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)" <de...@asterixdb.apache.org> on 2019/04/23 00:18:27 UTC

Change in asterixdb[master]: [NO ISSUE][CONF] Introduce ConfigValidator

Murtadha Hubail has uploaded this change for review. ( https://asterix-gerrit.ics.uci.edu/3359


Change subject: [NO ISSUE][CONF] Introduce ConfigValidator
......................................................................

[NO ISSUE][CONF] Introduce ConfigValidator

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

Details:
- Introduce ConfigValidator that can be used to validate
  runtime changes to configurations.
- On storage compression scheme configuration changes, ensure
  new value is a registered scheme.

Change-Id: I7e9dcde9c93feb2c9e2bdfe790f121b751d4c8ad
---
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/config/ConfigValidator.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IApplicationContext.java
A asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IConfigValidator.java
A asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IConfigValidatorFactory.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/INcApplicationContext.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/compression/CompressionManager.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/CcApplicationContext.java
10 files changed, 141 insertions(+), 9 deletions(-)



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

diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/config/ConfigValidator.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/config/ConfigValidator.java
new file mode 100644
index 0000000..138a2b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/config/ConfigValidator.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.app.config;
+
+import org.apache.asterix.common.api.IConfigValidator;
+import org.apache.asterix.common.config.StorageProperties;
+import org.apache.asterix.runtime.compression.CompressionManager;
+import org.apache.hyracks.api.config.IOption;
+
+public class ConfigValidator implements IConfigValidator {
+
+    @Override
+    public void validate(IOption option, Object value) {
+        boolean valid = true;
+        if (option == StorageProperties.Option.STORAGE_COMPRESSION_BLOCK) {
+            valid = CompressionManager.isRegisteredScheme((String) value);
+        }
+        if (!valid) {
+            throw new IllegalArgumentException("Invalid value " + value + " for option " + option.name());
+        }
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
index 8b8f5a0..182f982 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
@@ -31,6 +31,8 @@
 import java.util.stream.Collectors;
 
 import org.apache.asterix.active.ActiveManager;
+import org.apache.asterix.common.api.IConfigValidator;
+import org.apache.asterix.common.api.IConfigValidatorFactory;
 import org.apache.asterix.common.api.ICoordinationService;
 import org.apache.asterix.common.api.IDatasetLifecycleManager;
 import org.apache.asterix.common.api.IDatasetMemoryManager;
@@ -153,6 +155,7 @@
     private IReplicaManager replicaManager;
     private IReceptionist receptionist;
     private ICacheManager cacheManager;
+    private IConfigValidator configValidator;
 
     public NCAppRuntimeContext(INCServiceContext ncServiceContext, List<AsterixExtension> extensions,
             IPropertiesFactory propertiesFactory) throws AsterixException, InstantiationException,
@@ -184,7 +187,7 @@
 
     @Override
     public void initialize(IRecoveryManagerFactory recoveryManagerFactory, IReceptionistFactory receptionistFactory,
-            boolean initialRun) throws IOException {
+            IConfigValidatorFactory configValidatorFactory, boolean initialRun) throws IOException {
         ioManager = getServiceContext().getIoManager();
         int ioQueueLen = getServiceContext().getAppConfig().getInt(NCConfig.Option.IO_QUEUE_SIZE);
         threadExecutor =
@@ -201,6 +204,7 @@
                         persistedResourceRegistry);
         localResourceRepository =
                 (PersistentLocalResourceRepository) persistentLocalResourceRepositoryFactory.createRepository();
+        configValidator = configValidatorFactory.create();
         txnSubsystem = new TransactionSubsystem(this, recoveryManagerFactory);
         IRecoveryManager recoveryMgr = txnSubsystem.getRecoveryManager();
         SystemState systemState = recoveryMgr.getSystemState();
@@ -554,4 +558,9 @@
     public ICacheManager getCacheManager() {
         return cacheManager;
     }
+
+    @Override
+    public IConfigValidator getConfigValidator() {
+        return configValidator;
+    }
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
index b7ef0dc..76e1d88 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
@@ -50,9 +50,11 @@
 import org.apache.asterix.api.http.server.VersionApiServlet;
 import org.apache.asterix.app.active.ActiveNotificationHandler;
 import org.apache.asterix.app.cc.CCExtensionManager;
+import org.apache.asterix.app.config.ConfigValidator;
 import org.apache.asterix.app.external.ExternalLibraryUtils;
 import org.apache.asterix.app.replication.NcLifecycleCoordinator;
 import org.apache.asterix.common.api.AsterixThreadFactory;
+import org.apache.asterix.common.api.IConfigValidatorFactory;
 import org.apache.asterix.common.api.INodeJobTracker;
 import org.apache.asterix.common.api.IReceptionistFactory;
 import org.apache.asterix.common.cluster.IGlobalRecoveryManager;
@@ -154,7 +156,7 @@
         ccExtensionManager = new CCExtensionManager(extensions);
         IGlobalRecoveryManager globalRecoveryManager = createGlobalRecoveryManager();
         appCtx = createApplicationContext(libraryManager, globalRecoveryManager, lifecycleCoordinator,
-                () -> new Receptionist("CC"));
+                () -> new Receptionist("CC"), ConfigValidator::new);
         appCtx.setExtensionManager(ccExtensionManager);
         final CCConfig ccConfig = controllerService.getCCConfig();
         if (System.getProperty("java.rmi.server.hostname") == null) {
@@ -182,10 +184,11 @@
 
     protected ICcApplicationContext createApplicationContext(ILibraryManager libraryManager,
             IGlobalRecoveryManager globalRecoveryManager, INcLifecycleCoordinator lifecycleCoordinator,
-            IReceptionistFactory receptionistFactory) throws AlgebricksException, IOException {
+            IReceptionistFactory receptionistFactory, IConfigValidatorFactory configValidatorFactory)
+            throws AlgebricksException, IOException {
         return new CcApplicationContext(ccServiceCtx, getHcc(), libraryManager, () -> MetadataManager.INSTANCE,
                 globalRecoveryManager, lifecycleCoordinator, new ActiveNotificationHandler(), componentProvider,
-                new MetadataLockManager(), receptionistFactory);
+                new MetadataLockManager(), receptionistFactory, configValidatorFactory);
     }
 
     protected IGlobalRecoveryManager createGlobalRecoveryManager() throws Exception {
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java
index 4213509..9135688 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java
@@ -27,11 +27,13 @@
 import org.apache.asterix.api.http.server.NetDiagnosticsApiServlet;
 import org.apache.asterix.api.http.server.ServletConstants;
 import org.apache.asterix.api.http.server.StorageApiServlet;
+import org.apache.asterix.app.config.ConfigValidator;
 import org.apache.asterix.app.io.PersistedResourceRegistry;
 import org.apache.asterix.app.nc.NCAppRuntimeContext;
 import org.apache.asterix.app.nc.RecoveryManager;
 import org.apache.asterix.app.replication.message.RegistrationTasksRequestMessage;
 import org.apache.asterix.common.api.AsterixThreadFactory;
+import org.apache.asterix.common.api.IConfigValidatorFactory;
 import org.apache.asterix.common.api.INcApplicationContext;
 import org.apache.asterix.common.api.IPropertiesFactory;
 import org.apache.asterix.common.api.IReceptionistFactory;
@@ -130,7 +132,7 @@
             }
             updateOnNodeJoin();
         }
-        runtimeContext.initialize(getRecoveryManagerFactory(), getReceptionistFactory(),
+        runtimeContext.initialize(getRecoveryManagerFactory(), getReceptionistFactory(), getConfigValidatorFactory(),
                 runtimeContext.getNodeProperties().isInitialRun());
         MessagingProperties messagingProperties = runtimeContext.getMessagingProperties();
         NCMessageBroker messageBroker = new NCMessageBroker(controllerService, messagingProperties);
@@ -162,6 +164,10 @@
         return () -> new Receptionist(nodeId);
     }
 
+    protected IConfigValidatorFactory getConfigValidatorFactory() {
+        return ConfigValidator::new;
+    }
+
     @Override
     public void configureLoggingLevel(Level level) {
         super.configureLoggingLevel(level);
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IApplicationContext.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IApplicationContext.java
index 9e5801f..a9b306f 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IApplicationContext.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IApplicationContext.java
@@ -78,4 +78,9 @@
     ICoordinationService getCoordinationService();
 
     IReceptionist getReceptionist();
+
+    /**
+     * @return the configuration validator
+     */
+    IConfigValidator getConfigValidator();
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IConfigValidator.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IConfigValidator.java
new file mode 100644
index 0000000..a8df7be
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IConfigValidator.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.common.api;
+
+import org.apache.hyracks.api.config.IOption;
+
+public interface IConfigValidator {
+
+    /**
+     * Validates that {@code value} is a valid value for {@code option}
+     *
+     * @param option
+     * @param value
+     */
+    void validate(IOption option, Object value);
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IConfigValidatorFactory.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IConfigValidatorFactory.java
new file mode 100644
index 0000000..bb6e8d9
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IConfigValidatorFactory.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.common.api;
+
+@FunctionalInterface
+public interface IConfigValidatorFactory {
+
+    /**
+     * Creates a {@link IConfigValidator}
+     *
+     * @return a configValidator
+     */
+    IConfigValidator create();
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/INcApplicationContext.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/INcApplicationContext.java
index a18ff93..79828e1 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/INcApplicationContext.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/INcApplicationContext.java
@@ -69,7 +69,7 @@
     IResourceIdFactory getResourceIdFactory();
 
     void initialize(IRecoveryManagerFactory recoveryManagerFactory, IReceptionistFactory receptionistFactory,
-            boolean initialRun) throws IOException, AlgebricksException;
+            IConfigValidatorFactory configValidatorFactory, boolean initialRun) throws IOException, AlgebricksException;
 
     void setShuttingdown(boolean b);
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/compression/CompressionManager.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/compression/CompressionManager.java
index 3bffa9a..e34c4b1 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/compression/CompressionManager.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/compression/CompressionManager.java
@@ -94,7 +94,7 @@
      * @return
      *         true if it is registered
      */
-    private boolean isRegisteredScheme(String schemeName) {
+    public static boolean isRegisteredScheme(String schemeName) {
         return schemeName != null && REGISTERED_SCHEMES.containsKey(schemeName.toLowerCase());
     }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/CcApplicationContext.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/CcApplicationContext.java
index b92a15e..c72bf54 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/CcApplicationContext.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/CcApplicationContext.java
@@ -21,6 +21,8 @@
 import java.io.IOException;
 import java.util.function.Supplier;
 
+import org.apache.asterix.common.api.IConfigValidator;
+import org.apache.asterix.common.api.IConfigValidatorFactory;
 import org.apache.asterix.common.api.ICoordinationService;
 import org.apache.asterix.common.api.IMetadataLockManager;
 import org.apache.asterix.common.api.INodeJobTracker;
@@ -95,13 +97,14 @@
     private final ICompressionManager compressionManager;
     private final IReceptionist receptionist;
     private final IRequestTracker requestTracker;
+    private final IConfigValidator configValidator;
 
     public CcApplicationContext(ICCServiceContext ccServiceCtx, IHyracksClientConnection hcc,
             ILibraryManager libraryManager, Supplier<IMetadataBootstrap> metadataBootstrapSupplier,
             IGlobalRecoveryManager globalRecoveryManager, INcLifecycleCoordinator ftStrategy,
             IJobLifecycleListener activeLifeCycleListener, IStorageComponentProvider storageComponentProvider,
-            IMetadataLockManager mdLockManager, IReceptionistFactory receptionistFactory)
-            throws AlgebricksException, IOException {
+            IMetadataLockManager mdLockManager, IReceptionistFactory receptionistFactory,
+            IConfigValidatorFactory configValidatorFactory) throws AlgebricksException, IOException {
         this.ccServiceCtx = ccServiceCtx;
         this.hcc = hcc;
         this.libraryManager = libraryManager;
@@ -133,6 +136,7 @@
         compressionManager = new CompressionManager(storageProperties);
         receptionist = receptionistFactory.create();
         requestTracker = new RequestTracker(this);
+        configValidator = configValidatorFactory.create();
     }
 
     @Override
@@ -297,6 +301,11 @@
     }
 
     @Override
+    public IConfigValidator getConfigValidator() {
+        return configValidator;
+    }
+
+    @Override
     public IRequestTracker getRequestTracker() {
         return requestTracker;
     }

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

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

Change in asterixdb[master]: [NO ISSUE][CONF] Introduce ConfigValidator

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3359 )

Change subject: [NO ISSUE][CONF] Introduce ConfigValidator
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-txnlog/828/ (11/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I7e9dcde9c93feb2c9e2bdfe790f121b751d4c8ad
Gerrit-Change-Number: 3359
Gerrit-PatchSet: 1
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Tue, 23 Apr 2019 00:18:35 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][CONF] Introduce ConfigValidator

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3359 )

Change subject: [NO ISSUE][CONF] Introduce ConfigValidator
......................................................................


Patch Set 1:

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I7e9dcde9c93feb2c9e2bdfe790f121b751d4c8ad
Gerrit-Change-Number: 3359
Gerrit-PatchSet: 1
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Tue, 23 Apr 2019 00:18:33 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][CONF] Introduce ConfigValidator

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3359 )

