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 2016/09/08 16:30:33 UTC

asterixdb git commit: Expose Asterix Configuration on Cluster Servlet

Repository: asterixdb
Updated Branches:
  refs/heads/master 13ae9a3b2 -> d5b0afaca


Expose Asterix Configuration on Cluster Servlet

Also, a few test executor improvements

Change-Id: I828d6a61afe615f7826079ede4b1d638bbd7ac5d
Reviewed-on: https://asterix-gerrit.ics.uci.edu/1153
Sonar-Qube: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Till Westmann <ti...@apache.org>
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>


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

Branch: refs/heads/master
Commit: d5b0afacae8b23e155c73cefae1f1dc8b2a3cc8d
Parents: 13ae9a3
Author: Michael Blow <mb...@apache.org>
Authored: Thu Sep 8 11:17:31 2016 -0400
Committer: Michael Blow <mb...@apache.org>
Committed: Thu Sep 8 09:30:10 2016 -0700

----------------------------------------------------------------------
 .../api/http/servlet/ClusterAPIServlet.java     | 20 +++++-
 .../cluster_state_1.1.cstate.aql                | 24 +++++++
 .../cluster_state_2.1.cstate.aql                | 25 +++++++
 .../api/cluster_state_1/cluster_state_1.1.adm   | 58 ++++++++++++++++
 .../src/test/resources/runtimets/testsuite.xml  | 13 ++++
 .../config/AbstractAsterixProperties.java       | 39 +++++++++++
 .../config/AsterixCompilerProperties.java       | 18 +++--
 .../config/AsterixExternalProperties.java       | 28 +++++---
 .../common/config/AsterixFeedProperties.java    | 22 ++++++-
 .../config/AsterixReplicationProperties.java    | 29 ++++++--
 .../common/config/AsterixStorageProperties.java | 34 +++++++---
 .../config/AsterixTransactionProperties.java    | 35 ++++++----
 .../apache/asterix/test/aql/TestExecutor.java   | 69 ++++++++++----------
 .../installer/test/AbstractExecutionIT.java     |  8 +++
 .../server/test/NCServiceExecutionIT.java       |  6 +-
 15 files changed, 350 insertions(+), 78 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d5b0afac/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/ClusterAPIServlet.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/ClusterAPIServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/ClusterAPIServlet.java
index c7cf1ea..8a16cd7 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/ClusterAPIServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/ClusterAPIServlet.java
@@ -20,12 +20,16 @@ package org.apache.asterix.api.http.servlet;
 
 import java.io.IOException;
 import java.io.PrintWriter;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 
 import javax.servlet.http.HttpServlet;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 
 import org.apache.asterix.app.result.ResultUtil;
+import org.apache.asterix.common.config.AbstractAsterixProperties;
 import org.apache.asterix.runtime.util.AsterixClusterProperties;
 import org.json.JSONException;
 import org.json.JSONObject;
