You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by mb...@apache.org on 2017/03/07 03:06:26 UTC

[5/7] asterixdb git commit: Configuration Revamp

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4c7b5bfa/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/NodeProperties.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/NodeProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/NodeProperties.java
new file mode 100644
index 0000000..4175873
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/NodeProperties.java
@@ -0,0 +1,87 @@
+/*
+ * 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.asterix.common.config;
+
+import org.apache.hyracks.api.config.IOption;
+import org.apache.hyracks.api.config.IOptionType;
+import org.apache.hyracks.api.config.Section;
+import org.apache.hyracks.control.common.config.OptionTypes;
+import org.apache.hyracks.control.common.controllers.NCConfig;
+import org.apache.hyracks.util.file.FileUtil;
+
+public class NodeProperties extends AbstractProperties {
+
+    public enum Option implements IOption {
+        INITIAL_RUN(OptionTypes.BOOLEAN, false),
+        CORE_DUMP_DIR(OptionTypes.STRING, FileUtil.joinPath(NCConfig.defaultDir, "coredump")),
+        TXN_LOG_DIR(OptionTypes.STRING, FileUtil.joinPath(NCConfig.defaultDir, "txn-log")),
+        STORAGE_SUBDIR(OptionTypes.STRING, "storage"),
+        ;
+
+        private final IOptionType type;
+        private final Object defaultValue;
+
+        <T> Option(IOptionType<T> type, T defaultValue) {
+            this.type = type;
+            this.defaultValue = defaultValue;
+        }
+
+        @Override
+        public Section section() {
+            return Section.NC;
+        }
+
+        @Override
+        public String description() {
+            switch (this) {
+                case INITIAL_RUN:
+                    return "A flag indicating if it's the first time the NC is started";
+                default:
+                    return null;
+            }
+        }
+
+        @Override
+        public IOptionType type() {
+            return type;
+        }
+
+        @Override
+        public Object defaultValue() {
+            return defaultValue;
+        }
+
+        @Override
+        public boolean hidden() {
+            return this == INITIAL_RUN;
+        }
+    }
+
+    public NodeProperties(PropertiesAccessor accessor) {
+        super(accessor);
+    }
+
+    public boolean isInitialRun() {
+        return accessor.getBoolean(Option.INITIAL_RUN);
+    }
+
+    public boolean isVirtualNc() {
+        return accessor.getBoolean(NCConfig.Option.VIRTUAL_NC);
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4c7b5bfa/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/PropertiesAccessor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/PropertiesAccessor.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/PropertiesAccessor.java
index 1f503ec..233daa7 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/PropertiesAccessor.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/PropertiesAccessor.java
@@ -18,20 +18,29 @@
  */
 package org.apache.asterix.common.config;
 
+import static org.apache.asterix.common.config.MetadataProperties.Option.INSTANCE_NAME;
+import static org.apache.asterix.common.config.MetadataProperties.Option.METADATA_NODE;
+import static org.apache.asterix.common.config.NodeProperties.Option.STORAGE_SUBDIR;
+import static org.apache.hyracks.control.common.controllers.NCConfig.Option.IODEVICES;
+
 import java.io.File;
 import java.io.FileInputStream;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InputStream;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Objects;
 import java.util.Properties;
 import java.util.Set;
 import java.util.SortedMap;
 import java.util.TreeMap;
-import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.Predicate;
+import java.util.stream.Stream;
 
 import javax.xml.bind.JAXBContext;
 import javax.xml.bind.JAXBException;
@@ -48,22 +57,23 @@ import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.utils.ConfigUtil;
 import org.apache.commons.lang3.mutable.MutableInt;
 import org.apache.hyracks.algebricks.common.utils.Pair;
-import org.apache.hyracks.api.application.IApplicationConfig;
+import org.apache.hyracks.api.config.IApplicationConfig;
+import org.apache.hyracks.api.config.IOption;
+import org.apache.hyracks.api.config.IOptionType;
+import org.apache.hyracks.api.config.Section;
+import org.apache.hyracks.control.common.application.ConfigManagerApplicationConfig;
+import org.apache.hyracks.control.common.config.ConfigManager;
+import org.apache.hyracks.control.common.controllers.ControllerConfig;
+import org.apache.hyracks.control.common.controllers.NCConfig;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
 