Change subject: [NO ISSUE][CONF] Introduce ConfigValidator
......................................................................


Patch Set 1: Integration-Tests+1

Integration Tests Successful

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I7e9dcde9c93feb2c9e2bdfe790f121b751d4c8ad
Gerrit-Change-Number: 3359
Gerrit-PatchSet: 1
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Tue, 23 Apr 2019 01:45:21 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][CONF] Introduce ConfigValidator

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3359 )

Change subject: [NO ISSUE][CONF] Introduce ConfigValidator
......................................................................


Patch Set 1:

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I7e9dcde9c93feb2c9e2bdfe790f121b751d4c8ad
Gerrit-Change-Number: 3359
Gerrit-PatchSet: 1
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Tue, 23 Apr 2019 00:18:31 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][CONF] Introduce ConfigValidator

Posted by "Murtadha Hubail (Code Review)" <de...@asterixdb.apache.org>.
Murtadha Hubail has submitted this change and it was merged. ( https://asterix-gerrit.ics.uci.edu/3359 )

Change subject: [NO ISSUE][CONF] Introduce ConfigValidator
......................................................................

[NO ISSUE][CONF] Introduce ConfigValidator

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

Details:
- Introduce ConfigValidator that can be used to validate
  runtime changes to configurations.
- On storage compression scheme configuration changes, ensure
  new value is a registered scheme.

Change-Id: I7e9dcde9c93feb2c9e2bdfe790f121b751d4c8ad
Reviewed-on: https://asterix-gerrit.ics.uci.edu/3359
Contrib: Jenkins <je...@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Till Westmann <ti...@apache.org>
---
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/config/ConfigValidator.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IApplicationContext.java
A asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IConfigValidator.java
A asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IConfigValidatorFactory.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/INcApplicationContext.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/compression/CompressionManager.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/CcApplicationContext.java
10 files changed, 141 insertions(+), 9 deletions(-)

Approvals:
  Jenkins: Verified; ; Verified
  Anon. E. Moose (1000171): 
  Till Westmann: Looks good to me, approved

Objections:
  Jenkins: Violations found



diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/config/ConfigValidator.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/config/ConfigValidator.java
new file mode 100644
index 0000000..138a2b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/config/ConfigValidator.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.app.config;
+
+import org.apache.asterix.common.api.IConfigValidator;
+import org.apache.asterix.common.config.StorageProperties;
+import org.apache.asterix.runtime.compression.CompressionManager;
+import org.apache.hyracks.api.config.IOption;
+
+public class ConfigValidator implements IConfigValidator {
+
+    @Override
+    public void validate(IOption option, Object value) {
+        boolean valid = true;
+        if (option == StorageProperties.Option.STORAGE_COMPRESSION_BLOCK) {
+            valid = CompressionManager.isRegisteredScheme((String) value);
+        }
+        if (!valid) {
+            throw new IllegalArgumentException("Invalid value " + value + " for option " + option.name());
+        }
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
index 8b8f5a0..182f982 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
@@ -31,6 +31,8 @@
 import java.util.stream.Collectors;
 
 import org.apache.asterix.active.ActiveManager;
+import org.apache.asterix.common.api.IConfigValidator;
+import org.apache.asterix.common.api.IConfigValidatorFactory;
 import org.apache.asterix.common.api.ICoordinationService;
 import org.apache.asterix.common.api.IDatasetLifecycleManager;
 import org.apache.asterix.common.api.IDatasetMemoryManager;
@@ -153,6 +155,7 @@
     private IReplicaManager replicaManager;
     private IReceptionist receptionist;
     private ICacheManager cacheManager;
+    private IConfigValidator configValidator;
 
     public NCAppRuntimeContext(INCServiceContext ncServiceContext, List<AsterixExtension> extensions,
             IPropertiesFactory propertiesFactory) throws AsterixException, InstantiationException,
@@ -184,7 +187,7 @@
 
     @Override
     public void initialize(IRecoveryManagerFactory recoveryManagerFactory, IReceptionistFactory receptionistFactory,
-            boolean initialRun) throws IOException {
+            IConfigValidatorFactory configValidatorFactory, boolean initialRun) throws IOException {
         ioManager = getServiceContext().getIoManager();
         int ioQueueLen = getServiceContext().getAppConfig().getInt(NCConfig.Option.IO_QUEUE_SIZE);
         threadExecutor =
@@ -201,6 +204,7 @@
                         persistedResourceRegistry);
         localResourceRepository =
                 (PersistentLocalResourceRepository) persistentLocalResourceRepositoryFactory.createRepository();
+        configValidator = configValidatorFactory.create();
         txnSubsystem = new TransactionSubsystem(this, recoveryManagerFactory);
         IRecoveryManager recoveryMgr = txnSubsystem.getRecoveryManager();
         SystemState systemState = recoveryMgr.getSystemState();
@@ -554,4 +558,9 @@
     public ICacheManager getCacheManager() {
         return cacheManager;
     }
+
+    @Override
+    public IConfigValidator getConfigValidator() {
+        return configValidator;
+    }
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
index b7ef0dc..76e1d88 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
@@ -50,9 +50,11 @@
 import org.apache.asterix.api.http.server.VersionApiServlet;
 import org.apache.asterix.app.active.ActiveNotificationHandler;
 import org.apache.asterix.app.cc.CCExtensionManager;
+import org.apache.asterix.app.config.ConfigValidator;
 import org.apache.asterix.app.external.ExternalLibraryUtils;
 import org.apache.asterix.app.replication.NcLifecycleCoordinator;
 import org.apache.asterix.common.api.AsterixThreadFactory;
+import org.apache.asterix.common.api.IConfigValidatorFactory;
 import org.apache.asterix.common.api.INodeJobTracker;
 import org.apache.asterix.common.api.IReceptionistFactory;
 import org.apache.asterix.common.cluster.IGlobalRecoveryManager;
@@ -154,7 +156,7 @@
         ccExtensionManager = new CCExtensionManager(extensions);
         IGlobalRecoveryManager globalRecoveryManager = createGlobalRecoveryManager();
         appCtx = createApplicationContext(libraryManager, globalRecoveryManager, lifecycleCoordinator,
-                () -> new Receptionist("CC"));
+                () -> new Receptionist("CC"), ConfigValidator::new);
         appCtx.setExtensionManager(ccExtensionManager);
         final CCConfig ccConfig = controllerService.getCCConfig();
         if (System.getProperty("java.rmi.server.hostname") == null) {
@@ -182,10 +184,11 @@
 
     protected ICcApplicationContext createApplicationContext(ILibraryManager libraryManager,
             IGlobalRecoveryManager globalRecoveryManager, INcLifecycleCoordinator lifecycleCoordinator,
-            IReceptionistFactory receptionistFactory) throws AlgebricksException, IOException {
+            IReceptionistFactory receptionistFactory, IConfigValidatorFactory configValidatorFactory)
+            throws AlgebricksException, IOException {
         return new CcApplicationContext(ccServiceCtx, getHcc(), libraryManager, () -> MetadataManager.INSTANCE,
                 globalRecoveryManager, lifecycleCoordinator, new ActiveNotificationHandler(), componentProvider,
-                new MetadataLockManager(), receptionistFactory);
+                new MetadataLockManager(), receptionistFactory, configValidatorFactory);
     }
 
     protected IGlobalRecoveryManager createGlobalRecoveryManager() throws Exception {
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java
index 4213509..9135688 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java
@@ -27,11 +27,13 @@
 import org.apache.asterix.api.http.server.NetDiagnosticsApiServlet;
 import org.apache.asterix.api.http.server.ServletConstants;
 import org.apache.asterix.api.http.server.StorageApiServlet;
+import org.apache.asterix.app.config.ConfigValidator;
 import org.apache.asterix.app.io.PersistedResourceRegistry;
 import org.apache.asterix.app.nc.NCAppRuntimeContext;
 import org.apache.asterix.app.nc.RecoveryManager;
 import org.apache.asterix.app.replication.message.RegistrationTasksRequestMessage;
 import org.apache.asterix.common.api.AsterixThreadFactory;
+import org.apache.asterix.common.api.IConfigValidatorFactory;
 import org.apache.asterix.common.api.INcApplicationContext;
 import org.apache.asterix.common.api.IPropertiesFactory;
 import org.apache.asterix.common.api.IReceptionistFactory;
@@ -130,7 +132,7 @@
             }
             updateOnNodeJoin();
         }
-        runtimeContext.initialize(getRecoveryManagerFactory(), getReceptionistFactory(),
+        runtimeContext.initialize(getRecoveryManagerFactory(), getReceptionistFactory(), getConfigValidatorFactory(),
                 runtimeContext.getNodeProperties().isInitialRun());
         MessagingProperties messagingProperties = runtimeContext.getMessagingProperties();
         NCMessageBroker messageBroker = new NCMessageBroker(controllerService, messagingProperties);
@@ -162,6 +164,10 @@
         return () -> new Receptionist(nodeId);
     }
 
+    protected IConfigValidatorFactory getConfigValidatorFactory() {
+        return ConfigValidator::new;
+    }
+
     @Override
     public void configureLoggingLevel(Level level) {
         super.configureLoggingLevel(level);
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IApplicationContext.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IApplicationContext.java
index 9e5801f..a9b306f 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IApplicationContext.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IApplicationContext.java
@@ -78,4 +78,9 @@
     ICoordinationService getCoordinationService();
 
     IReceptionist getReceptionist();
+
+    /**
+     * @return the configuration validator
+     */
+    IConfigValidator getConfigValidator();
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IConfigValidator.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IConfigValidator.java
new file mode 100644
index 0000000..a8df7be
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IConfigValidator.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.common.api;
+
+import org.apache.hyracks.api.config.IOption;
+
+public interface IConfigValidator {
+
+    /**
+     * Validates that {@code value} is a valid value for {@code option}
+     *
+     * @param option
+     * @param value
+     */
+    void validate(IOption option, Object value);
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IConfigValidatorFactory.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IConfigValidatorFactory.java
new file mode 100644
index 0000000..bb6e8d9
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IConfigValidatorFactory.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.common.api;
+
+@FunctionalInterface
+public interface IConfigValidatorFactory {
+
+    /**
+     * Creates a {@link IConfigValidator}
+     *
+     * @return a configValidator
+     */
+    IConfigValidator create();
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/INcApplicationContext.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/INcApplicationContext.java
index a18ff93..79828e1 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/INcApplicationContext.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/INcApplicationContext.java
@@ -69,7 +69,7 @@
     IResourceIdFactory getResourceIdFactory();
 
     void initialize(IRecoveryManagerFactory recoveryManagerFactory, IReceptionistFactory receptionistFactory,
-            boolean initialRun) throws IOException, AlgebricksException;
+            IConfigValidatorFactory configValidatorFactory, boolean initialRun) throws IOException, AlgebricksException;
 
     void setShuttingdown(boolean b);
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/compression/CompressionManager.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/compression/CompressionManager.java
index 3bffa9a..e34c4b1 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/compression/CompressionManager.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/compression/CompressionManager.java
@@ -94,7 +94,7 @@
      * @return
      *         true if it is registered
      */
-    private boolean isRegisteredScheme(String schemeName) {
+    public static boolean isRegisteredScheme(String schemeName) {
         return schemeName != null && REGISTERED_SCHEMES.containsKey(schemeName.toLowerCase());
     }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/CcApplicationContext.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/CcApplicationContext.java
index b92a15e..c72bf54 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/CcApplicationContext.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/CcApplicationContext.java
@@ -21,6 +21,8 @@
 import java.io.IOException;
 import java.util.function.Supplier;
 
+import org.apache.asterix.common.api.IConfigValidator;
+import org.apache.asterix.common.api.IConfigValidatorFactory;
 import org.apache.asterix.common.api.ICoordinationService;
 import org.apache.asterix.common.api.IMetadataLockManager;
 import org.apache.asterix.common.api.INodeJobTracker;
@@ -95,13 +97,14 @@
     private final ICompressionManager compressionManager;
     private final IReceptionist receptionist;
     private final IRequestTracker requestTracker;
+    private final IConfigValidator configValidator;
 
     public CcApplicationContext(ICCServiceContext ccServiceCtx, IHyracksClientConnection hcc,
             ILibraryManager libraryManager, Supplier<IMetadataBootstrap> metadataBootstrapSupplier,
             IGlobalRecoveryManager globalRecoveryManager, INcLifecycleCoordinator ftStrategy,
             IJobLifecycleListener activeLifeCycleListener, IStorageComponentProvider storageComponentProvider,
-            IMetadataLockManager mdLockManager, IReceptionistFactory receptionistFactory)
-            throws AlgebricksException, IOException {
+            IMetadataLockManager mdLockManager, IReceptionistFactory receptionistFactory,
+            IConfigValidatorFactory configValidatorFactory) throws AlgebricksException, IOException {
         this.ccServiceCtx = ccServiceCtx;
         this.hcc = hcc;
         this.libraryManager = libraryManager;
@@ -133,6 +136,7 @@
         compressionManager = new CompressionManager(storageProperties);
         receptionist = receptionistFactory.create();
         requestTracker = new RequestTracker(this);
+        configValidator = configValidatorFactory.create();
     }
 
     @Override
@@ -297,6 +301,11 @@
     }
 
     @Override
+    public IConfigValidator getConfigValidator() {
+        return configValidator;
+    }
+
+    @Override
     public IRequestTracker getRequestTracker() {
         return requestTracker;
     }

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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: merged
Gerrit-Change-Id: I7e9dcde9c93feb2c9e2bdfe790f121b751d4c8ad
Gerrit-Change-Number: 3359
Gerrit-PatchSet: 2
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-Reviewer: Till Westmann <ti...@apache.org>

Change in asterixdb[master]: [NO ISSUE][CONF] Introduce ConfigValidator

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3359 )

Change subject: [NO ISSUE][CONF] Introduce ConfigValidator
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-cancellation-test/5660/ (2/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I7e9dcde9c93feb2c9e2bdfe790f121b751d4c8ad
Gerrit-Change-Number: 3359
Gerrit-PatchSet: 1
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Tue, 23 Apr 2019 00:18:32 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][CONF] Introduce ConfigValidator

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3359 )

Change subject: [NO ISSUE][CONF] Introduce ConfigValidator
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-spidersilk-tests/591/ (13/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I7e9dcde9c93feb2c9e2bdfe790f121b751d4c8ad
Gerrit-Change-Number: 3359
Gerrit-PatchSet: 1
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Tue, 23 Apr 2019 00:18:38 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][CONF] Introduce ConfigValidator

Posted by "Anon. E. Moose (Code Review)" <de...@asterixdb.apache.org>.
Anon. E. Moose (1000171) has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3359 )

