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/23 13:15:46 UTC

[1/4] incubator-joshua git commit: Major refactoring of core decoder components (Rule.java, FeatureVector.java and grammars). The core idea of this change is to simplify feature handling inside Joshua. Please note that this change is NOT backwards compat

Repository: incubator-joshua
Updated Branches:
  refs/heads/7 0ad45cc04 -> 82f9183ee


http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/main/java/org/apache/joshua/decoder/hypergraph/GrammarBuilderWalkerFunction.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/decoder/hypergraph/GrammarBuilderWalkerFunction.java b/joshua-core/src/main/java/org/apache/joshua/decoder/hypergraph/GrammarBuilderWalkerFunction.java
index c5d2398..286fcb8 100644
--- a/joshua-core/src/main/java/org/apache/joshua/decoder/hypergraph/GrammarBuilderWalkerFunction.java
+++ b/joshua-core/src/main/java/org/apache/joshua/decoder/hypergraph/GrammarBuilderWalkerFunction.java
@@ -23,9 +23,10 @@ import java.util.HashSet;
 
 import org.apache.joshua.corpus.Vocabulary;
 import org.apache.joshua.decoder.JoshuaConfiguration;
+import org.apache.joshua.decoder.ff.FeatureVector;
 import org.apache.joshua.decoder.ff.tm.Grammar;
+import org.apache.joshua.decoder.ff.tm.OwnerMap;
 import org.apache.joshua.decoder.ff.tm.Rule;
-import org.apache.joshua.decoder.ff.tm.format.HieroFormatReader;
 import org.apache.joshua.decoder.ff.tm.hash_based.MemoryBasedBatchGrammar;
 import org.apache.joshua.util.FormatUtils;
 import org.slf4j.Logger;
