You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@asterixdb.apache.org by "Murtadha Hubail (Code Review)" <do...@asterixdb.incubator.apache.org> on 2018/11/01 16:34:20 UTC

Change in asterixdb[master]: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options

Murtadha Hubail has uploaded a new change for review.

  https://asterix-gerrit.ics.uci.edu/3017

Change subject: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options
......................................................................

[ASTERIXDB-2472][CONF] More Restrictive Configuration Options

- user model changes: no
- storage format changes: no
- interface changes: no

Details:
- To avoid accepting invalid configuration types, more restrictive
  configurations types (UNSIGNED_INTEGER) and (POSITIVE_INTERGER)
  were introduced.
- Update configuration options that accept INTEGER type to use
  the more restrictive INTEGER types.

Change-Id: I7236089d93f90ebdb57a87a8efa0a64cf98df845
---
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ActiveProperties.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ExternalProperties.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/MessagingProperties.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/MetadataProperties.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ReplicationProperties.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/StorageProperties.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/TransactionProperties.java
M hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/config/OptionTypes.java
M hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/CCConfig.java
M hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/NCConfig.java
12 files changed, 133 insertions(+), 72 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/17/3017/1

diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
index 9727d46..f9ad13f 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
@@ -496,7 +496,7 @@
 
     // Gets the parallelism parameter.
     private static int getParallelism(String parameter, int parallelismInConfiguration) {
-        IOptionType<Integer> integerIPropertyInterpreter = OptionTypes.INTEGER;
+        IOptionType<Integer> integerIPropertyInterpreter = OptionTypes.UNSIGNED_INTEGER;
         return parameter == null ? parallelismInConfiguration : integerIPropertyInterpreter.parse(parameter);
     }
 
@@ -510,13 +510,8 @@
 
     private int getSortSamples(CompilerProperties compilerProperties, Map<String, Object> querySpecificConfig) {
         String valueInQuery = (String) querySpecificConfig.get(CompilerProperties.COMPILER_SORT_SAMPLES_KEY);
-        if (valueInQuery != null) {
-            int parsedNumSamples = OptionTypes.INTEGER.parse(valueInQuery);
-            if (parsedNumSamples > 0) {
-                return parsedNumSamples;
-            }
-        }
-        return compilerProperties.getSortSamples();
+        return valueInQuery == null ? compilerProperties.getSortSamples()
+                : OptionTypes.POSITIVE_INTEGER.parse(valueInQuery);
     }
 
     // Validates if the query contains unsupported query parameters.
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ActiveProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ActiveProperties.java
index c64c287..d343a3a 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ActiveProperties.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ActiveProperties.java
@@ -18,8 +18,8 @@
  */
 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.LONG_BYTE_UNIT;
+import static org.apache.hyracks.control.common.config.OptionTypes.POSITIVE_INTEGER;
 import static org.apache.hyracks.util.StorageUtil.StorageUnit.MEGABYTE;
 
 import org.apache.hyracks.api.config.IOption;
@@ -34,8 +34,14 @@
                 LONG_BYTE_UNIT,
                 StorageUtil.getLongSizeInBytes(64L, MEGABYTE),
                 "The memory budget (in bytes) for the active runtime"),
-        ACTIVE_STOP_TIMEOUT(INTEGER, 3600, "The maximum time to wait for a graceful stop of an active runtime"),
-        ACTIVE_SUSPEND_TIMEOUT(INTEGER, 3600, "The maximum time to wait for a graceful suspend of an active runtime");
+        ACTIVE_STOP_TIMEOUT(
+                POSITIVE_INTEGER,
+                3600,
+                "The maximum time to wait for a graceful stop of an active runtime"),
+        ACTIVE_SUSPEND_TIMEOUT(
+                POSITIVE_INTEGER,
+                3600,
+                "The maximum time to wait for a graceful suspend of an active runtime");
 
         private final IOptionType type;
         private final Object defaultValue;
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java
index 4c58ad7..cd41f86 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java
@@ -52,17 +52,17 @@
                 StorageUtil.getIntSizeInBytes(32, KILOBYTE),
                 "The page size (in bytes) for computation"),
         COMPILER_PARALLELISM(
-                INTEGER,
+                UNSIGNED_INTEGER,
                 COMPILER_PARALLELISM_AS_STORAGE,
                 "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"),
-        COMPILER_STRINGOFFSET(INTEGER, 0, "Position of a first character in a String/Binary (0 or 1)"),
+        COMPILER_STRINGOFFSET(UNSIGNED_INTEGER, 0, "Position of a first character in a String/Binary (0 or 1)"),
         COMPILER_SORT_PARALLEL(BOOLEAN, AlgebricksConfig.SORT_PARALLEL, "Enabling/Disabling full parallel sort"),
         COMPILER_SORT_SAMPLES(
-                INTEGER,
+                POSITIVE_INTEGER,
                 AlgebricksConfig.SORT_SAMPLES,
                 "The number of samples parallel sorting should " + "take from each partition");
 
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ExternalProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ExternalProperties.java
index 714bc92..620e47c 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ExternalProperties.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ExternalProperties.java
@@ -18,9 +18,10 @@
  */
 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.LEVEL;
+import static org.apache.hyracks.control.common.config.OptionTypes.POSITIVE_INTEGER;
 import static org.apache.hyracks.control.common.config.OptionTypes.STRING;
+import static org.apache.hyracks.control.common.config.OptionTypes.UNSIGNED_INTEGER;
 
 import org.apache.hyracks.api.config.IOption;
 import org.apache.hyracks.api.config.IOptionType;
