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/09/17 12:27:56 UTC

[07/14] incubator-joshua git commit: Joshua 7 configuration system

Joshua 7 configuration system


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

Branch: refs/heads/7_confsystem
Commit: f2edda0f14967f414f2550989cedf9a281536193
Parents: f7513ab
Author: Felix Hieber <fh...@amazon.com>
Authored: Wed Sep 14 20:03:51 2016 +0200
Committer: Hieber, Felix <fh...@amazon.de>
Committed: Thu Sep 15 17:34:00 2016 +0200

----------------------------------------------------------------------
 .../java/org/apache/joshua/adagrad/AdaGrad.java |   5 +-
 .../org/apache/joshua/adagrad/AdaGradCore.java  |  20 +-
 .../java/org/apache/joshua/decoder/Decoder.conf | 126 ++++
 .../java/org/apache/joshua/decoder/Decoder.java | 590 +++++++--------
 .../apache/joshua/decoder/DecoderConfig.java    | 123 ++++
 .../org/apache/joshua/decoder/DecoderTask.java  | 164 ++---
 .../org/apache/joshua/decoder/InputType.java    |   4 +
 .../joshua/decoder/JoshuaConfiguration.java     | 738 -------------------
 .../apache/joshua/decoder/JoshuaDecoder.java    | 160 ++--
 .../java/org/apache/joshua/decoder/OOVItem.java |  50 ++
 .../apache/joshua/decoder/SearchAlgorithm.java  |   7 +
 .../org/apache/joshua/decoder/ServerType.java   |   4 +
 .../joshua/decoder/StructuredTranslation.java   |   3 +-
 .../org/apache/joshua/decoder/Translation.java  |  50 +-
 .../joshua/decoder/TranslationRequest.java      |  24 +
 .../joshua/decoder/chart_parser/Chart.java      | 138 ++--
 .../decoder/chart_parser/ComputeNodeResult.java |  22 +-
 .../apache/joshua/decoder/ff/Accumulator.java   |  29 +
 .../joshua/decoder/ff/ArityPhrasePenalty.java   |  13 +-
 .../joshua/decoder/ff/FeatureAccumulator.java   |  36 +
 .../joshua/decoder/ff/FeatureFunction.java      | 131 +---
 .../joshua/decoder/ff/LabelCombinationFF.java   |   9 +-
 .../joshua/decoder/ff/LabelSubstitutionFF.java  |   7 +-
 .../joshua/decoder/ff/LexicalFeatures.java      |  22 +-
 .../apache/joshua/decoder/ff/OOVPenalty.java    |  35 +-
 .../apache/joshua/decoder/ff/PhraseModel.java   |  10 +-
 .../apache/joshua/decoder/ff/PhrasePenalty.java |  11 +-
 .../org/apache/joshua/decoder/ff/RuleFF.java    |  20 +-
 .../apache/joshua/decoder/ff/RuleLength.java    |   7 +-
 .../org/apache/joshua/decoder/ff/RuleShape.java |   7 +-
 .../joshua/decoder/ff/ScoreAccumulator.java     |  38 +
 .../apache/joshua/decoder/ff/SourcePathFF.java  |   7 +-
 .../apache/joshua/decoder/ff/StatefulFF.java    |   9 +-
 .../apache/joshua/decoder/ff/StatelessFF.java   |   7 +-
 .../apache/joshua/decoder/ff/TargetBigram.java  |  19 +-
 .../apache/joshua/decoder/ff/WordPenalty.java   |  17 +-
 .../decoder/ff/fragmentlm/FragmentLMFF.java     |  18 +-
 .../joshua/decoder/ff/lm/LanguageModelFF.java   |  31 +-
 .../ff/lm/StateMinimizingLanguageModel.java     |  22 +-
 .../joshua/decoder/ff/phrase/Distortion.java    |  14 +-
 .../joshua/decoder/ff/tm/AbstractGrammar.java   |  68 +-
 .../joshua/decoder/ff/tm/CreateGlueGrammar.java |   3 +-
 .../apache/joshua/decoder/ff/tm/Grammar.java    |   6 +-
 .../joshua/decoder/ff/tm/GrammarReader.java     |   3 +-
 .../decoder/ff/tm/SentenceFilteredGrammar.java  | 366 ---------
 .../decoder/ff/tm/format/MosesFormatReader.java |   1 -
 .../tm/hash_based/MemoryBasedBatchGrammar.java  | 248 -------
 .../decoder/ff/tm/hash_based/TextGrammar.java   | 216 ++++++
 .../ff/tm/hash_based/TextGrammarFactory.java    | 148 ++++
 .../decoder/ff/tm/packed/PackedGrammar.java     |  91 +--
 .../GrammarBuilderWalkerFunction.java           |  11 +-
 .../decoder/hypergraph/KBestExtractor.java      |  45 +-
 .../decoder/io/TranslationRequestStream.java    |  20 +-
 .../apache/joshua/decoder/phrase/Candidate.java |  19 +-
 .../joshua/decoder/phrase/PhraseChart.java      |  23 +-
 .../joshua/decoder/phrase/PhraseTable.java      |  53 +-
 .../org/apache/joshua/decoder/phrase/Stack.java |  11 +-
 .../apache/joshua/decoder/phrase/Stacks.java    |  61 +-
 .../decoder/segment_file/ParseTreeInput.java    |   6 +-
 .../decoder/segment_file/ParsedSentence.java    |   7 +-
 .../joshua/decoder/segment_file/Sentence.java   |  36 +-
 .../joshua/decoder/segment_file/Token.java      |   9 +-
 .../java/org/apache/joshua/lattice/Lattice.java |  21 +-
 .../main/java/org/apache/joshua/mira/MIRA.java  |   5 +-
 .../java/org/apache/joshua/mira/MIRACore.java   |  19 +-
 .../joshua/oracle/OracleExtractionHG.java       | 193 +++--
 .../main/java/org/apache/joshua/pro/PRO.java    |   5 +-
 .../java/org/apache/joshua/pro/PROCore.java     |  19 +-
 .../org/apache/joshua/server/ServerThread.java  |  29 +-
 .../org/apache/joshua/server/TcpServer.java     |  11 +-
 .../java/org/apache/joshua/util/Constants.java  |   6 +
 .../apache/joshua/util/PackedGrammarServer.java |  14 +-
 .../org/apache/joshua/util/io/LineReader.java   |   4 +-
 .../java/org/apache/joshua/zmert/MertCore.java  |  22 +-
 .../java/org/apache/joshua/zmert/ZMERT.java     |   5 +-
 .../apache/joshua/decoder/DecoderServlet.java   |   2 +-
 .../decoder/DecoderServletContextListener.java  |   8 +-
 77 files changed, 1885 insertions(+), 2676 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/f2edda0f/joshua-core/src/main/java/org/apache/joshua/adagrad/AdaGrad.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/adagrad/AdaGrad.java b/joshua-core/src/main/java/org/apache/joshua/adagrad/AdaGrad.java
index 0784318..15b8b88 100755
--- a/joshua-core/src/main/java/org/apache/joshua/adagrad/AdaGrad.java
+++ b/joshua-core/src/main/java/org/apache/joshua/adagrad/AdaGrad.java
@@ -18,13 +18,12 @@
  */
 package org.apache.joshua.adagrad;
 