Change subject: [NO ISSUE][CONF] Introduce ConfigValidator
......................................................................


Patch Set 1: Contrib-2

Analytics Compatibility Compilation Failed
https://cbjenkins.page.link/5gxxEUdfBnsU9zv46 : UNSTABLE


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I7e9dcde9c93feb2c9e2bdfe790f121b751d4c8ad
Gerrit-Change-Number: 3359
Gerrit-PatchSet: 1
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Tue, 23 Apr 2019 00:22:33 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][CONF] Introduce ConfigValidator

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3359 )

Change subject: [NO ISSUE][CONF] Introduce ConfigValidator
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/9608/ (6/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I7e9dcde9c93feb2c9e2bdfe790f121b751d4c8ad
Gerrit-Change-Number: 3359
Gerrit-PatchSet: 1
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Tue, 23 Apr 2019 00:18:33 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][CONF] Introduce ConfigValidator

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3359 )

Change subject: [NO ISSUE][CONF] Introduce ConfigValidator
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-openjdk11/1019/ (16/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I7e9dcde9c93feb2c9e2bdfe790f121b751d4c8ad
Gerrit-Change-Number: 3359
Gerrit-PatchSet: 1
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Tue, 23 Apr 2019 00:36:10 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][CONF] Introduce ConfigValidator

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3359 )

