You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@brooklyn.apache.org by neykov <gi...@git.apache.org> on 2015/03/17 19:16:23 UTC

[GitHub] incubator-brooklyn pull request: System Service enricher

GitHub user neykov opened a pull request:

    https://github.com/apache/incubator-brooklyn/pull/556

    System Service enricher

    Installs the entity being enriched as a system service. Extracts the launch script from the start effector, saves it and calls it from the service script.
    
    The functionality is wrapped in an enricher so that existing entities can be updated to work as services, but long term it makes sense to move it to the SoftwareProcess entity with INSTALL_AS_SYSTEM_SERVICE={true,false,auto}  where  auto is "true_unless_localhost" configuration.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/neykov/incubator-brooklyn system-service

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/incubator-brooklyn/pull/556.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #556
    
----
commit 49b056c3b26caf0767352e16e9e1f414ffa55970
Author: Svetoslav Neykov <sv...@cloudsoftcorp.com>
Date:   2015-03-17T18:08:49Z

    System Service enricher
    
    Installs the entity being enriched as a system service. Saves the launch script and uses it to start the entity.

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: System Service enricher

Posted by aledsage <gi...@git.apache.org>.
Github user aledsage commented on the pull request:

    https://github.com/apache/incubator-brooklyn/pull/556#issuecomment-88641216
  
    Excellent, thanks @neykov - merging now.
    
    Failed build (https://builds.apache.org/job/incubator-brooklyn-pull-requests/org.apache.brooklyn$brooklyn-launcher/974/testReport/junit/brooklyn.launcher/BrooklynWebServerTest/verifyHttps/) looks unreleated - `Unable to access URL brooklyn.webconsole.security.keystore.url: /home/jenkins/jenkins-slave/workspace/incubator-brooklyn-pull-requests%402/usage/launcher/target/test-classes/server.ks`; other tests passed fine.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: System Service enricher

Posted by aledsage <gi...@git.apache.org>.
Github user aledsage commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/556#discussion_r26650515
  
    --- Diff: software/base/src/main/java/brooklyn/entity/service/SystemServiceEnricher.java ---
    @@ -0,0 +1,152 @@
    +/*
    + * 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 brooklyn.entity.service;
    +
    +import java.util.Set;
    +
    +import brooklyn.config.ConfigKey;
    +import brooklyn.enricher.basic.AbstractEnricher;
    +import brooklyn.entity.Entity;
    +import brooklyn.entity.basic.Attributes;
    +import brooklyn.entity.basic.BrooklynTaskTags;
    +import brooklyn.entity.basic.BrooklynTaskTags.WrappedStream;
    +import brooklyn.entity.basic.ConfigKeys;
    +import brooklyn.entity.basic.Entities;
    +import brooklyn.entity.basic.EntityLocal;
    +import brooklyn.entity.basic.SoftwareProcess;
    +import brooklyn.entity.effector.EffectorTasks;
    +import brooklyn.location.basic.SshMachineLocation;
    +import brooklyn.management.ExecutionContext;
    +import brooklyn.management.Task;
    +import brooklyn.policy.Enricher;
    +import brooklyn.util.net.Urls;
    +import brooklyn.util.task.BasicExecutionManager;
    +import brooklyn.util.task.DynamicTasks;
    +import brooklyn.util.task.TaskBuilder;
    +import brooklyn.util.task.ssh.SshPutTaskWrapper;
    +import brooklyn.util.task.ssh.SshTasks;
    +import brooklyn.util.task.system.ProcessTaskFactory;
    +import brooklyn.util.task.system.ProcessTaskWrapper;
    +
    +import com.google.common.collect.ImmutableSet;
    +
    +public class SystemServiceEnricher extends AbstractEnricher implements Enricher {
    +    static final Set<String> LAUNCH_EFFECTOR_NAMES = ImmutableSet.of("start", "restart");
    +    public static final ConfigKey<String> LAUNCH_SCRIPT_NAME = ConfigKeys.newStringConfigKey(
    +            "service.script_name", "The name of the launch script to be created in the runtime directory of the entity.", "service-launch.sh");
    +    public static final ConfigKey<String> SERVICE_NAME = ConfigKeys.newStringConfigKey(
    +            "service.name", "The name of the system service. Can use ${entity_name} and ${id} variables to template the value.", "${entity_name}-${id}");
    +
    +    @Override
    +    public void setEntity(EntityLocal entity) {
    +        super.setEntity(entity);
    +        subscribeLaunch();
    +    }
    +
    +    private void subscribeLaunch() {
    +        if(checkCanSubscribe()) {
    +            getSubscriptionTracker().subscribe(entity, Attributes.SERVICE_STATE_ACTUAL, new EntityLaunchListener(this));
    --- End diff --
    
    Can just use `subscribe(...)` rather than `getSubscriptionTracker().subscribe(...)`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: System Service enricher

Posted by aledsage <gi...@git.apache.org>.
Github user aledsage commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/556#discussion_r26650453
  
    --- Diff: software/base/src/main/java/brooklyn/entity/service/InitdServiceInstaller.java ---
    @@ -0,0 +1,133 @@
    +/*
    + * 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 brooklyn.entity.service;
    +
    +import java.io.File;
    +import java.util.Map;
    +
    +import brooklyn.config.ConfigKey;
    +import brooklyn.entity.Entity;
    +import brooklyn.entity.basic.Attributes;
    +import brooklyn.entity.basic.ConfigKeys;
    +import brooklyn.entity.basic.EntityInternal;
    +import brooklyn.entity.basic.SoftwareProcess;
    +import brooklyn.entity.effector.EffectorTasks;
    +import brooklyn.entity.trait.HasShortName;
    +import brooklyn.location.basic.SshMachineLocation;
    +import brooklyn.location.cloud.CloudMachineNamer;
    +import brooklyn.management.Task;
    +import brooklyn.policy.Enricher;
    +import brooklyn.util.ResourceUtils;
    +import brooklyn.util.collections.MutableMap;
    +import brooklyn.util.os.Os;
    +import brooklyn.util.ssh.BashCommands;
    +import brooklyn.util.task.Tasks;
    +import brooklyn.util.task.ssh.SshPutTaskWrapper;
    +import brooklyn.util.task.ssh.SshTasks;
    +import brooklyn.util.task.system.ProcessTaskWrapper;
    +import brooklyn.util.text.TemplateProcessor;
    +
    +
    +public class InitdServiceInstaller implements SystemServiceInstaller {
    +    private static final ConfigKey<String> SERVICE_TEMPLATE = ConfigKeys.newStringConfigKey(
    +            "service.initd.service_template", "URL of the template to be used as the /etc/init.d service", "classpath:///brooklyn/entity/service/service.sh");
    +
    +    private Entity entity;
    +    private Enricher enricher;
    +
    +    public InitdServiceInstaller(Entity entity, Enricher enricher) {
    +        this.entity = entity;
    --- End diff --
    
    Preference for fields being final, and initialising with `this.entity = checkNotNull(entity, "entity")` to avoid later NPEs (that would be harder to debug).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: System Service enricher

Posted by aledsage <gi...@git.apache.org>.
Github user aledsage commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/556#discussion_r26651515
  
    --- Diff: software/base/src/main/java/brooklyn/entity/service/SystemdServiceInstaller.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 brooklyn.entity.service;
    +
    +import brooklyn.management.Task;
    +
    +public class SystemdServiceInstaller implements SystemServiceInstaller {
    +
    +    @Override
    +    public Task<?> getServiceInstallTask() {
    +        //TODO
    --- End diff --
    
    Needs either more comments here, or to delete this unused class (and to add a comment elsewhere about systemd support).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: System Service enricher

Posted by aledsage <gi...@git.apache.org>.
Github user aledsage commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/556#discussion_r26649849
  
    --- Diff: software/base/src/main/java/brooklyn/entity/service/EntityLaunchListener.java ---
    @@ -0,0 +1,109 @@
    +/*
    + * 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 brooklyn.entity.service;
    +
    +import java.util.Set;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +import brooklyn.entity.Entity;
    +import brooklyn.entity.basic.BrooklynTaskTags;
    +import brooklyn.entity.basic.BrooklynTaskTags.EffectorCallTag;
    +import brooklyn.entity.basic.Lifecycle;
    +import brooklyn.event.SensorEvent;
    +import brooklyn.event.SensorEventListener;
    +import brooklyn.management.ExecutionManager;
    +import brooklyn.management.Task;
    +import brooklyn.util.task.Tasks;
    +
    +public class EntityLaunchListener implements Runnable, SensorEventListener<Lifecycle> {
    +    private static final String SSH_LAUNCH_TASK_PREFIX = "ssh: launching";
    +    private static final String LAUNCH_CHECK_SKIP_TAG = "system-service-update";
    +
    +    private AtomicReference<Task<?>> launchTaskRef = new AtomicReference<Task<?>>();
    --- End diff --
    
    Personal preference for making fields "final" when we don't expect them to be changed. (Same for `enricher`).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: System Service enricher

Posted by aledsage <gi...@git.apache.org>.
Github user aledsage commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/556#discussion_r26649874
  
    --- Diff: software/base/src/main/java/brooklyn/entity/service/EntityLaunchListener.java ---
    @@ -0,0 +1,109 @@
    +/*
    + * 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 brooklyn.entity.service;
    +
    +import java.util.Set;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +import brooklyn.entity.Entity;
    +import brooklyn.entity.basic.BrooklynTaskTags;
    +import brooklyn.entity.basic.BrooklynTaskTags.EffectorCallTag;
    +import brooklyn.entity.basic.Lifecycle;
    +import brooklyn.event.SensorEvent;
    +import brooklyn.event.SensorEventListener;
    +import brooklyn.management.ExecutionManager;
    +import brooklyn.management.Task;
    +import brooklyn.util.task.Tasks;
    +
    +public class EntityLaunchListener implements Runnable, SensorEventListener<Lifecycle> {
    +    private static final String SSH_LAUNCH_TASK_PREFIX = "ssh: launching";
    +    private static final String LAUNCH_CHECK_SKIP_TAG = "system-service-update";
    +
    +    private AtomicReference<Task<?>> launchTaskRef = new AtomicReference<Task<?>>();
    +    private SystemServiceEnricher enricher;
    +
    +    public EntityLaunchListener(SystemServiceEnricher enricher) {
    +        this.enricher = enricher;
    --- End diff --
    
    I'd do `this.enricher = checkNotNull(enricher, "enricher")`, rather than risking subsequent NPEs.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: System Service enricher

Posted by aledsage <gi...@git.apache.org>.
Github user aledsage commented on the pull request:

    https://github.com/apache/incubator-brooklyn/pull/556#issuecomment-82883242
  
    Nice - very interesting and clever approach!
    
    Is it possible to get some tests?
    
    I'll reply to the e-mail on dev@brooklyn for more discussion.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: System Service enricher

Posted by neykov <gi...@git.apache.org>.
Github user neykov commented on the pull request:

    https://github.com/apache/incubator-brooklyn/pull/556#issuecomment-85982140
  
    Addressed comments and added a live test for the functionality.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: System Service enricher

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/incubator-brooklyn/pull/556


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: System Service enricher

Posted by neykov <gi...@git.apache.org>.
Github user neykov commented on the pull request:

    https://github.com/apache/incubator-brooklyn/pull/556#issuecomment-82886670
  
    Tests would definetly be nice! I didn't want to delay the functionality, but should add tests as part of the PR.
    
    Any pointers for the best approach on how to test this? For entities & enrichers I guess only integration/live tests make sense. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: System Service enricher

Posted by aledsage <gi...@git.apache.org>.
Github user aledsage commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/556#discussion_r26650620
  
    --- Diff: software/base/src/main/java/brooklyn/entity/service/SystemServiceEnricher.java ---
    @@ -0,0 +1,152 @@
    +/*
    + * 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 brooklyn.entity.service;
    +
    +import java.util.Set;
    +
    +import brooklyn.config.ConfigKey;
    +import brooklyn.enricher.basic.AbstractEnricher;
    +import brooklyn.entity.Entity;
    +import brooklyn.entity.basic.Attributes;
    +import brooklyn.entity.basic.BrooklynTaskTags;
    +import brooklyn.entity.basic.BrooklynTaskTags.WrappedStream;
    +import brooklyn.entity.basic.ConfigKeys;
    +import brooklyn.entity.basic.Entities;
    +import brooklyn.entity.basic.EntityLocal;
    +import brooklyn.entity.basic.SoftwareProcess;
    +import brooklyn.entity.effector.EffectorTasks;
    +import brooklyn.location.basic.SshMachineLocation;
    +import brooklyn.management.ExecutionContext;
    +import brooklyn.management.Task;
    +import brooklyn.policy.Enricher;
    +import brooklyn.util.net.Urls;
    +import brooklyn.util.task.BasicExecutionManager;
    +import brooklyn.util.task.DynamicTasks;
    +import brooklyn.util.task.TaskBuilder;
    +import brooklyn.util.task.ssh.SshPutTaskWrapper;
    +import brooklyn.util.task.ssh.SshTasks;
    +import brooklyn.util.task.system.ProcessTaskFactory;
    +import brooklyn.util.task.system.ProcessTaskWrapper;
    +
    +import com.google.common.collect.ImmutableSet;
    +
    +public class SystemServiceEnricher extends AbstractEnricher implements Enricher {
    +    static final Set<String> LAUNCH_EFFECTOR_NAMES = ImmutableSet.of("start", "restart");
    +    public static final ConfigKey<String> LAUNCH_SCRIPT_NAME = ConfigKeys.newStringConfigKey(
    +            "service.script_name", "The name of the launch script to be created in the runtime directory of the entity.", "service-launch.sh");
    +    public static final ConfigKey<String> SERVICE_NAME = ConfigKeys.newStringConfigKey(
    +            "service.name", "The name of the system service. Can use ${entity_name} and ${id} variables to template the value.", "${entity_name}-${id}");
    +
    +    @Override
    +    public void setEntity(EntityLocal entity) {
    +        super.setEntity(entity);
    +        subscribeLaunch();
    +    }
    +
    +    private void subscribeLaunch() {
    +        if(checkCanSubscribe()) {
    +            getSubscriptionTracker().subscribe(entity, Attributes.SERVICE_STATE_ACTUAL, new EntityLaunchListener(this));
    +        }
    +    }
    +
    +    public void onLaunched(Task<?> task) {
    +        WrappedStream streamStdin = BrooklynTaskTags.stream(task, BrooklynTaskTags.STREAM_STDIN);
    +        if (streamStdin == null) return;
    +
    +        WrappedStream streamEnv = BrooklynTaskTags.stream(task, BrooklynTaskTags.STREAM_ENV);
    +        String stdin = streamStdin.streamContents.get();
    +        String env = streamEnv.streamContents.get();
    +
    +        final SshMachineLocation sshMachine = EffectorTasks.getSshMachine(entity);
    +        final String launchScriptPath = Urls.mergePaths(getRunDir(), getStartScriptName());
    +
    +        Task<Void> installerTask = TaskBuilder.<Void>builder()
    +                .name("install (service)")
    +                .description("Install as a system service")
    +                .body(new Runnable() {
    +                    @Override
    +                    public void run() {
    +                        ProcessTaskFactory<Integer> taskFactory = SshTasks.newSshExecTaskFactory(sshMachine, "[ -e '" + launchScriptPath + "' ]")
    +                                .summary("check installed")
    +                                .allowingNonZeroExitCode();
    +                        boolean isInstalled = DynamicTasks.queue(taskFactory).get() == 0;
    +                        if (!isInstalled) {
    +                            Task<?> serviceInstallTask = SystemServiceInstallerFactory.forMachine(entity, SystemServiceEnricher.this).getServiceInstallTask();
    +                            DynamicTasks.queue(serviceInstallTask);
    +                        }
    +                    }
    +                })
    +                .build();
    +
    +        SshPutTaskWrapper updateLaunchScriptTask = SshTasks.newSshPutTaskFactory(sshMachine, launchScriptPath).contents(getLaunchScript(stdin, env)).newTask();
    +        ProcessTaskWrapper<Integer> makeExecutableTask = SshTasks.newSshExecTaskFactory(sshMachine, "chmod +x " + launchScriptPath)
    +            .requiringExitCodeZero()
    +            .newTask();
    +        Task<Void> udpateTask = TaskBuilder.<Void>builder()
    +            .name("update-launch")
    --- End diff --
    
    Personal preference for 8 char indent (to follow oracle/google coding conventions), as is done on lines 109-115 below. I particularly like consistency within a class.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---