@@ -48,24 +49,18 @@ public class GrammarBuilderWalkerFunction implements WalkerFunction {
 
   private static final Logger LOG = LoggerFactory.getLogger(GrammarBuilderWalkerFunction.class);
 
-  private MemoryBasedBatchGrammar grammar;
-  private static HieroFormatReader reader = new HieroFormatReader();
-  private PrintStream outStream;
-  private int goalSymbol;
-  private HashSet<Rule> rules;
+  private final MemoryBasedBatchGrammar grammar;
+  private final PrintStream outStream;
+  private final int goalSymbol;
+  private final HashSet<Rule> rules;
 
-  public GrammarBuilderWalkerFunction(String goal,JoshuaConfiguration joshuaConfiguration) {
-    grammar = new MemoryBasedBatchGrammar(reader, joshuaConfiguration, 1000);
+  public GrammarBuilderWalkerFunction(String goal, JoshuaConfiguration joshuaConfiguration, String owner) {
+    grammar = new MemoryBasedBatchGrammar(owner, joshuaConfiguration, 1000);
     outStream = null;
     goalSymbol = Vocabulary.id(goal);
     rules = new HashSet<Rule>();
   }
 
-  public GrammarBuilderWalkerFunction(String goal, PrintStream out,JoshuaConfiguration joshuaConfiguration) {
-    this(goal,joshuaConfiguration);
-    outStream = out;
-  }
-
   public void apply(HGNode node, int index) {
     // System.err.printf("VISITING NODE: %s\n", getLabelWithSpan(node));
     for (HyperEdge e : node.hyperedges) {
@@ -104,15 +99,19 @@ public class GrammarBuilderWalkerFunction implements WalkerFunction {
     // if this would be unary abstract, getNewSource will be null
     if (source == null) return null;
     int[] target = getNewTargetFromSource(source);
-    Rule result =
-        new Rule(headLabel, source, target, edgeRule.getFeatureString(), edgeRule.getArity());
-    // System.err.printf("new rule is %s\n", result);
-    return result;
+    return new Rule(
+        headLabel,
+        source,
+        target,
+        edgeRule.getArity(),
+        new FeatureVector(edgeRule.getFeatureVector()),
+        edgeRule.getAlignment(),
+        OwnerMap.UNKNOWN_OWNER_ID);
   }
 
   private static int[] getNewSource(boolean isGlue, HyperEdge edge) {
     Rule rule = edge.getRule();
-    int[] english = rule.getEnglish();
+    int[] english = rule.getTarget();
     // if this is a unary abstract rule, just return null
     // TODO: except glue rules!
     if (english.length == 1 && english[0] < 0 && !isGlue) return null;

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/main/java/org/apache/joshua/decoder/hypergraph/HyperGraph.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/decoder/hypergraph/HyperGraph.java b/joshua-core/src/main/java/org/apache/joshua/decoder/hypergraph/HyperGraph.java
index 499d4f3..3c4c3bb 100644
--- a/joshua-core/src/main/java/org/apache/joshua/decoder/hypergraph/HyperGraph.java
+++ b/joshua-core/src/main/java/org/apache/joshua/decoder/hypergraph/HyperGraph.java
@@ -119,7 +119,7 @@ public class HyperGraph {
           out.println(this.node_number);
           for (HyperEdge e: node.hyperedges) {
             if (e.getRule() != null) {
-              for (int id: e.getRule().getEnglish()) {
+              for (int id: e.getRule().getTarget()) {
                 if (id < 0) {
                   out.print(String.format("[%d] ", nodeMap.get(e.getTailNodes().get(-id-1))));
                 } else {

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/main/java/org/apache/joshua/decoder/hypergraph/KBestExtractor.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/decoder/hypergraph/KBestExtractor.java b/joshua-core/src/main/java/org/apache/joshua/decoder/hypergraph/KBestExtractor.java
index 8fc55df..131498b 100644
--- a/joshua-core/src/main/java/org/apache/joshua/decoder/hypergraph/KBestExtractor.java
+++ b/joshua-core/src/main/java/org/apache/joshua/decoder/hypergraph/KBestExtractor.java
@@ -18,9 +18,10 @@
  */
 package org.apache.joshua.decoder.hypergraph;
 
-import static org.apache.joshua.util.FormatUtils.unescapeSpecialSymbols;
-import static org.apache.joshua.util.FormatUtils.removeSentenceMarkers;
 import static java.util.Collections.emptyList;
+import static org.apache.joshua.decoder.ff.FeatureMap.hashFeature;
+import static org.apache.joshua.util.FormatUtils.removeSentenceMarkers;
+import static org.apache.joshua.util.FormatUtils.unescapeSpecialSymbols;
 
 import java.io.BufferedWriter;
 import java.io.IOException;
@@ -36,17 +37,18 @@ import java.util.PriorityQueue;
 import org.apache.joshua.corpus.Vocabulary;
 import org.apache.joshua.decoder.BLEU;
 import org.apache.joshua.decoder.JoshuaConfiguration;
+import org.apache.joshua.decoder.StructuredTranslation;
+import org.apache.joshua.decoder.StructuredTranslationFactory;
 import org.apache.joshua.decoder.ff.FeatureFunction;
 import org.apache.joshua.decoder.ff.FeatureVector;
 import org.apache.joshua.decoder.ff.fragmentlm.Tree;
 import org.apache.joshua.decoder.ff.state_maintenance.DPState;
+import org.apache.joshua.decoder.ff.tm.OwnerMap;
 import org.apache.joshua.decoder.ff.tm.Rule;
 import org.apache.joshua.decoder.io.DeNormalize;
 import org.apache.joshua.decoder.segment_file.Sentence;
 import org.apache.joshua.decoder.segment_file.Token;
 import org.apache.joshua.util.FormatUtils;
-import org.apache.joshua.decoder.StructuredTranslation;
-import org.apache.joshua.decoder.StructuredTranslationFactory;
 
 /**
  * <p>This class implements lazy k-best extraction on a hyper-graph.</p>
@@ -230,7 +232,7 @@ public class KBestExtractor {
        * If you want to output them, you have to replay them.
        */
 
-      FeatureVector features = new FeatureVector();
+      FeatureVector features = new FeatureVector(0);
       if (outputFormat.contains("%f") || outputFormat.contains("%d"))
         features = derivationState.getFeatures();
 
@@ -240,7 +242,7 @@ public class KBestExtractor {
           .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("%f", features.textFormat())
           .replace("%c", String.format("%.3f", derivationState.cost));
 
       if (outputFormat.contains("%t")) {
@@ -738,7 +740,7 @@ public class KBestExtractor {
      * @return float representing model cost plus the BLEU score
      */
     public float getCost() {
-      return cost - weights.getSparse("BLEU") * bleu;
+      return cost - weights.getOrDefault(hashFeature("BLEU")) * bleu;
     }
 
     public String toString() {
@@ -939,11 +941,11 @@ public class KBestExtractor {
       /* Find the fragment corresponding to this flattened rule in the fragment map; if it's not
        * there, just pretend it's a depth-one rule.
        */
-      Tree fragment = Tree.getFragmentFromYield(rule.getEnglishWords());
+      Tree fragment = Tree.getFragmentFromYield(rule.getTargetWords());
       if (fragment == null) {
         String subtree = String.format("(%s{%d-%d} %s)", unbracketedLHS, 
             state.parentNode.i, state.parentNode.j, 
-            quoteTerminals(rule.getEnglishWords()));
+            quoteTerminals(rule.getTargetWords()));
         fragment = Tree.fromString(subtree);
       }
       
@@ -1027,15 +1029,16 @@ public class KBestExtractor {
         // KBestExtractor.this.weights.innerProduct(features));
         sb.append(String.format("%d-%d", state.parentNode.i, state.parentNode.j));
         sb.append(" ||| " + Vocabulary.word(rule.getLHS()) + " -> "
-            + Vocabulary.getWords(rule.getFrench()) + " /// " + rule.getEnglishWords());
+            + Vocabulary.getWords(rule.getSource()) + " /// " + rule.getTargetWords());
         sb.append(" |||");
         for (DPState dpState : state.parentNode.getDPStates()) {
           sb.append(" " + dpState);
         }
-        sb.append(" ||| " + transitionFeatures);
+        sb.append(" ||| " + transitionFeatures.textFormat());
         sb.append(" ||| " + weights.innerProduct(transitionFeatures));
         if (rule.getAlignment() != null)
           sb.append(" ||| " + Arrays.toString(rule.getAlignment()));
+        sb.append(" ||| " + OwnerMap.getOwner(rule.getOwner()));
         sb.append("\n");
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/main/java/org/apache/joshua/decoder/hypergraph/OutputStringExtractor.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/decoder/hypergraph/OutputStringExtractor.java b/joshua-core/src/main/java/org/apache/joshua/decoder/hypergraph/OutputStringExtractor.java
index f20e063..1f5bb6b 100644
--- a/joshua-core/src/main/java/org/apache/joshua/decoder/hypergraph/OutputStringExtractor.java
+++ b/joshua-core/src/main/java/org/apache/joshua/decoder/hypergraph/OutputStringExtractor.java
@@ -54,7 +54,7 @@ public class OutputStringExtractor implements WalkerFunction, DerivationVisitor
   
   private void apply(Rule rule, int nodeIndex) {
     if (rule != null) {
-      final int[] words = extractSource ? rule.getFrench() : rule.getEnglish();
+      final int[] words = extractSource ? rule.getSource() : rule.getTarget();
       merge(new OutputString(words, rule.getArity(), nodeIndex));
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/main/java/org/apache/joshua/decoder/hypergraph/ViterbiExtractor.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/decoder/hypergraph/ViterbiExtractor.java b/joshua-core/src/main/java/org/apache/joshua/decoder/hypergraph/ViterbiExtractor.java
index 734e0aa..7cf7dc1 100644
--- a/joshua-core/src/main/java/org/apache/joshua/decoder/hypergraph/ViterbiExtractor.java
+++ b/joshua-core/src/main/java/org/apache/joshua/decoder/hypergraph/ViterbiExtractor.java
@@ -93,8 +93,9 @@ public class ViterbiExtractor {
       final HyperGraph hg,
       final List<FeatureFunction> featureFunctions,
       final Sentence sentence) {
-    if (hg == null)
-      return new FeatureVector();
+    if (hg == null) {
+      return new FeatureVector(0);
+    }
 
     final FeatureVectorExtractor extractor = new FeatureVectorExtractor(
         featureFunctions, sentence);

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/main/java/org/apache/joshua/decoder/hypergraph/WordAlignmentState.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/decoder/hypergraph/WordAlignmentState.java b/joshua-core/src/main/java/org/apache/joshua/decoder/hypergraph/WordAlignmentState.java
index f057f23..4b8c9e4 100644
--- a/joshua-core/src/main/java/org/apache/joshua/decoder/hypergraph/WordAlignmentState.java
+++ b/joshua-core/src/main/java/org/apache/joshua/decoder/hypergraph/WordAlignmentState.java
@@ -58,12 +58,12 @@ public class WordAlignmentState {
    */
   public WordAlignmentState(final Rule rule, final int start) {
     trgPoints = new LinkedList<AlignedSourceTokens>();
-    srcLength = rule.getFrench().length;
+    srcLength = rule.getSource().length;
     numNT = rule.getArity();
     srcStart = start;
     final Map<Integer, List<Integer>> alignmentMap = rule.getAlignmentMap();
     final int[] nonTerminalSourcePositions = rule.getNonTerminalSourcePositions();
-    final int[] trg = rule.getEnglish();
+    final int[] trg = rule.getTarget();
     // for each target index, create a TargetAlignmentPoint
     for (int trgIndex = 0; trgIndex < trg.length; trgIndex++) {
       final AlignedSourceTokens trgPoint = new AlignedSourceTokens();

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/main/java/org/apache/joshua/decoder/phrase/Candidate.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/decoder/phrase/Candidate.java b/joshua-core/src/main/java/org/apache/joshua/decoder/phrase/Candidate.java
index ee8a2a9..0098e9f 100644
--- a/joshua-core/src/main/java/org/apache/joshua/decoder/phrase/Candidate.java
+++ b/joshua-core/src/main/java/org/apache/joshua/decoder/phrase/Candidate.java
@@ -94,7 +94,7 @@ public class Candidate {
   public String toString() {
     return String.format("CANDIDATE(hyp %d/%d, phr %d/%d) [%s] phrase=[%s] span=%s",
         ranks[0], hypotheses.size(), ranks[1], phrases.size(),
-        getHypothesis(), getRule().getEnglishWords().replaceAll("\\[.*?\\] ",""), getSpan());
+        getHypothesis(), getRule().getTargetWords().replaceAll("\\[.*?\\] ",""), getSpan());
   }
   
   public Candidate(List<Hypothesis> hypotheses, TargetPhrases phrases, Span span, float delta) {

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/main/java/org/apache/joshua/decoder/phrase/Hypothesis.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/decoder/phrase/Hypothesis.java b/joshua-core/src/main/java/org/apache/joshua/decoder/phrase/Hypothesis.java
index 71d3df9..a302959 100644
--- a/joshua-core/src/main/java/org/apache/joshua/decoder/phrase/Hypothesis.java
+++ b/joshua-core/src/main/java/org/apache/joshua/decoder/phrase/Hypothesis.java
@@ -18,7 +18,9 @@
  */
 package org.apache.joshua.decoder.phrase;
 
-import java.util.List;	
+import static org.apache.joshua.decoder.ff.tm.OwnerMap.UNKNOWN_OWNER_ID;
+
+import java.util.List;
 
 import org.apache.joshua.corpus.Vocabulary;
 import org.apache.joshua.decoder.ff.state_maintenance.DPState;
@@ -40,15 +42,14 @@ public class Hypothesis extends HGNode implements Comparable<Hypothesis> {
 
   // The hypothesis' coverage vector
   private Coverage coverage;
-
-  public static Rule BEGIN_RULE = new HieroFormatReader().parseLine("[X] ||| <s> ||| <s> |||   ||| 0-0");
-  public static Rule END_RULE = new HieroFormatReader().parseLine("[GOAL] ||| [X,1] </s> ||| [X,1] </s> |||   ||| 0-0 1-1");
+  public static Rule BEGIN_RULE = new HieroFormatReader(UNKNOWN_OWNER_ID).parseLine("[X] ||| <s> ||| <s> |||   ||| 0-0");
+  public static Rule END_RULE = new HieroFormatReader(UNKNOWN_OWNER_ID).parseLine("[GOAL] ||| [X,1] </s> ||| [X,1] </s> |||   ||| 0-0 1-1");
 
   public String toString() {
     StringBuffer sb = new StringBuffer();
     for (DPState state: getDPStates())
       sb.append(state);
-    String words = bestHyperedge.getRule().getEnglishWords();
+    String words = bestHyperedge.getRule().getTargetWords();
 //  return String.format("HYP[%s] %.5f j=%d words=%s state=%s", coverage, score, j, words, sb);
     return String.format("HYP[%s] j=%d words=[%s] state=%s", coverage, j, words, sb);
   }

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/main/java/org/apache/joshua/decoder/phrase/PhraseChart.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/decoder/phrase/PhraseChart.java b/joshua-core/src/main/java/org/apache/joshua/decoder/phrase/PhraseChart.java
index 9803d9b..a676efe 100644
--- a/joshua-core/src/main/java/org/apache/joshua/decoder/phrase/PhraseChart.java
+++ b/joshua-core/src/main/java/org/apache/joshua/decoder/phrase/PhraseChart.java
@@ -109,7 +109,7 @@ public class PhraseChart {
             if (phrases != null) {
               LOG.debug("{} ({}-{})", source.source(i,j), i, j);
               for (Rule rule: phrases)
-                LOG.debug("    {} :: est={}", rule.getEnglishWords(), rule.getEstimatedCost());
+                LOG.debug("    {} :: est={}", rule.getTargetWords(), rule.getEstimatedCost());
             }
           }
         }

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/main/java/org/apache/joshua/decoder/phrase/PhraseTable.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/decoder/phrase/PhraseTable.java b/joshua-core/src/main/java/org/apache/joshua/decoder/phrase/PhraseTable.java
index 2c8a3e9..5f52358 100644
--- a/joshua-core/src/main/java/org/apache/joshua/decoder/phrase/PhraseTable.java
+++ b/joshua-core/src/main/java/org/apache/joshua/decoder/phrase/PhraseTable.java
@@ -18,8 +18,6 @@
  */
 package org.apache.joshua.decoder.phrase;
 
-import static org.apache.joshua.decoder.ff.tm.OwnerMap.UNKNOWN_OWNER;
-
 import java.io.File;
 import java.io.IOException;
 import java.util.List;
@@ -27,6 +25,7 @@ import java.util.List;
 import org.apache.joshua.corpus.Vocabulary;
 import org.apache.joshua.decoder.JoshuaConfiguration;
 import org.apache.joshua.decoder.ff.FeatureFunction;
+import org.apache.joshua.decoder.ff.FeatureVector;
 import org.apache.joshua.decoder.ff.tm.Grammar;
 import org.apache.joshua.decoder.ff.tm.OwnerId;
 import org.apache.joshua.decoder.ff.tm.Rule;
@@ -130,16 +129,15 @@ public class PhraseTable implements Grammar {
         : sourceWord;   
 
     int nt_i = Vocabulary.id("[X]");
-    Rule oovRule = new Rule(nt_i, new int[] { nt_i, sourceWord }, new int[] { -1, targetWord }, "0-0", 1);
+    Rule oovRule = new Rule(
+        nt_i,
+        new int[] { nt_i, sourceWord },
+        new int[] { -1, targetWord },
+        1,
+        new FeatureVector(0),
+        new byte[] {0,0}, backend.getOwner());
     addRule(oovRule);
     oovRule.estimateRuleCost(featureFunctions);
-        
-//    String ruleString = String.format("[X] ||| [X,1] %s ||| [X,1] %s", 
-//        Vocabulary.word(sourceWord), Vocabulary.word(targetWord));
-//    Rule oovRule = new HieroFormatReader().parseLine(ruleString);
-//    oovRule.setOwner(Vocabulary.id("oov"));
-//    addRule(oovRule);
-//    oovRule.estimateRuleCost(featureFunctions);
   }
 
   @Override
@@ -174,9 +172,4 @@ public class PhraseTable implements Grammar {
   public OwnerId getOwner() {
     return backend.getOwner();
   }
-
-  @Override
-  public int getNumDenseFeatures() {
-    return backend.getNumDenseFeatures();
-  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/main/java/org/apache/joshua/decoder/phrase/Stack.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/decoder/phrase/Stack.java b/joshua-core/src/main/java/org/apache/joshua/decoder/phrase/Stack.java
index d0ae2da..d4232cd 100644
--- a/joshua-core/src/main/java/org/apache/joshua/decoder/phrase/Stack.java
+++ b/joshua-core/src/main/java/org/apache/joshua/decoder/phrase/Stack.java
@@ -149,8 +149,8 @@ public class Stack extends ArrayList<Hypothesis> {
 
     // Constrained decoding
     if (sentence.target() != null) {
-      String oldWords = cand.getHypothesis().bestHyperedge.getRule().getEnglishWords().replace("[X,1] ",  "");
-      String newWords = cand.getRule().getEnglishWords().replace("[X,1] ",  "");
+      String oldWords = cand.getHypothesis().bestHyperedge.getRule().getTargetWords().replace("[X,1] ",  "");
+      String newWords = cand.getRule().getTargetWords().replace("[X,1] ",  "");
           
       // If the string is not found in the target sentence, explore the cube neighbors
       if (sentence.fullTarget().indexOf(oldWords + " " + newWords) == -1) {
@@ -217,10 +217,10 @@ public class Stack extends ArrayList<Hypothesis> {
     }
 
     if (LOG.isDebugEnabled()) {
-      LOG.debug("{} from ( ... {} )", taskName, complete.getHypothesis().getRule().getEnglishWords());
+      LOG.debug("{} from ( ... {} )", taskName, complete.getHypothesis().getRule().getTargetWords());
       LOG.debug("        base score {}", complete.getResult().getBaseCost());
       LOG.debug("        covering {}-{}", complete.getSpan().start - 1, complete.getSpan().end - 2);
-      LOG.debug("        translated as: {}", complete.getRule().getEnglishWords());
+      LOG.debug("        translated as: {}", complete.getRule().getTargetWords());
       LOG.debug("        score {} + future cost {} = {}",
           complete.getResult().getTransitionCost(), complete.getFutureEstimate(),
           complete.getResult().getTransitionCost() + complete.getFutureEstimate());

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/main/java/org/apache/joshua/oracle/OracleExtractionHG.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/oracle/OracleExtractionHG.java b/joshua-core/src/main/java/org/apache/joshua/oracle/OracleExtractionHG.java
index 575515a..572c7f1 100644
--- a/joshua-core/src/main/java/org/apache/joshua/oracle/OracleExtractionHG.java
+++ b/joshua-core/src/main/java/org/apache/joshua/oracle/OracleExtractionHG.java
@@ -360,7 +360,7 @@ public class OracleExtractionHG extends SplitHg {
     // computed
     int total_hyp_len = 0;
     int[] num_ngram_match = new int[g_bleu_order];
-    int[] en_words = dt.getRule().getEnglish();
+    int[] en_words = dt.getRule().getTarget();
 
     // ####calulate new and old ngram counts, and len
 

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/main/java/org/apache/joshua/server/ServerThread.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/server/ServerThread.java b/joshua-core/src/main/java/org/apache/joshua/server/ServerThread.java
index 0301e71..6cfea6c 100644
--- a/joshua-core/src/main/java/org/apache/joshua/server/ServerThread.java
+++ b/joshua-core/src/main/java/org/apache/joshua/server/ServerThread.java
@@ -18,6 +18,8 @@
  */
 package org.apache.joshua.server;
 
+import static org.apache.joshua.decoder.ff.FeatureMap.hashFeature;
+
 import java.io.BufferedReader;
 import java.io.IOException;
 import java.io.InputStreamReader;
@@ -32,10 +34,6 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
 
-import com.sun.net.httpserver.HttpExchange;
-import com.sun.net.httpserver.HttpHandler;
-
-import org.apache.joshua.corpus.Vocabulary;
 import org.apache.joshua.decoder.Decoder;
 import org.apache.joshua.decoder.JoshuaConfiguration;
 import org.apache.joshua.decoder.Translation;
@@ -48,6 +46,9 @@ import org.apache.joshua.decoder.io.TranslationRequestStream;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.sun.net.httpserver.HttpExchange;
+import com.sun.net.httpserver.HttpHandler;
+
 /**
  * This class handles a concurrent request for translations from a newly opened socket, for
  * both raw TCP/IP connections and for HTTP connections.
@@ -195,17 +196,18 @@ public class ServerThread extends Thread implements HttpHandler {
     
     if (type.equals("get_weight")) {
       String weight = tokens[1];
-      LOG.info("WEIGHT: %s = %.3f", weight, Decoder.weights.getWeight(weight));
+      LOG.info("WEIGHT: %s = %.3f", weight, Decoder.weights.getOrDefault(hashFeature(weight)));
 
     } else if (type.equals("set_weights")) {
       // Change a decoder weight
       String[] argTokens = args.split("\\s+");
       for (int i = 0; i < argTokens.length; i += 2) {
         String feature = argTokens[i];
+        int featureId = hashFeature(feature);
         String newValue = argTokens[i+1];
-        float old_weight = Decoder.weights.getWeight(feature);
-        Decoder.weights.set(feature, Float.parseFloat(newValue));
-        LOG.info("set_weights: {} {} -> {}", feature, old_weight, Decoder.weights.getWeight(feature));
+        float old_weight = Decoder.weights.getOrDefault(featureId);
+        Decoder.weights.put(featureId, Float.parseFloat(newValue));
+        LOG.info("set_weights: {} {} -> {}", feature, old_weight, Decoder.weights.getOrDefault(featureId));
       }
       
       message.addMetaData("weights " + Decoder.weights.toString());
@@ -235,7 +237,7 @@ public class ServerThread extends Thread implements HttpHandler {
           ? String.format("%s ||| [X,1] %s ||| [X,1] %s ||| custom=1 %s", lhs, source, target, featureStr)
           : String.format("%s ||| %s ||| %s ||| custom=1 %s", lhs, source, target, featureStr);
       
-      Rule rule = new HieroFormatReader().parseLine(ruleString);
+      Rule rule = new HieroFormatReader(decoder.getCustomPhraseTable().getOwner()).parseLine(ruleString);
       decoder.addCustomRule(rule);
       
       LOG.info("Added custom rule {}", rule.toString());
@@ -267,12 +269,12 @@ public class ServerThread extends Thread implements HttpHandler {
   
     } else if (type.equals("remove_rule")) {
       
-      Rule rule = new HieroFormatReader().parseLine(args);
+      Rule rule = new HieroFormatReader(decoder.getCustomPhraseTable().getOwner()).parseLine(args);
       
       LOG.info("remove_rule " + rule);
   
       Trie trie = decoder.getCustomPhraseTable().getTrieRoot();
-      int[] sourceTokens = rule.getFrench();
+      int[] sourceTokens = rule.getSource();
       for (int i = 0; i < sourceTokens.length; i++) {
         Trie nextTrie = trie.match(sourceTokens[i]);
         if (nextTrie == null)
@@ -283,7 +285,7 @@ public class ServerThread extends Thread implements HttpHandler {
 
       if (trie.hasRules()) {
         for (Rule ruleCand: trie.getRuleCollection().getRules()) {
-          if (Arrays.equals(rule.getEnglish(), ruleCand.getEnglish())) {
+          if (Arrays.equals(rule.getTarget(), ruleCand.getTarget())) {
             trie.getRuleCollection().getRules().remove(ruleCand);
             break;
           }

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/main/java/org/apache/joshua/tools/GrammarPacker.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/tools/GrammarPacker.java b/joshua-core/src/main/java/org/apache/joshua/tools/GrammarPacker.java
index b9208d2..6c02d19 100644
--- a/joshua-core/src/main/java/org/apache/joshua/tools/GrammarPacker.java
+++ b/joshua-core/src/main/java/org/apache/joshua/tools/GrammarPacker.java
@@ -18,6 +18,7 @@
  */
 package org.apache.joshua.tools;
 
+import static org.apache.joshua.decoder.ff.tm.OwnerMap.UNKNOWN_OWNER_ID;
 import static org.apache.joshua.decoder.ff.tm.packed.PackedGrammar.VOCABULARY_FILENAME;
 
 import java.io.BufferedOutputStream;
@@ -31,11 +32,13 @@ import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.LinkedList;
 import java.util.List;
+import java.util.Map.Entry;
 import java.util.Queue;
 import java.util.TreeMap;
 
 import org.apache.joshua.corpus.Vocabulary;
 import org.apache.joshua.decoder.ff.tm.Rule;
+import org.apache.joshua.decoder.ff.tm.RuleFactory;
 import org.apache.joshua.decoder.ff.tm.format.HieroFormatReader;
 import org.apache.joshua.decoder.ff.tm.format.MosesFormatReader;
 import org.apache.joshua.util.FormatUtils;
@@ -221,19 +224,20 @@ public class GrammarPacker {
   }
 
   /**
-   * Returns a reader that turns whatever file format is found into Hiero grammar rules.
+   * Returns a reader that turns whatever file format is found into unowned Hiero grammar rules.
+   * This means, features are NOT prepended with an owner string at packing time.
    * 
    * @param grammarFile
-   * @return
+   * @return GrammarReader of correct Format
    * @throws IOException
    */
   private HieroFormatReader getGrammarReader() throws IOException {
     LineReader reader = new LineReader(grammar);
     String line = reader.next();
     if (line.startsWith("[")) {
-      return new HieroFormatReader(grammar);
+      return new HieroFormatReader(grammar, UNKNOWN_OWNER_ID);
     } else {
-      return new MosesFormatReader(grammar);
+      return new MosesFormatReader(grammar, UNKNOWN_OWNER_ID);
     }
   }
 
@@ -247,9 +251,9 @@ public class GrammarPacker {
     // appear in the same order. They are assigned numeric names in order of appearance.
     this.types.setLabeled(true);
 
-    for (Rule rule: reader) {
+    for (Rule rule : reader) {
 
-      max_source_len = Math.max(max_source_len, rule.getFrench().length);
+      max_source_len = Math.max(max_source_len, rule.getSource().length);
 
       /* Add symbols to vocabulary.
        * NOTE: In case of nonterminals, we add both stripped versions ("[X]")
@@ -259,20 +263,9 @@ public class GrammarPacker {
        * {@link HieroFormatReader}, which is called by {@link MosesFormatReader}. 
        */
 
-      // Add feature names to vocabulary and pass the value through the
-      // appropriate encoder.
-      int feature_counter = 0;
-      String[] features = rule.getFeatureString().split("\\s+");
-      for (int f = 0; f < features.length; ++f) {
-        if (features[f].contains("=")) {
-          String[] fe = features[f].split("=");
-          if (fe[0].equals("Alignment"))
-            continue;
-          types.observe(Vocabulary.id(fe[0]), Float.parseFloat(fe[1]));
-        } else {
-          types.observe(Vocabulary.id(String.valueOf(feature_counter++)),
-              Float.parseFloat(features[f]));
-        }
+      // pass the value through the appropriate encoder.
+      for (final Entry<Integer, Float> entry : rule.getFeatureVector().entrySet()) {
+        types.observe(entry.getKey(), entry.getValue());
       }
     }
   }
@@ -303,14 +296,13 @@ public class GrammarPacker {
       alignment_buffer = new AlignmentBuffer();
 
     TreeMap<Integer, Float> features = new TreeMap<Integer, Float>();
-    for (Rule rule: grammarReader) {
+    for (Rule rule : grammarReader) {
       counter++;
       slice_counter++;
 
       String lhs_word = Vocabulary.word(rule.getLHS());
-      String[] source_words = rule.getFrenchWords().split("\\s+");
-      String[] target_words = rule.getEnglishWords().split("\\s+");
-      String[] feature_entries = rule.getFeatureString().split("\\s+");
+      String[] source_words = rule.getSourceWords().split("\\s+");
+      String[] target_words = rule.getTargetWords().split("\\s+");
 
       // Reached slice limit size, indicate that we're closing up.
       if (!ready_to_flush
@@ -347,24 +339,15 @@ public class GrammarPacker {
       int alignment_index = -1;
       // If present, process alignments.
       if (packAlignments) {
-        String alignment_line;
+        byte[] alignments = null;
         if (grammarAlignments) {
-          alignment_line = rule.getAlignmentString();
+          alignments = rule.getAlignment();
         } else {
           if (!alignment_reader.hasNext()) {
             LOG.error("No more alignments starting in line {}", counter);
             throw new RuntimeException("No more alignments starting in line " + counter);
           }
-          alignment_line = alignment_reader.next().trim();
-        }
-        String[] alignment_entries = alignment_line.split("\\s");
-        byte[] alignments = new byte[alignment_entries.length * 2];
-        if (alignment_line.length() > 0) {
-          for (int i = 0; i < alignment_entries.length; i++) {
-            String[] parts = alignment_entries[i].split("-");
-            alignments[2 * i] = Byte.parseByte(parts[0]);
-            alignments[2 * i + 1] = Byte.parseByte(parts[1]);
-          }
+          alignments = RuleFactory.parseAlignmentString(alignment_reader.next().trim());
         }
         alignment_index = alignment_buffer.add(alignments);
       }
@@ -373,24 +356,14 @@ public class GrammarPacker {
       // Implicitly sort via TreeMap, write to data buffer, remember position
       // to pass on to the source trie node.
       features.clear();
-      int feature_count = 0;
-      for (int f = 0; f < feature_entries.length; ++f) {
-        String feature_entry = feature_entries[f];
-        int feature_id;
-        float feature_value;
-        if (feature_entry.contains("=")) {
-          String[] parts = feature_entry.split("=");
-          if (parts[0].equals("Alignment"))
-            continue;
-          feature_id = Vocabulary.id(parts[0]);
-          feature_value = Float.parseFloat(parts[1]);
-        } else {
-          feature_id = Vocabulary.id(String.valueOf(feature_count++));
-          feature_value = Float.parseFloat(feature_entry);
+      for (Entry<Integer, Float> entry : rule.getFeatureVector().entrySet()) {
+        int featureId = entry.getKey();
+        float featureValue = entry.getValue();
+        if (featureValue != 0f) {
+          features.put(encoderConfig.innerId(featureId), featureValue);
         }
-        if (feature_value != 0)
-          features.put(encoderConfig.innerId(feature_id), feature_value);
       }
+
       int features_index = feature_buffer.add(features);
 
       // Sanity check on the data block index.

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/main/java/org/apache/joshua/util/Constants.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/util/Constants.java b/joshua-core/src/main/java/org/apache/joshua/util/Constants.java
index 3d4139d..bcabfe4 100644
--- a/joshua-core/src/main/java/org/apache/joshua/util/Constants.java
+++ b/joshua-core/src/main/java/org/apache/joshua/util/Constants.java
@@ -31,6 +31,14 @@ public final class Constants {
   public static final String STOP_SYM = "</s>";
   public static final String UNKNOWN_WORD = "<unk>";
   
-  public static final String fieldDelimiter = "\\s\\|{3}\\s";
+  public static final String fieldDelimiterPattern = "\\s\\|{3}\\s";
+  public static final String fieldDelimiter = " ||| ";
   public static final String spaceSeparator = "\\s+";
+  
+  public static final String NT_REGEX = "\\[[^\\]]+?\\]";
+  
+  public static final String TM_PREFIX = "tm";
+  
+  public static final String labeledFeatureSeparator = "=";
+  
 }

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/main/java/org/apache/joshua/util/PackedGrammarServer.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/util/PackedGrammarServer.java b/joshua-core/src/main/java/org/apache/joshua/util/PackedGrammarServer.java
index 74c8e4a..d764340 100644
--- a/joshua-core/src/main/java/org/apache/joshua/util/PackedGrammarServer.java
+++ b/joshua-core/src/main/java/org/apache/joshua/util/PackedGrammarServer.java
@@ -65,9 +65,11 @@ public class PackedGrammarServer {
       return null;
     
     int[] tgt = Vocabulary.addAll(target);
-    for (Rule r : rules)
-      if (Arrays.equals(tgt, r.getEnglish()))
-        return r.getFeatureVector().getMap();
+    for (Rule r : rules) {
+      if (Arrays.equals(tgt, r.getTarget())) {
+        return r.getFeatureVector().toStringMap();
+      }
+    }
     
     return null;
   }

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/main/java/org/apache/joshua/util/encoding/EncoderConfiguration.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/util/encoding/EncoderConfiguration.java b/joshua-core/src/main/java/org/apache/joshua/util/encoding/EncoderConfiguration.java
index 28b013f..23392a4 100644
--- a/joshua-core/src/main/java/org/apache/joshua/util/encoding/EncoderConfiguration.java
+++ b/joshua-core/src/main/java/org/apache/joshua/util/encoding/EncoderConfiguration.java
@@ -28,6 +28,7 @@ import java.util.HashMap;
 import java.util.Map;
 
 import org.apache.joshua.corpus.Vocabulary;
+import org.apache.joshua.decoder.ff.FeatureMap;
 
 public class EncoderConfiguration {
 
@@ -82,7 +83,7 @@ public class EncoderConfiguration {
       int outer_id;
       if (labeled) {
         String feature_name = in_stream.readUTF();
-        outer_id = Vocabulary.id(feature_name);
+        outer_id = FeatureMap.hashFeature(feature_name);
         try {
           Integer.parseInt(feature_name);
           numDenseFeatures++;
@@ -94,7 +95,7 @@ public class EncoderConfiguration {
       int encoder_index = in_stream.readInt();
       if (encoder_index >= num_encoders) {
         throw new RuntimeException("Error deserializing EncoderConfig. " + "Feature "
-            + (labeled ? Vocabulary.word(outer_id) : outer_id) + " referring to encoder "
+            + (labeled ? FeatureMap.getFeature(outer_id) : outer_id) + " referring to encoder "
             + encoder_index + " when only " + num_encoders + " known.");
       }
       encoderById[inner_id] = encoders[encoder_index];

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/main/java/org/apache/joshua/util/encoding/FeatureTypeAnalyzer.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/util/encoding/FeatureTypeAnalyzer.java b/joshua-core/src/main/java/org/apache/joshua/util/encoding/FeatureTypeAnalyzer.java
index 504859f..0aa41af 100644
--- a/joshua-core/src/main/java/org/apache/joshua/util/encoding/FeatureTypeAnalyzer.java
+++ b/joshua-core/src/main/java/org/apache/joshua/util/encoding/FeatureTypeAnalyzer.java
@@ -30,6 +30,7 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.joshua.corpus.Vocabulary;
+import org.apache.joshua.decoder.ff.FeatureMap;
 import org.apache.joshua.util.io.LineReader;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -80,7 +81,7 @@ public class FeatureTypeAnalyzer {
         String encoder_key = fields[1];
         ArrayList<Integer> feature_ids = new ArrayList<Integer>();
         for (int i = 2; i < fields.length; i++)
-          feature_ids.add(Vocabulary.id(fields[i]));
+          feature_ids.add(FeatureMap.hashFeature(fields[i]));
         addFeatures(encoder_key, feature_ids);
       }
     }
@@ -125,7 +126,7 @@ public class FeatureTypeAnalyzer {
     }
     if (LOG.isInfoEnabled()) {
       for (int id : featureToType.keySet()) {
-        LOG.info("Type inferred: {} is {}", (labeled ? Vocabulary.word(id) : "Feature " + id),
+        LOG.info("Type inferred: {} is {}", (labeled ? FeatureMap.getFeature(id) : "Feature " + id),
             types.get(featureToType.get(id)).encoder.getKey());
       }
     }
@@ -173,7 +174,7 @@ public class FeatureTypeAnalyzer {
     out_stream.writeInt(featureToType.size());
     for (int feature_id : featureToType.keySet()) {
       if (labeled)
-        out_stream.writeUTF(Vocabulary.word(feature_id));
+        out_stream.writeUTF(FeatureMap.getFeature(feature_id));
       else
         out_stream.writeInt(feature_id);
       out_stream.writeInt(featureIdMap.get(feature_id));
@@ -185,8 +186,9 @@ public class FeatureTypeAnalyzer {
   public String toString() {
     StringBuilder sb = new StringBuilder();
     for (int feature_id : featureToType.keySet()) {
-      sb.append(types.get(featureToType.get(feature_id)).analyzer.toString(Vocabulary.word(feature_id)));
+      sb.append(types.get(featureToType.get(feature_id)).analyzer.toString(FeatureMap.getFeature(feature_id)));
     }
+    System.out.println(sb.toString());
     return sb.toString();
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/test/java/org/apache/joshua/decoder/ff/FeatureVectorTest.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/test/java/org/apache/joshua/decoder/ff/FeatureVectorTest.java b/joshua-core/src/test/java/org/apache/joshua/decoder/ff/FeatureVectorTest.java
new file mode 100644
index 0000000..84939f1
--- /dev/null
+++ b/joshua-core/src/test/java/org/apache/joshua/decoder/ff/FeatureVectorTest.java
@@ -0,0 +1,120 @@
+/*
+ * 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.ff;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertFalse;
+import static org.testng.Assert.assertTrue;
+
+import org.testng.annotations.Test;
+
+import com.google.common.collect.ImmutableMap;
+
+public class FeatureVectorTest {
+    
+    private static final FeatureVector INIT =
+        new FeatureVector(ImmutableMap.of(0, 0.1f, 1, 500.0f));
+    
+    @Test
+    public void givenFeatureVector_whenRequestingAbsentValue_thenDefaultValue() {
+        FeatureVector v = new FeatureVector(0);
+        assertEquals(v.getOrDefault(0), 0.0f);
+    }
+    
+    @Test
+    public void givenFeatureVector_whenCopyConstructor_thenIsCopied() {
+        // GIVEN
+        FeatureVector newVector = new FeatureVector(INIT);
+        
+        // WHEN
+        newVector.add(0, 14f);
+        
+        // THEN
+        assertEquals(INIT.get(0), 0.1f);
+        assertEquals(newVector.get(0), 14.1f);
+    }
+    
+    @Test
+    public void givenFeatureVector_whenElementWiseAddition_thenEntriesAreCorrect() {
+        // GIVEN
+        FeatureVector v = new FeatureVector(INIT);
+        
+        // WHEN
+        v.add(1, 500f);
+        v.add(2, -1f);
+        
+        // THEN
+        assertEquals(v.get(1), 1000f);
+        assertEquals(v.get(2), -1f);
+    }
+    
+    @Test
+    public void givenFeatureVector_whenVectorAddition_thenVectorsAreCorrect() {
+        // GIVEN
+        FeatureVector v = new FeatureVector(INIT);
+        
+        // WHEN
+        v.addInPlace(INIT);
+        
+        // THEN
+        assertTrue(v.containsKey(0));
+        assertTrue(v.containsKey(1));
+        assertEquals(v.getOrDefault(0), 0.2f);
+        assertEquals(v.getOrDefault(1), 1000f);
+    }
+    
+    @Test
+    public void givenFeatureVector_whenPrune_thenCorrectEntriesAreRemoved() {
+        // GIVEN
+        FeatureVector v = new FeatureVector(INIT);
+        v.add(2, 0.0001f);
+        
+        // WHEN
+        v.prune(0.001f).size();
+        
+        // THEN
+        assertFalse(v.containsKey(2));
+    }
+    
+    @Test
+    public void givenFeatureVector_whenInnerProduct_thenResultIsCorrect() {
+        // GIVEN
+        FeatureVector v = new FeatureVector(INIT);
+        v.put(2, 12f);
+        float expectedDotProduct = 0.0f;
+        for (float value : v.values()) {
+            expectedDotProduct += value * value;
+        }
+        
+        // WHEN
+        float dotProduct = v.innerProduct(v);
+        
+        // THEN
+        assertTrue(Math.abs(dotProduct - expectedDotProduct) < 0.00001);
+    }
+
+    @Test
+    public void givenTwoFeatureVectors_thenEqualityIsCorrect() {
+        // GIVEN
+        FeatureVector v1 = new FeatureVector(INIT);
+        FeatureVector v2 = new FeatureVector(INIT);
+        assertEquals(v1, v2);
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/test/java/org/apache/joshua/decoder/ff/lm/LanguageModelFFTest.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/test/java/org/apache/joshua/decoder/ff/lm/LanguageModelFFTest.java b/joshua-core/src/test/java/org/apache/joshua/decoder/ff/lm/LanguageModelFFTest.java
index 5e71352..89f1e81 100644
--- a/joshua-core/src/test/java/org/apache/joshua/decoder/ff/lm/LanguageModelFFTest.java
+++ b/joshua-core/src/test/java/org/apache/joshua/decoder/ff/lm/LanguageModelFFTest.java
@@ -26,6 +26,7 @@ import static org.testng.Assert.assertEquals;
 import org.apache.joshua.corpus.Vocabulary;
 import org.apache.joshua.decoder.Decoder;
 import org.apache.joshua.decoder.JoshuaConfiguration;
+import org.apache.joshua.decoder.ff.FeatureMap;
 import org.apache.joshua.decoder.ff.FeatureVector;
 import org.apache.joshua.decoder.ff.state_maintenance.NgramDPState;
 import org.testng.annotations.AfterMethod;
@@ -42,8 +43,8 @@ public class LanguageModelFFTest {
   public void setUp() {
     Decoder.resetGlobalState();
 
-    FeatureVector weights = new FeatureVector();
-    weights.set("lm_0", WEIGHT);
+    FeatureVector weights = new FeatureVector(2);
+    weights.put(FeatureMap.hashFeature("lm_0"), WEIGHT);
     String[] args = {"-lm_type", "berkeleylm", "-lm_order", "2", "-lm_file", "./src/test/resources/lm/berkeley/lm"};
 
     JoshuaConfiguration config = new JoshuaConfiguration();

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/test/java/org/apache/joshua/decoder/ff/lm/berkeley_lm/LMGrammarBerkeleyTest.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/test/java/org/apache/joshua/decoder/ff/lm/berkeley_lm/LMGrammarBerkeleyTest.java b/joshua-core/src/test/java/org/apache/joshua/decoder/ff/lm/berkeley_lm/LMGrammarBerkeleyTest.java
index b0612d4..cc4a94c 100644
--- a/joshua-core/src/test/java/org/apache/joshua/decoder/ff/lm/berkeley_lm/LMGrammarBerkeleyTest.java
+++ b/joshua-core/src/test/java/org/apache/joshua/decoder/ff/lm/berkeley_lm/LMGrammarBerkeleyTest.java
@@ -35,9 +35,9 @@ import static org.testng.Assert.assertEquals;
 public class LMGrammarBerkeleyTest {
 
   private static final String INPUT = "the chat-rooms";
-  private static final String EXPECTED_OUTPUT = "tm_glue_0=2.000 lm_0=-7.153\n";
-  private static final String EXPECTED_OUTPUT_WITH_OOV = "tm_glue_0=2.000 lm_0=-7.153 lm_0_oov=0.000\n";
-  private static final String[] OPTIONS = "-v 0 -output-format %f".split(" ");
+  private static final String EXPECTED_OUTPUT = "glue_0=-2.000000 lm_0=-7.152632\n";
+  private static final String EXPECTED_OUTPUT_WITH_OOV = "glue_0=-2.000000 lm_0=-7.152632 lm_0_oov=0.000000\n";
+  private static final String[] OPTIONS = "-v 1 -output-format %f".split(" ");
 
   private JoshuaConfiguration joshuaConfig;
   private Decoder decoder;
@@ -77,9 +77,7 @@ public class LMGrammarBerkeleyTest {
     joshuaConfig.features.add("LanguageModel -lm_type berkeleylm -oov_feature -lm_order 2 -lm_file resources/berkeley_lm/lm");
     decoder = new Decoder(joshuaConfig, null);
     final String translation = decode(INPUT).toString();
-    assertEquals(Decoder.weights.getDenseFeatures().size(), 3);
     assertEquals(translation, EXPECTED_OUTPUT_WITH_OOV);
   }
 
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/test/java/org/apache/joshua/decoder/ff/lm/class_lm/ClassBasedLanguageModelTest.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/test/java/org/apache/joshua/decoder/ff/lm/class_lm/ClassBasedLanguageModelTest.java b/joshua-core/src/test/java/org/apache/joshua/decoder/ff/lm/class_lm/ClassBasedLanguageModelTest.java
index 5946abd..0d7a9c4 100644
--- a/joshua-core/src/test/java/org/apache/joshua/decoder/ff/lm/class_lm/ClassBasedLanguageModelTest.java
+++ b/joshua-core/src/test/java/org/apache/joshua/decoder/ff/lm/class_lm/ClassBasedLanguageModelTest.java
@@ -24,6 +24,7 @@ import static org.testng.Assert.assertTrue;
 import org.apache.joshua.corpus.Vocabulary;
 import org.apache.joshua.decoder.Decoder;
 import org.apache.joshua.decoder.JoshuaConfiguration;
+import org.apache.joshua.decoder.ff.FeatureMap;
 import org.apache.joshua.decoder.ff.FeatureVector;
 import org.apache.joshua.decoder.ff.lm.LanguageModelFF;
 import org.apache.joshua.decoder.ff.tm.OwnerMap;
@@ -46,8 +47,8 @@ public class ClassBasedLanguageModelTest {
   public void setUp() {
     Decoder.resetGlobalState();
 
-    FeatureVector weights = new FeatureVector();
-    weights.set("lm_0", WEIGHT);
+    FeatureVector weights = new FeatureVector(1);
+    weights.put(FeatureMap.hashFeature("lm_0"), WEIGHT);
     String[] args = { "-lm_type", "kenlm", "-lm_order", "9",
       "-lm_file", "./src/test/resources/lm/class_lm/class_lm_9gram.gz",
       "-class_map", "./src/test/resources/lm/class_lm/class.map" };
@@ -70,7 +71,7 @@ public class ClassBasedLanguageModelTest {
   @Test
   public void givenRuleWithSingleWord_whenGetRuleId_thenIsMappedToClass() {
     final int[] target = Vocabulary.addAll(new String[] { "professionalism" });
-    final Rule rule = new Rule(0, null, target, new FeatureVector(), 0, OwnerMap.register(OwnerMap.UNKNOWN_OWNER));
+    final Rule rule = new Rule(0, null, target, 0, new FeatureVector(0), null, OwnerMap.register(OwnerMap.UNKNOWN_OWNER));
     assertEquals(Vocabulary.word(ff.getRuleIds(rule)[0]), "13");
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/test/java/org/apache/joshua/decoder/ff/tm/OwnerMapTest.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/test/java/org/apache/joshua/decoder/ff/tm/OwnerMapTest.java b/joshua-core/src/test/java/org/apache/joshua/decoder/ff/tm/OwnerMapTest.java
index 8d129e1..5e0e672 100644
--- a/joshua-core/src/test/java/org/apache/joshua/decoder/ff/tm/OwnerMapTest.java
+++ b/joshua-core/src/test/java/org/apache/joshua/decoder/ff/tm/OwnerMapTest.java
@@ -1,3 +1,21 @@
+/*
+ * 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.ff.tm;
 
 import static org.testng.Assert.assertEquals;

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/test/java/org/apache/joshua/decoder/ff/tm/RuleFactoryTest.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/test/java/org/apache/joshua/decoder/ff/tm/RuleFactoryTest.java b/joshua-core/src/test/java/org/apache/joshua/decoder/ff/tm/RuleFactoryTest.java
new file mode 100644
index 0000000..ab6864b
--- /dev/null
+++ b/joshua-core/src/test/java/org/apache/joshua/decoder/ff/tm/RuleFactoryTest.java
@@ -0,0 +1,42 @@
+/*
+ * 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.ff.tm;
+
+import static org.testng.Assert.*;
+
+import java.util.Arrays;
+
+import org.testng.annotations.Test;
+
+public class RuleFactoryTest {
+  
+  @Test
+  public void givenAlignmentString_whenParseAlignmentString_thenAlignmentsAreCorrect() {
+    // GIVEN
+    String alignmentString = "0-0 2-1 4-1 5-3 5-5";
+    byte[] expectedAlignments = new byte[] {0, 0, 2, 1, 4, 1, 5, 3, 5, 5};
+    
+    // WHEN
+    byte[] alignments = RuleFactory.parseAlignmentString(alignmentString);
+    
+    // THEN
+    assertTrue(Arrays.equals(alignments, expectedAlignments));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/test/java/org/apache/joshua/decoder/kbest_extraction/KBestExtractionTest.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/test/java/org/apache/joshua/decoder/kbest_extraction/KBestExtractionTest.java b/joshua-core/src/test/java/org/apache/joshua/decoder/kbest_extraction/KBestExtractionTest.java
index 5b9db06..41569aa 100644
--- a/joshua-core/src/test/java/org/apache/joshua/decoder/kbest_extraction/KBestExtractionTest.java
+++ b/joshua-core/src/test/java/org/apache/joshua/decoder/kbest_extraction/KBestExtractionTest.java
@@ -26,7 +26,9 @@ import org.apache.joshua.util.io.KenLmTestUtil;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
+import org.testng.reporters.Files;
 
+import java.io.File;
 import java.io.IOException;
 import java.nio.file.Path;
 import java.nio.file.Paths;
@@ -52,6 +54,7 @@ public class KBestExtractionTest {
 
   @BeforeMethod
   public void setUp() throws Exception {
+    //BROKEN
     joshuaConfig = new JoshuaConfiguration();
     joshuaConfig.readConfigFile(CONFIG);
     joshuaConfig.outputFormat = "%i ||| %s ||| %c";
@@ -68,7 +71,8 @@ public class KBestExtractionTest {
   public void givenInput_whenKbestExtraction_thenOutputIsAsExpected() throws IOException {
     final String translation = decode(INPUT).toString();
     final String gold = new String(readAllBytes(GOLD_PATH), UTF_8);
-    assertEquals(gold, translation);
+    Files.writeFile(translation, new File("resources/kbest_extraction/output.actual"));
+    assertEquals(translation, gold);
   }
 
   private Translation decode(String input) {

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/test/java/org/apache/joshua/system/AlignmentMapTest.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/test/java/org/apache/joshua/system/AlignmentMapTest.java b/joshua-core/src/test/java/org/apache/joshua/system/AlignmentMapTest.java
index 97e4906..f1d7269 100644
--- a/joshua-core/src/test/java/org/apache/joshua/system/AlignmentMapTest.java
+++ b/joshua-core/src/test/java/org/apache/joshua/system/AlignmentMapTest.java
@@ -25,6 +25,7 @@ import java.util.Map;
 
 import org.apache.joshua.corpus.Vocabulary;
 import org.apache.joshua.decoder.ff.FeatureVector;
+import org.apache.joshua.decoder.ff.tm.OwnerMap;
 import org.apache.joshua.decoder.ff.tm.Rule;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
@@ -47,20 +48,21 @@ public class AlignmentMapTest {
     int[] sourceRhs = {Vocabulary.id("A1"),Vocabulary.id("A2"),-1,Vocabulary.id("B"),Vocabulary.id("C"),-2};
     int[] targetRhs = {Vocabulary.id("c"),Vocabulary.id("b1"),-1,Vocabulary.id("b2"),-4,Vocabulary.id("a")};
     int arity = 2; // 2 non terminals
-    String alignment = "0-5 1-5 3-1 3-3 4-0";
+    byte[] alignment = new byte[] {0, 5, 1, 5, 3, 1, 3, 3, 4, 0};
     expectedAlignmentMap = new HashMap<Integer, List<Integer>>();
     expectedAlignmentMap.put(0, Arrays.asList(4));
     expectedAlignmentMap.put(5, Arrays.asList(0,1));
     expectedAlignmentMap.put(1, Arrays.asList(3));
     expectedAlignmentMap.put(3, Arrays.asList(3));
-    rule1 = new Rule(-1, sourceRhs, targetRhs, "", arity, alignment);
-    rule2 = new Rule(-1, sourceRhs, targetRhs, new FeatureVector(), arity, null); // rule with no alignment
+    rule1 = new Rule(-1, sourceRhs, targetRhs, arity, new FeatureVector(0), alignment, OwnerMap.UNKNOWN_OWNER_ID);
+    rule2 = new Rule(-1, sourceRhs, targetRhs, arity, new FeatureVector(0), new byte[] {}, OwnerMap.UNKNOWN_OWNER_ID); // rule with no alignment
   }
 
   @Test
   public void test() {
     // test regular rule with arity 2
     Map<Integer, List<Integer>> alignmentMap1 = rule1.getAlignmentMap();
+    System.out.println(alignmentMap1);
     assertEquals(expectedAlignmentMap, alignmentMap1);
     int[] nonTerminalPositions1 = rule1.getNonTerminalSourcePositions();
     assertArrayEquals(expectedNonTerminalPositions, nonTerminalPositions1);

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/test/java/org/apache/joshua/system/LmOovFeatureTest.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/test/java/org/apache/joshua/system/LmOovFeatureTest.java b/joshua-core/src/test/java/org/apache/joshua/system/LmOovFeatureTest.java
index d6fc16d..9e2f622 100644
--- a/joshua-core/src/test/java/org/apache/joshua/system/LmOovFeatureTest.java
+++ b/joshua-core/src/test/java/org/apache/joshua/system/LmOovFeatureTest.java
@@ -35,7 +35,7 @@ public class LmOovFeatureTest {
   private static final String CONFIG = "resources/lm_oov/joshua.config";
   private static final String INPUT = "a chat-rooms full";
   // expecting 2 lm oovs ('a' & 'full') and 2 grammar OOVs ('chat-rooms' & 'full') and score -198.000
-  private static final String EXPECTED_FEATURES = "tm_pt_0=-2.000 tm_glue_0=3.000 lm_0=-206.718 lm_0_oov=2.000 OOVPenalty=-200.000 | -198.000";
+  private static final String EXPECTED_FEATURES = "lm_0=-206.718124 lm_0_oov=2.000000 OOVPenalty=-200.000000 pt_0=2.000000 glue_0=-3.000000 | -198.000";
 
   private JoshuaConfiguration joshuaConfig = null;
   private Decoder decoder = null;

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/test/java/org/apache/joshua/system/StructuredOutputTest.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/test/java/org/apache/joshua/system/StructuredOutputTest.java b/joshua-core/src/test/java/org/apache/joshua/system/StructuredOutputTest.java
index 2c9e0bd..0cc8721 100644
--- a/joshua-core/src/test/java/org/apache/joshua/system/StructuredOutputTest.java
+++ b/joshua-core/src/test/java/org/apache/joshua/system/StructuredOutputTest.java
@@ -67,13 +67,13 @@ public class StructuredOutputTest {
     joshuaConfig.goal_symbol = "[GOAL]";
     joshuaConfig.default_non_terminal = "[X]";
     joshuaConfig.features.add("OOVPenalty");
-    joshuaConfig.weights.add("tm_pt_0 1");
-    joshuaConfig.weights.add("tm_pt_1 1");
-    joshuaConfig.weights.add("tm_pt_2 1");
-    joshuaConfig.weights.add("tm_pt_3 1");
-    joshuaConfig.weights.add("tm_pt_4 1");
-    joshuaConfig.weights.add("tm_pt_5 1");
-    joshuaConfig.weights.add("tm_glue_0 1");
+    joshuaConfig.weights.add("pt_0 -1");
+    joshuaConfig.weights.add("pt_1 -1");
+    joshuaConfig.weights.add("pt_2 -1");
+    joshuaConfig.weights.add("pt_3 -1");
+    joshuaConfig.weights.add("pt_4 -1");
+    joshuaConfig.weights.add("pt_5 -1");
+    joshuaConfig.weights.add("glue_0 -1");
     joshuaConfig.weights.add("OOVPenalty 2");
     decoder = new Decoder(joshuaConfig, ""); // second argument (configFile
                                              // is not even used by the
@@ -99,18 +99,15 @@ public class StructuredOutputTest {
     joshuaConfig.use_structured_output = false;
     joshuaConfig.outputFormat = "%s | %a ";
     translation = decode(input);
-    Assert.assertEquals(expectedTranslation + " | "
-        + expectedWordAlignmentString, translation.toString().trim());
+    Assert.assertEquals(translation.toString().trim(), expectedTranslation + " | " + expectedWordAlignmentString);
 
     // test structured output
     joshuaConfig.use_structured_output = true; // set structured output creation to true
     translation = decode(input);
-    Assert.assertEquals(expectedTranslation, translation.getStructuredTranslations().get(0).getTranslationString());
-    Assert.assertEquals(Arrays.asList(expectedTranslation.split("\\s+")),
-        translation.getStructuredTranslations().get(0).getTranslationTokens());
-    Assert.assertEquals(expectedScore, translation.getStructuredTranslations().get(0).getTranslationScore(),
-        0.00001);
-    Assert.assertEquals(expectedWordAlignment, translation.getStructuredTranslations().get(0).getTranslationWordAlignments());
+    Assert.assertEquals(translation.getStructuredTranslations().get(0).getTranslationString(), expectedTranslation);
+    Assert.assertEquals(translation.getStructuredTranslations().get(0).getTranslationTokens(), Arrays.asList(expectedTranslation.split("\\s+")));
+    Assert.assertEquals(translation.getStructuredTranslations().get(0).getTranslationScore(), expectedScore, 0.00001);
+    Assert.assertEquals(translation.getStructuredTranslations().get(0).getTranslationWordAlignments(), expectedWordAlignment);
     Assert.assertEquals(translation.getStructuredTranslations().get(0).getTranslationWordAlignments().size(), translation
         .getStructuredTranslations().get(0).getTranslationTokens().size());
   }

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/test/java/org/apache/joshua/system/StructuredTranslationTest.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/test/java/org/apache/joshua/system/StructuredTranslationTest.java b/joshua-core/src/test/java/org/apache/joshua/system/StructuredTranslationTest.java
index 4977d08..3a5a939 100644
--- a/joshua-core/src/test/java/org/apache/joshua/system/StructuredTranslationTest.java
+++ b/joshua-core/src/test/java/org/apache/joshua/system/StructuredTranslationTest.java
@@ -61,15 +61,14 @@ public class StructuredTranslationTest {
   private static final Map<String,Float> EXPECTED_FEATURES = new HashMap<>();
   private static final int EXPECTED_NBEST_LIST_SIZE = 8;
   static {
-    EXPECTED_FEATURES.put("tm_glue_0", 1.0f);
-    EXPECTED_FEATURES.put("tm_pt_0", -3.0f);
-    EXPECTED_FEATURES.put("tm_pt_1", -3.0f);
-    EXPECTED_FEATURES.put("tm_pt_2", -3.0f);
-    EXPECTED_FEATURES.put("tm_pt_3", -3.0f);
-    EXPECTED_FEATURES.put("tm_pt_4", -3.0f);
-    EXPECTED_FEATURES.put("tm_pt_5", -3.0f);
-    EXPECTED_FEATURES.put("OOV", 7.0f);
-    EXPECTED_FEATURES.put("OOVPenalty", 0.0f);
+    EXPECTED_FEATURES.put("glue_0", -1.0f);
+    EXPECTED_FEATURES.put("pt_0", 3.0f);
+    EXPECTED_FEATURES.put("pt_1", 3.0f);
+    EXPECTED_FEATURES.put("pt_2", 3.0f);
+    EXPECTED_FEATURES.put("pt_3", 3.0f);
+    EXPECTED_FEATURES.put("pt_4", 3.0f);
+    EXPECTED_FEATURES.put("pt_5", 3.0f);
+    EXPECTED_FEATURES.put("pt_OOV", 7.0f);
   }
 
   @BeforeMethod
@@ -86,13 +85,13 @@ public class StructuredTranslationTest {
     joshuaConfig.goal_symbol = "[GOAL]";
     joshuaConfig.default_non_terminal = "[X]";
     joshuaConfig.features.add("OOVPenalty");
-    joshuaConfig.weights.add("tm_pt_0 1");
-    joshuaConfig.weights.add("tm_pt_1 1");
-    joshuaConfig.weights.add("tm_pt_2 1");
-    joshuaConfig.weights.add("tm_pt_3 1");
-    joshuaConfig.weights.add("tm_pt_4 1");
-    joshuaConfig.weights.add("tm_pt_5 1");
-    joshuaConfig.weights.add("tm_glue_0 1");
+    joshuaConfig.weights.add("pt_0 -1");
+    joshuaConfig.weights.add("pt_1 -1");
+    joshuaConfig.weights.add("pt_2 -1");
+    joshuaConfig.weights.add("pt_3 -1");
+    joshuaConfig.weights.add("pt_4 -1");
+    joshuaConfig.weights.add("pt_5 -1");
+    joshuaConfig.weights.add("glue_0 -1");
     joshuaConfig.weights.add("OOVPenalty 1");
     decoder = new Decoder(joshuaConfig, ""); // second argument (configFile
                                              // is not even used by the
@@ -120,7 +119,7 @@ public class StructuredTranslationTest {
     final String translation = decode(INPUT).toString().trim();
 
     // THEN
-    assertEquals(EXPECTED_TRANSLATION + " | " + EXPECTED_WORD_ALIGNMENT_STRING, translation);
+    assertEquals(translation, EXPECTED_TRANSLATION + " | " + EXPECTED_WORD_ALIGNMENT_STRING);
   }
 
   @Test
@@ -134,8 +133,8 @@ public class StructuredTranslationTest {
     final String translation = decode(INPUT).toString().trim();
 
     // THEN
-    assertEquals(EXPECTED_TRANSLATION + " | " + INPUT + " | " + EXPECTED_WORD_ALIGNMENT_STRING + String.format(" | %.3f", EXPECTED_SCORE),
-        translation);
+    assertEquals(translation,
+        EXPECTED_TRANSLATION + " | " + INPUT + " | " + EXPECTED_WORD_ALIGNMENT_STRING + String.format(" | %.3f", EXPECTED_SCORE));
   }
 
   @Test
@@ -155,12 +154,12 @@ public class StructuredTranslationTest {
 
     // THEN
     assertTrue(translation.getStructuredTranslations().size() == 1);
-    assertEquals(EXPECTED_TRANSLATION, translationString);
-    assertEquals(EXPECTED_TRANSLATED_TOKENS, translatedTokens);
-    assertEquals(EXPECTED_SCORE, translationScore, 0.00001);
-    assertEquals(EXPECTED_WORD_ALIGNMENT, wordAlignment);
-    assertEquals(wordAlignment.size(), translatedTokens.size());
-    assertEquals(EXPECTED_FEATURES.entrySet(), translationFeatures.entrySet());
+    assertEquals(translationString, EXPECTED_TRANSLATION);
+    assertEquals(translatedTokens, EXPECTED_TRANSLATED_TOKENS);
+    assertEquals(translationScore, EXPECTED_SCORE, 0.00001);
+    assertEquals(wordAlignment, EXPECTED_WORD_ALIGNMENT);
+    assertEquals(translatedTokens.size(), wordAlignment.size());
+    assertEquals(translationFeatures.entrySet(), EXPECTED_FEATURES.entrySet());
   }
 
   @Test
@@ -179,14 +178,14 @@ public class StructuredTranslationTest {
     final List<List<Integer>> wordAlignment = structuredTranslation.getTranslationWordAlignments();
     final Map<String,Float> translationFeatures = structuredTranslation.getTranslationFeatures();
 
-    // THEN
+    // THEN   
     assertTrue(structuredTranslations.size() == 1);
-    assertEquals(EXPECTED_TRANSLATION, translationString);
-    assertEquals(EXPECTED_TRANSLATED_TOKENS, translatedTokens);
-    assertEquals(EXPECTED_SCORE, translationScore, 0.00001);
-    assertEquals(EXPECTED_WORD_ALIGNMENT, wordAlignment);
-    assertEquals(wordAlignment.size(), translatedTokens.size());
-    assertEquals(EXPECTED_FEATURES.entrySet(), translationFeatures.entrySet());
+    assertEquals(translationString, EXPECTED_TRANSLATION);
+    assertEquals(translatedTokens, EXPECTED_TRANSLATED_TOKENS);
+    assertEquals(translationScore, EXPECTED_SCORE, 0.00001);
+    assertEquals(wordAlignment, EXPECTED_WORD_ALIGNMENT);
+    assertEquals(translatedTokens.size(), wordAlignment.size());
+    assertEquals(translationFeatures.entrySet(), EXPECTED_FEATURES.entrySet());
   }
 
   @Test
@@ -204,14 +203,14 @@ public class StructuredTranslationTest {
     // THEN
     assertEquals(structuredTranslations.size(), EXPECTED_NBEST_LIST_SIZE);
     assertTrue(structuredTranslations.size() > 1);
-    assertEquals(EXPECTED_TRANSLATION, viterbiTranslation.getTranslationString());
-    assertEquals(EXPECTED_TRANSLATED_TOKENS, viterbiTranslation.getTranslationTokens());
-    assertEquals(EXPECTED_SCORE, viterbiTranslation.getTranslationScore(), 0.00001);
-    assertEquals(EXPECTED_WORD_ALIGNMENT, viterbiTranslation.getTranslationWordAlignments());
-    assertEquals(EXPECTED_FEATURES.entrySet(), viterbiTranslation.getTranslationFeatures().entrySet());
+    assertEquals(viterbiTranslation.getTranslationString(), EXPECTED_TRANSLATION);
+    assertEquals(viterbiTranslation.getTranslationTokens(), EXPECTED_TRANSLATED_TOKENS);
+    assertEquals(viterbiTranslation.getTranslationScore(), EXPECTED_SCORE, 0.00001);
+    assertEquals(viterbiTranslation.getTranslationWordAlignments(), EXPECTED_WORD_ALIGNMENT);
+    assertEquals(viterbiTranslation.getTranslationFeatures().entrySet(), EXPECTED_FEATURES.entrySet());
     // last entry in KBEST is all input words untranslated, should have 8 OOVs.
-    assertEquals(INPUT, lastKBest.getTranslationString());
-    assertEquals(-800.0, lastKBest.getTranslationFeatures().get("OOVPenalty"), 0.0001);
+    assertEquals(lastKBest.getTranslationString(), INPUT);
+    assertEquals(lastKBest.getTranslationFeatures().get("OOVPenalty"), -800.0, 0.0001);
 
   }
 


[3/4] incubator-joshua git commit: Major refactoring of core decoder components (Rule.java, FeatureVector.java and grammars). The core idea of this change is to simplify feature handling inside Joshua. Please note that this change is NOT backwards compat

Posted by mj...@apache.org.
Major refactoring of core decoder components (Rule.java, FeatureVector.java and grammars). The core idea of this change is to simplify feature handling inside Joshua. Please note that this change is NOT backwards compatible. The following changes were made:
- No distinction between sparse and dense features inside the decoder anymore. Each feature stored at the rule is 'owned' by the grammar that contains the rule. An 'owned' feature simply means that its name is prepended with the owner string: 0=0.2 becomes <owner>_0=0.2. This applies to both dense features (features that occur at every rule), as well as sparse features. Please note that the old prefix 'tm_' is no longer used.
- Having only one type of feature, a revised version of FeatureVector.java was built that is greatly simplified. It is basically a HashMap of FeatureId (typed as ints) to feature values. FeatureIds are created/hashed by the new global mapping FeatureMap.java, which maintains a bidirectional mapping between feature ids and feature names. This also allowed getting rid of storing feature names in the vocabulary.
- The simplified FeatureVector cause removal of all 'reportDenseFeatures'/'getNumDenseFeatures' method in the decoder and the grammar interface.
- The tradition but very obscure way of flipping the sign of dense features but not sparse features was removed. The feature value in the decoder is now just the value as you see it stored at the rule.
- The Rule class was changed to adhere to object-oriented principles. It now has only one constructor that requires all of its dependencies and these can not be changed later. This forces Rule creators to finalize the dependencies (deciding on an owner of the rule and the hashing of the feature vector).
- Also the unused concept of the precomputableCost in a rule was removed. Rules still 'cache' their estimated cost.
- The various Grammar and MemoryBasedBatchGrammar constructors were unified and a lot of old obscure code was removed.
- Due to the change above, the PhraseModel feature function that fires feature values for features stored at rules is greatly simplified.
- As featureVectors at Rules are final and have to have an owner, feature sharing across multiple grammars would need to be handled by a separate feature function implementation which is transparent.

This commit also updates all existing (and enabled) Unit tests which also pass. Existing regression tests do NOT work in this commit since many of the grammars are packed and would need to be re-packed.


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

Branch: refs/heads/7
Commit: 20afddf85263f3def242f721728ac148ef143ad5
Parents: 0ad45cc
Author: Felix Hieber <fh...@amazon.com>
Authored: Wed Aug 17 13:52:39 2016 +0200
Committer: Felix Hieber <fh...@amazon.com>
Committed: Thu Aug 18 09:32:29 2016 +0200

----------------------------------------------------------------------
 .../resources/kbest_extraction/joshua.config    |   4 +-
 joshua-core/resources/wa_grammar.packed/config  |   2 +-
 .../resources/wa_grammar.packed/encoding        | Bin 154 -> 154 bytes
 .../wa_grammar.packed/slice_00000.source        | Bin 204 -> 204 bytes
 .../wa_grammar.packed/slice_00000.target        | Bin 128 -> 128 bytes
 .../resources/wa_grammar.packed/vocabulary      | Bin 238 -> 187 bytes
 .../java/org/apache/joshua/decoder/BLEU.java    |   2 +-
 .../java/org/apache/joshua/decoder/Decoder.java |  95 +---
 .../apache/joshua/decoder/DecoderThread.java    |   2 +-
 .../joshua/decoder/JoshuaConfiguration.java     |   7 +-
 .../joshua/decoder/StructuredTranslation.java   |   3 -
 .../decoder/StructuredTranslationFactory.java   |   5 +-
 .../org/apache/joshua/decoder/Translation.java  |  16 +-
 .../decoder/chart_parser/ComputeNodeResult.java |  15 +-
 .../joshua/decoder/ff/ArityPhrasePenalty.java   |   3 +-
 .../joshua/decoder/ff/FeatureFunction.java      |  63 +--
 .../apache/joshua/decoder/ff/FeatureMap.java    | 107 +++++
 .../apache/joshua/decoder/ff/FeatureVector.java | 401 ++++-------------
 .../joshua/decoder/ff/LabelCombinationFF.java   |   4 +-
 .../joshua/decoder/ff/LabelSubstitutionFF.java  |   8 +-
 .../joshua/decoder/ff/LexicalFeatures.java      |  31 +-
 .../apache/joshua/decoder/ff/OOVPenalty.java    |  22 +-
 .../apache/joshua/decoder/ff/PhraseModel.java   |  85 +---
 .../apache/joshua/decoder/ff/PhrasePenalty.java |  19 +-
 .../apache/joshua/decoder/ff/RuleCountBin.java  |  77 ----
 .../org/apache/joshua/decoder/ff/RuleFF.java    |  20 +-
 .../apache/joshua/decoder/ff/RuleLength.java    |  12 +-
 .../org/apache/joshua/decoder/ff/RuleShape.java |  12 +-
 .../apache/joshua/decoder/ff/SourcePathFF.java  |  12 +-
 .../apache/joshua/decoder/ff/TargetBigram.java  |   8 +-
 .../apache/joshua/decoder/ff/WordPenalty.java   |  15 +-
 .../decoder/ff/fragmentlm/FragmentLMFF.java     |  54 +--
 .../joshua/decoder/ff/fragmentlm/Tree.java      |  16 +-
 .../joshua/decoder/ff/lm/LanguageModelFF.java   |  43 +-
 .../ff/lm/StateMinimizingLanguageModel.java     |   8 +-
 .../joshua/decoder/ff/phrase/Distortion.java    |  19 +-
 .../ff/similarity/EdgePhraseSimilarityFF.java   |   4 +-
 .../joshua/decoder/ff/tm/AbstractGrammar.java   |   4 +-
 .../apache/joshua/decoder/ff/tm/Grammar.java    |   7 -
 .../joshua/decoder/ff/tm/GrammarReader.java     |  44 +-
 .../org/apache/joshua/decoder/ff/tm/Rule.java   | 440 +++++--------------
 .../joshua/decoder/ff/tm/RuleFactory.java       |  88 ++++
 .../decoder/ff/tm/format/HieroFormatReader.java |  28 +-
 .../decoder/ff/tm/format/MosesFormatReader.java |  18 +-
 .../tm/hash_based/MemoryBasedBatchGrammar.java  |  96 ++--
 .../ff/tm/hash_based/MemoryBasedRuleBin.java    |   6 +-
 .../decoder/ff/tm/packed/PackedGrammar.java     | 157 +++----
 .../hypergraph/FeatureVectorExtractor.java      |   6 +-
 .../GrammarBuilderWalkerFunction.java           |  35 +-
 .../joshua/decoder/hypergraph/HyperGraph.java   |   2 +-
 .../decoder/hypergraph/KBestExtractor.java      |  25 +-
 .../hypergraph/OutputStringExtractor.java       |   2 +-
 .../decoder/hypergraph/ViterbiExtractor.java    |   5 +-
 .../decoder/hypergraph/WordAlignmentState.java  |   4 +-
 .../apache/joshua/decoder/phrase/Candidate.java |   2 +-
 .../joshua/decoder/phrase/Hypothesis.java       |  11 +-
 .../joshua/decoder/phrase/PhraseChart.java      |   2 +-
 .../joshua/decoder/phrase/PhraseTable.java      |  23 +-
 .../org/apache/joshua/decoder/phrase/Stack.java |   8 +-
 .../joshua/oracle/OracleExtractionHG.java       |   2 +-
 .../org/apache/joshua/server/ServerThread.java  |  26 +-
 .../org/apache/joshua/tools/GrammarPacker.java  |  77 ++--
 .../java/org/apache/joshua/util/Constants.java  |  10 +-
 .../apache/joshua/util/PackedGrammarServer.java |   8 +-
 .../util/encoding/EncoderConfiguration.java     |   5 +-
 .../util/encoding/FeatureTypeAnalyzer.java      |  10 +-
 .../joshua/decoder/ff/FeatureVectorTest.java    | 120 +++++
 .../decoder/ff/lm/LanguageModelFFTest.java      |   5 +-
 .../lm/berkeley_lm/LMGrammarBerkeleyTest.java   |   8 +-
 .../class_lm/ClassBasedLanguageModelTest.java   |   7 +-
 .../joshua/decoder/ff/tm/OwnerMapTest.java      |  18 +
 .../joshua/decoder/ff/tm/RuleFactoryTest.java   |  42 ++
 .../kbest_extraction/KBestExtractionTest.java   |   6 +-
 .../apache/joshua/system/AlignmentMapTest.java  |   8 +-
 .../apache/joshua/system/LmOovFeatureTest.java  |   2 +-
 .../joshua/system/StructuredOutputTest.java     |  27 +-
 .../system/StructuredTranslationTest.java       |  77 ++--
 77 files changed, 1098 insertions(+), 1567 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/resources/kbest_extraction/joshua.config
----------------------------------------------------------------------
diff --git a/joshua-core/resources/kbest_extraction/joshua.config b/joshua-core/resources/kbest_extraction/joshua.config
index cdab98e..4a78956 100644
--- a/joshua-core/resources/kbest_extraction/joshua.config
+++ b/joshua-core/resources/kbest_extraction/joshua.config
@@ -22,6 +22,6 @@ feature_function = OOVPenalty
 # Model Weights ####
 
 lm_0 1
-tm_pt_0 1
-tm_glue_0 1
+pt_0 -1
+glue_0 -1
 OOVPenalty 10000

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/resources/wa_grammar.packed/config
----------------------------------------------------------------------
diff --git a/joshua-core/resources/wa_grammar.packed/config b/joshua-core/resources/wa_grammar.packed/config
index fbc07d0..2984a0e 100644
--- a/joshua-core/resources/wa_grammar.packed/config
+++ b/joshua-core/resources/wa_grammar.packed/config
@@ -1,2 +1,2 @@
-max-source-len = 6
 version = 3
+max-source-len = 6

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/resources/wa_grammar.packed/encoding
----------------------------------------------------------------------
diff --git a/joshua-core/resources/wa_grammar.packed/encoding b/joshua-core/resources/wa_grammar.packed/encoding
index 630f69f..64cf663 100644
Binary files a/joshua-core/resources/wa_grammar.packed/encoding and b/joshua-core/resources/wa_grammar.packed/encoding differ

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/resources/wa_grammar.packed/slice_00000.source
----------------------------------------------------------------------
diff --git a/joshua-core/resources/wa_grammar.packed/slice_00000.source b/joshua-core/resources/wa_grammar.packed/slice_00000.source
index 4607b89..e3b83de 100644
Binary files a/joshua-core/resources/wa_grammar.packed/slice_00000.source and b/joshua-core/resources/wa_grammar.packed/slice_00000.source differ

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/resources/wa_grammar.packed/slice_00000.target
----------------------------------------------------------------------
diff --git a/joshua-core/resources/wa_grammar.packed/slice_00000.target b/joshua-core/resources/wa_grammar.packed/slice_00000.target
index fe11a38..761ec14 100644
Binary files a/joshua-core/resources/wa_grammar.packed/slice_00000.target and b/joshua-core/resources/wa_grammar.packed/slice_00000.target differ

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/resources/wa_grammar.packed/vocabulary
----------------------------------------------------------------------
diff --git a/joshua-core/resources/wa_grammar.packed/vocabulary b/joshua-core/resources/wa_grammar.packed/vocabulary
index 637651e..46c401f 100644
Binary files a/joshua-core/resources/wa_grammar.packed/vocabulary and b/joshua-core/resources/wa_grammar.packed/vocabulary differ

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/main/java/org/apache/joshua/decoder/BLEU.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/decoder/BLEU.java b/joshua-core/src/main/java/org/apache/joshua/decoder/BLEU.java
index 8b51403..a6e02b2 100644
--- a/joshua-core/src/main/java/org/apache/joshua/decoder/BLEU.java
+++ b/joshua-core/src/main/java/org/apache/joshua/decoder/BLEU.java
@@ -367,7 +367,7 @@ public class BLEU {
 
     Rule rule = edge.getRule();
     if (rule != null) {
-      int[] symbols = rule.getEnglish();
+      int[] symbols = rule.getTarget();
 
 //      System.err.println(String.format("compute(%s)", rule));
       

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/main/java/org/apache/joshua/decoder/Decoder.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/decoder/Decoder.java b/joshua-core/src/main/java/org/apache/joshua/decoder/Decoder.java
index e1eadb1..5b0ae0f 100644
--- a/joshua-core/src/main/java/org/apache/joshua/decoder/Decoder.java
+++ b/joshua-core/src/main/java/org/apache/joshua/decoder/Decoder.java
@@ -18,27 +18,28 @@
  */
 package org.apache.joshua.decoder;
 
-import static org.apache.joshua.decoder.ff.FeatureVector.DENSE_FEATURE_NAMES;
+import static org.apache.joshua.decoder.ff.FeatureMap.hashFeature;
 import static org.apache.joshua.decoder.ff.tm.OwnerMap.getOwner;
+import static org.apache.joshua.util.Constants.spaceSeparator;
 
 import java.io.BufferedWriter;
 import java.io.File;
-import java.io.IOException;
 import java.io.FileNotFoundException;
+import java.io.IOException;
 import java.lang.reflect.Constructor;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
+import java.util.Map.Entry;
 import java.util.Set;
 import java.util.concurrent.ArrayBlockingQueue;
 import java.util.concurrent.BlockingQueue;
 
-import com.google.common.base.Strings;
-
 import org.apache.joshua.corpus.Vocabulary;
-import org.apache.joshua.decoder.ff.FeatureVector;
 import org.apache.joshua.decoder.ff.FeatureFunction;
+import org.apache.joshua.decoder.ff.FeatureMap;
+import org.apache.joshua.decoder.ff.FeatureVector;
 import org.apache.joshua.decoder.ff.PhraseModel;
 import org.apache.joshua.decoder.ff.StatefulFF;
 import org.apache.joshua.decoder.ff.lm.LanguageModelFF;
@@ -59,6 +60,8 @@ import org.apache.joshua.util.io.LineReader;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.base.Strings;
+
 /**
  * This class handles decoder initialization and the complication introduced by multithreading.
  *
@@ -100,9 +103,9 @@ public class Decoder {
    * overhead, but it can be problematic because of unseen dependencies (for example, in the
    * Vocabulary shared by language model, translation grammar, etc).
    */
-  private List<Grammar> grammars;
-  private ArrayList<FeatureFunction> featureFunctions;
-  private Grammar customPhraseTable;
+  private final List<Grammar> grammars = new ArrayList<Grammar>();
+  private final ArrayList<FeatureFunction> featureFunctions = new ArrayList<>();
+  private Grammar customPhraseTable = null;
 
   /* The feature weights. */
   public static FeatureVector weights;
@@ -146,10 +149,8 @@ public class Decoder {
    */
   private Decoder(JoshuaConfiguration joshuaConfiguration) {
     this.joshuaConfiguration = joshuaConfiguration;
-    this.grammars = new ArrayList<Grammar>();
     this.threadPool = new ArrayBlockingQueue<DecoderThread>(
         this.joshuaConfiguration.num_parallel_decoders, true);
-    this.customPhraseTable = null;
   }
 
   /**
@@ -211,32 +212,6 @@ public class Decoder {
       }
     }
 
-    /**
-     * Strips the nonterminals from the lefthand side of the rule.
-     *
-     * @param rule
-     * @return
-     */
-    private String formatRule(Rule rule) {
-      String ruleString = "";
-      boolean first = true;
-      for (int word: rule.getFrench()) {
-        if (!first)
-          ruleString += " " + Vocabulary.word(word);
-        first = false;
-      }
-
-      ruleString += " |||"; // space will get added with first English word
-      first = true;
-      for (int word: rule.getEnglish()) {
-        if (!first)
-          ruleString += " " + Vocabulary.word(word);
-        first = false;
-      }
-
-      // strip of the leading space
-      return ruleString.substring(1);
-    }
   }
 
   /**
@@ -368,7 +343,7 @@ public class Decoder {
   public static void resetGlobalState() {
     // clear/reset static variables
     OwnerMap.clear();
-    DENSE_FEATURE_NAMES.clear();
+    FeatureMap.clear();
     Vocabulary.clear();
     Vocabulary.unregisterLanguageModels();
     LanguageModelFF.resetLmIndex();
@@ -438,23 +413,6 @@ public class Decoder {
   // ===============================================================
 
   /**
-   * Moses requires the pattern .*_.* for sparse features, and prohibits underscores in dense features. 
-   * This conforms to that pattern. We assume non-conforming dense features start with tm_ or lm_,
-   * and the only sparse feature that needs converting is OOVPenalty.
-   *
-   * @param feature
-   * @return the feature in Moses format
-   */
-  private String mosesize(String feature) {
-    if (joshuaConfiguration.moses) {
-      if (feature.startsWith("tm_") || feature.startsWith("lm_"))
-        return feature.replace("_", "-");
-    }
-
-    return feature;
-  }
-
-  /**
    * Initialize all parts of the JoshuaDecoder.
    *
    * @param configFile File containing configuration options
@@ -501,13 +459,13 @@ public class Decoder {
           throw new RuntimeException(errMsg.toString());
         }
 
-        weights.set(pair[0], Float.parseFloat(pair[1]));
+        weights.add(hashFeature(pair[0]), Float.parseFloat(pair[1]));
       }
 
-      LOG.info("Read {} weights ({} of them dense)", weights.size(), DENSE_FEATURE_NAMES.size());
+      LOG.info("Read {} weights", weights.size());
 
       // Do this before loading the grammars and the LM.
-      this.featureFunctions = new ArrayList<FeatureFunction>();
+      this.featureFunctions.clear();
 
       // Initialize and load grammars. This must happen first, since the vocab gets defined by
       // the packed grammar (if any)
@@ -520,13 +478,10 @@ public class Decoder {
 
       // This is mostly for compatibility with the Moses tuning script
       if (joshuaConfiguration.show_weights_and_quit) {
-        for (int i = 0; i < DENSE_FEATURE_NAMES.size(); i++) {
-          String name = DENSE_FEATURE_NAMES.get(i);
-          if (joshuaConfiguration.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)));
+        for (Entry<Integer, Float> entry : weights.entrySet()) {
+          System.out.println(String.format("%s=%.5f", FeatureMap.getFeature(entry.getKey()), entry.getValue()));
         }
+        // TODO (fhieber): this functionality should not be in main Decoder class and simply exit.
         System.exit(0);
       }
 
@@ -617,7 +572,7 @@ public class Decoder {
     
     /* Add the grammar for custom entries */
     if (joshuaConfiguration.search_algorithm.equals("stack"))
-      this.customPhraseTable = new PhraseTable(null, "custom", "phrase", joshuaConfiguration);
+      this.customPhraseTable = new PhraseTable("custom", joshuaConfiguration);
     else
       this.customPhraseTable = new MemoryBasedBatchGrammar("custom", joshuaConfiguration, 20);
     this.grammars.add(this.customPhraseTable);
@@ -626,7 +581,7 @@ public class Decoder {
     if (joshuaConfiguration.lattice_decoding) {
       LOG.info("Creating an epsilon-deleting grammar");
       MemoryBasedBatchGrammar latticeGrammar = new MemoryBasedBatchGrammar("lattice", joshuaConfiguration, -1);
-      HieroFormatReader reader = new HieroFormatReader();
+      HieroFormatReader reader = new HieroFormatReader(OwnerMap.register("lattice"));
 
       String goalNT = FormatUtils.cleanNonTerminal(joshuaConfiguration.goal_symbol);
       String defaultNT = FormatUtils.cleanNonTerminal(joshuaConfiguration.default_non_terminal);
@@ -686,7 +641,7 @@ public class Decoder {
    * FEATURE_NAME WEIGHT
    */
   private void readWeights(String fileName) {
-    Decoder.weights = new FeatureVector();
+    Decoder.weights = new FeatureVector(5);
 
     if (fileName.equals(""))
       return;
@@ -695,13 +650,13 @@ public class Decoder {
       LineReader lineReader = new LineReader(fileName);
 
       for (String line : lineReader) {
-        line = line.replaceAll("\\s+", " ");
+        line = line.replaceAll(spaceSeparator, " ");
 
         if (line.equals("") || line.startsWith("#") || line.startsWith("//")
             || line.indexOf(' ') == -1)
           continue;
 
-        String tokens[] = line.split("\\s+");
+        String tokens[] = line.split(spaceSeparator);
         String feature = tokens[0];
         Float value = Float.parseFloat(tokens[1]);
 
@@ -710,7 +665,7 @@ public class Decoder {
           feature = demoses(feature);
         }
 
-        weights.increment(feature, value);
+        weights.add(hashFeature(feature), value);
       }
     } catch (IOException ioe) {
       throw new RuntimeException(ioe);
@@ -765,8 +720,6 @@ public class Decoder {
     for (FeatureFunction feature : featureFunctions) {
       LOG.info("FEATURE: {}", feature.logString());
     }
-
-    weights.registerDenseFeatures(featureFunctions);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/main/java/org/apache/joshua/decoder/DecoderThread.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/decoder/DecoderThread.java b/joshua-core/src/main/java/org/apache/joshua/decoder/DecoderThread.java
index d6f5233..d095e8d 100644
--- a/joshua-core/src/main/java/org/apache/joshua/decoder/DecoderThread.java
+++ b/joshua-core/src/main/java/org/apache/joshua/decoder/DecoderThread.java
@@ -193,7 +193,7 @@ public class DecoderThread extends Thread {
   }
 
   private Grammar getGrammarFromHyperGraph(String goal, HyperGraph hg) {
-    GrammarBuilderWalkerFunction f = new GrammarBuilderWalkerFunction(goal,joshuaConfiguration);
+    GrammarBuilderWalkerFunction f = new GrammarBuilderWalkerFunction(goal, joshuaConfiguration, "pt");
     ForestWalker walker = new ForestWalker();
     walker.walk(hg.goalNode, f);
     return f.getGrammar();

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/main/java/org/apache/joshua/decoder/JoshuaConfiguration.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/decoder/JoshuaConfiguration.java b/joshua-core/src/main/java/org/apache/joshua/decoder/JoshuaConfiguration.java
index e7ad3b4..9b07a04 100644
--- a/joshua-core/src/main/java/org/apache/joshua/decoder/JoshuaConfiguration.java
+++ b/joshua-core/src/main/java/org/apache/joshua/decoder/JoshuaConfiguration.java
@@ -18,15 +18,16 @@
  */
 package org.apache.joshua.decoder;
 
+import static org.apache.joshua.util.Constants.TM_PREFIX;
 import static org.apache.joshua.util.FormatUtils.cleanNonTerminal;
 import static org.apache.joshua.util.FormatUtils.ensureNonTerminalBrackets;
 
+import java.io.BufferedReader;
 import java.io.File;
+import java.io.FileReader;
 import java.io.FileWriter;
 import java.io.IOException;
 import java.io.PrintWriter;
-import java.io.BufferedReader;
-import java.io.FileReader;
 import java.util.ArrayList;
 import java.util.Collections;
 
@@ -409,7 +410,7 @@ public class JoshuaConfiguration {
               features.add(String.format("LanguageModel -lm_type %s -lm_order %s -lm_file %s",
                   tokens[0], tokens[1], tokens[5]));
 
-          } else if (parameter.equals(normalize_key("tm"))) {
+          } else if (parameter.equals(normalize_key(TM_PREFIX))) {
             /* If found, convert old format:
              *   tm = TYPE OWNER MAXSPAN PATH
              * to new format

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/main/java/org/apache/joshua/decoder/StructuredTranslation.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/decoder/StructuredTranslation.java b/joshua-core/src/main/java/org/apache/joshua/decoder/StructuredTranslation.java
index b44a7f0..0e167d1 100644
--- a/joshua-core/src/main/java/org/apache/joshua/decoder/StructuredTranslation.java
+++ b/joshua-core/src/main/java/org/apache/joshua/decoder/StructuredTranslation.java
@@ -21,9 +21,6 @@ package org.apache.joshua.decoder;
 import java.util.List;
 import java.util.Map;
 
-import org.apache.joshua.decoder.ff.FeatureVector;
-import org.apache.joshua.decoder.hypergraph.KBestExtractor.DerivationState;
-import org.apache.joshua.decoder.io.DeNormalize;
 import org.apache.joshua.decoder.segment_file.Sentence;
 import org.apache.joshua.decoder.segment_file.Token;
 import org.apache.joshua.util.FormatUtils;

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/main/java/org/apache/joshua/decoder/StructuredTranslationFactory.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/decoder/StructuredTranslationFactory.java b/joshua-core/src/main/java/org/apache/joshua/decoder/StructuredTranslationFactory.java
index 916a5a7..ce4f32c 100644
--- a/joshua-core/src/main/java/org/apache/joshua/decoder/StructuredTranslationFactory.java
+++ b/joshua-core/src/main/java/org/apache/joshua/decoder/StructuredTranslationFactory.java
@@ -29,7 +29,6 @@ import static org.apache.joshua.util.FormatUtils.removeSentenceMarkers;
 import java.util.List;
 
 import org.apache.joshua.decoder.ff.FeatureFunction;
-import org.apache.joshua.decoder.ff.FeatureVector;
 import org.apache.joshua.decoder.hypergraph.HyperGraph;
 import org.apache.joshua.decoder.hypergraph.KBestExtractor.DerivationState;
 import org.apache.joshua.decoder.segment_file.Sentence;
@@ -62,7 +61,7 @@ public class StructuredTranslationFactory {
         extractTranslationTokens(translationString),
         extractTranslationScore(hypergraph),
         getViterbiWordAlignmentList(hypergraph),
-        getViterbiFeatures(hypergraph, featureFunctions, sourceSentence).getMap(),
+        getViterbiFeatures(hypergraph, featureFunctions, sourceSentence).toStringMap(),
         (System.currentTimeMillis() - startTime) / 1000.0f);
   }
   
@@ -93,7 +92,7 @@ public class StructuredTranslationFactory {
         extractTranslationTokens(translationString),
         derivationState.getModelCost(),
         derivationState.getWordAlignmentList(),
-        derivationState.getFeatures().getMap(),
+        derivationState.getFeatures().toStringMap(),
         (System.currentTimeMillis() - startTime) / 1000.0f);
   }
   

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/main/java/org/apache/joshua/decoder/Translation.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/decoder/Translation.java b/joshua-core/src/main/java/org/apache/joshua/decoder/Translation.java
index 46f3061..cdda63e 100644
--- a/joshua-core/src/main/java/org/apache/joshua/decoder/Translation.java
+++ b/joshua-core/src/main/java/org/apache/joshua/decoder/Translation.java
@@ -18,12 +18,13 @@
  */
 package org.apache.joshua.decoder;
 
+import static java.util.Arrays.asList;
+import static org.apache.joshua.decoder.StructuredTranslationFactory.fromViterbiDerivation;
+import static org.apache.joshua.decoder.ff.FeatureMap.hashFeature;
 import static org.apache.joshua.decoder.hypergraph.ViterbiExtractor.getViterbiFeatures;
 import static org.apache.joshua.decoder.hypergraph.ViterbiExtractor.getViterbiString;
 import static org.apache.joshua.decoder.hypergraph.ViterbiExtractor.getViterbiWordAlignments;
-import static org.apache.joshua.decoder.StructuredTranslationFactory.fromViterbiDerivation;
 import static org.apache.joshua.util.FormatUtils.removeSentenceMarkers;
-import static java.util.Arrays.asList;
 
 import java.io.BufferedWriter;
 import java.io.IOException;
@@ -110,10 +111,6 @@ public class Translation {
           
           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 */
@@ -138,7 +135,7 @@ public class Translation {
 
             if (joshuaConfiguration.outputFormat.contains("%f")) {
               final FeatureVector features = getViterbiFeatures(hypergraph, featureFunctions, source);
-              translation = translation.replace("%f", joshuaConfiguration.moses ? features.mosesString() : features.toString());
+              translation = translation.replace("%f", features.textFormat());
             }
 
             out.write(translation);
@@ -151,10 +148,11 @@ public class Translation {
             kBestExtractor.lazyKBestExtractOnHG(hypergraph, joshuaConfiguration.topN, out);
 
             if (joshuaConfiguration.rescoreForest) {
-              Decoder.weights.increment("BLEU", joshuaConfiguration.rescoreForestWeight);
+              final int bleuFeatureHash = hashFeature("BLEU");
+              Decoder.weights.add(bleuFeatureHash, joshuaConfiguration.rescoreForestWeight);
               kBestExtractor.lazyKBestExtractOnHG(hypergraph, joshuaConfiguration.topN, out);
 
-              Decoder.weights.increment("BLEU", -joshuaConfiguration.rescoreForestWeight);
+              Decoder.weights.add(bleuFeatureHash, -joshuaConfiguration.rescoreForestWeight);
               kBestExtractor.lazyKBestExtractOnHG(hypergraph, joshuaConfiguration.topN, out);
             }
           }

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/main/java/org/apache/joshua/decoder/chart_parser/ComputeNodeResult.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/decoder/chart_parser/ComputeNodeResult.java b/joshua-core/src/main/java/org/apache/joshua/decoder/chart_parser/ComputeNodeResult.java
index 9833734..1fb1031 100644
--- a/joshua-core/src/main/java/org/apache/joshua/decoder/chart_parser/ComputeNodeResult.java
+++ b/joshua-core/src/main/java/org/apache/joshua/decoder/chart_parser/ComputeNodeResult.java
@@ -18,14 +18,15 @@
  */
 package org.apache.joshua.decoder.chart_parser;
 
-import java.util.ArrayList;
+import static org.apache.joshua.decoder.ff.FeatureMap.hashFeature;
 
+import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.joshua.decoder.Decoder;
-import org.apache.joshua.decoder.ff.StatefulFF;
 import org.apache.joshua.decoder.ff.FeatureFunction;
 import org.apache.joshua.decoder.ff.FeatureVector;
+import org.apache.joshua.decoder.ff.StatefulFF;
 import org.apache.joshua.decoder.ff.state_maintenance.DPState;
 import org.apache.joshua.decoder.ff.tm.Rule;
 import org.apache.joshua.decoder.hypergraph.HGNode;
@@ -120,8 +121,8 @@ public class ComputeNodeResult {
 
       if (LOG.isDebugEnabled()) {
         LOG.debug("FEATURE {} = {} * {} = {}", feature.getName(),
-            acc.getScore() / Decoder.weights.getSparse(feature.getName()),
-            Decoder.weights.getSparse(feature.getName()), acc.getScore());
+            acc.getScore() / Decoder.weights.getOrDefault(hashFeature(feature.getName())),
+            Decoder.weights.getOrDefault(hashFeature(feature.getName())), acc.getScore());
       }
 
       if (feature.isStateful()) {
@@ -174,15 +175,15 @@ public class ComputeNodeResult {
       HyperEdge edge, int i, int j, Sentence sentence) {
 
     // Initialize the set of features with those that were present with the rule in the grammar.
-    FeatureVector featureDelta = new FeatureVector();
+    FeatureVector featureDelta = new FeatureVector(featureFunctions.size());
 
     // === compute feature logPs
     for (FeatureFunction ff : featureFunctions) {
       // A null rule signifies the final transition.
       if (edge.getRule() == null)
-        featureDelta.add(ff.computeFinalFeatures(edge.getTailNodes().get(0), i, j, edge.getSourcePath(), sentence));
+        featureDelta.addInPlace(ff.computeFinalFeatures(edge.getTailNodes().get(0), i, j, edge.getSourcePath(), sentence));
       else {
-        featureDelta.add(ff.computeFeatures(edge.getRule(), edge.getTailNodes(), i, j, edge.getSourcePath(), sentence));
+        featureDelta.addInPlace(ff.computeFeatures(edge.getRule(), edge.getTailNodes(), i, j, edge.getSourcePath(), sentence));
       }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/main/java/org/apache/joshua/decoder/ff/ArityPhrasePenalty.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/ArityPhrasePenalty.java b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/ArityPhrasePenalty.java
index f544f50..996f40d 100644
--- a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/ArityPhrasePenalty.java
+++ b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/ArityPhrasePenalty.java
@@ -46,7 +46,6 @@ public class ArityPhrasePenalty extends StatelessFF {
 
   public ArityPhrasePenalty(final FeatureVector weights, String[] args, JoshuaConfiguration config) {
     super(weights, "ArityPenalty", args, config);
-
     this.owner = OwnerMap.register(parsedArgs.get("owner"));
     this.minArity = Integer.parseInt(parsedArgs.get("min-arity"));
     this.maxArity = Integer.parseInt(parsedArgs.get("max-arity"));
@@ -66,7 +65,7 @@ public class ArityPhrasePenalty extends StatelessFF {
   @Override
   public DPState compute(Rule rule, List<HGNode> tailNodes, int i, int j, SourcePath sourcePath,
       Sentence sentence, Accumulator acc) {
-    acc.add(name, isEligible(rule));
+    acc.add(featureId, isEligible(rule));
     
     return null;
   }

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/main/java/org/apache/joshua/decoder/ff/FeatureFunction.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/FeatureFunction.java b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/FeatureFunction.java
index cb4a4bd..fb8c789 100644
--- a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/FeatureFunction.java
+++ b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/FeatureFunction.java
@@ -18,7 +18,8 @@
  */
 package org.apache.joshua.decoder.ff;
 
-import java.util.ArrayList;
+import static org.apache.joshua.decoder.ff.FeatureMap.hashFeature;
+
 import java.util.HashMap;
 import java.util.List;
 import java.util.regex.Matcher;
@@ -73,27 +74,22 @@ public abstract class FeatureFunction {
    * names, for templates that define multiple features.
    */
   protected String name = null;
-
-  /*
-   * The list of features each function can contribute, along with the dense feature IDs.
-   */
-  protected String[] denseFeatureNames = null;
-  protected int[] denseFeatureIDs = null;
-
+  
   /*
-   * The first dense feature index
+   * The hashed feature id correspondig to name. This can be changed if name is changed as well
+   * but provides a good default id for most cases. 
    */
-  protected int denseFeatureIndex = -1; 
+  protected int featureId;
 
   // The list of arguments passed to the feature, and the hash for the parsed args
-  protected String[] args;
-  protected HashMap<String, String> parsedArgs = null; 
+  protected final String[] args;
+  protected final HashMap<String, String> parsedArgs; 
 
   /*
    * The global weight vector used by the decoder, passed it when the feature is
    * instantiated
    */
-  protected FeatureVector weights;
+  protected final FeatureVector weights;
 
   /* The config */
   protected JoshuaConfiguration config;
@@ -108,30 +104,14 @@ public abstract class FeatureFunction {
   public FeatureFunction(FeatureVector weights, String name, String[] args, JoshuaConfiguration config) {
     this.weights = weights;
     this.name = name;
+    this.featureId = FeatureMap.hashFeature(this.name);
     this.args = args;
     this.config = config;
-
     this.parsedArgs = FeatureFunction.parseArgs(args);
   }
 
-  /**
-   * Any feature function can use this to report dense features names to the master code. The 
-   * parameter tells the feature function the index of the first available dense feature ID; the feature
-   * function will then use IDs (id..id+names.size()-1).
-   * 
-   * @param id the id of the first dense feature id to use
-   * @return a list of dense feature names
-   */
-  public ArrayList<String> reportDenseFeatures(int id) {
-    return new ArrayList<String>();
-  }
-
   public String logString() {
-    try {
-      return String.format("%s (weight %.3f)", name, weights.getSparse(name));
-    } catch (RuntimeException e) {
-      return name;
-    }
+    return String.format("%s (weight %.3f)", name, weights.getOrDefault(hashFeature(name)));
   }
 
   /**
@@ -320,8 +300,7 @@ public abstract class FeatureFunction {
    * (for k-best extraction).
    */
   public interface Accumulator {
-    public void add(String name, float value);
-    public void add(int id, float value);
+    public void add(int featureId, float value);
   }
 
   public class ScoreAccumulator implements Accumulator {
@@ -332,13 +311,8 @@ public abstract class FeatureFunction {
     }
 
     @Override
-    public void add(String name, float value) {
-      score += value * weights.getSparse(name);
-    }
-
-    @Override
-    public void add(int id, float value) {
-      score += value * weights.getDense(id);
+    public void add(int featureId, float value) {
+      score += value * weights.getOrDefault(featureId);
     }
 
     public float getScore() {
@@ -350,17 +324,12 @@ public abstract class FeatureFunction {
     private FeatureVector features;
 
     public FeatureAccumulator() {
-      this.features = new FeatureVector();
-    }
-
-    @Override
-    public void add(String name, float value) {
-      features.increment(name, value);
+      this.features = new FeatureVector(10);
     }
 
     @Override
     public void add(int id, float value) {
-      features.increment(id,  value);
+      features.add(id, value);
     }
 
     public FeatureVector getFeatures() {

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/main/java/org/apache/joshua/decoder/ff/FeatureMap.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/FeatureMap.java b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/FeatureMap.java
new file mode 100644
index 0000000..e95339a
--- /dev/null
+++ b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/FeatureMap.java
@@ -0,0 +1,107 @@
+/*
+ * 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.ff;
+
+import java.util.concurrent.locks.StampedLock;
+
+import org.apache.joshua.decoder.ff.tm.OwnerId;
+import org.apache.joshua.decoder.ff.tm.OwnerMap;
+
+import com.google.common.collect.BiMap;
+import com.google.common.collect.HashBiMap;
+
+/**
+ * FeatureMap maintains a decoder-wide bi-directional mapping between feature names and
+ * corresponding IDs, typed as ints.
+ * This separates feature ids (i.e. simple hashes) from word ids.
+ * The retrieval of the feature name given a feature id is strict, in that it throws 
+ * a {@link RuntimeException} if the feature was not hashed/added to this mapping before.
+ * 
+ * @author fhieber
+ *
+ */
+public class FeatureMap {
+  
+  /** bi-directional mapping between feature ids and feature names */
+  private static BiMap<Integer, String> map = HashBiMap.create();
+  
+  private static final StampedLock lock = new StampedLock();
+  
+  static {
+    map.clear();
+  }
+  
+  /**
+   * Return a feature id for the given featureName. If the id does not exist
+   * yet, it is added to the mapping.
+   * @param featureName the featureName to be hashed
+   * @return
+   */
+  public static synchronized int hashFeature(String featureName) {
+    if (map.inverse().containsKey(featureName)) {
+      return map.inverse().get(featureName);
+    }
+    final int newId = map.size();
+    map.put(newId, featureName);
+    return newId;
+  }
+
+  /**
+   * Returns a feature id corresponding to a feature prepended an owner string if ownerId != OwnerMap.UNKNOWN_OWNER_ID.
+   * This function is used to hash features precomputed on rules (i.e. dense & sparse features stored in the grammar).
+   * @param featureName the featureName to be hashed
+   * @param ownerId the ownerId of the grammar owning this feature.
+   * @return feature id corresponding to the (owner-prefixed) feature name.
+   */
+  public static int hashFeature(final String featureName, final OwnerId ownerId) {
+    if (ownerId.equals(OwnerMap.UNKNOWN_OWNER_ID)) {
+      return hashFeature(featureName);
+    } else {
+      return hashFeature(OwnerMap.getOwner(ownerId) + "_" + featureName);
+    }
+  }
+  
+  /**
+   * Reverse lookup a feature id to retrieve the stored feature name.
+   * Throws a {@link RuntimeException} if mapping is not present.
+   * @param id a feature id that must be contained in the mapping.
+   * @return featureName corresponding to the given feature id.
+   */
+  public static String getFeature(final int id) {
+    long lock_stamp = lock.readLock();
+    try {
+      if (map.containsKey(id)) {
+        return map.get(id);
+      }
+      throw new IllegalArgumentException(
+          String.format("FeatureMap does not contain mapping for %s", id));
+    } finally {
+      lock.unlockRead(lock_stamp);
+    }
+  }
+  
+  public static boolean hasFeature(final String featureName) {
+    return map.inverse().containsKey(featureName);
+  }
+  
+  public static synchronized void clear() {
+    map.clear();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/main/java/org/apache/joshua/decoder/ff/FeatureVector.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/FeatureVector.java b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/FeatureVector.java
index 1b39c78..7be7317 100644
--- a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/FeatureVector.java
+++ b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/FeatureVector.java
@@ -18,368 +18,131 @@
  */
 package org.apache.joshua.decoder.ff;
 
-import java.util.ArrayList;
-import java.util.Collections;
+import static java.util.stream.Collectors.joining;
+
+import java.util.AbstractMap;
+import java.util.Collection;
 import java.util.HashMap;
-import java.util.HashSet;
+import java.util.Iterator;
 import java.util.Map;
 import java.util.Set;
 
 /**
- * An implementation of a sparse feature vector, using for representing both weights and feature
- * values.
- *
- * This class is used to hold both the decoder weights and the feature values accumulated across
- * each edge. When features are read in upon decoder startup, they all start out as sparse features
- * and are stored in the hash table. After the feature functions have been loaded, the decoder
- * queries each of them for their sparse features via {@link registerDenseFeatures}. Those features
- * returned by each decoder are then *removed* from the sparse feature hash and placed in the dense
- * feature array. Therefore, when a feature registers a dense feature, it should take care to
- * query either {@link org.apache.joshua.decoder.ff.FeatureVector#getDense(int)} or
- * {@link org.apache.joshua.decoder.ff.FeatureVector#getSparse(String)} when asking for the feature
- * values later on.
- *
- * @author Matt Post post@cs.jhu.edu
+ * Simplified version of a sparse feature vector.
+ * @author fhieber
  */
+public class FeatureVector extends AbstractMap<Integer, Float> {
 
-public class FeatureVector {
-  /*
-   * A list of the dense feature names. Increased via calls to registerDenseFeatures()
-   */
-  public static ArrayList<String> DENSE_FEATURE_NAMES = new ArrayList<String>();
-
-  /*
-   * The values of each of the dense features, defaulting to 0.
-   */
-  private ArrayList<Float> denseFeatures = null;
-
-  /*
-   * Value of sparse features.
-   */
-  private HashMap<String, Float> sparseFeatures;
+  private static final float DEFAULT_VALUE = 0.0f;
+  private final Map<Integer, Float> internalMap;
 
-  public FeatureVector() {
-    sparseFeatures = new HashMap<String, Float>();
-    denseFeatures = new ArrayList<Float>(DENSE_FEATURE_NAMES.size());
-    for (int i = 0; i < denseFeatures.size(); i++)
-      denseFeatures.set(i, 0.0f);
+  public FeatureVector(int size) {
+    internalMap = new HashMap<Integer, Float>(size);
   }
 
   /**
-   * This version of the constructor takes an uninitialized feature with potentially intermingled
-   * labeled and unlabeled feature values, of the format:
-   *
-   * [feature1=]value [feature2=]value
-   *
-   * It produces a Feature Vector where all unlabeled features have been labeled by appending the
-   * unlabeled feature index (starting at 0) to the defaultPrefix value.
-   *
-   * **IMPORTANT** The feature values are inverted, for historical reasons, which leads to a lot
-   * of confusion. They have to be inverted here and when the score is actually computed. They
-   * are inverted here (which is used to build the feature vector representation of a rule's dense
-   * features) and in {@link org.apache.joshua.decoder.ff.tm.Rule#estimateRuleCost(java.util.List)}
-   * , where the rule's precomputable (weighted) score is cached.
-   *
-   * @param featureString, the string of labeled and unlabeled features (probably straight from the
-   *          grammar text file)
-   * @param prefix, the prefix to use for unlabeled features (probably "tm_OWNER_")
+   * Copy constructor
    */
-  public FeatureVector(String featureString, String prefix) {
-
-//    System.err.println(String.format("FEATURES_OF(%s, %s)", featureString, prefix));
-
-    /*
-     * Read through the features on this rule, adding them to the feature vector. Unlabeled features
-     * are converted to a canonical form.
-     *
-     * Note that it's bad form to mix unlabeled features and the named feature index they are mapped
-     * to, but we are being liberal in what we accept.
-     *
-     * IMPORTANT: Note that, for historical reasons, the sign is reversed on all *dense* scores.
-     * This is the source of *no end* of confusion and should be done away with.
-     */
-    this();
-
-    int denseFeatureIndex = 0;
-
-    if (!featureString.trim().equals("")) {
-      for (String token : featureString.split("\\s+")) {
-        if (token.indexOf('=') == -1) {
-          /*
-           * If we encounter an unlabeled feature, it is the next dense feature
-           */
-          while (denseFeatures.size() <= denseFeatureIndex)
-            denseFeatures.add(0.0f);
-          denseFeatures.set(denseFeatureIndex, -Float.parseFloat(token));
-          denseFeatureIndex++;
-        } else {
-          /*
-           * Labeled features are of two types: if they start with the prefix, they are actually
-           * dense feature in disguise; otherwise, they are proper sparse features.
-           */
-          int splitPoint = token.indexOf('=');
-          if (token.startsWith(prefix)) {
-//            System.err.println(String.format("  PREFIX=%s '%s'.substring(%d,%d) = %s", prefix, token, prefix.length(), splitPoint,
-//                token.substring(prefix.length(), splitPoint)));
-            int index = Integer.parseInt(token.substring(prefix.length(), splitPoint));
-            while (denseFeatures.size() <= index)
-              denseFeatures.add(0.0f);
-            denseFeatures.set(index, 1.0f * Float.parseFloat(token.substring(splitPoint + 1)));
-          } else {
-            sparseFeatures.put(token.substring(0, splitPoint),
-                Float.parseFloat(token.substring(splitPoint + 1)));
-          }
-        }
-      }
-    }
+  public FeatureVector(Map<Integer, Float> m) {
+    internalMap = new HashMap<Integer, Float>(m);
   }
-
+  
   /**
-   * Register one or more dense features with the global weight vector. This assumes them global
-   * IDs, and then returns the index of the first feature (from which the calling feature function
-   * can infer them all). This *must* be called by every feature function wishing to register
-   * dense features!
-   *
-   * @param featureFunctions {@link java.util.ArrayList} of {@link org.apache.joshua.decoder.ff.FeatureFunction}'s
+   * Returns a Map of feature names to values.
+   * @return map from strings to floats
    */
-  public void registerDenseFeatures(ArrayList<FeatureFunction> featureFunctions) {
-    for (FeatureFunction feature: featureFunctions) {
-      ArrayList<String> names = feature.reportDenseFeatures(denseFeatures.size());
-      for (String name: names) {
-        DENSE_FEATURE_NAMES.add(name);
-        denseFeatures.add(getSparse(name));
-        sparseFeatures.remove(name);
-      }
+  public Map<String, Float> toStringMap() {
+    final Map<String, Float> result = new HashMap<>(this.size());
+    for (Map.Entry<Integer, Float> entry : this.entrySet()) {
+      result.put(FeatureMap.getFeature(entry.getKey()), entry.getValue());
     }
-  }
-
-  public ArrayList<Float> getDenseFeatures() {
-    return denseFeatures;
-  }
-
-  public HashMap<String,Float> getSparseFeatures() {
-    return sparseFeatures;
-  }
-
-  public Set<String> keySet() {
-    return sparseFeatures.keySet();
-  }
-
-  public int size() {
-    return sparseFeatures.size() + denseFeatures.size();
-  }
-
-  public FeatureVector clone() {
-    FeatureVector newOne = new FeatureVector();
-    for (String key : this.sparseFeatures.keySet())
-      newOne.set(key, this.sparseFeatures.get(key));
-    for (int i = 0; i < denseFeatures.size(); i++)
-      newOne.set(i, getDense(i));
-    return newOne;
+    return result;
   }
 
   /**
-   * Subtracts the weights in the other feature vector from this one. Note that this is not set
-   * subtraction; keys found in the other FeatureVector but not in this one will be initialized with
-   * a value of 0.0f before subtraction.
-   *
-   * @param other another {@link org.apache.joshua.decoder.ff.FeatureVector} from which to subtract its score
+   * Like Map.getOrDefault but with default value 0.0f.
    */
-  public void subtract(FeatureVector other) {
-    for (int i = 0; i < denseFeatures.size(); i++)
-      denseFeatures.set(i, getDense(i) - other.getDense(i));
-
-    for (String key : other.keySet()) {
-      float oldValue = (sparseFeatures.containsKey(key)) ? sparseFeatures.get(key) : 0.0f;
-      sparseFeatures.put(key, oldValue - other.getSparse(key));
-    }
+  public Float getOrDefault(Integer key) {
+    return internalMap.getOrDefault(key, DEFAULT_VALUE);
+  }
+  
+  public Float get(Integer key) {
+    return getOrDefault(key);
+  }
+  
+  public boolean containsKey(Integer key) {
+    return internalMap.containsKey(key);
+  };
+  
+  @Override
+  public Float put(Integer featureId, Float value) {
+    return internalMap.put(featureId, value);
   }
 
   /**
-   * Adds the weights in the other feature vector to this one. This is set union, with values shared
-   * between the two being summed.
-   *
-   * @param other another {@link org.apache.joshua.decoder.ff.FeatureVector} from which to add its score
+   * Adds values of other to this. Returns a reference to this.
+   * 
+   * @param other
+   * @return this
    */
-  public void add(FeatureVector other) {
-    while (denseFeatures.size() < other.denseFeatures.size())
-      denseFeatures.add(0.0f);
-
-    for (int i = 0; i < other.denseFeatures.size(); i++)
-      increment(i, other.getDense(i));
-
-    for (String key : other.keySet()) {
-      if (!sparseFeatures.containsKey(key))
-        sparseFeatures.put(key, other.getSparse(key));
-      else
-        sparseFeatures.put(key, sparseFeatures.get(key) + other.getSparse(key));
+  public FeatureVector addInPlace(FeatureVector other) {
+    for (Entry<Integer, Float> e : other.entrySet()) {
+      add(e.getKey(), e.getValue());
     }
+    return this;
   }
 
-  /**
-   * Return the weight of a feature by name, after checking to determine if it is sparse or dense.
-   *
-   * @param feature String name of some feature
-   * @return the feature's weight
-   */
-  public float getWeight(String feature) {
-    for (int i = 0; i < DENSE_FEATURE_NAMES.size(); i++) {
-      if (DENSE_FEATURE_NAMES.get(i).equals(feature)) {
-        return getDense(i);
-      }
-    }
-    return getSparse(feature);
+  public void add(Integer key, float value) {
+    this.put(key, getOrDefault(key) + value);
   }
 
   /**
-   * Return the weight of a sparse feature, indexed by its name.
-   *
-   * @param feature String name of some feature
-   * @return the sparse feature's weight, or 0 if not found.
+   * Computes dot product of this and other FeatureVector.
    */
-  public float getSparse(String feature) {
-    if (sparseFeatures.containsKey(feature))
-      return sparseFeatures.get(feature);
-    return 0.0f;
-  }
-
-  public boolean hasValue(String name) {
-    return sparseFeatures.containsKey(name);
-  }
-
-  /**
-   * Return the weight of a dense feature, indexed by its feature index, or 0.0f, if the feature
-   * is not found. In other words, this is a safe way to query the dense feature vector.
-   *
-   * @param id int representing of some dense feature
-   * @return the dense feature's value, or 0 if not found.
-   */
-  public float getDense(int id) {
-    if (id < denseFeatures.size())
-      return denseFeatures.get(id);
-    return 0.0f;
-  }
-
-  public void increment(String feature, float value) {
-    sparseFeatures.put(feature, getSparse(feature) + value);
-  }
-
-  public void increment(int id, float value) {
-    while (id >= denseFeatures.size())
-      denseFeatures.add(0.0f);
-    denseFeatures.set(id, getDense(id) + value);
-  }
-
-  /**
-   * Set the value of a feature. We need to first determine whether the feature is a dense or
-   * sparse one, then set accordingly.
-   *
-   * @param feature String name of some feature
-   * @param value float value to set to the featue with the associated name
-   */
-  public void set(String feature, float value) {
-    for (int i = 0; i < DENSE_FEATURE_NAMES.size(); i++) {
-      if (DENSE_FEATURE_NAMES.get(i).equals(feature)) {
-        denseFeatures.set(i, value);
-        return;
+  public float innerProduct(FeatureVector other) {
+    float product = 0.0f;
+    if (other.size() >= this.size()) {
+      for (Entry<Integer, Float> e : this.entrySet()) {
+        product += e.getValue() * other.getOrDefault(e.getKey());
+      }
+    } else {
+      for (Entry<Integer, Float> e : other.entrySet()) {
+        product += e.getValue() * this.getOrDefault(e.getKey());
       }
     }
-    // No dense feature was found; assume it's sparse
-    sparseFeatures.put(feature, value);
+    return product;
   }
 
-  public void set(int id, float value) {
-    while (id >= denseFeatures.size())
-      denseFeatures.add(0.0f);
-    denseFeatures.set(id, value);
+  @Override
+  public Set<Map.Entry<Integer, Float>> entrySet() {
+    return internalMap.entrySet();
   }
 
-  public Map<String, Float> getMap() {
-    Map<String, Float> allFeatures = new HashMap<>(sparseFeatures.size() + denseFeatures.size());
-    allFeatures.putAll(sparseFeatures);
-    for (int i = 0; i < DENSE_FEATURE_NAMES.size(); i++) {
-      allFeatures.put(DENSE_FEATURE_NAMES.get(i), getDense(i));
-    }
-    return allFeatures;
+  @Override
+  public Collection<Float> values() {
+    return internalMap.values();
   }
 
   /**
-   * Computes the inner product between this feature vector and another one.
-   *
-   * @param other a {@link org.apache.joshua.decoder.ff.FeatureVector} with which to compute the inner product
-   * @return float value representing the computation
+   * Prunes elements from the vector whose absolute values are smaller than
+   * threshold.
+   * 
+   * @return the pruned feature vector
    */
-  public float innerProduct(FeatureVector other) {
-    float cost = 0.0f;
-    for (int i = 0; i < DENSE_FEATURE_NAMES.size(); i++)
-      cost += getDense(i) * other.getDense(i);
-
-    for (String key : sparseFeatures.keySet())
-      cost += sparseFeatures.get(key) * other.getSparse(key);
-
-    return cost;
-  }
-
-  public void times(float value) {
-    for (String key : sparseFeatures.keySet())
-      sparseFeatures.put(key, sparseFeatures.get(key) * value);
-  }
-
-  /***
-   * Moses distinguishes sparse features as those containing an underscore, so we have to fake it
-   * to be compatible with their tuners.
-   *
-   * @return trimmed Moses output string
-   */
-  public String mosesString() {
-    StringBuilder outputString = new StringBuilder();
-
-    HashSet<String> printed_keys = new HashSet<String>();
-
-    // First print all the dense feature names in order
-    for (int i = 0; i < DENSE_FEATURE_NAMES.size(); i++) {
-      outputString.append(String.format("%s=%.3f ", DENSE_FEATURE_NAMES.get(i).replaceAll("_", "-"), getDense(i)));
-      printed_keys.add(DENSE_FEATURE_NAMES.get(i));
-    }
-
-    // Now print the sparse features
-    ArrayList<String> keys = new ArrayList<String>(sparseFeatures.keySet());
-    Collections.sort(keys);
-    for (String key: keys) {
-      if (! printed_keys.contains(key)) {
-        float value = sparseFeatures.get(key);
-        if (key.equals("OOVPenalty"))
-          // force moses to see it as sparse
-          key = "OOV_Penalty";
-        outputString.append(String.format("%s=%.3f ", key, value));
+  public FeatureVector prune(final float threshold) {
+    for (Iterator<Map.Entry<Integer, Float>> it = internalMap.entrySet().iterator(); it.hasNext();) {
+      if (Math.abs(it.next().getValue()) < threshold) {
+        it.remove();
       }
     }
-    return outputString.toString().trim();
+    return this;
   }
-
-  /***
-   * Outputs a list of feature names. All dense features are printed. Feature names are printed
-   * in the order they were read in.
-   */
-  @Override
-  public String toString() {
-    StringBuilder outputString = new StringBuilder();
-
-    HashSet<String> printed_keys = new HashSet<String>();
-
-    // First print all the dense feature names in order
-    for (int i = 0; i < DENSE_FEATURE_NAMES.size(); i++) {
-      outputString.append(String.format("%s=%.3f ", DENSE_FEATURE_NAMES.get(i), getDense(i)));
-      printed_keys.add(DENSE_FEATURE_NAMES.get(i));
-    }
-
-    // Now print the rest of the features
-    ArrayList<String> keys = new ArrayList<String>(sparseFeatures.keySet());
-    Collections.sort(keys);
-    for (String key: keys)
-      if (! printed_keys.contains(key))
-        outputString.append(String.format("%s=%.3f ", key, sparseFeatures.get(key)));
-
-    return outputString.toString().trim();
+  
+  public String textFormat() {
+    return internalMap.entrySet()
+      .stream()
+      .map(e -> String.format("%s=%.6f", FeatureMap.getFeature(e.getKey()), e.getValue())  )
+      .collect(joining(" "));
   }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/main/java/org/apache/joshua/decoder/ff/LabelCombinationFF.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/LabelCombinationFF.java b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/LabelCombinationFF.java
index bfebaa5..edeae6c 100644
--- a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/LabelCombinationFF.java
+++ b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/LabelCombinationFF.java
@@ -41,14 +41,14 @@ public class LabelCombinationFF extends StatelessFF {
     return name.toLowerCase();
   }
 
-  private final String computeRuleLabelCombinationDescriptor(Rule rule) {
+  private final int computeRuleLabelCombinationDescriptor(Rule rule) {
     StringBuilder result = new StringBuilder(getLowerCasedFeatureName() + "_");
     result.append(RulePropertiesQuerying.getLHSAsString(rule));
     // System.out.println("Rule: " + rule);
     for (String foreignNonterminalString : RulePropertiesQuerying.getRuleSourceNonterminalStrings(rule)) {
       result.append("_").append(foreignNonterminalString);
     }
-    return result.toString();
+    return FeatureMap.hashFeature(result.toString());
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/main/java/org/apache/joshua/decoder/ff/LabelSubstitutionFF.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/LabelSubstitutionFF.java b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/LabelSubstitutionFF.java
index 8735be6..766ea0b 100644
--- a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/LabelSubstitutionFF.java
+++ b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/LabelSubstitutionFF.java
@@ -18,6 +18,8 @@
  */
 package org.apache.joshua.decoder.ff;
 
+import static org.apache.joshua.decoder.ff.FeatureMap.hashFeature;
+
 /***
  * @author Gideon Wenniger
  */
@@ -121,10 +123,10 @@ public class LabelSubstitutionFF extends StatelessFF {
       for (int nonterinalIndex = 0; nonterinalIndex < ruleSourceNonterminals.size(); nonterinalIndex++) {
         String ruleNonterminal = ruleSourceNonterminals.get(nonterinalIndex);
         String substitutionNonterminal = substitutionNonterminals.get(nonterinalIndex);
-        acc.add(computeLabelMatchingFeature(ruleNonterminal, substitutionNonterminal), 1);
-        acc.add(computeLabelSubstitutionFeature(ruleNonterminal, substitutionNonterminal), 1);
+        acc.add(hashFeature(computeLabelMatchingFeature(ruleNonterminal, substitutionNonterminal)), 1);
+        acc.add(hashFeature(computeLabelSubstitutionFeature(ruleNonterminal, substitutionNonterminal)), 1);
       }
-      acc.add(getGapLabelsForRuleSubstitutionSuffix(rule, tailNodes), 1);
+      acc.add(hashFeature(getGapLabelsForRuleSubstitutionSuffix(rule, tailNodes)), 1);
     }
     return null;
   }

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/main/java/org/apache/joshua/decoder/ff/LexicalFeatures.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/LexicalFeatures.java b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/LexicalFeatures.java
index 75158d0..4eacd26 100644
--- a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/LexicalFeatures.java
+++ b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/LexicalFeatures.java
@@ -19,6 +19,7 @@
 package org.apache.joshua.decoder.ff;
 
 import static com.google.common.cache.CacheBuilder.newBuilder;
+import static org.apache.joshua.decoder.ff.FeatureMap.hashFeature;
 
 import java.util.ArrayList;
 import java.util.List;
@@ -55,7 +56,7 @@ public class LexicalFeatures extends StatelessFF {
   // Strings separating words
   private static final String SEPARATOR = "~";
   
-  private final Cache<Rule, List<String>> featureCache;
+  private final Cache<Rule, List<Integer>> featureCache;
   
   public LexicalFeatures(FeatureVector weights, String[] args, JoshuaConfiguration config) {
     super(weights, NAME, args, config);
@@ -83,13 +84,13 @@ public class LexicalFeatures extends StatelessFF {
       return null;
     }
 
-    List<String> featureNames = featureCache.getIfPresent(rule);
-    if (featureNames == null) {
-      featureNames = getFeatures(rule);
-      featureCache.put(rule, featureNames);
+    List<Integer> featureIds = featureCache.getIfPresent(rule);
+    if (featureIds == null) {
+      featureIds = getFeatures(rule);
+      featureCache.put(rule, featureIds);
     }
-    for (String feature : featureNames) {
-      acc.add(feature, VALUE);
+    for (int featureId : featureIds) {
+      acc.add(featureId, VALUE);
     }
     
     return null;
@@ -100,15 +101,15 @@ public class LexicalFeatures extends StatelessFF {
    * @param rule
    * @return String representing the feature name.s
    */
-  private List<String> getFeatures(final Rule rule) {
-    final List<String> result = new ArrayList<>();
+  private List<Integer> getFeatures(final Rule rule) {
+    final List<Integer> result = new ArrayList<>();
     
     byte[] alignments = rule.getAlignment();
     if (alignments == null) {
       return result;
     }
-    int[] sourceWords = rule.getFrench();
-    int[] targetWords = rule.getEnglish();
+    int[] sourceWords = rule.getSource();
+    int[] targetWords = rule.getTarget();
     
     // sourceAligned & targetAligned indicate whether an index is covered by alignments
     boolean[] sourceAligned = new boolean[sourceWords.length];
@@ -121,11 +122,11 @@ public class LexicalFeatures extends StatelessFF {
       sourceAligned[sourceIndex] = true;
       targetAligned[targetIndex] = true;
       if (useAlignments) {
-        result.add(
+        result.add(hashFeature(
             "T:" + 
             Vocabulary.word(sourceWords[sourceIndex]) + 
             SEPARATOR + 
-            Vocabulary.word(targetWords[targetIndex]));
+            Vocabulary.word(targetWords[targetIndex])));
       }
     }
     
@@ -133,7 +134,7 @@ public class LexicalFeatures extends StatelessFF {
     if (useDeletions) {
       for (int i = 0; i < sourceAligned.length; i++) {
         if (!sourceAligned[i] && ! FormatUtils.isNonterminal(sourceWords[i])) {
-          result.add("D:" + Vocabulary.word(sourceWords[i]));
+          result.add(hashFeature("D:" + Vocabulary.word(sourceWords[i])));
         }
       }
     }
@@ -142,7 +143,7 @@ public class LexicalFeatures extends StatelessFF {
     if (useInsertions) {
       for (int i = 0; i < targetAligned.length; i++) {
         if (useInsertions && !targetAligned[i] && ! FormatUtils.isNonterminal(targetWords[i])) {
-          result.add("I:" + Vocabulary.word(targetWords[i]));
+          result.add(hashFeature("I:" + Vocabulary.word(targetWords[i])));
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/main/java/org/apache/joshua/decoder/ff/OOVPenalty.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/OOVPenalty.java b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/OOVPenalty.java
index 92ee740..6eb1293 100644
--- a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/OOVPenalty.java
+++ b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/OOVPenalty.java
@@ -18,20 +18,19 @@
  */
 package org.apache.joshua.decoder.ff;
 
-import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 
+import org.apache.joshua.corpus.Vocabulary;
 import org.apache.joshua.decoder.JoshuaConfiguration;
 import org.apache.joshua.decoder.JoshuaConfiguration.OOVItem;
+import org.apache.joshua.decoder.chart_parser.SourcePath;
 import org.apache.joshua.decoder.ff.state_maintenance.DPState;
 import org.apache.joshua.decoder.ff.tm.OwnerId;
 import org.apache.joshua.decoder.ff.tm.OwnerMap;
 import org.apache.joshua.decoder.ff.tm.Rule;
 import org.apache.joshua.decoder.hypergraph.HGNode;
 import org.apache.joshua.decoder.segment_file.Sentence;
-import org.apache.joshua.corpus.Vocabulary;
-import org.apache.joshua.decoder.chart_parser.SourcePath;
 
 /**
  * This feature is fired when an out-of-vocabulary word (with respect to the translation model) is
@@ -52,7 +51,6 @@ public class OOVPenalty extends StatelessFF {
 
   public OOVPenalty(FeatureVector weights, String[] args, JoshuaConfiguration config) {
     super(weights, "OOVPenalty", args, config);
-
     ownerID = OwnerMap.register("oov");
     oovWeights = new HashMap<Integer,Float>();
     
@@ -63,15 +61,6 @@ public class OOVPenalty extends StatelessFF {
     }
   }
   
-  @Override
-  public ArrayList<String> reportDenseFeatures(int index) {
-    denseFeatureIndex = index;
-    
-    ArrayList<String> names = new ArrayList<>(1);
-    names.add(name);
-    return names;
-  }
-
   /**
    * OOV rules cover exactly one word, and such rules belong to a grammar whose owner is "oov". Each
    * OOV fires the OOVPenalty feature with a value of 1, so the cost is simply the weight, which was
@@ -82,7 +71,7 @@ public class OOVPenalty extends StatelessFF {
       Sentence sentence, Accumulator acc) {
     
     if (rule != null && this.ownerID.equals(rule.getOwner())) {
-      acc.add(denseFeatureIndex, getValue(rule.getLHS()));
+      acc.add(featureId, getValue(rule.getLHS()));
     }
 
     return null;
@@ -97,8 +86,9 @@ public class OOVPenalty extends StatelessFF {
    */
   @Override
   public float estimateCost(Rule rule, Sentence sentence) {
-    if (rule != null && this.ownerID.equals(rule.getOwner()))
-      return weights.getDense(denseFeatureIndex) * getValue(rule.getLHS());
+    if (rule != null && this.ownerID.equals(rule.getOwner())) {
+      return weights.getOrDefault(featureId) * getValue(rule.getLHS());
+    }
     return 0.0f;
   }
   

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/main/java/org/apache/joshua/decoder/ff/PhraseModel.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/PhraseModel.java b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/PhraseModel.java
index 7ae3dbc..e1ba328 100644
--- a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/PhraseModel.java
+++ b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/PhraseModel.java
@@ -18,8 +18,8 @@
  */
 package org.apache.joshua.decoder.ff;
 
-import java.util.ArrayList;
 import java.util.List;
+import java.util.Map.Entry;
 
 import org.apache.joshua.decoder.JoshuaConfiguration;
 import org.apache.joshua.decoder.chart_parser.SourcePath;
@@ -32,52 +32,23 @@ import org.apache.joshua.decoder.hypergraph.HGNode;
 import org.apache.joshua.decoder.segment_file.Sentence;
 
 /**
- * This feature handles the list of features that are found with grammar rules in the grammar file.
- * dense features that may be associated with the rules in a grammar file. The feature names of
- * these dense rules are a function of the phrase model owner. When the feature is loaded, it
- * queries the weights for the set of features that are active for this grammar, storing them in an
- * array.
+ * This feature handles the list of features that are stored with grammar rules in the grammar file.
+ * These are by convention bound to the PhraseModel feature function and will be prepended by the owner of this
+ * PhraseModel instance, i.e. 'p_e_given_f' will be hashed as '<owner>_p_e_given_f'.
+ * If multiple grammars exist and feature sharing is needed, one must implement a separate feature function for this.
  * 
  * @author Matt Post post@cs.jhu.edu
  * @author Zhifei Li zhifei.work@gmail.com
  */
 
 public class PhraseModel extends StatelessFF {
-
-  /* The owner of the grammar. */
-  private final OwnerId ownerID;
-  private final String owner;
-
-  private float[] phrase_weights = null;
+  
+  private final OwnerId owner;
 
   public PhraseModel(FeatureVector weights, String[] args, JoshuaConfiguration config, Grammar g) {
-    super(weights, "tm_", args, config);
-
-    // Store the owner and name
-    this.owner = parsedArgs.get("owner");
-    this.ownerID = OwnerMap.register(owner);
-    this.name = String.format("tm_%s", this.owner);
-
-    /*
-     * Determine the number of features by querying the example grammar that was passed in.
-     */
-    phrase_weights = new float[g.getNumDenseFeatures()];
-    for (int i = 0; i < phrase_weights.length; i++)
-      phrase_weights[i] = weights.getSparse(String.format("tm_%s_%d", owner, i));
-    
-  }
-
-  /**
-   * Just register a single weight, tm_OWNER, and use that to set its precomputed cost
-   */
-  @Override
-  public ArrayList<String> reportDenseFeatures(int index) {
-    denseFeatureIndex = index;
-
-    ArrayList<String> names = new ArrayList<String>();
-    for (int i = 0; i < phrase_weights.length; i++)
-      names.add(String.format("tm_%s_%d", owner, i));
-    return names;
+    // name of this feature is the owner of the grammar
+    super(weights, OwnerMap.getOwner(g.getOwner()), args, config);
+    this.owner = g.getOwner();
   }
 
   /**
@@ -86,45 +57,27 @@ public class PhraseModel extends StatelessFF {
    */
   @Override
   public float estimateCost(final Rule rule, Sentence sentence) {
-
-    if (rule != null && rule.getOwner().equals(ownerID)) {
-      if (rule.getPrecomputableCost() <= Float.NEGATIVE_INFINITY)
-        rule.setPrecomputableCost(phrase_weights, weights);
-
-      return rule.getPrecomputableCost();
+    // check if the rule belongs to this PhraseModel
+    if (rule != null && rule.getOwner().equals(owner)) {
+      return rule.getFeatureVector().innerProduct(weights);
     }
-
     return 0.0f;
   }
 
   /**
-   * Just chain to computeFeatures(rule), since this feature doesn't use the sourcePath or sentID. *
+   * Accumulates the cost of applying this rule if it belongs to the owner.
    */
   @Override
   public DPState compute(Rule rule, List<HGNode> tailNodes, int i, int j, SourcePath sourcePath,
       Sentence sentence, Accumulator acc) {
 
-    if (rule != null && rule.getOwner().equals(ownerID)) {
-      /*
-       * Here, we peak at the Accumulator object. If it's asking for scores, then we don't bother to
-       * add each feature, but rather compute the inner product and add *that*. This is totally
-       * cheating; the Accumulator is supposed to be a generic object. But without this cheat
-       */
-      if (rule.getPrecomputableCost() <= Float.NEGATIVE_INFINITY) {
-        // float score = rule.getFeatureVector().innerProduct(weights);
-        rule.setPrecomputableCost(phrase_weights, weights);
-      }
-      
-//      System.err.println(String.format("RULE = %s / %f", rule.getEnglishWords(), rule.getPrecomputableCost()));
-      for (int k = 0; k < phrase_weights.length; k++) {
-//        System.err.println(String.format("k = %d, denseFeatureIndex = %d, owner = %s, ownerID = %d", k, denseFeatureIndex, owner, ownerID));
-        acc.add(k + denseFeatureIndex, rule.getDenseFeature(k));
+    if (rule != null && rule.getOwner().equals(owner)) {
+      for (Entry<Integer, Float> entry : rule.getFeatureVector().entrySet()) {
+        final int featureId = entry.getKey();
+        final float featureValue = entry.getValue();
+        acc.add(featureId, featureValue);
       }
-      
-      for (String key: rule.getFeatureVector().keySet())
-        acc.add(key, rule.getFeatureVector().getSparse(key));
     }
-
     return null;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/main/java/org/apache/joshua/decoder/ff/PhrasePenalty.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/PhrasePenalty.java b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/PhrasePenalty.java
index 9eecd0c..acda1d2 100644
--- a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/PhrasePenalty.java
+++ b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/PhrasePenalty.java
@@ -18,7 +18,6 @@
  */
 package org.apache.joshua.decoder.ff;
 
-import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.joshua.decoder.JoshuaConfiguration;
@@ -59,19 +58,12 @@ public class PhrasePenalty extends StatelessFF {
       Sentence sentence, Accumulator acc) {
 
     if (rule != null && rule != Hypothesis.BEGIN_RULE && rule != Hypothesis.END_RULE 
-        && (rule.getOwner().equals(owner)))
-      acc.add(denseFeatureIndex, value);
+        && (rule.getOwner().equals(owner))) {
+      acc.add(featureId, value);
+    }
 
     return null;
   }
-    
-  @Override
-  public ArrayList<String> reportDenseFeatures(int index) {
-    denseFeatureIndex = index;
-    ArrayList<String> names = new ArrayList<String>();
-    names.add(name);
-    return names;
-  }
   
   /**
    * Returns the *weighted* estimate.
@@ -80,8 +72,9 @@ public class PhrasePenalty extends StatelessFF {
   @Override
   public float estimateCost(Rule rule, Sentence sentence) {
     if (rule != null && rule != Hypothesis.BEGIN_RULE && rule != Hypothesis.END_RULE 
-        && (rule.getOwner().equals(owner)))
-      return weights.getDense(denseFeatureIndex) * value;
+        && (rule.getOwner().equals(owner))) {
+      return weights.getOrDefault(featureId) * value;
+    }
     return 0.0f;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/main/java/org/apache/joshua/decoder/ff/RuleCountBin.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/RuleCountBin.java b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/RuleCountBin.java
deleted file mode 100644
index 3ffbf65..0000000
--- a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/RuleCountBin.java
+++ /dev/null
@@ -1,77 +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.ff;
-
-import java.util.List;
-
-import org.apache.joshua.decoder.JoshuaConfiguration;
-import org.apache.joshua.decoder.chart_parser.SourcePath;
-import org.apache.joshua.decoder.ff.state_maintenance.DPState;
-import org.apache.joshua.decoder.ff.tm.OwnerId;
-import org.apache.joshua.decoder.ff.tm.OwnerMap;
-import org.apache.joshua.decoder.ff.tm.Rule;
-import org.apache.joshua.decoder.hypergraph.HGNode;
-import org.apache.joshua.decoder.segment_file.Sentence;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/*
- * This feature computes a bin for the rule and activates a feature for it. It requires access to
- * the index of the RarityPenalty field, from which the rule count can be computed.
- */
-public class RuleCountBin extends StatelessFF {
-
-  private static final Logger LOG = LoggerFactory.getLogger(RuleCountBin.class);
-  private int field = -1;
-  private final OwnerId owner;
-
-  public RuleCountBin(FeatureVector weights, String[] args, JoshuaConfiguration config) {
-    super(weights, "RuleCountBin", args, config);
-    owner = OwnerMap.register("pt");
-
-    field = Integer.parseInt(parsedArgs.get("field"));
-  }
-
-  @Override
-  public DPState compute(Rule rule, List<HGNode> tailNodes, int i, int j, SourcePath sourcePath,
-      Sentence sentence, Accumulator acc) {
-
-    if (rule.getOwner().equals(owner))
-      return null;
-    
-    float rarityPenalty = -rule.getFeatureVector().getSparse(String.format("tm_pt_%d", field));
-    int count = (int) (1.0 - Math.log(rarityPenalty));
-
-    String feature = "RuleCountBin_inf";
-
-    int[] bins = { 1, 2, 4, 8, 16, 32, 64, 128, 1000, 10000 };
-    for (int k : bins) {
-      if (count <= k) {
-        feature = String.format("RuleCountBin_%d", k);
-        break;
-      }
-    }
-
-    LOG.debug("RuleCountBin({}) = {} ==> {}", rarityPenalty, count, feature);
-    
-    acc.add(feature, 1.0f);
-
-    return null;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/main/java/org/apache/joshua/decoder/ff/RuleFF.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/RuleFF.java b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/RuleFF.java
index 308d38a..df2b180 100644
--- a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/RuleFF.java
+++ b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/RuleFF.java
@@ -57,7 +57,7 @@ public class RuleFF extends StatelessFF {
   private static final String SEPARATOR = "~";
   private static final String SIDES_SEPARATOR = "->";
   
-  private final Cache<Rule, String> featureCache;
+  private final Cache<Rule, Integer> featureCache;
   
   public RuleFF(FeatureVector weights, String[] args, JoshuaConfiguration config) {
     super(weights, NAME, args, config);
@@ -96,12 +96,12 @@ public class RuleFF extends StatelessFF {
       return null;
     }
 
-    String featureName = featureCache.getIfPresent(rule);
-    if (featureName == null) {
-      featureName = getRuleString(rule);
-      featureCache.put(rule, featureName);
+    Integer featureId = featureCache.getIfPresent(rule);
+    if (featureId == null) {
+      featureId = hashRuleFeature(rule);
+      featureCache.put(rule, featureId);
     }
-    acc.add(featureName, VALUE);
+    acc.add(featureId, VALUE);
     
     return null;
   }
@@ -111,16 +111,16 @@ public class RuleFF extends StatelessFF {
    * @param rule
    * @return String representing the feature name.s
    */
-  private String getRuleString(final Rule rule) {
+  private int hashRuleFeature(final Rule rule) {
     final StringBuilder sb = new StringBuilder(Vocabulary.word(rule.getLHS()))
       .append(SIDES_SEPARATOR);
     if (sides == Sides.SOURCE || sides == Sides.BOTH) {
-      sb.append(Vocabulary.getWords(rule.getFrench(), SEPARATOR));
+      sb.append(Vocabulary.getWords(rule.getSource(), SEPARATOR));
     }
     sb.append(SIDES_SEPARATOR);
     if (sides == Sides.TARGET || sides == Sides.BOTH) {
-      sb.append(Vocabulary.getWords(rule.getEnglish(), SEPARATOR));
+      sb.append(Vocabulary.getWords(rule.getTarget(), SEPARATOR));
     }
-    return sb.toString();
+    return FeatureMap.hashFeature(sb.toString());
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/main/java/org/apache/joshua/decoder/ff/RuleLength.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/RuleLength.java b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/RuleLength.java
index 02c520b..f6fcefe 100644
--- a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/RuleLength.java
+++ b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/RuleLength.java
@@ -18,6 +18,8 @@
  */
 package org.apache.joshua.decoder.ff;
 
+import static org.apache.joshua.decoder.ff.FeatureMap.hashFeature;
+
 import java.util.List;
 
 import org.apache.joshua.decoder.JoshuaConfiguration;
@@ -42,11 +44,11 @@ public abstract class RuleLength extends StatelessFF {
   @Override
   public DPState compute(Rule rule, List<HGNode> tailNodes, int i, int j, SourcePath sourcePath,
       Sentence sentence, Accumulator acc) {
-    int sourceLength = rule.getFrench().length;
-    int targetLength = rule.getEnglish().length;
-    acc.add(name + "_source" + sourceLength, VALUE);
-    acc.add(name + "_target" + sourceLength, VALUE);
-    acc.add(name + "_sourceTarget" + sourceLength + "-" + targetLength, VALUE);
+    int sourceLength = rule.getSource().length;
+    int targetLength = rule.getTarget().length;
+    acc.add(hashFeature(name + "_source" + sourceLength), VALUE);
+    acc.add(hashFeature(name + "_target" + sourceLength), VALUE);
+    acc.add(hashFeature(name + "_sourceTarget" + sourceLength + "-" + targetLength), VALUE);
     return null;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/main/java/org/apache/joshua/decoder/ff/RuleShape.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/RuleShape.java b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/RuleShape.java
index 8483ad6..b389774 100644
--- a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/RuleShape.java
+++ b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/RuleShape.java
@@ -18,6 +18,8 @@
  */
 package org.apache.joshua.decoder.ff;
 
+import static org.apache.joshua.decoder.ff.FeatureMap.hashFeature;
+
 import java.util.List;
 
 import org.apache.joshua.decoder.JoshuaConfiguration;
@@ -89,11 +91,11 @@ public class RuleShape extends StatelessFF {
   @Override
   public DPState compute(Rule rule, List<HGNode> tailNodes, int i_, int j, SourcePath sourcePath,
       Sentence sentence, Accumulator acc) {
-    final String sourceShape = getRulePattern(rule.getFrench());
-    final String targetShape = getRulePattern(rule.getEnglish());
-    acc.add(name + "_source_" + sourceShape, 1);
-    acc.add(name + "_target_" + sourceShape, 1);
-    acc.add(name + "_sourceTarget_" + sourceShape + "_" + targetShape, 1);
+    final String sourceShape = getRulePattern(rule.getSource());
+    final String targetShape = getRulePattern(rule.getTarget());
+    acc.add(hashFeature(name + "_source_" + sourceShape), 1);
+    acc.add(hashFeature(name + "_target_" + sourceShape), 1);
+    acc.add(hashFeature(name + "_sourceTarget_" + sourceShape + "_" + targetShape), 1);
     return null;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/main/java/org/apache/joshua/decoder/ff/SourcePathFF.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/SourcePathFF.java b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/SourcePathFF.java
index b138426..cb902a0 100644
--- a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/SourcePathFF.java
+++ b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/SourcePathFF.java
@@ -18,7 +18,6 @@
  */
 package org.apache.joshua.decoder.ff;
 
-import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.joshua.decoder.JoshuaConfiguration;
@@ -43,21 +42,12 @@ public final class SourcePathFF extends StatelessFF {
   public SourcePathFF(FeatureVector weights, String[] args, JoshuaConfiguration config) {
     super(weights, "SourcePath", args, config);
   }
-
-  @Override
-  public ArrayList<String> reportDenseFeatures(int index) {
-    denseFeatureIndex = index;
-    
-    ArrayList<String> names = new ArrayList<String>();
-    names.add(name);
-    return names;
-  }
   
   @Override
   public DPState compute(Rule rule, List<HGNode> tailNodes, int i, int j, SourcePath sourcePath,
       Sentence sentence, Accumulator acc) {
 
-    acc.add(denseFeatureIndex,  sourcePath.getPathCost());
+    acc.add(featureId,  sourcePath.getPathCost());
     return null;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/main/java/org/apache/joshua/decoder/ff/TargetBigram.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/TargetBigram.java b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/TargetBigram.java
index e7de1f8..888fa03 100644
--- a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/TargetBigram.java
+++ b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/TargetBigram.java
@@ -18,6 +18,8 @@
  */
 package org.apache.joshua.decoder.ff;
 
+import static org.apache.joshua.decoder.ff.FeatureMap.hashFeature;
+
 import java.io.IOException;
 import java.util.HashSet;
 import java.util.LinkedList;
@@ -104,7 +106,7 @@ public class TargetBigram extends StatefulFF {
   public DPState compute(Rule rule, List<HGNode> tailNodes, int spanStart, int spanEnd,
       SourcePath sourcePath, Sentence sentence, Accumulator acc) {
 
-    int[] enWords = rule.getEnglish();
+    int[] enWords = rule.getTarget();
 
     int left = -1;
     int right = -1;
@@ -127,7 +129,7 @@ public class TargetBigram extends StatefulFF {
           right = token;
           if (currentNgram.size() == 2) {
             String ngram = join(currentNgram);
-            acc.add(String.format("%s_%s", name, ngram), 1);
+            acc.add(hashFeature(String.format("%s_%s", name, ngram)), 1);
             //            System.err.println(String.format("ADDING %s_%s", name, ngram));
             currentNgram.remove(0);
           }
@@ -144,7 +146,7 @@ public class TargetBigram extends StatefulFF {
         right = curID;
         if (currentNgram.size() == 2) {
           String ngram = join(currentNgram);
-          acc.add(String.format("%s_%s", name, ngram), 1);
+          acc.add(hashFeature(String.format("%s_%s", name, ngram)), 1);
           //          System.err.println(String.format("ADDING %s_%s", name, ngram));
           currentNgram.remove(0);
         }



[4/4] incubator-joshua git commit: Merge branch '7-feature_refactoring' into 7

Posted by mj...@apache.org.
Merge branch '7-feature_refactoring' into 7


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

Branch: refs/heads/7
Commit: 82f9183ee6fd2e0f1ff616ed92845064b92d9b76
Parents: 0ad45cc 20afddf
Author: Matt Post <po...@cs.jhu.edu>
Authored: Tue Aug 23 08:11:34 2016 -0500
Committer: Matt Post <po...@cs.jhu.edu>
Committed: Tue Aug 23 08:11:34 2016 -0500

----------------------------------------------------------------------
 .../resources/kbest_extraction/joshua.config    |   4 +-
 joshua-core/resources/wa_grammar.packed/config  |   2 +-
 .../resources/wa_grammar.packed/encoding        | Bin 154 -> 154 bytes
 .../wa_grammar.packed/slice_00000.source        | Bin 204 -> 204 bytes
 .../wa_grammar.packed/slice_00000.target        | Bin 128 -> 128 bytes
 .../resources/wa_grammar.packed/vocabulary      | Bin 238 -> 187 bytes
 .../java/org/apache/joshua/decoder/BLEU.java    |   2 +-
 .../java/org/apache/joshua/decoder/Decoder.java |  95 +---
 .../apache/joshua/decoder/DecoderThread.java    |   2 +-
 .../joshua/decoder/JoshuaConfiguration.java     |   7 +-
 .../joshua/decoder/StructuredTranslation.java   |   3 -
 .../decoder/StructuredTranslationFactory.java   |   5 +-
 .../org/apache/joshua/decoder/Translation.java  |  16 +-
 .../decoder/chart_parser/ComputeNodeResult.java |  15 +-
 .../joshua/decoder/ff/ArityPhrasePenalty.java   |   3 +-
 .../joshua/decoder/ff/FeatureFunction.java      |  63 +--
 .../apache/joshua/decoder/ff/FeatureMap.java    | 107 +++++
 .../apache/joshua/decoder/ff/FeatureVector.java | 401 ++++-------------
 .../joshua/decoder/ff/LabelCombinationFF.java   |   4 +-
 .../joshua/decoder/ff/LabelSubstitutionFF.java  |   8 +-
 .../joshua/decoder/ff/LexicalFeatures.java      |  31 +-
 .../apache/joshua/decoder/ff/OOVPenalty.java    |  22 +-
 .../apache/joshua/decoder/ff/PhraseModel.java   |  85 +---
 .../apache/joshua/decoder/ff/PhrasePenalty.java |  19 +-
 .../apache/joshua/decoder/ff/RuleCountBin.java  |  77 ----
 .../org/apache/joshua/decoder/ff/RuleFF.java    |  20 +-
 .../apache/joshua/decoder/ff/RuleLength.java    |  12 +-
 .../org/apache/joshua/decoder/ff/RuleShape.java |  12 +-
 .../apache/joshua/decoder/ff/SourcePathFF.java  |  12 +-
 .../apache/joshua/decoder/ff/TargetBigram.java  |   8 +-
 .../apache/joshua/decoder/ff/WordPenalty.java   |  15 +-
 .../decoder/ff/fragmentlm/FragmentLMFF.java     |  54 +--
 .../joshua/decoder/ff/fragmentlm/Tree.java      |  16 +-
 .../joshua/decoder/ff/lm/LanguageModelFF.java   |  43 +-
 .../ff/lm/StateMinimizingLanguageModel.java     |   8 +-
 .../joshua/decoder/ff/phrase/Distortion.java    |  19 +-
 .../ff/similarity/EdgePhraseSimilarityFF.java   |   4 +-
 .../joshua/decoder/ff/tm/AbstractGrammar.java   |   4 +-
 .../apache/joshua/decoder/ff/tm/Grammar.java    |   7 -
 .../joshua/decoder/ff/tm/GrammarReader.java     |  44 +-
 .../org/apache/joshua/decoder/ff/tm/Rule.java   | 440 +++++--------------
 .../joshua/decoder/ff/tm/RuleFactory.java       |  88 ++++
 .../decoder/ff/tm/format/HieroFormatReader.java |  28 +-
 .../decoder/ff/tm/format/MosesFormatReader.java |  18 +-
 .../tm/hash_based/MemoryBasedBatchGrammar.java  |  96 ++--
 .../ff/tm/hash_based/MemoryBasedRuleBin.java    |   6 +-
 .../decoder/ff/tm/packed/PackedGrammar.java     | 157 +++----
 .../hypergraph/FeatureVectorExtractor.java      |   6 +-
 .../GrammarBuilderWalkerFunction.java           |  35 +-
 .../joshua/decoder/hypergraph/HyperGraph.java   |   2 +-
 .../decoder/hypergraph/KBestExtractor.java      |  25 +-
 .../hypergraph/OutputStringExtractor.java       |   2 +-
 .../decoder/hypergraph/ViterbiExtractor.java    |   5 +-
 .../decoder/hypergraph/WordAlignmentState.java  |   4 +-
 .../apache/joshua/decoder/phrase/Candidate.java |   2 +-
 .../joshua/decoder/phrase/Hypothesis.java       |  11 +-
 .../joshua/decoder/phrase/PhraseChart.java      |   2 +-
 .../joshua/decoder/phrase/PhraseTable.java      |  23 +-
 .../org/apache/joshua/decoder/phrase/Stack.java |   8 +-
 .../joshua/oracle/OracleExtractionHG.java       |   2 +-
 .../org/apache/joshua/server/ServerThread.java  |  26 +-
 .../org/apache/joshua/tools/GrammarPacker.java  |  77 ++--
 .../java/org/apache/joshua/util/Constants.java  |  10 +-
 .../apache/joshua/util/PackedGrammarServer.java |   8 +-
 .../util/encoding/EncoderConfiguration.java     |   5 +-
 .../util/encoding/FeatureTypeAnalyzer.java      |  10 +-
 .../joshua/decoder/ff/FeatureVectorTest.java    | 120 +++++
 .../decoder/ff/lm/LanguageModelFFTest.java      |   5 +-
 .../lm/berkeley_lm/LMGrammarBerkeleyTest.java   |   8 +-
 .../class_lm/ClassBasedLanguageModelTest.java   |   7 +-
 .../joshua/decoder/ff/tm/OwnerMapTest.java      |  18 +
 .../joshua/decoder/ff/tm/RuleFactoryTest.java   |  42 ++
 .../kbest_extraction/KBestExtractionTest.java   |   6 +-
 .../apache/joshua/system/AlignmentMapTest.java  |   8 +-
 .../apache/joshua/system/LmOovFeatureTest.java  |   2 +-
 .../joshua/system/StructuredOutputTest.java     |  27 +-
 .../system/StructuredTranslationTest.java       |  77 ++--
 77 files changed, 1098 insertions(+), 1567 deletions(-)
----------------------------------------------------------------------



[2/4] incubator-joshua git commit: Major refactoring of core decoder components (Rule.java, FeatureVector.java and grammars). The core idea of this change is to simplify feature handling inside Joshua. Please note that this change is NOT backwards compat

Posted by mj...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/main/java/org/apache/joshua/decoder/ff/WordPenalty.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/WordPenalty.java b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/WordPenalty.java
index da315ec..c5c4a14 100644
--- a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/WordPenalty.java
+++ b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/WordPenalty.java
@@ -18,13 +18,12 @@
  */
 package org.apache.joshua.decoder.ff;
 
-import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.joshua.decoder.JoshuaConfiguration;
+import org.apache.joshua.decoder.chart_parser.SourcePath;
 import org.apache.joshua.decoder.ff.state_maintenance.DPState;
 import org.apache.joshua.decoder.ff.tm.Rule;
-import org.apache.joshua.decoder.chart_parser.SourcePath;
 import org.apache.joshua.decoder.hypergraph.HGNode;
 import org.apache.joshua.decoder.phrase.Hypothesis;
 import org.apache.joshua.decoder.segment_file.Sentence;
@@ -56,7 +55,7 @@ public final class WordPenalty extends StatelessFF {
       // TODO: this is an inefficient way to do this. Find a better way to not apply this rule
       // to start and stop glue rules when phrase-based decoding.
       if (isCky || (rule != Hypothesis.BEGIN_RULE && rule != Hypothesis.END_RULE)) {
-        acc.add(denseFeatureIndex, OMEGA * (rule.getEnglish().length - rule.getArity()));
+        acc.add(featureId, OMEGA * (rule.getTarget().length - rule.getArity()));
       }
     }
       
@@ -64,17 +63,9 @@ public final class WordPenalty extends StatelessFF {
   }
 
   @Override
-  public ArrayList<String> reportDenseFeatures(int index) {
-    denseFeatureIndex = index;
-    ArrayList<String> names = new ArrayList<>(1);
-    names.add(name);
-    return names;
-  }
-
-  @Override
   public float estimateCost(Rule rule, Sentence sentence) {
     if (rule != null)
-      return weights.getDense(denseFeatureIndex) * OMEGA * (rule.getEnglish().length - rule.getArity());
+      return weights.getOrDefault(featureId) * OMEGA * (rule.getTarget().length - rule.getArity());
     return 0.0f;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/main/java/org/apache/joshua/decoder/ff/fragmentlm/FragmentLMFF.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/fragmentlm/FragmentLMFF.java b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/fragmentlm/FragmentLMFF.java
index 861cf35..5d6780b 100644
--- a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/fragmentlm/FragmentLMFF.java
+++ b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/fragmentlm/FragmentLMFF.java
@@ -18,10 +18,11 @@
  */
 package org.apache.joshua.decoder.ff.fragmentlm;
 
+import static org.apache.joshua.decoder.ff.FeatureMap.hashFeature;
+
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
-import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Stack;
@@ -31,12 +32,8 @@ import org.apache.joshua.decoder.chart_parser.SourcePath;
 import org.apache.joshua.decoder.ff.FeatureVector;
 import org.apache.joshua.decoder.ff.StatefulFF;
 import org.apache.joshua.decoder.ff.state_maintenance.DPState;
-import org.apache.joshua.decoder.ff.tm.OwnerId;
-import org.apache.joshua.decoder.ff.tm.OwnerMap;
 import org.apache.joshua.decoder.ff.tm.Rule;
-import org.apache.joshua.decoder.ff.tm.format.HieroFormatReader;
 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;
@@ -210,12 +207,12 @@ public class FragmentLMFF extends StatefulFF {
 
           if (fragment.getLabel() == tree.getLabel() && match(fragment, tree)) {
 //             System.err.println(String.format("  FIRING: matched %s against %s", fragment, tree));
-            acc.add(fragment.escapedString(), 1);
+            acc.add(hashFeature(fragment.escapedString()), 1);
             if (OPTS_DEPTH)
               if (fragment.isLexicalized())
-                acc.add(String.format("FragmentFF_lexdepth%d", fragment.getDepth()), 1);
+                acc.add(hashFeature(String.format("FragmentFF_lexdepth%d", fragment.getDepth())), 1);
               else
-                acc.add(String.format("FragmentFF_depth%d", fragment.getDepth()), 1);
+                acc.add(hashFeature(String.format("FragmentFF_depth%d", fragment.getDepth())), 1);
           }
         }
       }
@@ -289,47 +286,6 @@ public class FragmentLMFF extends StatefulFF {
     // TODO Auto-generated method stub
     return 0;
   }
-  
-  public static void main(String[] args) {
-    /* Add an LM fragment, then create a dummy multi-level hypergraph to match the fragment against. */
-    // FragmentLMFF fragmentLMFF = new FragmentLMFF(new FeatureVector(), (StateComputer) null, "");
-    FragmentLMFF fragmentLMFF = new FragmentLMFF(new FeatureVector(),
-        new String[] {"-lm", "test/fragments.txt", "-map", "test/mapping.txt"}, null);
-  
-    Tree fragment = Tree.fromString("(S NP (VP (VBD \"said\") SBAR) (. \".\"))");
-  
-    Rule ruleS = new HieroFormatReader()
-        .parseLine("[S] ||| the man [VP,1] [.,2] ||| the man [VP,1] [.,2] ||| 0");
-    Rule ruleVP = new HieroFormatReader()
-        .parseLine("[VP] ||| said [SBAR,1] ||| said [SBAR,1] ||| 0");
-    Rule ruleSBAR = new HieroFormatReader()
-        .parseLine("[SBAR] ||| that he was done ||| that he was done ||| 0");
-    Rule rulePERIOD = new HieroFormatReader().parseLine("[.] ||| . ||| . ||| 0");
-  
-    final OwnerId owner = OwnerMap.register("0");
-    ruleS.setOwner(owner);
-    ruleVP.setOwner(owner);
-    ruleSBAR.setOwner(owner);
-    rulePERIOD.setOwner(owner);
-  
-    HyperEdge edgeSBAR = new HyperEdge(ruleSBAR, 0.0f, 0.0f, null, (SourcePath) null);
-  
-    HGNode nodeSBAR = new HGNode(3, 7, ruleSBAR.getLHS(), null, edgeSBAR, 0.0f);
-    ArrayList<HGNode> tailNodesVP = new ArrayList<HGNode>();
-    Collections.addAll(tailNodesVP, nodeSBAR);
-    HyperEdge edgeVP = new HyperEdge(ruleVP, 0.0f, 0.0f, tailNodesVP, (SourcePath) null);
-    HGNode nodeVP = new HGNode(2, 7, ruleVP.getLHS(), null, edgeVP, 0.0f);
-  
-    HyperEdge edgePERIOD = new HyperEdge(rulePERIOD, 0.0f, 0.0f, null, (SourcePath) null);
-    HGNode nodePERIOD = new HGNode(7, 8, rulePERIOD.getLHS(), null, edgePERIOD, 0.0f);
-  
-    ArrayList<HGNode> tailNodes = new ArrayList<HGNode>();
-    Collections.addAll(tailNodes, nodeVP, nodePERIOD);
-  
-    Tree tree = Tree.buildTree(ruleS, tailNodes, 1);
-    boolean matched = fragmentLMFF.match(fragment, tree);
-    LOG.info("Does\n  {} match\n  {}??\n  -> {}", fragment, tree, matched);
-  }
 
   /**
    * Maintains a state pointer used by KenLM to implement left-state minimization. 

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/main/java/org/apache/joshua/decoder/ff/fragmentlm/Tree.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/fragmentlm/Tree.java b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/fragmentlm/Tree.java
index 07c7ecd..7e22b97 100644
--- a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/fragmentlm/Tree.java
+++ b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/fragmentlm/Tree.java
@@ -547,7 +547,7 @@ public class Tree implements Serializable {
    * @return the Tree 
    */
   public static Tree buildTree(Rule rule, DerivationState[] derivationStates, int maxDepth) {
-    Tree tree = getFragmentFromYield(rule.getEnglishWords());
+    Tree tree = getFragmentFromYield(rule.getTargetWords());
 
     if (tree == null) {
       return null;
@@ -572,7 +572,7 @@ public class Tree implements Serializable {
      * the incoming DerivationState items, which are ordered by the source side.
      */
     ArrayList<Integer> tailIndices = new ArrayList<Integer>();
-    int[] englishInts = rule.getEnglish();
+    int[] englishInts = rule.getTarget();
     for (int i = 0; i < englishInts.length; i++)
       if (englishInts[i] < 0)
         tailIndices.add(-(englishInts[i] + 1));
@@ -623,7 +623,7 @@ public class Tree implements Serializable {
   public static Tree buildTree(DerivationState derivationState, int maxDepth) {
     Rule rule = derivationState.edge.getRule();
     
-    Tree tree = getFragmentFromYield(rule.getEnglishWords());
+    Tree tree = getFragmentFromYield(rule.getTargetWords());
 
     if (tree == null) {
       return null;
@@ -644,7 +644,7 @@ public class Tree implements Serializable {
        * the incoming DerivationState items, which are ordered by the source side.
        */
       ArrayList<Integer> tailIndices = new ArrayList<Integer>();
-      int[] englishInts = rule.getEnglish();
+      int[] englishInts = rule.getTarget();
       for (int i = 0; i < englishInts.length; i++)
         if (englishInts[i] < 0)
           tailIndices.add(-(englishInts[i] + 1));
@@ -685,10 +685,10 @@ public class Tree implements Serializable {
    * @return shallow clone of the Tree object
    */
   public static Tree buildTree(Rule rule, List<HGNode> tailNodes, int maxDepth) {
-    Tree tree = getFragmentFromYield(rule.getEnglishWords());
+    Tree tree = getFragmentFromYield(rule.getTargetWords());
 
     if (tree == null) {
-      tree = new Tree(String.format("(%s %s)", Vocabulary.word(rule.getLHS()), rule.getEnglishWords()));
+      tree = new Tree(String.format("(%s %s)", Vocabulary.word(rule.getLHS()), rule.getTargetWords()));
       // System.err.println("COULDN'T FIND " + rule.getEnglishWords());
       // System.err.println("RULE " + rule);
       // for (Entry<String, Tree> pair: rulesToFragments.entrySet())
@@ -703,7 +703,7 @@ public class Tree implements Serializable {
       List<Tree> frontier = tree.getNonterminalYield();
 
       ArrayList<Integer> tailIndices = new ArrayList<Integer>();
-      int[] englishInts = rule.getEnglish();
+      int[] englishInts = rule.getTarget();
       for (int i = 0; i < englishInts.length; i++)
         if (englishInts[i] < 0)
           tailIndices.add(-1 * englishInts[i] - 1);
@@ -734,7 +734,7 @@ public class Tree implements Serializable {
           }
         } catch (IndexOutOfBoundsException e) {
           LOG.error("ERROR at index {}", i);
-          LOG.error("RULE: {}  TREE: {}", rule.getEnglishWords(), tree);
+          LOG.error("RULE: {}  TREE: {}", rule.getTargetWords(), tree);
           LOG.error("  FRONTIER:");
           for (Tree kid : frontier) {
             LOG.error("    {}", kid);

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/main/java/org/apache/joshua/decoder/ff/lm/LanguageModelFF.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/lm/LanguageModelFF.java b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/lm/LanguageModelFF.java
index 7b0bac8..c460cb0 100644
--- a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/lm/LanguageModelFF.java
+++ b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/lm/LanguageModelFF.java
@@ -30,6 +30,7 @@ import org.apache.joshua.corpus.Vocabulary;
 import org.apache.joshua.decoder.JoshuaConfiguration;
 import org.apache.joshua.decoder.Support;
 import org.apache.joshua.decoder.chart_parser.SourcePath;
+import org.apache.joshua.decoder.ff.FeatureMap;
 import org.apache.joshua.decoder.ff.FeatureVector;
 import org.apache.joshua.decoder.ff.StatefulFF;
 import org.apache.joshua.decoder.ff.lm.berkeley_lm.LMGrammarBerkeley;
@@ -84,7 +85,7 @@ public class LanguageModelFF extends StatefulFF {
   
   protected final static String NAME_PREFIX = "lm_";
   protected final static String OOV_SUFFIX = "_oov";
-  protected final String oovFeatureName;
+  protected final int oovFeatureId;
 
   /**
    * We always use this order of ngram, though the LMGrammar may provide higher order probability.
@@ -94,8 +95,6 @@ public class LanguageModelFF extends StatefulFF {
   /**
    * We cache the weight of the feature since there is only one.
    */
-  protected float weight;
-  protected float oovWeight;
   protected String type;
   protected String path;
 
@@ -105,11 +104,10 @@ public class LanguageModelFF extends StatefulFF {
   
   /** Whether this feature function fires LM oov indicators */ 
   protected boolean withOovFeature;
-  protected int oovDenseFeatureIndex = -1;
 
   public LanguageModelFF(FeatureVector weights, String[] args, JoshuaConfiguration config) {
     super(weights, NAME_PREFIX + LM_INDEX, args, config);
-    this.oovFeatureName = NAME_PREFIX + LM_INDEX + OOV_SUFFIX;
+    this.oovFeatureId = FeatureMap.hashFeature(NAME_PREFIX + LM_INDEX + OOV_SUFFIX);
     LM_INDEX++;
 
     this.type = parsedArgs.get("lm_type");
@@ -125,26 +123,9 @@ public class LanguageModelFF extends StatefulFF {
       this.withOovFeature = true;
     }
 
-    // The dense feature initialization hasn't happened yet, so we have to retrieve this as sparse
-    this.weight = weights.getSparse(name);
-    this.oovWeight = weights.getSparse(oovFeatureName);
-
     initializeLM();
   }
 
-  @Override
-  public ArrayList<String> reportDenseFeatures(int index) {
-    denseFeatureIndex = index;
-    oovDenseFeatureIndex = denseFeatureIndex + 1;
-
-    final ArrayList<String> names = new ArrayList<String>(2);
-    names.add(name);
-    if (withOovFeature) {
-      names.add(oovFeatureName);
-    }
-    return names;
-  }
-
   /**
    * Initializes the underlying language model.
    */
@@ -176,7 +157,7 @@ public class LanguageModelFF extends StatefulFF {
   }
 
   public String logString() {
-    return String.format("%s, order %d (weight %.3f), classLm=%s", name, languageModel.getOrder(), weight, isClassLM);
+    return String.format("%s, order %d (weight %.3f), classLm=%s", name, languageModel.getOrder(), weights.getOrDefault(featureId), isClassLM);
   }
 
   /**
@@ -201,7 +182,7 @@ public class LanguageModelFF extends StatefulFF {
     }
     
     if (withOovFeature) {
-      acc.add(oovDenseFeatureIndex, getOovs(words));
+      acc.add(oovFeatureId, getOovs(words));
     }
 
     return computeTransition(words, tailNodes, acc);
@@ -222,7 +203,7 @@ public class LanguageModelFF extends StatefulFF {
       return getClasses(rule);
     }
     // Regular LM: use rule word ids
-    return rule.getEnglish();
+    return rule.getTarget();
   }
   
   /**
@@ -254,7 +235,7 @@ public class LanguageModelFF extends StatefulFF {
    */
   protected int[] getTags(Rule rule, int begin, int end, Sentence sentence) {
     /* Very important to make a copy here, so the original rule is not modified */
-    int[] tokens = Arrays.copyOf(rule.getEnglish(), rule.getEnglish().length);
+    int[] tokens = Arrays.copyOf(rule.getTarget(), rule.getTarget().length);
     byte[] alignments = rule.getAlignment();
 
     //    System.err.println(String.format("getTags() %s", rule.getRuleString()));
@@ -302,7 +283,7 @@ public class LanguageModelFF extends StatefulFF {
       throw new RuntimeException("The class map is not set. Cannot use the class LM ");
     }
     /* Very important to make a copy here, so the original rule is not modified */
-    int[] tokens = Arrays.copyOf(rule.getEnglish(), rule.getEnglish().length);
+    int[] tokens = Arrays.copyOf(rule.getTarget(), rule.getTarget().length);
     for (int i = 0; i < tokens.length; i++) {
       if (tokens[i] > 0 ) { // skip non-terminals
         tokens[i] = this.classMap.getClassID(tokens[i]);
@@ -350,7 +331,7 @@ public class LanguageModelFF extends StatefulFF {
     
     final float oovEstimate = (withOovFeature) ? getOovs(enWords) : 0f;
 
-    return weight * lmEstimate + oovWeight * oovEstimate;
+    return weights.getOrDefault(featureId) * lmEstimate + weights.getOrDefault(oovFeatureId) * oovEstimate;
   }
 
   /**
@@ -372,7 +353,7 @@ public class LanguageModelFF extends StatefulFF {
       estimate += scoreChunkLogP(leftContext, true, skipStart);
     }
     // NOTE: no future cost for oov weight
-    return weight * estimate;
+    return weights.getOrDefault(featureId) * estimate;
   }
 
   /**
@@ -446,7 +427,7 @@ public class LanguageModelFF extends StatefulFF {
       }
     }
     //    acc.add(name, transitionLogP);
-    acc.add(denseFeatureIndex, transitionLogP);
+    acc.add(featureId, transitionLogP);
 
     if (left_context != null) {
       return new NgramDPState(left_context, Arrays.copyOfRange(current, ccount - this.ngramOrder
@@ -489,7 +470,7 @@ public class LanguageModelFF extends StatefulFF {
 
     // Tell the accumulator
     //    acc.add(name, res);
-    acc.add(denseFeatureIndex, res);
+    acc.add(featureId, res);
 
     // State is the same
     return new NgramDPState(leftContext, rightContext);

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/main/java/org/apache/joshua/decoder/ff/lm/StateMinimizingLanguageModel.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/lm/StateMinimizingLanguageModel.java b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/lm/StateMinimizingLanguageModel.java
index 4bec379..baf8b1a 100644
--- a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/lm/StateMinimizingLanguageModel.java
+++ b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/lm/StateMinimizingLanguageModel.java
@@ -82,8 +82,8 @@ public class StateMinimizingLanguageModel extends LanguageModelFF {
     final long[] words = mapToKenLmIds(ruleWords, null, true);
     
     // Get the probability of applying the rule and the new state
-    float lmCost = weight * ((KenLM) languageModel).estimateRule(words);
-    float oovCost = oovWeight * ((withOovFeature) ? getOovs(ruleWords) : 0f);
+    float lmCost = weights.getOrDefault(featureId) * ((KenLM) languageModel).estimateRule(words);
+    float oovCost = weights.getOrDefault(oovFeatureId) * ((withOovFeature) ? getOovs(ruleWords) : 0f);
     return lmCost + oovCost;
   }
 
@@ -109,7 +109,7 @@ public class StateMinimizingLanguageModel extends LanguageModelFF {
     
     // Record the oov count
     if (withOovFeature) {
-      acc.add(oovDenseFeatureIndex, getOovs(ruleWords));
+      acc.add(oovFeatureId, getOovs(ruleWords));
     }
 
      // map to ken lm ids
@@ -125,7 +125,7 @@ public class StateMinimizingLanguageModel extends LanguageModelFF {
     final StateProbPair pair = ((KenLM) languageModel).probRule(words, poolMap.get(sentID));
 
     // Record the prob
-    acc.add(denseFeatureIndex, pair.prob);
+    acc.add(featureId, pair.prob);
 
     // Return the state
     return pair.state;

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/main/java/org/apache/joshua/decoder/ff/phrase/Distortion.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/phrase/Distortion.java b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/phrase/Distortion.java
index f9e6a29..fc6c831 100644
--- a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/phrase/Distortion.java
+++ b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/phrase/Distortion.java
@@ -18,8 +18,7 @@
  */
 package org.apache.joshua.decoder.ff.phrase;
 
-import java.util.ArrayList;
-import java.util.List;	
+import java.util.List;
 
 import org.apache.joshua.decoder.JoshuaConfiguration;
 import org.apache.joshua.decoder.chart_parser.SourcePath;
@@ -42,29 +41,19 @@ public class Distortion extends StatelessFF {
       throw new RuntimeException(msg);
     }
   }
-  
-  @Override
-  public ArrayList<String> reportDenseFeatures(int index) {
-    denseFeatureIndex = index;
-    
-    ArrayList<String> names = new ArrayList<String>();
-    names.add(name);
-    return names;
-  }
 
   @Override
   public DPState compute(Rule rule, List<HGNode> tailNodes, int i, int j, SourcePath sourcePath,
       Sentence sentence, Accumulator acc) {
 
     if (rule != Hypothesis.BEGIN_RULE && rule != Hypothesis.END_RULE) {
-        int start_point = j - rule.getFrench().length + rule.getArity();
+        int start_point = j - rule.getSource().length + rule.getArity();
 
         int jump_size = Math.abs(tailNodes.get(0).j - start_point);
-//        acc.add(name, -jump_size);
-        acc.add(denseFeatureIndex, -jump_size); 
+        acc.add(featureId, -jump_size); 
     }
     
-//    System.err.println(String.format("DISTORTION(%d, %d) from %d = %d", i, j, tailNodes != null ? tailNodes.get(0).j : -1, jump_size));
+    // System.err.println(String.format("DISTORTION(%d, %d) from %d = %d", i, j, tailNodes != null ? tailNodes.get(0).j : -1, jump_size));
 
     return null;
   }

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/main/java/org/apache/joshua/decoder/ff/similarity/EdgePhraseSimilarityFF.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/similarity/EdgePhraseSimilarityFF.java b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/similarity/EdgePhraseSimilarityFF.java
index 91af58b..9f402cb 100644
--- a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/similarity/EdgePhraseSimilarityFF.java
+++ b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/similarity/EdgePhraseSimilarityFF.java
@@ -84,7 +84,7 @@ public class EdgePhraseSimilarityFF extends StatefulFF implements SourceDependen
       Sentence sentence, Accumulator acc) {
 
     float value = computeScore(rule, tailNodes);
-    acc.add(name, value);
+    acc.add(featureId, value);
 
     // TODO 07/2013: EdgePhraseSimilarity needs to know its order rather than inferring it from tail
     // nodes.
@@ -102,7 +102,7 @@ public class EdgePhraseSimilarityFF extends StatefulFF implements SourceDependen
 
     // System.err.println("RULE [" + spanStart + ", " + spanEnd + "]: " + rule.toString());
 
-    int[] target = rule.getEnglish();
+    int[] target = rule.getTarget();
     int lm_state_size = 0;
     for (HGNode node : tailNodes) {
       NgramDPState state = (NgramDPState) node.getDPState(stateIndex);

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/main/java/org/apache/joshua/decoder/ff/tm/AbstractGrammar.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/tm/AbstractGrammar.java b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/tm/AbstractGrammar.java
index 01260ab..70e4daf 100644
--- a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/tm/AbstractGrammar.java
+++ b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/tm/AbstractGrammar.java
@@ -165,8 +165,8 @@ public abstract class AbstractGrammar implements Grammar {
         if (LOG.isDebugEnabled()) {
           StringBuilder s = new StringBuilder();
           for (Rule r : rules.getSortedRules(models)) {
-            s.append("\n\t" + r.getLHS() + " ||| " + Arrays.toString(r.getFrench()) + " ||| "
-                + Arrays.toString(r.getEnglish()) + " ||| " + r.getFeatureVector() + " ||| "
+            s.append("\n\t" + r.getLHS() + " ||| " + Arrays.toString(r.getSource()) + " ||| "
+                + Arrays.toString(r.getTarget()) + " ||| " + r.getFeatureVector() + " ||| "
                 + r.getEstimatedCost() + "  " + r.getClass().getName() + "@"
                 + Integer.toHexString(System.identityHashCode(r)));
           }

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/main/java/org/apache/joshua/decoder/ff/tm/Grammar.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/tm/Grammar.java b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/tm/Grammar.java
index 8f90d1b..8497c17 100644
--- a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/tm/Grammar.java
+++ b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/tm/Grammar.java
@@ -83,13 +83,6 @@ public interface Grammar {
    * @return the number of rules stored in the grammar
    */
   int getNumRules();
-  
-  /**
-   * Returns the number of dense features.
-   * 
-   * @return the number of dense features
-   */
-  int getNumDenseFeatures();
 
   /**
    * Return the grammar's owner.

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/main/java/org/apache/joshua/decoder/ff/tm/GrammarReader.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/tm/GrammarReader.java b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/tm/GrammarReader.java
index df00255..7e3d6cd 100644
--- a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/tm/GrammarReader.java
+++ b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/tm/GrammarReader.java
@@ -22,6 +22,8 @@ import java.io.IOException;
 import java.util.Iterator;
 
 import org.apache.joshua.decoder.Decoder;
+import org.apache.joshua.decoder.ff.tm.format.HieroFormatReader;
+import org.apache.joshua.decoder.ff.tm.format.MosesFormatReader;
 import org.apache.joshua.util.io.LineReader;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -36,27 +38,52 @@ public abstract class GrammarReader<R extends Rule> implements Iterable<R>, Iter
 
   private static final Logger LOG = LoggerFactory.getLogger(GrammarReader.class);
 
-  protected static String fieldDelimiter;
   protected static String description;
 
-  protected String fileName;
-  protected LineReader reader;
+  protected final String fileName;
+  protected final LineReader reader;
   protected String lookAhead;
   protected int numRulesRead;
+  
+  /** A grammar reader requires an owner to correctly parse and hash the rule's feature values (prepended by the ownwer string) */
+  protected final OwnerId owner;
 
-
-  // dummy constructor for
-  public GrammarReader() {
+  /**
+   * Constructor for in-memory grammars where rules are added later
+   * @param ownerId the owner of the resulting grammar
+   */
+  public GrammarReader(OwnerId ownerId) {
+    this.owner = ownerId;
     this.fileName = null;
+    this.reader = null;
   }
 
-  public GrammarReader(String fileName) throws IOException {
+  /**
+   * Constructor for in-memory grammars read from a text file.
+   * @param fileName
+   * @param ownerId
+   * @throws IOException
+   */
+  public GrammarReader(String fileName, OwnerId ownerId) throws IOException {
     this.fileName = fileName;
+    this.owner = ownerId;
     this.reader = new LineReader(fileName);
     LOG.info("Reading grammar from file {}...", fileName);
     numRulesRead = 0;
     advanceReader();
   }
+  
+  /**
+   * Given a grammar format, returns the appropriate GrammarReader implementation. 
+   */
+  public static GrammarReader<Rule> createReader(String format, String grammarFile, OwnerId ownerId) throws IOException {
+    if ("hiero".equals(format) || "thrax".equals(format)) {
+      return new HieroFormatReader(grammarFile, ownerId);
+    } else if ("moses".equals(format)) {
+      return new MosesFormatReader(grammarFile, ownerId);
+    }
+    throw new RuntimeException(String.format("* FATAL: unknown grammar format '%s'", format));
+  }
 
   // the reader is the iterator itself
   public Iterator<R> iterator() {
@@ -74,9 +101,8 @@ public abstract class GrammarReader<R extends Rule> implements Iterable<R>, Iter
         this.reader.close();
       } catch (IOException e) {
         LOG.warn(e.getMessage(), e);
-        LOG.error("Error closing grammar file stream: {}",  this.fileName);
+        LOG.error("Error closing grammar file stream: {}", this.fileName);
       }
-      this.reader = null;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/main/java/org/apache/joshua/decoder/ff/tm/Rule.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/tm/Rule.java b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/tm/Rule.java
index 15fbec1..601dde7 100644
--- a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/tm/Rule.java
+++ b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/tm/Rule.java
@@ -19,212 +19,87 @@
 package org.apache.joshua.decoder.ff.tm;
 
 import static org.apache.joshua.decoder.ff.tm.OwnerMap.UNKNOWN_OWNER_ID;
+import static org.apache.joshua.util.Constants.NT_REGEX;
+import static org.apache.joshua.util.Constants.fieldDelimiter;
 
 import java.util.ArrayList;
-import java.util.Arrays;  
+import java.util.Arrays;
 import java.util.Comparator;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.regex.Pattern;
 
-import com.google.common.base.Supplier;
-import com.google.common.base.Suppliers;
-
 import org.apache.joshua.corpus.Vocabulary;
 import org.apache.joshua.decoder.ff.FeatureFunction;
 import org.apache.joshua.decoder.ff.FeatureVector;
+import org.apache.joshua.decoder.ff.tm.format.HieroFormatReader;
 import org.apache.joshua.decoder.segment_file.Sentence;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * This class define the interface for Rule. 
- * 
- * All feature scores are interpreted as negative log probabilities, and are therefore negated.
- * Note that not all features need to be negative log probs, but you should be aware that they
- * will be negated, so if you want a positive count, it should come in as negative.
- * 
- * Normally, the feature score in the rule should be *cost* (i.e., -LogP), so that the feature
- * weight should be positive
+ * This class defines the interface for Rule. Components of a rule (left-hand-side,
+ * source words, target words, features, alignments) are final and can not be modified.
+ * This forces creators of Rule instances to decide on feature namespaces and owner in advances and greatly
+ * simplifies the code.
  * 
  * @author Zhifei Li, zhifei.work@gmail.com
  * @author Matt Post post@cs.jhu.edu
+ * @author fhieber
  */
 public class Rule implements Comparator<Rule>, Comparable<Rule> {
-
+  
   private static final Logger LOG = LoggerFactory.getLogger(Rule.class);
-  private int lhs; // tag of this rule
-  private int[] source; // pointer to the RuleCollection, as all the rules under it share the same
-                         // Source side
-  protected int arity;
+  
+  /** left hand side vocabulary id */
+  private final int lhs;
+
+  /** source vocabulary ids */
+  private final int[] source;
+  
+  /** target vocabulary ids */
+  private final int[] target;
+  
+  /** arity of the rule (number of non-terminals) */
+  protected final int arity;
 
-  // And a string containing the sparse ones
-  //protected final String sparseFeatureString;
-  protected final Supplier<String> sparseFeatureStringSupplier;
-  private final Supplier<FeatureVector> featuresSupplier;
+  /** the {@link FeatureVector} associated with this {@link Rule} */
+  private final FeatureVector featureVector;
 
-  /*
-   * a feature function will be fired for this rule only if the owner of the rule matches the owner
-   * of the feature function
-   */
+  /** The {@link OwnerId} this rule belongs to. */
   private OwnerId owner = UNKNOWN_OWNER_ID;
 
   /**
-   * This is the cost computed only from the features present with the grammar rule. This cost is
+   * This is the rule cost computed only from local rule context. This cost is
    * needed to sort the rules in the grammar for cube pruning, but isn't the full cost of applying
    * the rule (which will include contextual features that can't be computed until the rule is
    * applied).
    */
   private float estimatedCost = Float.NEGATIVE_INFINITY;
 
-  private float precomputableCost = Float.NEGATIVE_INFINITY;
-
-  private int[] target;
-
-  // The alignment string, e.g., 0-0 0-1 1-1 2-1
-  private String alignmentString;
-  private final Supplier<byte[]> alignmentSupplier;
-
+  private final byte[] alignments;
+  
   /**
-   * Constructs a new rule using the provided parameters. Rule id for this rule is
-   * undefined. Note that some of the sparse features may be unlabeled, but they cannot be mapped to
-   * their default names ("tm_OWNER_INDEX") until later, when we know the owner of the rule. This is
-   * not known until the rule is actually added to a grammar in Grammar::addRule().
-   * 
-   * Constructor used by other constructors below;
-   * 
-   * @param lhs Left-hand side of the rule.
-   * @param source Source language right-hand side of the rule.
-   * @param target Target language right-hand side of the rule.
-   * @param sparseFeatures Feature value scores for the rule.
-   * @param arity Number of nonterminals in the source language right-hand side.
-   * @param owner todo
+   * Constructs a rule given its dependencies. ownerId should be the same as used for
+   * 'hashing'/creating the {@link FeatureVector} features.
    */
-  public Rule(int lhs, int[] source, int[] target, String sparseFeatures, int arity, OwnerId owner) {
+  public Rule(int lhs, int[] source, int[] target, int arity, FeatureVector features, byte[] alignments, OwnerId ownerId) {
     this.lhs = lhs;
     this.source = source;
-    this.arity = arity;
-    this.owner = owner;
     this.target = target;
-    this.sparseFeatureStringSupplier = Suppliers.memoize(() -> { return sparseFeatures; });
-    this.featuresSupplier = initializeFeatureSupplierFromString();
-    this.alignmentSupplier = initializeAlignmentSupplier();
-  }
-  
-  /**
-   * Constructor used by PackedGrammar's sortRules()
-   * @param lhs todo
-   * @param sourceRhs todo
-   * @param targetRhs todo
-   * @param features todo
-   * @param arity todo
-   * @param owner todo
-   */
-  public Rule(int lhs, int[] sourceRhs, int[] targetRhs, FeatureVector features, int arity, OwnerId owner) {
-    this.lhs = lhs;
-    this.source = sourceRhs;
     this.arity = arity;
-    this.owner = owner;
-    this.target = targetRhs;
-    this.featuresSupplier = Suppliers.memoize(() -> { return features; });
-    this.sparseFeatureStringSupplier = initializeSparseFeaturesStringSupplier();
-    this.alignmentSupplier = initializeAlignmentSupplier();
-  }
-
-  /**
-   * Constructor used for SamtFormatReader and GrammarBuilderWalkerFunction's getRuleWithSpans()
-   * Rule is unowned.
-   * @param lhs todo
-   * @param sourceRhs todo
-   * @param targetRhs todo
-   * @param sparseFeatures todo
-   * @param arity todo
-   */
-  public Rule(int lhs, int[] sourceRhs, int[] targetRhs, String sparseFeatures, int arity) {
-    this(lhs, sourceRhs, targetRhs, sparseFeatures, arity, OwnerMap.UNKNOWN_OWNER_ID);
+    this.featureVector = features;
+    this.alignments = alignments;
+    this.owner = ownerId;
   }
 
-  /**
-   * Constructor used for addOOVRules(), HieroFormatReader and PhraseRule.
-   * @param lhs todo
-   * @param sourceRhs todo
-   * @param targetRhs todo
-   * @param sparseFeatures todo
-   * @param arity todo
-   * @param alignment todo
-   */
-  public Rule(int lhs, int[] sourceRhs, int[] targetRhs, String sparseFeatures, int arity, String alignment) {
-    this(lhs, sourceRhs, targetRhs, sparseFeatures, arity);
-    this.alignmentString = alignment;
-  }
-  
-  /**
-   * Constructor (implicitly) used by PackedRule
-   */
-  public Rule() {
-    this.lhs = -1;
-    this.sparseFeatureStringSupplier = initializeSparseFeaturesStringSupplier();
-    this.featuresSupplier = initializeFeatureSupplierFromString();
-    this.alignmentSupplier = initializeAlignmentSupplier();
-  }
-
-  // ==========================================================================
-  // Lazy loading Suppliers for alignments, feature vector, and feature strings
-  // ==========================================================================
-  
-  private Supplier<byte[]> initializeAlignmentSupplier(){
-    return Suppliers.memoize(() ->{
-      byte[] alignment = null;
-      String alignmentString = getAlignmentString();
-      if (alignmentString != null) {
-        String[] tokens = alignmentString.split("[-\\s]+");
-        alignment = new byte[tokens.length];
-        for (int i = 0; i < tokens.length; i++)
-          alignment[i] = (byte) Short.parseShort(tokens[i]);
-      }
-      return alignment;
-    });
-  }
-  
-  /**
-   * If Rule was constructed with sparseFeatures String, we lazily populate the
-   * FeatureSupplier.
-   */
-  private Supplier<FeatureVector> initializeFeatureSupplierFromString(){
-    return Suppliers.memoize(() ->{
-      if (!owner.equals(UNKNOWN_OWNER_ID)) {
-        return new FeatureVector(getFeatureString(), "tm_" + OwnerMap.getOwner(owner) + "_");
-      } else {
-        return new FeatureVector();
-      }
-    });
-  }
-  
-  /**
-   * If Rule was constructed with a FeatureVector, we lazily populate the sparseFeaturesStringSupplier.
-   */
-  private Supplier<String> initializeSparseFeaturesStringSupplier() {
-    return Suppliers.memoize(() -> {
-      return getFeatureVector().toString();
-    });
-  }
-
-  // ===============================================================
-  // Attributes
-  // ===============================================================
-
-  public void setEnglish(int[] eng) {
-    this.target = eng;
-  }
-
-  public int[] getEnglish() {
+  public int[] getTarget() {
     return this.target;
   }
 
   /**
-   * Two Rules are equal of they have the same LHS, the same source RHS and the same target
-   * RHS.
+   * Two Rules are equal of they have the same LHS, the same source RHS and the same target RHS.
    * 
    * @param o the object to check for equality
    * @return true if o is the same Rule as this rule, false otherwise
@@ -237,10 +112,10 @@ public class Rule implements Comparator<Rule>, Comparable<Rule> {
     if (getLHS() != other.getLHS()) {
       return false;
     }
-    if (!Arrays.equals(getFrench(), other.getFrench())) {
+    if (!Arrays.equals(getSource(), other.getSource())) {
       return false;
     }
-    if (!Arrays.equals(target, other.getEnglish())) {
+    if (!Arrays.equals(target, other.getTarget())) {
       return false;
     }
     return true;
@@ -249,61 +124,29 @@ public class Rule implements Comparator<Rule>, Comparable<Rule> {
   public int hashCode() {
     // I just made this up. If two rules are equal they'll have the
     // same hashcode. Maybe someone else can do a better job though?
-    int frHash = Arrays.hashCode(getFrench());
+    int frHash = Arrays.hashCode(getSource());
     int enHash = Arrays.hashCode(target);
     return frHash ^ enHash ^ getLHS();
   }
-
-  // ===============================================================
-  // Attributes
-  // ===============================================================
-
-  public void setArity(int arity) {
-    this.arity = arity;
-  }
-
+  
   public int getArity() {
     return this.arity;
   }
 
-  public void setOwner(final OwnerId owner) {
-    this.owner = owner;
-  }
-
   public OwnerId getOwner() {
     return this.owner;
   }
 
-  public void setLHS(int lhs) {
-    this.lhs = lhs;
-  }
-
   public int getLHS() {
     return this.lhs;
   }
 
-  public void setFrench(int[] french) {
-    this.source = french;
-  }
-
-  public int[] getFrench() {
+  public int[] getSource() {
     return this.source;
   }
 
-  /**
-   * This function does the work of turning the string version of the sparse features (passed in
-   * when the rule was created) into an actual set of features. This is a bit complicated because we
-   * support intermingled labeled and unlabeled features, where the unlabeled features are mapped to
-   * a default name template of the form "tm_OWNER_INDEX".
-   * 
-   * This function returns the dense (phrasal) features discovered when the rule was loaded. Dense
-   * features are the list of unlabeled features that preceded labeled ones. They can also be
-   * specified as labeled features of the form "tm_OWNER_INDEX", but the former format is preferred.
-   * 
-   * @return the {@link org.apache.joshua.decoder.ff.FeatureVector} for this rule
-   */
   public FeatureVector getFeatureVector() {
-    return featuresSupplier.get();
+    return featureVector;
   }
 
   /**
@@ -321,44 +164,6 @@ public class Rule implements Comparator<Rule>, Comparable<Rule> {
   public float getEstimatedCost() {
     return estimatedCost;
   }
-
-  /**
-   * Precomputable costs is the inner product of the weights found on each grammar rule and the
-   * weight vector. This is slightly different from the estimated rule cost, which can include other
-   * features (such as a language model estimate). This getter and setter should also be cached, and
-   * is basically provided to allow the PhraseModel feature to cache its (expensive) computation for
-   * each rule.
-   *
-   * The weights are passed in as dense weights and sparse weights. This allows the dense weight
-   * precomputation to be even faster (since we don't have to query a hash map. 
-   *
-   * @param dense_weights the dense weights from the model
-   * @param weights the sparse weights from the model
-   */
-  public void setPrecomputableCost(float[] dense_weights, FeatureVector weights) {
-    float cost = 0.0f;
-    FeatureVector features = getFeatureVector();
-    for (int i = 0; i < features.getDenseFeatures().size() && i < dense_weights.length; i++) {
-      cost += dense_weights[i] * features.getDense(i);
-    }
-
-    for (String key: features.getSparseFeatures().keySet()) {
-      cost += weights.getSparse(key) * features.getSparse(key);
-    }
-    
-    this.precomputableCost = cost;
-  }
-  
-  /**
-   * @return the precomputed model cost of each rule
-   */
-  public float getPrecomputableCost() {
-    return precomputableCost;
-  }
-  
-  public float getDenseFeature(int k) {
-    return getFeatureVector().getDense(k);
-  }
   
   /**
    * This function estimates the cost of a rule, which is used for sorting the rules for cube
@@ -374,104 +179,86 @@ public class Rule implements Comparator<Rule>, Comparable<Rule> {
    * @return estimated cost of the rule
    */
   public float estimateRuleCost(List<FeatureFunction> models) {
-    if (null == models)
-      return 0.0f;
-
+    
     if (this.estimatedCost <= Float.NEGATIVE_INFINITY) {
-      this.estimatedCost = 0.0f; // weights.innerProduct(computeFeatures());
-
-      LOG.debug("estimateCost({} ;; {})", getFrenchWords(), getEnglishWords());
-      for (FeatureFunction ff : models) {
+      float result = 0.0f;
+      LOG.debug("estimateRuleCost({} ;; {})", getSourceWords(), getTargetWords());
+      for (final FeatureFunction ff : models) {
         float val = ff.estimateCost(this, null);
         LOG.debug("  FEATURE {} -> {}", ff.getName(), val);
-        this.estimatedCost += val; 
+        result += val; 
       }
+      this.estimatedCost = result;
     }
-    
+
     return estimatedCost;
   }
 
-  // ===============================================================
-  // Methods
-  // ===============================================================
-
+  /**
+   * Returns an informative String for the rule, including estimated cost and the rule's owner.
+   */
+  @Override
   public String toString() {
-    StringBuffer sb = new StringBuffer();
-    sb.append(Vocabulary.word(this.getLHS()));
-    sb.append(" ||| ");
-    sb.append(getFrenchWords());
-    sb.append(" ||| ");
-    sb.append(getEnglishWords());
-    sb.append(" |||");
-    sb.append(" " + getFeatureVector());
-    sb.append(String.format(" ||| est=%.3f", getEstimatedCost()));
-    sb.append(String.format(" pre=%.3f", getPrecomputableCost()));
-    return sb.toString();
+    return new StringBuffer(textFormat())
+        .append(fieldDelimiter)
+        .append(getEstimatedCost())
+        .append(fieldDelimiter)
+        .append(OwnerMap.getOwner(getOwner()))
+        .toString();
   }
   
   /**
-   * Returns a version of the rule suitable for reading in from a text file.
-   * 
-   * @return string version of the rule
+   * Returns a string version of the rule parsable by the {@link HieroFormatReader}.
    */
   public String textFormat() {
-    StringBuffer sb = new StringBuffer();
-    sb.append(Vocabulary.word(this.getLHS()));
-    sb.append(" |||");
-    
-    int nt = 1;
-    for (int i = 0; i < getFrench().length; i++) {
-      if (getFrench()[i] < 0)
-        sb.append(" " + Vocabulary.word(getFrench()[i]).replaceFirst("\\]", String.format(",%d]", nt++)));
-      else
-        sb.append(" " + Vocabulary.word(getFrench()[i]));
-    }
-    sb.append(" |||");
-    nt = 1;
-    for (int i = 0; i < getEnglish().length; i++) {
-      if (getEnglish()[i] < 0)
-        sb.append(" " + Vocabulary.word(getEnglish()[i]).replaceFirst("\\]", String.format(",%d]", nt++)));
-      else
-        sb.append(" " + Vocabulary.word(getEnglish()[i]));
-    }
-    sb.append(" |||");
-    sb.append(" " + getFeatureString());
-    if (getAlignmentString() != null)
-      sb.append(" ||| " + getAlignmentString());
-    return sb.toString();
-  }
-
-  public String getFeatureString() {
-    return sparseFeatureStringSupplier.get();
+    return new StringBuffer()
+        .append(Vocabulary.word(this.getLHS()))
+        .append(fieldDelimiter)
+        .append(getSourceWords())
+        .append(fieldDelimiter)
+        .append(getTargetWords())
+        .append(fieldDelimiter)
+        .append(getFeatureVector().textFormat())
+        .append(fieldDelimiter)
+        .append(getAlignmentString())
+        .toString();
   }
 
   /**
    * Returns an alignment as a sequence of integers. The integers at positions i and i+1 are paired,
    * with position i indexing the source and i+1 the target.
    * 
-   * @return a byte[] from the {@link com.google.common.base.Supplier}
+   * @return a byte[]
    */
   public byte[] getAlignment() {
-    return this.alignmentSupplier.get();
+    return this.alignments;
   }
   
   public String getAlignmentString() {
-    return this.alignmentString;
+    byte[] alignments = getAlignment();
+    if (alignments == null || alignments.length == 0) {
+      return "";
+    }
+    final StringBuilder b = new StringBuilder();
+    for (int i = 0; i < alignments.length - 1; i+=2) {
+      b.append(alignments[i]).append("-").append(alignments[i+1]).append(" ");
+    }
+    return b.toString().trim();
   }
 
   /**
-   * The nonterminals on the English side are pointers to the source side nonterminals (-1 and -2),
+   * The nonterminals on the target side are pointers to the source side nonterminals (-1 and -2),
    * rather than being directly encoded. These number indicate the correspondence between the
    * nonterminals on each side, introducing a level of indirection however when we want to resolve
    * them. So to get the ID, we need to look up the corresponding source side ID.
    * 
-   * @return The string of English words
+   * @return The string of target words
    */
-  public String getEnglishWords() {
+  public String getTargetWords() {
     int[] foreignNTs = getForeignNonTerminals();
   
     StringBuilder sb = new StringBuilder();
-    for (Integer index : getEnglish()) {
+    for (Integer index : getTarget()) {
       if (index >= 0)
         sb.append(Vocabulary.word(index) + " ");
       else
@@ -482,23 +269,15 @@ public class Rule implements Comparator<Rule>, Comparable<Rule> {
     return sb.toString().trim();
   }
 
-  public boolean isTerminal() {
-    for (int i = 0; i < getEnglish().length; i++)
-      if (getEnglish()[i] < 0)
-        return false;
-  
-    return true;
-  }
-
   /**
-   * Return the French (source) nonterminals as list of Strings
+   * Return the source nonterminals as list of Strings
    * 
    * @return a list of strings
    */
   public int[] getForeignNonTerminals() {
     int[] nts = new int[getArity()];
     int index = 0;
-    for (int id : getFrench())
+    for (int id : getSource())
       if (id < 0)
         nts[index++] = -id;
     return nts;
@@ -512,8 +291,8 @@ public class Rule implements Comparator<Rule>, Comparable<Rule> {
   public int[] getNonTerminalSourcePositions() {
     int[] nonTerminalPositions = new int[getArity()];
     int ntPos = 0;
-    for (int sourceIdx = 0; sourceIdx < getFrench().length; sourceIdx++) {
-      if (getFrench()[sourceIdx] < 0)
+    for (int sourceIdx = 0; sourceIdx < getSource().length; sourceIdx++) {
+      if (getSource()[sourceIdx] < 0)
         nonTerminalPositions[ntPos++] = sourceIdx;
     }
     return nonTerminalPositions;
@@ -542,28 +321,28 @@ public class Rule implements Comparator<Rule>, Comparable<Rule> {
   }
 
   /**
-   * Return the English (target) nonterminals as list of Strings
+   * Return the target nonterminals as list of Strings
    * 
    * @return list of strings
    */
-  public int[] getEnglishNonTerminals() {
+  public int[] getTargetNonTerminals() {
     int[] nts = new int[getArity()];
     int[] foreignNTs = getForeignNonTerminals();
     int index = 0;
   
-    for (int i : getEnglish()) {
+    for (int i : getTarget()) {
       if (i < 0)
-        nts[index++] = foreignNTs[Math.abs(getEnglish()[i]) - 1];
+        nts[index++] = foreignNTs[Math.abs(getTarget()[i]) - 1];
     }
   
     return nts;
   }
 
-  private int[] getNormalizedEnglishNonterminalIndices() {
+  private int[] getNormalizedTargetNonterminalIndices() {
     int[] result = new int[getArity()];
   
     int ntIndex = 0;
-    for (Integer index : getEnglish()) {
+    for (Integer index : getTarget()) {
       if (index < 0)
         result[ntIndex++] = -index - 1;
     }
@@ -572,23 +351,21 @@ public class Rule implements Comparator<Rule>, Comparable<Rule> {
   }
 
   public boolean isInverting() {
-    int[] normalizedEnglishNonTerminalIndices = getNormalizedEnglishNonterminalIndices();
-    if (normalizedEnglishNonTerminalIndices.length == 2) {
-      if (normalizedEnglishNonTerminalIndices[0] == 1) {
+    int[] normalizedTargetNonTerminalIndices = getNormalizedTargetNonterminalIndices();
+    if (normalizedTargetNonTerminalIndices.length == 2) {
+      if (normalizedTargetNonTerminalIndices[0] == 1) {
         return true;
       }
     }
     return false;
   }
 
-  public String getFrenchWords() {
-    return Vocabulary.getWords(getFrench());
+  public String getSourceWords() {
+    return Vocabulary.getWords(getSource());
   }
 
-  public static final String NT_REGEX = "\\[[^\\]]+?\\]";
-
   private Pattern getPattern() {
-    String source = getFrenchWords();
+    String source = getSourceWords();
     String pattern = Pattern.quote(source);
     pattern = pattern.replaceAll(NT_REGEX, "\\\\E.+\\\\Q");
     pattern = pattern.replaceAll("\\\\Q\\\\E", "");
@@ -603,10 +380,7 @@ public class Rule implements Comparator<Rule>, Comparable<Rule> {
    * @return true if there is a match
    */
   public boolean matches(Sentence sentence) {
-    boolean match = getPattern().matcher(sentence.fullSource()).find();
-    // System.err.println(String.format("match(%s,%s) = %s", Pattern.quote(getFrenchWords()),
-    // sentence.annotatedSource(), match));
-    return match;
+    return getPattern().matcher(sentence.fullSource()).find();
   }
 
   /**
@@ -615,9 +389,7 @@ public class Rule implements Comparator<Rule>, Comparable<Rule> {
    */
   public static Comparator<Rule> EstimatedCostComparator = new Comparator<Rule>() {
     public int compare(Rule rule1, Rule rule2) {
-      float cost1 = rule1.getEstimatedCost();
-      float cost2 = rule2.getEstimatedCost();
-      return Float.compare(cost2,  cost1);
+      return Float.compare(rule1.getEstimatedCost(),  rule2.getEstimatedCost());
     }
   };
   
@@ -628,8 +400,4 @@ public class Rule implements Comparator<Rule>, Comparable<Rule> {
   public int compareTo(Rule other) {
     return EstimatedCostComparator.compare(this, other);
   }
-
-  public String getRuleString() {
-    return String.format("%s -> %s ||| %s", Vocabulary.word(getLHS()), getFrenchWords(), getEnglishWords());
-  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/main/java/org/apache/joshua/decoder/ff/tm/RuleFactory.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/tm/RuleFactory.java b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/tm/RuleFactory.java
new file mode 100644
index 0000000..18d3415
--- /dev/null
+++ b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/tm/RuleFactory.java
@@ -0,0 +1,88 @@
+/*
+ * 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.ff.tm;
+
+import static org.apache.joshua.decoder.ff.FeatureMap.hashFeature;
+import static org.apache.joshua.util.Constants.labeledFeatureSeparator;
+import static org.apache.joshua.util.Constants.spaceSeparator;
+
+import org.apache.joshua.decoder.ff.FeatureVector;
+
+/**
+ * Provides static functions to instantiate rules from its dependencies.
+ * @author fhieber
+ *
+ */
+public class RuleFactory {
+  
+  /**
+   * Parses an alignment string of the form '0-0 1-1 2-1'
+   * into a byte array {0, 0, 1, 1, 2, 1}.
+   * @param alignmentString a string of the form '0-0 1-1 2-1'
+   * @return byte[] containing alignment indices or null if alignmentString is null.
+   */
+  public static byte[] parseAlignmentString(String alignmentString) {
+    byte[] alignment = null;
+    if (alignmentString != null) {
+      String[] tokens = alignmentString.split("[-\\s]+");
+      if (tokens.length % 2 != 0) {
+        throw new RuntimeException(
+            String.format("Can not parse alignment string: '%s'", alignmentString));
+      }
+      alignment = new byte[tokens.length];
+      for (int i = 0; i < tokens.length; i++) {
+        alignment[i] = (byte) Short.parseShort(tokens[i]);
+      }
+    }
+    return alignment;
+  }
+  
+  /**
+   * Creates a {@link FeatureVector} from a string of the form '0.4 <name>=-1 ...'.
+   * This means, features can be either labeled or unlabeled.
+   * In the latter case the feature name will be an increasing index.
+   * Further, the feature names are prepended by the <owner> prefix before being hashed.
+   * @param featureString
+   * @param ownerId the owner id
+   * @return a {@link FeatureVector} with ids corresponding to tm-owned feature names.
+   */
+  public static FeatureVector parseFeatureString(final String featureString, final OwnerId ownerId) {
+    final String[] fields = featureString.split(spaceSeparator);
+    final FeatureVector result = new FeatureVector(fields.length);
+    int unlabeledFeatureIndex = 0;
+    String featureName;
+    float featureValue;
+
+    for (final String token : fields) {
+      final int splitIndex = token.indexOf(labeledFeatureSeparator);
+      final boolean isUnlabeledFeature = (splitIndex == -1);
+      if (isUnlabeledFeature) {
+        featureName = Integer.toString(unlabeledFeatureIndex);
+        featureValue = Float.parseFloat(token);
+        unlabeledFeatureIndex++;
+      } else {
+        featureName = token.substring(0, splitIndex);
+        featureValue = Float.parseFloat(token.substring(splitIndex + 1));
+      }
+      result.put(hashFeature(featureName, ownerId), featureValue);
+    }
+    return result;
+  }
+  
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/main/java/org/apache/joshua/decoder/ff/tm/format/HieroFormatReader.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/tm/format/HieroFormatReader.java b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/tm/format/HieroFormatReader.java
index 45c8c33..e68c11a 100644
--- a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/tm/format/HieroFormatReader.java
+++ b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/tm/format/HieroFormatReader.java
@@ -21,8 +21,11 @@ package org.apache.joshua.decoder.ff.tm.format;
 import java.io.IOException;
 
 import org.apache.joshua.corpus.Vocabulary;
+import org.apache.joshua.decoder.ff.FeatureVector;
 import org.apache.joshua.decoder.ff.tm.GrammarReader;
+import org.apache.joshua.decoder.ff.tm.OwnerId;
 import org.apache.joshua.decoder.ff.tm.Rule;
+import org.apache.joshua.decoder.ff.tm.RuleFactory;
 import org.apache.joshua.util.Constants;
 import org.apache.joshua.util.FormatUtils;
 
@@ -37,18 +40,18 @@ public class HieroFormatReader extends GrammarReader<Rule> {
   static {
     description = "Original Hiero format";
   }
-
-  public HieroFormatReader() {
-    super();
+  
+  public HieroFormatReader(OwnerId ownerId) {
+    super(ownerId);
   }
 
-  public HieroFormatReader(String grammarFile) throws IOException {
-    super(grammarFile);
+  public HieroFormatReader(String grammarFile, OwnerId ownerId) throws IOException {
+    super(grammarFile, ownerId);
   }
 
   @Override
   public Rule parseLine(String line) {
-    String[] fields = line.split(Constants.fieldDelimiter);
+    String[] fields = line.split(Constants.fieldDelimiterPattern);
     if (fields.length < 3) {
       throw new RuntimeException(String.format("Rule '%s' does not have four fields", line));
     }
@@ -94,10 +97,17 @@ public class HieroFormatReader extends GrammarReader<Rule> {
       }
     }
 
-    String sparse_features = (fields.length > 3 ? fields[3] : "");
-    String alignment = (fields.length > 4) ? fields[4] : null;
+    FeatureVector features = new FeatureVector(0);
+    if (fields.length > 3) {
+      features = RuleFactory.parseFeatureString(fields[3], owner);
+    }
+    
+    byte[] alignments = new byte[] {};
+    if (fields.length > 4) {
+      alignments = RuleFactory.parseAlignmentString(fields[4]);
+    }
 
-    return new Rule(lhs, sourceIDs, targetIDs, sparse_features, arity, alignment);
+    return new Rule(lhs, sourceIDs, targetIDs, arity, features, alignments, owner);
   }
   
   public static boolean isNonTerminal(final String word) {

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/main/java/org/apache/joshua/decoder/ff/tm/format/MosesFormatReader.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/tm/format/MosesFormatReader.java b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/tm/format/MosesFormatReader.java
index 7811b3b..959e607 100644
--- a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/tm/format/MosesFormatReader.java
+++ b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/tm/format/MosesFormatReader.java
@@ -21,10 +21,12 @@ package org.apache.joshua.decoder.ff.tm.format;
 import java.io.IOException;
 
 import org.apache.joshua.corpus.Vocabulary;
+import org.apache.joshua.decoder.ff.tm.OwnerId;
+import org.apache.joshua.decoder.ff.tm.OwnerMap;
 import org.apache.joshua.decoder.ff.tm.Rule;
-import org.apache.joshua.util.io.LineReader;
 import org.apache.joshua.util.Constants;
 import org.apache.joshua.util.FormatUtils;
+import org.apache.joshua.util.io.LineReader;
 
 /***
  * This class reads in the Moses phrase table format, with support for the source and target side,
@@ -46,13 +48,13 @@ import org.apache.joshua.util.FormatUtils;
 
 public class MosesFormatReader extends HieroFormatReader {
 
-  public MosesFormatReader(String grammarFile) throws IOException {
-    super(grammarFile);
+  public MosesFormatReader(String grammarFile, OwnerId ownerId) throws IOException {
+    super(grammarFile, ownerId);
     Vocabulary.id(Constants.defaultNT);
   }
   
-  public MosesFormatReader() {
-    super();
+  public MosesFormatReader(OwnerId ownerId) {
+    super(ownerId);
     Vocabulary.id(Constants.defaultNT);
   }
   
@@ -75,7 +77,7 @@ public class MosesFormatReader extends HieroFormatReader {
    */
   @Override
   public Rule parseLine(String line) {
-    String[] fields = line.split(Constants.fieldDelimiter);
+    String[] fields = line.split(Constants.fieldDelimiterPattern);
     
     String nt = FormatUtils.cleanNonTerminal(Constants.defaultNT);
     StringBuffer hieroLine = new StringBuffer(Constants.defaultNT + " ||| [" + nt + ",1] " + fields[0] + " ||| [" + nt + ",1] " + fields[1] + " |||");
@@ -89,7 +91,7 @@ public class MosesFormatReader extends HieroFormatReader {
     // alignments
     if (fields.length >= 4)
       hieroLine.append(" ||| " + fields[3]);
-
+    
     return super.parseLine(hieroLine.toString());
   }
   
@@ -99,7 +101,7 @@ public class MosesFormatReader extends HieroFormatReader {
    * @param args the command-line arguments
    */
   public static void main(String[] args) {
-    MosesFormatReader reader = new MosesFormatReader();
+    MosesFormatReader reader = new MosesFormatReader(OwnerMap.UNKNOWN_OWNER_ID);
     for (String line: new LineReader(System.in)) {
       Rule rule = reader.parseLine(line);
       System.out.println(rule.textFormat());

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/main/java/org/apache/joshua/decoder/ff/tm/hash_based/MemoryBasedBatchGrammar.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/tm/hash_based/MemoryBasedBatchGrammar.java b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/tm/hash_based/MemoryBasedBatchGrammar.java
index 234fe0f..0bc577a 100644
--- a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/tm/hash_based/MemoryBasedBatchGrammar.java
+++ b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/tm/hash_based/MemoryBasedBatchGrammar.java
@@ -18,6 +18,8 @@
  */
 package org.apache.joshua.decoder.ff.tm.hash_based;
 
+import static org.apache.joshua.decoder.ff.tm.GrammarReader.createReader;
+
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashMap;
@@ -27,13 +29,11 @@ import org.apache.joshua.corpus.Vocabulary;
 import org.apache.joshua.decoder.JoshuaConfiguration;
 import org.apache.joshua.decoder.JoshuaConfiguration.OOVItem;
 import org.apache.joshua.decoder.ff.FeatureFunction;
+import org.apache.joshua.decoder.ff.FeatureVector;
 import org.apache.joshua.decoder.ff.tm.AbstractGrammar;
-import org.apache.joshua.decoder.ff.tm.OwnerMap;
 import org.apache.joshua.decoder.ff.tm.Rule;
-import org.apache.joshua.decoder.ff.tm.GrammarReader;
 import org.apache.joshua.decoder.ff.tm.Trie;
 import org.apache.joshua.decoder.ff.tm.format.HieroFormatReader;
-import org.apache.joshua.decoder.ff.tm.format.MosesFormatReader;
 import org.apache.joshua.util.FormatUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -58,16 +58,12 @@ public class MemoryBasedBatchGrammar extends AbstractGrammar {
   /* The number of distinct source sides. */
   private int qtyRuleBins = 0;
 
-  private int numDenseFeatures = 0;
-
   /* The trie root. */
   private MemoryBasedTrie root = new MemoryBasedTrie();
 
   /* The file containing the grammar. */
   private String grammarFile;
 
-  private GrammarReader<Rule> modelReader;
-
   /**
    * Constructor used by Decoder mostly. Default spanLimit of 20
    * @param owner the associated decoder-wide {@link org.apache.joshua.decoder.ff.tm.OwnerMap}
@@ -78,18 +74,6 @@ public class MemoryBasedBatchGrammar extends AbstractGrammar {
     super(owner, config, spanLimit);
   }
 
-  /**
-   * Constructor to initialize a GrammarReader (unowned)
-   * @param reader the GrammarReader used for storing ASCII line-based grammars on disk.
-   * @param config a {@link org.apache.joshua.decoder.JoshuaConfiguration} object
-   * @param spanLimit the maximum span of the input grammar rule(s) can be applied to.
-   */
-  public MemoryBasedBatchGrammar(
-      final GrammarReader<Rule> reader, final JoshuaConfiguration config, final int spanLimit) {
-    super(OwnerMap.UNKNOWN_OWNER, config, spanLimit);
-    modelReader = reader;
-  }
-
   public MemoryBasedBatchGrammar(String formatKeyword, String grammarFile, String owner,
       String defaultLHSSymbol, int spanLimit, JoshuaConfiguration joshuaConfiguration)
       throws IOException {
@@ -99,34 +83,14 @@ public class MemoryBasedBatchGrammar extends AbstractGrammar {
     this.grammarFile = grammarFile;
 
     // ==== loading grammar
-    this.modelReader = createReader(formatKeyword, grammarFile);
-    if (modelReader != null) {
-      for (Rule rule : modelReader)
-        if (rule != null) {
-          addRule(rule);
-        }
-    } else {
-      LOG.info("Couldn't create a GrammarReader for file {} with format {}",
-          grammarFile, formatKeyword);
-    }
+    for (Rule rule : createReader(formatKeyword, grammarFile, getOwner()))
+      if (rule != null) {
+        addRule(rule);
+      }
 
     this.printGrammar();
   }
 
-  protected GrammarReader<Rule> createReader(String format, String grammarFile) throws IOException {
-
-    if (grammarFile != null) {
-      if ("hiero".equals(format) || "thrax".equals(format)) {
-        return new HieroFormatReader(grammarFile);
-      } else if ("moses".equals(format)) {
-        return new MosesFormatReader(grammarFile);
-      } else {
-        throw new RuntimeException(String.format("* FATAL: unknown grammar format '%s'", format));
-      }
-    }
-    return null;
-  }
-
   // ===============================================================
   // Methods
   // ===============================================================
@@ -160,14 +124,9 @@ public class MemoryBasedBatchGrammar extends AbstractGrammar {
 
     this.qtyRulesRead++;
 
-    rule.setOwner(owner);
-
-    if (numDenseFeatures == 0)
-      numDenseFeatures = rule.getFeatureVector().getDenseFeatures().size();
-
     // === identify the position, and insert the trie nodes as necessary
     MemoryBasedTrie pos = root;
-    int[] french = rule.getFrench();
+    int[] french = rule.getSource();
 
     maxSourcePhraseLength = Math.max(maxSourcePhraseLength, french.length);
 
@@ -195,7 +154,7 @@ public class MemoryBasedBatchGrammar extends AbstractGrammar {
 
     // === add the rule into the trie node
     if (!pos.hasRules()) {
-      pos.ruleBin = new MemoryBasedRuleBin(rule.getArity(), rule.getFrench());
+      pos.ruleBin = new MemoryBasedRuleBin(rule.getArity(), rule.getSource());
       this.qtyRuleBins++;
     }
     pos.ruleBin.addRule(rule);
@@ -221,22 +180,39 @@ public class MemoryBasedBatchGrammar extends AbstractGrammar {
     final int targetWord = this.joshuaConfiguration.mark_oovs ? Vocabulary.id(Vocabulary
         .word(sourceWord) + "_OOV") : sourceWord;
 
-    int[] sourceWords = { sourceWord };
-    int[] targetWords = { targetWord };
-    final String oovAlignment = "0-0";
+    final int[] sourceWords = { sourceWord };
+    final int[] targetWords = { targetWord };
+    final byte[] alignment = { 0, 0 };
+    final FeatureVector features = new FeatureVector(0);
 
     if (this.joshuaConfiguration.oovList != null && this.joshuaConfiguration.oovList.size() != 0) {
+      
       for (OOVItem item : this.joshuaConfiguration.oovList) {
-        Rule oovRule = new Rule(Vocabulary.id(item.label), sourceWords, targetWords, "", 0,
-            oovAlignment);
+        final Rule oovRule = new Rule(
+            Vocabulary.id(item.label),
+            sourceWords,
+            targetWords,
+            0,
+            features,
+            alignment,
+            getOwner());
         addRule(oovRule);
         oovRule.estimateRuleCost(featureFunctions);
       }
+      
     } else {
-      int nt_i = Vocabulary.id(this.joshuaConfiguration.default_non_terminal);
-      Rule oovRule = new Rule(nt_i, sourceWords, targetWords, "", 0, oovAlignment);
+      
+      final Rule oovRule = new Rule(
+          Vocabulary.id(this.joshuaConfiguration.default_non_terminal),
+          sourceWords,
+          targetWords,
+          0,
+          features,
+          alignment,
+          getOwner());
       addRule(oovRule);
       oovRule.estimateRuleCost(featureFunctions);
+      
     }
   }
 
@@ -246,7 +222,7 @@ public class MemoryBasedBatchGrammar extends AbstractGrammar {
    * @param featureFunctions an {@link java.util.ArrayList} of {@link org.apache.joshua.decoder.ff.FeatureFunction}'s
    */
   public void addGlueRules(ArrayList<FeatureFunction> featureFunctions) {
-    HieroFormatReader reader = new HieroFormatReader();
+    final HieroFormatReader reader = new HieroFormatReader(getOwner());
 
     String goalNT = FormatUtils.cleanNonTerminal(joshuaConfiguration.goal_symbol);
     String defaultNT = FormatUtils.cleanNonTerminal(joshuaConfiguration.default_non_terminal);
@@ -266,8 +242,4 @@ public class MemoryBasedBatchGrammar extends AbstractGrammar {
     }
   }
 
-  @Override
-  public int getNumDenseFeatures() {
-    return numDenseFeatures;
-  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/main/java/org/apache/joshua/decoder/ff/tm/hash_based/MemoryBasedRuleBin.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/tm/hash_based/MemoryBasedRuleBin.java b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/tm/hash_based/MemoryBasedRuleBin.java
index f91df1e..8daeb21 100644
--- a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/tm/hash_based/MemoryBasedRuleBin.java
+++ b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/tm/hash_based/MemoryBasedRuleBin.java
@@ -18,6 +18,8 @@
  */
 package org.apache.joshua.decoder.ff.tm.hash_based;
 
+import java.util.Arrays;
+
 import org.apache.joshua.decoder.ff.tm.BasicRuleCollection;
 import org.apache.joshua.decoder.ff.tm.Rule;
 
@@ -47,13 +49,13 @@ public class MemoryBasedRuleBin extends BasicRuleCollection {
     // XXX This if clause seems bogus.
     if (rules.size() <= 0) { // first time
       this.arity = rule.getArity();
-      this.sourceTokens = rule.getFrench();
+      this.sourceTokens = rule.getSource();
     }
     if (rule.getArity() != this.arity) {
       return;
     }
     rules.add(rule);
     sorted = false;
-    rule.setFrench(this.sourceTokens);
+    assert(Arrays.equals(rule.getSource(), this.sourceTokens));
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/main/java/org/apache/joshua/decoder/ff/tm/packed/PackedGrammar.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/tm/packed/PackedGrammar.java b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/tm/packed/PackedGrammar.java
index 37bffb7..23e64a1 100644
--- a/joshua-core/src/main/java/org/apache/joshua/decoder/ff/tm/packed/PackedGrammar.java
+++ b/joshua-core/src/main/java/org/apache/joshua/decoder/ff/tm/packed/PackedGrammar.java
@@ -55,6 +55,8 @@ package org.apache.joshua.decoder.ff.tm.packed;
  */
 
 import static java.util.Collections.sort;
+import static org.apache.joshua.decoder.ff.FeatureMap.getFeature;
+import static org.apache.joshua.decoder.ff.FeatureMap.hashFeature;
 
 import java.io.File;
 import java.io.FileInputStream;
@@ -94,15 +96,14 @@ import org.apache.joshua.util.FormatUtils;
 import org.apache.joshua.util.encoding.EncoderConfiguration;
 import org.apache.joshua.util.encoding.FloatEncoder;
 import org.apache.joshua.util.io.LineReader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Supplier;
 import com.google.common.base.Suppliers;
 import com.google.common.cache.Cache;
 import com.google.common.cache.CacheBuilder;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 public class PackedGrammar extends AbstractGrammar {
 
   private static final Logger LOG = LoggerFactory.getLogger(PackedGrammar.class);
@@ -183,11 +184,6 @@ public class PackedGrammar extends AbstractGrammar {
     return num_rules;
   }
 
-  @Override
-  public int getNumDenseFeatures() {
-    return encoding.getNumDenseFeatures();
-  }
-
   /**
    * Computes the MD5 checksum of the vocabulary file.
    * Can be used for comparing vocabularies across multiple packedGrammars.
@@ -344,7 +340,6 @@ public class PackedGrammar extends AbstractGrammar {
     private final int[] targetLookup;
     private int featureSize;
     private float[] estimated;
-    private float[] precomputable;
 
     private final static int BUFFER_HEADER_POSITION = 8;
 
@@ -386,9 +381,7 @@ public class PackedGrammar extends AbstractGrammar {
     private void initializeFeatureStructures() {
       int num_blocks = features.getInt(0);
       estimated = new float[num_blocks];
-      precomputable = new float[num_blocks];
       Arrays.fill(estimated, Float.NEGATIVE_INFINITY);
-      Arrays.fill(precomputable, Float.NEGATIVE_INFINITY);
       featureSize = features.getInt(4);
     }
 
@@ -430,7 +423,7 @@ public class PackedGrammar extends AbstractGrammar {
       }
     }
 
-    private final int[] getTarget(int pointer) {
+    private final int[] getTargetArray(int pointer) {
       // Figure out level.
       int tgt_length = 1;
       while (tgt_length < (targetLookup.length + 1) && targetLookup[tgt_length] <= pointer)
@@ -468,34 +461,27 @@ public class PackedGrammar extends AbstractGrammar {
 
     /**
      * Returns the FeatureVector associated with a rule (represented as a block ID).
-     * These features are in the form "feature1=value feature2=value...". By default, unlabeled
-     * features are named using the pattern.
+     * The feature ids are hashed corresponding to feature names prepended with the owner string:
+     * i.e. '0' becomes '<owner>_0'.
      * @param block_id
      * @return feature vector
      */
-
-    private final FeatureVector loadFeatureVector(int block_id) {
+    private final FeatureVector loadFeatureVector(int block_id, OwnerId ownerId) {
       int featurePosition = getIntFromByteBuffer(block_id, features);
       final int numFeatures = encoding.readId(features, featurePosition);
 
       featurePosition += EncoderConfiguration.ID_SIZE;
-      final FeatureVector featureVector = new FeatureVector();
+      final FeatureVector featureVector = new FeatureVector(encoding.getNumDenseFeatures());
       FloatEncoder encoder;
-      String featureName;
 
       for (int i = 0; i < numFeatures; i++) {
         final int innerId = encoding.readId(features, featurePosition);
-        final int outerId = encoding.outerId(innerId);
         encoder = encoding.encoder(innerId);
-        // TODO (fhieber): why on earth are dense feature ids (ints) encoded in the vocabulary?
-        featureName = Vocabulary.word(outerId);
+        final int outerId = encoding.outerId(innerId);
+        final int ownedFeatureId = hashFeature(getFeature(outerId), ownerId);
         final float value = encoder.read(features, featurePosition);
-        try {
-          int index = Integer.parseInt(featureName);
-          featureVector.increment(index, -value);
-        } catch (NumberFormatException e) {
-          featureVector.increment(featureName, value);
-        }
+        
+        featureVector.add(ownedFeatureId, value);
         featurePosition += EncoderConfiguration.ID_SIZE + encoder.size();
       }
       
@@ -668,7 +654,10 @@ public class PackedGrammar extends AbstractGrammar {
         return sorted;
       }
 
-      private synchronized void sortRules(List<FeatureFunction> models) {
+      /**
+       * Estimates rule costs for all rules at this trie node.
+       */
+      private synchronized void sortRules(List<FeatureFunction> featureFunctions) {
         int num_children = source[position];
         int rule_position = position + 2 * (num_children + 1);
         int num_rules = source[rule_position - 1];
@@ -676,19 +665,24 @@ public class PackedGrammar extends AbstractGrammar {
           this.sorted = true;
           return;
         }
-        Integer[] rules = new Integer[num_rules];
+        final Integer[] rules = new Integer[num_rules];
 
-        int target_address;
         int block_id;
+        int lhs;
+        int[] target;
+        byte[] alignments;
+        FeatureVector features;
+        
         for (int i = 0; i < num_rules; ++i) {
-          target_address = source[rule_position + 1 + 3 * i];
+          // we construct very short-lived rule objects for sorting
           rules[i] = rule_position + 2 + 3 * i;
           block_id = source[rules[i]];
-
-          Rule rule = new Rule(source[rule_position + 3 * i], src,
-              getTarget(target_address), loadFeatureVector(block_id), arity, owner);
-          estimated[block_id] = rule.estimateRuleCost(models);
-          precomputable[block_id] = rule.getPrecomputableCost();
+          lhs = source[rule_position + 3 * i];
+          target = getTargetArray(source[rule_position + 1 + 3 * i]);
+          features = loadFeatureVector(block_id, owner);
+          alignments = getAlignmentArray(block_id);
+          final Rule rule = new Rule(lhs, src, target, arity, features, alignments, owner);
+          estimated[block_id] = rule.estimateRuleCost(featureFunctions);
         }
 
         Arrays.sort(rules, new Comparator<Integer>() {
@@ -803,12 +797,12 @@ public class PackedGrammar extends AbstractGrammar {
        */
       public final class PackedPhrasePair extends PackedRule {
 
-        private final Supplier<int[]> englishSupplier;
+        private final Supplier<int[]> targetSupplier;
         private final Supplier<byte[]> alignmentSupplier;
 
         public PackedPhrasePair(int address) {
           super(address);
-          englishSupplier = initializeEnglishSupplier();
+          targetSupplier = initializeTargetSupplier();
           alignmentSupplier = initializeAlignmentSupplier();
         }
 
@@ -826,9 +820,9 @@ public class PackedGrammar extends AbstractGrammar {
          * This means this implementation should be as thread-safe and performant as possible.
          */
 
-        private Supplier<int[]> initializeEnglishSupplier(){
+        private Supplier<int[]> initializeTargetSupplier(){
           Supplier<int[]> result = Suppliers.memoize(() ->{
-            int[] phrase = getTarget(source[address + 1]);
+            int[] phrase = getTargetArray(source[address + 1]);
             int[] tgt = new int[phrase.length + 1];
             tgt[0] = -1;
             for (int i = 0; i < phrase.length; i++)
@@ -851,22 +845,22 @@ public class PackedGrammar extends AbstractGrammar {
         }
 
         /**
-         * Take the English phrase of the underlying rule and prepend an [X].
+         * Take the target phrase of the underlying rule and prepend an [X].
          * 
          * @return the augmented phrase
          */
         @Override
-        public int[] getEnglish() {
-          return this.englishSupplier.get();
+        public int[] getTarget() {
+          return this.targetSupplier.get();
         }
         
         /**
-         * Take the French phrase of the underlying rule and prepend an [X].
+         * Take the source phrase of the underlying rule and prepend an [X].
          * 
-         * @return the augmented French phrase
+         * @return the augmented source phrase
          */
         @Override
-        public int[] getFrench() {
+        public int[] getSource() {
           int phrase[] = new int[src.length + 1];
           int ntid = Vocabulary.id(PackedGrammar.this.joshuaConfiguration.default_non_terminal);
           phrase[0] = ntid;
@@ -892,27 +886,28 @@ public class PackedGrammar extends AbstractGrammar {
 
       public class PackedRule extends Rule {
         protected final int address;
-        private final Supplier<int[]> englishSupplier;
+        private final Supplier<int[]> targetSupplier;
         private final Supplier<FeatureVector> featureVectorSupplier;
         private final Supplier<byte[]> alignmentsSupplier;
 
         public PackedRule(int address) {
+          super(source[address], src, null, PackedTrie.this.getArity(), null, null, owner);
           this.address = address;
-          this.englishSupplier = intializeEnglishSupplier();
+          this.targetSupplier = intializeTargetSupplier();
           this.featureVectorSupplier = initializeFeatureVectorSupplier();
           this.alignmentsSupplier = initializeAlignmentsSupplier();
         }
 
-        private Supplier<int[]> intializeEnglishSupplier(){
+        private Supplier<int[]> intializeTargetSupplier(){
           Supplier<int[]> result = Suppliers.memoize(() ->{
-            return getTarget(source[address + 1]);
+            return getTargetArray(source[address + 1]);
           });
           return result;
         }
 
         private Supplier<FeatureVector> initializeFeatureVectorSupplier(){
           Supplier<FeatureVector> result = Suppliers.memoize(() ->{
-            return loadFeatureVector(source[address + 2]);
+            return loadFeatureVector(source[address + 2], owner);
          });
           return result;
         }
@@ -929,47 +924,22 @@ public class PackedGrammar extends AbstractGrammar {
         }
 
         @Override
-        public void setArity(int arity) {
-        }
-
-        @Override
         public int getArity() {
           return PackedTrie.this.getArity();
         }
 
         @Override
-        public void setOwner(OwnerId owner) {
-        }
-
-        @Override
-        public OwnerId getOwner() {
-          return owner;
-        }
-
-        @Override
-        public void setLHS(int lhs) {
-        }
-
-        @Override
         public int getLHS() {
           return source[address];
         }
 
         @Override
-        public void setEnglish(int[] eng) {
-        }
-
-        @Override
-        public int[] getEnglish() {
-          return this.englishSupplier.get();
-        }
-
-        @Override
-        public void setFrench(int[] french) {
+        public int[] getTarget() {
+          return this.targetSupplier.get();
         }
 
         @Override
-        public int[] getFrench() {
+        public int[] getSource() {
           return src;
         }
 
@@ -982,45 +952,16 @@ public class PackedGrammar extends AbstractGrammar {
         public byte[] getAlignment() {
           return this.alignmentsSupplier.get();
         }
-        
-        @Override
-        public String getAlignmentString() {
-            throw new RuntimeException("AlignmentString not implemented for PackedRule!");
-        }
 
         @Override
         public float getEstimatedCost() {
           return estimated[source[address + 2]];
         }
 
-//        @Override
-//        public void setPrecomputableCost(float cost) {
-//          precomputable[source[address + 2]] = cost;
-//        }
-
-        @Override
-        public float getPrecomputableCost() {
-          return precomputable[source[address + 2]];
-        }
-
         @Override
         public float estimateRuleCost(List<FeatureFunction> models) {
           return estimated[source[address + 2]];
         }
-
-        @Override
-        public String toString() {
-          StringBuffer sb = new StringBuffer();
-          sb.append(Vocabulary.word(this.getLHS()));
-          sb.append(" ||| ");
-          sb.append(getFrenchWords());
-          sb.append(" ||| ");
-          sb.append(getEnglishWords());
-          sb.append(" |||");
-          sb.append(" " + getFeatureVector());
-          sb.append(String.format(" ||| %.3f", getEstimatedCost()));
-          return sb.toString();
-        }
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/20afddf8/joshua-core/src/main/java/org/apache/joshua/decoder/hypergraph/FeatureVectorExtractor.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/decoder/hypergraph/FeatureVectorExtractor.java b/joshua-core/src/main/java/org/apache/joshua/decoder/hypergraph/FeatureVectorExtractor.java
index a8525be..4edd466 100644
--- a/joshua-core/src/main/java/org/apache/joshua/decoder/hypergraph/FeatureVectorExtractor.java
+++ b/joshua-core/src/main/java/org/apache/joshua/decoder/hypergraph/FeatureVectorExtractor.java
@@ -43,7 +43,7 @@ public class FeatureVectorExtractor implements WalkerFunction, DerivationVisitor
   public FeatureVectorExtractor(
       final List<FeatureFunction> featureFunctions,
       final Sentence sourceSentence) {
-    this.features = new FeatureVector();
+    this.features = new FeatureVector(10);
     this.featureFunctions = featureFunctions;
     this.sourceSentence = sourceSentence;
   }
@@ -51,7 +51,7 @@ public class FeatureVectorExtractor implements WalkerFunction, DerivationVisitor
   /** Accumulate edge features from Viterbi path */
   @Override
   public void apply(HGNode node, int nodeIndex) {
-    features.add(
+    features.addInPlace(
         computeTransitionFeatures(
           featureFunctions,
           node.bestHyperedge,
@@ -62,7 +62,7 @@ public class FeatureVectorExtractor implements WalkerFunction, DerivationVisitor
   /** Accumulate edge features for that DerivationState */
   @Override
   public void before(DerivationState state, int level, int tailNodeIndex) {
-    features.add(
+    features.addInPlace(
         computeTransitionFeatures(
           featureFunctions,
           state.edge,