Change subject: [NO ISSUE][CONF] Introduce ConfigValidator
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-stabilization-f69489-compat/926/ (10/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I7e9dcde9c93feb2c9e2bdfe790f121b751d4c8ad
Gerrit-Change-Number: 3359
Gerrit-PatchSet: 1
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Tue, 23 Apr 2019 00:18:34 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][CONF] Introduce ConfigValidator

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3359 )

Change subject: [NO ISSUE][CONF] Introduce ConfigValidator
......................................................................


Patch Set 1:

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I7e9dcde9c93feb2c9e2bdfe790f121b751d4c8ad
Gerrit-Change-Number: 3359
Gerrit-PatchSet: 1
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Tue, 23 Apr 2019 00:21:32 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][CONF] Introduce ConfigValidator

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3359 )

Change subject: [NO ISSUE][CONF] Introduce ConfigValidator
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ensure-ancestor/3662/ (8/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I7e9dcde9c93feb2c9e2bdfe790f121b751d4c8ad
Gerrit-Change-Number: 3359
Gerrit-PatchSet: 1
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Tue, 23 Apr 2019 00:18:33 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][CONF] Introduce ConfigValidator

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3359 )

Change subject: [NO ISSUE][CONF] Introduce ConfigValidator
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-assemblies/5872/ (14/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I7e9dcde9c93feb2c9e2bdfe790f121b751d4c8ad
Gerrit-Change-Number: 3359
Gerrit-PatchSet: 1
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Tue, 23 Apr 2019 00:18:40 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][CONF] Introduce ConfigValidator

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3359 )

