You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@asterixdb.apache.org by "abdullah alamoudi (Code Review)" <do...@asterixdb.incubator.apache.org> on 2016/10/29 03:06:22 UTC

Change in asterixdb[master]: Remove Server Context

abdullah alamoudi has uploaded a new change for review.

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

Change subject: Remove Server Context
......................................................................

Remove Server Context

Change-Id: I4261cf44410b498e84ffcac5e217281a306a7f03
---
M hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/service/IControllerService.java
M hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClusterControllerService.java
M hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/application/CCApplicationContext.java
M hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/web/ApplicationInstallationHandler.java
M hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/CliDeployBinaryWork.java
M hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/CliUnDeployBinaryWork.java
M hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/application/ApplicationContext.java
D hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/context/ServerContext.java
M hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/deployment/DeploymentUtils.java
M hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java
M hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/application/NCApplicationContext.java
M hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/DeployBinaryWork.java
M hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/UnDeployBinaryWork.java
13 files changed, 62 insertions(+), 92 deletions(-)


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

diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/service/IControllerService.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/service/IControllerService.java
index 2e742ac..e2b13a1 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/service/IControllerService.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/service/IControllerService.java
@@ -18,8 +18,26 @@
  */
 package org.apache.hyracks.api.service;
 
-public interface IControllerService {
-    public void start() throws Exception;
+import java.io.File;
 
-    public void stop() throws Exception;
+/**
+ * The base interface for the cluster controller service and the node controller service
+ */
+public interface IControllerService {
+    /**
+     * Start the service
+     * @throws Exception
+     */
+    void start() throws Exception;
+
+    /**
+     * Stop the service
+     * @throws Exception
+     */
+    void stop() throws Exception;
+
+    /**
+     * @return the baseDir for the service process (used for binary deployment)
+     */
+    File getBaseDir();
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClusterControllerService.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClusterControllerService.java
index ac38523..80eb159 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClusterControllerService.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClusterControllerService.java
@@ -68,11 +68,11 @@
 import org.apache.hyracks.control.cc.work.GetJobInfoWork;
 import org.apache.hyracks.control.cc.work.GetJobStatusWork;
 import org.apache.hyracks.control.cc.work.GetNodeControllersInfoWork;
-import org.apache.hyracks.control.cc.work.GetThreadDumpWork;
-import org.apache.hyracks.control.cc.work.GetThreadDumpWork.ThreadDumpRun;
 import org.apache.hyracks.control.cc.work.GetNodeDetailsJSONWork;
 import org.apache.hyracks.control.cc.work.GetResultPartitionLocationsWork;
 import org.apache.hyracks.control.cc.work.GetResultStatusWork;
+import org.apache.hyracks.control.cc.work.GetThreadDumpWork;
+import org.apache.hyracks.control.cc.work.GetThreadDumpWork.ThreadDumpRun;
 import org.apache.hyracks.control.cc.work.JobStartWork;
 import org.apache.hyracks.control.cc.work.JobletCleanupNotificationWork;
 import org.apache.hyracks.control.cc.work.NodeHeartbeatWork;
@@ -94,7 +94,6 @@
 import org.apache.hyracks.control.cc.work.TriggerNCWork;
 import org.apache.hyracks.control.cc.work.UnregisterNodeWork;
 import org.apache.hyracks.control.cc.work.WaitForJobCompletionWork;
-import org.apache.hyracks.control.common.context.ServerContext;
 import org.apache.hyracks.control.common.controllers.CCConfig;
 import org.apache.hyracks.control.common.controllers.IniUtils;
 import org.apache.hyracks.control.common.deployment.DeploymentRun;
@@ -130,7 +129,7 @@
 
     private final Map<InetAddress, Set<String>> ipAddressNodeNameMap;
 
-    private final ServerContext serverCtx;
+    private final File baseDir;
 
     private final WebServer webServer;
 
@@ -174,7 +173,7 @@
         jobLog = new LogFile(jobLogFolder);
         nodeRegistry = new LinkedHashMap<>();
         ipAddressNodeNameMap = new HashMap<>();
-        serverCtx = new ServerContext(ServerContext.ServerType.CLUSTER_CONTROLLER, new File(ccConfig.ccRoot));
+        baseDir = new File(ccConfig.ccRoot);
         IIPCI ccIPCI = new ClusterControllerIPCI();
         clusterIPC = new IPCSystem(new InetSocketAddress(ccConfig.clusterNetPort), ccIPCI,
                 new CCNCFunctions.SerializerDeserializer());
@@ -249,7 +248,7 @@
     }
 
     private void startApplication() throws Exception {
-        appCtx = new CCApplicationContext(this, serverCtx, ccContext, ccConfig.getAppConfig());
+        appCtx = new CCApplicationContext(this, ccContext, ccConfig.getAppConfig());
         appCtx.addJobLifecycleListener(datasetDirectoryService);
         executor = Executors.newCachedThreadPool(appCtx.getThreadFactory());
         String className = ccConfig.appCCMainClass;
@@ -337,10 +336,6 @@
         if (aep != null) {
             aep.stop();
         }
-    }
-
-    public ServerContext getServerContext() {
-        return serverCtx;
     }
 
     public ICCContext getCCContext() {
@@ -514,7 +509,7 @@
                     HyracksClientInterfaceFunctions.WaitForCompletionFunction wfcf =
                             (HyracksClientInterfaceFunctions.WaitForCompletionFunction) fn;
                     workQueue.schedule(new WaitForJobCompletionWork(ClusterControllerService.this, wfcf.getJobId(),
-                            new IPCResponder<Object>(handle, mid)));
+                            new IPCResponder<>(handle, mid)));
                     return;
                 }
 