-import org.apache.joshua.decoder.JoshuaConfiguration;
+import org.apache.joshua.decoder.Decoder;
 import org.apache.joshua.util.FileUtility;
 import org.apache.joshua.util.StreamGobbler;
 
 public class AdaGrad {
   public static void main(String[] args) throws Exception {
-    JoshuaConfiguration joshuaConfiguration = new JoshuaConfiguration();
     boolean external = false; // should each AdaGrad iteration be launched externally?
 
     if (args.length == 1) {
@@ -42,7 +41,7 @@ public class AdaGrad {
     }
 
     if (!external) {
-      AdaGradCore myAdaGrad = new AdaGradCore(args[0], joshuaConfiguration);
+      AdaGradCore myAdaGrad = new AdaGradCore(args[0], Decoder.getDefaultFlags());
       myAdaGrad.run_AdaGrad(); // optimize lambda[]
       myAdaGrad.finish();
     } else {

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/f2edda0f/joshua-core/src/main/java/org/apache/joshua/adagrad/AdaGradCore.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/adagrad/AdaGradCore.java b/joshua-core/src/main/java/org/apache/joshua/adagrad/AdaGradCore.java
index b21ab71..4fe7e07 100755
--- a/joshua-core/src/main/java/org/apache/joshua/adagrad/AdaGradCore.java
+++ b/joshua-core/src/main/java/org/apache/joshua/adagrad/AdaGradCore.java
@@ -47,13 +47,14 @@ import java.util.zip.GZIPOutputStream;
 
 import org.apache.joshua.corpus.Vocabulary;
 import org.apache.joshua.decoder.Decoder;
-import org.apache.joshua.decoder.JoshuaConfiguration;
 import org.apache.joshua.metrics.EvaluationMetric;
 import org.apache.joshua.util.StreamGobbler;
 import org.apache.joshua.util.io.ExistingUTF8EncodedTextFile;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.typesafe.config.Config;
+
 /**
  * This code was originally written by Yuan Cao, who copied the MERT code to produce this file.
  */
@@ -66,8 +67,6 @@ public class AdaGradCore {
   private final static double epsilon = 1.0 / 1000000;
   private final static DecimalFormat f4 = new DecimalFormat("###0.0000");
 
-  private final JoshuaConfiguration joshuaConfiguration;
-
   private TreeSet<Integer>[] indicesOfInterest_all;
 
   private int verbosity; // anything of priority <= verbosity will be printed
@@ -249,6 +248,7 @@ public class AdaGradCore {
   private String sourceFileName, refFileName, decoderOutFileName;
   private String decoderConfigFileName, decoderCommandFileName;
   private String fakeFileNameTemplate, fakeFileNamePrefix, fakeFileNameSuffix;
+  private Config config;
 
   // e.g. output.it[1-x].someOldRun would be specified as:
   // output.it?.someOldRun
@@ -256,19 +256,19 @@ public class AdaGradCore {
 
   // private int useDisk;
 
-  public AdaGradCore(JoshuaConfiguration joshuaConfiguration) {
-    this.joshuaConfiguration = joshuaConfiguration;
+  public AdaGradCore(Config config) {
+    this.config = config;
   }
 
-  public AdaGradCore(String[] args, JoshuaConfiguration joshuaConfiguration) throws FileNotFoundException, IOException {
-    this.joshuaConfiguration = joshuaConfiguration;
+  public AdaGradCore(String[] args, Config config) throws FileNotFoundException, IOException {
+    this.config = config;
     EvaluationMetric.set_knownMetrics();
     processArgsArray(args);
     initialize(0);
   }
 
-  public AdaGradCore(String configFileName, JoshuaConfiguration joshuaConfiguration) throws FileNotFoundException, IOException {
-    this.joshuaConfiguration = joshuaConfiguration;
+  public AdaGradCore(String configFileName, Config config) throws FileNotFoundException, IOException {
+    this.config = config;
     EvaluationMetric.set_knownMetrics();
     processArgsArray(cfgFileToArgsArray(configFileName));
     initialize(0);
@@ -480,7 +480,7 @@ public class AdaGradCore {
     // by default, load joshua decoder
     if (decoderCommand == null && fakeFileNameTemplate == null) {
       println("Loading Joshua decoder...", 1);
-      myDecoder = new Decoder(joshuaConfiguration);
+      myDecoder = new Decoder(config);
       println("...finished loading @ " + (new Date()), 1);
       println("");
     } else {

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/f2edda0f/joshua-core/src/main/java/org/apache/joshua/decoder/Decoder.conf
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/decoder/Decoder.conf b/joshua-core/src/main/java/org/apache/joshua/decoder/Decoder.conf
new file mode 100644
index 0000000..3b86fb6
--- /dev/null
+++ b/joshua-core/src/main/java/org/apache/joshua/decoder/Decoder.conf
@@ -0,0 +1,126 @@
+# This default Joshua config file is written in TypesafeConfig syntax
+# [https://github.com/typesafehub/config] and is parsed at
+# initialization time as the default configuration.
+# It may be overriden by any other given Joshua config file and/or
+# commandline overrides.
+
+# Joshua supported search algorithms: "cky" or "stack"
+search_algorithm = cky
+
+# default grammar symbols. The symbols here should be enclosed in square brackets.
+default_non_terminal = "[X]"
+goal_symbol = "[GOAL]"
+
+# whether to construct a StructuredTranslation object for each request instead of
+# printing to stdout. Used when the Decoder is used from Java directly.
+use_structured_output = false
+
+# If set to true, Joshua will recapitalize the output by projecting the case from aligned source-side words
+project_case = false
+
+output_format = "%i ||| %s ||| %f ||| %c"
+
+# When true, _OOV is appended to all words that are passed through (useful for something like
+# transliteration on the target side
+mark_oovs = false
+
+# n-best configuration
+# Make sure output strings in the n-best list are unique.
+use_unique_nbest = true
+# Include the phrasal alignments in the output.
+include_align_index = false
+# The number of hypotheses to output by default.
+top_n = 1
+# Write n-best output to this file
+n_best_file = ""
+
+# The cube pruning pop limit. Set to 0 for exhaustive pruning.
+pop_limit = 100
+
+# Maximum sentence length. Sentences longer than this are truncated.
+maximum_sentence_length = 200
+
+# The number of target sides considered for each source side (after sorting by model weight)
+num_translation_options = 20
+
+# The distortion limit
+reordering_limit = 8
+
+# If set to true, Joshua will lowercase the input, creating an annotation that marks the original case
+lowercase = false
+
+# Whether to segment OOVs into a lattice
+segment_oovs = false
+
+# Enable lattice decoding.
+lattice_decoding = false
+
+# If false, sorting of the complete grammar is done at load time. If true, grammar tries are not
+# sorted till they are first accessed. Amortized sorting means you get your first translation
+# much, much quicker (good for debugging), but that per-sentence decoding is a bit slower.
+amortized_sorting = true
+
+# syntax-constrained decoding
+constrain_parse = false
+
+use_pos_labels = false
+
+# oov-specific
+true_oovs_only = false
+
+# Dynamic sentence-level filtering.
+filter_grammar = false
+
+# The number of decoding threads to use (-threads).
+num_parallel_decoders = 1
+
+# Enables synchronous parsing.
+parse = false
+  
+# Whether to do forest rescoring. If set to true, the references are expected on STDIN along with
+# the input sentences in the following format: 
+# input sentence ||| ||| reference1 ||| reference2 ... 
+# (The second field is reserved for the output sentence for alignment and forced decoding).
+rescore_forest = false
+rescore_forest_weight = 10.0f
+
+
+# If true, decode using a dot chart (standard CKY+); if false, use the much more efficient
+# version of Sennrich (SSST 2014)
+use_dot_chart = true
+
+# moses compatibility
+moses = false
+
+# Read input from a file (Moses compatible flag)
+input_file = ""
+
+# Timeout in seconds for threads
+translation_thread_timeout = 30_000
+
+# grammars are a list of sub configs for each grammar
+grammars = []
+
+# the list of feature functions with their configurations
+feature_functions = []
+
+# an external weights file to be specified
+weights_file = ""
+
+# The decoder weights directly specified as key-value pairs. These take precedence over weights in weights_file
+weights = {}
+
+serverSettings = {	
+
+	# Input typr: plain or json
+	input_type = plain
+
+	# Type of server: none, TCP, HTTP
+	server_type = none
+
+	# If set, Joshua will start a (multi-threaded) TCP/IP server on this port.
+	server_port = 0
+}
+
+# the default size of the cache used by packed grammars
+default_packed_grammar_cache_size = 5000
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/f2edda0f/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 3d6f3bc..e448ec8 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
@@ -19,16 +19,14 @@
 package org.apache.joshua.decoder;
 
 import static org.apache.joshua.decoder.ff.FeatureMap.hashFeature;
-import static org.apache.joshua.decoder.ff.tm.OwnerMap.getOwner;
+import static org.apache.joshua.decoder.ff.tm.hash_based.TextGrammarFactory.createCustomGrammar;
+import static org.apache.joshua.decoder.ff.tm.hash_based.TextGrammarFactory.createGlueTextGrammar;
 import static org.apache.joshua.util.Constants.spaceSeparator;
 
-import java.io.BufferedWriter;
 import java.io.File;
-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;
@@ -49,21 +47,21 @@ import org.apache.joshua.decoder.ff.tm.Grammar;
 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.ff.tm.hash_based.MemoryBasedBatchGrammar;
+import org.apache.joshua.decoder.ff.tm.hash_based.TextGrammarFactory;
 import org.apache.joshua.decoder.ff.tm.packed.PackedGrammar;
 import org.apache.joshua.decoder.io.TranslationRequestStream;
-import org.apache.joshua.decoder.phrase.PhraseTable;
 import org.apache.joshua.decoder.segment_file.Sentence;
-import org.apache.joshua.util.FileUtility;
-import org.apache.joshua.util.FormatUtils;
-import org.apache.joshua.util.Regex;
 import org.apache.joshua.util.io.LineReader;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.base.Strings;
+import com.google.common.base.Throwables;
+import com.google.common.collect.ImmutableList;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import com.typesafe.config.Config;
+import com.typesafe.config.ConfigFactory;
+import com.typesafe.config.ConfigParseOptions;
+import com.typesafe.config.ConfigValue;
 
 /**
  * This class handles decoder initialization and the complication introduced by multithreading.
@@ -80,6 +78,7 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder;
  * but also ensures that round-robin parallelization occurs, since RequestParallelizer uses the
  * thread pool before translating each request.
  *
+ * TODO(fhieber): this documentation should be updated
  * A decoding thread is handled by DecoderTask and launched from DecoderThreadRunner. The purpose
  * of the runner is to record where to place the translated sentence when it is done (i.e., which
  * TranslationResponseStream object). TranslationResponseStream itself is an iterator whose next() call blocks until the next
@@ -90,40 +89,49 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder;
  * @author wren ng thornton wren@users.sourceforge.net
  * @author Lane Schwartz dowobeha@users.sourceforge.net
  * @author Kellen Sunderland kellen.sunderland@gmail.com
+ * @author Felix Hieber felix.hieber@gmail.com
  */
 public class Decoder {
 
   private static final Logger LOG = LoggerFactory.getLogger(Decoder.class);
 
-  private final JoshuaConfiguration joshuaConfiguration;
-
-  public JoshuaConfiguration getJoshuaConfiguration() {
-    return joshuaConfiguration;
-  }
-
   /*
-   * Many of these objects themselves are global objects. We pass them in when constructing other
-   * objects, so that they all share pointers to the same object. This is good because it reduces
-   * overhead, but it can be problematic because of unseen dependencies (for example, in the
-   * Vocabulary shared by language model, translation grammar, etc).
+   * Holds the common (immutable) decoder state (features, grammars etc.) after initialization
    */
-  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;
-
-  public static int VERBOSE = 1;
-
+  private final DecoderConfig decoderConfig;
+  
+  private static final ImmutableList<String> GRAMMAR_PACKAGES = ImmutableList.of(
+      "org.apache.joshua.decoder.ff.tm.hash_based",
+      "org.apache.joshua.decoder.ff.tm.packed",
+      "org.apache.joshua.decoder.phrase");
+  
+  private static final ImmutableList<String> FEATURE_PACKAGES = ImmutableList.of(
+      "org.apache.joshua.decoder.ff",
+      "org.apache.joshua.decoder.ff.lm",
+      "org.apache.joshua.decoder.ff.phrase");
+  
   /**
    * Constructor method that creates a new decoder using the specified configuration file.
    *
    * @param joshuaConfiguration a populated {@link org.apache.joshua.decoder.JoshuaConfiguration}
    */
-  public Decoder(JoshuaConfiguration joshuaConfiguration) {
-    this.joshuaConfiguration = joshuaConfiguration;
-    this.initialize();
+  public Decoder(Config config) {
+    this.decoderConfig = initialize(config); 
+  }
+  
+  /**
+   * Returns the default Decoder flags.
+   */
+  public static Config getDefaultFlags() {
+    final ConfigParseOptions options = ConfigParseOptions.defaults().setAllowMissing(false);
+    return ConfigFactory.parseResources(Decoder.class, "Decoder.conf", options).resolve();
+  }
+  
+  /**
+   * Returns the DecoderConfig
+   */
+  public DecoderConfig getDecoderConfig() {
+    return decoderConfig;
   }
 
   /**
@@ -149,8 +157,8 @@ public class Decoder {
             .setNameFormat("TranslationWorker-%d")
             .setDaemon(true)
             .build();
-    ExecutorService executor = Executors.newFixedThreadPool(this.joshuaConfiguration.num_parallel_decoders,
-            threadFactory);
+    int numParallelDecoders = this.decoderConfig.getFlags().getInt("num_parallel_decoders");
+    ExecutorService executor = Executors.newFixedThreadPool(numParallelDecoders, threadFactory);
     try {
       for (; ; ) {
         Sentence sentence = request.next();
@@ -176,15 +184,49 @@ public class Decoder {
 
 
   /**
-   * We can also just decode a single sentence in the same thread.
+   * Decode call for a single sentence.
+   * Creates a sentence-specific {@link DecoderConfig} including
+   * sentence-specific OOVGrammar.
    *
    * @param sentence {@link org.apache.joshua.lattice.Lattice} input
-   * @throws RuntimeException if any fatal errors occur during translation
    * @return the sentence {@link org.apache.joshua.decoder.Translation}
    */
   public Translation decode(Sentence sentence) {
-    DecoderTask decoderTask = new DecoderTask(this.grammars, this.featureFunctions, joshuaConfiguration);
-    return decoderTask.translate(sentence);
+    final DecoderConfig sentenceConfig = createSentenceDecoderConfig(sentence, decoderConfig);
+    final DecoderTask decoderTask = new DecoderTask(sentenceConfig, sentence);
+    return decoderTask.translate();
+  }
+  
+  /**
+   * Creates a sentence-specific {@link DecoderConfig}.
+   * Most importantly, adds an OOV grammar for the words of this
+   * sentence.
+   */
+  private static DecoderConfig createSentenceDecoderConfig(
+      final Sentence sentence, final DecoderConfig globalConfig) {
+    
+    // create a new list of grammars that includes the OOVgrammar
+    // this is specific to the search algorithm
+    final ImmutableList.Builder<Grammar> grammars = new ImmutableList.Builder<>();
+    switch (globalConfig.getSearchAlgorithm()) {
+    case cky:
+      grammars
+        .add(TextGrammarFactory.createOovGrammarForSentence(sentence, globalConfig));
+    case stack:
+      grammars 
+        .add(TextGrammarFactory.createEndRulePhraseTable(sentence, globalConfig))
+        .add(TextGrammarFactory.createOovPhraseTable(sentence, globalConfig));
+    }
+    
+    return new DecoderConfig(
+        globalConfig.getFlags(),
+        globalConfig.getFeatureFunctions(),
+        grammars.addAll(globalConfig.getGrammars()).build(),
+        globalConfig.getCustomGrammar(),
+        globalConfig.getVocabulary(),
+        globalConfig.getWeights(),
+        globalConfig.getFeatureMap(),
+        globalConfig.getOwnerMap());
   }
 
   /**
@@ -206,248 +248,105 @@ public class Decoder {
     StatefulFF.resetGlobalStateIndex();
   }
 
-  public static void writeConfigFile(double[] newWeights, String template, String outputFile,
-      String newDiscriminativeModel) {
-    try {
-      int columnID = 0;
-
-      try (LineReader reader = new LineReader(template);
-           BufferedWriter writer = FileUtility.getWriteFileStream(outputFile)) {
-        for (String line : reader) {
-          line = line.trim();
-          if (Regex.commentOrEmptyLine.matches(line) || line.contains("=")) {
-            // comment, empty line, or parameter lines: just copy
-            writer.write(line);
-            writer.newLine();
-
-          } else { // models: replace the weight
-            String[] fds = Regex.spaces.split(line);
-            StringBuilder newSent = new StringBuilder();
-            if (!Regex.floatingNumber.matches(fds[fds.length - 1])) {
-              throw new IllegalArgumentException("last field is not a number; the field is: "
-                      + fds[fds.length - 1]);
-            }
-
-            if (newDiscriminativeModel != null && "discriminative".equals(fds[0])) {
-              newSent.append(fds[0]).append(' ');
-              newSent.append(newDiscriminativeModel).append(' ');// change the
-              // file name
-              for (int i = 2; i < fds.length - 1; i++) {
-                newSent.append(fds[i]).append(' ');
-              }
-            } else {// regular
-              for (int i = 0; i < fds.length - 1; i++) {
-                newSent.append(fds[i]).append(' ');
-              }
-            }
-            if (newWeights != null)
-              newSent.append(newWeights[columnID++]);// change the weight
-            else
-              newSent.append(fds[fds.length - 1]);// do not change
-
-            writer.write(newSent.toString());
-            writer.newLine();
-          }
-        }
-      }
-
-      if (newWeights != null && columnID != newWeights.length) {
-        throw new IllegalArgumentException("number of models does not match number of weights");
-      }
-
-    } catch (IOException e) {
-      e.printStackTrace();
-    }
-  }
-
   /**
-   * Initialize all parts of the JoshuaDecoder.
+   * Initialize all parts of the Decoder.
    */
-  private void initialize() {
-    try {
-
-      long pre_load_time = System.currentTimeMillis();
-      resetGlobalState();
-
-      /* Weights can be listed in a separate file (denoted by parameter "weights-file") or directly
-       * in the Joshua config file. Config file values take precedent.
-       */
-      this.readWeights(joshuaConfiguration.weights_file);
-
-
-      /* Add command-line-passed weights to the weights array for processing below */
-      if (!Strings.isNullOrEmpty(joshuaConfiguration.weight_overwrite)) {
-        String[] tokens = joshuaConfiguration.weight_overwrite.split("\\s+");
-        for (int i = 0; i < tokens.length; i += 2) {
-          String feature = tokens[i];
-          float value = Float.parseFloat(tokens[i+1]);
-
-          if (joshuaConfiguration.moses)
-            feature = demoses(feature);
-
-          joshuaConfiguration.weights.add(String.format("%s %s", feature, tokens[i+1]));
-          LOG.info("COMMAND LINE WEIGHT: {} -> {}", feature, value);
-        }
-      }
-
-      /* Read the weights found in the config file */
-      for (String pairStr: joshuaConfiguration.weights) {
-        String pair[] = pairStr.split("\\s+");
-
-        /* Sanity check for old-style unsupported feature invocations. */
-        if (pair.length != 2) {
-          String errMsg = "FATAL: Invalid feature weight line found in config file.\n" +
-              String.format("The line was '%s'\n", pairStr) +
-              "You might be using an old version of the config file that is no longer supported\n" +
-              "Check joshua.apache.org or email dev@joshua.apache.org for help\n" +
-              "Code = " + 17;
-          throw new RuntimeException(errMsg);
-        }
-
-        weights.add(hashFeature(pair[0]), Float.parseFloat(pair[1]));
-      }
-
-      LOG.info("Read {} weights", weights.size());
-
-      // Do this before loading the grammars and the LM.
-      this.featureFunctions.clear();
-
-      // Initialize and load grammars. This must happen first, since the vocab gets defined by
-      // the packed grammar (if any)
-      this.initializeTranslationGrammars();
-      LOG.info("Grammar loading took: {} seconds.",
-          (System.currentTimeMillis() - pre_load_time) / 1000);
-
-      // Initialize the features: requires that LM model has been initialized.
-      this.initializeFeatureFunctions();
-
-      // This is mostly for compatibility with the Moses tuning script
-      if (joshuaConfiguration.show_weights_and_quit) {
-        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);
-      }
-
-      // Sort the TM grammars (needed to do cube pruning)
-      if (joshuaConfiguration.amortized_sorting) {
-        LOG.info("Grammar sorting happening lazily on-demand.");
-      } else {
-        long pre_sort_time = System.currentTimeMillis();
-        for (Grammar grammar : this.grammars) {
-          grammar.sortGrammar(this.featureFunctions);
-        }
-        LOG.info("Grammar sorting took {} seconds.",
-            (System.currentTimeMillis() - pre_sort_time) / 1000);
+  private DecoderConfig initialize(final Config config) {
+    
+    LOG.info("Initializing decoder ...");
+    long initTime = System.currentTimeMillis();
+    
+    /*
+     * (1) read weights (denoted by parameter "weights-file")
+     * or directly in the Joshua config. Config file values take precedent.
+     */
+    final FeatureVector weights = readWeights(config);
+    
+    /*
+     * (2) initialize/instantiate translation grammars
+     * Unfortunately this can not be static due to customPhraseTable member.
+     */
+    final List<Grammar> grammars = initializeTranslationGrammars(config);
+    final Grammar customGrammar = createCustomGrammar(SearchAlgorithm.valueOf(config.getString("search_algorithm")));
+    grammars.add(customGrammar);
+    
+    /*
+     * (3) initialize/instantiate feature functions 
+     */
+    final ImmutableList<FeatureFunction> featureFunctions = initializeFeatureFunctions(config, grammars, weights);
+    
+    /*
+     * (4) Optionally sort the grammars for cube-pruning
+     */
+    if (config.getBoolean("amortized_sorting")) {
+      LOG.info("Grammar sorting happening lazily on-demand.");
+    } else {
+      long preSortTime = System.currentTimeMillis();
+      for (final Grammar grammar : grammars) {
+        grammar.sortGrammar(featureFunctions);
       }
-
-    } catch (IOException e) {
-      LOG.warn(e.getMessage(), e);
+      LOG.info("Grammar sorting took {} seconds.", (System.currentTimeMillis() - preSortTime) / 1000);
     }
+    
+    LOG.info("Initialization done ({} seconds)", (System.currentTimeMillis() - initTime) / 1000);
+    // TODO(fhieber): right now we still rely on static variables for vocab etc.
+    // this should be changed and then we pass the instance of vocab etc. in here
+    return new DecoderConfig(
+        config,
+        featureFunctions,
+        ImmutableList.copyOf(grammars),
+        customGrammar,
+        null,
+        weights,
+        null,
+        null);
   }
 
   /**
-   * Initializes translation grammars Retained for backward compatibility
-   *
-   * @throws IOException Several grammar elements read from disk that can
-   * cause IOExceptions.
+   * Returns a list of initialized {@link Grammar}s
    */
-  private void initializeTranslationGrammars() throws IOException {
-
-    if (joshuaConfiguration.tms.size() > 0) {
-
-      // collect packedGrammars to check if they use a shared vocabulary
-      final List<PackedGrammar> packed_grammars = new ArrayList<>();
-
-      // tm = {thrax/hiero,packed,samt,moses} OWNER LIMIT FILE
-      for (String tmLine : joshuaConfiguration.tms) {
-
-        String type = tmLine.substring(0,  tmLine.indexOf(' '));
-        String[] args = tmLine.substring(tmLine.indexOf(' ')).trim().split("\\s+");
-        HashMap<String, String> parsedArgs = FeatureFunction.parseArgs(args);
-
-        String owner = parsedArgs.get("owner");
-        int span_limit = Integer.parseInt(parsedArgs.get("maxspan"));
-        String path = parsedArgs.get("path");
-
-        Grammar grammar;
-        if (! type.equals("moses") && ! type.equals("phrase")) {
-          if (new File(path).isDirectory()) {
-            try {
-              PackedGrammar packed_grammar = new PackedGrammar(path, span_limit, owner, type, joshuaConfiguration);
-              packed_grammars.add(packed_grammar);
-              grammar = packed_grammar;
-            } catch (FileNotFoundException e) {
-              String msg = String.format("Couldn't load packed grammar from '%s'", path)
-                  + "Perhaps it doesn't exist, or it may be an old packed file format.";
-              throw new RuntimeException(msg);
-            }
-          } else {
-            // thrax, hiero, samt
-            grammar = new MemoryBasedBatchGrammar(type, path, owner,
-                joshuaConfiguration.default_non_terminal, span_limit, joshuaConfiguration);
-          }
-
-        } else {
-
-          joshuaConfiguration.search_algorithm = "stack";
-          grammar = new PhraseTable(path, owner, type, joshuaConfiguration);
-        }
-
-        this.grammars.add(grammar);
+  private List<Grammar> initializeTranslationGrammars(final Config config) {
+    
+    final List<Grammar> result = new ArrayList<>();
+    
+    // collect packedGrammars to check if they use a shared vocabulary
+    final List<PackedGrammar> packedGrammars = new ArrayList<>();
+    
+    final long startTime = System.currentTimeMillis();
+    
+    for (final Config grammarConfig : config.getConfigList("grammars")) {
+      final Class<?> clazz = getClassFromPackages(grammarConfig.getString("class"), GRAMMAR_PACKAGES);
+      try {
+        final Constructor<?> constructor = clazz.getConstructor(Config.class);
+        final Grammar grammar = (Grammar) constructor.newInstance(grammarConfig);
+        result.add(grammar);
+      } catch (Exception e) {
+        LOG.error("Unable to instantiate grammar '{}'", clazz.getName());
+        Throwables.propagate(e);
       }
-
-      checkSharedVocabularyChecksumsForPackedGrammars(packed_grammars);
-
-    } else {
-      LOG.warn("no grammars supplied!  Supplying dummy glue grammar.");
-      MemoryBasedBatchGrammar glueGrammar = new MemoryBasedBatchGrammar("glue", joshuaConfiguration, -1);
-      glueGrammar.addGlueRules(featureFunctions);
-      this.grammars.add(glueGrammar);
-    }
-
-    /* Add the grammar for custom entries */
-    if (joshuaConfiguration.search_algorithm.equals("stack"))
-      this.customPhraseTable = new PhraseTable("custom", joshuaConfiguration);
-    else
-      this.customPhraseTable = new MemoryBasedBatchGrammar("custom", joshuaConfiguration, 20);
-    this.grammars.add(this.customPhraseTable);
-
-    /* Create an epsilon-deleting grammar */
-    if (joshuaConfiguration.lattice_decoding) {
-      LOG.info("Creating an epsilon-deleting grammar");
-      MemoryBasedBatchGrammar latticeGrammar = new MemoryBasedBatchGrammar("lattice", joshuaConfiguration, -1);
-      HieroFormatReader reader = new HieroFormatReader(OwnerMap.register("lattice"));
-
-      String goalNT = FormatUtils.cleanNonTerminal(joshuaConfiguration.goal_symbol);
-      String defaultNT = FormatUtils.cleanNonTerminal(joshuaConfiguration.default_non_terminal);
-
-      //FIXME: arguments changed to match string format on best effort basis.  Author please review.
-      String ruleString = String.format("[%s] ||| [%s,1] <eps> ||| [%s,1] ||| ", goalNT, defaultNT, defaultNT);
-
-      Rule rule = reader.parseLine(ruleString);
-      latticeGrammar.addRule(rule);
-      rule.estimateRuleCost(featureFunctions);
-
-      this.grammars.add(latticeGrammar);
     }
-
-    /* Now create a feature function for each owner */
-    final Set<OwnerId> ownersSeen = new HashSet<>();
-
-    for (Grammar grammar: this.grammars) {
-      OwnerId owner = grammar.getOwner();
-      if (! ownersSeen.contains(owner)) {
-        this.featureFunctions.add(
-            new PhraseModel(
-                weights, new String[] { "tm", "-owner", getOwner(owner) }, joshuaConfiguration, grammar));
-        ownersSeen.add(owner);
+    
+    if (result.isEmpty()) {
+      
+      LOG.warn("no grammars supplied! Supplying dummy glue grammar.");
+      result.add(createGlueTextGrammar(
+          config.getString("goal_symbol"),
+          config.getString("default_non_terminal")));
+      
+    } else {
+      
+      checkSharedVocabularyChecksumsForPackedGrammars(packedGrammars);
+      
+      if (config.getBoolean("lattice_decoding")) {
+        LOG.info("Creating an epsilon-deleting grammar");
+        result.add(TextGrammarFactory.addEpsilonDeletingGrammar(
+            config.getString("goal_symbol"),
+            config.getString("default_non_terminal")));
       }
     }
-
-    LOG.info("Memory used {} MB",
-        ((Runtime.getRuntime().totalMemory() - Runtime.getRuntime().freeMemory()) / 1000000.0));
+    
+    LOG.info("Grammar loading took: {} seconds.", (System.currentTimeMillis() - startTime) / 1000);
+    return result;
   }
 
   /**
@@ -470,44 +369,55 @@ public class Decoder {
     }
   }
 
-  /*
-   * This function reads the weights for the model. Feature names and their weights are listed one
-   * per line in the following format:
-   *
-   * FEATURE_NAME WEIGHT
-   */
-  private void readWeights(String fileName) {
-    Decoder.weights = new FeatureVector(5);
-
-    if (fileName.equals(""))
-      return;
-
-    try (LineReader lineReader = new LineReader(fileName);) {
-      for (String line : lineReader) {
-        line = line.replaceAll(spaceSeparator, " ");
-
-        if (line.equals("") || line.startsWith("#") || line.startsWith("//")
-            || line.indexOf(' ') == -1)
-          continue;
+  /**
+    * This function reads the weights for the model either
+    * from the weights_file in format
+    * NAME VALUE
+    * once per line;
+    * or from the weights section in the {@link Config} object.
+    * The latter take precedence.
+    */
+  private static FeatureVector readWeights(final Config config) {
+    final FeatureVector weights = new FeatureVector(5);
+    
+    // read from optional weights_file
+    if (config.hasPath("weights_file")
+        && new File(config.getString("weights_file")).exists()) {
+      final String weightsFilename = config.getString("weights_file");
+      try (LineReader lineReader = new LineReader(weightsFilename);) {
+        for (String line : lineReader) {
+          line = line.replaceAll(spaceSeparator, " ");
+          if (line.equals("") || line.startsWith("#") || line.startsWith("//") || line.indexOf(' ') == -1) {
+            continue;
+          }
+          final String tokens[] = line.split(spaceSeparator);
+          String feature = tokens[0];
+          final float value = Float.parseFloat(tokens[1]);
 
-        String tokens[] = line.split(spaceSeparator);
-        String feature = tokens[0];
-        Float value = Float.parseFloat(tokens[1]);
+          // Kludge for compatibility with Moses tuners
+          if (config.getBoolean("moses")) {
+            feature = demoses(feature);
+          }
 
-        // Kludge for compatibility with Moses tuners
-        if (joshuaConfiguration.moses) {
-          feature = demoses(feature);
+          weights.put(hashFeature(feature), value);
         }
-
-        weights.add(hashFeature(feature), value);
+        LOG.info("Read {} weights from file '{}'", weights.size(), weightsFilename);
+      } catch (IOException e) {
+        Throwables.propagate(e);
       }
-    } catch (IOException ioe) {
-      throw new RuntimeException(ioe);
     }
-    LOG.info("Read {} weights from file '{}'", weights.size(), fileName);
+    
+    // overwrite with config values
+    for (Entry<String, ConfigValue> entry : config.getConfig("weights").entrySet()) {
+      final String name = entry.getKey();
+      float value = ((Number) entry.getValue().unwrapped()).floatValue();
+      weights.put(hashFeature(name), value);
+    }
+    LOG.info("Read {} weights", weights.size());
+    return weights;
   }
 
-  private String demoses(String feature) {
+  private static String demoses(String feature) {
     if (feature.endsWith("="))
       feature = feature.replace("=", "");
     if (feature.equals("OOV_Penalty"))
@@ -518,60 +428,58 @@ public class Decoder {
   }
 
   /**
-   * Feature functions are instantiated with a line of the form
-   *
-   * <pre>
-   *   FEATURE OPTIONS
-   * </pre>
-   *
-   * Weights for features are listed separately.
-   *
+   * Initializes & instantiates feature functions.
+   * Required a list of previously loaded grammars to instantiate the PhraseModel feature function
+   * as well.
    */
-  private void initializeFeatureFunctions() {
-
-    for (String featureLine : joshuaConfiguration.features) {
-      // line starts with NAME, followed by args
-      // 1. create new class named NAME, pass it config, weights, and the args
-
-      String fields[] = featureLine.split("\\s+");
-      String featureName = fields[0];
-
+  private static ImmutableList<FeatureFunction> initializeFeatureFunctions(
+      final Config config, final List<Grammar> grammars, final FeatureVector weights) {
+    
+    final ImmutableList.Builder<FeatureFunction> result = new ImmutableList.Builder<>();
+    
+    // (1) create PhraseModel feature function for every owner
+    final Set<OwnerId> ownersSeen = new HashSet<>();
+    for (final Grammar grammar: grammars) {
+      final OwnerId owner = grammar.getOwner();
+      if (!ownersSeen.contains(owner)) {
+        result.add(new PhraseModel(owner, ConfigFactory.empty(), weights));
+        ownersSeen.add(owner);
+      }
+    }
+    
+    // (2) instantiate other feature functions by class name
+    for (Config featureConfig : config.getConfigList("feature_functions")) {
+      final Class<?> clazz = getClassFromPackages(featureConfig.getString("class"), FEATURE_PACKAGES);
       try {
-
-        Class<?> clas = getFeatureFunctionClass(featureName);
-        Constructor<?> constructor = clas.getConstructor(FeatureVector.class,
-            String[].class, JoshuaConfiguration.class);
-        FeatureFunction feature = (FeatureFunction) constructor.newInstance(weights, fields, joshuaConfiguration);
-        this.featureFunctions.add(feature);
-
+        final Constructor<?> constructor = clazz.getConstructor(Config.class, FeatureVector.class);
+        final FeatureFunction feature = (FeatureFunction) constructor.newInstance(featureConfig, weights);
+        result.add(feature);
       } catch (Exception e) {
-        throw new RuntimeException(String.format("Unable to instantiate feature function '%s'!", featureLine), e);
+        LOG.error("Unable to instantiate feature '{}'", clazz.getName());
+        Throwables.propagate(e);
       }
     }
-
-    for (FeatureFunction feature : featureFunctions) {
+    
+    final ImmutableList<FeatureFunction> features = result.build(); 
+    for (final FeatureFunction feature : features) {
       LOG.info("FEATURE: {}", feature.logString());
     }
+    return features;
   }
-
+  
   /**
-   * Searches a list of predefined paths for classes, and returns the first one found. Meant for
-   * instantiating feature functions.
-   *
-   * @param featureName Class name of the feature to return.
-   * @return the class, found in one of the search paths
+   * Searches a list of paths for classes and returns the first one found.
+   * Used for instantiating grammars and feature functions.
    */
-  private Class<?> getFeatureFunctionClass(String featureName) {
+  private static Class<?> getClassFromPackages(String className, ImmutableList<String> packages) {
     Class<?> clas = null;
-
-    String[] packages = { "org.apache.joshua.decoder.ff", "org.apache.joshua.decoder.ff.lm", "org.apache.joshua.decoder.ff.phrase" };
     for (String path : packages) {
       try {
-        clas = Class.forName(String.format("%s.%s", path, featureName));
+        clas = Class.forName(String.format("%s.%s", path, className));
         break;
       } catch (ClassNotFoundException e) {
         try {
-          clas = Class.forName(String.format("%s.%sFF", path, featureName));
+          clas = Class.forName(String.format("%s.%sFF", path, className));
           break;
         } catch (ClassNotFoundException e2) {
           // do nothing
@@ -587,11 +495,11 @@ public class Decoder {
    * @param rule the rule to add
    */
   public void addCustomRule(Rule rule) {
-    customPhraseTable.addRule(rule);
-    rule.estimateRuleCost(featureFunctions);
+    decoderConfig.getCustomGrammar().addRule(rule);
+    rule.estimateRuleCost(decoderConfig.getFeatureFunctions());
   }
 
   public Grammar getCustomPhraseTable() {
-    return customPhraseTable;
+    return decoderConfig.getCustomGrammar();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/f2edda0f/joshua-core/src/main/java/org/apache/joshua/decoder/DecoderConfig.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/decoder/DecoderConfig.java b/joshua-core/src/main/java/org/apache/joshua/decoder/DecoderConfig.java
new file mode 100644
index 0000000..71ee40c
--- /dev/null
+++ b/joshua-core/src/main/java/org/apache/joshua/decoder/DecoderConfig.java
@@ -0,0 +1,123 @@
+/*
+ * 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;
+
+import org.apache.joshua.corpus.Vocabulary;
+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.tm.Grammar;
+import org.apache.joshua.decoder.ff.tm.OwnerMap;
+
+import com.google.common.collect.ImmutableList;
+import com.typesafe.config.Config;
+
+/**
+ * This is the central config/state object that holds references to relevant attributes
+ * of the decoder (features, grammars, etc.).
+ * A sentence-specific instance of this object is created before
+ * translating a single sentence.
+ * 
+ * @author Felix Hieber, felix.hieber@gmail.com
+ */
+public class DecoderConfig {
+  
+  /** Decoder flags */ 
+  private final Config flags;
+  
+  /** Decoder feature functions */
+  private final ImmutableList<FeatureFunction> featureFunctions;
+  
+  /** Decoder grammars/phrase tables */
+  private final ImmutableList<Grammar> grammars;
+  
+  /** Decoder custom grammar where rules can be added */
+  private final Grammar customGrammar;
+  
+  /** Decoder vocabulary */
+  private final Vocabulary vocabulary;
+  
+  /** Decoder weights */
+  private final FeatureVector weights;
+  
+  /** Decoder feature mapping */
+  private final FeatureMap featureMap;
+  
+  /** Decoder grammar owner mapping */
+  private final OwnerMap ownerMap;
+  
+  private final SearchAlgorithm searchAlgorithm;
+  
+  public DecoderConfig(
+      final Config flags,
+      final ImmutableList<FeatureFunction> featureFunctions,
+      final ImmutableList<Grammar> grammars,
+      final Grammar customGrammar,
+      final Vocabulary vocabulary,
+      final FeatureVector weights,
+      final FeatureMap featureMap,
+      final OwnerMap ownerMap) {
+    this.flags = flags;
+    this.featureFunctions = featureFunctions;
+    this.grammars = grammars;
+    this.customGrammar = customGrammar;
+    this.vocabulary = vocabulary;
+    this.weights = weights;
+    this.featureMap = featureMap;
+    this.ownerMap = ownerMap;
+    this.searchAlgorithm = SearchAlgorithm.valueOf(flags.getString("search_algorithm"));
+  }
+
+  public Config getFlags() {
+    return flags;
+  }
+
+  public ImmutableList<FeatureFunction> getFeatureFunctions() {
+    return featureFunctions;
+  }
+
+  public ImmutableList<Grammar> getGrammars() {
+    return grammars;
+  }
+  
+  public Grammar getCustomGrammar() {
+    return customGrammar;
+  }
+
+  public Vocabulary getVocabulary() {
+    return vocabulary;
+  }
+
+  public FeatureVector getWeights() {
+    return weights;
+  }
+
+  public FeatureMap getFeatureMap() {
+    return featureMap;
+  }
+
+  public OwnerMap getOwnerMap() {
+    return ownerMap;
+  }
+  
+  public SearchAlgorithm getSearchAlgorithm() {
+    return searchAlgorithm;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/f2edda0f/joshua-core/src/main/java/org/apache/joshua/decoder/DecoderTask.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/decoder/DecoderTask.java b/joshua-core/src/main/java/org/apache/joshua/decoder/DecoderTask.java
index 0c7a76b..d820d7d 100644
--- a/joshua-core/src/main/java/org/apache/joshua/decoder/DecoderTask.java
+++ b/joshua-core/src/main/java/org/apache/joshua/decoder/DecoderTask.java
@@ -18,22 +18,21 @@
  */
 package org.apache.joshua.decoder;
 
-import java.util.ArrayList;
-import java.util.List;
-
+import org.apache.joshua.corpus.Vocabulary;
 import org.apache.joshua.decoder.chart_parser.Chart;
-import org.apache.joshua.decoder.ff.FeatureFunction;
-import org.apache.joshua.decoder.ff.SourceDependentFF;
 import org.apache.joshua.decoder.ff.tm.Grammar;
 import org.apache.joshua.decoder.hypergraph.ForestWalker;
 import org.apache.joshua.decoder.hypergraph.GrammarBuilderWalkerFunction;
 import org.apache.joshua.decoder.hypergraph.HyperGraph;
 import org.apache.joshua.decoder.phrase.Stacks;
 import org.apache.joshua.decoder.segment_file.Sentence;
-import org.apache.joshua.corpus.Vocabulary;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.collect.ImmutableMap;
+import com.typesafe.config.Config;
+import com.typesafe.config.ConfigFactory;
+
 /**
  * This class handles decoding of individual Sentence objects (which can represent plain sentences
  * or lattices). A single sentence can be decoded by a call to translate() and, if an InputHandler
@@ -41,122 +40,110 @@ import org.slf4j.LoggerFactory;
  * translateAll(), which continually queries the InputHandler for sentences until they have all been
  * consumed and translated.
  * 
- * The DecoderFactory class is responsible for launching the threads.
- * 
  * @author Matt Post post@cs.jhu.edu
  * @author Zhifei Li, zhifei.work@gmail.com
+ * @author Felix Hieber, felix.hieber@gmail.com
  */
-
 public class DecoderTask {
+  
   private static final Logger LOG = LoggerFactory.getLogger(DecoderTask.class);
-
-  private final JoshuaConfiguration joshuaConfiguration;
-  /*
-   * these variables may be the same across all threads (e.g., just copy from DecoderFactory), or
-   * differ from thread to thread
+  
+  /** sentence-specific DecoderConfig,
+   * mostly shared with the global decoderConfig, but can have adaptations
    */
-  private final List<Grammar> allGrammars;
-  private final List<FeatureFunction> featureFunctions;
-
-  public DecoderTask(List<Grammar> grammars, List<FeatureFunction> featureFunctions,
-                       JoshuaConfiguration joshuaConfiguration) {
-
-    this.joshuaConfiguration = joshuaConfiguration;
-    this.allGrammars = grammars;
-
-    this.featureFunctions = new ArrayList<>();
-    for (FeatureFunction ff : featureFunctions) {
-      if (ff instanceof SourceDependentFF) {
-        this.featureFunctions.add(((SourceDependentFF) ff).clone());
-      } else {
-        this.featureFunctions.add(ff);
-      }
-    }
+  private final DecoderConfig sentenceConfig;
+  private final Sentence sentence;
+  private final boolean segmentOovs;
+  private final boolean useDotChart;
+  private final boolean doParsing;
+
+  public DecoderTask(final DecoderConfig sentenceConfig, final Sentence sentence) {
+    this.sentenceConfig = sentenceConfig;
+    this.sentence = sentence;
+    this.segmentOovs = sentenceConfig.getFlags().getBoolean("segment_oovs");
+    this.useDotChart = sentenceConfig.getFlags().getBoolean("use_dot_chart");
+    this.doParsing = sentenceConfig.getFlags().getBoolean("parse");
   }
 
   /**
-   * Translate a sentence.
-   * 
-   * @param sentence The sentence to be translated.
-   * @return the sentence {@link org.apache.joshua.decoder.Translation}
+   * Translate the sentence.
+   * @return translation of the sentence {@link org.apache.joshua.decoder.Translation}
    */
-  public Translation translate(Sentence sentence) {
+  public Translation translate() {
 
     LOG.info("Input {}: {}", sentence.id(), sentence.fullSource());
 
-    if (sentence.target() != null)
+    if (sentence.target() != null) {
       LOG.info("Input {}: Constraining to target sentence '{}'",
           sentence.id(), sentence.target());
+    }
 
     // skip blank sentences
     if (sentence.isEmpty()) {
       LOG.info("Translation {}: Translation took 0 seconds", sentence.id());
-      return new Translation(sentence, null, featureFunctions, joshuaConfiguration);
+      return new Translation(sentence, null, sentenceConfig);
     }
 
     long startTime = System.currentTimeMillis();
 
-    int numGrammars = allGrammars.size();
-    Grammar[] grammars = new Grammar[numGrammars];
-
-    for (int i = 0; i < allGrammars.size(); i++)
-      grammars[i] = allGrammars.get(i);
-
-    if (joshuaConfiguration.segment_oovs)
-      sentence.segmentOOVs(grammars);
+    // TODO(fhieber): this should be done in the constructor maybe?
+    // But it should not modify the sentence object.
+    if (segmentOovs) {
+      sentence.segmentOOVs(sentenceConfig.getGrammars());
+    }
 
     /*
      * Joshua supports (as of September 2014) both phrase-based and hierarchical decoding. Here
      * we build the appropriate chart. The output of both systems is a hypergraph, which is then
      * used for further processing (e.g., k-best extraction).
      */
-    HyperGraph hypergraph;
-    try {
-
-      if (joshuaConfiguration.search_algorithm.equals("stack")) {
-        Stacks stacks = new Stacks(sentence, this.featureFunctions, grammars, joshuaConfiguration);
+    final HyperGraph hypergraph = createHypergraph();
 
-        hypergraph = stacks.search();
-      } else {
-        /* Seeding: the chart only sees the grammars, not the factories */
-        Chart chart = new Chart(sentence, this.featureFunctions, grammars,
-            joshuaConfiguration.goal_symbol, joshuaConfiguration);
+    float decodingTime = (System.currentTimeMillis() - startTime) / 1000.0f;
+    float usedMemory = (Runtime.getRuntime().totalMemory() - Runtime.getRuntime().freeMemory()) / 1000000.0f;
+    LOG.info("Input {}: Translation took {} seconds", sentence.id(), decodingTime);
+    LOG.info("Input {}: Memory used is {} MB", sentence.id(), usedMemory);
 
-        hypergraph = (joshuaConfiguration.use_dot_chart) 
-            ? chart.expand() 
-                : chart.expandSansDotChart();
+    /* Return the translation unless we're doing synchronous parsing. */
+    if (!doParsing || hypergraph == null) {
+      return new Translation(sentence, hypergraph, sentenceConfig);
+    } else {
+      return parse(hypergraph);
+    }
+  }
+  
+  private HyperGraph createHypergraph() {
+    try {
+      switch (sentenceConfig.getSearchAlgorithm()) {
+      case stack:
+        final Stacks stacks = new Stacks(sentence, sentenceConfig);
+        return stacks.search();
+      case cky:
+        final Chart chart = new Chart(sentence, sentenceConfig);
+        return useDotChart ? chart.expand() : chart.expandSansDotChart();
+      default:
+        return null;
       }
-
     } catch (java.lang.OutOfMemoryError e) {
-      LOG.error("Input {}: out of memory", sentence.id());
-      hypergraph = null;
+      return null;
     }
-
-    float seconds = (System.currentTimeMillis() - startTime) / 1000.0f;
-    LOG.info("Input {}: Translation took {} seconds", sentence.id(), seconds);
-    LOG.info("Input {}: Memory used is {} MB", sentence.id(), (Runtime
-        .getRuntime().totalMemory() - Runtime.getRuntime().freeMemory()) / 1000000.0);
-
-    /* Return the translation unless we're doing synchronous parsing. */
-    if (!joshuaConfiguration.parse || hypergraph == null) {
-      return new Translation(sentence, hypergraph, featureFunctions, joshuaConfiguration);
-    }
-
-    /*
-     * Synchronous parsing.
-     * 
-     * Step 1. Traverse the hypergraph to create a grammar for the second-pass parse.
-     */
-    Grammar newGrammar = getGrammarFromHyperGraph(joshuaConfiguration.goal_symbol, hypergraph);
-    newGrammar.sortGrammar(this.featureFunctions);
+  }
+  
+  /**
+   * Synchronous parsing.
+   */
+  private Translation parse(final HyperGraph hypergraph) {
+    long startTime = System.currentTimeMillis();
+    // Step 1. Traverse the hypergraph to create a grammar for the second-pass parse.
+    final Grammar newGrammar = getGrammarFromHyperGraph(sentenceConfig.getFlags().getString("goal_symbol"), hypergraph);
+    newGrammar.sortGrammar(sentenceConfig.getFeatureFunctions());
     long sortTime = System.currentTimeMillis();
     LOG.info("Sentence {}: New grammar has {} rules.", sentence.id(),
         newGrammar.getNumRules());
 
     /* Step 2. Create a new chart and parse with the instantiated grammar. */
-    Grammar[] newGrammarArray = new Grammar[] { newGrammar };
-    Sentence targetSentence = new Sentence(sentence.target(), sentence.id(), joshuaConfiguration);
-    Chart chart = new Chart(targetSentence, featureFunctions, newGrammarArray, "GOAL",joshuaConfiguration);
+    final Sentence targetSentence = new Sentence(sentence.target(), sentence.id(), sentenceConfig.getFlags());
+    final Chart chart = new Chart(targetSentence, sentenceConfig);
     int goalSymbol = GrammarBuilderWalkerFunction.goalSymbol(hypergraph);
     String goalSymbolString = Vocabulary.word(goalSymbol);
     LOG.info("Sentence {}: goal symbol is {} ({}).", sentence.id(),
@@ -172,14 +159,17 @@ public class DecoderTask {
         (secondParseTime - startTime) / 1000);
     LOG.info("Memory used after sentence {} is {} MB", sentence.id(), (Runtime
         .getRuntime().totalMemory() - Runtime.getRuntime().freeMemory()) / 1000000.0);
-    return new Translation(sentence, englishParse, featureFunctions, joshuaConfiguration); // or do something else
+    return new Translation(sentence, englishParse, sentenceConfig); // or do something else
   }
 
-  private Grammar getGrammarFromHyperGraph(String goal, HyperGraph hg) {
-    GrammarBuilderWalkerFunction f = new GrammarBuilderWalkerFunction(goal,joshuaConfiguration,
-            "pt");
+  private static Grammar getGrammarFromHyperGraph(String goal, HyperGraph hg) {
+    final Config grammarConfig = ConfigFactory.parseMap(
+        ImmutableMap.of("owner", "pt", "span_limit", "1000"), "");
+    GrammarBuilderWalkerFunction f = new GrammarBuilderWalkerFunction(
+        goal, grammarConfig);
     ForestWalker walker = new ForestWalker();
     walker.walk(hg.goalNode, f);
     return f.getGrammar();
   }
+  
 }

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/f2edda0f/joshua-core/src/main/java/org/apache/joshua/decoder/InputType.java
----------------------------------------------------------------------
diff --git a/joshua-core/src/main/java/org/apache/joshua/decoder/InputType.java b/joshua-core/src/main/java/org/apache/joshua/decoder/InputType.java
new file mode 100644
index 0000000..e04da79
--- /dev/null
+++ b/joshua-core/src/main/java/org/apache/joshua/decoder/InputType.java
@@ -0,0 +1,4 @@
+package org.apache.joshua.decoder;
+
+/* Determines whether to expect JSON input or plain lines */
+public enum InputType { plain, json }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-joshua/blob/f2edda0f/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
deleted file mode 100644
index ddf24ea..0000000
--- a/joshua-core/src/main/java/org/apache/joshua/decoder/JoshuaConfiguration.java
+++ /dev/null
@@ -1,738 +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;
-
-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.util.ArrayList;
-import java.util.Collections;
-
-import org.apache.joshua.decoder.ff.StatefulFF;
-import org.apache.joshua.decoder.ff.fragmentlm.Tree;
-import org.apache.joshua.util.FormatUtils;
-import org.apache.joshua.util.Regex;
-import org.apache.joshua.util.io.LineReader;
-import org.apache.log4j.Level;
-import org.apache.log4j.LogManager;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Configuration file for Joshua decoder.
- *
- * When adding new features to Joshua, any new configurable parameters should be added to this
- * class.
- *
- * @author Zhifei Li, zhifei.work@gmail.com
- * @author Matt Post post@cs.jhu.edu
- */
-public class JoshuaConfiguration {
-
-  private static final Logger LOG = LoggerFactory.getLogger(JoshuaConfiguration.class);
-
-  // whether to construct a StructuredTranslation object for each request instead of
-  // printing to stdout. Used when the Decoder is used from Java directly.
-  public Boolean use_structured_output = false;
-
-  // If set to true, Joshua will lowercase the input, creating an annotation that marks the
-  // original case
-  public boolean lowercase = false;
-
-  // If set to true, Joshua will recapitalize the output by projecting the case from aligned
-  // source-side words
-  public boolean project_case = false;
-
-  // List of grammar files to read
-  public ArrayList<String> tms = new ArrayList<>();
-
-  // A rule cache for commonly used tries to avoid excess object allocations
-  // Testing shows there's up to ~95% hit rate when cache size is 5000 Trie nodes.
-  public Integer cachedRuleSize = 5000;
-
-  /*
-   * The file to read the weights from (part of the sparse features implementation). Weights can
-   * also just be listed in the main config file.
-   */
-  public String weights_file = "";
-  // Default symbols. The symbol here should be enclosed in square brackets.
-  public String default_non_terminal = FormatUtils.ensureNonTerminalBrackets("X");
-  public String goal_symbol = FormatUtils.ensureNonTerminalBrackets("GOAL");
-
-  /*
-   * A list of OOV symbols in the form
-   *
-   * [X1] weight [X2] weight [X3] weight ...
-   *
-   * where the [X] symbols are nonterminals and the weights are weights. For each OOV word w in the
-   * input sentence, Joshua will create rules of the form
-   *
-   * X1 -> w (weight)
-   *
-   * If this is empty, an unweighted default_non_terminal is used.
-   */
-  public class OOVItem implements Comparable<OOVItem> {
-    public final String label;
-
-    public final float weight;
-
-    OOVItem(String l, float w) {
-      label = l;
-      weight = w;
-    }
-    @Override
-    public int compareTo(OOVItem other) {
-      if (weight > other.weight)
-        return -1;
-      else if (weight < other.weight)
-        return 1;
-      return 0;
-    }
-  }
-
-  public ArrayList<OOVItem> oovList = null;
-
-  /*
-   * Whether to segment OOVs into a lattice
-   */
-  public boolean segment_oovs = false;
-
-  /*
-   * Enable lattice decoding.
-   */
-  public boolean lattice_decoding = false;
-
-  /*
-   * If false, sorting of the complete grammar is done at load time. If true, grammar tries are not
-   * sorted till they are first accessed. Amortized sorting means you get your first translation
-   * much, much quicker (good for debugging), but that per-sentence decoding is a bit slower.
-   */
-  public boolean amortized_sorting = true;
-  // syntax-constrained decoding
-  public boolean constrain_parse = false;
-
-  public boolean use_pos_labels = false;
-
-  // oov-specific
-  public boolean true_oovs_only = false;
-
-  /* Dynamic sentence-level filtering. */
-  public boolean filter_grammar = false;
-
-  /* The cube pruning pop limit. Set to 0 for exhaustive pruning. */
-  public int pop_limit = 100;
-
-  /* Maximum sentence length. Sentences longer than this are truncated. */
-  public int maxlen = 200;
-
-  /*
-   * N-best configuration.
-   */
-  // Make sure output strings in the n-best list are unique.
-  public boolean use_unique_nbest = true;
-
-  /* Include the phrasal alignments in the output (not word-level alignmetns at the moment). */
-  public boolean include_align_index = false;
-
-  /* The number of hypotheses to output by default. */
-  public int topN = 1;
-
-  /**
-   * This string describes the format of each line of output from the decoder (i.e., the
-   * translations). The string can include arbitrary text and also variables. The following
-   * variables are available:
-   *
-   * <pre>
-   * - %i the 0-indexed sentence number
-   * - %e the source string %s the translated sentence
-   * - %S the translated sentence with some basic capitalization and denormalization
-   * - %t the synchronous derivation
-   * - %f the list of feature values (as name=value pairs)
-   * - %c the model cost
-   * - %w the weight vector
-   * - %a the alignments between source and target words (currently unimplemented)
-   * - %d a verbose, many-line version of the derivation
-   * </pre>
-   */
-  public String outputFormat = "%i ||| %s ||| %f ||| %c";
-
-  /* The number of decoding threads to use (-threads). */
-  public int num_parallel_decoders = 1;
-
-  /*
-   * When true, _OOV is appended to all words that are passed through (useful for something like
-   * transliteration on the target side
-   */
-  public boolean mark_oovs = false;
-
-  /* Enables synchronous parsing. */
-  public boolean parse = false; // perform synchronous parsing
-
-
-  /* A list of the feature functions. */
-  public ArrayList<String> features = new ArrayList<>();
-
-  /* A list of weights found in the main config file (instead of in a separate weights file) */
-  public ArrayList<String> weights = new ArrayList<>();
-
-  /* Determines whether to expect JSON input or plain lines */
-  public enum INPUT_TYPE { plain, json }
-
-  public INPUT_TYPE input_type = INPUT_TYPE.plain;
-
-  /* Type of server. Not sure we need to keep the regular TCP one around. */
-  public enum SERVER_TYPE { none, TCP, HTTP }
-
-  public SERVER_TYPE server_type = SERVER_TYPE.TCP;
-
-  /* If set, Joshua will start a (multi-threaded, per "threads") TCP/IP server on this port. */
-  public int server_port = 0;
-
-  /*
-   * Whether to do forest rescoring. If set to true, the references are expected on STDIN along with
-   * the input sentences in the following format:
-   * 
-   * input sentence ||| ||| reference1 ||| reference2 ...
-   * 
-   * (The second field is reserved for the output sentence for alignment and forced decoding).
-   */
-
-  public boolean rescoreForest = false;
-  public float rescoreForestWeight = 10.0f;
-
-  /*
-   * Location of fragment mapping file, which maps flattened SCFG rules to their internal
-   * representation.
-   */
-  public String fragmentMapFile = null;
-
-  /*
-   * Whether to use soft syntactic constraint decoding /fuzzy matching, which allows that any
-   * nonterminal may be substituted for any other nonterminal (except for OOV and GOAL)
-   */
-  public boolean fuzzy_matching = false;
-
-  public static final String SOFT_SYNTACTIC_CONSTRAINT_DECODING_PROPERTY_NAME = "fuzzy_matching";
-
-  /***
-   * Phrase-based decoding parameters.
-   */
-  
-  /* The search algorithm: currently either "cky" or "stack" */
-  public String search_algorithm = "cky";
-
-  /* The distortion limit */
-  public int reordering_limit = 8;
-
-  /* The number of target sides considered for each source side (after sorting by model weight) */
-  public int num_translation_options = 20;
-
-  /* If true, decode using a dot chart (standard CKY+); if false, use the much more efficient
-   * version of Sennrich (SSST 2014)
-   */
-  public boolean use_dot_chart = true;
-
-  /* Moses compatibility */
-  public boolean moses = false;
-
-  /* If true, just print out the weights found in the config file, and exit. */
-  public boolean show_weights_and_quit = false;
-
-  /* Read input from a file (Moses compatible flag) */
-  public String input_file = null;
-
-  /* Write n-best output to this file */
-  public String n_best_file = null;
-
-  /* Whether to look at source side for special annotations */
-  public boolean source_annotations = false;
-
-  /* Weights overridden from the command line */
-  public String weight_overwrite = "";
-
-  /* Timeout in seconds for threads */
-  public long translation_thread_timeout = 30_000;
-
-  /**
-   * This method resets the state of JoshuaConfiguration back to the state after initialization.
-   * This is useful when for example making different calls to the decoder within the same java
-   * program, which otherwise leads to potential errors due to inconsistent state as a result of
-   * loading the configuration multiple times without resetting etc.
-   *
-   * This leads to the insight that in fact it may be an even better idea to refactor the code and
-   * make JoshuaConfiguration an object that is is created and passed as an argument, rather than a
-   * shared static object. This is just a suggestion for the next step.
-   *
-   */
-  public void reset() {
-    LOG.info("Resetting the JoshuaConfiguration to its defaults ...");
-    LOG.info("\n\tResetting the StatefullFF global state index ...");
-    LOG.info("\n\t...done");
-    StatefulFF.resetGlobalStateIndex();
-    tms = new ArrayList<>();
-    weights_file = "";
-    default_non_terminal = "[X]";
-    oovList = new ArrayList<>();
-    oovList.add(new OOVItem(default_non_terminal, 1.0f));
-    goal_symbol = "[GOAL]";
-    amortized_sorting = true;
-    constrain_parse = false;
-    use_pos_labels = false;
-    true_oovs_only = false;
-    filter_grammar = false;
-    pop_limit = 100;
-    maxlen = 200;
-    use_unique_nbest = false;
-    include_align_index = false;
-    topN = 1;
-    outputFormat = "%i ||| %s ||| %f ||| %c";
-    num_parallel_decoders = 1;
-    mark_oovs = false;
-    // oracleFile = null;
-    parse = false; // perform synchronous parsing
-    features = new ArrayList<>();
-    weights = new ArrayList<>();
-    server_port = 0;
-
-    reordering_limit = 8;
-    num_translation_options = 20;
-    LOG.info("...done");
-  }
-
-  // ===============================================================
-  // Methods
-  // ===============================================================
-
-  /**
-   * To process command-line options, we write them to a file that looks like the config file, and
-   * then call readConfigFile() on it. It would be more general to define a class that sits on a
-   * stream and knows how to chop it up, but this was quicker to implement.
-   * 
-   * @param options string array of command line options
-   */
-  public void processCommandLineOptions(String[] options) {
-    try {
-      File tmpFile = File.createTempFile("options", null, null);
-      PrintWriter out = new PrintWriter(new FileWriter(tmpFile));
-
-      for (int i = 0; i < options.length; i++) {
-        String key = options[i].substring(1);
-        if (i + 1 == options.length || options[i + 1].startsWith("-")) {
-          // if this is the last item, or if the next item
-          // is another flag, then this is a boolean flag
-          out.println(key + " = true");
-
-        } else {
-          out.print(key + " =");
-          while (i + 1 < options.length && ! options[i + 1].startsWith("-")) {
-            out.print(String.format(" %s", options[i + 1]));
-            i++;
-          }
-          out.println();
-        }
-      }
-      out.close();
-      
-//      LOG.info("Parameters overridden from the command line:");
-      this.readConfigFile(tmpFile.getCanonicalPath());
-
-      tmpFile.delete();
-
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  public void readConfigFile(String configFile) throws IOException {
-
-    LineReader configReader = new LineReader(configFile, false);
-    try {
-      for (String line : configReader) {
-        line = line.trim(); // .toLowerCase();
-
-        if (Regex.commentOrEmptyLine.matches(line))
-          continue;
-
-        /*
-         * There are two kinds of substantive (non-comment, non-blank) lines: parameters and feature
-         * values. Parameters match the pattern "key = value"; all other substantive lines are
-         * interpreted as features.
-         */
-
-        if (line.contains("=")) { // parameters; (not feature function)
-          String[] fds = Regex.equalsWithSpaces.split(line, 2);
-          if (fds.length < 2) {
-            LOG.warn("skipping config file line '{}'", line);
-            continue;
-          }
-
-          String parameter = normalize_key(fds[0]);
-
-          if (parameter.equals(normalize_key("lm"))) {
-            /* This is deprecated. This support old LM lines of the form
-             * 
-             *   lm = berkeleylm 5 false false 100 lm.gz
-             * 
-             * LMs are now loaded as general feature functions, so we transform that to either
-             * 
-             *   LanguageModel -lm_order 5 -lm_type berkeleylm -lm_file lm.gz
-             * 
-             * If the line were state minimizing:
-             * 
-             *   lm = kenlm 5 true false 100 lm.gz
-             *              
-             * StateMinimizingLanguageModel -lm_order 5 -lm_file lm.gz
-             */
-
-            String[] tokens = fds[1].split("\\s+");
-            if (tokens[2].equals("true"))
-              features.add(String.format("StateMinimizingLanguageModel -lm_type kenlm -lm_order %s -lm_file %s",
-                  tokens[1], tokens[5]));
-            else
-              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_PREFIX))) {
-            /* If found, convert old format:
-             *   tm = TYPE OWNER MAXSPAN PATH
-             * to new format
-             *   tm = TYPE -owner OWNER -maxspan MAXSPAN -path PATH    
-             */
-            String tmLine = fds[1];
-
-            String[] tokens = fds[1].split("\\s+");
-            if (! tokens[1].startsWith("-")) { // old format
-              tmLine = String.format("%s -owner %s -maxspan %s -path %s", tokens[0], tokens[1], tokens[2], tokens[3]);
-              LOG.warn("Converting deprecated TM line from '{}' -> '{}'", fds[1], tmLine);
-            }
-            tms.add(tmLine);
-
-          } else if (parameter.equals("v")) {
-
-            // This is already handled in ArgsParser, skip it here, easier than removing it there
-
-          } else if (parameter.equals(normalize_key("parse"))) {
-            parse = Boolean.parseBoolean(fds[1]);
-            LOG.debug("parse: {}", parse);
-
-          } else if (parameter.equals(normalize_key("oov-list"))) {
-            if (new File(fds[1]).exists()) {
-              oovList = new ArrayList<>();
-              try {
-                File file = new File(fds[1]);
-                BufferedReader br = new BufferedReader(new FileReader(file));
-                try {
-                  String str = br.readLine();
-                  while (str != null) {
-                    String[] tokens = str.trim().split("\\s+");
-
-                    oovList.add(new OOVItem(FormatUtils.ensureNonTerminalBrackets(tokens[0]),
-                            (float) Math.log(Float.parseFloat(tokens[1]))));
-
-                    str = br.readLine();
-                  }
-                  br.close();
-                } catch(IOException e){
-                  System.out.println(e);
-                }
-              } catch(IOException e){
-                System.out.println(e);
-              }
-              Collections.sort(oovList);
-
-            } else {
-              String[] tokens = fds[1].trim().split("\\s+");
-              if (tokens.length % 2 != 0) {
-                throw new RuntimeException(String.format("* FATAL: invalid format for '%s'", fds[0]));
-              }
-              oovList = new ArrayList<>();
-
-              for (int i = 0; i < tokens.length; i += 2)
-                oovList.add(new OOVItem(FormatUtils.ensureNonTerminalBrackets(tokens[i]),
-                    (float) Math.log(Float.parseFloat(tokens[i + 1]))));
-
-              Collections.sort(oovList);
-            }
-
-          } else if (parameter.equals(normalize_key("lattice-decoding"))) {
-            lattice_decoding = true;
-
-          } else if (parameter.equals(normalize_key("segment-oovs"))) {
-            segment_oovs = true;
-            lattice_decoding = true;
-
-          } else if (parameter.equals(normalize_key("default-non-terminal"))) {
-            default_non_terminal = ensureNonTerminalBrackets(cleanNonTerminal(fds[1].trim()));
-            LOG.debug("default_non_terminal: {}", default_non_terminal);
-
-          } else if (parameter.equals(normalize_key("goal-symbol"))) {
-            goal_symbol = ensureNonTerminalBrackets(cleanNonTerminal(fds[1].trim()));
-            LOG.debug("goalSymbol: {}", goal_symbol);
-
-          } else if (parameter.equals(normalize_key("weights-file"))) {
-            weights_file = fds[1];
-
-          } else if (parameter.equals(normalize_key("constrain_parse"))) {
-            constrain_parse = Boolean.parseBoolean(fds[1]);
-
-          } else if (parameter.equals(normalize_key("true_oovs_only"))) {
-            true_oovs_only = Boolean.parseBoolean(fds[1]);
-
-          } else if (parameter.equals(normalize_key("filter-grammar"))) {
-            filter_grammar = Boolean.parseBoolean(fds[1]);
-
-          } else if (parameter.equals(normalize_key("amortize"))) {
-            amortized_sorting = Boolean.parseBoolean(fds[1]);
-
-          } else if (parameter.equals(normalize_key("use_pos_labels"))) {
-            use_pos_labels = Boolean.parseBoolean(fds[1]);
-
-          } else if (parameter.equals(normalize_key("use_unique_nbest"))) {
-            use_unique_nbest = Boolean.valueOf(fds[1]);
-            LOG.debug("use_unique_nbest: {}", use_unique_nbest);
-
-          } else if (parameter.equals(normalize_key("output-format"))) {
-            outputFormat = fds[1];
-            LOG.debug("output-format: {}", outputFormat);
-
-          } else if (parameter.equals(normalize_key("include_align_index"))) {
-            include_align_index = Boolean.valueOf(fds[1]);
-            LOG.debug("include_align_index: {}", include_align_index);
-
-          } else if (parameter.equals(normalize_key("top_n"))) {
-            topN = Integer.parseInt(fds[1]);
-            LOG.debug("topN: {}", topN);
-
-          } else if (parameter.equals(normalize_key("num_parallel_decoders"))
-              || parameter.equals(normalize_key("threads"))) {
-            num_parallel_decoders = Integer.parseInt(fds[1]);
-            if (num_parallel_decoders <= 0) {
-              throw new IllegalArgumentException(
-                  "Must specify a positive number for num_parallel_decoders");
-            }
-            LOG.debug("num_parallel_decoders: {}", num_parallel_decoders);
-
-          } else if (parameter.equals(normalize_key("mark_oovs"))) {
-            mark_oovs = Boolean.valueOf(fds[1]);
-            LOG.debug("mark_oovs: {}", mark_oovs);
-
-          } else if (parameter.equals(normalize_key("pop-limit"))) {
-            pop_limit = Integer.parseInt(fds[1]);
-            LOG.info("pop-limit: {}", pop_limit);
-
-          } else if (parameter.equals(normalize_key("input-type"))) {
-            switch (fds[1]) {
-            case "json":
-              input_type = INPUT_TYPE.json;
-              break;
-            case "plain":
-              input_type = INPUT_TYPE.plain;
-              break;
-            default:
-              throw new RuntimeException(
-                  String.format("* FATAL: invalid server type '%s'", fds[1]));
-            }
-            LOG.info("    input-type: {}", input_type);
-
-          } else if (parameter.equals(normalize_key("server-type"))) {
-            if (fds[1].toLowerCase().equals("tcp"))
-              server_type = SERVER_TYPE.TCP;
-            else if (fds[1].toLowerCase().equals("http"))
-              server_type = SERVER_TYPE.HTTP;
-
-            LOG.info("    server-type: {}", server_type);
-
-          } else if (parameter.equals(normalize_key("server-port"))) {
-            server_port = Integer.parseInt(fds[1]);
-            LOG.info("    server-port: {}", server_port);
-
-          } else if (parameter.equals(normalize_key("rescore-forest"))) {
-            rescoreForest = true;
-            LOG.info("    rescore-forest: {}", rescoreForest);
-
-          } else if (parameter.equals(normalize_key("rescore-forest-weight"))) {
-            rescoreForestWeight = Float.parseFloat(fds[1]);
-            LOG.info("    rescore-forest-weight: {}", rescoreForestWeight);
-
-          } else if (parameter.equals(normalize_key("maxlen"))) {
-            // reset the maximum length
-            maxlen = Integer.parseInt(fds[1]);
-
-          } else if (parameter.equals("c") || parameter.equals("config")) {
-            // this was used to send in the config file, just ignore it
-
-          } else if (parameter.equals(normalize_key("feature-function"))) {
-            // add the feature to the list of features for later processing
-            features.add(fds[1]);
-
-          } else if (parameter.equals(normalize_key("maxlen"))) {
-            // add the feature to the list of features for later processing
-            maxlen = Integer.parseInt(fds[1]);
-
-          } else if (parameter
-              .equals(normalize_key(SOFT_SYNTACTIC_CONSTRAINT_DECODING_PROPERTY_NAME))) {
-            fuzzy_matching = Boolean.parseBoolean(fds[1]);
-            LOG.debug("fuzzy_matching: {}", fuzzy_matching);
-
-          } else if (parameter.equals(normalize_key("fragment-map"))) {
-            fragmentMapFile = fds[1];
-            Tree.readMapping(fragmentMapFile);
-
-            /** PHRASE-BASED PARAMETERS **/
-          } else if (parameter.equals(normalize_key("search"))) {
-            search_algorithm = fds[1];
-
-            if (!search_algorithm.equals("cky") && !search_algorithm.equals("stack")) {
-              throw new RuntimeException(
-                  "-search must be one of 'stack' (for phrase-based decoding) " +
-                      "or 'cky' (for hierarchical / syntactic decoding)");
-            }
-
-            if (search_algorithm.equals("cky") && include_align_index) {
-              throw new RuntimeException(
-                  "include_align_index is currently not supported with cky search");
-            }
-
-          } else if (parameter.equals(normalize_key("reordering-limit"))) {
-            reordering_limit = Integer.parseInt(fds[1]);
-
-          } else if (parameter.equals(normalize_key("num-translation-options"))) {
-            num_translation_options = Integer.parseInt(fds[1]);
-
-          } else if (parameter.equals(normalize_key("no-dot-chart"))) {
-            use_dot_chart = false;
-
-          } else if (parameter.equals(normalize_key("moses"))) {
-            moses = true; // triggers some Moses-specific compatibility options
-
-          } else if (parameter.equals(normalize_key("show-weights"))) {
-            show_weights_and_quit = true;
-
-          } else if (parameter.equals(normalize_key("n-best-list"))) {
-            // for Moses compatibility
-            String[] tokens = fds[1].split("\\s+");
-            n_best_file = tokens[0];
-            if (tokens.length > 1)
-              topN = Integer.parseInt(tokens[1]);
-
-          } else if (parameter.equals(normalize_key("input-file"))) {
-            // for Moses compatibility
-            input_file = fds[1];
-
-          } else if (parameter.equals(normalize_key("weight-file"))) {
-            // for Moses, ignore
-
-          } else if (parameter.equals(normalize_key("weight-overwrite"))) {
-            weight_overwrite = fds[1];
-
-          } else if (parameter.equals(normalize_key("source-annotations"))) {
-            // Check source sentence
-            source_annotations = true;
-
-          } else if (parameter.equals(normalize_key("cached-rules-size"))) {
-            // Check source sentence
-            cachedRuleSize = Integer.parseInt(fds[1]);
-          } else if (parameter.equals(normalize_key("lowercase"))) {
-            lowercase = true;
-
-          } else if (parameter.equals(normalize_key("project-case"))) {
-            project_case = true;
-
-          } else {
-
-            if (parameter.equals(normalize_key("use-sent-specific-tm"))
-                || parameter.equals(normalize_key("add-combined-cost"))
-                || parameter.equals(normalize_key("use-tree-nbest"))
-                || parameter.equals(normalize_key("use-kenlm"))
-                || parameter.equals(normalize_key("useCubePrune"))
-                || parameter.equals(normalize_key("useBeamAndThresholdPrune"))
-                || parameter.equals(normalize_key("regexp-grammar"))) {
-              LOG.warn("ignoring deprecated parameter '{}'", fds[0]);
-
-            } else {
-              throw new RuntimeException("FATAL: unknown configuration parameter '" + fds[0] + "'");
-            }
-          }
-
-          LOG.info("    {} = '{}'", normalize_key(fds[0]), fds[1]);
-
-        } else {
-          /*
-           * Lines that don't have an equals sign and are not blank lines, empty lines, or comments,
-           * are feature values, which can be present in this file
-           */
-
-          weights.add(line);
-        }
-      }
-    } finally {
-      configReader.close();
-    }
-  }
-
-  /**
-   * Checks for invalid variable configurations
-   */
-  public void sanityCheck() {
-  }
-  
-  /**
-   * Sets the verbosity level to v (0: OFF; 1: INFO; 2: DEBUG).
-   * 
-   * @param v the verbosity level (0, 1, or 2)
-   */
-  public void setVerbosity(int v) {
-    Decoder.VERBOSE = v;
-    switch (Decoder.VERBOSE) {
-    case 0:
-      LogManager.getRootLogger().setLevel(Level.OFF);
-      break;
-    case 1:
-      LogManager.getRootLogger().setLevel(Level.INFO);
-      break;
-    case 2:
-      LogManager.getRootLogger().setLevel(Level.DEBUG);
-      break;
-    }
-  }
-
-  /**
-   * Normalizes parameter names by removing underscores and hyphens and lowercasing. This defines
-   * equivalence classes on external use of parameter names, permitting arbitrary_under_scores and
-   * camelCasing in paramter names without forcing the user to memorize them all. Here are some
-   * examples of equivalent ways to refer to parameter names:
-   * <pre>
-   * {pop-limit, poplimit, PopLimit, popLimit, pop_lim_it} {lmfile, lm-file, LM-FILE, lm_file}
-   * </pre>
-   * 
-   * @param text the string to be normalized
-   * @return normalized key
-   * 
-   */
-  public static String normalize_key(String text) {
-    return text.replaceAll("[-_]", "").toLowerCase();
-  }
-}