-public class PropertiesAccessor {
+public class PropertiesAccessor implements IApplicationConfig {
     private static final Logger LOGGER = Logger.getLogger(PropertiesAccessor.class.getName());
 
-    private static final AtomicReference<PropertiesAccessor> instanceHolder = new AtomicReference<>();
-    private final String instanceName;
-    private final String metadataNodeName;
-    private final List<String> nodeNames = new ArrayList<>();;
-    private final Map<String, String[]> stores = new HashMap<>();;
+    private static final Map<IApplicationConfig, PropertiesAccessor> instances = new ConcurrentHashMap<>();
+    private final Map<String, String[]> stores = new HashMap<>();
     private final Map<String, String> coredumpConfig = new HashMap<>();
-
-    // This can be removed when asterix-configuration.xml is no longer required.
-    private final Map<String, Property> asterixConfigurationParams;
     private final IApplicationConfig cfg;
     private final Map<String, String> transactionLogDirs = new HashMap<>();
     private final Map<String, String> asterixBuildProperties = new HashMap<>();
@@ -73,61 +83,105 @@ public class PropertiesAccessor {
     private final List<AsterixExtension> extensions;
 
     /**
-     * Constructor which reads asterix-configuration.xml, the old way.
-     *
-     * @throws AsterixException
-     * @throws IOException
+     * Constructor which wraps an IApplicationConfig.
      */
-    private PropertiesAccessor() throws AsterixException, IOException {
-        String fileName = System.getProperty(GlobalConfig.CONFIG_FILE_PROPERTY);
-        if (fileName == null) {
-            fileName = GlobalConfig.DEFAULT_CONFIG_FILE_NAME;
-        }
-        AsterixConfiguration asterixConfiguration = configure(fileName);
-        cfg = null;
-        instanceName = asterixConfiguration.getInstanceName();
-        metadataNodeName = asterixConfiguration.getMetadataNode();
-        List<Store> configuredStores = asterixConfiguration.getStore();
+    private PropertiesAccessor(IApplicationConfig cfg) throws AsterixException, IOException {
+        this.cfg = cfg;
         nodePartitionsMap = new HashMap<>();
-        int uniquePartitionId = 0;
-        // Here we iterate through all <store> elements in asterix-configuration.xml.
-        // For each one, we create an array of ClusterPartitions and store this array
-        // in nodePartitionsMap, keyed by the node name. The array is the same length
-        // as the comma-separated <storeDirs> child element, because Managix will have
-        // arranged for that element to be populated with the full paths to each
-        // partition directory (as formed by appending the <store> subdirectory to
-        // each <iodevices> path from the user's original cluster.xml).
-        for (Store store : configuredStores) {
-            String trimmedStoreDirs = store.getStoreDirs().trim();
-            String[] nodeStores = trimmedStoreDirs.split(",");
-            ClusterPartition[] nodePartitions = new ClusterPartition[nodeStores.length];
-            for (int i = 0; i < nodePartitions.length; i++) {
-                ClusterPartition partition = new ClusterPartition(uniquePartitionId++, store.getNcId(), i);
-                clusterPartitions.put(partition.getPartitionId(), partition);
-                nodePartitions[i] = partition;
-            }
-            stores.put(store.getNcId(), nodeStores);
-            nodePartitionsMap.put(store.getNcId(), nodePartitions);
-            nodeNames.add(store.getNcId());
-        }
-
-        // Get extensions
         extensions = new ArrayList<>();
-        if (asterixConfiguration.getExtensions() != null) {
-            for (Extension ext : asterixConfiguration.getExtensions().getExtension()) {
-                extensions.add(ConfigUtil.toAsterixExtension(ext));
-            }
+        // Determine whether to use old-style asterix-configuration.xml or new-style configuration.
+        // QQQ strip this out eventually
+        // QQQ this is NOT a good way to determine whether to use config file
+        ConfigManager configManager = ((ConfigManagerApplicationConfig)cfg).getConfigManager();
+        boolean usingConfigFile = Stream
+                .of((IOption) ControllerConfig.Option.CONFIG_FILE, ControllerConfig.Option.CONFIG_FILE_URL)
+                .map(configManager::get).anyMatch(Objects::nonNull);
+        AsterixConfiguration asterixConfiguration = null;
+        try {
+            asterixConfiguration = configure(System.getProperty(GlobalConfig.CONFIG_FILE_PROPERTY,
+                    GlobalConfig.DEFAULT_CONFIG_FILE_NAME));
+        } catch (Exception e) {
+            // cannot load config file, assume new-style config
         }
 
-        asterixConfigurationParams = new HashMap<>();
-        for (Property p : asterixConfiguration.getProperty()) {
-            asterixConfigurationParams.put(p.getName(), p);
-        }
-        for (Coredump cd : asterixConfiguration.getCoredump()) {
-            coredumpConfig.put(cd.getNcId(), cd.getCoredumpPath());
-        }
-        for (TransactionLogDir txnLogDir : asterixConfiguration.getTransactionLogDir()) {
-            transactionLogDirs.put(txnLogDir.getNcId(), txnLogDir.getTxnLogDirPath());
+        if (!usingConfigFile && asterixConfiguration != null) {
+            LOGGER.info("using old-style configuration: " + System.getProperty(GlobalConfig.CONFIG_FILE_PROPERTY));
+            if (asterixConfiguration.getInstanceName() != null) {
+                configManager.set(INSTANCE_NAME, asterixConfiguration.getInstanceName());
+            }
+            if (asterixConfiguration.getMetadataNode() != null) {
+                configManager.set(METADATA_NODE, asterixConfiguration.getMetadataNode());
+            }
+            List<Store> configuredStores = asterixConfiguration.getStore();
+
+            int uniquePartitionId = 0;
+            // Here we iterate through all <store> elements in asterix-configuration.xml.
+            // For each one, we create an array of ClusterPartitions and store this array
+            // in nodePartitionsMap, keyed by the node name. The array is the same length
+            // as the comma-separated <storeDirs> child element, because Managix will have
+            // arranged for that element to be populated with the full paths to each
+            // partition directory (as formed by appending the <store> subdirectory to
+            // each <iodevices> path from the user's original cluster.xml).
+            for (Store store : configuredStores) {
+                String trimmedStoreDirs = store.getStoreDirs().trim();
+                String[] nodeStores = trimmedStoreDirs.split(",");
+                ClusterPartition[] nodePartitions = new ClusterPartition[nodeStores.length];
+                for (int i = 0; i < nodePartitions.length; i++) {
+                    ClusterPartition partition = new ClusterPartition(uniquePartitionId++, store.getNcId(), i);
+                    clusterPartitions.put(partition.getPartitionId(), partition);
+                    nodePartitions[i] = partition;
+                }
+                stores.put(store.getNcId(), nodeStores);
+                nodePartitionsMap.put(store.getNcId(), nodePartitions);
+                configManager.registerVirtualNode(store.getNcId());
+                // marking node as virtual, as we're not using NCServices with old-style config
+                configManager.set(store.getNcId(), NCConfig.Option.VIRTUAL_NC, true);
+            }
+            // Get extensions
+            if (asterixConfiguration.getExtensions() != null) {
+                for (Extension ext : asterixConfiguration.getExtensions().getExtension()) {
+                    extensions.add(ConfigUtil.toAsterixExtension(ext));
+                }
+            }
+            for (Property p : asterixConfiguration.getProperty()) {
+                IOption option = null;
+                for (Section section : Arrays.asList(Section.COMMON, Section.CC, Section.NC)) {
+                    IOption optionTemp = cfg.lookupOption(section.sectionName(), p.getName());
+                    if (optionTemp == null) {
+                        continue;
+                    }
+                    if (option != null) {
+                        throw new IllegalStateException("ERROR: option found in multiple sections: " +
+                                Arrays.asList(option, optionTemp));
+                    }
+                    option = optionTemp;
+                }
+                if (option == null) {
+                    LOGGER.warn("Ignoring unknown property: " + p.getName());
+                } else {
+                    configManager.set(option, option.type().parse(p.getValue()));
+                }
+            }
+            for (Coredump cd : asterixConfiguration.getCoredump()) {
+                coredumpConfig.put(cd.getNcId(), cd.getCoredumpPath());
+            }
+            for (TransactionLogDir txnLogDir : asterixConfiguration.getTransactionLogDir()) {
+                transactionLogDirs.put(txnLogDir.getNcId(), txnLogDir.getTxnLogDirPath());
+            }
+        } else {
+            LOGGER.info("using new-style configuration");
+            MutableInt uniquePartitionId = new MutableInt(0);
+            // Iterate through each configured NC.
+            for (String ncName : cfg.getNCNames()) {
+                configureNc(ncName, uniquePartitionId);
+            }
+            for (String section : cfg.getSectionNames()) {
+                if (section.startsWith(AsterixProperties.SECTION_PREFIX_EXTENSION)) {
+                    String className = AsterixProperties.getSectionId(
+                            AsterixProperties.SECTION_PREFIX_EXTENSION, section);
+                    configureExtension(className, section);
+                }
+            }
         }
         loadAsterixBuildProperties();
     }
@@ -163,31 +217,6 @@ public class PropertiesAccessor {
         }
     }
 
-    /**
-     * Constructor which wraps an IApplicationConfig.
-     */
-    private PropertiesAccessor(IApplicationConfig cfg) throws AsterixException {
-        this.cfg = cfg;
-        instanceName = cfg.getString(AsterixProperties.SECTION_ASTERIX, AsterixProperties.PROPERTY_INSTANCE_NAME,
-                AsterixProperties.DEFAULT_INSTANCE_NAME);
-        nodePartitionsMap = new HashMap<>();
-        MutableInt uniquePartitionId = new MutableInt(0);
-        extensions = new ArrayList<>();
-        // Iterate through each configured NC.
-        for (String section : cfg.getSections()) {
-            if (section.startsWith(AsterixProperties.SECTION_PREFIX_NC)) {
-                configureNc(section, uniquePartitionId);
-            } else if (section.startsWith(AsterixProperties.SECTION_PREFIX_EXTENSION)) {
-                String className = AsterixProperties.getSectionId(AsterixProperties.SECTION_PREFIX_EXTENSION, section);
-                configureExtension(className, section);
-            }
-        }
-        metadataNodeName = getProperty(AsterixProperties.PROPERTY_METADATA_NODE,
-                nodeNames.isEmpty() ? "" : nodeNames.get(0), PropertyInterpreters.getStringPropertyInterpreter());
-        asterixConfigurationParams = null;
-        loadAsterixBuildProperties();
-    }
-
     private void configureExtension(String className, String section) {
         Set<String> keys = cfg.getKeys(section);
         List<Pair<String, String>> kvs = new ArrayList<>();
@@ -198,27 +227,24 @@ public class PropertiesAccessor {
         extensions.add(new AsterixExtension(className, kvs));
     }
 
-    private void configureNc(String section, MutableInt uniquePartitionId) {
-        String ncId = AsterixProperties.getSectionId(AsterixProperties.SECTION_PREFIX_NC, section);
+    private void configureNc(String ncId, MutableInt uniquePartitionId) {
 
         // Now we assign the coredump and txnlog directories for this node.
         // QQQ Default values? Should they be specified here? Or should there
         // be a default.ini? Certainly wherever they are, they should be platform-dependent.
-        coredumpConfig.put(ncId, cfg.getString(section, AsterixProperties.PROPERTY_COREDUMP_DIR,
-                AsterixProperties.DEFAULT_COREDUMP_DIR));
+        IApplicationConfig nodeCfg = cfg.getNCEffectiveConfig(ncId);
+        coredumpConfig.put(ncId, nodeCfg.getString(NodeProperties.Option.CORE_DUMP_DIR));
         transactionLogDirs.put(ncId,
-                cfg.getString(section, AsterixProperties.PROPERTY_TXN_LOG_DIR, AsterixProperties.DEFAULT_TXN_LOG_DIR));
+                nodeCfg.getString(NodeProperties.Option.TXN_LOG_DIR));
 
         // Now we create an array of ClusterPartitions for all the partitions
         // on this NC.
-        String[] iodevices = cfg.getString(section, AsterixProperties.PROPERTY_IO_DEV,
-                AsterixProperties.DEFAULT_IO_DEV).split(",");
-        String storageSubdir = cfg.getString(section, AsterixProperties.PROPERTY_STORAGE_DIR,
-                AsterixProperties.DEFAULT_STORAGE_DIR);
+        String[] iodevices = nodeCfg.getStringArray(IODEVICES);
+        String storageSubdir = nodeCfg.getString(STORAGE_SUBDIR);
         String[] nodeStores = new String[iodevices.length];
         ClusterPartition[] nodePartitions = new ClusterPartition[iodevices.length];
         for (int i = 0; i < nodePartitions.length; i++) {
-            // Construct final storage path from iodevice dir + storage subdir.s
+            // Construct final storage path from iodevice dir + storage subdirs
             nodeStores[i] = iodevices[i] + File.separator + storageSubdir;
             // Create ClusterPartition instances for this NC.
             ClusterPartition partition = new ClusterPartition(uniquePartitionId.getValue(), ncId, i);
@@ -228,7 +254,6 @@ public class PropertiesAccessor {
         }
         stores.put(ncId, nodeStores);
         nodePartitionsMap.put(ncId, nodePartitions);
-        nodeNames.add(ncId);
     }
 
     private void loadAsterixBuildProperties() throws AsterixException {
@@ -248,18 +273,10 @@ public class PropertiesAccessor {
         }
     }
 
-    public String getMetadataNodeName() {
-        return metadataNodeName;
-    }
-
     public Map<String, String[]> getStores() {
         return stores;
     }
 
-    public List<String> getNodeNames() {
-        return nodeNames;
-    }
-
     public String getCoredumpPath(String nodeId) {
         return coredumpConfig.get(nodeId);
     }
@@ -276,50 +293,19 @@ public class PropertiesAccessor {
         return asterixBuildProperties;
     }
 
-    public <T> T getProperty(String property, T defaultValue, IPropertyInterpreter<T> interpreter) {
-        String value;
-        Property p = null;
-        if (asterixConfigurationParams != null) {
-            p = asterixConfigurationParams.get(property);
-            value = (p == null) ? null : p.getValue();
-        } else {
-            value = cfg.getString("app", property);
-            if (value == null) {
-                value = cfg.getString("asterix", property);
-                if (value != null) {
-                    LOGGER.warn("[asterix] config section deprecated and will be removed in a future release;" +
-                            " please update to [app] (found: " + property + ')');
-                }
-            }
-        }
-        if (value == null) {
-            return defaultValue;
-        }
+    public <T> T getProperty(String property, T defaultValue, IOptionType<T> interpreter) {
+        String value = cfg.getString("common", property);
         try {
-            return interpreter.interpret(value);
+            return value == null ? defaultValue : interpreter.parse(value);
         } catch (IllegalArgumentException e) {
             if (LOGGER.isEnabledFor(Level.ERROR)) {
-                StringBuilder msg = new StringBuilder(
-                        "Invalid property value '" + value + "' for property '" + property + "'.\n");
-                if (p != null) {
-                    msg.append("See the description: \n" + p.getDescription() + "\n");
-                }
-                msg.append("Default = " + defaultValue);
-                LOGGER.error(msg.toString());
+                LOGGER.error("Invalid property value '" + value + "' for property '" + property + "'.\n" +
+                        "Default = " + defaultValue);
             }
             throw e;
         }
     }
 
-    public String getInstanceName() {
-        return instanceName;
-    }
-
-    public ClusterPartition getMetadataPartition() {
-        // metadata partition is always the first partition on the metadata node
-        return nodePartitionsMap.get(metadataNodeName)[0];
-    }
-
     public Map<String, ClusterPartition[]> getNodePartitions() {
         return nodePartitionsMap;
     }
@@ -333,22 +319,78 @@ public class PropertiesAccessor {
     }
 
     public static PropertiesAccessor getInstance(IApplicationConfig cfg) throws IOException, AsterixException {
-        // Determine whether to use old-style asterix-configuration.xml or new-style configuration.
-        // QQQ strip this out eventually
-        // QQQ this is NOT a good way to determine whether the config is valid
-        PropertiesAccessor propertiesAccessor;
-        if (cfg != null && cfg.getString("cc", "cluster.address") != null) {
-            propertiesAccessor = new PropertiesAccessor(cfg);
-        } else {
-            propertiesAccessor = new PropertiesAccessor();
-        }
-        if (!instanceHolder.compareAndSet(null, propertiesAccessor)) {
-            propertiesAccessor = instanceHolder.get();
+        PropertiesAccessor accessor = instances.get(cfg);
+        if (accessor == null) {
+            accessor = new PropertiesAccessor(cfg);
+            if (instances.putIfAbsent(cfg, accessor) != null) {
+                return instances.get(cfg);
+            }
         }
-        return propertiesAccessor;
+        return accessor;
+    }
+
+    @Override
+    public Object getStatic(IOption option) {
+        return cfg.getStatic(option);
+    }
+
+    @Override
+    public String getString(String section, String key) {
+        return cfg.getString(section, key);
+    }
+
+    @Override
+    public int getInt(String section, String key) {
+        return cfg.getInt(section, key);
+    }
+
+    @Override
+    public long getLong(String section, String key) {
+        return cfg.getLong(section, key);
+    }
+
+    @Override
+    public Set<String> getSectionNames() {
+        return cfg.getSectionNames();
+    }
+
+    @Override
+    public Set<String> getKeys(String section) {
+        return cfg.getKeys(section);
+    }
+
+    @Override
+    public List<String> getNCNames() {
+        return cfg.getNCNames();
+    }
+
+    @Override
+    public IOption lookupOption(String sectionName, String propertyName) {
+        return cfg.lookupOption(sectionName, propertyName);
+    }
+
+    @Override
+    public IApplicationConfig getNCEffectiveConfig(String nodeId) {
+        return cfg.getNCEffectiveConfig(nodeId);
+    }
+
+    @Override
+    public Set<IOption> getOptions() {
+        return cfg.getOptions();
+    }
+
+    @Override
+    public Set<IOption> getOptions(Section section) {
+        return cfg.getOptions(section);
+    }
+
+    @Override
+    public Set<Section> getSections() {
+        return cfg.getSections();
     }
 
-    public static PropertiesAccessor getInstance() throws IOException, AsterixException {
-        return getInstance(null);
+    @Override
+    public Set<Section> getSections(Predicate<Section> predicate) {
+        return cfg.getSections(predicate);
     }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4c7b5bfa/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/PropertyInterpreters.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/PropertyInterpreters.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/PropertyInterpreters.java
deleted file mode 100644
index 6703fcd..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/PropertyInterpreters.java
+++ /dev/null
@@ -1,67 +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.asterix.common.config;
-
-import java.util.logging.Level;
-
-import org.apache.hyracks.util.StorageUtil;
-
-public class PropertyInterpreters {
-
-    public static IPropertyInterpreter<Integer> getIntegerPropertyInterpreter() {
-        return Integer::parseInt;
-    }
-
-    public static IPropertyInterpreter<Boolean> getBooleanPropertyInterpreter() {
-        return Boolean::parseBoolean;
-    }
-
-    public static IPropertyInterpreter<Long> getLongPropertyInterpreter() {
-        return Long::parseLong;
-    }
-
-    public static IPropertyInterpreter<Level> getLevelPropertyInterpreter() {
-        return Level::parse;
-    }
-
-    public static IPropertyInterpreter<String> getStringPropertyInterpreter() {
-        return s -> s;
-    }
-
-    public static IPropertyInterpreter<Double> getDoublePropertyInterpreter() {
-        return Double::parseDouble;
-    }
-
-    public static IPropertyInterpreter<Long> getLongBytePropertyInterpreter() {
-        return StorageUtil::getByteValue;
-    }
-
-    public static IPropertyInterpreter<Integer> getIntegerBytePropertyInterpreter() {
-        return s -> {
-                long result = StorageUtil.getByteValue(s);
-            if (result > Integer.MAX_VALUE || result < Integer.MIN_VALUE) {
-                throw new IllegalArgumentException(
-                        "The given value: " + result + " is not within the int range.");
-            } else {
-                return (int) result;
-            }
-        };
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4c7b5bfa/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ReplicationProperties.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ReplicationProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ReplicationProperties.java
index 116609e..efa5296 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ReplicationProperties.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ReplicationProperties.java
@@ -18,6 +18,9 @@
  */
 package org.apache.asterix.common.config;
 
+import static org.apache.hyracks.control.common.config.OptionTypes.INTEGER;
+import static org.apache.hyracks.control.common.config.OptionTypes.INTEGER_BYTE_UNIT;
+
 import java.util.Set;
 import java.util.stream.Collectors;
 
@@ -25,48 +28,108 @@ import org.apache.asterix.common.replication.IReplicationStrategy;
 import org.apache.asterix.common.replication.Replica;
 import org.apache.asterix.event.schema.cluster.Cluster;
 import org.apache.asterix.event.schema.cluster.Node;
+import org.apache.hyracks.api.config.IApplicationConfig;
+import org.apache.hyracks.api.config.IOption;
+import org.apache.hyracks.api.config.IOptionType;
+import org.apache.hyracks.api.config.Section;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.util.StorageUtil;
 import org.apache.hyracks.util.StorageUtil.StorageUnit;
 
 public class ReplicationProperties extends AbstractProperties {
 
-    private static final int REPLICATION_DATAPORT_DEFAULT = 2000;
+    public enum Option implements IOption {
+        REPLICATION_MAX_REMOTE_RECOVERY_ATTEMPTS(INTEGER, 5),
+        REPLICATION_LOG_BUFFER_PAGESIZE(INTEGER_BYTE_UNIT, StorageUtil.getIntSizeInBytes(128,
+                StorageUnit.KILOBYTE)),
+        REPLICATION_LOG_BUFFER_NUMPAGES(INTEGER, 8),
+        REPLICATION_LOG_BATCHSIZE(INTEGER_BYTE_UNIT, StorageUtil.getIntSizeInBytes(4, StorageUnit.KILOBYTE)),
+        REPLICATION_TIMEOUT(INTEGER, REPLICATION_TIME_OUT_DEFAULT),
+        ;
+
+        private final IOptionType type;
+        private final Object defaultValue;
+
+        Option(IOptionType type, Object defaultValue) {
+            this.type = type;
+            this.defaultValue = defaultValue;
+        }
 
-    private static final String REPLICATION_TIMEOUT_KEY = "replication.timeout";
-    private static final int REPLICATION_TIME_OUT_DEFAULT = 15;
+        @Override
+        public Section section() {
+            return Section.COMMON;
+        }
 
-    private static final String REPLICATION_MAX_REMOTE_RECOVERY_ATTEMPTS_KEY =
-            "replication.max.remote.recovery.attempts";
-    private static final int MAX_REMOTE_RECOVERY_ATTEMPTS = 5;
+        @Override
+        public String description() {
+            // TODO(mblow): add missing descriptions
+            return null;
+        }
 
-    private static final String NODE_IP_ADDRESS_DEFAULT = "127.0.0.1";
+        @Override
+        public IOptionType type() {
+            return type;
+        }
 
-    private static final String REPLICATION_LOG_BATCH_SIZE_KEY = "replication.log.batchsize";
-    private static final int REPLICATION_LOG_BATCH_SIZE_DEFAULT = StorageUtil.getSizeInBytes(4, StorageUnit.KILOBYTE);
+        @Override
+        public Object defaultValue() {
+            return defaultValue;
+        }
 
-    private static final String REPLICATION_LOG_BUFFER_NUM_PAGES_KEY = "replication.log.buffer.numpages";
-    private static final int REPLICATION_LOG_BUFFER_NUM_PAGES_DEFAULT = 8;
+        @Override
+        public Object get(IApplicationConfig config) {
+            switch (this) {
+                case REPLICATION_TIMEOUT:
+                    final Cluster cluster = ClusterProperties.INSTANCE.getCluster();
+                    if (cluster != null
+                            && cluster.getHighAvailability() != null
+                            && cluster.getHighAvailability().getDataReplication() != null
+                            && cluster.getHighAvailability().getDataReplication().getReplicationTimeOut() != null) {
+                        return cluster.getHighAvailability().getDataReplication().getReplicationTimeOut().intValue();
+                    }
+                    return REPLICATION_TIME_OUT_DEFAULT;
+                default:
+                    return config.getStatic(this);
+            }
+        }
+    }
 
-    private static final String REPLICATION_LOG_BUFFER_PAGE_SIZE_KEY = "replication.log.buffer.pagesize";
-    private static final int REPLICATION_LOG_BUFFER_PAGE_SIZE_DEFAULT = StorageUtil.getSizeInBytes(128,
-            StorageUnit.KILOBYTE);
+    private static final int REPLICATION_DATAPORT_DEFAULT = 2000;
+
+    private static final int REPLICATION_TIME_OUT_DEFAULT = 15;
+
+    private static final String NODE_IP_ADDRESS_DEFAULT = "127.0.0.1";
 
-    private final Cluster cluster;
     private final IReplicationStrategy repStrategy;
 
     public ReplicationProperties(PropertiesAccessor accessor) throws HyracksDataException {
         super(accessor);
-        this.cluster = ClusterProperties.INSTANCE.getCluster();
         this.repStrategy = ClusterProperties.INSTANCE.getReplicationStrategy();
     }
 
+    public int getMaxRemoteRecoveryAttempts() {
+        return accessor.getInt(Option.REPLICATION_MAX_REMOTE_RECOVERY_ATTEMPTS);
+    }
+
+    public int getLogBufferPageSize() {
+        return accessor.getInt(Option.REPLICATION_LOG_BUFFER_PAGESIZE);
+    }
+
+    public int getLogBufferNumOfPages() {
+        return accessor.getInt(Option.REPLICATION_LOG_BUFFER_NUMPAGES);
+    }
+
+    public int getLogBatchSize() {
+        return accessor.getInt(Option.REPLICATION_LOG_BATCHSIZE);
+    }
+
     public String getReplicaIPAddress(String nodeId) {
         Node node = ClusterProperties.INSTANCE.getNodeById(nodeId);
         return node != null ? node.getClusterIp() : NODE_IP_ADDRESS_DEFAULT;
     }
 
     public int getDataReplicationPort(String nodeId) {
+        final Cluster cluster = ClusterProperties.INSTANCE.getCluster();
         Node node = ClusterProperties.INSTANCE.getNodeById(nodeId);
         if (node != null) {
             return node.getReplicationPort() != null ? node.getReplicationPort().intValue()
@@ -98,35 +161,8 @@ public class ReplicationProperties extends AbstractProperties {
         return remoteReplicasIds;
     }
 
-    @PropertyKey(REPLICATION_TIMEOUT_KEY)
     public int getReplicationTimeOut() {
-        if (cluster != null) {
-            return cluster.getHighAvailability().getDataReplication().getReplicationTimeOut().intValue();
-        }
-        return REPLICATION_TIME_OUT_DEFAULT;
-    }
-
-    @PropertyKey(REPLICATION_MAX_REMOTE_RECOVERY_ATTEMPTS_KEY)
-    public int getMaxRemoteRecoveryAttempts() {
-        return MAX_REMOTE_RECOVERY_ATTEMPTS;
-    }
-
-    @PropertyKey(REPLICATION_LOG_BUFFER_PAGE_SIZE_KEY)
-    public int getLogBufferPageSize() {
-        return accessor.getProperty(REPLICATION_LOG_BUFFER_PAGE_SIZE_KEY, REPLICATION_LOG_BUFFER_PAGE_SIZE_DEFAULT,
-                PropertyInterpreters.getIntegerBytePropertyInterpreter());
-    }
-
-    @PropertyKey(REPLICATION_LOG_BUFFER_NUM_PAGES_KEY)
-    public int getLogBufferNumOfPages() {
-        return accessor.getProperty(REPLICATION_LOG_BUFFER_NUM_PAGES_KEY, REPLICATION_LOG_BUFFER_NUM_PAGES_DEFAULT,
-                PropertyInterpreters.getIntegerPropertyInterpreter());
-    }
-
-    @PropertyKey(REPLICATION_LOG_BATCH_SIZE_KEY)
-    public int getLogBatchSize() {
-        return accessor.getProperty(REPLICATION_LOG_BATCH_SIZE_KEY, REPLICATION_LOG_BATCH_SIZE_DEFAULT,
-                PropertyInterpreters.getIntegerBytePropertyInterpreter());
+        return accessor.getInt(Option.REPLICATION_TIMEOUT);
     }
 
     public boolean isParticipant(String nodeId) {

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4c7b5bfa/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/StorageProperties.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/StorageProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/StorageProperties.java
index 68ad80c..d363f3d 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/StorageProperties.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/StorageProperties.java
@@ -18,117 +18,159 @@
  */
 package org.apache.asterix.common.config;
 
+import static org.apache.hyracks.control.common.config.OptionTypes.DOUBLE;
+import static org.apache.hyracks.control.common.config.OptionTypes.INTEGER;
+import static org.apache.hyracks.control.common.config.OptionTypes.INTEGER_BYTE_UNIT;
+import static org.apache.hyracks.control.common.config.OptionTypes.LONG_BYTE_UNIT;
 import static org.apache.hyracks.util.StorageUtil.StorageUnit.KILOBYTE;
 
+import java.util.function.Function;
+
+import org.apache.hyracks.api.config.IApplicationConfig;
+import org.apache.hyracks.api.config.IOption;
+import org.apache.hyracks.api.config.IOptionType;
+import org.apache.hyracks.api.config.Section;
 import org.apache.hyracks.storage.common.buffercache.IBufferCache;
 import org.apache.hyracks.util.StorageUtil;
 
 public class StorageProperties extends AbstractProperties {
 
-    private static final String STORAGE_BUFFERCACHE_PAGESIZE_KEY = "storage.buffercache.pagesize";
-    private static final int STORAGE_BUFFERCACHE_PAGESIZE_DEFAULT = StorageUtil.getSizeInBytes(128, KILOBYTE);
-
-    private static final String STORAGE_BUFFERCACHE_SIZE_KEY = "storage.buffercache.size";
-
-    private static final String STORAGE_BUFFERCACHE_MAXOPENFILES_KEY = "storage.buffercache.maxopenfiles";
-    private static final int STORAGE_BUFFERCACHE_MAXOPENFILES_DEFAULT = Integer.MAX_VALUE;
-
-    private static final String STORAGE_MEMORYCOMPONENT_PAGESIZE_KEY = "storage.memorycomponent.pagesize";
-    private static final int STORAGE_MEMORYCOMPONENT_PAGESIZE_DEFAULT = StorageUtil.getSizeInBytes(128, KILOBYTE);
-
-    private static final String STORAGE_MEMORYCOMPONENT_NUMPAGES_KEY = "storage.memorycomponent.numpages";
-
-    private static final String STORAGE_METADATA_MEMORYCOMPONENT_NUMPAGES_KEY =
-            "storage.metadata.memorycomponent.numpages";
-
-    private static final String STORAGE_MEMORYCOMPONENT_NUMCOMPONENTS_KEY = "storage.memorycomponent.numcomponents";
-    private static final int STORAGE_MEMORYCOMPONENT_NUMCOMPONENTS_DEFAULT = 2; // 2 components
-
-    private static final String STORAGE_MEMORYCOMPONENT_GLOBALBUDGET_KEY = "storage.memorycomponent.globalbudget";
-
-    private static final String STORAGE_LSM_BLOOMFILTER_FALSEPOSITIVERATE_KEY =
-            "storage.lsm.bloomfilter.falsepositiverate";
-    private static final double STORAGE_LSM_BLOOMFILTER_FALSEPOSITIVERATE_DEFAULT = 0.01;
-
-    private final long storageBufferCacheSizeDefault;
-    private final int storageMemoryComponentNumPages;
-    private final int storageMetadataMemoryComponentNumPages;
-    private final long storageMemorycomponentGlobalbudgetDefault;
+    public enum Option implements IOption {
+        STORAGE_BUFFERCACHE_PAGESIZE(INTEGER_BYTE_UNIT, StorageUtil.getIntSizeInBytes(128, KILOBYTE)),
+        // By default, uses 1/4 of the maximum heap size for read cache, i.e., disk buffer cache.
+        STORAGE_BUFFERCACHE_SIZE(LONG_BYTE_UNIT, Runtime.getRuntime().maxMemory() / 4),
+        STORAGE_BUFFERCACHE_MAXOPENFILES(INTEGER, Integer.MAX_VALUE),
+        STORAGE_MEMORYCOMPONENT_GLOBALBUDGET(LONG_BYTE_UNIT, Runtime.getRuntime().maxMemory() / 4),
+        STORAGE_MEMORYCOMPONENT_PAGESIZE(INTEGER_BYTE_UNIT, StorageUtil.getIntSizeInBytes(128, KILOBYTE)),
+        STORAGE_MEMORYCOMPONENT_NUMPAGES(INTEGER, (Function<IApplicationConfig, Integer>) accessor ->
+                // By default, uses 1/16 of the STORAGE_MEMORYCOMPONENT_GLOBALBUDGET for the write buffer
+                // budget for a dataset, including data and indexes.
+                (int) (accessor.getLong(STORAGE_MEMORYCOMPONENT_GLOBALBUDGET) /
+                        (16 * accessor.getInt(STORAGE_MEMORYCOMPONENT_PAGESIZE)))),
+        STORAGE_MEMORYCOMPONENT_NUMCOMPONENTS(INTEGER, 2),
+        STORAGE_METADATA_MEMORYCOMPONENT_NUMPAGES(INTEGER, (Function<IApplicationConfig, Integer>) accessor ->
+                // By default, uses the min of 1/64 of the STORAGE_MEMORYCOMPONENT_GLOBALBUDGET and 256 pages
+                // for the write buffer budget for a metadata dataset, including data and indexes.
+                Math.min((int) (accessor.getLong(STORAGE_MEMORYCOMPONENT_GLOBALBUDGET)
+                        / (64 * accessor.getInt(STORAGE_MEMORYCOMPONENT_PAGESIZE))), 256)),
+        STORAGE_LSM_BLOOMFILTER_FALSEPOSITIVERATE(DOUBLE, 0.01d);
+
+        private final IOptionType interpreter;
+        private final Object defaultValue;
+
+        <T> Option(IOptionType<T> interpreter, T defaultValue) {
+            this.interpreter = interpreter;
+            this.defaultValue = defaultValue;
+        }
+
+        <T> Option(IOptionType<T> interpreter, Function<IApplicationConfig, T> defaultValueFunction) {
+            this.interpreter = interpreter;
+            this.defaultValue = defaultValueFunction;
+        }
+
+        @Override
+        public Section section() {
+            return Section.NC;
+        }
+
+        @Override
+        public String description() {
+            switch (this) {
+                case STORAGE_BUFFERCACHE_PAGESIZE:
+                    return "The page size in bytes for pages in the buffer cache";
+                case STORAGE_BUFFERCACHE_SIZE:
+                    return "The size of memory allocated to the disk buffer cache.  The value should be a multiple" +
+                            " of the buffer cache page size.";
+                case STORAGE_BUFFERCACHE_MAXOPENFILES:
+                    return "The maximum number of open files in the buffer cache";
+                case STORAGE_MEMORYCOMPONENT_GLOBALBUDGET:
+                    return "The size of memory allocated to the memory components.  The value should be a multiple " +
+                            "of the memory component page size";
+                case STORAGE_MEMORYCOMPONENT_PAGESIZE:
+                    return "The page size in bytes for pages allocated to memory components";
+                case STORAGE_MEMORYCOMPONENT_NUMPAGES:
+                    return "The number of pages to allocate for a memory component.  This budget is shared by all " +
+                            "the memory components of the primary index and all its secondary indexes across all I/O " +
+                            "devices on a node.  Note: in-memory components usually has fill factor of 75% since " +
+                            "the pages are 75% full and the remaining 25% is un-utilized";
+                case STORAGE_MEMORYCOMPONENT_NUMCOMPONENTS:
+                    return "The number of memory components to be used per lsm index";
+                case STORAGE_METADATA_MEMORYCOMPONENT_NUMPAGES:
+                    return "The number of pages to allocate for a metadata memory component";
+                case STORAGE_LSM_BLOOMFILTER_FALSEPOSITIVERATE:
+                    return "The maximum acceptable false positive rate for bloom filters associated with LSM indexes";
+                default:
+                    throw new IllegalStateException("NYI: " + this);
+            }
+        }
+
+        @Override
+        public IOptionType type() {
+            return interpreter;
+        }
+
+        @Override
+        public Object defaultValue() {
+            return defaultValue;
+        }
+
+
+        @Override
+        public String usageDefaultOverride(IApplicationConfig accessor, Function<IOption, String> optionPrinter) {
+            switch (this) {
+                case STORAGE_MEMORYCOMPONENT_NUMPAGES:
+                    return "1/16th of the " + optionPrinter.apply(Option.STORAGE_MEMORYCOMPONENT_GLOBALBUDGET) +
+                            " value";
+                case STORAGE_METADATA_MEMORYCOMPONENT_NUMPAGES:
+                    return "1/64th of the " + optionPrinter.apply(Option.STORAGE_MEMORYCOMPONENT_GLOBALBUDGET) +
+                            " value or 256, whichever is larger";
+                default:
+                    return null;
+            }
+        }
+    }
 
     public StorageProperties(PropertiesAccessor accessor) {
         super(accessor);
-
-        // Gets the -Xmx value for the JVM.
-        long maxHeapSize = Runtime.getRuntime().maxMemory();
-        // By default, uses 1/4 of the maximum heap size for read cache, i.e., disk buffer cache.
-        storageBufferCacheSizeDefault = maxHeapSize / 4;
-        // By default, uses 1/4 of the maximum heap size for the write buffer, i.e., globalbudget for memory components.
-        storageMemorycomponentGlobalbudgetDefault = maxHeapSize / 4;
-        // By default, uses 1/16 of the storageMemorycomponentGlobalbudgetDefault for the write buffer budget
-        // for a dataset, including data and indexes.
-        storageMemoryComponentNumPages = (int) (storageMemorycomponentGlobalbudgetDefault
-                / (16 * getMemoryComponentPageSize()));
-        // By default, uses the min of 1/64 of the storageMemorycomponentGlobalbudgetDefault and 256 pages
-        // for the write buffer budget for a metadata dataset, including data and indexes.
-        storageMetadataMemoryComponentNumPages = Math
-                .min((int) (storageMemorycomponentGlobalbudgetDefault / (64 * getMemoryComponentPageSize())), 256);
     }
 
-    @PropertyKey(STORAGE_BUFFERCACHE_PAGESIZE_KEY)
     public int getBufferCachePageSize() {
-        return accessor.getProperty(STORAGE_BUFFERCACHE_PAGESIZE_KEY, STORAGE_BUFFERCACHE_PAGESIZE_DEFAULT,
-                PropertyInterpreters.getIntegerBytePropertyInterpreter());
+        return accessor.getInt(Option.STORAGE_BUFFERCACHE_PAGESIZE);
     }
 
-    @PropertyKey(STORAGE_BUFFERCACHE_SIZE_KEY)
     public long getBufferCacheSize() {
-        return accessor.getProperty(STORAGE_BUFFERCACHE_SIZE_KEY, storageBufferCacheSizeDefault,
-                PropertyInterpreters.getLongBytePropertyInterpreter());
+        return accessor.getLong(Option.STORAGE_BUFFERCACHE_SIZE);
     }
 
-    public int getBufferCacheNumPages() {
-        return (int) (getBufferCacheSize() / (getBufferCachePageSize() + IBufferCache.RESERVED_HEADER_BYTES));
-    }
-
-    @PropertyKey(STORAGE_BUFFERCACHE_MAXOPENFILES_KEY)
     public int getBufferCacheMaxOpenFiles() {
-        return accessor.getProperty(STORAGE_BUFFERCACHE_MAXOPENFILES_KEY, STORAGE_BUFFERCACHE_MAXOPENFILES_DEFAULT,
-                PropertyInterpreters.getIntegerPropertyInterpreter());
+        return accessor.getInt(Option.STORAGE_BUFFERCACHE_MAXOPENFILES);
     }
 
-    @PropertyKey(STORAGE_MEMORYCOMPONENT_PAGESIZE_KEY)
     public int getMemoryComponentPageSize() {
-        return accessor.getProperty(STORAGE_MEMORYCOMPONENT_PAGESIZE_KEY, STORAGE_MEMORYCOMPONENT_PAGESIZE_DEFAULT,
-                PropertyInterpreters.getIntegerBytePropertyInterpreter());
+        return accessor.getInt(Option.STORAGE_MEMORYCOMPONENT_PAGESIZE);
     }
 
-    @PropertyKey(STORAGE_MEMORYCOMPONENT_NUMPAGES_KEY)
     public int getMemoryComponentNumPages() {
-        return accessor.getProperty(STORAGE_MEMORYCOMPONENT_NUMPAGES_KEY, storageMemoryComponentNumPages,
-                PropertyInterpreters.getIntegerPropertyInterpreter());
+        return accessor.getInt(Option.STORAGE_MEMORYCOMPONENT_NUMPAGES);
     }
 
-    @PropertyKey(STORAGE_METADATA_MEMORYCOMPONENT_NUMPAGES_KEY)
     public int getMetadataMemoryComponentNumPages() {
-        return accessor.getProperty(STORAGE_METADATA_MEMORYCOMPONENT_NUMPAGES_KEY,
-                storageMetadataMemoryComponentNumPages, PropertyInterpreters.getIntegerPropertyInterpreter());
+        return accessor.getInt(Option.STORAGE_METADATA_MEMORYCOMPONENT_NUMPAGES);
     }
 
-    @PropertyKey(STORAGE_MEMORYCOMPONENT_NUMCOMPONENTS_KEY)
     public int getMemoryComponentsNum() {
-        return accessor.getProperty(STORAGE_MEMORYCOMPONENT_NUMCOMPONENTS_KEY,
-                STORAGE_MEMORYCOMPONENT_NUMCOMPONENTS_DEFAULT, PropertyInterpreters.getIntegerPropertyInterpreter());
+        return accessor.getInt(Option.STORAGE_MEMORYCOMPONENT_NUMCOMPONENTS);
     }
 
-    @PropertyKey(STORAGE_MEMORYCOMPONENT_GLOBALBUDGET_KEY)
     public long getMemoryComponentGlobalBudget() {
-        return accessor.getProperty(STORAGE_MEMORYCOMPONENT_GLOBALBUDGET_KEY, storageMemorycomponentGlobalbudgetDefault,
-                PropertyInterpreters.getLongBytePropertyInterpreter());
+        return accessor.getLong(Option.STORAGE_MEMORYCOMPONENT_GLOBALBUDGET);
     }
 
-    @PropertyKey(STORAGE_LSM_BLOOMFILTER_FALSEPOSITIVERATE_KEY)
     public double getBloomFilterFalsePositiveRate() {
-        return accessor.getProperty(STORAGE_LSM_BLOOMFILTER_FALSEPOSITIVERATE_KEY,
-                STORAGE_LSM_BLOOMFILTER_FALSEPOSITIVERATE_DEFAULT, PropertyInterpreters.getDoublePropertyInterpreter());
+        return accessor.getDouble(Option.STORAGE_LSM_BLOOMFILTER_FALSEPOSITIVERATE);
+    }
+
+    public int getBufferCacheNumPages() {
+        return (int) (getBufferCacheSize() / (getBufferCachePageSize() + IBufferCache.RESERVED_HEADER_BYTES));
     }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4c7b5bfa/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/TransactionProperties.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/TransactionProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/TransactionProperties.java
index e4fd71d..73e8c4a 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/TransactionProperties.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/TransactionProperties.java
@@ -18,50 +18,65 @@
  */
 package org.apache.asterix.common.config;
 
-import static org.apache.hyracks.util.StorageUtil.StorageUnit.KILOBYTE;
-import static org.apache.hyracks.util.StorageUtil.StorageUnit.MEGABYTE;
+import static org.apache.hyracks.control.common.config.OptionTypes.*;
+import static org.apache.hyracks.util.StorageUtil.StorageUnit.*;
 
 import java.util.Map;
 
+import org.apache.hyracks.api.config.IOption;
+import org.apache.hyracks.api.config.IOptionType;
+import org.apache.hyracks.api.config.Section;
 import org.apache.hyracks.util.StorageUtil;
 
 public class TransactionProperties extends AbstractProperties {
 
-    private static final String TXN_LOG_BUFFER_NUMPAGES_KEY = "txn.log.buffer.numpages";
-    private static final int TXN_LOG_BUFFER_NUMPAGES_DEFAULT = 8;
-
-    private static final String TXN_LOG_BUFFER_PAGESIZE_KEY = "txn.log.buffer.pagesize";
-    private static final int TXN_LOG_BUFFER_PAGESIZE_DEFAULT = StorageUtil.getSizeInBytes(128, KILOBYTE);
-
-    public static final String TXN_LOG_PARTITIONSIZE_KEY = "txn.log.partitionsize";
-    private static final long TXN_LOG_PARTITIONSIZE_DEFAULT = StorageUtil.getSizeInBytes(256L, MEGABYTE);
-
-    private static final String TXN_LOG_CHECKPOINT_LSNTHRESHOLD_KEY = "txn.log.checkpoint.lsnthreshold";
-    private static final int TXN_LOG_CHECKPOINT_LSNTHRESHOLD_DEFAULT = StorageUtil.getSizeInBytes(64, MEGABYTE);
-
-    public static final String TXN_LOG_CHECKPOINT_POLLFREQUENCY_KEY = "txn.log.checkpoint.pollfrequency";
-    private static final int TXN_LOG_CHECKPOINT_POLLFREQUENCY_DEFAULT = 120; // 120s
-
-    private static final String TXN_LOG_CHECKPOINT_HISTORY_KEY = "txn.log.checkpoint.history";
-    private static final int TXN_LOG_CHECKPOINT_HISTORY_DEFAULT = 0;
-
-    private static final String TXN_LOCK_ESCALATIONTHRESHOLD_KEY = "txn.lock.escalationthreshold";
-    private static final int TXN_LOCK_ESCALATIONTHRESHOLD_DEFAULT = 1000;
-
-    private static final String TXN_LOCK_SHRINKTIMER_KEY = "txn.lock.shrinktimer";
-    private static final int TXN_LOCK_SHRINKTIMER_DEFAULT = 5000; // 5s
-
-    private static final String TXN_LOCK_TIMEOUT_WAITTHRESHOLD_KEY = "txn.lock.timeout.waitthreshold";
-    private static final int TXN_LOCK_TIMEOUT_WAITTHRESHOLD_DEFAULT = 60000; // 60s
-
-    private static final String TXN_LOCK_TIMEOUT_SWEEPTHRESHOLD_KEY = "txn.lock.timeout.sweepthreshold";
-    private static final int TXN_LOCK_TIMEOUT_SWEEPTHRESHOLD_DEFAULT = 10000; // 10s
+    public enum Option implements IOption {
+        TXN_LOG_BUFFER_NUMPAGES(INTEGER, 8),
+        TXN_LOG_BUFFER_PAGESIZE(INTEGER_BYTE_UNIT, StorageUtil.getIntSizeInBytes(128, KILOBYTE)),
+        TXN_LOG_PARTITIONSIZE(LONG_BYTE_UNIT, StorageUtil.getLongSizeInBytes(256L, MEGABYTE)),
+        TXN_LOG_CHECKPOINT_LSNTHRESHOLD(INTEGER_BYTE_UNIT, StorageUtil.getIntSizeInBytes(64, MEGABYTE)),
+        TXN_LOG_CHECKPOINT_POLLFREQUENCY(INTEGER, 120),
+        TXN_LOG_CHECKPOINT_HISTORY(INTEGER, 0),
+        TXN_LOCK_ESCALATIONTHRESHOLD(INTEGER, 1000),
+        TXN_LOCK_SHRINKTIMER(INTEGER, 5000),
+        TXN_LOCK_TIMEOUT_WAITTHRESHOLD(INTEGER, 60000),
+        TXN_LOCK_TIMEOUT_SWEEPTHRESHOLD(INTEGER, 10000),
+        TXN_COMMITPROFILER_REPORTINTERVAL(INTEGER, 5),
+        TXN_JOB_RECOVERY_MEMORYSIZE(LONG_BYTE_UNIT, StorageUtil.getLongSizeInBytes(64L, MEGABYTE));
+
+        private final IOptionType type;
+        private final Object defaultValue;
+
+        Option(IOptionType type, Object defaultValue) {
+            this.type = type;
+            this.defaultValue = defaultValue;
+        }
+
+        @Override
+        public Section section() {
+            return Section.COMMON;
+        }
+
+        @Override
+        public String description() {
+            // TODO(mblow): add missing descriptions
+            return null;
+        }
+
+        @Override
+        public IOptionType type() {
+            return type;
+        }
+
+        @Override
+        public Object defaultValue() {
+            return defaultValue;
+        }
+    }
 
-    private static final String TXN_COMMIT_PROFILER_REPORT_INTERVAL_KEY = "txn.commitprofiler.reportinterval";
-    private static final int TXN_COMMIT_PROFILER_REPORT_INTERVAL_DEFAULT = 5; // 5 seconds
+    public static final String TXN_LOG_PARTITIONSIZE_KEY = Option.TXN_LOG_PARTITIONSIZE.ini();
 
-    private static final String TXN_JOB_RECOVERY_MEMORY_SIZE_KEY = "txn.job.recovery.memorysize";
-    private static final long TXN_JOB_RECOVERY_MEMORY_SIZE_DEFAULT = StorageUtil.getSizeInBytes(64L, MEGABYTE);
+    public static final String TXN_LOG_CHECKPOINT_POLLFREQUENCY_KEY = Option.TXN_LOG_CHECKPOINT_POLLFREQUENCY.ini();
 
     public TransactionProperties(PropertiesAccessor accessor) {
         super(accessor);
@@ -75,75 +90,51 @@ public class TransactionProperties extends AbstractProperties {
         return accessor.getTransactionLogDirs();
     }
 
-    @PropertyKey(TXN_LOG_BUFFER_NUMPAGES_KEY)
     public int getLogBufferNumPages() {
-        return accessor.getProperty(TXN_LOG_BUFFER_NUMPAGES_KEY, TXN_LOG_BUFFER_NUMPAGES_DEFAULT,
-                PropertyInterpreters.getIntegerPropertyInterpreter());
+        return accessor.getInt(Option.TXN_LOG_BUFFER_NUMPAGES);
     }
 
-    @PropertyKey(TXN_LOG_BUFFER_PAGESIZE_KEY)
     public int getLogBufferPageSize() {
-        return accessor.getProperty(TXN_LOG_BUFFER_PAGESIZE_KEY, TXN_LOG_BUFFER_PAGESIZE_DEFAULT,
-                PropertyInterpreters.getIntegerBytePropertyInterpreter());
+        return accessor.getInt(Option.TXN_LOG_BUFFER_PAGESIZE);
     }
 
-    @PropertyKey(TXN_LOG_PARTITIONSIZE_KEY)
     public long getLogPartitionSize() {
-        return accessor.getProperty(TXN_LOG_PARTITIONSIZE_KEY, TXN_LOG_PARTITIONSIZE_DEFAULT,
-                PropertyInterpreters.getLongBytePropertyInterpreter());
+        return accessor.getLong(Option.TXN_LOG_PARTITIONSIZE);
     }
 
-    @PropertyKey(TXN_LOG_CHECKPOINT_LSNTHRESHOLD_KEY)
     public int getCheckpointLSNThreshold() {
-        return accessor.getProperty(TXN_LOG_CHECKPOINT_LSNTHRESHOLD_KEY, TXN_LOG_CHECKPOINT_LSNTHRESHOLD_DEFAULT,
-                PropertyInterpreters.getIntegerPropertyInterpreter());
+        return accessor.getInt(Option.TXN_LOG_CHECKPOINT_LSNTHRESHOLD);
     }
 
-    @PropertyKey(TXN_LOG_CHECKPOINT_POLLFREQUENCY_KEY)
     public int getCheckpointPollFrequency() {
-        return accessor.getProperty(TXN_LOG_CHECKPOINT_POLLFREQUENCY_KEY, TXN_LOG_CHECKPOINT_POLLFREQUENCY_DEFAULT,
-                PropertyInterpreters.getIntegerPropertyInterpreter());
+        return accessor.getInt(Option.TXN_LOG_CHECKPOINT_POLLFREQUENCY);
     }
 
-    @PropertyKey(TXN_LOG_CHECKPOINT_HISTORY_KEY)
     public int getCheckpointHistory() {
-        return accessor.getProperty(TXN_LOG_CHECKPOINT_HISTORY_KEY, TXN_LOG_CHECKPOINT_HISTORY_DEFAULT,
-                PropertyInterpreters.getIntegerPropertyInterpreter());
+        return accessor.getInt(Option.TXN_LOG_CHECKPOINT_HISTORY);
     }
 
-    @PropertyKey(TXN_LOCK_ESCALATIONTHRESHOLD_KEY)
     public int getEntityToDatasetLockEscalationThreshold() {
-        return accessor.getProperty(TXN_LOCK_ESCALATIONTHRESHOLD_KEY, TXN_LOCK_ESCALATIONTHRESHOLD_DEFAULT,
-                PropertyInterpreters.getIntegerPropertyInterpreter());
+        return accessor.getInt(Option.TXN_LOCK_ESCALATIONTHRESHOLD);
     }
 
-    @PropertyKey(TXN_LOCK_SHRINKTIMER_KEY)
     public int getLockManagerShrinkTimer() {
-        return accessor.getProperty(TXN_LOCK_SHRINKTIMER_KEY, TXN_LOCK_SHRINKTIMER_DEFAULT,
-                PropertyInterpreters.getIntegerPropertyInterpreter());
+        return accessor.getInt(Option.TXN_LOCK_SHRINKTIMER);
     }
 
-    @PropertyKey(TXN_LOCK_TIMEOUT_WAITTHRESHOLD_KEY)
     public int getTimeoutWaitThreshold() {
-        return accessor.getProperty(TXN_LOCK_TIMEOUT_WAITTHRESHOLD_KEY, TXN_LOCK_TIMEOUT_WAITTHRESHOLD_DEFAULT,
-                PropertyInterpreters.getIntegerPropertyInterpreter());
+        return accessor.getInt(Option.TXN_LOCK_TIMEOUT_WAITTHRESHOLD);
     }
 
-    @PropertyKey(TXN_LOCK_TIMEOUT_SWEEPTHRESHOLD_KEY)
     public int getTimeoutSweepThreshold() {
-        return accessor.getProperty(TXN_LOCK_TIMEOUT_SWEEPTHRESHOLD_KEY, TXN_LOCK_TIMEOUT_SWEEPTHRESHOLD_DEFAULT,
-                PropertyInterpreters.getIntegerPropertyInterpreter());
+        return accessor.getInt(Option.TXN_LOCK_TIMEOUT_SWEEPTHRESHOLD);
     }
 
-    @PropertyKey(TXN_COMMIT_PROFILER_REPORT_INTERVAL_KEY)
     public int getCommitProfilerReportInterval() {
-        return accessor.getProperty(TXN_COMMIT_PROFILER_REPORT_INTERVAL_KEY,
-                TXN_COMMIT_PROFILER_REPORT_INTERVAL_DEFAULT, PropertyInterpreters.getIntegerPropertyInterpreter());
+        return accessor.getInt(Option.TXN_COMMITPROFILER_REPORTINTERVAL);
     }
 
-    @PropertyKey(TXN_JOB_RECOVERY_MEMORY_SIZE_KEY)
     public long getJobRecoveryMemorySize() {
-        return accessor.getProperty(TXN_JOB_RECOVERY_MEMORY_SIZE_KEY, TXN_JOB_RECOVERY_MEMORY_SIZE_DEFAULT,
-                PropertyInterpreters.getLongBytePropertyInterpreter());
+        return accessor.getLong(Option.TXN_JOB_RECOVERY_MEMORYSIZE);
     }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4c7b5bfa/asterixdb/asterix-common/src/main/resources/schema/cluster.xsd
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-common/src/main/resources/schema/cluster.xsd b/asterixdb/asterix-common/src/main/resources/schema/cluster.xsd
index 098b4e7..8cd7b42 100644
--- a/asterixdb/asterix-common/src/main/resources/schema/cluster.xsd
+++ b/asterixdb/asterix-common/src/main/resources/schema/cluster.xsd
@@ -52,7 +52,6 @@
     <xs:element name="heartbeat_period" type="xs:integer" />
     <xs:element name="max_heartbeat_lapse_periods" type="xs:integer" />
     <xs:element name="profile_dump_period" type="xs:integer" />
-    <xs:element name="default_max_job_attempts" type="xs:integer" />
     <xs:element name="job_history_size" type="xs:integer" />
     <xs:element name="result_time_to_live" type="xs:long" />
     <xs:element name="result_sweep_threshold" type="xs:long" />
@@ -185,7 +184,6 @@
                 <xs:element ref="cl:heartbeat_period" minOccurs="0" />
                 <xs:element ref="cl:max_heartbeat_lapse_periods" minOccurs="0" />
                 <xs:element ref="cl:profile_dump_period" minOccurs="0" />
-                <xs:element ref="cl:default_max_job_attempts" minOccurs="0" />
                 <xs:element ref="cl:job_history_size" minOccurs="0" />
                 <xs:element ref="cl:result_time_to_live" minOccurs="0" />
                 <xs:element ref="cl:result_sweep_threshold" minOccurs="0" />

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4c7b5bfa/asterixdb/asterix-doc/src/site/markdown/ncservice.md
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-doc/src/site/markdown/ncservice.md b/asterixdb/asterix-doc/src/site/markdown/ncservice.md
index 3215183..67b1970 100644
--- a/asterixdb/asterix-doc/src/site/markdown/ncservice.md
+++ b/asterixdb/asterix-doc/src/site/markdown/ncservice.md
@@ -126,25 +126,25 @@ The second configuration file is
 `cc.conf`:
 
     [nc/red]
-    txnlogdir=/tmp/asterix/red/txnlog
-    coredumpdir=/tmp/asterix/red/coredump
+    txn.log.dir=/tmp/asterix/red/txnlog
+    core.dump.dir=/tmp/asterix/red/coredump
     iodevices=/tmp/asterix/red
 
     [nc/blue]
     port=9091
-    txnlogdir=/tmp/asterix/blue/txnlog
-    coredumpdir=/tmp/asterix/blue/coredump
+    txn.log.dir=/tmp/asterix/blue/txnlog
+    core.dump.dir=/tmp/asterix/blue/coredump
     iodevices=/tmp/asterix/blue
 
     [nc]
     app.class=org.apache.asterix.hyracks.bootstrap.NCApplicationEntryPoint
-    storagedir=storage
+    storage.subdir=storage
     address=127.0.0.1
     command=asterixnc
 
     [cc]
-    cluster.address = 127.0.0.1
-    http.port = 12345
+    address = 127.0.0.1
+    console.listen.port = 12345
 
 This is the configuration file for the cluster and it contains information
 that each `NCService` will use when starting the corresponding `NCDriver` as
@@ -257,10 +257,8 @@ The following parameters are for the master process, under the "[cc]" section.
 
 | Parameter | Meaning |  Default |
 |----------|--------|-------|
-| instance.name  |  The name of the AsterixDB instance   | "DEFAULT_INSTANCE" |
 | max.wait.active.cluster | The max pending time (in seconds) for cluster startup. After the threshold, if the cluster still is not up and running, it is considered unavailable.    | 60 |
-| metadata.callback.port | The port for metadata communication | 0 |
-| cluster.address | The binding IP address for the AsterixDB instance | N/A |
+| address | The binding IP address for the AsterixDB instance | N/A |
 
 
 The following parameters for slave processes, under "[nc]" sections.
@@ -269,13 +267,28 @@ The following parameters for slave processes, under "[nc]" sections.
 |----------|--------|-------|
 | address | The binding IP address for the slave process |  N/A   |
 | command | The command for the slave process | N/A (for AsterixDB, it should be "asterixnc") |
-| coredumpdir | The path for core dump | N/A |
+| core.dump.dir | The path for core dump | N/A |
 | iodevices | Comma separated directory paths for both storage files and temporary files | N/A |
 | jvm.args | The JVM arguments | -Xmx1536m |
-| metadata.port | The metadata communication port on the metadata node. This parameter should only be present in the section of the metadata NC | 0 |
+| ncservice.port | The port on which the NCService for this NC is listening | 9090 |
+| txn.log.dir  | The directory for transaction logs | N/A |
+
+
+The following parameters are configured under the "[common]" section.
+
+| Parameter | Meaning |  Default |
+|----------|--------|-------|
+| instance.name  |  The name of the AsterixDB instance   | "DEFAULT_INSTANCE" |
+| log.level | The logging level for master and slave processes | "INFO" |
+| compiler.framesize |  The page size (in bytes) for computation  | 32768 |
+| compiler.groupmemory |  The memory budget (in bytes) for a group by operator instance in a partition | 33554432 |
+| compiler.joinmemory | The memory budget (in bytes) for a join operator instance in a partition | 33554432 |
+| compiler.sortmemory | The memory budget (in bytes) for a sort operator instance in a partition | 33554432 |
+| compiler.parallelism | The degree of parallelism for query execution. Zero means to use the storage parallelism as the query execution parallelism, while other integer values dictate the number of query execution parallel partitions. The system will fall back to use the number of all available CPU cores in the cluster as the degree of parallelism if the number set by a user is too large or too small.  | 0 |
+| metadata.callback.port | The port for metadata communication | 0 |
+| metadata.listen.port | The metadata communication port on the metadata node. This parameter should only be present in the section of the metadata NC | 0 |
 | metadata.registration.timeout.secs | The time out threshold (in seconds) for metadata node registration | 60 |
-| port | The port for the NCService that starts the slave process |  N/A |
-| storagedir | The directory for storage files  |  N/A |
+| storage.subdir | The directory for storage files  |  N/A |
 | storage.buffercache.maxopenfiles | The maximum number of open files for the buffer cache.  Note that this is the parameter for the AsterixDB and setting the operating system parameter is still required. | 2147483647 |
 | storage.buffercache.pagesize |  The page size (in bytes) for the disk buffer cache (for reads) | 131072 |
 | storage.buffercache.size | The overall budget (in bytes) of the disk buffer cache (for reads) | 536870912 |
@@ -285,7 +298,6 @@ The following parameters for slave processes, under "[nc]" sections.
 | storage.memorycomponent.numpages | The number of pages for all memory components of a dataset, including those for secondary indexes | 256 |
 | storage.memorycomponent.pagesize | The page size (in bytes) of memory components | 131072 |
 | storage.metadata.memorycomponent.numpages | The number of pages for all memory components of a metadata dataset | 256 |
-| txnlogdir  | The directory for transaction logs | N/A |
 | txn.commitprofiler.reportinterval |  The interval for reporting commit statistics | 5 |
 | txn.job.recovery.memorysize  | The memory budget (in bytes) used for recovery | 67108864 |
 | txn.lock.timeout.sweepthreshold | Interval (in milliseconds) for checking lock timeout | 10000 |
@@ -296,13 +308,10 @@ The following parameters for slave processes, under "[nc]" sections.
 | txn.log.checkpoint.lsnthreshold | The checkpoint threshold (in terms of LSNs (log sequence numbers) that have been written to the transaction log, i.e., the length of the transaction log) for transection logs | 67108864 |
 
 
-The following parameter is for both master and slave processes, under the "[app]" section.
+# For the optional NCService process configuration file, the following parameters, under "[ncservice]" section.
 
 | Parameter | Meaning |  Default |
 |----------|--------|-------|
-| log.level | The logging level for master and slave processes | "INFO" |
-| compiler.framesize |  The page size (in bytes) for computation  | 32768 |
-| compiler.groupmemory |  The memory budget (in bytes) for a group by operator instance in a partition | 33554432 |
-| compiler.joinmemory | The memory budget (in bytes) for a join operator instance in a partition  | 33554432 |
-| compiler.sortmemory | The memory budget (in bytes) for a sort operator instance in a partition | 33554432 |
-| compiler.parallelism | The degree of parallelism for query execution. Zero means to use the storage parallelism as the query execution parallelism, while other integer values dictate the number of query execution parallel partitions. The system will fall back to use the number of all available CPU cores in the cluster as the degree of parallelism if the number set by a user is too large or too small.  | 0 |
+| address | The address the NCService listens on for commands from the CC | (all addresses) |
+| port | The port for the NCService listens on for commands from the CC | 9090 |
+| logdir | Directory where NCService logs should be written ('-' indicates that output should go to stdout) | ${app.home}/logs (${user.home} if 'app.home' not present in NCService Java system properties. |

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4c7b5bfa/asterixdb/asterix-docker/docker/supervisord.conf
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-docker/docker/supervisord.conf b/asterixdb/asterix-docker/docker/supervisord.conf
index 9f70dd1..b6ce1e0 100644
--- a/asterixdb/asterix-docker/docker/supervisord.conf
+++ b/asterixdb/asterix-docker/docker/supervisord.conf
@@ -19,21 +19,21 @@
 nodaemon=true
 
 [program:asterixcc]
-command=/asterixdb/bin/asterixcc -cluster-net-ip-address localhost -client-net-ip-address localhost
+command=/asterixdb/bin/asterixcc -address localhost -client-listen-address localhost
 stdout_logfile=/dev/stdout
 stdout_logfile_maxbytes=0
 stderr_logfile=/dev/stderr
 stderr_logfile_maxbytes=0
 
 [program:asterixnc1]
-command=/asterixdb/bin/asterixnc -node-id nc1 -cc-host localhost -iodevices /asterixdb/nc1 -cluster-net-ip-address localhost -data-ip-address localhost -result-ip-address localhost --
+command=/asterixdb/bin/asterixnc -node-id nc1 -cluster-address localhost -iodevices /asterixdb/nc1 -address localhost -data-listen-address localhost -result-listen-address localhost
 stdout_logfile=/dev/stdout
 stdout_logfile_maxbytes=0
 stderr_logfile=/dev/stderr
 stderr_logfile_maxbytes=0
 
 [program:asterixnc2]
-command=/asterixdb/bin/asterixnc -node-id nc2 -cc-host localhost -iodevices /asterixdb/nc2 -cluster-net-ip-address localhost -data-ip-address localhost -result-ip-address localhost --
+command=/asterixdb/bin/asterixnc -node-id nc2 -cluster-address localhost -iodevices /asterixdb/nc2 -address localhost -data-listen-address localhost -result-listen-address localhost
 stdout_logfile=/dev/stdout
 stdout_logfile_maxbytes=0
 stderr_logfile=/dev/stderr

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4c7b5bfa/asterixdb/asterix-events/src/main/java/org/apache/asterix/event/service/AsterixEventServiceUtil.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-events/src/main/java/org/apache/asterix/event/service/AsterixEventServiceUtil.java b/asterixdb/asterix-events/src/main/java/org/apache/asterix/event/service/AsterixEventServiceUtil.java
index b6cc339..b48e311 100644
--- a/asterixdb/asterix-events/src/main/java/org/apache/asterix/event/service/AsterixEventServiceUtil.java
+++ b/asterixdb/asterix-events/src/main/java/org/apache/asterix/event/service/AsterixEventServiceUtil.java
@@ -23,7 +23,6 @@ import java.io.BufferedOutputStream;
 import java.io.File;
 import java.io.FileFilter;
 import java.io.FileInputStream;
-import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
 import java.io.FileWriter;
 import java.io.IOException;
@@ -157,10 +156,6 @@ public class AsterixEventServiceUtil {
             clusterProperties.add(
                     new Property("PROFILE_DUMP_PERIOD", String.valueOf(cluster.getProfileDumpPeriod().intValue())));
         }
-        if (cluster.getDefaultMaxJobAttempts() != null) {
-            clusterProperties.add(new Property("DEFAULT_MAX_JOB_ATTEMPTS",
-                    String.valueOf(cluster.getDefaultMaxJobAttempts().intValue())));
-        }
         if (cluster.getJobHistorySize() != null) {
             clusterProperties
                     .add(new Property("JOB_HISTORY_SIZE", String.valueOf(cluster.getJobHistorySize().intValue())));
@@ -288,15 +283,13 @@ public class AsterixEventServiceUtil {
         }
         configuration.setStore(stores);
         List<Coredump> coredump = new ArrayList<Coredump>();
-        String coredumpDir = null;
         List<TransactionLogDir> txnLogDirs = new ArrayList<TransactionLogDir>();
-        String txnLogDir = null;
         for (Node node : cluster.getNode()) {
-            coredumpDir = node.getLogDir() == null ? cluster.getLogDir() : node.getLogDir();
+            String coredumpdir = node.getLogDir() == null ? cluster.getLogDir() : node.getLogDir();
             coredump.add(new Coredump(asterixInstanceName + "_" + node.getId(),
-                    coredumpDir + File.separator + asterixInstanceName + "_" + node.getId()));
+                    coredumpdir + File.separator + asterixInstanceName + "_" + node.getId()));
 
-            txnLogDir = node.getTxnLogDir() == null ? cluster.getTxnLogDir() : node.getTxnLogDir();
+            String txnLogDir = node.getTxnLogDir() == null ? cluster.getTxnLogDir() : node.getTxnLogDir();
             txnLogDirs.add(new TransactionLogDir(asterixInstanceName + "_" + node.getId(), txnLogDir));
         }
         configuration.setCoredump(coredump);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4c7b5bfa/asterixdb/asterix-events/src/main/resources/events/cc_start/cc_start.sh
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-events/src/main/resources/events/cc_start/cc_start.sh b/asterixdb/asterix-events/src/main/resources/events/cc_start/cc_start.sh
index c33ba6c..3950547 100644
--- a/asterixdb/asterix-events/src/main/resources/events/cc_start/cc_start.sh
+++ b/asterixdb/asterix-events/src/main/resources/events/cc_start/cc_start.sh
@@ -18,14 +18,14 @@
 # under the License.
 
 if [ ! -d $LOG_DIR ];
-then 
+then
   mkdir -p $LOG_DIR
 fi
-ccArgs='-client-net-ip-address '$CLIENT_NET_IP
-ccArgs=$ccArgs' -client-net-port '$CLIENT_NET_PORT
-ccArgs=$ccArgs' -cluster-net-ip-address '$CLUSTER_NET_IP
-ccArgs=$ccArgs' -cluster-net-port '$CLUSTER_NET_PORT
-ccArgs=$ccArgs' -http-port '$HTTP_PORT
+ccArgs='-client-listen-address '$CLIENT_NET_IP
+ccArgs=$ccArgs' -client-listen-port '$CLIENT_NET_PORT
+ccArgs=$ccArgs' -address '$CLUSTER_NET_IP
+ccArgs=$ccArgs' -cluster-listen-port '$CLUSTER_NET_PORT
+ccArgs=$ccArgs' -console-listen-port '$HTTP_PORT
 
 if [ ! -z $HEARTBEAT_PERIOD ]
 then
@@ -34,7 +34,7 @@ fi
 
 if [ ! -z $MAX_HEARTBEAT_LAPSE_PERIODS ]
 then
-ccArgs=$ccArgs' -max-heartbeat-lapse-periods '$MAX_HEARTBEAT_LAPSE_PERIODS
+ccArgs=$ccArgs' -heartbeat-max-misses '$MAX_HEARTBEAT_LAPSE_PERIODS
 fi
 
 if [ ! -z $PROFILE_DUMP_PERIOD ]
@@ -42,11 +42,6 @@ then
 ccArgs=$ccArgs' -profile-dump-period '$PROFILE_DUMP_PERIOD
 fi
 
-if [ ! -z $DEFAULT_MAX_JOB_ATTEMPTS ]
-then
-ccArgs=$ccArgs' -default-max-job-attempts '$DEFAULT_MAX_JOB_ATTEMPTS
-fi
-
 if [ ! -z $JOB_HISTORY_SIZE ]
 then
 ccArgs=$ccArgs' -job-history-size '$JOB_HISTORY_SIZE
@@ -54,7 +49,7 @@ fi
 
 if [ ! -z $RESULT_TIME_TO_LIVE ]
 then
-ccArgs=$ccArgs' "-result-time-to-live '$RESULT_TIME_TO_LIVE
+ccArgs=$ccArgs' -result-ttl '$RESULT_TIME_TO_LIVE
 fi
 
 if [ ! -z $RESULT_SWEEP_THRESHOLD ]
@@ -64,14 +59,17 @@ fi
 
 if [ ! -z $CC_ROOT ]
 then
-ccArgs=$ccArgs' -cc-root '$CC_ROOT
+ccArgs=$ccArgs' -root-dir '$CC_ROOT
 fi
 cd $WORKING_DIR
-DATE=`date`
+# TODO(mblow): ensure logging configuration is correct here...
+# export JAVA_OPTS="$JAVA_OPTS -Dlog4j.configuration=file:$WORKING_DIR/conf/log4j.properties"
 
+DATE=`date`
 cat <<EOF >> $LOG_DIR/cc.log
 --------------------------------------------------------------------------------
 LOG START: $DATE
+JAVA_OPTS: $JAVA_OPTS
 --------------------------------------------------------------------------------
 EOF
-$ASTERIX_HOME/bin/asterixcc echo $ccArgs >> $LOG_DIR/cc.log 2>&1
\ No newline at end of file
+$ASTERIX_HOME/bin/asterixcc $ccArgs >> $LOG_DIR/cc.log 2>&1
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4c7b5bfa/asterixdb/asterix-events/src/main/resources/events/node_join/nc_join.sh
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-events/src/main/resources/events/node_join/nc_join.sh b/asterixdb/asterix-events/src/main/resources/events/node_join/nc_join.sh
index 7f86379..626739a 100644
--- a/asterixdb/asterix-events/src/main/resources/events/node_join/nc_join.sh
+++ b/asterixdb/asterix-events/src/main/resources/events/node_join/nc_join.sh
@@ -21,8 +21,8 @@ CC_HOST=$1
 NC_ID=$2
 IO_DEVICES=$3
 INITIAL_RUN_FLAG=$4
-if [ ! -d $LOG_DIR ]; 
-then 
+if [ ! -d $LOG_DIR ];
+then
   mkdir -p $LOG_DIR
 fi
 
@@ -32,6 +32,7 @@ DATE=`date`
 cat <<EOF >> $LOG_DIR/${NC_ID}.log
 --------------------------------------------------------------------------------
 LOG START: $DATE
+JAVA_OPTS: $JAVA_OPTS
 --------------------------------------------------------------------------------
 EOF
-$ASTERIX_HOME/bin/asterixnc -node-id $NC_ID -cc-host $CC_HOST -cc-port $CLUSTER_NET_PORT  -cluster-net-ip-address $IP_LOCATION  -data-ip-address $IP_LOCATION -iodevices $IO_DEVICES -result-ip-address $IP_LOCATION -- $INITIAL_RUN_FLAG >> $LOG_DIR/${NC_ID}.log 2>&1
+$ASTERIX_HOME/bin/asterixnc -node-id $NC_ID -cluster-address $CC_HOST -cluster-port $CLUSTER_NET_PORT  -address $IP_LOCATION  -data-listen-address $IP_LOCATION -iodevices $IO_DEVICES -result-listen-address $IP_LOCATION $INITIAL_RUN_FLAG >> $LOG_DIR/${NC_ID}.log 2>&1

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4c7b5bfa/asterixdb/asterix-events/src/main/resources/events/node_restart/nc_restart.sh
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-events/src/main/resources/events/node_restart/nc_restart.sh b/asterixdb/asterix-events/src/main/resources/events/node_restart/nc_restart.sh
index b3b5303..5ed2f7e 100644
--- a/asterixdb/asterix-events/src/main/resources/events/node_restart/nc_restart.sh
+++ b/asterixdb/asterix-events/src/main/resources/events/node_restart/nc_restart.sh
@@ -23,18 +23,18 @@ SLEEP_TIME=$3
 
 if [ $NC_ID == 'ANY' ]
 then
-  NC_ID="." 
-  PARENT_ID=`ps -ej | tr -s " " | grep nc_join | grep -v grep | grep -v ssh |  cut -d " " -f2 | head -n 1` 
+  NC_ID="."
+  PARENT_ID=`ps -ej | tr -s " " | grep nc_join | grep -v grep | grep -v ssh |  cut -d " " -f2 | head -n 1`
   PARENT_PROCESS_ENTRY=`ps -ef | grep $PARENT_ID | grep -v grep   | head -n 1`
   NC_ID=`echo ${PARENT_PROCESS_ENTRY##* }`
   echo "NCid is $NC_ID" >> ~/try.txt
-else 
-  PARENT_ID=`ps -ej | tr -s " " | grep nc_join | grep -v grep | grep -v ssh | grep $NC_ID | cut -d " " -f2 | head -n 1` 
-fi 
+else
+  PARENT_ID=`ps -ej | tr -s " " | grep nc_join | grep -v grep | grep -v ssh | grep $NC_ID | cut -d " " -f2 | head -n 1`
+fi
 
 PID=`ps -ej | tr -s " " | grep hyracks | grep -v grep | grep -v nc_join |  grep $PARENT_ID | cut -d " " -f2 | head -n 1`
 kill -9 $PID
 
 sleep $3
 
-$HYRACKS_HOME/hyracks-server/target/hyracks-server-0.2.2-SNAPSHOT-binary-assembly/bin/hyracksnc -node-id $NC_ID -cc-host $CC_HOST -cc-port 1099 -cluster-net-ip-address $IP_LOCATION  -data-ip-address $IP_LOCATION
+$HYRACKS_HOME/hyracks-server/target/hyracks-server-0.2.2-SNAPSHOT-binary-assembly/bin/hyracksnc -node-id $NC_ID -cluster-address $CC_HOST -cluster-port 1099 -address $IP_LOCATION  -data-listen-address $IP_LOCATION

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4c7b5bfa/asterixdb/asterix-installer/pom.xml
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/pom.xml b/asterixdb/asterix-installer/pom.xml
index df36c80..8b200bb 100644
--- a/asterixdb/asterix-installer/pom.xml
+++ b/asterixdb/asterix-installer/pom.xml
@@ -260,7 +260,7 @@
         <artifactId>maven-dependency-plugin</artifactId>
         <version>2.10</version>
         <configuration>
-          <usedDependencies>
+          <usedDependencies combine.children="append">
             <usedDependency>org.apache.asterix:asterix-external-data</usedDependency>
             <usedDependency>org.apache.asterix:asterix-server</usedDependency>
             <usedDependency>org.apache.hadoop:hadoop-common</usedDependency>
@@ -478,5 +478,10 @@
       <artifactId>hadoop-hdfs</artifactId>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-util</artifactId>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 </project>

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4c7b5bfa/asterixdb/asterix-installer/src/main/resources/clusters/local/local.xml
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/main/resources/clusters/local/local.xml b/asterixdb/asterix-installer/src/main/resources/clusters/local/local.xml
index bd3bde0..f2afe5e 100644
--- a/asterixdb/asterix-installer/src/main/resources/clusters/local/local.xml
+++ b/asterixdb/asterix-installer/src/main/resources/clusters/local/local.xml
@@ -32,8 +32,6 @@
   <max_heartbeat_lapse_periods>5</max_heartbeat_lapse_periods>
   <!-- Sets the time duration between two profile dumps from each node controller in milliseconds. 0 to disable. (default: 0) -->
   <profile_dump_period>0</profile_dump_period>
-  <!-- Sets the default number of job attempts allowed if not specified in the job specification. (default: 5) -->
-  <default_max_job_attempts>5</default_max_job_attempts>
   <!-- Limits the number of historical jobs remembered by the system to the specified value. (default: 10) -->
   <job_history_size>10</job_history_size>
   <!-- Limits the amount of time results for asynchronous jobs should be retained by the system in milliseconds. (default: 24 hours) -->

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4c7b5bfa/asterixdb/asterix-installer/src/main/resources/clusters/local/local_chained_declustering_rep.xml
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/main/resources/clusters/local/local_chained_declustering_rep.xml b/asterixdb/asterix-installer/src/main/resources/clusters/local/local_chained_declustering_rep.xml
index 954a311..57d04c7 100644
--- a/asterixdb/asterix-installer/src/main/resources/clusters/local/local_chained_declustering_rep.xml
+++ b/asterixdb/asterix-installer/src/main/resources/clusters/local/local_chained_declustering_rep.xml
@@ -32,8 +32,6 @@
   <max_heartbeat_lapse_periods>5</max_heartbeat_lapse_periods>
   <!-- Sets the time duration between two profile dumps from each node controller in milliseconds. 0 to disable. (default: 0) -->
   <profile_dump_period>0</profile_dump_period>
-  <!-- Sets the default number of job attempts allowed if not specified in the job specification. (default: 5) -->
-  <default_max_job_attempts>5</default_max_job_attempts>
   <!-- Limits the number of historical jobs remembered by the system to the specified value. (default: 10) -->
   <job_history_size>10</job_history_size>
   <!-- Limits the amount of time results for asynchronous jobs should be retained by the system in milliseconds. (default: 24 hours) -->

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4c7b5bfa/asterixdb/asterix-installer/src/test/java/org/apache/asterix/installer/test/AbstractExecutionIT.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/java/org/apache/asterix/installer/test/AbstractExecutionIT.java b/asterixdb/asterix-installer/src/test/java/org/apache/asterix/installer/test/AbstractExecutionIT.java
index 03e4f75..9d0a1db 100644
--- a/asterixdb/asterix-installer/src/test/java/org/apache/asterix/installer/test/AbstractExecutionIT.java
+++ b/asterixdb/asterix-installer/src/test/java/org/apache/asterix/installer/test/AbstractExecutionIT.java
@@ -31,6 +31,7 @@ import org.apache.asterix.testframework.context.TestFileContext;
 import org.apache.asterix.testframework.xml.TestCase.CompilationUnit;
 import org.apache.asterix.testframework.xml.TestGroup;
 import org.apache.commons.lang3.StringUtils;
+import org.apache.hyracks.util.file.FileUtil;
 import org.codehaus.plexus.util.FileUtils;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -62,7 +63,7 @@ public abstract class AbstractExecutionIT {
     private static final List<String> badTestCases = new ArrayList<>();
 
     private static String reportPath =
-            new File(StringUtils.join(new String[] { "target", "failsafe-reports" }, File.separator)).getAbsolutePath();
+            new File(FileUtil.joinPath("target", "failsafe-reports")).getAbsolutePath();
 
     @Rule
     public TestRule retainLogs = new RetainLogsRule(
@@ -100,14 +101,14 @@ public abstract class AbstractExecutionIT {
         AsterixLifecycleIT.restartInstance();
 
         FileUtils.copyDirectoryStructure(
-                new File(StringUtils.join(new String[] { "..", "asterix-app", "data" }, File.separator)),
+                new File(FileUtil.joinPath("..", "asterix-app", "data")),
                 new File(AsterixInstallerIntegrationUtil.getManagixHome() + "/clusters/local/working_dir/data"));
 
         FileUtils.copyDirectoryStructure(
-                new File(StringUtils.join(new String[] { "..", "asterix-app", "target", "data" }, File.separator)),
+                new File(FileUtil.joinPath("..", "asterix-app", "target", "data")),
                 new File(AsterixInstallerIntegrationUtil.getManagixHome() + "/clusters/local/working_dir/target/data"));
 
-        FileUtils.copyDirectoryStructure(new File(StringUtils.join(new String[] { "target", "data" }, File.separator)),
+        FileUtils.copyDirectoryStructure(new File(FileUtil.joinPath("target", "data")),
                 new File(AsterixInstallerIntegrationUtil.getManagixHome()
                         + "/clusters/local/working_dir/target/data/csv"));
 
@@ -116,7 +117,7 @@ public abstract class AbstractExecutionIT {
         System.setProperty(ExternalDataConstants.NODE_RESOLVER_FACTORY_PROPERTY,
                 IdentitiyResolverFactory.class.getName());
 
-        reportPath = new File(StringUtils.join(new String[] { "target", "failsafe-reports" }, File.separator))
+        reportPath = new File(FileUtil.joinPath("target", "failsafe-reports"))
                 .getAbsolutePath();
     }
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4c7b5bfa/asterixdb/asterix-installer/src/test/java/org/apache/asterix/installer/test/AsterixExternalLibraryIT.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/java/org/apache/asterix/installer/test/AsterixExternalLibraryIT.java b/asterixdb/asterix-installer/src/test/java/org/apache/asterix/installer/test/AsterixExternalLibraryIT.java
index b3a5dcc..ac5beaf 100644
--- a/asterixdb/asterix-installer/src/test/java/org/apache/asterix/installer/test/AsterixExternalLibraryIT.java
+++ b/asterixdb/asterix-installer/src/test/java/org/apache/asterix/installer/test/AsterixExternalLibraryIT.java
@@ -26,7 +26,7 @@ import org.apache.asterix.event.model.AsterixInstance.State;
 import org.apache.asterix.test.base.RetainLogsRule;
 import org.apache.asterix.test.common.TestExecutor;
 import org.apache.asterix.testframework.context.TestCaseContext;
-import org.apache.commons.lang3.StringUtils;
+import org.apache.hyracks.util.file.FileUtil;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Rule;
@@ -44,7 +44,7 @@ public class AsterixExternalLibraryIT {
     private static final Logger LOGGER = Logger.getLogger(AsterixExternalLibraryIT.class.getName());
     private static List<TestCaseContext> testCaseCollection;
     private static String reportPath =
-            new File(StringUtils.join(new String[] { "target", "failsafe-reports" }, File.separator)).getAbsolutePath();
+            new File(FileUtil.joinPath("target", "failsafe-reports")).getAbsolutePath();
 
     private final TestExecutor testExecutor = new TestExecutor();