@@ -31,21 +32,21 @@
 public class ExternalProperties extends AbstractProperties {
 
     public enum Option implements IOption {
-        WEB_PORT(INTEGER, 19001, "The listen port of the legacy query interface"),
-        WEB_QUERYINTERFACE_PORT(INTEGER, 19006, "The listen port of the query web interface"),
-        API_PORT(INTEGER, 19002, "The listen port of the API server"),
-        ACTIVE_PORT(INTEGER, 19003, "The listen port of the active server"),
-        NC_API_PORT(INTEGER, 19004, "The listen port of the node controller API server"),
+        WEB_PORT(UNSIGNED_INTEGER, 19001, "The listen port of the legacy query interface"),
+        WEB_QUERYINTERFACE_PORT(UNSIGNED_INTEGER, 19006, "The listen port of the query web interface"),
+        API_PORT(UNSIGNED_INTEGER, 19002, "The listen port of the API server"),
+        ACTIVE_PORT(UNSIGNED_INTEGER, 19003, "The listen port of the active server"),
+        NC_API_PORT(UNSIGNED_INTEGER, 19004, "The listen port of the node controller API server"),
         LOG_LEVEL(LEVEL, Level.WARN, "The logging level for master and slave processes"),
         MAX_WAIT_ACTIVE_CLUSTER(
-                INTEGER,
+                POSITIVE_INTEGER,
                 60,
                 "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"),
         CC_JAVA_OPTS(STRING, "-Xmx1024m", "The JVM options passed to the cluster controller process by managix"),
         NC_JAVA_OPTS(STRING, "-Xmx1024m", "The JVM options passed to the node controller process(es) by managix"),
         MAX_WEB_REQUEST_SIZE(
-                INTEGER,
+                UNSIGNED_INTEGER,
                 StorageUtil.getIntSizeInBytes(50, StorageUtil.StorageUnit.MEGABYTE),
                 "The maximum accepted web request size in bytes");
 
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/MessagingProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/MessagingProperties.java
index e382293..c53becb 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/MessagingProperties.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/MessagingProperties.java
@@ -18,8 +18,8 @@
  */
 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 static org.apache.hyracks.control.common.config.OptionTypes.POSITIVE_INTEGER;
 import static org.apache.hyracks.util.StorageUtil.StorageUnit.KILOBYTE;
 
 import org.apache.hyracks.api.config.IOption;
@@ -34,7 +34,7 @@
                 INTEGER_BYTE_UNIT,
                 StorageUtil.getIntSizeInBytes(4, KILOBYTE),
                 "The frame size to be used for NC to NC messaging"),
-        MESSAGING_FRAME_COUNT(INTEGER, 512, "Number of reusable frames for NC to NC messaging");
+        MESSAGING_FRAME_COUNT(POSITIVE_INTEGER, 512, "Number of reusable frames for NC to NC messaging");
 
         private final IOptionType type;
         private final Object defaultValue;
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/MetadataProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/MetadataProperties.java
index 7325c98..7a3e707 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/MetadataProperties.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/MetadataProperties.java
@@ -18,8 +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.POSITIVE_INTEGER;
 import static org.apache.hyracks.control.common.config.OptionTypes.STRING;
+import static org.apache.hyracks.control.common.config.OptionTypes.UNSIGNED_INTEGER;
 
 import java.util.List;
 import java.util.Map;
@@ -35,9 +36,9 @@
 
     public enum Option implements IOption {
         METADATA_NODE(STRING, null),
-        METADATA_REGISTRATION_TIMEOUT_SECS(INTEGER, 60),
-        METADATA_LISTEN_PORT(INTEGER, 0),
-        METADATA_CALLBACK_PORT(INTEGER, 0);
+        METADATA_REGISTRATION_TIMEOUT_SECS(POSITIVE_INTEGER, 60),
+        METADATA_LISTEN_PORT(UNSIGNED_INTEGER, 0),
+        METADATA_CALLBACK_PORT(UNSIGNED_INTEGER, 0);
 
         private final IOptionType type;
         private final Object defaultValue;
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 18503b2..afa2c75 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
@@ -19,10 +19,11 @@
 package org.apache.asterix.common.config;
 
 import static org.apache.hyracks.control.common.config.OptionTypes.BOOLEAN;
-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;
+import static org.apache.hyracks.control.common.config.OptionTypes.POSITIVE_INTEGER;
 import static org.apache.hyracks.control.common.config.OptionTypes.STRING;
+import static org.apache.hyracks.control.common.config.OptionTypes.UNSIGNED_INTEGER;
 
 import java.util.concurrent.TimeUnit;
 
@@ -40,7 +41,7 @@
                 INTEGER_BYTE_UNIT,
                 StorageUtil.getIntSizeInBytes(128, StorageUnit.KILOBYTE),
                 "The size in bytes of each log buffer page"),
-        REPLICATION_LOG_BUFFER_NUMPAGES(INTEGER, 8, "The number of log buffer pages"),
+        REPLICATION_LOG_BUFFER_NUMPAGES(POSITIVE_INTEGER, 8, "The number of log buffer pages"),
         REPLICATION_LOG_BATCHSIZE(
                 INTEGER_BYTE_UNIT,
                 StorageUtil.getIntSizeInBytes(4, StorageUnit.KILOBYTE),
@@ -50,7 +51,7 @@
                 TimeUnit.SECONDS.toSeconds(30),
                 "The time in seconds to timeout waiting for master or replica to ack"),
         REPLICATION_ENABLED(BOOLEAN, false, "Whether or not data replication is enabled"),
-        REPLICATION_FACTOR(INTEGER, 2, "Number of replicas (backups) to maintain per master replica"),
+        REPLICATION_FACTOR(UNSIGNED_INTEGER, 2, "Number of replicas (backups) to maintain per master replica"),
         REPLICATION_STRATEGY(STRING, "none", "Replication strategy to choose");
 
         private final IOptionType type;
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 6a6e649..85e44ea 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
@@ -19,9 +19,10 @@
 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.control.common.config.OptionTypes.POSITIVE_INTEGER;
+import static org.apache.hyracks.control.common.config.OptionTypes.UNSIGNED_INTEGER;
 import static org.apache.hyracks.util.StorageUtil.StorageUnit.KILOBYTE;
 
 import java.util.function.Function;
@@ -40,13 +41,13 @@
         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_BUFFERCACHE_MAXOPENFILES(UNSIGNED_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_NUMCOMPONENTS(INTEGER, 2),
-        STORAGE_METADATA_MEMORYCOMPONENT_NUMPAGES(INTEGER, 8),
+        STORAGE_MEMORYCOMPONENT_NUMCOMPONENTS(POSITIVE_INTEGER, 2),
+        STORAGE_METADATA_MEMORYCOMPONENT_NUMPAGES(POSITIVE_INTEGER, 8),
         STORAGE_LSM_BLOOMFILTER_FALSEPOSITIVERATE(DOUBLE, 0.01d),
-        STORAGE_MAX_ACTIVE_WRITABLE_DATASETS(INTEGER, 8);
+        STORAGE_MAX_ACTIVE_WRITABLE_DATASETS(UNSIGNED_INTEGER, 8);
 
         private final IOptionType interpreter;
         private final Object defaultValue;
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 3215cd6..6ff51ca 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
@@ -19,9 +19,10 @@
 package org.apache.asterix.common.config;
 
 import static org.apache.hyracks.control.common.config.OptionTypes.BOOLEAN;
-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.control.common.config.OptionTypes.POSITIVE_INTEGER;
+import static org.apache.hyracks.control.common.config.OptionTypes.UNSIGNED_INTEGER;
 import static org.apache.hyracks.util.StorageUtil.StorageUnit.MEGABYTE;
 
 import java.util.Map;
@@ -34,7 +35,7 @@
 public class TransactionProperties extends AbstractProperties {
 
     public enum Option implements IOption {
-        TXN_LOG_BUFFER_NUMPAGES(INTEGER, 8, "The number of pages in the transaction log tail"),
+        TXN_LOG_BUFFER_NUMPAGES(POSITIVE_INTEGER, 8, "The number of pages in the transaction log tail"),
         TXN_LOG_BUFFER_PAGESIZE(
                 INTEGER_BYTE_UNIT,
                 StorageUtil.getIntSizeInBytes(4, MEGABYTE),
@@ -49,22 +50,26 @@
                 "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 transaction logs"),
         TXN_LOG_CHECKPOINT_POLLFREQUENCY(
-                INTEGER,
+                POSITIVE_INTEGER,
                 120,
-                "The frequency (in seconds) the checkpoint thread should check to see if a checkpoint should be written"),
-        TXN_LOG_CHECKPOINT_HISTORY(INTEGER, 0, "The number of checkpoints to keep in the transaction log"),
+                "The frequency (in seconds) the checkpoint thread should check to see if a checkpoint should be "
+                        + "written"),
+        TXN_LOG_CHECKPOINT_HISTORY(UNSIGNED_INTEGER, 0, "The number of checkpoints to keep in the transaction log"),
         TXN_LOCK_ESCALATIONTHRESHOLD(
-                INTEGER,
+                UNSIGNED_INTEGER,
                 1000,
                 "The maximum number of entity locks to obtain before upgrading to a dataset lock"),
         TXN_LOCK_SHRINKTIMER(
-                INTEGER,
+                POSITIVE_INTEGER,
                 5000,
                 "The time (in milliseconds) where under utilization of resources will trigger a shrink phase"),
-        TXN_LOCK_TIMEOUT_WAITTHRESHOLD(INTEGER, 60000, "Time out (in milliseconds) of waiting for a lock"),
-        TXN_LOCK_TIMEOUT_SWEEPTHRESHOLD(INTEGER, 10000, "Interval (in milliseconds) for checking lock timeout"),
+        TXN_LOCK_TIMEOUT_WAITTHRESHOLD(POSITIVE_INTEGER, 60000, "Time out (in milliseconds) of waiting for a lock"),
+        TXN_LOCK_TIMEOUT_SWEEPTHRESHOLD(
+                POSITIVE_INTEGER,
+                10000,
+                "Interval (in milliseconds) for checking lock " + "timeout"),
         TXN_COMMITPROFILER_ENABLED(BOOLEAN, false, "Enable output of commit profiler logs"),
-        TXN_COMMITPROFILER_REPORTINTERVAL(INTEGER, 5, "Interval (in seconds) to report commit profiler logs"),
+        TXN_COMMITPROFILER_REPORTINTERVAL(POSITIVE_INTEGER, 5, "Interval (in seconds) to report commit profiler logs"),
         TXN_JOB_RECOVERY_MEMORYSIZE(
                 LONG_BYTE_UNIT,
                 StorageUtil.getLongSizeInBytes(64L, MEGABYTE),
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/config/OptionTypes.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/config/OptionTypes.java
index ae424b7..7088e08 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/config/OptionTypes.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/config/OptionTypes.java
@@ -340,6 +340,54 @@
         }
     };
 
+    public static final IOptionType<Integer> UNSIGNED_INTEGER = new IOptionType<Integer>() {
+        @Override
+        public Integer parse(String s) {
+            return Integer.parseUnsignedInt(s);
+        }
+
+        @Override
+        public Integer parse(JsonNode node) {
+            return node.isNull() ? null : parse(node.asText());
+        }
+
+        @Override
+        public Class<Integer> targetType() {
+            return Integer.class;
+        }
+
+        @Override
+        public void serializeJSONField(String fieldName, Object value, ObjectNode node) {
+            node.put(fieldName, (int) value);
+        }
+    };
+
+    public static final IOptionType<Integer> POSITIVE_INTEGER = new IOptionType<Integer>() {
+        @Override
+        public Integer parse(String s) {
+            final int value = Integer.parseUnsignedInt(s);
+            if (value == 0) {
+                throw new IllegalArgumentException("Value must be greater than zero");
+            }
+            return value;
+        }
+
+        @Override
+        public Integer parse(JsonNode node) {
+            return node.isNull() ? null : parse(node.asText());
+        }
+
+        @Override
+        public Class<Integer> targetType() {
+            return Integer.class;
+        }
+
+        @Override
+        public void serializeJSONField(String fieldName, Object value, ObjectNode node) {
+            node.put(fieldName, (int) value);
+        }
+    };
+
     private OptionTypes() {
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/CCConfig.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/CCConfig.java
index 1cb2d05..5417513 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/CCConfig.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/CCConfig.java
@@ -19,10 +19,11 @@
 package org.apache.hyracks.control.common.controllers;
 
 import static org.apache.hyracks.control.common.config.OptionTypes.BOOLEAN;
-import static org.apache.hyracks.control.common.config.OptionTypes.INTEGER;
 import static org.apache.hyracks.control.common.config.OptionTypes.LONG;
+import static org.apache.hyracks.control.common.config.OptionTypes.POSITIVE_INTEGER;
 import static org.apache.hyracks.control.common.config.OptionTypes.SHORT;
 import static org.apache.hyracks.control.common.config.OptionTypes.STRING;
+import static org.apache.hyracks.control.common.config.OptionTypes.UNSIGNED_INTEGER;
 
 import java.io.File;
 import java.net.InetAddress;
@@ -47,32 +48,32 @@
         ADDRESS(STRING, InetAddress.getLoopbackAddress().getHostAddress()),
         PUBLIC_ADDRESS(STRING, ADDRESS),
         CLUSTER_LISTEN_ADDRESS(STRING, ADDRESS),
-        CLUSTER_LISTEN_PORT(INTEGER, 1099),
+        CLUSTER_LISTEN_PORT(UNSIGNED_INTEGER, 1099),
         CLUSTER_PUBLIC_ADDRESS(STRING, PUBLIC_ADDRESS),
-        CLUSTER_PUBLIC_PORT(INTEGER, CLUSTER_LISTEN_PORT),
+        CLUSTER_PUBLIC_PORT(UNSIGNED_INTEGER, CLUSTER_LISTEN_PORT),
         CLIENT_LISTEN_ADDRESS(STRING, ADDRESS),
-        CLIENT_LISTEN_PORT(INTEGER, 1098),
+        CLIENT_LISTEN_PORT(UNSIGNED_INTEGER, 1098),
         CLIENT_PUBLIC_ADDRESS(STRING, PUBLIC_ADDRESS),
-        CLIENT_PUBLIC_PORT(INTEGER, CLIENT_LISTEN_PORT),
+        CLIENT_PUBLIC_PORT(UNSIGNED_INTEGER, CLIENT_LISTEN_PORT),
         CONSOLE_LISTEN_ADDRESS(STRING, ADDRESS),
-        CONSOLE_LISTEN_PORT(INTEGER, 16001),
+        CONSOLE_LISTEN_PORT(UNSIGNED_INTEGER, 16001),
         CONSOLE_PUBLIC_ADDRESS(STRING, PUBLIC_ADDRESS),
-        CONSOLE_PUBLIC_PORT(INTEGER, CONSOLE_LISTEN_PORT),
+        CONSOLE_PUBLIC_PORT(UNSIGNED_INTEGER, CONSOLE_LISTEN_PORT),
         HEARTBEAT_PERIOD(LONG, 10000L), // TODO (mblow): add time unit
-        HEARTBEAT_MAX_MISSES(INTEGER, 5),
+        HEARTBEAT_MAX_MISSES(UNSIGNED_INTEGER, 5),
         DEAD_NODE_SWEEP_THRESHOLD(LONG, HEARTBEAT_PERIOD),
-        PROFILE_DUMP_PERIOD(INTEGER, 0),
-        JOB_HISTORY_SIZE(INTEGER, 10),
+        PROFILE_DUMP_PERIOD(UNSIGNED_INTEGER, 0),
+        JOB_HISTORY_SIZE(UNSIGNED_INTEGER, 10),
         RESULT_TTL(LONG, 86400000L), // TODO(mblow): add time unit
         RESULT_SWEEP_THRESHOLD(LONG, 60000L), // TODO(mblow): add time unit
         @SuppressWarnings("RedundantCast") // not redundant- false positive from IDEA
         ROOT_DIR(STRING, (Function<IApplicationConfig, String>) appConfig -> FileUtil.joinPath(appConfig.getString(ControllerConfig.Option.DEFAULT_DIR), "ClusterControllerService"), "<value of " + ControllerConfig.Option.DEFAULT_DIR.cmdline() + ">/ClusterControllerService"),
         CLUSTER_TOPOLOGY(STRING),
         JOB_QUEUE_CLASS(STRING, "org.apache.hyracks.control.cc.scheduler.FIFOJobQueue"),
-        JOB_QUEUE_CAPACITY(INTEGER, 4096),
+        JOB_QUEUE_CAPACITY(POSITIVE_INTEGER, 4096),
         JOB_MANAGER_CLASS(STRING, "org.apache.hyracks.control.cc.job.JobManager"),
         ENFORCE_FRAME_WRITER_PROTOCOL(BOOLEAN, false),
-        CORES_MULTIPLIER(INTEGER, 3),
+        CORES_MULTIPLIER(POSITIVE_INTEGER, 3),
         CONTROLLER_ID(SHORT, (short) 0x0000);
 
         private final IOptionType parser;
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/NCConfig.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/NCConfig.java
index 95c67df..1d94dda 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/NCConfig.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/NCConfig.java
@@ -21,8 +21,10 @@
 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;
+import static org.apache.hyracks.control.common.config.OptionTypes.POSITIVE_INTEGER;
 import static org.apache.hyracks.control.common.config.OptionTypes.STRING;
 import static org.apache.hyracks.control.common.config.OptionTypes.STRING_ARRAY;
+import static org.apache.hyracks.control.common.config.OptionTypes.UNSIGNED_INTEGER;
 
 import java.net.InetAddress;
 import java.util.ArrayList;
@@ -43,38 +45,38 @@
         ADDRESS(STRING, InetAddress.getLoopbackAddress().getHostAddress()),
         PUBLIC_ADDRESS(STRING, ADDRESS),
         CLUSTER_LISTEN_ADDRESS(STRING, ADDRESS),
-        CLUSTER_LISTEN_PORT(INTEGER, 0),
+        CLUSTER_LISTEN_PORT(UNSIGNED_INTEGER, 0),
         NCSERVICE_ADDRESS(STRING, PUBLIC_ADDRESS),
         NCSERVICE_PORT(INTEGER, 9090),
         CLUSTER_ADDRESS(STRING, (String) null),
-        CLUSTER_PORT(INTEGER, 1099),
+        CLUSTER_PORT(UNSIGNED_INTEGER, 1099),
         CLUSTER_PUBLIC_ADDRESS(STRING, PUBLIC_ADDRESS),
-        CLUSTER_PUBLIC_PORT(INTEGER, CLUSTER_LISTEN_PORT),
+        CLUSTER_PUBLIC_PORT(UNSIGNED_INTEGER, CLUSTER_LISTEN_PORT),
         NODE_ID(STRING, (String) null),
         DATA_LISTEN_ADDRESS(STRING, ADDRESS),
-        DATA_LISTEN_PORT(INTEGER, 0),
+        DATA_LISTEN_PORT(UNSIGNED_INTEGER, 0),
         DATA_PUBLIC_ADDRESS(STRING, PUBLIC_ADDRESS),
-        DATA_PUBLIC_PORT(INTEGER, DATA_LISTEN_PORT),
+        DATA_PUBLIC_PORT(UNSIGNED_INTEGER, DATA_LISTEN_PORT),
         RESULT_LISTEN_ADDRESS(STRING, ADDRESS),
-        RESULT_LISTEN_PORT(INTEGER, 0),
+        RESULT_LISTEN_PORT(UNSIGNED_INTEGER, 0),
         RESULT_PUBLIC_ADDRESS(STRING, PUBLIC_ADDRESS),
-        RESULT_PUBLIC_PORT(INTEGER, RESULT_LISTEN_PORT),
+        RESULT_PUBLIC_PORT(UNSIGNED_INTEGER, RESULT_LISTEN_PORT),
         MESSAGING_LISTEN_ADDRESS(STRING, ADDRESS),
-        MESSAGING_LISTEN_PORT(INTEGER, 0),
+        MESSAGING_LISTEN_PORT(UNSIGNED_INTEGER, 0),
         MESSAGING_PUBLIC_ADDRESS(STRING, PUBLIC_ADDRESS),
-        MESSAGING_PUBLIC_PORT(INTEGER, MESSAGING_LISTEN_PORT),
+        MESSAGING_PUBLIC_PORT(UNSIGNED_INTEGER, MESSAGING_LISTEN_PORT),
         REPLICATION_LISTEN_ADDRESS(STRING, ADDRESS),
-        REPLICATION_LISTEN_PORT(INTEGER, 2000),
+        REPLICATION_LISTEN_PORT(UNSIGNED_INTEGER, 2000),
         REPLICATION_PUBLIC_ADDRESS(STRING, PUBLIC_ADDRESS),
-        REPLICATION_PUBLIC_PORT(INTEGER, REPLICATION_LISTEN_PORT),
-        CLUSTER_CONNECT_RETRIES(INTEGER, 5),
+        REPLICATION_PUBLIC_PORT(UNSIGNED_INTEGER, REPLICATION_LISTEN_PORT),
+        CLUSTER_CONNECT_RETRIES(UNSIGNED_INTEGER, 5),
         IODEVICES(
                 STRING_ARRAY,
                 appConfig -> new String[] {
                         FileUtil.joinPath(appConfig.getString(ControllerConfig.Option.DEFAULT_DIR), "iodevice") },
                 "<value of " + ControllerConfig.Option.DEFAULT_DIR.cmdline() + ">/iodevice"),
-        NET_THREAD_COUNT(INTEGER, 1),
-        NET_BUFFER_COUNT(INTEGER, 1),
+        NET_THREAD_COUNT(POSITIVE_INTEGER, 1),
+        NET_BUFFER_COUNT(POSITIVE_INTEGER, 1),
         RESULT_TTL(LONG, 86400000L),
         RESULT_SWEEP_THRESHOLD(LONG, 60000L),
         RESULT_MANAGER_MEMORY(INTEGER_BYTE_UNIT, -1),

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3017
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newchange
Gerrit-Change-Id: I7236089d93f90ebdb57a87a8efa0a64cf98df845
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>

Change in asterixdb[master]: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-sql-execution/4681/ (1/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3017
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7236089d93f90ebdb57a87a8efa0a64cf98df845
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-sql-execution/4680/ (2/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3017
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7236089d93f90ebdb57a87a8efa0a64cf98df845
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options

Posted by "Murtadha Hubail (Code Review)" <do...@asterixdb.incubator.apache.org>.
Murtadha Hubail has posted comments on this change.

Change subject: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options
......................................................................


Patch Set 2:

(1 comment)

https://asterix-gerrit.ics.uci.edu/#/c/3017/2/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ExternalProperties.java
File asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ExternalProperties.java:

PS2, Line 35: UNSIGNED_INTEGER
> should we add a PORT type?
Currently we have cases where we are using -1 in ports to indicate that they shouldn't be used (e.g. ncservice_port). We need to fix it then we can add a valid port type.


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3017
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7236089d93f90ebdb57a87a8efa0a64cf98df845
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-HasComments: Yes

Change in asterixdb[master]: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options
......................................................................


Patch Set 2: Integration-Tests+1

Integration Tests Successful

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/7505/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3017
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7236089d93f90ebdb57a87a8efa0a64cf98df845
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options

Posted by "Murtadha Hubail (Code Review)" <do...@asterixdb.incubator.apache.org>.
Murtadha Hubail has submitted this change and it was merged.

Change subject: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options
......................................................................


[ASTERIXDB-2472][CONF] More Restrictive Configuration Options

- user model changes: no
- storage format changes: no
- interface changes: no

Details:
- To avoid accepting invalid configuration types, more restrictive
  configurations types (UNSIGNED_INTEGER) and (POSITIVE_INTERGER)
  were introduced.
- Update configuration options that accept INTEGER type to use
  the more restrictive INTEGER types.

Change-Id: I7236089d93f90ebdb57a87a8efa0a64cf98df845
Reviewed-on: https://asterix-gerrit.ics.uci.edu/3017
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
Contrib: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Murtadha Hubail <mh...@apache.org>
Reviewed-by: Michael Blow <mb...@apache.org>
---
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ActiveProperties.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ExternalProperties.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/MessagingProperties.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/MetadataProperties.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ReplicationProperties.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/StorageProperties.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/TransactionProperties.java
M hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/config/OptionTypes.java
M hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/CCConfig.java
M hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/NCConfig.java
12 files changed, 132 insertions(+), 71 deletions(-)

Approvals:
  Anon. E. Moose #1000171: 
  Jenkins: Verified; ; Verified
  Michael Blow: Looks good to me, approved
  Murtadha Hubail: Looks good to me, but someone else must approve

Objections:
  Jenkins: Violations found



diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
index 9727d46..f9ad13f 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
@@ -496,7 +496,7 @@
 
     // Gets the parallelism parameter.
     private static int getParallelism(String parameter, int parallelismInConfiguration) {
-        IOptionType<Integer> integerIPropertyInterpreter = OptionTypes.INTEGER;
+        IOptionType<Integer> integerIPropertyInterpreter = OptionTypes.UNSIGNED_INTEGER;
         return parameter == null ? parallelismInConfiguration : integerIPropertyInterpreter.parse(parameter);
     }
 
@@ -510,13 +510,8 @@
 
     private int getSortSamples(CompilerProperties compilerProperties, Map<String, Object> querySpecificConfig) {
         String valueInQuery = (String) querySpecificConfig.get(CompilerProperties.COMPILER_SORT_SAMPLES_KEY);
-        if (valueInQuery != null) {
-            int parsedNumSamples = OptionTypes.INTEGER.parse(valueInQuery);
-            if (parsedNumSamples > 0) {
-                return parsedNumSamples;
-            }
-        }
-        return compilerProperties.getSortSamples();
+        return valueInQuery == null ? compilerProperties.getSortSamples()
+                : OptionTypes.POSITIVE_INTEGER.parse(valueInQuery);
     }
 
     // Validates if the query contains unsupported query parameters.
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ActiveProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ActiveProperties.java
index c64c287..d343a3a 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ActiveProperties.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ActiveProperties.java
@@ -18,8 +18,8 @@
  */
 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.LONG_BYTE_UNIT;
+import static org.apache.hyracks.control.common.config.OptionTypes.POSITIVE_INTEGER;
 import static org.apache.hyracks.util.StorageUtil.StorageUnit.MEGABYTE;
 
 import org.apache.hyracks.api.config.IOption;
@@ -34,8 +34,14 @@
                 LONG_BYTE_UNIT,
                 StorageUtil.getLongSizeInBytes(64L, MEGABYTE),
                 "The memory budget (in bytes) for the active runtime"),
-        ACTIVE_STOP_TIMEOUT(INTEGER, 3600, "The maximum time to wait for a graceful stop of an active runtime"),
-        ACTIVE_SUSPEND_TIMEOUT(INTEGER, 3600, "The maximum time to wait for a graceful suspend of an active runtime");
+        ACTIVE_STOP_TIMEOUT(
+                POSITIVE_INTEGER,
+                3600,
+                "The maximum time to wait for a graceful stop of an active runtime"),
+        ACTIVE_SUSPEND_TIMEOUT(
+                POSITIVE_INTEGER,
+                3600,
+                "The maximum time to wait for a graceful suspend of an active runtime");
 
         private final IOptionType type;
         private final Object defaultValue;
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java
index 66c95ee..4bfbf11 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java
@@ -59,10 +59,10 @@
                         + "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"),
-        COMPILER_STRINGOFFSET(INTEGER, 0, "Position of a first character in a String/Binary (0 or 1)"),
+        COMPILER_STRINGOFFSET(UNSIGNED_INTEGER, 0, "Position of a first character in a String/Binary (0 or 1)"),
         COMPILER_SORT_PARALLEL(BOOLEAN, AlgebricksConfig.SORT_PARALLEL, "Enabling/Disabling full parallel sort"),
         COMPILER_SORT_SAMPLES(
-                INTEGER,
+                POSITIVE_INTEGER,
                 AlgebricksConfig.SORT_SAMPLES,
                 "The number of samples which parallel sorting should take from each partition");
 
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ExternalProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ExternalProperties.java
index 714bc92..620e47c 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ExternalProperties.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ExternalProperties.java
@@ -18,9 +18,10 @@
  */
 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.LEVEL;
+import static org.apache.hyracks.control.common.config.OptionTypes.POSITIVE_INTEGER;
 import static org.apache.hyracks.control.common.config.OptionTypes.STRING;
+import static org.apache.hyracks.control.common.config.OptionTypes.UNSIGNED_INTEGER;
 
 import org.apache.hyracks.api.config.IOption;
 import org.apache.hyracks.api.config.IOptionType;
@@ -31,21 +32,21 @@
 public class ExternalProperties extends AbstractProperties {
 
     public enum Option implements IOption {
-        WEB_PORT(INTEGER, 19001, "The listen port of the legacy query interface"),
-        WEB_QUERYINTERFACE_PORT(INTEGER, 19006, "The listen port of the query web interface"),
-        API_PORT(INTEGER, 19002, "The listen port of the API server"),
-        ACTIVE_PORT(INTEGER, 19003, "The listen port of the active server"),
-        NC_API_PORT(INTEGER, 19004, "The listen port of the node controller API server"),
+        WEB_PORT(UNSIGNED_INTEGER, 19001, "The listen port of the legacy query interface"),
+        WEB_QUERYINTERFACE_PORT(UNSIGNED_INTEGER, 19006, "The listen port of the query web interface"),
+        API_PORT(UNSIGNED_INTEGER, 19002, "The listen port of the API server"),
+        ACTIVE_PORT(UNSIGNED_INTEGER, 19003, "The listen port of the active server"),
+        NC_API_PORT(UNSIGNED_INTEGER, 19004, "The listen port of the node controller API server"),
         LOG_LEVEL(LEVEL, Level.WARN, "The logging level for master and slave processes"),
         MAX_WAIT_ACTIVE_CLUSTER(
-                INTEGER,
+                POSITIVE_INTEGER,
                 60,
                 "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"),
         CC_JAVA_OPTS(STRING, "-Xmx1024m", "The JVM options passed to the cluster controller process by managix"),
         NC_JAVA_OPTS(STRING, "-Xmx1024m", "The JVM options passed to the node controller process(es) by managix"),
         MAX_WEB_REQUEST_SIZE(
-                INTEGER,
+                UNSIGNED_INTEGER,
                 StorageUtil.getIntSizeInBytes(50, StorageUtil.StorageUnit.MEGABYTE),
                 "The maximum accepted web request size in bytes");
 
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/MessagingProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/MessagingProperties.java
index e382293..c53becb 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/MessagingProperties.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/MessagingProperties.java
@@ -18,8 +18,8 @@
  */
 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 static org.apache.hyracks.control.common.config.OptionTypes.POSITIVE_INTEGER;
 import static org.apache.hyracks.util.StorageUtil.StorageUnit.KILOBYTE;
 
 import org.apache.hyracks.api.config.IOption;
@@ -34,7 +34,7 @@
                 INTEGER_BYTE_UNIT,
                 StorageUtil.getIntSizeInBytes(4, KILOBYTE),
                 "The frame size to be used for NC to NC messaging"),
-        MESSAGING_FRAME_COUNT(INTEGER, 512, "Number of reusable frames for NC to NC messaging");
+        MESSAGING_FRAME_COUNT(POSITIVE_INTEGER, 512, "Number of reusable frames for NC to NC messaging");
 
         private final IOptionType type;
         private final Object defaultValue;
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/MetadataProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/MetadataProperties.java
index 7325c98..7a3e707 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/MetadataProperties.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/MetadataProperties.java
@@ -18,8 +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.POSITIVE_INTEGER;
 import static org.apache.hyracks.control.common.config.OptionTypes.STRING;
+import static org.apache.hyracks.control.common.config.OptionTypes.UNSIGNED_INTEGER;
 
 import java.util.List;
 import java.util.Map;
@@ -35,9 +36,9 @@
 
     public enum Option implements IOption {
         METADATA_NODE(STRING, null),
-        METADATA_REGISTRATION_TIMEOUT_SECS(INTEGER, 60),
-        METADATA_LISTEN_PORT(INTEGER, 0),
-        METADATA_CALLBACK_PORT(INTEGER, 0);
+        METADATA_REGISTRATION_TIMEOUT_SECS(POSITIVE_INTEGER, 60),
+        METADATA_LISTEN_PORT(UNSIGNED_INTEGER, 0),
+        METADATA_CALLBACK_PORT(UNSIGNED_INTEGER, 0);
 
         private final IOptionType type;
         private final Object defaultValue;
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 18503b2..afa2c75 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
@@ -19,10 +19,11 @@
 package org.apache.asterix.common.config;
 
 import static org.apache.hyracks.control.common.config.OptionTypes.BOOLEAN;
-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;
+import static org.apache.hyracks.control.common.config.OptionTypes.POSITIVE_INTEGER;
 import static org.apache.hyracks.control.common.config.OptionTypes.STRING;
+import static org.apache.hyracks.control.common.config.OptionTypes.UNSIGNED_INTEGER;
 
 import java.util.concurrent.TimeUnit;
 
@@ -40,7 +41,7 @@
                 INTEGER_BYTE_UNIT,
                 StorageUtil.getIntSizeInBytes(128, StorageUnit.KILOBYTE),
                 "The size in bytes of each log buffer page"),
-        REPLICATION_LOG_BUFFER_NUMPAGES(INTEGER, 8, "The number of log buffer pages"),
+        REPLICATION_LOG_BUFFER_NUMPAGES(POSITIVE_INTEGER, 8, "The number of log buffer pages"),
         REPLICATION_LOG_BATCHSIZE(
                 INTEGER_BYTE_UNIT,
                 StorageUtil.getIntSizeInBytes(4, StorageUnit.KILOBYTE),
@@ -50,7 +51,7 @@
                 TimeUnit.SECONDS.toSeconds(30),
                 "The time in seconds to timeout waiting for master or replica to ack"),
         REPLICATION_ENABLED(BOOLEAN, false, "Whether or not data replication is enabled"),
-        REPLICATION_FACTOR(INTEGER, 2, "Number of replicas (backups) to maintain per master replica"),
+        REPLICATION_FACTOR(UNSIGNED_INTEGER, 2, "Number of replicas (backups) to maintain per master replica"),
         REPLICATION_STRATEGY(STRING, "none", "Replication strategy to choose");
 
         private final IOptionType type;
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 6a6e649..85e44ea 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
@@ -19,9 +19,10 @@
 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.control.common.config.OptionTypes.POSITIVE_INTEGER;
+import static org.apache.hyracks.control.common.config.OptionTypes.UNSIGNED_INTEGER;
 import static org.apache.hyracks.util.StorageUtil.StorageUnit.KILOBYTE;
 
 import java.util.function.Function;
@@ -40,13 +41,13 @@
         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_BUFFERCACHE_MAXOPENFILES(UNSIGNED_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_NUMCOMPONENTS(INTEGER, 2),
-        STORAGE_METADATA_MEMORYCOMPONENT_NUMPAGES(INTEGER, 8),
+        STORAGE_MEMORYCOMPONENT_NUMCOMPONENTS(POSITIVE_INTEGER, 2),
+        STORAGE_METADATA_MEMORYCOMPONENT_NUMPAGES(POSITIVE_INTEGER, 8),
         STORAGE_LSM_BLOOMFILTER_FALSEPOSITIVERATE(DOUBLE, 0.01d),
-        STORAGE_MAX_ACTIVE_WRITABLE_DATASETS(INTEGER, 8);
+        STORAGE_MAX_ACTIVE_WRITABLE_DATASETS(UNSIGNED_INTEGER, 8);
 
         private final IOptionType interpreter;
         private final Object defaultValue;
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 3215cd6..6ff51ca 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
@@ -19,9 +19,10 @@
 package org.apache.asterix.common.config;
 
 import static org.apache.hyracks.control.common.config.OptionTypes.BOOLEAN;
-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.control.common.config.OptionTypes.POSITIVE_INTEGER;
+import static org.apache.hyracks.control.common.config.OptionTypes.UNSIGNED_INTEGER;
 import static org.apache.hyracks.util.StorageUtil.StorageUnit.MEGABYTE;
 
 import java.util.Map;
@@ -34,7 +35,7 @@
 public class TransactionProperties extends AbstractProperties {
 
     public enum Option implements IOption {
-        TXN_LOG_BUFFER_NUMPAGES(INTEGER, 8, "The number of pages in the transaction log tail"),
+        TXN_LOG_BUFFER_NUMPAGES(POSITIVE_INTEGER, 8, "The number of pages in the transaction log tail"),
         TXN_LOG_BUFFER_PAGESIZE(
                 INTEGER_BYTE_UNIT,
                 StorageUtil.getIntSizeInBytes(4, MEGABYTE),
@@ -49,22 +50,26 @@
                 "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 transaction logs"),
         TXN_LOG_CHECKPOINT_POLLFREQUENCY(
-                INTEGER,
+                POSITIVE_INTEGER,
                 120,
-                "The frequency (in seconds) the checkpoint thread should check to see if a checkpoint should be written"),
-        TXN_LOG_CHECKPOINT_HISTORY(INTEGER, 0, "The number of checkpoints to keep in the transaction log"),
+                "The frequency (in seconds) the checkpoint thread should check to see if a checkpoint should be "
+                        + "written"),
+        TXN_LOG_CHECKPOINT_HISTORY(UNSIGNED_INTEGER, 0, "The number of checkpoints to keep in the transaction log"),
         TXN_LOCK_ESCALATIONTHRESHOLD(
-                INTEGER,
+                UNSIGNED_INTEGER,
                 1000,
                 "The maximum number of entity locks to obtain before upgrading to a dataset lock"),
         TXN_LOCK_SHRINKTIMER(
-                INTEGER,
+                POSITIVE_INTEGER,
                 5000,
                 "The time (in milliseconds) where under utilization of resources will trigger a shrink phase"),
-        TXN_LOCK_TIMEOUT_WAITTHRESHOLD(INTEGER, 60000, "Time out (in milliseconds) of waiting for a lock"),
-        TXN_LOCK_TIMEOUT_SWEEPTHRESHOLD(INTEGER, 10000, "Interval (in milliseconds) for checking lock timeout"),
+        TXN_LOCK_TIMEOUT_WAITTHRESHOLD(POSITIVE_INTEGER, 60000, "Time out (in milliseconds) of waiting for a lock"),
+        TXN_LOCK_TIMEOUT_SWEEPTHRESHOLD(
+                POSITIVE_INTEGER,
+                10000,
+                "Interval (in milliseconds) for checking lock " + "timeout"),
         TXN_COMMITPROFILER_ENABLED(BOOLEAN, false, "Enable output of commit profiler logs"),
-        TXN_COMMITPROFILER_REPORTINTERVAL(INTEGER, 5, "Interval (in seconds) to report commit profiler logs"),
+        TXN_COMMITPROFILER_REPORTINTERVAL(POSITIVE_INTEGER, 5, "Interval (in seconds) to report commit profiler logs"),
         TXN_JOB_RECOVERY_MEMORYSIZE(
                 LONG_BYTE_UNIT,
                 StorageUtil.getLongSizeInBytes(64L, MEGABYTE),
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/config/OptionTypes.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/config/OptionTypes.java
index ae424b7..7088e08 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/config/OptionTypes.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/config/OptionTypes.java
@@ -340,6 +340,54 @@
         }
     };
 
+    public static final IOptionType<Integer> UNSIGNED_INTEGER = new IOptionType<Integer>() {
+        @Override
+        public Integer parse(String s) {
+            return Integer.parseUnsignedInt(s);
+        }
+
+        @Override
+        public Integer parse(JsonNode node) {
+            return node.isNull() ? null : parse(node.asText());
+        }
+
+        @Override
+        public Class<Integer> targetType() {
+            return Integer.class;
+        }
+
+        @Override
+        public void serializeJSONField(String fieldName, Object value, ObjectNode node) {
+            node.put(fieldName, (int) value);
+        }
+    };
+
+    public static final IOptionType<Integer> POSITIVE_INTEGER = new IOptionType<Integer>() {
+        @Override
+        public Integer parse(String s) {
+            final int value = Integer.parseUnsignedInt(s);
+            if (value == 0) {
+                throw new IllegalArgumentException("Value must be greater than zero");
+            }
+            return value;
+        }
+
+        @Override
+        public Integer parse(JsonNode node) {
+            return node.isNull() ? null : parse(node.asText());
+        }
+
+        @Override
+        public Class<Integer> targetType() {
+            return Integer.class;
+        }
+
+        @Override
+        public void serializeJSONField(String fieldName, Object value, ObjectNode node) {
+            node.put(fieldName, (int) value);
+        }
+    };
+
     private OptionTypes() {
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/CCConfig.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/CCConfig.java
index 1cb2d05..5417513 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/CCConfig.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/CCConfig.java
@@ -19,10 +19,11 @@
 package org.apache.hyracks.control.common.controllers;
 
 import static org.apache.hyracks.control.common.config.OptionTypes.BOOLEAN;
-import static org.apache.hyracks.control.common.config.OptionTypes.INTEGER;
 import static org.apache.hyracks.control.common.config.OptionTypes.LONG;
+import static org.apache.hyracks.control.common.config.OptionTypes.POSITIVE_INTEGER;
 import static org.apache.hyracks.control.common.config.OptionTypes.SHORT;
 import static org.apache.hyracks.control.common.config.OptionTypes.STRING;
+import static org.apache.hyracks.control.common.config.OptionTypes.UNSIGNED_INTEGER;
 
 import java.io.File;
 import java.net.InetAddress;
@@ -47,32 +48,32 @@
         ADDRESS(STRING, InetAddress.getLoopbackAddress().getHostAddress()),
         PUBLIC_ADDRESS(STRING, ADDRESS),
         CLUSTER_LISTEN_ADDRESS(STRING, ADDRESS),
-        CLUSTER_LISTEN_PORT(INTEGER, 1099),
+        CLUSTER_LISTEN_PORT(UNSIGNED_INTEGER, 1099),
         CLUSTER_PUBLIC_ADDRESS(STRING, PUBLIC_ADDRESS),
-        CLUSTER_PUBLIC_PORT(INTEGER, CLUSTER_LISTEN_PORT),
+        CLUSTER_PUBLIC_PORT(UNSIGNED_INTEGER, CLUSTER_LISTEN_PORT),
         CLIENT_LISTEN_ADDRESS(STRING, ADDRESS),
-        CLIENT_LISTEN_PORT(INTEGER, 1098),
+        CLIENT_LISTEN_PORT(UNSIGNED_INTEGER, 1098),
         CLIENT_PUBLIC_ADDRESS(STRING, PUBLIC_ADDRESS),
-        CLIENT_PUBLIC_PORT(INTEGER, CLIENT_LISTEN_PORT),
+        CLIENT_PUBLIC_PORT(UNSIGNED_INTEGER, CLIENT_LISTEN_PORT),
         CONSOLE_LISTEN_ADDRESS(STRING, ADDRESS),
-        CONSOLE_LISTEN_PORT(INTEGER, 16001),
+        CONSOLE_LISTEN_PORT(UNSIGNED_INTEGER, 16001),
         CONSOLE_PUBLIC_ADDRESS(STRING, PUBLIC_ADDRESS),
-        CONSOLE_PUBLIC_PORT(INTEGER, CONSOLE_LISTEN_PORT),
+        CONSOLE_PUBLIC_PORT(UNSIGNED_INTEGER, CONSOLE_LISTEN_PORT),
         HEARTBEAT_PERIOD(LONG, 10000L), // TODO (mblow): add time unit
-        HEARTBEAT_MAX_MISSES(INTEGER, 5),
+        HEARTBEAT_MAX_MISSES(UNSIGNED_INTEGER, 5),
         DEAD_NODE_SWEEP_THRESHOLD(LONG, HEARTBEAT_PERIOD),
-        PROFILE_DUMP_PERIOD(INTEGER, 0),
-        JOB_HISTORY_SIZE(INTEGER, 10),
+        PROFILE_DUMP_PERIOD(UNSIGNED_INTEGER, 0),
+        JOB_HISTORY_SIZE(UNSIGNED_INTEGER, 10),
         RESULT_TTL(LONG, 86400000L), // TODO(mblow): add time unit
         RESULT_SWEEP_THRESHOLD(LONG, 60000L), // TODO(mblow): add time unit
         @SuppressWarnings("RedundantCast") // not redundant- false positive from IDEA
         ROOT_DIR(STRING, (Function<IApplicationConfig, String>) appConfig -> FileUtil.joinPath(appConfig.getString(ControllerConfig.Option.DEFAULT_DIR), "ClusterControllerService"), "<value of " + ControllerConfig.Option.DEFAULT_DIR.cmdline() + ">/ClusterControllerService"),
         CLUSTER_TOPOLOGY(STRING),
         JOB_QUEUE_CLASS(STRING, "org.apache.hyracks.control.cc.scheduler.FIFOJobQueue"),
-        JOB_QUEUE_CAPACITY(INTEGER, 4096),
+        JOB_QUEUE_CAPACITY(POSITIVE_INTEGER, 4096),
         JOB_MANAGER_CLASS(STRING, "org.apache.hyracks.control.cc.job.JobManager"),
         ENFORCE_FRAME_WRITER_PROTOCOL(BOOLEAN, false),
-        CORES_MULTIPLIER(INTEGER, 3),
+        CORES_MULTIPLIER(POSITIVE_INTEGER, 3),
         CONTROLLER_ID(SHORT, (short) 0x0000);
 
         private final IOptionType parser;
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/NCConfig.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/NCConfig.java
index 95c67df..1d94dda 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/NCConfig.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/NCConfig.java
@@ -21,8 +21,10 @@
 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;
+import static org.apache.hyracks.control.common.config.OptionTypes.POSITIVE_INTEGER;
 import static org.apache.hyracks.control.common.config.OptionTypes.STRING;
 import static org.apache.hyracks.control.common.config.OptionTypes.STRING_ARRAY;
+import static org.apache.hyracks.control.common.config.OptionTypes.UNSIGNED_INTEGER;
 
 import java.net.InetAddress;
 import java.util.ArrayList;
@@ -43,38 +45,38 @@
         ADDRESS(STRING, InetAddress.getLoopbackAddress().getHostAddress()),
         PUBLIC_ADDRESS(STRING, ADDRESS),
         CLUSTER_LISTEN_ADDRESS(STRING, ADDRESS),
-        CLUSTER_LISTEN_PORT(INTEGER, 0),
+        CLUSTER_LISTEN_PORT(UNSIGNED_INTEGER, 0),
         NCSERVICE_ADDRESS(STRING, PUBLIC_ADDRESS),
         NCSERVICE_PORT(INTEGER, 9090),
         CLUSTER_ADDRESS(STRING, (String) null),
-        CLUSTER_PORT(INTEGER, 1099),
+        CLUSTER_PORT(UNSIGNED_INTEGER, 1099),
         CLUSTER_PUBLIC_ADDRESS(STRING, PUBLIC_ADDRESS),
-        CLUSTER_PUBLIC_PORT(INTEGER, CLUSTER_LISTEN_PORT),
+        CLUSTER_PUBLIC_PORT(UNSIGNED_INTEGER, CLUSTER_LISTEN_PORT),
         NODE_ID(STRING, (String) null),
         DATA_LISTEN_ADDRESS(STRING, ADDRESS),
-        DATA_LISTEN_PORT(INTEGER, 0),
+        DATA_LISTEN_PORT(UNSIGNED_INTEGER, 0),
         DATA_PUBLIC_ADDRESS(STRING, PUBLIC_ADDRESS),
-        DATA_PUBLIC_PORT(INTEGER, DATA_LISTEN_PORT),
+        DATA_PUBLIC_PORT(UNSIGNED_INTEGER, DATA_LISTEN_PORT),
         RESULT_LISTEN_ADDRESS(STRING, ADDRESS),
-        RESULT_LISTEN_PORT(INTEGER, 0),
+        RESULT_LISTEN_PORT(UNSIGNED_INTEGER, 0),
         RESULT_PUBLIC_ADDRESS(STRING, PUBLIC_ADDRESS),
-        RESULT_PUBLIC_PORT(INTEGER, RESULT_LISTEN_PORT),
+        RESULT_PUBLIC_PORT(UNSIGNED_INTEGER, RESULT_LISTEN_PORT),
         MESSAGING_LISTEN_ADDRESS(STRING, ADDRESS),
-        MESSAGING_LISTEN_PORT(INTEGER, 0),
+        MESSAGING_LISTEN_PORT(UNSIGNED_INTEGER, 0),
         MESSAGING_PUBLIC_ADDRESS(STRING, PUBLIC_ADDRESS),
-        MESSAGING_PUBLIC_PORT(INTEGER, MESSAGING_LISTEN_PORT),
+        MESSAGING_PUBLIC_PORT(UNSIGNED_INTEGER, MESSAGING_LISTEN_PORT),
         REPLICATION_LISTEN_ADDRESS(STRING, ADDRESS),
-        REPLICATION_LISTEN_PORT(INTEGER, 2000),
+        REPLICATION_LISTEN_PORT(UNSIGNED_INTEGER, 2000),
         REPLICATION_PUBLIC_ADDRESS(STRING, PUBLIC_ADDRESS),
-        REPLICATION_PUBLIC_PORT(INTEGER, REPLICATION_LISTEN_PORT),
-        CLUSTER_CONNECT_RETRIES(INTEGER, 5),
+        REPLICATION_PUBLIC_PORT(UNSIGNED_INTEGER, REPLICATION_LISTEN_PORT),
+        CLUSTER_CONNECT_RETRIES(UNSIGNED_INTEGER, 5),
         IODEVICES(
                 STRING_ARRAY,
                 appConfig -> new String[] {
                         FileUtil.joinPath(appConfig.getString(ControllerConfig.Option.DEFAULT_DIR), "iodevice") },
                 "<value of " + ControllerConfig.Option.DEFAULT_DIR.cmdline() + ">/iodevice"),
-        NET_THREAD_COUNT(INTEGER, 1),
-        NET_BUFFER_COUNT(INTEGER, 1),
+        NET_THREAD_COUNT(POSITIVE_INTEGER, 1),
+        NET_BUFFER_COUNT(POSITIVE_INTEGER, 1),
         RESULT_TTL(LONG, 86400000L),
         RESULT_SWEEP_THRESHOLD(LONG, 60000L),
         RESULT_MANAGER_MEMORY(INTEGER_BYTE_UNIT, -1),

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3017
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: merged
Gerrit-Change-Id: I7236089d93f90ebdb57a87a8efa0a64cf98df845
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>

Change in asterixdb[master]: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options

Posted by "Michael Blow (Code Review)" <do...@asterixdb.incubator.apache.org>.
Michael Blow has posted comments on this change.

Change subject: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options
......................................................................


Patch Set 2: Code-Review+2

(1 comment)

https://asterix-gerrit.ics.uci.edu/#/c/3017/2/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ExternalProperties.java
File asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ExternalProperties.java:

PS2, Line 35: UNSIGNED_INTEGER
should we add a PORT type?


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3017
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7236089d93f90ebdb57a87a8efa0a64cf98df845
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-HasComments: Yes

Change in asterixdb[master]: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/8627/ (3/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3017
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7236089d93f90ebdb57a87a8efa0a64cf98df845
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-asterix-app/5065/ (7/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3017
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7236089d93f90ebdb57a87a8efa0a64cf98df845
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/4589/ (1/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3017
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7236089d93f90ebdb57a87a8efa0a64cf98df845
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-stabilization-f69489-compat/108/ (12/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3017
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7236089d93f90ebdb57a87a8efa0a64cf98df845
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-no-installer-app/5019/ (13/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3017
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7236089d93f90ebdb57a87a8efa0a64cf98df845
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-stabilization-f69489-compat/109/ (12/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3017
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7236089d93f90ebdb57a87a8efa0a64cf98df845
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-openjdk11/110/ (10/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3017
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7236089d93f90ebdb57a87a8efa0a64cf98df845
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options
......................................................................


Patch Set 1: Integration-Tests+1

Integration Tests Successful

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/7504/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3017
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7236089d93f90ebdb57a87a8efa0a64cf98df845
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ensure-ancestor/2688/ (7/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3017
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7236089d93f90ebdb57a87a8efa0a64cf98df845
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/5249/ (9/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3017
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7236089d93f90ebdb57a87a8efa0a64cf98df845
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-format/4644/ (11/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3017
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7236089d93f90ebdb57a87a8efa0a64cf98df845
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-cancellation-test/4677/ (3/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3017
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7236089d93f90ebdb57a87a8efa0a64cf98df845
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/8626/ (8/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3017
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7236089d93f90ebdb57a87a8efa0a64cf98df845
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ensure-ancestor/2689/ (11/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3017
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7236089d93f90ebdb57a87a8efa0a64cf98df845
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-assemblies/4899/ (10/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3017
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7236089d93f90ebdb57a87a8efa0a64cf98df845
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options
......................................................................


Patch Set 2: Contrib+1

BAD Compatibility Tests Successful

https://asterix-jenkins.ics.uci.edu/job/asterixbad-compat/3764/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3017
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7236089d93f90ebdb57a87a8efa0a64cf98df845
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/5248/ (9/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3017
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7236089d93f90ebdb57a87a8efa0a64cf98df845
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options
......................................................................


Patch Set 2:

BAD Compatibility Tests Started https://asterix-jenkins.ics.uci.edu/job/asterixbad-compat/3764/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3017
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7236089d93f90ebdb57a87a8efa0a64cf98df845
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-cancellation-test/4678/ (6/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3017
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7236089d93f90ebdb57a87a8efa0a64cf98df845
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-asterix-app/5063/ (6/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3017
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7236089d93f90ebdb57a87a8efa0a64cf98df845
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-assemblies/4900/ (4/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3017
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7236089d93f90ebdb57a87a8efa0a64cf98df845
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-openjdk11/109/ (4/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3017
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7236089d93f90ebdb57a87a8efa0a64cf98df845
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-no-installer-app/5020/ (13/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3017
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7236089d93f90ebdb57a87a8efa0a64cf98df845
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options
......................................................................


Patch Set 2: Contrib+1

Analytics Compatibility Tests Successful
https://goo.gl/3rSZRc : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3017
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7236089d93f90ebdb57a87a8efa0a64cf98df845
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options
......................................................................


Patch Set 1:

Analytics Compatibility Compilation Successful
https://goo.gl/rBr3TL : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3017
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7236089d93f90ebdb57a87a8efa0a64cf98df845
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/4590/ (2/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3017
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7236089d93f90ebdb57a87a8efa0a64cf98df845
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options
......................................................................


Patch Set 2:

Integration Tests Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/7505/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3017
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7236089d93f90ebdb57a87a8efa0a64cf98df845
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options
......................................................................


Patch Set 1:

WARNING: THIS CHANGE CONTAINS CROSS-PRODUCT CHANGES IN:
* asterixdb
* hyracks-fullstack

PLEASE REVIEW CAREFULLY AND LOOK FOR API CHANGES!

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3017
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7236089d93f90ebdb57a87a8efa0a64cf98df845
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options
......................................................................


Patch Set 1:

Integration Tests Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/7504/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3017
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7236089d93f90ebdb57a87a8efa0a64cf98df845
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options
......................................................................


Patch Set 2:

WARNING: THIS CHANGE CONTAINS CROSS-PRODUCT CHANGES IN:
* asterixdb
* hyracks-fullstack

PLEASE REVIEW CAREFULLY AND LOOK FOR API CHANGES!

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3017
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7236089d93f90ebdb57a87a8efa0a64cf98df845
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options

Posted by "Murtadha Hubail (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Jenkins,

I'd like you to reexamine a change.  Please visit

    https://asterix-gerrit.ics.uci.edu/3017

to look at the new patch set (#2).

Change subject: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options
......................................................................

[ASTERIXDB-2472][CONF] More Restrictive Configuration Options

- user model changes: no
- storage format changes: no
- interface changes: no

Details:
- To avoid accepting invalid configuration types, more restrictive
  configurations types (UNSIGNED_INTEGER) and (POSITIVE_INTERGER)
  were introduced.
- Update configuration options that accept INTEGER type to use
  the more restrictive INTEGER types.

Change-Id: I7236089d93f90ebdb57a87a8efa0a64cf98df845
---
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ActiveProperties.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ExternalProperties.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/MessagingProperties.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/MetadataProperties.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ReplicationProperties.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/StorageProperties.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/TransactionProperties.java
M hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/config/OptionTypes.java
M hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/CCConfig.java
M hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/NCConfig.java
12 files changed, 132 insertions(+), 71 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/17/3017/2
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3017
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I7236089d93f90ebdb57a87a8efa0a64cf98df845
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>

Change in asterixdb[master]: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-format/4645/ (5/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3017
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7236089d93f90ebdb57a87a8efa0a64cf98df845
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/10160/ (8/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3017
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7236089d93f90ebdb57a87a8efa0a64cf98df845
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/10159/ (5/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3017
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7236089d93f90ebdb57a87a8efa0a64cf98df845
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options
......................................................................


Patch Set 2:

Analytics Compatibility Compilation Successful
https://goo.gl/pmnE3W : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3017
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7236089d93f90ebdb57a87a8efa0a64cf98df845
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options

Posted by "Murtadha Hubail (Code Review)" <do...@asterixdb.incubator.apache.org>.
Murtadha Hubail has posted comments on this change.

Change subject: [ASTERIXDB-2472][CONF] More Restrictive Configuration Options
......................................................................


Patch Set 2: Code-Review+1

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3017
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7236089d93f90ebdb57a87a8efa0a64cf98df845
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-HasComments: No