@@ -40,7 +44,9 @@ public class ClusterAPIServlet extends HttpServlet {
         PrintWriter responseWriter = response.getWriter();
         try {
             JSONObject responseObject = AsterixClusterProperties.INSTANCE.getClusterStateDescription();
-            responseWriter.write(responseObject.toString());
+            Map<String, Object> allProperties = getAllClusterProperties();
+            responseObject.put("config", allProperties);
+            responseWriter.write(responseObject.toString(4));
             response.setStatus(HttpServletResponse.SC_OK);
         } catch (JSONException e) {
             ResultUtil.apiErrorHandler(responseWriter, e);
@@ -48,4 +54,16 @@ public class ClusterAPIServlet extends HttpServlet {
         }
         responseWriter.flush();
     }
+
+    protected Map<String, Object> getAllClusterProperties() {
+        Map<String, Object> allProperties = new HashMap<>();
+        for (AbstractAsterixProperties properties : getPropertiesInstances()) {
+            allProperties.putAll(properties.getProperties());
+        }
+        return allProperties;
+    }
+
+    protected List<AbstractAsterixProperties> getPropertiesInstances() {
+        return AbstractAsterixProperties.getImplementations();
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d5b0afac/asterixdb/asterix-app/src/test/resources/runtimets/queries/api/cluster_state_1/cluster_state_1.1.cstate.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/api/cluster_state_1/cluster_state_1.1.cstate.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/api/cluster_state_1/cluster_state_1.1.cstate.aql
new file mode 100644
index 0000000..517b5b0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/api/cluster_state_1/cluster_state_1.1.cstate.aql
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ */
+/*
+ * Test case Name  : cluster_state_1
+ * Description     : test cluster state api
+ * Expected Result : Success
+ * Date            : 7th September 2016
+ */

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d5b0afac/asterixdb/asterix-app/src/test/resources/runtimets/queries/api/cluster_state_2/cluster_state_2.1.cstate.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/api/cluster_state_2/cluster_state_2.1.cstate.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/api/cluster_state_2/cluster_state_2.1.cstate.aql
new file mode 100644
index 0000000..b88aed8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/api/cluster_state_2/cluster_state_2.1.cstate.aql
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+/*
+ * Test case Name  : cluster_state_2
+ * Description     : test cluster state api nonexistent node
+ * Expected Result : Negative
+ * Date            : 7th September 2016
+ */
+/node/foo

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d5b0afac/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.adm
new file mode 100644
index 0000000..0bc8f17
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.adm
@@ -0,0 +1,58 @@
+{
+    "Metadata_Node": "asterix_nc1",
+    "State": "ACTIVE",
+    "config": {
+        "api.port": 19002,
+        "cc.java.opts": "-Xmx1024m",
+        "compiler.framesize": 32768,
+        "compiler.groupmemory": 163840,
+        "compiler.joinmemory": 163840,
+        "compiler.pregelix.home": "~/pregelix",
+        "compiler.sortmemory": 327680,
+        "feed.central.manager.port": 4500,
+        "feed.max.threshold.period": 5,
+        "feed.memory.available.wait.timeout": 10,
+        "feed.memory.global.budget": 67108864,
+        "feed.pending.work.threshold": 50,
+        "feed.port": 19003,
+        "log.level": "WARNING",
+        "max.wait.active.cluster": 60,
+        "nc.java.opts": "-Xmx1024m",
+        "plot.activate": false,
+        "replication.enabled": false,
+        "replication.factor": 2,
+        "replication.log.batchsize": 4096,
+        "replication.log.buffer.numpages": 8,
+        "replication.log.buffer.pagesize": 131072,
+        "replication.max.remote.recovery.attempts": 5,
+        "replication.timeout": 30,
+        "storage.buffercache.maxopenfiles": 2147483647,
+        "storage.buffercache.pagesize": 32768,
+        "storage.buffercache.size": 33554432,
+        "storage.lsm.bloomfilter.falsepositiverate": 0.01,
+        "storage.memorycomponent.globalbudget": 536870912,
+        "storage.memorycomponent.numcomponents": 2,
+        "storage.memorycomponent.numpages": 8,
+        "storage.memorycomponent.pagesize": 131072,
+        "storage.metadata.memorycomponent.numpages": 256,
+        "txn.commitprofiler.reportinterval": 5,
+        "txn.job.recovery.memorysize": 67108864,
+        "txn.lock.escalationthreshold": 1000,
+        "txn.lock.shrinktimer": 5000,
+        "txn.lock.timeout.sweepthreshold": 10000,
+        "txn.lock.timeout.waitthreshold": 60000,
+        "txn.log.buffer.numpages": 8,
+        "txn.log.buffer.pagesize": 131072,
+        "txn.log.checkpoint.history": 0,
+        "txn.log.checkpoint.lsnthreshold": 67108864,
+        "txn.log.checkpoint.pollfrequency": 120,
+        "txn.log.partitionsize": 268435456,
+        "web.port": 19001,
+        "web.queryinterface.port": 19006,
+        "web.secondary.port": 19005
+    },
+    "partition_0": "asterix_nc1",
+    "partition_1": "asterix_nc1",
+    "partition_2": "asterix_nc2",
+    "partition_3": "asterix_nc2"
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d5b0afac/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
index 6c65a62..34a0805 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
@@ -7303,4 +7303,17 @@
       </compilation-unit>
     </test-case>
   </test-group>
+  <test-group name="api">
+    <test-case FilePath="api">
+      <compilation-unit name="cluster_state_1">
+        <output-dir compare="Text">cluster_state_1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="api">
+      <compilation-unit name="cluster_state_2">
+        <output-dir compare="Text">cluster_state_2</output-dir>
+        <expected-error>HTTP/1.1 404 Not Found</expected-error>
+      </compilation-unit>
+    </test-case>
+  </test-group>
 </test-suite>

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d5b0afac/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AbstractAsterixProperties.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AbstractAsterixProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AbstractAsterixProperties.java
index e68676e..845483e 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AbstractAsterixProperties.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AbstractAsterixProperties.java
@@ -18,10 +18,49 @@
  */
 package org.apache.asterix.common.config;
 
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.reflect.Method;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
 public abstract class AbstractAsterixProperties {
+    private static final Logger LOGGER = Logger.getLogger(AbstractAsterixProperties.class.getName());
+    private static final List<AbstractAsterixProperties> IMPLS = Collections.synchronizedList(new ArrayList<>());
+
     protected final AsterixPropertiesAccessor accessor;
 
     public AbstractAsterixProperties(AsterixPropertiesAccessor accessor) {
         this.accessor = accessor;
+        IMPLS.add(this);
+    }
+
+    public Map<String, Object> getProperties() {
+        Map<String, Object> properties = new HashMap<>();
+        for (Method m : getClass().getMethods()) {
+            PropertyKey key = m.getAnnotation(PropertyKey.class);
+            if (key != null) {
+                try {
+                    properties.put(key.value(), m.invoke(this));
+                } catch (Exception e) {
+                    LOGGER.log(Level.INFO, "Error accessing property: " + key.value(), e);
+                }
+            }
+        }
+        return properties;
+    }
+
+    @Retention(RetentionPolicy.RUNTIME)
+    public @interface PropertyKey {
+        String value();
+    }
+
+    public static List<AbstractAsterixProperties> getImplementations() {
+        return Collections.unmodifiableList(IMPLS);
     }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d5b0afac/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixCompilerProperties.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixCompilerProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixCompilerProperties.java
index 5bfb5f8..af5124b 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixCompilerProperties.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixCompilerProperties.java
@@ -18,18 +18,23 @@
  */
 package org.apache.asterix.common.config;
 
+import org.apache.hyracks.util.StorageUtil;
+
+import static org.apache.hyracks.util.StorageUtil.StorageUnit.KILOBYTE;
+import static org.apache.hyracks.util.StorageUtil.StorageUnit.MEGABYTE;
+
 public class AsterixCompilerProperties extends AbstractAsterixProperties {
     private static final String COMPILER_SORTMEMORY_KEY = "compiler.sortmemory";
-    private static final long COMPILER_SORTMEMORY_DEFAULT = (32 << 20); // 32MB
+    private static final long COMPILER_SORTMEMORY_DEFAULT = StorageUtil.getSizeInBytes(32, MEGABYTE);
 
     private static final String COMPILER_GROUPMEMORY_KEY = "compiler.groupmemory";
-    private static final long COMPILER_GROUPMEMORY_DEFAULT = (32 << 20); // 32MB
+    private static final long COMPILER_GROUPMEMORY_DEFAULT = StorageUtil.getSizeInBytes(32, MEGABYTE);
 
     private static final String COMPILER_JOINMEMORY_KEY = "compiler.joinmemory";
-    private static final long COMPILER_JOINMEMORY_DEFAULT = (32 << 20); // 32MB
+    private static final long COMPILER_JOINMEMORY_DEFAULT = StorageUtil.getSizeInBytes(32, MEGABYTE);
 
     private static final String COMPILER_FRAMESIZE_KEY = "compiler.framesize";
-    private static int COMPILER_FRAMESIZE_DEFAULT = (32 << 10); // 32KB
+    private static final int COMPILER_FRAMESIZE_DEFAULT = StorageUtil.getSizeInBytes(32, KILOBYTE);
 
     private static final String COMPILER_PREGELIX_HOME = "compiler.pregelix.home";
     private static final String COMPILER_PREGELIX_HOME_DEFAULT = "~/pregelix";
@@ -38,26 +43,31 @@ public class AsterixCompilerProperties extends AbstractAsterixProperties {
         super(accessor);
     }
 
+    @PropertyKey(COMPILER_SORTMEMORY_KEY)
     public long getSortMemorySize() {
         return accessor.getProperty(COMPILER_SORTMEMORY_KEY, COMPILER_SORTMEMORY_DEFAULT,
                 PropertyInterpreters.getLongBytePropertyInterpreter());
     }
 
+    @PropertyKey(COMPILER_JOINMEMORY_KEY)
     public long getJoinMemorySize() {
         return accessor.getProperty(COMPILER_JOINMEMORY_KEY, COMPILER_JOINMEMORY_DEFAULT,
                 PropertyInterpreters.getLongBytePropertyInterpreter());
     }
 
+    @PropertyKey(COMPILER_GROUPMEMORY_KEY)
     public long getGroupMemorySize() {
         return accessor.getProperty(COMPILER_GROUPMEMORY_KEY, COMPILER_GROUPMEMORY_DEFAULT,
                 PropertyInterpreters.getLongBytePropertyInterpreter());
     }
 
+    @PropertyKey(COMPILER_FRAMESIZE_KEY)
     public int getFrameSize() {
         return accessor.getProperty(COMPILER_FRAMESIZE_KEY, COMPILER_FRAMESIZE_DEFAULT,
                 PropertyInterpreters.getIntegerBytePropertyInterpreter());
     }
 
+    @PropertyKey(COMPILER_PREGELIX_HOME)
     public String getPregelixHome() {
         return accessor.getProperty(COMPILER_PREGELIX_HOME, COMPILER_PREGELIX_HOME_DEFAULT,
                 PropertyInterpreters.getStringPropertyInterpreter());

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d5b0afac/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixExternalProperties.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixExternalProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixExternalProperties.java
index b2a43e3..1a9096d 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixExternalProperties.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixExternalProperties.java
@@ -23,84 +23,94 @@ import java.util.logging.Level;
 public class AsterixExternalProperties extends AbstractAsterixProperties {
 
     private static final String EXTERNAL_WEBPORT_KEY = "web.port";
-    private static int EXTERNAL_WEBPORT_DEFAULT = 19001;
+    private static final int EXTERNAL_WEBPORT_DEFAULT = 19001;
 
     private static final String EXTERNAL_SECONDARY_WEBPORT_KEY = "web.secondary.port";
-    private static int EXTERNAL_SECONDARY_WEBPORT_DEFAULT = 19005;
+    private static final int EXTERNAL_SECONDARY_WEBPORT_DEFAULT = 19005;
 
     private static final String QUERY_WEBPORT_KEY = "web.queryinterface.port";
     private static final int QUERY_WEBPORT_DEFAULT = 19006;
 
     private static final String EXTERNAL_LOGLEVEL_KEY = "log.level";
-    private static Level EXTERNAL_LOGLEVEL_DEFAULT = Level.WARNING;
+    private static final Level EXTERNAL_LOGLEVEL_DEFAULT = Level.WARNING;
 
     private static final String EXTERNAL_APISERVER_KEY = "api.port";
-    private static int EXTERNAL_APISERVER_DEFAULT = 19002;
+    private static final int EXTERNAL_APISERVER_DEFAULT = 19002;
 
     private static final String EXTERNAL_FEEDSERVER_KEY = "feed.port";
-    private static int EXTERNAL_FEEDSERVER_DEFAULT = 19003;
+    private static final int EXTERNAL_FEEDSERVER_DEFAULT = 19003;
 
     private static final String EXTERNAL_CC_JAVA_OPTS_KEY = "cc.java.opts";
-    private static String EXTERNAL_CC_JAVA_OPTS_DEFAULT = "-Xmx1024m";
+    private static final String EXTERNAL_CC_JAVA_OPTS_DEFAULT = "-Xmx1024m";
 
     private static final String EXTERNAL_NC_JAVA_OPTS_KEY = "nc.java.opts";
-    private static String EXTERNAL_NC_JAVA_OPTS_DEFAULT = "-Xmx1024m";
+    private static final String EXTERNAL_NC_JAVA_OPTS_DEFAULT = "-Xmx1024m";
 
     private static final String EXTERNAL_MAX_WAIT_FOR_ACTIVE_CLUSTER = "max.wait.active.cluster";
-    private static int EXTERNAL_MAX_WAIT_FOR_ACTIVE_CLUSTER_DEFAULT = 60;
+    private static final int EXTERNAL_MAX_WAIT_FOR_ACTIVE_CLUSTER_DEFAULT = 60;
 
     private static final String EXTERNAL_PLOT_ACTIVATE = "plot.activate";
-    private static Boolean EXTERNAL_PLOT_ACTIVATE_DEFAULT = new Boolean(false);
+    private static final boolean EXTERNAL_PLOT_ACTIVATE_DEFAULT = false;
 
     public AsterixExternalProperties(AsterixPropertiesAccessor accessor) {
         super(accessor);
     }
 
+    @PropertyKey(EXTERNAL_WEBPORT_KEY)
     public int getWebInterfacePort() {
         return accessor.getProperty(EXTERNAL_WEBPORT_KEY, EXTERNAL_WEBPORT_DEFAULT,
                 PropertyInterpreters.getIntegerPropertyInterpreter());
     }
 
+    @PropertyKey(EXTERNAL_SECONDARY_WEBPORT_KEY)
     public int getSecondaryWebInterfacePort() {
         return accessor.getProperty(EXTERNAL_SECONDARY_WEBPORT_KEY, EXTERNAL_SECONDARY_WEBPORT_DEFAULT,
                 PropertyInterpreters.getIntegerPropertyInterpreter());
     }
 
+    @PropertyKey(QUERY_WEBPORT_KEY)
     public int getQueryWebInterfacePort() {
         return accessor.getProperty(QUERY_WEBPORT_KEY, QUERY_WEBPORT_DEFAULT,
                 PropertyInterpreters.getIntegerPropertyInterpreter());
     }
 
+    @PropertyKey(EXTERNAL_APISERVER_KEY)
     public int getAPIServerPort() {
         return accessor.getProperty(EXTERNAL_APISERVER_KEY, EXTERNAL_APISERVER_DEFAULT,
                 PropertyInterpreters.getIntegerPropertyInterpreter());
     }
 
+    @PropertyKey(EXTERNAL_FEEDSERVER_KEY)
     public int getFeedServerPort() {
         return accessor.getProperty(EXTERNAL_FEEDSERVER_KEY, EXTERNAL_FEEDSERVER_DEFAULT,
                 PropertyInterpreters.getIntegerPropertyInterpreter());
     }
 
+    @PropertyKey(EXTERNAL_LOGLEVEL_KEY)
     public Level getLogLevel() {
         return accessor.getProperty(EXTERNAL_LOGLEVEL_KEY, EXTERNAL_LOGLEVEL_DEFAULT,
                 PropertyInterpreters.getLevelPropertyInterpreter());
     }
 
+    @PropertyKey(EXTERNAL_NC_JAVA_OPTS_KEY)
     public String getNCJavaParams() {
         return accessor.getProperty(EXTERNAL_NC_JAVA_OPTS_KEY, EXTERNAL_NC_JAVA_OPTS_DEFAULT,
                 PropertyInterpreters.getStringPropertyInterpreter());
     }
 
+    @PropertyKey(EXTERNAL_CC_JAVA_OPTS_KEY)
     public String getCCJavaParams() {
         return accessor.getProperty(EXTERNAL_CC_JAVA_OPTS_KEY, EXTERNAL_CC_JAVA_OPTS_DEFAULT,
                 PropertyInterpreters.getStringPropertyInterpreter());
     }
 
+    @PropertyKey(EXTERNAL_MAX_WAIT_FOR_ACTIVE_CLUSTER)
     public int getMaxWaitClusterActive() {
         return accessor.getProperty(EXTERNAL_MAX_WAIT_FOR_ACTIVE_CLUSTER, EXTERNAL_MAX_WAIT_FOR_ACTIVE_CLUSTER_DEFAULT,
                 PropertyInterpreters.getIntegerPropertyInterpreter());
     }
 
+    @PropertyKey(EXTERNAL_PLOT_ACTIVATE)
     public Boolean getIsPlottingEnabled() {
         return accessor.getProperty(EXTERNAL_PLOT_ACTIVATE, EXTERNAL_PLOT_ACTIVATE_DEFAULT,
                 PropertyInterpreters.getBooleanPropertyInterpreter());

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d5b0afac/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixFeedProperties.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixFeedProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixFeedProperties.java
index 0f3951e..ff4d81e 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixFeedProperties.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixFeedProperties.java
@@ -18,19 +18,24 @@
  */
 package org.apache.asterix.common.config;
 
+import org.apache.hyracks.util.StorageUtil;
+
+import static org.apache.hyracks.util.StorageUtil.StorageUnit.MEGABYTE;
+
 public class AsterixFeedProperties extends AbstractAsterixProperties {
 
     private static final String FEED_CENTRAL_MANAGER_PORT_KEY = "feed.central.manager.port";
-    private static final int FEED_CENTRAL_MANAGER_PORT_DEFAULT = 4500; // port at which the Central Feed Manager listens for control messages from local Feed Managers
+    private static final int FEED_CENTRAL_MANAGER_PORT_DEFAULT = 4500;
 
     private static final String FEED_MEMORY_GLOBALBUDGET_KEY = "feed.memory.global.budget";
-    private static final long FEED_MEMORY_GLOBALBUDGET_DEFAULT = 67108864; // 64MB or 2048 frames (assuming 32768 as frame size)
+    private static final long FEED_MEMORY_GLOBALBUDGET_DEFAULT = StorageUtil.getSizeInBytes(64, MEGABYTE);
+                                                                 // i.e. 2048 frames (assuming 32768 as frame size)
 
     private static final String FEED_MEMORY_AVAILABLE_WAIT_TIMEOUT_KEY = "feed.memory.available.wait.timeout";
     private static final long FEED_MEMORY_AVAILABLE_WAIT_TIMEOUT_DEFAULT = 10; // 10 seconds
 
     private static final String FEED_PENDING_WORK_THRESHOLD_KEY = "feed.pending.work.threshold";
-    private static final int FEED_PENDING_WORK_THRESHOLD_DEFAULT = 50; // maximum length of input queue before triggering corrective action
+    private static final int FEED_PENDING_WORK_THRESHOLD_DEFAULT = 50;
 
     private static final String FEED_MAX_SUCCESSIVE_THRESHOLD_PERIOD_KEY = "feed.max.threshold.period";
     private static final int FEED_MAX_SUCCESSIVE_THRESHOLD_PERIOD_DEFAULT = 5;
@@ -39,26 +44,37 @@ public class AsterixFeedProperties extends AbstractAsterixProperties {
         super(accessor);
     }
 
+    @PropertyKey(FEED_MEMORY_GLOBALBUDGET_KEY)
     public long getMemoryComponentGlobalBudget() {
         return accessor.getProperty(FEED_MEMORY_GLOBALBUDGET_KEY, FEED_MEMORY_GLOBALBUDGET_DEFAULT,
                 PropertyInterpreters.getLongBytePropertyInterpreter());
     }
 
+    @PropertyKey(FEED_MEMORY_AVAILABLE_WAIT_TIMEOUT_KEY)
     public long getMemoryAvailableWaitTimeout() {
         return accessor.getProperty(FEED_MEMORY_AVAILABLE_WAIT_TIMEOUT_KEY, FEED_MEMORY_AVAILABLE_WAIT_TIMEOUT_DEFAULT,
                 PropertyInterpreters.getLongPropertyInterpreter());
     }
 
+    /**
+     * @return port at which the Central Feed Manager listens for control messages from local Feed Managers
+     */
+    @PropertyKey(FEED_CENTRAL_MANAGER_PORT_KEY)
     public int getFeedCentralManagerPort() {
         return accessor.getProperty(FEED_CENTRAL_MANAGER_PORT_KEY, FEED_CENTRAL_MANAGER_PORT_DEFAULT,
                 PropertyInterpreters.getIntegerPropertyInterpreter());
     }
 
+    /**
+     * @return maximum length of input queue before triggering corrective action
+     */
+    @PropertyKey(FEED_PENDING_WORK_THRESHOLD_KEY)
     public int getPendingWorkThreshold() {
         return accessor.getProperty(FEED_PENDING_WORK_THRESHOLD_KEY, FEED_PENDING_WORK_THRESHOLD_DEFAULT,
                 PropertyInterpreters.getIntegerPropertyInterpreter());
     }
 
+    @PropertyKey(FEED_MAX_SUCCESSIVE_THRESHOLD_PERIOD_KEY)
     public int getMaxSuccessiveThresholdPeriod() {
         return accessor.getProperty(FEED_MAX_SUCCESSIVE_THRESHOLD_PERIOD_KEY,
                 FEED_MAX_SUCCESSIVE_THRESHOLD_PERIOD_DEFAULT, PropertyInterpreters.getIntegerPropertyInterpreter());

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d5b0afac/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixReplicationProperties.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixReplicationProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixReplicationProperties.java
index eec4947..86ace28 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixReplicationProperties.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixReplicationProperties.java
@@ -33,13 +33,22 @@ public class AsterixReplicationProperties extends AbstractAsterixProperties {
 
     private static final Logger LOGGER = Logger.getLogger(AsterixReplicationProperties.class.getName());
 
+
     private static final int REPLICATION_DATAPORT_DEFAULT = 2000;
+
+    private static final String REPLICATION_ENABLED_KEY = "replication.enabled";
+
+    private static final String REPLICATION_FACTOR_KEY = "replication.factor";
     private static final int REPLICATION_FACTOR_DEFAULT = 1;
+
+    private static final String REPLICATION_TIMEOUT_KEY = "replication.timeout";
     private static final int REPLICATION_TIME_OUT_DEFAULT = 15;
+
+    private static final String REPLICATION_MAX_REMOTE_RECOVERY_ATTEMPTS_KEY =
+            "replication.max.remote.recovery.attempts";
     private static final int MAX_REMOTE_RECOVERY_ATTEMPTS = 5;
+
     private static final String NODE_IP_ADDRESS_DEFAULT = "127.0.0.1";
-    private final String NODE_NAME_PREFIX;
-    private final Cluster cluster;
 
     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);
@@ -51,17 +60,21 @@ public class AsterixReplicationProperties extends AbstractAsterixProperties {
     private static final int REPLICATION_LOG_BUFFER_PAGE_SIZE_DEFAULT = StorageUtil.getSizeInBytes(128,
             StorageUnit.KILOBYTE);
 
+    private final String nodeNamePrefix;
+    private final Cluster cluster;
+
     public AsterixReplicationProperties(AsterixPropertiesAccessor accessor, Cluster cluster) {
         super(accessor);
         this.cluster = cluster;
 
         if (cluster != null) {
-            NODE_NAME_PREFIX = cluster.getInstanceName() + "_";
+            nodeNamePrefix = cluster.getInstanceName() + "_";
         } else {
-            NODE_NAME_PREFIX = "";
+            nodeNamePrefix = "";
         }
     }
 
+    @PropertyKey(REPLICATION_ENABLED_KEY)
     public boolean isReplicationEnabled() {
         if (cluster != null && cluster.getDataReplication() != null) {
             if (getReplicationFactor() == 1) {
@@ -184,7 +197,7 @@ public class AsterixReplicationProperties extends AbstractAsterixProperties {
     }
 
     public String getRealCluserNodeID(String nodeId) {
-        return NODE_NAME_PREFIX + nodeId;
+        return nodeNamePrefix + nodeId;
     }
 
     public Set<String> getNodeReplicasIds(String nodeId) {
@@ -194,6 +207,7 @@ public class AsterixReplicationProperties extends AbstractAsterixProperties {
         return replicaIds;
     }
 
+    @PropertyKey(REPLICATION_FACTOR_KEY)
     public int getReplicationFactor() {
         if (cluster != null) {
             if (cluster.getDataReplication() == null || cluster.getDataReplication().getReplicationFactor() == null) {
@@ -204,6 +218,7 @@ public class AsterixReplicationProperties extends AbstractAsterixProperties {
         return REPLICATION_FACTOR_DEFAULT;
     }
 
+    @PropertyKey(REPLICATION_TIMEOUT_KEY)
     public int getReplicationTimeOut() {
         if (cluster != null) {
             return cluster.getDataReplication().getReplicationTimeOut().intValue();
@@ -254,20 +269,24 @@ public class AsterixReplicationProperties extends AbstractAsterixProperties {
         return clientReplicas;
     }
 
+    @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());

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d5b0afac/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixStorageProperties.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixStorageProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixStorageProperties.java
index 5fdbc1c..daeb099 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixStorageProperties.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixStorageProperties.java
@@ -19,45 +19,53 @@
 package org.apache.asterix.common.config;
 
 import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.apache.hyracks.util.StorageUtil;
+
+import static org.apache.hyracks.util.StorageUtil.StorageUnit.KILOBYTE;
+import static org.apache.hyracks.util.StorageUtil.StorageUnit.MEGABYTE;
 
 public class AsterixStorageProperties extends AbstractAsterixProperties {
 
     private static final String STORAGE_BUFFERCACHE_PAGESIZE_KEY = "storage.buffercache.pagesize";
-    private static int STORAGE_BUFFERCACHE_PAGESIZE_DEFAULT = (128 << 10); // 128KB
+    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 long STORAGE_BUFFERCACHE_SIZE_DEFAULT = (512 << 20); // 512 MB
+    private static final long STORAGE_BUFFERCACHE_SIZE_DEFAULT = StorageUtil.getSizeInBytes(512, MEGABYTE);
 
     private static final String STORAGE_BUFFERCACHE_MAXOPENFILES_KEY = "storage.buffercache.maxopenfiles";
-    private static int STORAGE_BUFFERCACHE_MAXOPENFILES_DEFAULT = Integer.MAX_VALUE;
+    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 = (128 << 10); // 128KB
+    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 int STORAGE_MEMORYCOMPONENT_NUMPAGES_DEFAULT = 256; // ... so 32MB components
 
-    private static final String STORAGE_METADATA_MEMORYCOMPONENT_NUMPAGES_KEY = "storage.metadata.memorycomponent.numpages";
+    private static final String STORAGE_METADATA_MEMORYCOMPONENT_NUMPAGES_KEY =
+            "storage.metadata.memorycomponent.numpages";
     private static final int STORAGE_METADATA_MEMORYCOMPONENT_NUMPAGES_DEFAULT = 256; // ... so 32MB components
 
     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 long STORAGE_MEMORYCOMPONENT_GLOBALBUDGET_DEFAULT = 536870912; // 512MB
+    private static final long STORAGE_MEMORYCOMPONENT_GLOBALBUDGET_DEFAULT = StorageUtil.getSizeInBytes(512, MEGABYTE);
 
-    private static final String STORAGE_LSM_BLOOMFILTER_FALSEPOSITIVERATE_KEY = "storage.lsm.bloomfilter.falsepositiverate";
-    private static double STORAGE_LSM_BLOOMFILTER_FALSEPOSITIVERATE_DEFAULT = 0.01;
+    private static final String STORAGE_LSM_BLOOMFILTER_FALSEPOSITIVERATE_KEY =
+            "storage.lsm.bloomfilter.falsepositiverate";
+    private static final double STORAGE_LSM_BLOOMFILTER_FALSEPOSITIVERATE_DEFAULT = 0.01;
 
     public AsterixStorageProperties(AsterixPropertiesAccessor accessor) {
         super(accessor);
     }
 
+    @PropertyKey(STORAGE_BUFFERCACHE_PAGESIZE_KEY)
     public int getBufferCachePageSize() {
         return accessor.getProperty(STORAGE_BUFFERCACHE_PAGESIZE_KEY, STORAGE_BUFFERCACHE_PAGESIZE_DEFAULT,
                 PropertyInterpreters.getIntegerBytePropertyInterpreter());
     }
 
+    @PropertyKey(STORAGE_BUFFERCACHE_SIZE_KEY)
     public long getBufferCacheSize() {
         return accessor.getProperty(STORAGE_BUFFERCACHE_SIZE_KEY, STORAGE_BUFFERCACHE_SIZE_DEFAULT,
                 PropertyInterpreters.getLongBytePropertyInterpreter());
@@ -67,38 +75,44 @@ public class AsterixStorageProperties extends AbstractAsterixProperties {
         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());
     }
 
+    @PropertyKey(STORAGE_MEMORYCOMPONENT_PAGESIZE_KEY)
     public int getMemoryComponentPageSize() {
         return accessor.getProperty(STORAGE_MEMORYCOMPONENT_PAGESIZE_KEY, STORAGE_MEMORYCOMPONENT_PAGESIZE_DEFAULT,
                 PropertyInterpreters.getIntegerBytePropertyInterpreter());
     }
 
+    @PropertyKey(STORAGE_MEMORYCOMPONENT_NUMPAGES_KEY)
     public int getMemoryComponentNumPages() {
         return accessor.getProperty(STORAGE_MEMORYCOMPONENT_NUMPAGES_KEY, STORAGE_MEMORYCOMPONENT_NUMPAGES_DEFAULT,
                 PropertyInterpreters.getIntegerPropertyInterpreter());
     }
 
+    @PropertyKey(STORAGE_METADATA_MEMORYCOMPONENT_NUMPAGES_KEY)
     public int getMetadataMemoryComponentNumPages() {
-        return accessor
-                .getProperty(STORAGE_METADATA_MEMORYCOMPONENT_NUMPAGES_KEY,
+        return accessor.getProperty(STORAGE_METADATA_MEMORYCOMPONENT_NUMPAGES_KEY,
                         STORAGE_METADATA_MEMORYCOMPONENT_NUMPAGES_DEFAULT,
                         PropertyInterpreters.getIntegerPropertyInterpreter());
     }
 
+    @PropertyKey(STORAGE_MEMORYCOMPONENT_NUMCOMPONENTS_KEY)
     public int getMemoryComponentsNum() {
         return accessor.getProperty(STORAGE_MEMORYCOMPONENT_NUMCOMPONENTS_KEY,
                 STORAGE_MEMORYCOMPONENT_NUMCOMPONENTS_DEFAULT, PropertyInterpreters.getIntegerPropertyInterpreter());
     }
 
+    @PropertyKey(STORAGE_MEMORYCOMPONENT_GLOBALBUDGET_KEY)
     public long getMemoryComponentGlobalBudget() {
         return accessor.getProperty(STORAGE_MEMORYCOMPONENT_GLOBALBUDGET_KEY,
                 STORAGE_MEMORYCOMPONENT_GLOBALBUDGET_DEFAULT, PropertyInterpreters.getLongBytePropertyInterpreter());
     }
 
+    @PropertyKey(STORAGE_LSM_BLOOMFILTER_FALSEPOSITIVERATE_KEY)
     public double getBloomFilterFalsePositiveRate() {
         return accessor.getProperty(STORAGE_LSM_BLOOMFILTER_FALSEPOSITIVERATE_KEY,
                 STORAGE_LSM_BLOOMFILTER_FALSEPOSITIVERATE_DEFAULT, PropertyInterpreters.getDoublePropertyInterpreter());

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d5b0afac/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixTransactionProperties.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixTransactionProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixTransactionProperties.java
index 00bcdc9..afc9e4f 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixTransactionProperties.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixTransactionProperties.java
@@ -21,33 +21,35 @@ package org.apache.asterix.common.config;
 import java.util.Map;
 
 import org.apache.hyracks.util.StorageUtil;
-import org.apache.hyracks.util.StorageUtil.StorageUnit;
+
+import static org.apache.hyracks.util.StorageUtil.StorageUnit.KILOBYTE;
+import static org.apache.hyracks.util.StorageUtil.StorageUnit.MEGABYTE;
 
 public class AsterixTransactionProperties extends AbstractAsterixProperties {
 
     private static final String TXN_LOG_BUFFER_NUMPAGES_KEY = "txn.log.buffer.numpages";
-    private static int TXN_LOG_BUFFER_NUMPAGES_DEFAULT = 8;
+    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 = (128 << 10); // 128KB
+    private static final int TXN_LOG_BUFFER_PAGESIZE_DEFAULT = StorageUtil.getSizeInBytes(128, KILOBYTE);
 
     private static final String TXN_LOG_PARTITIONSIZE_KEY = "txn.log.partitionsize";
-    private static final long TXN_LOG_PARTITIONSIZE_DEFAULT = StorageUtil.getSizeInBytes(256L, StorageUnit.MEGABYTE);
+    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 = (64 << 20); // 64M
+    private static final int TXN_LOG_CHECKPOINT_LSNTHRESHOLD_DEFAULT = StorageUtil.getSizeInBytes(64, MEGABYTE);
 
     private static final String TXN_LOG_CHECKPOINT_POLLFREQUENCY_KEY = "txn.log.checkpoint.pollfrequency";
-    private static int TXN_LOG_CHECKPOINT_POLLFREQUENCY_DEFAULT = 120; // 120s
+    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 int TXN_LOG_CHECKPOINT_HISTORY_DEFAULT = 0;
+    private static final int TXN_LOG_CHECKPOINT_HISTORY_DEFAULT = 0;
 
     private static final String TXN_LOCK_ESCALATIONTHRESHOLD_KEY = "txn.lock.escalationthreshold";
-    private static int TXN_LOCK_ESCALATIONTHRESHOLD_DEFAULT = 1000;
+    private static final int TXN_LOCK_ESCALATIONTHRESHOLD_DEFAULT = 1000;
 
     private static final String TXN_LOCK_SHRINKTIMER_KEY = "txn.lock.shrinktimer";
-    private static int TXN_LOCK_SHRINKTIMER_DEFAULT = 5000; // 5s
+    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
@@ -59,8 +61,7 @@ public class AsterixTransactionProperties extends AbstractAsterixProperties {
     private static final int TXN_COMMIT_PROFILER_REPORT_INTERVAL_DEFAULT = 5; // 5 seconds
 
     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,
-            StorageUnit.MEGABYTE);
+    private static final long TXN_JOB_RECOVERY_MEMORY_SIZE_DEFAULT = StorageUtil.getSizeInBytes(64L, MEGABYTE);
 
     public AsterixTransactionProperties(AsterixPropertiesAccessor accessor) {
         super(accessor);
@@ -74,61 +75,73 @@ public class AsterixTransactionProperties extends AbstractAsterixProperties {
         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());
     }
 
+    @PropertyKey(TXN_LOG_BUFFER_PAGESIZE_KEY)
     public int getLogBufferPageSize() {
         return accessor.getProperty(TXN_LOG_BUFFER_PAGESIZE_KEY, TXN_LOG_BUFFER_PAGESIZE_DEFAULT,
                 PropertyInterpreters.getIntegerBytePropertyInterpreter());
     }
 
+    @PropertyKey(TXN_LOG_PARTITIONSIZE_KEY)
     public long getLogPartitionSize() {
         return accessor.getProperty(TXN_LOG_PARTITIONSIZE_KEY, TXN_LOG_PARTITIONSIZE_DEFAULT,
                 PropertyInterpreters.getLongBytePropertyInterpreter());
     }
 
+    @PropertyKey(TXN_LOG_CHECKPOINT_LSNTHRESHOLD_KEY)
     public int getCheckpointLSNThreshold() {
         return accessor.getProperty(TXN_LOG_CHECKPOINT_LSNTHRESHOLD_KEY, TXN_LOG_CHECKPOINT_LSNTHRESHOLD_DEFAULT,
                 PropertyInterpreters.getIntegerPropertyInterpreter());
     }
 
+    @PropertyKey(TXN_LOG_CHECKPOINT_POLLFREQUENCY_KEY)
     public int getCheckpointPollFrequency() {
         return accessor.getProperty(TXN_LOG_CHECKPOINT_POLLFREQUENCY_KEY, TXN_LOG_CHECKPOINT_POLLFREQUENCY_DEFAULT,
                 PropertyInterpreters.getIntegerPropertyInterpreter());
     }
 
+    @PropertyKey(TXN_LOG_CHECKPOINT_HISTORY_KEY)
     public int getCheckpointHistory() {
         return accessor.getProperty(TXN_LOG_CHECKPOINT_HISTORY_KEY, TXN_LOG_CHECKPOINT_HISTORY_DEFAULT,
                 PropertyInterpreters.getIntegerPropertyInterpreter());
     }
 
+    @PropertyKey(TXN_LOCK_ESCALATIONTHRESHOLD_KEY)
     public int getEntityToDatasetLockEscalationThreshold() {
         return accessor.getProperty(TXN_LOCK_ESCALATIONTHRESHOLD_KEY, TXN_LOCK_ESCALATIONTHRESHOLD_DEFAULT,
                 PropertyInterpreters.getIntegerPropertyInterpreter());
     }
 
+    @PropertyKey(TXN_LOCK_SHRINKTIMER_KEY)
     public int getLockManagerShrinkTimer() {
         return accessor.getProperty(TXN_LOCK_SHRINKTIMER_KEY, TXN_LOCK_SHRINKTIMER_DEFAULT,
                 PropertyInterpreters.getIntegerPropertyInterpreter());
     }
 
+    @PropertyKey(TXN_LOCK_TIMEOUT_WAITTHRESHOLD_KEY)
     public int getTimeoutWaitThreshold() {
         return accessor.getProperty(TXN_LOCK_TIMEOUT_WAITTHRESHOLD_KEY, TXN_LOCK_TIMEOUT_WAITTHRESHOLD_DEFAULT,
                 PropertyInterpreters.getIntegerPropertyInterpreter());
     }
 
+    @PropertyKey(TXN_LOCK_TIMEOUT_SWEEPTHRESHOLD_KEY)
     public int getTimeoutSweepThreshold() {
         return accessor.getProperty(TXN_LOCK_TIMEOUT_SWEEPTHRESHOLD_KEY, TXN_LOCK_TIMEOUT_SWEEPTHRESHOLD_DEFAULT,
                 PropertyInterpreters.getIntegerPropertyInterpreter());
     }
 
+    @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());
     }
 
+    @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());

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d5b0afac/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/aql/TestExecutor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/aql/TestExecutor.java b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/aql/TestExecutor.java
index 52237d8..08cad0e 100644
--- a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/aql/TestExecutor.java
+++ b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/aql/TestExecutor.java
@@ -29,6 +29,7 @@ import java.io.PrintWriter;
 import java.io.StringWriter;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
+import java.net.Inet4Address;
 import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -38,6 +39,7 @@ import java.util.List;
 import java.util.Set;
 import java.util.logging.Level;
 import java.util.logging.Logger;
+import java.util.regex.Pattern;
 
 import org.apache.asterix.common.config.GlobalConfig;
 import org.apache.asterix.common.utils.ServletUtil.Servlets;
@@ -59,7 +61,6 @@ import org.apache.http.entity.StringEntity;
 import org.apache.http.impl.client.HttpClients;
 import org.apache.http.impl.client.StandardHttpRequestRetryHandler;
 import org.apache.http.util.EntityUtils;
-import org.json.JSONException;
 import org.json.JSONObject;
 
 public class TestExecutor {
@@ -71,6 +72,8 @@ public class TestExecutor {
     // see
     // https://stackoverflow.com/questions/417142/what-is-the-maximum-length-of-a-url-in-different-browsers/417184
     private static final long MAX_URL_LENGTH = 2000l;
+    private static final Pattern JAVA_BLOCK_COMMENT_PATTERN = Pattern.compile("/\\*.*\\*/",
+            Pattern.MULTILINE | Pattern.DOTALL);
     private static Method managixExecuteMethod = null;
     private static final HashMap<Integer, ITestServer> runningTestServers = new HashMap<>();
 
@@ -87,7 +90,7 @@ public class TestExecutor {
     }
 
     public TestExecutor() {
-        this("127.0.0.1", 19002);
+        this(Inet4Address.getLoopbackAddress().getHostAddress(), 19002);
     }
 
     public void setLibrarian(ITestLibrarian librarian) {
@@ -269,26 +272,26 @@ public class TestExecutor {
 
     protected HttpResponse checkResponse(HttpResponse httpResponse) throws Exception {
         if (httpResponse.getStatusLine().getStatusCode() != HttpStatus.SC_OK) {
-            // QQQ For now, we are indeed assuming we get back JSON errors.
-            // In future this may be changed depending on the requested
-            // output format sent to the servlet.
             String errorBody = EntityUtils.toString(httpResponse.getEntity());
+            String exceptionMsg;
             try {
+                // First try to parse the response for a JSON error response.
+
                 JSONObject result = new JSONObject(errorBody);
                 String[] errors = { result.getJSONArray("error-code").getString(0), result.getString("summary"),
                         result.getString("stacktrace") };
                 GlobalConfig.ASTERIX_LOGGER.log(Level.SEVERE, errors[2]);
-                String exceptionMsg = "HTTP operation failed: " + errors[0]
+                exceptionMsg = "HTTP operation failed: " + errors[0]
                         + "\nSTATUS LINE: " + httpResponse.getStatusLine()
                         + "\nSUMMARY: " + errors[1] + "\nSTACKTRACE: " + errors[2];
-                throw new Exception(exceptionMsg);
-            } catch (JSONException e) {
+            } catch (Exception e) {
+                // whoops, not JSON (e.g. 404) - just include the body
                 GlobalConfig.ASTERIX_LOGGER.log(Level.SEVERE, errorBody);
-                String exceptionMsg = "HTTP operation failed: response is not valid-JSON (see nested exception)"
+                exceptionMsg = "HTTP operation failed:"
                         + "\nSTATUS LINE: " + httpResponse.getStatusLine()
                         + "\nERROR_BODY: " + errorBody;
-                throw new Exception(exceptionMsg, e);
             }
+            throw new Exception(exceptionMsg);
         }
         return httpResponse;
     }
@@ -679,29 +682,22 @@ public class TestExecutor {
                 }
                 break;
             case "vmgx": // a managix command that will be executed on vagrant cc node
-                try {
-                    String output = executeVagrantManagix(pb, statement);
-                    if (output.contains("ERROR")) {
-                        throw new Exception(output);
-                    }
-                } catch (Exception e) {
-                    throw new Exception("Test \"" + testFile + "\" FAILED!\n", e);
+                String output = executeVagrantManagix(pb, statement);
+                if (output.contains("ERROR")) {
+                    throw new Exception(output);
                 }
                 break;
             case "cstate": // cluster state query
-                try {
-                    fmt = OutputFormat.forCompilationUnit(cUnit);
-                    resultStream = executeClusterStateQuery(fmt, getEndpoint(Servlets.CLUSTER_STATE));
-                    expectedResultFile = expectedResultFileCtxs.get(queryCount.intValue()).getFile();
-                    actualResultFile = testCaseCtx.getActualResultFile(cUnit, expectedResultFile, new File(actualPath));
-                    actualResultFile.getParentFile().mkdirs();
-                    writeOutputToFile(actualResultFile, resultStream);
-                    runScriptAndCompareWithResult(testFile, new PrintWriter(System.err), expectedResultFile,
-                            actualResultFile);
-                    queryCount.increment();
-                } catch (Exception e) {
-                    throw new Exception("Test \"" + testFile + "\" FAILED!\n", e);
-                }
+                fmt = OutputFormat.forCompilationUnit(cUnit);
+                String extra = stripJavaComments(statement).trim();
+                resultStream = executeClusterStateQuery(fmt, getEndpoint(Servlets.CLUSTER_STATE) + extra);
+                expectedResultFile = expectedResultFileCtxs.get(queryCount.intValue()).getFile();
+                actualResultFile = testCaseCtx.getActualResultFile(cUnit, expectedResultFile, new File(actualPath));
+                actualResultFile.getParentFile().mkdirs();
+                writeOutputToFile(actualResultFile, resultStream);
+                runScriptAndCompareWithResult(testFile, new PrintWriter(System.err), expectedResultFile,
+                        actualResultFile);
+                queryCount.increment();
                 break;
             case "server": // (start <test server name> <port>
                            // [<arg1>][<arg2>][<arg3>]...|stop (<port>|all))
@@ -809,15 +805,15 @@ public class TestExecutor {
                     executeTest(testCaseCtx, ctx, statement, isDmlRecoveryTest, pb, cUnit, queryCount,
                             expectedResultFileCtxs, testFile, actualPath);
                 } catch (Exception e) {
-                    System.err.println("testFile " + testFile.toString() + " raised an exception:");
+                    System.err.println("testFile " + testFile.toString() + " raised an exception: " + e);
                     boolean unExpectedFailure = false;
                     numOfErrors++;
+                    String expectedError = null;
                     if (cUnit.getExpectedError().size() < numOfErrors) {
                         unExpectedFailure = true;
                     } else {
                         // Get the expected exception
-                        String expectedError = cUnit.getExpectedError().get(numOfErrors - 1);
-                        System.err.println("+++++\n" + expectedError + "\n+++++\n");
+                        expectedError = cUnit.getExpectedError().get(numOfErrors - 1);
                         if (e.toString().contains(expectedError)) {
                             System.err.println("...but that was expected.");
                         } else {
@@ -827,6 +823,9 @@ public class TestExecutor {
                     if (unExpectedFailure) {
                         e.printStackTrace();
                         System.err.println("...Unexpected!");
+                        if (expectedError != null) {
+                            System.err.println("Expected to find the following in error text:\n+++++\n" + expectedError + "\n+++++");
+                        }
                         if (failedGroup != null) {
                             failedGroup.getTestCase().add(testCaseCtx.getTestCase());
                         }
@@ -862,4 +861,8 @@ public class TestExecutor {
     protected String getEndpoint(Servlets servlet) {
         return "http://" + host + ":" + port + getPath(servlet);
     }
+
+    public static String stripJavaComments(String text) {
+        return JAVA_BLOCK_COMMENT_PATTERN.matcher(text).replaceAll("");
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d5b0afac/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 e901584..a20ca50 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
@@ -29,6 +29,7 @@ import org.apache.asterix.test.runtime.HDFSCluster;
 import org.apache.asterix.testframework.context.TestCaseContext;
 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.codehaus.plexus.util.FileUtils;
 import org.junit.AfterClass;
@@ -165,6 +166,13 @@ public abstract class AbstractExecutionIT {
                 }
             }
         }
+        // For now we skip api tests.
+        for (TestGroup group : tcCtx.getTestGroups()) {
+            if (group != null && "api".equals(group.getName())) {
+                LOGGER.info("Skipping test: " + tcCtx.toString());
+                return true;
+            }
+        }
         return false;
     }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d5b0afac/asterixdb/asterix-server/src/test/java/org/apache/asterix/server/test/NCServiceExecutionIT.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-server/src/test/java/org/apache/asterix/server/test/NCServiceExecutionIT.java b/asterixdb/asterix-server/src/test/java/org/apache/asterix/server/test/NCServiceExecutionIT.java
index 003137d..ab50b51 100644
--- a/asterixdb/asterix-server/src/test/java/org/apache/asterix/server/test/NCServiceExecutionIT.java
+++ b/asterixdb/asterix-server/src/test/java/org/apache/asterix/server/test/NCServiceExecutionIT.java
@@ -164,9 +164,11 @@ public class NCServiceExecutionIT {
     }
 
     private static boolean skip(TestCaseContext tcCtx) {
-        // For now we skip feeds tests and external-library tests.
+        // For now we skip feeds tests, external-library, and api tests.
         for (TestGroup group : tcCtx.getTestGroups()) {
-            if (group.getName().startsWith("external-") || group.getName().equals("feeds")) {
+            if (group.getName().startsWith("external-")
+                    || group.getName().equals("feeds")
+                    || group.getName().equals("api")) {
                 LOGGER.info("Skipping test: " + tcCtx.toString());
                 return true;
             }