@@ -782,4 +777,9 @@
     public ThreadDumpRun removeThreadDumpRun(String requestKey) {
         return threadDumpRunMap.remove(requestKey);
     }
+
+    @Override
+    public File getBaseDir() {
+        return baseDir;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/application/CCApplicationContext.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/application/CCApplicationContext.java
index dd6f83b..e798b3f 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/application/CCApplicationContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/application/CCApplicationContext.java
@@ -37,7 +37,6 @@
 import org.apache.hyracks.api.service.IControllerService;
 import org.apache.hyracks.control.cc.ClusterControllerService;
 import org.apache.hyracks.control.common.application.ApplicationContext;
-import org.apache.hyracks.control.common.context.ServerContext;
 import org.apache.hyracks.control.common.utils.HyracksThreadFactory;
 import org.apache.hyracks.control.common.work.IResultCallback;
 
@@ -54,9 +53,9 @@
     private List<IClusterLifecycleListener> clusterLifecycleListeners;
     private final ClusterControllerService ccs;
 
-    public CCApplicationContext(ClusterControllerService ccs, ServerContext serverCtx, ICCContext ccContext,
+    public CCApplicationContext(ClusterControllerService ccs, ICCContext ccContext,
             IApplicationConfig appConfig) throws IOException {
-        super(serverCtx, appConfig, new HyracksThreadFactory("ClusterController"));
+        super(ccs.getBaseDir(), appConfig, new HyracksThreadFactory("ClusterController"));
         this.ccContext = ccContext;
         this.ccs = ccs;
         initPendingNodeIds = new HashSet<>();
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/web/ApplicationInstallationHandler.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/web/ApplicationInstallationHandler.java
index 75b7473..4dc39dc 100755
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/web/ApplicationInstallationHandler.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/web/ApplicationInstallationHandler.java
@@ -60,7 +60,7 @@
             }
             final String[] params = parts[0].split("&");
             String deployIdString = params[0];
-            String rootDir = ccs.getServerContext().getBaseDir().toString();
+            String rootDir = ccs.getBaseDir().toString();
             final String deploymentDir = rootDir.endsWith(File.separator) ? rootDir + "applications/" + deployIdString
                     : rootDir + File.separator + "/applications/" + File.separator + deployIdString;
             switch (HttpMethod.valueOf(request.getMethod())) {
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/CliDeployBinaryWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/CliDeployBinaryWork.java
index 741c641..abc496b 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/CliDeployBinaryWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/CliDeployBinaryWork.java
@@ -67,14 +67,14 @@
              * Deploy for the cluster controller
              */
             DeploymentUtils.deploy(deploymentId, binaryURLs, ccs.getApplicationContext()
-                    .getJobSerializerDeserializerContainer(), ccs.getServerContext(), false);
+                    .getJobSerializerDeserializerContainer(), ccs.getBaseDir(), false);
 
             /**
              * Deploy for the node controllers
              */
             Map<String, NodeControllerState> nodeControllerStateMap = ccs.getNodeMap();
 
-            Set<String> nodeIds = new TreeSet<String>();
+            Set<String> nodeIds = new TreeSet<>();
             for (String nc : nodeControllerStateMap.keySet()) {
                 nodeIds.add(nc);
             }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/CliUnDeployBinaryWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/CliUnDeployBinaryWork.java
index 5f97ce2..9944583 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/CliUnDeployBinaryWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/CliUnDeployBinaryWork.java
@@ -63,14 +63,14 @@
              * Deploy for the cluster controller
              */
             DeploymentUtils.undeploy(deploymentId, ccs.getApplicationContext().getJobSerializerDeserializerContainer(),
-                    ccs.getServerContext());
+                    ccs.getBaseDir());
 
             /**
              * Deploy for the node controllers
              */
             Map<String, NodeControllerState> nodeControllerStateMap = ccs.getNodeMap();
 
-            Set<String> nodeIds = new TreeSet<String>();
+            Set<String> nodeIds = new TreeSet<>();
             for (String nc : nodeControllerStateMap.keySet()) {
                 nodeIds.add(nc);
             }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/application/ApplicationContext.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/application/ApplicationContext.java
index 06bcda3..2357345 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/application/ApplicationContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/application/ApplicationContext.java
@@ -18,6 +18,7 @@
  */
 package org.apache.hyracks.control.common.application;
 
+import java.io.File;
 import java.io.Serializable;
 import java.util.concurrent.ThreadFactory;
 
@@ -26,18 +27,17 @@
 import org.apache.hyracks.api.job.IJobSerializerDeserializerContainer;
 import org.apache.hyracks.api.job.JobSerializerDeserializerContainer;
 import org.apache.hyracks.api.messages.IMessageBroker;
-import org.apache.hyracks.control.common.context.ServerContext;
 
 public abstract class ApplicationContext implements IApplicationContext {
-    protected final ServerContext serverCtx;
+    protected final File baseDir;
     protected final IApplicationConfig appConfig;
     protected ThreadFactory threadFactory;
     protected Serializable distributedState;
     protected IMessageBroker messageBroker;
     protected IJobSerializerDeserializerContainer jobSerDeContainer = new JobSerializerDeserializerContainer();
 
-    public ApplicationContext(ServerContext serverCtx, IApplicationConfig appConfig, ThreadFactory threadFactory) {
-        this.serverCtx = serverCtx;
+    public ApplicationContext(File baseDir, IApplicationConfig appConfig, ThreadFactory threadFactory) {
+        this.baseDir = baseDir;
         this.appConfig = appConfig;
         this.threadFactory = threadFactory;
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/context/ServerContext.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/context/ServerContext.java
deleted file mode 100644
index d4dc054..0000000
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/context/ServerContext.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.hyracks.control.common.context;
-
-import java.io.File;
-
-public class ServerContext {
-    public enum ServerType {
-        CLUSTER_CONTROLLER,
-        NODE_CONTROLLER,
-    }
-
-    private final ServerType type;
-    private final File baseDir;
-
-    public ServerContext(ServerType type, File baseDir) throws Exception {
-        this.type = type;
-        this.baseDir = baseDir;
-    }
-
-    public ServerType getServerType() {
-        return type;
-    }
-
-    public File getBaseDir() {
-        return baseDir;
-    }
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/deployment/DeploymentUtils.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/deployment/DeploymentUtils.java
index e033ff9..8cc8238 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/deployment/DeploymentUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/deployment/DeploymentUtils.java
@@ -40,7 +40,6 @@
 import org.apache.hyracks.api.job.IJobSerializerDeserializer;
 import org.apache.hyracks.api.job.IJobSerializerDeserializerContainer;
 import org.apache.hyracks.api.util.JavaSerializationUtils;
-import org.apache.hyracks.control.common.context.ServerContext;
 
 /**
  * A utility class which is in charge of the actual work of deployments.
@@ -61,9 +60,9 @@
      * @throws HyracksException
      */
     public static void undeploy(DeploymentId deploymentId, IJobSerializerDeserializerContainer container,
-            ServerContext ctx) throws HyracksException {
+            File baseDir) throws HyracksException {
         container.removeJobSerializerDeserializer(deploymentId);
-        String rootDir = ctx.getBaseDir().toString();
+        String rootDir = baseDir.toString();
         String deploymentDir = rootDir.endsWith(File.separator) ? rootDir + DEPLOYMENT + File.separator + deploymentId
                 : rootDir + File.separator + DEPLOYMENT + File.separator + deploymentId;
         try {
@@ -92,13 +91,13 @@
      * @throws HyracksException
      */
     public static void deploy(DeploymentId deploymentId, List<URL> urls, IJobSerializerDeserializerContainer container,
-            ServerContext ctx, boolean isNC) throws HyracksException {
+            File baseDir, boolean isNC) throws HyracksException {
         IJobSerializerDeserializer jobSerDe = container.getJobSerializerDeserializer(deploymentId);
         if (jobSerDe == null) {
             jobSerDe = new ClassLoaderJobSerializerDeserializer();
             container.addJobSerializerDeserializer(deploymentId, jobSerDe);
         }
-        String rootDir = ctx.getBaseDir().toString();
+        String rootDir = baseDir.toString();
         String deploymentDir = rootDir.endsWith(File.separator) ? rootDir + DEPLOYMENT + File.separator + deploymentId
                 : rootDir + File.separator + DEPLOYMENT + File.separator + deploymentId;
         jobSerDe.addClassPathURLs(downloadURLs(urls, deploymentDir, isNC));
@@ -185,7 +184,7 @@
         while (tried < retryCount) {
             try {
                 tried++;
-                List<URL> downloadedFileURLs = new ArrayList<URL>();
+                List<URL> downloadedFileURLs = new ArrayList<>();
                 File dir = new File(deploymentDir);
                 if (!dir.exists()) {
                     FileUtils.forceMkdir(dir);
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java
index ed46b53..5692216 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java
@@ -55,7 +55,6 @@
 import org.apache.hyracks.api.lifecycle.LifeCycleComponentManager;
 import org.apache.hyracks.api.service.IControllerService;
 import org.apache.hyracks.control.common.base.IClusterController;
-import org.apache.hyracks.control.common.context.ServerContext;
 import org.apache.hyracks.control.common.controllers.NCConfig;
 import org.apache.hyracks.control.common.controllers.NodeParameters;
 import org.apache.hyracks.control.common.controllers.NodeRegistration;
@@ -79,16 +78,16 @@
 import org.apache.hyracks.control.nc.partitions.PartitionManager;
 import org.apache.hyracks.control.nc.resources.memory.MemoryManager;
 import org.apache.hyracks.control.nc.runtime.RootHyracksContext;
+import org.apache.hyracks.control.nc.task.ShutdownTask;
+import org.apache.hyracks.control.nc.task.ThreadDumpTask;
 import org.apache.hyracks.control.nc.work.AbortTasksWork;
 import org.apache.hyracks.control.nc.work.ApplicationMessageWork;
 import org.apache.hyracks.control.nc.work.BuildJobProfilesWork;
 import org.apache.hyracks.control.nc.work.CleanupJobletWork;
 import org.apache.hyracks.control.nc.work.DeployBinaryWork;
 import org.apache.hyracks.control.nc.work.ReportPartitionAvailabilityWork;
-import org.apache.hyracks.control.nc.task.ShutdownTask;
 import org.apache.hyracks.control.nc.work.StartTasksWork;
 import org.apache.hyracks.control.nc.work.StateDumpWork;
-import org.apache.hyracks.control.nc.task.ThreadDumpTask;
 import org.apache.hyracks.control.nc.work.UnDeployBinaryWork;
 import org.apache.hyracks.ipc.api.IIPCHandle;
 import org.apache.hyracks.ipc.api.IIPCI;
@@ -136,7 +135,7 @@
 
     private HeartbeatTask heartbeatTask;
 
-    private final ServerContext serverCtx;
+    private final File baseDir;
 
     private NCApplicationContext appCtx;
 
@@ -184,8 +183,7 @@
         queue = new WorkQueue(id, Thread.NORM_PRIORITY); // Reserves MAX_PRIORITY of the heartbeat thread.
         jobletMap = new Hashtable<>();
         timer = new Timer(true);
-        serverCtx = new ServerContext(ServerContext.ServerType.NODE_CONTROLLER,
-                new File(new File(NodeControllerService.class.getName()), id));
+        baseDir = new File(new File(NodeControllerService.class.getName()), id);
         memoryMXBean = ManagementFactory.getMemoryMXBean();
         gcMXBeans = ManagementFactory.getGarbageCollectorMXBeans();
         threadMXBean = ManagementFactory.getThreadMXBean();
@@ -332,7 +330,7 @@
     }
 
     private void startApplication() throws Exception {
-        appCtx = new NCApplicationContext(this, serverCtx, ctx, id, memoryManager, lccm, ncConfig.getAppConfig());
+        appCtx = new NCApplicationContext(this, ctx, id, memoryManager, lccm, ncConfig.getAppConfig());
         String className = ncConfig.appNCMainClass;
         if (className != null) {
             Class<?> c = Class.forName(className);
@@ -375,10 +373,6 @@
 
     public String getId() {
         return id;
-    }
-
-    public ServerContext getServerContext() {
-        return serverCtx;
     }
 
     public Map<JobId, Joblet> getJobletMap() {
@@ -621,4 +615,9 @@
             }
         }
     }
+
+    @Override
+    public File getBaseDir() {
+        return baseDir;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/application/NCApplicationContext.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/application/NCApplicationContext.java
index 44549b4..da0ff0e 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/application/NCApplicationContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/application/NCApplicationContext.java
@@ -31,7 +31,6 @@
 import org.apache.hyracks.api.resources.memory.IMemoryManager;
 import org.apache.hyracks.api.service.IControllerService;
 import org.apache.hyracks.control.common.application.ApplicationContext;
-import org.apache.hyracks.control.common.context.ServerContext;
 import org.apache.hyracks.control.common.utils.HyracksThreadFactory;
 import org.apache.hyracks.control.nc.NodeControllerService;
 import org.apache.hyracks.control.nc.resources.memory.MemoryManager;
@@ -46,10 +45,10 @@
     private final NodeControllerService ncs;
     private IChannelInterfaceFactory messagingChannelInterfaceFactory;
 
-    public NCApplicationContext(NodeControllerService ncs, ServerContext serverCtx, IHyracksRootContext rootCtx,
+    public NCApplicationContext(NodeControllerService ncs, IHyracksRootContext rootCtx,
             String nodeId, MemoryManager memoryManager, ILifeCycleComponentManager lifeCyclecomponentManager,
             IApplicationConfig appConfig) throws IOException {
-        super(serverCtx, appConfig, new HyracksThreadFactory(nodeId));
+        super(ncs.getBaseDir(), appConfig, new HyracksThreadFactory(nodeId));
         this.lccm = lifeCyclecomponentManager;
         this.nodeId = nodeId;
         this.rootCtx = rootCtx;
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/DeployBinaryWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/DeployBinaryWork.java
index 264a131..c86f67d 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/DeployBinaryWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/DeployBinaryWork.java
@@ -52,7 +52,7 @@
         DeploymentStatus status;
         try {
             DeploymentUtils.deploy(deploymentId, binaryURLs, ncs.getApplicationContext()
-                    .getJobSerializerDeserializerContainer(), ncs.getServerContext(), true);
+                    .getJobSerializerDeserializerContainer(), ncs.getBaseDir(), true);
             status = DeploymentStatus.SUCCEED;
         } catch (Exception e) {
             status = DeploymentStatus.FAIL;
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/UnDeployBinaryWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/UnDeployBinaryWork.java
index f564ff4..f05135d 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/UnDeployBinaryWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/UnDeployBinaryWork.java
@@ -46,7 +46,7 @@
         DeploymentStatus status;
         try {
             DeploymentUtils.undeploy(deploymentId, ncs.getApplicationContext().getJobSerializerDeserializerContainer(),
-                    ncs.getServerContext());
+                    ncs.getBaseDir());
             status = DeploymentStatus.SUCCEED;
         } catch (Exception e) {
             status = DeploymentStatus.FAIL;

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

Gerrit-MessageType: newchange
Gerrit-Change-Id: I4261cf44410b498e84ffcac5e217281a306a7f03
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>

Change in asterixdb[master]: Remove Server Context

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

Change subject: Remove Server Context
......................................................................


Patch Set 1:

>> too many contexts = confusion.
>> not only for existing developers but also for new ones. 

I don't think they really cause troubles. The stuffs really causing troubles are those that lacks facets and extensibility,  for example, AqlMetdataProvider and various storage code.

>> and those are wrappers with unused methods.

Pls don't think they are "wrappers".   Tomorrow you can add x, y, z into a context class and doesn't need to change interface contracts, but you cannot x, y, z into the File class. 


>> We can't add contexts for potential future use. 
>> Once we have the need for additional contexts, we can add them.

Once the contexts are removed, ClusterControllerService etc will start becoming a kitchen sink like AqlMetadataProvider.

>> For the SonarQube argument, show me a place 
>> where having the ICCContext or the ServerContext 
>> will get rid of that.

I was talking about design principles like facets and separating concerns. For places lacking those, we have many warnings, e.g., AqlMetadataProvider and various storage classes.

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I4261cf44410b498e84ffcac5e217281a306a7f03
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: Remove Server Context

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

Change subject: Remove Server Context
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/3188/

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I4261cf44410b498e84ffcac5e217281a306a7f03
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: Remove Server Context

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

Change subject: Remove Server Context
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/3186/

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I4261cf44410b498e84ffcac5e217281a306a7f03
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: Remove Server Context

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

Change subject: Remove Server Context
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/3185/

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I4261cf44410b498e84ffcac5e217281a306a7f03
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: Remove Server Context

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

Change subject: Remove Server Context
......................................................................


Patch Set 1:

Also, forgot the IHyracksRootContext which was a wrapper for IOManager :S

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I4261cf44410b498e84ffcac5e217281a306a7f03
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: Remove Server Context

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

Change subject: Remove Server Context
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I4261cf44410b498e84ffcac5e217281a306a7f03
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: Remove Server Context

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

Change subject: Remove Server Context
......................................................................


Patch Set 1:

(3 comments)

https://asterix-gerrit.ics.uci.edu/#/c/1320/1/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/service/IControllerService.java
File hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/service/IControllerService.java:

Line 42:     File getBaseDir();
Since there was no ServerContext here, it seems that we also shouldn't need the base dir on this interface ...


https://asterix-gerrit.ics.uci.edu/#/c/1320/1/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClusterControllerService.java
File hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClusterControllerService.java:

Line 782:     public File getBaseDir() {
Put this in the place where getServerContext was?


https://asterix-gerrit.ics.uci.edu/#/c/1320/1/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java
File hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java:

Line 620:     public File getBaseDir() {
Put this in the place where getServerContext was?


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I4261cf44410b498e84ffcac5e217281a306a7f03
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: Yes

Change in asterixdb[master]: Remove Server Context

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

Change subject: Remove Server Context
......................................................................


Patch Set 4:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I4261cf44410b498e84ffcac5e217281a306a7f03
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: Remove Server Context

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

Change subject: Remove Server Context
......................................................................


Patch Set 1: Integration-Tests+1

Integration Tests Successful

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I4261cf44410b498e84ffcac5e217281a306a7f03
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: Remove Server Context

Posted by "abdullah alamoudi (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/1320

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

Change subject: Remove Server Context
......................................................................

Remove Server Context

Change-Id: I4261cf44410b498e84ffcac5e217281a306a7f03
---
M hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/service/IControllerService.java
M hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClusterControllerService.java
M hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/application/CCApplicationContext.java
M hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/web/ApplicationInstallationHandler.java
M hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/CliDeployBinaryWork.java
M hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/CliUnDeployBinaryWork.java
M hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/application/ApplicationContext.java
D hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/context/ServerContext.java
M hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/NCConfig.java
M hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/deployment/DeploymentUtils.java
M hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java
M hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/application/NCApplicationContext.java
M hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/DeployBinaryWork.java
M hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/UnDeployBinaryWork.java
14 files changed, 61 insertions(+), 88 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/20/1320/3
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1320
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I4261cf44410b498e84ffcac5e217281a306a7f03
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>

Change in asterixdb[master]: Remove Server Context

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

Change subject: Remove Server Context
......................................................................


Patch Set 1:

NoddControllerService and ClusterControllerService are semi kitchen sinks right now and I am stopping that from happening.

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I4261cf44410b498e84ffcac5e217281a306a7f03
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: Remove Server Context

Posted by "abdullah alamoudi (Code Review)" <do...@asterixdb.incubator.apache.org>.
abdullah alamoudi has abandoned this change.

Change subject: Remove Server Context
......................................................................


Abandoned

Seems like there is a direction to contextualize everything. Regardless of confusion introduced by creating more contexts with no apparent benefits (such as this ServerContext which only serve to hide the base directory).

Lesson learned: create more contexts

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

Gerrit-MessageType: abandon
Gerrit-Change-Id: I4261cf44410b498e84ffcac5e217281a306a7f03
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>

Change in asterixdb[master]: Remove Server Context

Posted by "abdullah alamoudi (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/1320

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

Change subject: Remove Server Context
......................................................................

Remove Server Context

Change-Id: I4261cf44410b498e84ffcac5e217281a306a7f03
---
M hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/service/IControllerService.java
M hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClusterControllerService.java
M hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/application/CCApplicationContext.java
M hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/web/ApplicationInstallationHandler.java
M hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/CliDeployBinaryWork.java
M hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/CliUnDeployBinaryWork.java
M hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/application/ApplicationContext.java
D hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/context/ServerContext.java
M hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/NCConfig.java
M hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/deployment/DeploymentUtils.java
M hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java
M hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/application/NCApplicationContext.java
M hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/DeployBinaryWork.java
M hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/UnDeployBinaryWork.java
14 files changed, 65 insertions(+), 92 deletions(-)


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

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I4261cf44410b498e84ffcac5e217281a306a7f03
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>

Change in asterixdb[master]: Remove Server Context

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

Change subject: Remove Server Context
......................................................................


Patch Set 1:

(1 comment)

I like various context classes.

At the high level, it provides:

(1)  a facet for separating concerns and modularizing code.  For example, AqlMetadataProvider is an example that lacks those, which deserves more refactoring.

(2)  extensibility.  If you want to pass more things to a calling client, you just need to pass the context, instead of every individual parameters.  If tomorrow you want to add things, add things to the context and no interface (contract) will need to change.  That's also why SonarQube warns us if we pass more parameters to a method.

https://asterix-gerrit.ics.uci.edu/#/c/1320/1/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/application/ApplicationContext.java
File hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/application/ApplicationContext.java:

Line 39:     public ApplicationContext(File baseDir, IApplicationConfig appConfig, ThreadFactory threadFactory) {
Passing a serverCtx is easier to extend.

For example, if tomorrow you need to additional things into the class, you don't need to add additional parameters.  So as other interfaces if any.

Again, maybe it's under utilized, but the design was well conceived.


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I4261cf44410b498e84ffcac5e217281a306a7f03
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: Yes

Change in asterixdb[master]: Remove Server Context

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

Change subject: Remove Server Context
......................................................................


Patch Set 1:

(1 comment)

https://asterix-gerrit.ics.uci.edu/#/c/1320/1/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/service/IControllerService.java
File hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/service/IControllerService.java:

Line 42:     File getBaseDir();
> Since there was no ServerContext here, it seems that we also shouldn't need
there was no server context but all the sub interfaces had this. so when there is a method which returns an IControllerService, one still needs to cast before calling getBaseDir().


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I4261cf44410b498e84ffcac5e217281a306a7f03
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: Yes

Change in asterixdb[master]: Remove Server Context

Posted by "abdullah alamoudi (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/1320

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

Change subject: Remove Server Context
......................................................................

Remove Server Context

Change-Id: I4261cf44410b498e84ffcac5e217281a306a7f03
---
M hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/service/IControllerService.java
M hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClusterControllerService.java
M hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/application/CCApplicationContext.java
M hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/web/ApplicationInstallationHandler.java
M hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/CliDeployBinaryWork.java
M hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/CliUnDeployBinaryWork.java
M hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/application/ApplicationContext.java
D hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/context/ServerContext.java
M hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/NCConfig.java
M hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/deployment/DeploymentUtils.java
M hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java
M hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/application/NCApplicationContext.java
M hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/DeployBinaryWork.java
M hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/UnDeployBinaryWork.java
14 files changed, 61 insertions(+), 88 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/20/1320/4
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1320
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I4261cf44410b498e84ffcac5e217281a306a7f03
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>

Change in asterixdb[master]: Remove Server Context

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

Change subject: Remove Server Context
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/3176/

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I4261cf44410b498e84ffcac5e217281a306a7f03
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: Remove Server Context

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

Change subject: Remove Server Context
......................................................................


Patch Set 1:

Again, moving the inner class ClusterControllerContext out would be nice, if you can do so.  But inlining baseDir into ClusterControllerService doesn't seem good to me.

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I4261cf44410b498e84ffcac5e217281a306a7f03
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: Remove Server Context

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

Change subject: Remove Server Context
......................................................................


Patch Set 1:

too many contexts = confusion. not only for existing developers but also for new ones.
and those are wrappers with unused methods.

right now, on CC, we have:
1. ServerContext (wrapper for baseDir)
2. ICCContext (wrapper for unused cluster topology?)
3. ICCApplicationContext (the only used one)

We can't add contexts for potential future use. Once we have the need for additional contexts, we can add them.

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I4261cf44410b498e84ffcac5e217281a306a7f03
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: Remove Server Context

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

Change subject: Remove Server Context
......................................................................


Patch Set 4: Integration-Tests+1

Integration Tests Successful

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I4261cf44410b498e84ffcac5e217281a306a7f03
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: Remove Server Context

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

Change subject: Remove Server Context
......................................................................


Patch Set 1:

For the SonarQube argument, show me a place where having the
ICCContext or the ServerContext will get rid of that.

We have that issue in so many other places

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I4261cf44410b498e84ffcac5e217281a306a7f03
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: Remove Server Context

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

Change subject: Remove Server Context
......................................................................


Patch Set 1:

(1 comment)

https://asterix-gerrit.ics.uci.edu/#/c/1320/1/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java
File hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java:

Line 186:         baseDir = new File(new File(NodeControllerService.class.getName()), id);
> what is this used for?  seems like this should be configurable, and that th
it is used for binary deployment. if we want to make it configurable. let's do that in a different change,

let's keep changes small always.


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I4261cf44410b498e84ffcac5e217281a306a7f03
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: Yes

Change in asterixdb[master]: Remove Server Context

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

Change subject: Remove Server Context
......................................................................


Patch Set 1:

(3 comments)

https://asterix-gerrit.ics.uci.edu/#/c/1320/1/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClusterControllerService.java
File hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClusterControllerService.java:

Line 782:     public File getBaseDir() {
> Put this in the place where getServerContext was?
Done


https://asterix-gerrit.ics.uci.edu/#/c/1320/1/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java
File hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java:

Line 186:         baseDir = new File(new File(NodeControllerService.class.getName()), id);
> It seems not to add much to the change to add something equivalent to:
Done


Line 620:     public File getBaseDir() {
> Put this in the place where getServerContext was?
Done


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I4261cf44410b498e84ffcac5e217281a306a7f03
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: Yes

Change in asterixdb[master]: Remove Server Context

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

Change subject: Remove Server Context
......................................................................


Patch Set 1:

(1 comment)

https://asterix-gerrit.ics.uci.edu/#/c/1320/1/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java
File hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java:

Line 186:         baseDir = new File(new File(NodeControllerService.class.getName()), id);
> it is used for binary deployment. if we want to make it configurable. let's
It seems not to add much to the change to add something equivalent to:

    @Option(name = "-cc-root",
            usage = "Sets the root folder used for file operations. (default: ClusterControllerService)")
    public String ccRoot = "ClusterControllerService";

to NCConfig and reference here?


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I4261cf44410b498e84ffcac5e217281a306a7f03
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: Yes

Change in asterixdb[master]: Remove Server Context

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

Change subject: Remove Server Context
......................................................................


Patch Set 1:

(1 comment)

https://asterix-gerrit.ics.uci.edu/#/c/1320/1/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java
File hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java:

Line 186:         baseDir = new File(new File(NodeControllerService.class.getName()), id);
what is this used for?  seems like this should be configurable, and that this is a pretty obnoxious default.


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I4261cf44410b498e84ffcac5e217281a306a7f03
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-HasComments: Yes