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:47 UTC

[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

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,