Change subject: [NO ISSUE][CONF] Introduce ConfigValidator
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-sql-execution/5649/ (12/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I7e9dcde9c93feb2c9e2bdfe790f121b751d4c8ad
Gerrit-Change-Number: 3359
Gerrit-PatchSet: 1
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Tue, 23 Apr 2019 00:18:37 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][CONF] Introduce ConfigValidator

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3359 )

Change subject: [NO ISSUE][CONF] Introduce ConfigValidator
......................................................................


Patch Set 1:

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I7e9dcde9c93feb2c9e2bdfe790f121b751d4c8ad
Gerrit-Change-Number: 3359
Gerrit-PatchSet: 1
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Tue, 23 Apr 2019 00:27:47 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][CONF] Introduce ConfigValidator

Posted by "Anon. E. Moose (Code Review)" <de...@asterixdb.apache.org>.
Anon. E. Moose (1000171) has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3359 )

Change subject: [NO ISSUE][CONF] Introduce ConfigValidator
......................................................................


Patch Set 1: -Contrib

Analytics Compatibility Compilation Successful
https://cbjenkins.page.link/bwMczX1QKFwRVcCWA : SUCCESS


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I7e9dcde9c93feb2c9e2bdfe790f121b751d4c8ad
Gerrit-Change-Number: 3359
Gerrit-PatchSet: 1
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Tue, 23 Apr 2019 00:41:57 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][CONF] Introduce ConfigValidator

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3359 )

Change subject: [NO ISSUE][CONF] Introduce ConfigValidator
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ssl-compression/430/ (3/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I7e9dcde9c93feb2c9e2bdfe790f121b751d4c8ad
Gerrit-Change-Number: 3359
Gerrit-PatchSet: 1
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Tue, 23 Apr 2019 00:18:32 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][CONF] Introduce ConfigValidator

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3359 )

Change subject: [NO ISSUE][CONF] Introduce ConfigValidator
......................................................................


Patch Set 1:

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I7e9dcde9c93feb2c9e2bdfe790f121b751d4c8ad
Gerrit-Change-Number: 3359
Gerrit-PatchSet: 1
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Tue, 23 Apr 2019 00:18:33 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][CONF] Introduce ConfigValidator

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3359 )

Change subject: [NO ISSUE][CONF] Introduce ConfigValidator
......................................................................


Patch Set 1:

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I7e9dcde9c93feb2c9e2bdfe790f121b751d4c8ad
Gerrit-Change-Number: 3359
Gerrit-PatchSet: 1
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Tue, 23 Apr 2019 00:33:31 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][CONF] Introduce ConfigValidator

Posted by "Till Westmann (Code Review)" <de...@asterixdb.apache.org>.
Till Westmann has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3359 )

Change subject: [NO ISSUE][CONF] Introduce ConfigValidator
......................................................................


Patch Set 1: Code-Review+2


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I7e9dcde9c93feb2c9e2bdfe790f121b751d4c8ad
Gerrit-Change-Number: 3359
Gerrit-PatchSet: 1
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: Till Westmann <ti...@apache.org>
Gerrit-Comment-Date: Tue, 23 Apr 2019 15:00:06 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][CONF] Introduce ConfigValidator

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3359 )

Change subject: [NO ISSUE][CONF] Introduce ConfigValidator
......................................................................


Patch Set 1:

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I7e9dcde9c93feb2c9e2bdfe790f121b751d4c8ad
Gerrit-Change-Number: 3359
Gerrit-PatchSet: 1
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Tue, 23 Apr 2019 00:18:33 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][CONF] Introduce ConfigValidator

Posted by "Anon. E. Moose (Code Review)" <de...@asterixdb.apache.org>.
Anon. E. Moose (1000171) has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3359 )

Change subject: [NO ISSUE][CONF] Introduce ConfigValidator
......................................................................


Patch Set 1: Contrib+1

Analytics Compatibility Tests Successful
https://cbjenkins.page.link/mKwUooa53YdXJWh6A : SUCCESS


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I7e9dcde9c93feb2c9e2bdfe790f121b751d4c8ad
Gerrit-Change-Number: 3359
Gerrit-PatchSet: 1
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Tue, 23 Apr 2019 04:20:54 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][CONF] Introduce ConfigValidator

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3359 )

Change subject: [NO ISSUE][CONF] Introduce ConfigValidator
......................................................................


Patch Set 1: Contrib+1

BAD Compatibility Tests Successful

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I7e9dcde9c93feb2c9e2bdfe790f121b751d4c8ad
Gerrit-Change-Number: 3359
Gerrit-PatchSet: 1
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Tue, 23 Apr 2019 00:38:08 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][CONF] Introduce ConfigValidator

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3359 )

Change subject: [NO ISSUE][CONF] Introduce ConfigValidator
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-format/5626/ (4/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I7e9dcde9c93feb2c9e2bdfe790f121b751d4c8ad
Gerrit-Change-Number: 3359
Gerrit-PatchSet: 1
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Tue, 23 Apr 2019 00:18:33 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][CONF] Introduce ConfigValidator

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3359 )

Change subject: [NO ISSUE][CONF] Introduce ConfigValidator
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-cancellation-test/5661/ (16/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I7e9dcde9c93feb2c9e2bdfe790f121b751d4c8ad
Gerrit-Change-Number: 3359
Gerrit-PatchSet: 1
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Tue, 23 Apr 2019 02:09:06 +0000
Gerrit-HasComments: No