You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by GitBox <gi...@apache.org> on 2022/05/24 14:27:02 UTC

[GitHub] [nifi] mattyb149 opened a new pull request, #6075: NIFI-9666 Implement base C2 client functionality

mattyb149 opened a new pull request, #6075:
URL: https://github.com/apache/nifi/pull/6075

   
   # Summary
   
   [NIFI-9666](https://issues.apache.org/jira/browse/NIFI-9666) This PR offers initial functionality for a C2 client to interact with a server using the [C2 protocol](https://cwiki.apache.org/confluence/display/MINIFI/C2+Design). The multiple commits represent multiple authors for which we'd like to retain attribution, perhaps we can squash and use the co-author annotations if desired.
   
   # Tracking
   
   Please complete the following tracking steps prior to pull request creation.
   
   ### Issue Tracking
   
   - [x] [Apache NiFi Jira](https://issues.apache.org/jira/browse/NIFI) issue created
   
   ### Pull Request Tracking
   
   - [x] Pull Request title starts with Apache NiFi Jira issue number, such as `NIFI-00000`
   - [x] Pull Request commit message starts with Apache NiFi Jira issue number, as such `NIFI-00000`
   
   ### Pull Request Formatting
   
   - [x] Pull Request based on current revision of the `main` branch
   - [ ] Pull Request refers to a feature branch with one commit containing changes
   
   # Verification
   
   Please indicate the verification steps performed prior to pull request creation.
   
   ### Build
   
   - [ ] Build completed using `mvn clean install -P contrib-check`
     - [x] JDK 8
     - [x] JDK 11
     - [ ] JDK 17
   
   ### Licensing
   
   - [ ] New dependencies are compatible with the [Apache License 2.0](https://apache.org/licenses/LICENSE-2.0) according to the [License Policy](https://www.apache.org/legal/resolved.html)
   - [ ] New dependencies are documented in applicable `LICENSE` and `NOTICE` files
   
   ### Documentation
   
   - [ ] Documentation formatting appears as expected in rendered files
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] ferencerdei commented on a diff in pull request #6075: NIFI-9666 Implement base C2 client functionality

Posted by GitBox <gi...@apache.org>.
ferencerdei commented on code in PR #6075:
URL: https://github.com/apache/nifi/pull/6075#discussion_r893253234


##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/command/EnvRunner.java:
##########
@@ -0,0 +1,107 @@
+/*
+ * 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.nifi.minifi.bootstrap.command;
+
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CMD_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.DEFAULT_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.Status.ERROR;
+import static org.apache.nifi.minifi.bootstrap.Status.MINIFI_NOT_RUNNING;
+import static org.apache.nifi.minifi.bootstrap.Status.OK;
+
+import java.lang.reflect.Method;
+import java.util.Map.Entry;
+import java.util.Properties;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.apache.nifi.minifi.bootstrap.MiNiFiStatus;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiStatusProvider;
+
+public class EnvRunner implements CommandRunner {
+    private final MiNiFiParameters miNiFiParameters;
+    private final MiNiFiStatusProvider miNiFiStatusProvider;
+
+    public EnvRunner(MiNiFiParameters miNiFiParameters, MiNiFiStatusProvider miNiFiStatusProvider) {
+        this.miNiFiParameters = miNiFiParameters;
+        this.miNiFiStatusProvider = miNiFiStatusProvider;
+    }
+
+    /**
+     * Returns information about the MiNiFi's virtual machine.
+     * @param args the input arguments
+     * @return status code
+     */
+    @Override
+    public int runCommand(String[] args) {
+        return env();
+    }
+
+    private int env() {
+        MiNiFiStatus status = miNiFiStatusProvider.getStatus(miNiFiParameters.getMiNiFiPort(), miNiFiParameters.getMinifiPid());
+        if (status.getPid() == null) {
+            CMD_LOGGER.error("Apache MiNiFi is not running");
+            return MINIFI_NOT_RUNNING.getStatusCode();
+        }
+
+        Class<?> virtualMachineClass;
+        try {
+            virtualMachineClass = Class.forName("com.sun.tools.attach.VirtualMachine");

Review Comment:
   Updated it, so now the MiNiFi process sends back the properties through the socket (no reflection used)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] kevdoran commented on a diff in pull request #6075: NIFI-9666 Implement base C2 client functionality

Posted by GitBox <gi...@apache.org>.
kevdoran commented on code in PR #6075:
URL: https://github.com/apache/nifi/pull/6075#discussion_r899229069


##########
minifi/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/bootstrap.conf:
##########
@@ -129,3 +128,36 @@ java.arg.7=-Djava.security.egd=file:/dev/urandom
 
 #Set headless mode by default
 java.arg.14=-Djava.awt.headless=true
+
+# MiNiFi Command & Control Configuration
+# C2 Properties
+# Enabling C2 Uncomment each of the following options
+#c2.enable=true
+## define protocol parameters
+#c2.rest.url=
+#c2.rest.url.ack=
+## c2 timeouts
+#c2.rest.connectionTimeout=5 sec
+#c2.rest.readTimeout=5 sec
+#c2.rest.callTimeout=10 sec
+## heartbeat in milliseconds
+#c2.agent.heartbeat.period=5000
+## define parameters about your agent
+#c2.agent.class=
+#c2.config.directory=./conf
+#c2.runtime.manifest.identifier=minifi
+#c2.runtime.type=minifi-java
+# Optional.  Defaults to a hardware based unique identifier
+#c2.agent.identifier=
+## Define TLS security properties for C2 communications
+#c2.security.truststore.location=
+#c2.security.truststore.password=
+#c2.security.truststore.type=JKS
+#c2.security.keystore.location=
+#c2.security.keystore.password=
+#c2.security.keystore.type=JKS
+#c2.security.need.client.auth=true

Review Comment:
   Agreed it looks like this could be removed. This is exclusively a client-side TLS Context.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory closed pull request #6075: NIFI-9666 Implement base C2 client functionality

Posted by GitBox <gi...@apache.org>.
exceptionfactory closed pull request #6075: NIFI-9666 Implement base C2 client functionality
URL: https://github.com/apache/nifi/pull/6075


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] bejancsaba commented on a diff in pull request #6075: NIFI-9666 Implement base C2 client functionality

Posted by GitBox <gi...@apache.org>.
bejancsaba commented on code in PR #6075:
URL: https://github.com/apache/nifi/pull/6075#discussion_r891075210


##########
c2/c2-client-bundle/c2-client-http/src/main/java/org/apache/nifi/c2/client/http/C2HttpClient.java:
##########
@@ -0,0 +1,250 @@
+/*
+ * 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.nifi.c2.client.http;
+
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.security.KeyStore;
+import java.security.NoSuchAlgorithmException;
+import java.util.Optional;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLSocketFactory;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+import okhttp3.MediaType;
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.RequestBody;
+import okhttp3.Response;
+import okhttp3.ResponseBody;
+import okhttp3.logging.HttpLoggingInterceptor;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.c2.client.C2ClientConfig;
+import org.apache.nifi.c2.client.api.C2Client;
+import org.apache.nifi.c2.client.api.C2Serializer;
+import org.apache.nifi.c2.protocol.api.C2Heartbeat;
+import org.apache.nifi.c2.protocol.api.C2HeartbeatResponse;
+import org.apache.nifi.c2.protocol.api.C2OperationAck;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class C2HttpClient implements C2Client {
+
+    private static final Logger logger = LoggerFactory.getLogger(C2HttpClient.class);
+    private static final MediaType MEDIA_TYPE_APPLICATION_JSON = MediaType.parse("application/json");
+
+    private final AtomicReference<OkHttpClient> httpClientReference = new AtomicReference<>();
+    private final C2ClientConfig clientConfig;
+    private final C2Serializer serializer;
+
+    public C2HttpClient(C2ClientConfig clientConfig, C2Serializer serializer) {
+        super();
+        this.clientConfig = clientConfig;
+        this.serializer = serializer;
+        final OkHttpClient.Builder okHttpClientBuilder = new OkHttpClient.Builder();
+
+        // Configure request and response logging
+        HttpLoggingInterceptor logging = new HttpLoggingInterceptor();
+        logging.setLevel(HttpLoggingInterceptor.Level.BASIC);
+        okHttpClientBuilder.addInterceptor(logging);
+
+        // Set whether to follow redirects
+        okHttpClientBuilder.followRedirects(true);
+
+        // Timeout for calls made to the server
+        okHttpClientBuilder.callTimeout(clientConfig.getCallTimeout(), TimeUnit.MILLISECONDS);
+
+        // check if the ssl path is set and add the factory if so
+        if (StringUtils.isNotBlank(clientConfig.getKeystoreFilename())) {
+            try {
+                setSslSocketFactory(okHttpClientBuilder);
+            } catch (Exception e) {
+                throw new IllegalStateException("OkHttp TLS configuration failed", e);
+            }
+        }
+
+        httpClientReference.set(okHttpClientBuilder.build());
+    }
+
+    @Override
+    public Optional<C2HeartbeatResponse> publishHeartbeat(C2Heartbeat heartbeat) {
+        return serializer.serialize(heartbeat).flatMap(this::sendHeartbeat);
+    }
+
+    private Optional<C2HeartbeatResponse> sendHeartbeat(String heartbeat) {
+        logger.debug("Sending heartbeat to {}", clientConfig.getC2Url());
+
+        Optional<C2HeartbeatResponse> c2HeartbeatResponse = Optional.empty();
+        Request request = new Request.Builder()
+            .post(RequestBody.create(heartbeat, MEDIA_TYPE_APPLICATION_JSON))
+            .url(clientConfig.getC2Url())
+            .build();
+
+        try (Response heartbeatResponse = httpClientReference.get().newCall(request).execute()) {
+            c2HeartbeatResponse = getResponseBody(heartbeatResponse).flatMap(response -> serializer.deserialize(response, C2HeartbeatResponse.class));
+        } catch (IOException ce) {
+            logger.error("Send Heartbeat failed [{}]", clientConfig.getC2Url(), ce);
+        }
+
+        return c2HeartbeatResponse;
+    }
+
+    private Optional<String> getResponseBody(Response response) {
+        String responseBody = null;
+
+        try {
+            responseBody = response.body().string();
+            logger.debug("Received response body {}", responseBody);
+        } catch (IOException e) {
+            logger.error("HTTP Request failed", e);
+        }
+
+        return Optional.ofNullable(responseBody);
+    }
+
+    private void setSslSocketFactory(OkHttpClient.Builder okHttpClientBuilder) throws Exception {
+        final String keystoreLocation = clientConfig.getKeystoreFilename();
+        final String keystoreType = clientConfig.getKeystoreType();
+        final String keystorePass = clientConfig.getKeystorePass();
+
+        assertKeystorePropertiesSet(keystoreLocation, keystorePass, keystoreType);
+
+        // prepare the keystore
+        final KeyStore keyStore = KeyStore.getInstance(keystoreType);
+
+        try (FileInputStream keyStoreStream = new FileInputStream(keystoreLocation)) {
+            keyStore.load(keyStoreStream, keystorePass.toCharArray());
+        }
+
+        final KeyManagerFactory keyManagerFactory = KeyManagerFactory.getInstance(KeyManagerFactory.getDefaultAlgorithm());
+        keyManagerFactory.init(keyStore, keystorePass.toCharArray());
+
+        // load truststore
+        final String truststoreLocation = clientConfig.getTruststoreFilename();
+        final String truststorePass = clientConfig.getTruststorePass();
+        final String truststoreType = clientConfig.getTruststoreType();
+        assertTruststorePropertiesSet(truststoreLocation, truststorePass, truststoreType);
+
+        KeyStore truststore = KeyStore.getInstance(truststoreType);
+        final TrustManagerFactory trustManagerFactory = TrustManagerFactory.getInstance("X509");
+        truststore.load(new FileInputStream(truststoreLocation), truststorePass.toCharArray());
+        trustManagerFactory.init(truststore);
+
+        final X509TrustManager x509TrustManager;
+        TrustManager[] trustManagers = trustManagerFactory.getTrustManagers();
+        if (trustManagers[0] != null) {
+            x509TrustManager = (X509TrustManager) trustManagers[0];
+        } else {
+            throw new IllegalStateException("List of trust managers is null");
+        }
+
+        SSLContext tempSslContext;
+        try {
+            tempSslContext = SSLContext.getInstance("TLS");
+        } catch (NoSuchAlgorithmException e) {
+            throw new IllegalStateException("SSLContext creation failed", e);
+        }
+
+        final SSLContext sslContext = tempSslContext;
+        sslContext.init(keyManagerFactory.getKeyManagers(), trustManagerFactory.getTrustManagers(), null);
+
+        final SSLSocketFactory sslSocketFactory = sslContext.getSocketFactory();
+        okHttpClientBuilder.sslSocketFactory(sslSocketFactory, x509TrustManager);
+    }
+
+    private void assertKeystorePropertiesSet(String location, String password, String type) {
+        if (location == null || location.isEmpty()) {
+            throw new IllegalArgumentException(clientConfig.getKeystoreFilename() + " is null or is empty");
+        }
+
+        if (password == null || password.isEmpty()) {
+            throw new IllegalArgumentException("The client's keystore filename is set but its password is not (or is empty). If the location is set, the password must also be.");
+        }
+
+        if (type == null || type.isEmpty()) {
+            throw new IllegalArgumentException("The client's keystore filename is set but its type is not (or is empty). If the location is set, the type must also be.");
+        }
+    }
+
+    private void assertTruststorePropertiesSet(String location, String password, String type) {
+        if (location == null || location.isEmpty()) {
+            throw new IllegalArgumentException("The client's truststore filename is not set or is empty");
+        }
+
+        if (password == null || password.isEmpty()) {
+            throw new IllegalArgumentException("The client's truststore filename is set but its password is not (or is empty). If the location is set, the password must also be.");
+        }
+
+        if (type == null || type.isEmpty()) {
+            throw new IllegalArgumentException("The client's truststore filename is set but its type is not (or is empty). If the location is set, the type must also be.");
+        }
+    }
+
+    @Override
+    public byte[] retrieveUpdateContent(String flowUpdateUrl) {
+        final Request.Builder requestBuilder = new Request.Builder()
+                .get()
+                .url(flowUpdateUrl);
+        final Request request = requestBuilder.build();
+
+        ResponseBody body;
+        try (final Response response = httpClientReference.get().newCall(request).execute()) {
+            logger.debug("Response received: {}", response);

Review Comment:
   You are right, I removed it.



##########
c2/c2-client-bundle/c2-client-service/src/main/java/org/apache/nifi/c2/client/service/operation/UpdateConfigurationOperationHandler.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.nifi.c2.client.service.operation;
+
+import static org.apache.commons.lang3.StringUtils.EMPTY;
+import static org.apache.nifi.c2.protocol.api.OperandType.CONFIGURATION;
+import static org.apache.nifi.c2.protocol.api.OperationType.UPDATE;
+
+import java.net.URI;
+import java.util.Optional;
+import java.util.function.Function;
+import org.apache.nifi.c2.client.api.C2Client;
+import org.apache.nifi.c2.client.service.FlowIdHolder;
+import org.apache.nifi.c2.protocol.api.C2Operation;
+import org.apache.nifi.c2.protocol.api.C2OperationAck;
+import org.apache.nifi.c2.protocol.api.C2OperationState;
+import org.apache.nifi.c2.protocol.api.OperandType;
+import org.apache.nifi.c2.protocol.api.OperationType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class UpdateConfigurationOperationHandler implements C2OperationHandler {
+
+    private static final Logger logger = LoggerFactory.getLogger(UpdateConfigurationOperationHandler.class);
+
+    private static final String LOCATION = "location";
+
+    private final C2Client client;
+    private final Function<byte[], Boolean> updateFlow;
+    private final FlowIdHolder flowIdHolder;
+
+    public UpdateConfigurationOperationHandler(C2Client client, FlowIdHolder flowIdHolder, Function<byte[], Boolean> updateFlow) {
+        this.client = client;
+        this.updateFlow = updateFlow;
+        this.flowIdHolder = flowIdHolder;
+    }
+
+    @Override
+    public OperationType getOperationType() {
+        return UPDATE;
+    }
+
+    @Override
+    public OperandType getOperandType() {
+        return CONFIGURATION;
+    }
+
+    @Override
+    public C2OperationAck handle(C2Operation operation) {
+        String opIdentifier = Optional.ofNullable(operation.getIdentifier())
+            .orElse(EMPTY);
+        C2OperationAck operationAck = new C2OperationAck();
+        C2OperationState state = new C2OperationState();
+        operationAck.setOperationState(state);
+        operationAck.setOperationId(opIdentifier);
+
+        String updateLocation = Optional.ofNullable(operation.getArgs())
+            .map(map -> map.get(LOCATION))
+            .orElse(EMPTY);
+
+        String newFlowId = parseFlowId(updateLocation);
+        if (flowIdHolder.getFlowId() == null || !flowIdHolder.getFlowId().equals(newFlowId)) {
+            logger.info("Will perform flow update from {} for operation #{}. Previous flow id was {}, replacing with new id {}", updateLocation, opIdentifier,
+                flowIdHolder.getFlowId() == null ? "not set" : flowIdHolder.getFlowId(), newFlowId);
+        } else {
+            logger.info("Flow is current, no update is necessary...");
+        }
+
+        flowIdHolder.setFlowId(newFlowId);
+        byte[] updateContent = client.retrieveUpdateContent(updateLocation);
+        if (updateContent != null) {
+            if (updateFlow.apply(updateContent)) {
+                state.setState(C2OperationState.OperationState.FULLY_APPLIED);
+                logger.debug("Update configuration applied for operation #{}.", opIdentifier);
+            } else {
+                state.setState(C2OperationState.OperationState.NOT_APPLIED);
+                logger.error("Update resulted in error for operation #{}.", opIdentifier);
+            }
+        } else {
+            state.setState(C2OperationState.OperationState.NOT_APPLIED);
+            logger.error("Update content retrieval resulted in empty content so flow update was omitted for operation #{}.", opIdentifier);
+        }
+
+        return operationAck;
+    }
+
+    private String parseFlowId(String flowUpdateUrl) {
+        try {
+            URI flowUri = new URI(flowUpdateUrl);
+            String flowUriPath = flowUri.getPath();
+            String[] split = flowUriPath.split("/");
+            if (split.length > 4) {
+                return split[4];
+            } else {
+                throw new Exception();

Review Comment:
   Applied



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] bejancsaba commented on a diff in pull request #6075: NIFI-9666 Implement base C2 client functionality

Posted by GitBox <gi...@apache.org>.
bejancsaba commented on code in PR #6075:
URL: https://github.com/apache/nifi/pull/6075#discussion_r891078615


##########
c2/c2-client-bundle/c2-client-http/src/main/java/org/apache/nifi/c2/client/http/C2HttpClient.java:
##########
@@ -0,0 +1,250 @@
+/*
+ * 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.nifi.c2.client.http;
+
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.security.KeyStore;
+import java.security.NoSuchAlgorithmException;
+import java.util.Optional;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLSocketFactory;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+import okhttp3.MediaType;
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.RequestBody;
+import okhttp3.Response;
+import okhttp3.ResponseBody;
+import okhttp3.logging.HttpLoggingInterceptor;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.c2.client.C2ClientConfig;
+import org.apache.nifi.c2.client.api.C2Client;
+import org.apache.nifi.c2.client.api.C2Serializer;
+import org.apache.nifi.c2.protocol.api.C2Heartbeat;
+import org.apache.nifi.c2.protocol.api.C2HeartbeatResponse;
+import org.apache.nifi.c2.protocol.api.C2OperationAck;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class C2HttpClient implements C2Client {
+
+    private static final Logger logger = LoggerFactory.getLogger(C2HttpClient.class);
+    private static final MediaType MEDIA_TYPE_APPLICATION_JSON = MediaType.parse("application/json");
+
+    private final AtomicReference<OkHttpClient> httpClientReference = new AtomicReference<>();
+    private final C2ClientConfig clientConfig;
+    private final C2Serializer serializer;
+
+    public C2HttpClient(C2ClientConfig clientConfig, C2Serializer serializer) {
+        super();
+        this.clientConfig = clientConfig;
+        this.serializer = serializer;
+        final OkHttpClient.Builder okHttpClientBuilder = new OkHttpClient.Builder();
+
+        // Configure request and response logging
+        HttpLoggingInterceptor logging = new HttpLoggingInterceptor();
+        logging.setLevel(HttpLoggingInterceptor.Level.BASIC);
+        okHttpClientBuilder.addInterceptor(logging);
+
+        // Set whether to follow redirects
+        okHttpClientBuilder.followRedirects(true);
+
+        // Timeout for calls made to the server
+        okHttpClientBuilder.callTimeout(clientConfig.getCallTimeout(), TimeUnit.MILLISECONDS);
+
+        // check if the ssl path is set and add the factory if so
+        if (StringUtils.isNotBlank(clientConfig.getKeystoreFilename())) {
+            try {
+                setSslSocketFactory(okHttpClientBuilder);
+            } catch (Exception e) {
+                throw new IllegalStateException("OkHttp TLS configuration failed", e);
+            }
+        }
+
+        httpClientReference.set(okHttpClientBuilder.build());
+    }
+
+    @Override
+    public Optional<C2HeartbeatResponse> publishHeartbeat(C2Heartbeat heartbeat) {
+        return serializer.serialize(heartbeat).flatMap(this::sendHeartbeat);
+    }
+
+    private Optional<C2HeartbeatResponse> sendHeartbeat(String heartbeat) {
+        logger.debug("Sending heartbeat to {}", clientConfig.getC2Url());
+
+        Optional<C2HeartbeatResponse> c2HeartbeatResponse = Optional.empty();
+        Request request = new Request.Builder()
+            .post(RequestBody.create(heartbeat, MEDIA_TYPE_APPLICATION_JSON))
+            .url(clientConfig.getC2Url())
+            .build();
+
+        try (Response heartbeatResponse = httpClientReference.get().newCall(request).execute()) {
+            c2HeartbeatResponse = getResponseBody(heartbeatResponse).flatMap(response -> serializer.deserialize(response, C2HeartbeatResponse.class));

Review Comment:
   Yes, in the deserializer the objectmapper if not able to map to the target class will provide an exception about the problem which we are logging there.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory commented on pull request #6075: NIFI-9666 Implement base C2 client functionality

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on PR #6075:
URL: https://github.com/apache/nifi/pull/6075#issuecomment-1150584129

   Thanks for working through the feedback @bejancsaba and @ferencerdei! The latest updates are helpful.
   
   The one remaining concern is the use of the JDK Virtual Machine classes through reflection, which apparently doesn't work on Java 17. Given that those are not part of the public JDK, it seems like a different approach should be evaluated.
   
   I will take another look and run through some tests soon. This includes a lot of great improvements and features, so thanks for working through the feedback process!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory commented on pull request #6075: NIFI-9666 Implement base C2 client functionality

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on PR #6075:
URL: https://github.com/apache/nifi/pull/6075#issuecomment-1157835163

   Thanks @joewitt and @kevdoran! I think a number of the commits could be squashed down to two or three so that each contributor can be represented. Will collaborate with @kevdoran on orchestrating the merge when ready.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] ferencerdei commented on a diff in pull request #6075: NIFI-9666 Implement base C2 client functionality

Posted by GitBox <gi...@apache.org>.
ferencerdei commented on code in PR #6075:
URL: https://github.com/apache/nifi/pull/6075#discussion_r893252008


##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/util/ProcessUtils.java:
##########
@@ -0,0 +1,131 @@
+/*
+ * 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.nifi.minifi.bootstrap.util;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.Reader;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ProcessUtils {

Review Comment:
   ok, thanks. I renamed already and I'll create a ticket after this PR is merged.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #6075: NIFI-9666 Implement base C2 client functionality

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on code in PR #6075:
URL: https://github.com/apache/nifi/pull/6075#discussion_r892997205


##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/command/EnvRunner.java:
##########
@@ -0,0 +1,107 @@
+/*
+ * 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.nifi.minifi.bootstrap.command;
+
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CMD_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.DEFAULT_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.Status.ERROR;
+import static org.apache.nifi.minifi.bootstrap.Status.MINIFI_NOT_RUNNING;
+import static org.apache.nifi.minifi.bootstrap.Status.OK;
+
+import java.lang.reflect.Method;
+import java.util.Map.Entry;
+import java.util.Properties;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.apache.nifi.minifi.bootstrap.MiNiFiStatus;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiStatusProvider;
+
+public class EnvRunner implements CommandRunner {
+    private final MiNiFiParameters miNiFiParameters;
+    private final MiNiFiStatusProvider miNiFiStatusProvider;
+
+    public EnvRunner(MiNiFiParameters miNiFiParameters, MiNiFiStatusProvider miNiFiStatusProvider) {
+        this.miNiFiParameters = miNiFiParameters;
+        this.miNiFiStatusProvider = miNiFiStatusProvider;
+    }
+
+    /**
+     * Returns information about the MiNiFi's virtual machine.
+     * @param args the input arguments
+     * @return status code
+     */
+    @Override
+    public int runCommand(String[] args) {
+        return env();
+    }
+
+    private int env() {
+        MiNiFiStatus status = miNiFiStatusProvider.getStatus(miNiFiParameters.getMiNiFiPort(), miNiFiParameters.getMinifiPid());
+        if (status.getPid() == null) {
+            CMD_LOGGER.error("Apache MiNiFi is not running");
+            return MINIFI_NOT_RUNNING.getStatusCode();
+        }
+
+        Class<?> virtualMachineClass;
+        try {
+            virtualMachineClass = Class.forName("com.sun.tools.attach.VirtualMachine");
+        } catch (ClassNotFoundException cnfe) {
+            CMD_LOGGER.error("Seems tools.jar (Linux / Windows JDK) or classes.jar (Mac OS) is not available in classpath");
+            return ERROR.getStatusCode();
+        }
+
+        Method attachMethod;
+        Method detachMethod;
+        try {
+            attachMethod = virtualMachineClass.getMethod("attach", String.class);
+            detachMethod = virtualMachineClass.getDeclaredMethod("detach");
+        } catch (Exception e) {
+            CMD_LOGGER.error("Methods required for getting environment not available");
+            DEFAULT_LOGGER.error("Exception:", e);
+            return ERROR.getStatusCode();
+        }
+
+        Object virtualMachine;
+        try {
+            virtualMachine = attachMethod.invoke(null, status.getPid());
+        } catch (Exception e) {
+            CMD_LOGGER.error("Problem attaching to MiNiFi");
+            DEFAULT_LOGGER.error("Exception:", e);
+            return ERROR.getStatusCode();
+        }
+
+        try {
+            Method getSystemPropertiesMethod = virtualMachine.getClass().getMethod("getSystemProperties");
+
+            Properties sysProps = (Properties) getSystemPropertiesMethod.invoke(virtualMachine);

Review Comment:
   Thanks for the explanation, that makes sense.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #6075: NIFI-9666 Implement base C2 client functionality

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on code in PR #6075:
URL: https://github.com/apache/nifi/pull/6075#discussion_r892996699


##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/MiNiFiStatus.java:
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.nifi.minifi.bootstrap;
+
+public class MiNiFiStatus {
+
+    private final Integer port;
+    private final String pid;

Review Comment:
   Thanks for the reply, changing to a `Long` sounds good.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] kevdoran commented on pull request #6075: NIFI-9666 Implement base C2 client functionality

Posted by GitBox <gi...@apache.org>.
kevdoran commented on PR #6075:
URL: https://github.com/apache/nifi/pull/6075#issuecomment-1154437187

   Thansk @ferencerdei. I've verified the SIGTERM fix works as expected.
   
   Regarding the logging output: I'm comparing the output of the docker image from this PR to the [latest release of nifi-minifi ](https://hub.docker.com/r/apache/nifi-minifi/tags), and I get the full output (more than just the bootstrap process) when I run this:
   
   ```
   docker run apache/nifi-minifi:latest
   ```
   
   compared to this (after building this PR branch), which only has the bootstrap output:
   
   ```
   docker run apacheminifi:latest
   ```
   
   I could be missing something, but I would expect those to more or less be the same in docker container output behavior?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] bejancsaba commented on a diff in pull request #6075: NIFI-9666 Implement base C2 client functionality

Posted by GitBox <gi...@apache.org>.
bejancsaba commented on code in PR #6075:
URL: https://github.com/apache/nifi/pull/6075#discussion_r891074965


##########
c2/c2-client-bundle/c2-client-base/src/main/java/org/apache/nifi/c2/client/C2ClientConfig.java:
##########
@@ -0,0 +1,269 @@
+/*
+ * 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.nifi.c2.client;
+
+import javax.net.ssl.HostnameVerifier;
+
+/**
+ * Configuration for a C2 Client.
+ */
+public class C2ClientConfig {
+
+    private final String c2Url;
+    private final String c2AckUrl;
+    private final String agentClass;
+    private final String agentIdentifier;
+    private final String confDirectory;
+    private final String runtimeManifestIdentifier;
+    private final String runtimeType;
+    private final Long heartbeatPeriod;
+    private final Long callTimeout;
+    private final String keystoreFilename;
+    private final String keystorePass;
+    private final String keyPass;
+    private final String keystoreType;
+    private final String truststoreFilename;
+    private final String truststorePass;
+    private final String truststoreType;
+    private final HostnameVerifier hostnameVerifier;
+    private final Integer readTimeout;
+    private final Integer connectTimeout;

Review Comment:
   I suppose there is no reason for that moved to primitives.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #6075: NIFI-9666 Implement base C2 client functionality

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on code in PR #6075:
URL: https://github.com/apache/nifi/pull/6075#discussion_r890254387


##########
c2/c2-client-bundle/c2-client-http/src/main/java/org/apache/nifi/c2/client/http/C2HttpClient.java:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.nifi.c2.client.http;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.security.KeyStore;
+import java.security.NoSuchAlgorithmException;
+import java.util.Optional;
+import java.util.concurrent.atomic.AtomicReference;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLSocketFactory;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+import okhttp3.MediaType;
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.RequestBody;
+import okhttp3.Response;
+import okhttp3.ResponseBody;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.c2.client.C2ClientConfig;
+import org.apache.nifi.c2.client.api.C2Client;
+import org.apache.nifi.c2.client.api.C2Serializer;
+import org.apache.nifi.c2.protocol.api.C2Heartbeat;
+import org.apache.nifi.c2.protocol.api.C2HeartbeatResponse;
+import org.apache.nifi.c2.protocol.api.C2OperationAck;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class C2HttpClient implements C2Client {
+
+    private static final Logger logger = LoggerFactory.getLogger(C2HttpClient.class);
+    private static final MediaType MEDIA_TYPE_APPLICATION_JSON = MediaType.parse("application/json");
+
+    private final AtomicReference<OkHttpClient> httpClientReference = new AtomicReference<>();
+    private final C2ClientConfig clientConfig;
+    private final C2Serializer serializer;
+
+    public C2HttpClient(C2ClientConfig clientConfig, C2Serializer serializer) {
+        super();
+        this.clientConfig = clientConfig;
+        this.serializer = serializer;
+        final OkHttpClient.Builder okHttpClientBuilder = new OkHttpClient.Builder();
+
+        // Set whether to follow redirects
+        okHttpClientBuilder.followRedirects(true);
+
+        // check if the ssl path is set and add the factory if so
+        if (StringUtils.isNotBlank(clientConfig.getKeystoreFilename())) {
+            try {
+                setSslSocketFactory(okHttpClientBuilder);
+            } catch (Exception e) {
+                throw new IllegalStateException(e);
+            }
+        }
+
+        httpClientReference.set(okHttpClientBuilder.build());
+    }
+
+    @Override
+    public Optional<C2HeartbeatResponse> publishHeartbeat(C2Heartbeat heartbeat) {
+        return serializer.serialize(heartbeat).flatMap(this::sendHeartbeat);
+    }
+
+    private Optional<C2HeartbeatResponse> sendHeartbeat(String heartbeat) {
+        logger.debug("Sending heartbeat to {}", clientConfig.getC2Url());
+
+        Optional<C2HeartbeatResponse> c2HeartbeatResponse = Optional.empty();
+        Request request = new Request.Builder()
+            .post(RequestBody.create(heartbeat, MEDIA_TYPE_APPLICATION_JSON))
+            .url(clientConfig.getC2Url())
+            .build();
+
+        try {
+            Response heartbeatResponse = httpClientReference.get().newCall(request).execute();
+            c2HeartbeatResponse = getResponseBody(heartbeatResponse).flatMap(response -> serializer.deserialize(response, C2HeartbeatResponse.class));
+        } catch (IOException ce) {
+            logger.error("Error while sending heartbeat", ce);
+        }
+
+        return c2HeartbeatResponse;
+    }
+
+    private Optional<String> getResponseBody(Response response) {
+        String responseBody = null;
+
+        try {
+            responseBody = response.body().string();
+            logger.debug("Received response (Status={}) {}", response.code(), responseBody);
+        } catch (IOException e) {
+            logger.error("Could not get response body: ", e);
+        }
+
+        return Optional.ofNullable(responseBody);
+    }
+
+    private void setSslSocketFactory(OkHttpClient.Builder okHttpClientBuilder) throws Exception {
+        final String keystoreLocation = clientConfig.getKeystoreFilename();
+        final String keystoreType = clientConfig.getKeystoreType().getType();
+        final String keystorePass = clientConfig.getKeystorePass();
+
+        assertKeystorePropertiesSet(keystoreLocation, keystorePass, keystoreType);
+
+        // prepare the keystore
+        final KeyStore keyStore = KeyStore.getInstance(keystoreType);
+
+        try (FileInputStream keyStoreStream = new FileInputStream(keystoreLocation)) {
+            keyStore.load(keyStoreStream, keystorePass.toCharArray());
+        }
+
+        final KeyManagerFactory keyManagerFactory = KeyManagerFactory.getInstance(KeyManagerFactory.getDefaultAlgorithm());
+        keyManagerFactory.init(keyStore, keystorePass.toCharArray());
+
+        // load truststore
+        final String truststoreLocation = clientConfig.getTruststoreFilename();
+        final String truststorePass = clientConfig.getTruststorePass();
+        final String truststoreType = clientConfig.getTruststoreType().getType();
+        assertTruststorePropertiesSet(truststoreLocation, truststorePass, truststoreType);
+
+        KeyStore truststore = KeyStore.getInstance(truststoreType);
+        final TrustManagerFactory trustManagerFactory = TrustManagerFactory.getInstance("X509");
+        truststore.load(new FileInputStream(truststoreLocation), truststorePass.toCharArray());
+        trustManagerFactory.init(truststore);
+
+        final X509TrustManager x509TrustManager;
+        TrustManager[] trustManagers = trustManagerFactory.getTrustManagers();
+        if (trustManagers[0] != null) {
+            x509TrustManager = (X509TrustManager) trustManagers[0];
+        } else {
+            throw new IllegalStateException("List of trust managers is null");
+        }
+
+        SSLContext tempSslContext;
+        try {
+            tempSslContext = SSLContext.getInstance("TLS");
+        } catch (NoSuchAlgorithmException e) {
+            logger.warn("Unable to use 'TLS' for the PullHttpChangeIngestor due to NoSuchAlgorithmException. Will attempt to use the default algorithm.", e);
+            tempSslContext = SSLContext.getDefault();
+        }
+
+        final SSLContext sslContext = tempSslContext;
+        sslContext.init(keyManagerFactory.getKeyManagers(), trustManagerFactory.getTrustManagers(), null);
+
+        final SSLSocketFactory sslSocketFactory = sslContext.getSocketFactory();
+        okHttpClientBuilder.sslSocketFactory(sslSocketFactory, x509TrustManager);
+    }
+
+    private void assertKeystorePropertiesSet(String location, String password, String type) {
+        if (location == null || location.isEmpty()) {
+            throw new IllegalArgumentException(clientConfig.getKeystoreFilename() + " is null or is empty");
+        }
+
+        if (password == null || password.isEmpty()) {
+            throw new IllegalArgumentException("The client's keystore filename is set but its password is not (or is empty). If the location is set, the password must also be.");
+        }
+
+        if (type == null || type.isEmpty()) {
+            throw new IllegalArgumentException("The client's keystore filename is set but its type is not (or is empty). If the location is set, the type must also be.");
+        }
+    }
+
+    private void assertTruststorePropertiesSet(String location, String password, String type) {
+        if (location == null || location.isEmpty()) {
+            throw new IllegalArgumentException("The client's truststore filename is not set or is empty");
+        }
+
+        if (password == null || password.isEmpty()) {
+            throw new IllegalArgumentException("The client's truststore filename is set but its password is not (or is empty). If the location is set, the password must also be.");
+        }
+
+        if (type == null || type.isEmpty()) {
+            throw new IllegalArgumentException("The client's truststore filename is set but its type is not (or is empty). If the location is set, the type must also be.");
+        }
+    }
+
+    @Override
+    public ByteBuffer retrieveUpdateContent(String flowUpdateUrl) {
+        final Request.Builder requestBuilder = new Request.Builder()
+                .get()
+                .url(flowUpdateUrl);
+        final Request request = requestBuilder.build();
+
+        ResponseBody body;
+        try (final Response response = httpClientReference.get().newCall(request).execute()) {
+            logger.debug("Response received: {}", response);
+
+            int code = response.code();
+            if (code >= 400) {
+                throw new IOException("Got response code " + code + " while trying to pull configuration: " + response.body().string());

Review Comment:
   That sounds good.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] bejancsaba commented on a diff in pull request #6075: NIFI-9666 Implement base C2 client functionality

Posted by GitBox <gi...@apache.org>.
bejancsaba commented on code in PR #6075:
URL: https://github.com/apache/nifi/pull/6075#discussion_r891077359


##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/MiNiFiConfigurationChangeListener.java:
##########
@@ -0,0 +1,215 @@
+/*
+ * 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.nifi.minifi.bootstrap.service;
+
+import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CONF_DIR_KEY;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.MINIFI_CONFIG_FILE_KEY;
+import static org.apache.nifi.minifi.bootstrap.configuration.ingestors.PullHttpChangeIngestor.OVERRIDE_SECURITY;
+import static org.apache.nifi.minifi.bootstrap.configuration.ingestors.PullHttpChangeIngestor.PULL_HTTP_BASE_KEY;
+import static org.apache.nifi.minifi.bootstrap.util.ConfigTransformer.generateConfigFiles;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.util.Properties;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.commons.io.IOUtils;
+import org.apache.nifi.minifi.bootstrap.RunMiNiFi;
+import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeException;
+import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeListener;
+import org.apache.nifi.minifi.bootstrap.util.ByteBufferInputStream;
+import org.apache.nifi.minifi.bootstrap.util.ConfigTransformer;
+import org.apache.nifi.minifi.commons.schema.ConfigSchema;
+import org.apache.nifi.minifi.commons.schema.common.ConvertableSchema;
+import org.apache.nifi.minifi.commons.schema.serialization.SchemaLoader;
+import org.slf4j.Logger;
+
+public class MiNiFiConfigurationChangeListener implements ConfigurationChangeListener {
+
+    private final RunMiNiFi runner;
+    private final Logger logger;
+    private final BootstrapFileProvider bootstrapFileProvider;
+
+    private static final ReentrantLock handlingLock = new ReentrantLock();
+
+    public MiNiFiConfigurationChangeListener(RunMiNiFi runner, Logger logger, BootstrapFileProvider bootstrapFileProvider) {
+        this.runner = runner;
+        this.logger = logger;
+        this.bootstrapFileProvider = bootstrapFileProvider;
+    }
+
+    @Override
+    public void handleChange(InputStream configInputStream) throws ConfigurationChangeException {
+        logger.info("Received notification of a change");
+
+        if (!handlingLock.tryLock()) {
+            throw new ConfigurationChangeException("Instance is already handling another change");
+        }
+        // Store the incoming stream as a byte array to be shared among components that need it
+        try(ByteArrayOutputStream bufferedConfigOs = new ByteArrayOutputStream()) {
+
+            Properties bootstrapProperties = bootstrapFileProvider.getBootstrapProperties();
+            File configFile = new File(bootstrapProperties.getProperty(MINIFI_CONFIG_FILE_KEY));
+
+            byte[] copyArray = new byte[1024];
+            int available;
+            while ((available = configInputStream.read(copyArray)) > 0) {
+                bufferedConfigOs.write(copyArray, 0, available);
+            }
+
+            File swapConfigFile = bootstrapFileProvider.getSwapFile();
+            logger.info("Persisting old configuration to {}", swapConfigFile.getAbsolutePath());
+
+            try (FileInputStream configFileInputStream = new FileInputStream(configFile)) {
+                Files.copy(configFileInputStream, swapConfigFile.toPath(), REPLACE_EXISTING);
+            }
+
+            persistBackNonFlowSectionsFromOriginalSchema(bufferedConfigOs.toByteArray(), bootstrapProperties, configFile);
+
+            // Create an input stream to feed to the config transformer
+            try (ByteArrayInputStream newConfigBais = new ByteArrayInputStream(IOUtils.toByteArray(new FileInputStream(configFile)))) {
+                newConfigBais.mark(-1);

Review Comment:
   You are right I went with FileInputStream (there were a few rewrites and code movement around here so the added pair of eyes is welcome)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] bejancsaba commented on a diff in pull request #6075: NIFI-9666 Implement base C2 client functionality

Posted by GitBox <gi...@apache.org>.
bejancsaba commented on code in PR #6075:
URL: https://github.com/apache/nifi/pull/6075#discussion_r891079721


##########
c2/c2-client-bundle/c2-client-service/src/main/java/org/apache/nifi/c2/client/service/operation/UpdateConfigurationOperationHandler.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.nifi.c2.client.service.operation;
+
+import static org.apache.commons.lang3.StringUtils.EMPTY;
+import static org.apache.nifi.c2.protocol.api.OperandType.CONFIGURATION;
+import static org.apache.nifi.c2.protocol.api.OperationType.UPDATE;
+
+import java.net.URI;
+import java.util.Optional;
+import java.util.function.Function;
+import org.apache.nifi.c2.client.api.C2Client;
+import org.apache.nifi.c2.client.service.FlowIdHolder;
+import org.apache.nifi.c2.protocol.api.C2Operation;
+import org.apache.nifi.c2.protocol.api.C2OperationAck;
+import org.apache.nifi.c2.protocol.api.C2OperationState;
+import org.apache.nifi.c2.protocol.api.OperandType;
+import org.apache.nifi.c2.protocol.api.OperationType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class UpdateConfigurationOperationHandler implements C2OperationHandler {
+
+    private static final Logger logger = LoggerFactory.getLogger(UpdateConfigurationOperationHandler.class);
+
+    private static final String LOCATION = "location";
+
+    private final C2Client client;
+    private final Function<byte[], Boolean> updateFlow;
+    private final FlowIdHolder flowIdHolder;
+
+    public UpdateConfigurationOperationHandler(C2Client client, FlowIdHolder flowIdHolder, Function<byte[], Boolean> updateFlow) {
+        this.client = client;
+        this.updateFlow = updateFlow;
+        this.flowIdHolder = flowIdHolder;
+    }
+
+    @Override
+    public OperationType getOperationType() {
+        return UPDATE;
+    }
+
+    @Override
+    public OperandType getOperandType() {
+        return CONFIGURATION;
+    }
+
+    @Override
+    public C2OperationAck handle(C2Operation operation) {
+        String opIdentifier = Optional.ofNullable(operation.getIdentifier())
+            .orElse(EMPTY);
+        C2OperationAck operationAck = new C2OperationAck();
+        C2OperationState state = new C2OperationState();
+        operationAck.setOperationState(state);
+        operationAck.setOperationId(opIdentifier);
+
+        String updateLocation = Optional.ofNullable(operation.getArgs())
+            .map(map -> map.get(LOCATION))
+            .orElse(EMPTY);
+
+        String newFlowId = parseFlowId(updateLocation);
+        if (flowIdHolder.getFlowId() == null || !flowIdHolder.getFlowId().equals(newFlowId)) {
+            logger.info("Will perform flow update from {} for operation #{}. Previous flow id was {}, replacing with new id {}", updateLocation, opIdentifier,
+                flowIdHolder.getFlowId() == null ? "not set" : flowIdHolder.getFlowId(), newFlowId);
+        } else {
+            logger.info("Flow is current, no update is necessary...");
+        }
+
+        flowIdHolder.setFlowId(newFlowId);
+        byte[] updateContent = client.retrieveUpdateContent(updateLocation);
+        if (updateContent != null) {
+            if (updateFlow.apply(updateContent)) {
+                state.setState(C2OperationState.OperationState.FULLY_APPLIED);
+                logger.debug("Update configuration applied for operation #{}.", opIdentifier);
+            } else {
+                state.setState(C2OperationState.OperationState.NOT_APPLIED);
+                logger.error("Update resulted in error for operation #{}.", opIdentifier);
+            }
+        } else {
+            state.setState(C2OperationState.OperationState.NOT_APPLIED);
+            logger.error("Update content retrieval resulted in empty content so flow update was omitted for operation #{}.", opIdentifier);
+        }
+
+        return operationAck;
+    }
+
+    private String parseFlowId(String flowUpdateUrl) {
+        try {
+            URI flowUri = new URI(flowUpdateUrl);
+            String flowUriPath = flowUri.getPath();
+            String[] split = flowUriPath.split("/");
+            if (split.length > 4) {
+                return split[4];

Review Comment:
   Thanks and we are in agreement this needs to be hardened. For now we have compatibility with the c2 server reference implementation but we plan to enhance that one too and as part of that effort we will be hardening this approach as well. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] bejancsaba commented on pull request #6075: NIFI-9666 Implement base C2 client functionality

Posted by GitBox <gi...@apache.org>.
bejancsaba commented on PR #6075:
URL: https://github.com/apache/nifi/pull/6075#issuecomment-1158717071

   All outstanding comments have been addressed., thanks everyone for the thorough review. It is quite a complex feature so any help is highly appreciated.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] bejancsaba commented on a diff in pull request #6075: NIFI-9666 Implement base C2 client functionality

Posted by GitBox <gi...@apache.org>.
bejancsaba commented on code in PR #6075:
URL: https://github.com/apache/nifi/pull/6075#discussion_r888827087


##########
c2/c2-client-bundle/c2-client-api/src/main/java/org/apache/nifi/c2/client/api/C2Client.java:
##########
@@ -0,0 +1,37 @@
+/*
+ * 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.nifi.c2.client.api;
+
+import java.util.Optional;
+import org.apache.nifi.c2.protocol.api.C2Heartbeat;
+import org.apache.nifi.c2.protocol.api.C2HeartbeatResponse;
+import org.apache.nifi.c2.protocol.api.C2OperationAck;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+/**
+ * Defines interface methods used to implement a C2 Client. The controller can be application-specific but is used for such tasks as updating the flow.
+ */
+public interface C2Client {
+
+    Optional<C2HeartbeatResponse> publishHeartbeat(C2Heartbeat heartbeat) throws IOException;
+
+    ByteBuffer retrieveUpdateContent(String flowUpdateUrl);

Review Comment:
   There is no particular reason for ByteBuffer. I went with byte[] as that is coming natively from okhttp



##########
c2/c2-client-bundle/c2-client-api/src/main/java/org/apache/nifi/c2/client/api/C2Client.java:
##########
@@ -0,0 +1,37 @@
+/*
+ * 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.nifi.c2.client.api;
+
+import java.util.Optional;
+import org.apache.nifi.c2.protocol.api.C2Heartbeat;
+import org.apache.nifi.c2.protocol.api.C2HeartbeatResponse;
+import org.apache.nifi.c2.protocol.api.C2OperationAck;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+/**
+ * Defines interface methods used to implement a C2 Client. The controller can be application-specific but is used for such tasks as updating the flow.
+ */
+public interface C2Client {
+
+    Optional<C2HeartbeatResponse> publishHeartbeat(C2Heartbeat heartbeat) throws IOException;

Review Comment:
   Added javadoc here and the neighbouring interfaces



##########
c2/c2-client-bundle/c2-client-base/src/main/java/org/apache/nifi/c2/serializer/C2JacksonSerializer.java:
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.nifi.c2.serializer;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.util.Optional;
+import org.apache.nifi.c2.client.api.C2Serializer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class C2JacksonSerializer implements C2Serializer {
+
+    private static final Logger logger = LoggerFactory.getLogger(C2JacksonSerializer.class);
+
+    private final ObjectMapper objectMapper;
+
+    public C2JacksonSerializer() {
+        objectMapper = new ObjectMapper();
+        objectMapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
+        objectMapper.setSerializationInclusion(JsonInclude.Include.NON_NULL);
+        objectMapper.enable(DeserializationFeature.READ_ENUMS_USING_TO_STRING);
+    }
+
+    @Override
+    public <T> Optional<String> serialize(T object) {
+        if (object == null) {
+            logger.trace("C2 Object was null. Nothing to serialize. Returning empty.");
+            return Optional.empty();
+        }
+
+        String contentString = null;
+        try {
+            contentString = objectMapper.writeValueAsString(object);
+            logger.trace("Serialized C2 Object: {}", contentString);
+        } catch (JsonProcessingException e) {
+            logger.error("Can't serialise C2 Object: ", e);
+        }
+
+        return Optional.ofNullable(contentString);
+    }
+
+    @Override
+    public <T> Optional<T> deserialize(String content, Class<T> valueType) {
+        if (content == null) {
+            logger.trace("Content for deserialization was null. Returning empty.");
+            return Optional.empty();
+        }
+
+        T responseObject = null;
+        try {
+            responseObject = objectMapper.readValue(content, valueType);
+        } catch (JsonProcessingException e) {
+            logger.error("Can't deserialize response object: ", e);

Review Comment:
   applied



##########
c2/c2-client-bundle/c2-client-http/pom.xml:
##########
@@ -0,0 +1,48 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+ -->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <artifactId>c2-client-bundle</artifactId>
+        <groupId>org.apache.nifi</groupId>
+        <version>1.17.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>c2-client-http</artifactId>
+    <packaging>jar</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>c2-client-api</artifactId>
+            <version>1.17.0-SNAPSHOT</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>c2-client-base</artifactId>
+            <version>1.17.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>com.squareup.okhttp3</groupId>
+            <artifactId>okhttp</artifactId>
+            <version>${okhttp.version}</version>

Review Comment:
   removed, thanks



##########
c2/c2-client-bundle/c2-client-http/src/main/java/org/apache/nifi/c2/client/http/C2HttpClient.java:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.nifi.c2.client.http;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.security.KeyStore;
+import java.security.NoSuchAlgorithmException;
+import java.util.Optional;
+import java.util.concurrent.atomic.AtomicReference;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLSocketFactory;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+import okhttp3.MediaType;
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.RequestBody;
+import okhttp3.Response;
+import okhttp3.ResponseBody;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.c2.client.C2ClientConfig;
+import org.apache.nifi.c2.client.api.C2Client;
+import org.apache.nifi.c2.client.api.C2Serializer;
+import org.apache.nifi.c2.protocol.api.C2Heartbeat;
+import org.apache.nifi.c2.protocol.api.C2HeartbeatResponse;
+import org.apache.nifi.c2.protocol.api.C2OperationAck;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class C2HttpClient implements C2Client {
+
+    private static final Logger logger = LoggerFactory.getLogger(C2HttpClient.class);
+    private static final MediaType MEDIA_TYPE_APPLICATION_JSON = MediaType.parse("application/json");
+
+    private final AtomicReference<OkHttpClient> httpClientReference = new AtomicReference<>();
+    private final C2ClientConfig clientConfig;
+    private final C2Serializer serializer;
+
+    public C2HttpClient(C2ClientConfig clientConfig, C2Serializer serializer) {
+        super();
+        this.clientConfig = clientConfig;
+        this.serializer = serializer;
+        final OkHttpClient.Builder okHttpClientBuilder = new OkHttpClient.Builder();
+
+        // Set whether to follow redirects
+        okHttpClientBuilder.followRedirects(true);
+
+        // check if the ssl path is set and add the factory if so
+        if (StringUtils.isNotBlank(clientConfig.getKeystoreFilename())) {
+            try {
+                setSslSocketFactory(okHttpClientBuilder);
+            } catch (Exception e) {
+                throw new IllegalStateException(e);
+            }
+        }
+
+        httpClientReference.set(okHttpClientBuilder.build());
+    }
+
+    @Override
+    public Optional<C2HeartbeatResponse> publishHeartbeat(C2Heartbeat heartbeat) {
+        return serializer.serialize(heartbeat).flatMap(this::sendHeartbeat);
+    }
+
+    private Optional<C2HeartbeatResponse> sendHeartbeat(String heartbeat) {
+        logger.debug("Sending heartbeat to {}", clientConfig.getC2Url());
+
+        Optional<C2HeartbeatResponse> c2HeartbeatResponse = Optional.empty();
+        Request request = new Request.Builder()
+            .post(RequestBody.create(heartbeat, MEDIA_TYPE_APPLICATION_JSON))
+            .url(clientConfig.getC2Url())
+            .build();
+
+        try {
+            Response heartbeatResponse = httpClientReference.get().newCall(request).execute();
+            c2HeartbeatResponse = getResponseBody(heartbeatResponse).flatMap(response -> serializer.deserialize(response, C2HeartbeatResponse.class));
+        } catch (IOException ce) {
+            logger.error("Error while sending heartbeat", ce);
+        }
+
+        return c2HeartbeatResponse;
+    }
+
+    private Optional<String> getResponseBody(Response response) {
+        String responseBody = null;
+
+        try {
+            responseBody = response.body().string();
+            logger.debug("Received response (Status={}) {}", response.code(), responseBody);

Review Comment:
   Added HttpLoggingInterceptor so additional information with context can be logged for each request and response. Thanks for the idea it looks better now.



##########
c2/c2-client-bundle/c2-client-http/src/main/java/org/apache/nifi/c2/client/http/C2HttpClient.java:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.nifi.c2.client.http;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.security.KeyStore;
+import java.security.NoSuchAlgorithmException;
+import java.util.Optional;
+import java.util.concurrent.atomic.AtomicReference;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLSocketFactory;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+import okhttp3.MediaType;
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.RequestBody;
+import okhttp3.Response;
+import okhttp3.ResponseBody;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.c2.client.C2ClientConfig;
+import org.apache.nifi.c2.client.api.C2Client;
+import org.apache.nifi.c2.client.api.C2Serializer;
+import org.apache.nifi.c2.protocol.api.C2Heartbeat;
+import org.apache.nifi.c2.protocol.api.C2HeartbeatResponse;
+import org.apache.nifi.c2.protocol.api.C2OperationAck;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class C2HttpClient implements C2Client {
+
+    private static final Logger logger = LoggerFactory.getLogger(C2HttpClient.class);
+    private static final MediaType MEDIA_TYPE_APPLICATION_JSON = MediaType.parse("application/json");
+
+    private final AtomicReference<OkHttpClient> httpClientReference = new AtomicReference<>();
+    private final C2ClientConfig clientConfig;
+    private final C2Serializer serializer;
+
+    public C2HttpClient(C2ClientConfig clientConfig, C2Serializer serializer) {
+        super();
+        this.clientConfig = clientConfig;
+        this.serializer = serializer;
+        final OkHttpClient.Builder okHttpClientBuilder = new OkHttpClient.Builder();
+
+        // Set whether to follow redirects
+        okHttpClientBuilder.followRedirects(true);
+
+        // check if the ssl path is set and add the factory if so
+        if (StringUtils.isNotBlank(clientConfig.getKeystoreFilename())) {
+            try {
+                setSslSocketFactory(okHttpClientBuilder);
+            } catch (Exception e) {
+                throw new IllegalStateException(e);
+            }
+        }
+
+        httpClientReference.set(okHttpClientBuilder.build());
+    }
+
+    @Override
+    public Optional<C2HeartbeatResponse> publishHeartbeat(C2Heartbeat heartbeat) {
+        return serializer.serialize(heartbeat).flatMap(this::sendHeartbeat);
+    }
+
+    private Optional<C2HeartbeatResponse> sendHeartbeat(String heartbeat) {
+        logger.debug("Sending heartbeat to {}", clientConfig.getC2Url());
+
+        Optional<C2HeartbeatResponse> c2HeartbeatResponse = Optional.empty();
+        Request request = new Request.Builder()
+            .post(RequestBody.create(heartbeat, MEDIA_TYPE_APPLICATION_JSON))
+            .url(clientConfig.getC2Url())
+            .build();
+
+        try {
+            Response heartbeatResponse = httpClientReference.get().newCall(request).execute();
+            c2HeartbeatResponse = getResponseBody(heartbeatResponse).flatMap(response -> serializer.deserialize(response, C2HeartbeatResponse.class));
+        } catch (IOException ce) {
+            logger.error("Error while sending heartbeat", ce);
+        }
+
+        return c2HeartbeatResponse;
+    }
+
+    private Optional<String> getResponseBody(Response response) {
+        String responseBody = null;
+
+        try {
+            responseBody = response.body().string();
+            logger.debug("Received response (Status={}) {}", response.code(), responseBody);
+        } catch (IOException e) {
+            logger.error("Could not get response body: ", e);
+        }
+
+        return Optional.ofNullable(responseBody);
+    }
+
+    private void setSslSocketFactory(OkHttpClient.Builder okHttpClientBuilder) throws Exception {
+        final String keystoreLocation = clientConfig.getKeystoreFilename();
+        final String keystoreType = clientConfig.getKeystoreType().getType();
+        final String keystorePass = clientConfig.getKeystorePass();
+
+        assertKeystorePropertiesSet(keystoreLocation, keystorePass, keystoreType);
+
+        // prepare the keystore
+        final KeyStore keyStore = KeyStore.getInstance(keystoreType);
+
+        try (FileInputStream keyStoreStream = new FileInputStream(keystoreLocation)) {
+            keyStore.load(keyStoreStream, keystorePass.toCharArray());
+        }
+
+        final KeyManagerFactory keyManagerFactory = KeyManagerFactory.getInstance(KeyManagerFactory.getDefaultAlgorithm());
+        keyManagerFactory.init(keyStore, keystorePass.toCharArray());
+
+        // load truststore
+        final String truststoreLocation = clientConfig.getTruststoreFilename();
+        final String truststorePass = clientConfig.getTruststorePass();
+        final String truststoreType = clientConfig.getTruststoreType().getType();
+        assertTruststorePropertiesSet(truststoreLocation, truststorePass, truststoreType);
+
+        KeyStore truststore = KeyStore.getInstance(truststoreType);
+        final TrustManagerFactory trustManagerFactory = TrustManagerFactory.getInstance("X509");
+        truststore.load(new FileInputStream(truststoreLocation), truststorePass.toCharArray());
+        trustManagerFactory.init(truststore);
+
+        final X509TrustManager x509TrustManager;
+        TrustManager[] trustManagers = trustManagerFactory.getTrustManagers();
+        if (trustManagers[0] != null) {
+            x509TrustManager = (X509TrustManager) trustManagers[0];
+        } else {
+            throw new IllegalStateException("List of trust managers is null");
+        }
+
+        SSLContext tempSslContext;
+        try {
+            tempSslContext = SSLContext.getInstance("TLS");
+        } catch (NoSuchAlgorithmException e) {
+            logger.warn("Unable to use 'TLS' for the PullHttpChangeIngestor due to NoSuchAlgorithmException. Will attempt to use the default algorithm.", e);

Review Comment:
   Applied



##########
c2/c2-client-bundle/c2-client-http/src/main/java/org/apache/nifi/c2/client/http/C2HttpClient.java:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.nifi.c2.client.http;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.security.KeyStore;
+import java.security.NoSuchAlgorithmException;
+import java.util.Optional;
+import java.util.concurrent.atomic.AtomicReference;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLSocketFactory;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+import okhttp3.MediaType;
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.RequestBody;
+import okhttp3.Response;
+import okhttp3.ResponseBody;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.c2.client.C2ClientConfig;
+import org.apache.nifi.c2.client.api.C2Client;
+import org.apache.nifi.c2.client.api.C2Serializer;
+import org.apache.nifi.c2.protocol.api.C2Heartbeat;
+import org.apache.nifi.c2.protocol.api.C2HeartbeatResponse;
+import org.apache.nifi.c2.protocol.api.C2OperationAck;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class C2HttpClient implements C2Client {
+
+    private static final Logger logger = LoggerFactory.getLogger(C2HttpClient.class);
+    private static final MediaType MEDIA_TYPE_APPLICATION_JSON = MediaType.parse("application/json");
+
+    private final AtomicReference<OkHttpClient> httpClientReference = new AtomicReference<>();
+    private final C2ClientConfig clientConfig;
+    private final C2Serializer serializer;
+
+    public C2HttpClient(C2ClientConfig clientConfig, C2Serializer serializer) {
+        super();
+        this.clientConfig = clientConfig;
+        this.serializer = serializer;
+        final OkHttpClient.Builder okHttpClientBuilder = new OkHttpClient.Builder();
+
+        // Set whether to follow redirects
+        okHttpClientBuilder.followRedirects(true);
+
+        // check if the ssl path is set and add the factory if so
+        if (StringUtils.isNotBlank(clientConfig.getKeystoreFilename())) {
+            try {
+                setSslSocketFactory(okHttpClientBuilder);
+            } catch (Exception e) {
+                throw new IllegalStateException(e);
+            }
+        }
+
+        httpClientReference.set(okHttpClientBuilder.build());
+    }
+
+    @Override
+    public Optional<C2HeartbeatResponse> publishHeartbeat(C2Heartbeat heartbeat) {
+        return serializer.serialize(heartbeat).flatMap(this::sendHeartbeat);
+    }
+
+    private Optional<C2HeartbeatResponse> sendHeartbeat(String heartbeat) {
+        logger.debug("Sending heartbeat to {}", clientConfig.getC2Url());
+
+        Optional<C2HeartbeatResponse> c2HeartbeatResponse = Optional.empty();
+        Request request = new Request.Builder()
+            .post(RequestBody.create(heartbeat, MEDIA_TYPE_APPLICATION_JSON))
+            .url(clientConfig.getC2Url())
+            .build();
+
+        try {
+            Response heartbeatResponse = httpClientReference.get().newCall(request).execute();
+            c2HeartbeatResponse = getResponseBody(heartbeatResponse).flatMap(response -> serializer.deserialize(response, C2HeartbeatResponse.class));
+        } catch (IOException ce) {
+            logger.error("Error while sending heartbeat", ce);
+        }
+
+        return c2HeartbeatResponse;
+    }
+
+    private Optional<String> getResponseBody(Response response) {
+        String responseBody = null;
+
+        try {
+            responseBody = response.body().string();
+            logger.debug("Received response (Status={}) {}", response.code(), responseBody);
+        } catch (IOException e) {
+            logger.error("Could not get response body: ", e);
+        }
+
+        return Optional.ofNullable(responseBody);
+    }
+
+    private void setSslSocketFactory(OkHttpClient.Builder okHttpClientBuilder) throws Exception {
+        final String keystoreLocation = clientConfig.getKeystoreFilename();
+        final String keystoreType = clientConfig.getKeystoreType().getType();
+        final String keystorePass = clientConfig.getKeystorePass();
+
+        assertKeystorePropertiesSet(keystoreLocation, keystorePass, keystoreType);
+
+        // prepare the keystore
+        final KeyStore keyStore = KeyStore.getInstance(keystoreType);
+
+        try (FileInputStream keyStoreStream = new FileInputStream(keystoreLocation)) {
+            keyStore.load(keyStoreStream, keystorePass.toCharArray());
+        }
+
+        final KeyManagerFactory keyManagerFactory = KeyManagerFactory.getInstance(KeyManagerFactory.getDefaultAlgorithm());
+        keyManagerFactory.init(keyStore, keystorePass.toCharArray());
+
+        // load truststore
+        final String truststoreLocation = clientConfig.getTruststoreFilename();
+        final String truststorePass = clientConfig.getTruststorePass();
+        final String truststoreType = clientConfig.getTruststoreType().getType();
+        assertTruststorePropertiesSet(truststoreLocation, truststorePass, truststoreType);
+
+        KeyStore truststore = KeyStore.getInstance(truststoreType);
+        final TrustManagerFactory trustManagerFactory = TrustManagerFactory.getInstance("X509");
+        truststore.load(new FileInputStream(truststoreLocation), truststorePass.toCharArray());
+        trustManagerFactory.init(truststore);
+
+        final X509TrustManager x509TrustManager;
+        TrustManager[] trustManagers = trustManagerFactory.getTrustManagers();
+        if (trustManagers[0] != null) {
+            x509TrustManager = (X509TrustManager) trustManagers[0];
+        } else {
+            throw new IllegalStateException("List of trust managers is null");
+        }
+
+        SSLContext tempSslContext;
+        try {
+            tempSslContext = SSLContext.getInstance("TLS");
+        } catch (NoSuchAlgorithmException e) {
+            logger.warn("Unable to use 'TLS' for the PullHttpChangeIngestor due to NoSuchAlgorithmException. Will attempt to use the default algorithm.", e);
+            tempSslContext = SSLContext.getDefault();
+        }
+
+        final SSLContext sslContext = tempSslContext;
+        sslContext.init(keyManagerFactory.getKeyManagers(), trustManagerFactory.getTrustManagers(), null);
+
+        final SSLSocketFactory sslSocketFactory = sslContext.getSocketFactory();
+        okHttpClientBuilder.sslSocketFactory(sslSocketFactory, x509TrustManager);
+    }
+
+    private void assertKeystorePropertiesSet(String location, String password, String type) {
+        if (location == null || location.isEmpty()) {
+            throw new IllegalArgumentException(clientConfig.getKeystoreFilename() + " is null or is empty");
+        }
+
+        if (password == null || password.isEmpty()) {
+            throw new IllegalArgumentException("The client's keystore filename is set but its password is not (or is empty). If the location is set, the password must also be.");
+        }
+
+        if (type == null || type.isEmpty()) {
+            throw new IllegalArgumentException("The client's keystore filename is set but its type is not (or is empty). If the location is set, the type must also be.");
+        }
+    }
+
+    private void assertTruststorePropertiesSet(String location, String password, String type) {
+        if (location == null || location.isEmpty()) {
+            throw new IllegalArgumentException("The client's truststore filename is not set or is empty");
+        }
+
+        if (password == null || password.isEmpty()) {
+            throw new IllegalArgumentException("The client's truststore filename is set but its password is not (or is empty). If the location is set, the password must also be.");
+        }
+
+        if (type == null || type.isEmpty()) {
+            throw new IllegalArgumentException("The client's truststore filename is set but its type is not (or is empty). If the location is set, the type must also be.");
+        }
+    }
+
+    @Override
+    public ByteBuffer retrieveUpdateContent(String flowUpdateUrl) {
+        final Request.Builder requestBuilder = new Request.Builder()
+                .get()
+                .url(flowUpdateUrl);
+        final Request request = requestBuilder.build();
+
+        ResponseBody body;
+        try (final Response response = httpClientReference.get().newCall(request).execute()) {
+            logger.debug("Response received: {}", response);
+
+            int code = response.code();
+            if (code >= 400) {
+                throw new IOException("Got response code " + code + " while trying to pull configuration: " + response.body().string());
+            }
+
+            body = response.body();
+
+            if (body == null) {
+                logger.warn("No body returned when pulling a new configuration");
+                return null;
+            }
+
+            final ByteBuffer bodyByteBuffer = ByteBuffer.wrap(body.bytes());
+            return bodyByteBuffer;
+
+        } catch (Exception e) {
+            logger.warn("Hit an exception while trying to pull", e);
+            return null;
+        }
+    }
+
+    @Override
+    public void acknowledgeOperation(C2OperationAck operationAck) {
+        logger.info("Performing acknowledgement request to {} for operation {}", clientConfig.getC2AckUrl(), operationAck.getOperationId());
+        final ObjectMapper jacksonObjectMapper = new ObjectMapper();
+        jacksonObjectMapper.setSerializationInclusion(JsonInclude.Include.NON_NULL);
+        try {
+            final String operationAckBody = jacksonObjectMapper.writeValueAsString(operationAck);
+
+            final RequestBody requestBody = RequestBody.create(operationAckBody, MediaType.parse("application/json"));
+            final Request.Builder requestBuilder = new Request.Builder()
+                    .post(requestBody)
+                    .url(clientConfig.getC2AckUrl());
+            final Response heartbeatResponse = httpClientReference.get().newCall(requestBuilder.build()).execute();

Review Comment:
   Applied



##########
c2/c2-client-bundle/c2-client-service/src/main/java/org/apache/nifi/c2/client/service/C2HeartbeatFactory.java:
##########
@@ -0,0 +1,248 @@
+/*
+ * 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.nifi.c2.client.service;
+
+import static org.apache.commons.lang3.StringUtils.isNotBlank;
+
+import java.io.File;
+import java.lang.management.ManagementFactory;
+import java.lang.management.OperatingSystemMXBean;
+import java.net.InetAddress;
+import java.net.NetworkInterface;
+import java.util.Enumeration;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.nifi.c2.client.C2ClientConfig;
+import org.apache.nifi.c2.client.PersistentUuidGenerator;
+import org.apache.nifi.c2.client.service.model.RuntimeInfoWrapper;
+import org.apache.nifi.c2.protocol.api.AgentInfo;
+import org.apache.nifi.c2.protocol.api.AgentRepositories;
+import org.apache.nifi.c2.protocol.api.AgentStatus;
+import org.apache.nifi.c2.protocol.api.C2Heartbeat;
+import org.apache.nifi.c2.protocol.api.DeviceInfo;
+import org.apache.nifi.c2.protocol.api.FlowInfo;
+import org.apache.nifi.c2.protocol.api.FlowQueueStatus;
+import org.apache.nifi.c2.protocol.api.NetworkInfo;
+import org.apache.nifi.c2.protocol.api.SystemInfo;
+import org.apache.nifi.c2.protocol.component.api.RuntimeManifest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class C2HeartbeatFactory {
+
+    private static final Logger logger = LoggerFactory.getLogger(C2HeartbeatFactory.class);
+
+    private static final String AGENT_IDENTIFIER_FILENAME = "agent-identifier";
+    private static final String DEVICE_IDENTIFIER_FILENAME = "device-identifier";
+
+    private final C2ClientConfig clientConfig;
+    private final FlowIdHolder flowIdHolder;
+
+    private String agentId;
+    private String deviceId;
+    private File confDirectory;
+
+    public C2HeartbeatFactory(C2ClientConfig clientConfig, FlowIdHolder flowIdHolder) {
+        this.clientConfig = clientConfig;
+        this.flowIdHolder = flowIdHolder;
+    }
+
+    public C2Heartbeat create(RuntimeInfoWrapper runtimeInfoWrapper) {
+        C2Heartbeat heartbeat = new C2Heartbeat();
+
+        heartbeat.setAgentInfo(getAgentInfo(runtimeInfoWrapper.getAgentRepositories(), runtimeInfoWrapper.getManifest()));
+        heartbeat.setDeviceInfo(generateDeviceInfo());
+        heartbeat.setFlowInfo(getFlowInfo(runtimeInfoWrapper.getQueueStatus()));
+        heartbeat.setCreated(System.currentTimeMillis());
+
+        return heartbeat;
+    }
+
+    //////
+    // FlowInfo
+    //////
+
+    private FlowInfo getFlowInfo(Map<String, FlowQueueStatus> queueStatus) {
+        FlowInfo flowInfo = new FlowInfo();
+        flowInfo.setQueues(queueStatus);
+        Optional.ofNullable(flowIdHolder.getFlowId()).ifPresent(flowInfo::setFlowId);
+        return flowInfo;
+    }
+
+    //////
+    // AgentInfo
+    //////
+
+    private AgentInfo getAgentInfo(AgentRepositories repos, RuntimeManifest manifest) {
+        AgentInfo agentInfo = new AgentInfo();
+        agentInfo.setAgentClass(clientConfig.getAgentClass());
+        agentInfo.setIdentifier(getAgentId());
+
+        AgentStatus agentStatus = new AgentStatus();
+        // TODO: Look into this, it doesn't look right
+        agentStatus.setUptime(System.currentTimeMillis());

Review Comment:
   Corrected



##########
c2/c2-client-bundle/c2-client-service/src/main/java/org/apache/nifi/c2/client/service/FlowIdHolder.java:
##########
@@ -0,0 +1,85 @@
+/*
+ * 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.nifi.c2.client.service;
+
+import static java.util.Collections.singletonList;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.List;
+import org.apache.nifi.util.file.FileUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class FlowIdHolder {
+    private static final Logger LOGGER = LoggerFactory.getLogger(FlowIdHolder.class);
+    private static final String FLOW_IDENTIFIER_FILENAME = "flow-identifier";
+
+    private volatile String flowId;
+    private final String configDirectoryName;
+
+    public FlowIdHolder(String configDirectoryName) {
+        this.configDirectoryName = configDirectoryName;
+        this.flowId = readFlowId();
+    }
+
+    public String getFlowId() {
+        return flowId;
+    }
+
+    public void setFlowId(String flowId) {
+        this.flowId = flowId;
+        persistFlowId(flowId);
+    }
+
+    private void persistFlowId(String flowId) {
+        File flowIdFile = new File(configDirectoryName, FLOW_IDENTIFIER_FILENAME);
+        try {
+            FileUtils.ensureDirectoryExistAndCanAccess(flowIdFile.getParentFile());
+            saveFlowId(flowIdFile, flowId);
+        } catch (IOException e) {
+            LOGGER.error("Failed to save flow information due to: {}", e.getMessage());
+        }
+    }
+
+    private void saveFlowId(File flowUpdateInfoFile, String flowId) {
+        try {
+            Files.write(flowUpdateInfoFile.toPath(), singletonList(flowId));
+        } catch (IOException e) {
+            LOGGER.error("Failed to persist flow data", e);

Review Comment:
   applied



##########
c2/c2-client-bundle/c2-client-api/src/main/java/org/apache/nifi/c2/client/api/C2Properties.java:
##########
@@ -0,0 +1,141 @@
+/*
+ * 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.nifi.c2.client.api;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Properties;
+import java.util.concurrent.TimeUnit;
+
+public class C2Properties {
+
+    private static final Logger logger = LoggerFactory.getLogger(C2Properties.class);
+
+    public static final String NIFI_PREFIX = "nifi.";
+
+    public static final String C2_PROPERTY_BASE = NIFI_PREFIX + "c2.";
+    public static final String C2_ENABLE_KEY = C2_PROPERTY_BASE + "enable";
+    public static final String C2_AGENT_PROTOCOL_KEY = C2_PROPERTY_BASE + "agent.protocol.class";
+    public static final String C2_REST_URL_KEY = C2_PROPERTY_BASE + "rest.url";
+    public static final String C2_REST_URL_ACK_KEY = C2_PROPERTY_BASE + "rest.url.ack";
+    public static final String C2_AGENT_HEARTBEAT_PERIOD_KEY = C2_PROPERTY_BASE + "agent.heartbeat.period";
+    public static final String C2_AGENT_CLASS_KEY = C2_PROPERTY_BASE + "agent.class";
+    public static final String C2_AGENT_IDENTIFIER_KEY = C2_PROPERTY_BASE + "agent.identifier";
+
+    public static final String C2_ROOT_CLASS_DEFINITIONS_KEY = C2_PROPERTY_BASE + "root.class.definitions";
+    public static final String C2_METRICS_NAME_KEY = C2_ROOT_CLASS_DEFINITIONS_KEY + ".metrics.name";
+    public static final String C2_METRICS_METRICS_KEY = C2_ROOT_CLASS_DEFINITIONS_KEY + ".metrics.metrics";
+    public static final String C2_METRICS_METRICS_TYPED_METRICS_NAME_KEY = C2_ROOT_CLASS_DEFINITIONS_KEY + ".metrics.metrics.typedmetrics.name";
+    public static final String C2_METRICS_METRICS_QUEUED_METRICS_NAME_KEY = C2_ROOT_CLASS_DEFINITIONS_KEY + ".metrics.metrics.queuemetrics.name";
+    public static final String C2_METRICS_METRICS_QUEUE_METRICS_CLASSES_KEY = C2_ROOT_CLASS_DEFINITIONS_KEY + ".metrics.metrics.queuemetrics.classes";
+    public static final String C2_METRICS_METRICS_TYPED_METRICS_CLASSES_KEY = C2_ROOT_CLASS_DEFINITIONS_KEY + ".metrics.metrics.typedmetrics.classes";
+    public static final String C2_METRICS_METRICS_PROCESSOR_METRICS_NAME_KEY = C2_ROOT_CLASS_DEFINITIONS_KEY + ".metrics.metrics.processorMetrics.name";
+    public static final String C2_METRICS_METRICS_PROCESSOR_METRICS_CLASSES_KEY = C2_ROOT_CLASS_DEFINITIONS_KEY + ".metrics.metrics.processorMetrics.classes";
+
+    /* REST Heartbeat Reporter Security Properties */
+    private static final String C2_REST_SECURITY_BASE_KEY = NIFI_PREFIX + "c2.security";
+    public static final String TRUSTSTORE_LOCATION_KEY = C2_REST_SECURITY_BASE_KEY + ".truststore.location";
+    public static final String TRUSTSTORE_PASSWORD_KEY = C2_REST_SECURITY_BASE_KEY + ".truststore.password";
+    public static final String TRUSTSTORE_TYPE_KEY = C2_REST_SECURITY_BASE_KEY + ".truststore.type";
+    public static final String KEYSTORE_LOCATION_KEY = C2_REST_SECURITY_BASE_KEY + ".keystore.location";
+    public static final String KEYSTORE_PASSWORD_KEY = C2_REST_SECURITY_BASE_KEY + ".keystore.password";
+    public static final String KEYSTORE_TYPE_KEY = C2_REST_SECURITY_BASE_KEY + ".keystore.type";
+    public static final String NEED_CLIENT_AUTH_KEY = C2_REST_SECURITY_BASE_KEY + ".need.client.auth";
+
+    // Defaults
+    // Heartbeat period of 1 second
+    public static final long C2_AGENT_DEFAULT_HEARTBEAT_PERIOD = TimeUnit.SECONDS.toMillis(1);

Review Comment:
   As you point out in one of your other comments this is a duplicated class (I suppose came back accidentally at a rebase) so I deleted it now.
   Reflecting on your comment: As you said there is already a lot of cleanup in the PR. We already identified other cleanup / refactor / hardening opportunities which we will cover after this PR gets the green light (didn't wanted to further extend this one). Would you be ok if we would look into this enum based approach there? I added to my list.



##########
c2/c2-client-bundle/c2-client-service/src/main/java/org/apache/nifi/c2/client/service/operation/UpdateConfigurationOperationHandler.java:
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.nifi.c2.client.service.operation;
+
+import static org.apache.commons.lang3.StringUtils.EMPTY;
+import static org.apache.nifi.c2.protocol.api.OperandType.CONFIGURATION;
+import static org.apache.nifi.c2.protocol.api.OperationType.UPDATE;
+
+import java.net.URI;
+import java.nio.ByteBuffer;
+import java.util.Optional;
+import java.util.function.Function;
+import org.apache.nifi.c2.client.api.C2Client;
+import org.apache.nifi.c2.client.service.FlowIdHolder;
+import org.apache.nifi.c2.protocol.api.C2Operation;
+import org.apache.nifi.c2.protocol.api.C2OperationAck;
+import org.apache.nifi.c2.protocol.api.C2OperationState;
+import org.apache.nifi.c2.protocol.api.OperandType;
+import org.apache.nifi.c2.protocol.api.OperationType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class UpdateConfigurationOperationHandler implements C2OperationHandler {
+
+    private static final Logger logger = LoggerFactory.getLogger(UpdateConfigurationOperationHandler.class);
+
+    private static final String LOCATION = "location";
+
+    private final C2Client client;
+    private final Function<ByteBuffer, Boolean> updateFlow;
+    private final FlowIdHolder flowIdHolder;
+
+    public UpdateConfigurationOperationHandler(C2Client client, FlowIdHolder flowIdHolder, Function<ByteBuffer, Boolean> updateFlow) {
+        this.client = client;
+        this.updateFlow = updateFlow;
+        this.flowIdHolder = flowIdHolder;
+    }
+
+    @Override
+    public OperationType getOperationType() {
+        return UPDATE;
+    }
+
+    @Override
+    public OperandType getOperandType() {
+        return CONFIGURATION;
+    }
+
+    @Override
+    public C2OperationAck handle(C2Operation operation) {
+        String opIdentifier = Optional.ofNullable(operation.getIdentifier())
+            .orElse(EMPTY);
+        C2OperationAck operationAck = new C2OperationAck();
+        C2OperationState state = new C2OperationState();
+        operationAck.setOperationState(state);
+        operationAck.setOperationId(opIdentifier);
+
+        String updateLocation = Optional.ofNullable(operation.getArgs())
+            .map(map -> map.get(LOCATION))
+            .orElse(EMPTY);
+
+        String newFlowId = parseFlowId(updateLocation);
+        if (flowIdHolder.getFlowId() == null || !flowIdHolder.getFlowId().equals(newFlowId)) {
+            logger.info("Will perform flow update from {} for operation #{}. Previous flow id was {}, replacing with new id {}", updateLocation, opIdentifier,
+                flowIdHolder.getFlowId() == null ? "not set" : flowIdHolder.getFlowId(), newFlowId);
+        } else {
+            logger.info("Flow is current, no update is necessary...");

Review Comment:
   If you are ok with it I would leave it as is because flow update is a very rare activity while the outcome of the operation (whatever that may be) is really informative and descriptive in the logs.



##########
c2/c2-client-bundle/c2-client-base/pom.xml:
##########
@@ -0,0 +1,71 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+ -->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <artifactId>c2-client-bundle</artifactId>
+        <groupId>org.apache.nifi</groupId>
+        <version>1.17.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>c2-client-base</artifactId>
+    <packaging>jar</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>c2-client-api</artifactId>
+            <version>1.17.0-SNAPSHOT</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>c2-protocol-api</artifactId>
+            <version>1.17.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-security-utils-api</artifactId>
+            <version>1.17.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-security-utils</artifactId>
+            <version>1.17.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-utils</artifactId>
+            <version>1.17.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.commons</groupId>
+            <artifactId>commons-lang3</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>com.fasterxml.jackson.core</groupId>
+            <artifactId>jackson-databind</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>com.fasterxml.jackson.core</groupId>
+            <artifactId>jackson-annotations</artifactId>
+            <version>2.9.0</version>

Review Comment:
   removed, thanks



##########
c2/c2-client-bundle/c2-client-base/pom.xml:
##########
@@ -0,0 +1,71 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+ -->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <artifactId>c2-client-bundle</artifactId>
+        <groupId>org.apache.nifi</groupId>
+        <version>1.17.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>c2-client-base</artifactId>
+    <packaging>jar</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>c2-client-api</artifactId>
+            <version>1.17.0-SNAPSHOT</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>c2-protocol-api</artifactId>
+            <version>1.17.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-security-utils-api</artifactId>
+            <version>1.17.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-security-utils</artifactId>
+            <version>1.17.0-SNAPSHOT</version>
+        </dependency>

Review Comment:
   **nifi-utils:** was used to check directory exists and can be accessed. Moved those few lines of code here removed the dependency
   **nifi-security-utils:** was not needed at all so it was removed
   **nifi-security-utils-api:** the KeyStoreType enum was used. I followed the whole flow around C2NiFiClientService, httpClient and clientConfig and it looks like we don't need to rely on it. So remved and with that removed all nifi dependencies form here which is a good thing I suppose :)
   



##########
c2/c2-client-bundle/c2-client-base/src/main/java/org/apache/nifi/c2/serializer/C2JacksonSerializer.java:
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.nifi.c2.serializer;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.util.Optional;
+import org.apache.nifi.c2.client.api.C2Serializer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class C2JacksonSerializer implements C2Serializer {
+
+    private static final Logger logger = LoggerFactory.getLogger(C2JacksonSerializer.class);
+
+    private final ObjectMapper objectMapper;
+
+    public C2JacksonSerializer() {
+        objectMapper = new ObjectMapper();
+        objectMapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
+        objectMapper.setSerializationInclusion(JsonInclude.Include.NON_NULL);
+        objectMapper.enable(DeserializationFeature.READ_ENUMS_USING_TO_STRING);
+    }
+
+    @Override
+    public <T> Optional<String> serialize(T object) {
+        if (object == null) {
+            logger.trace("C2 Object was null. Nothing to serialize. Returning empty.");
+            return Optional.empty();
+        }
+
+        String contentString = null;
+        try {
+            contentString = objectMapper.writeValueAsString(object);
+            logger.trace("Serialized C2 Object: {}", contentString);
+        } catch (JsonProcessingException e) {
+            logger.error("Can't serialise C2 Object: ", e);

Review Comment:
   applied



##########
c2/c2-client-bundle/c2-client-base/src/main/java/org/apache/nifi/c2/serializer/C2JacksonSerializer.java:
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.nifi.c2.serializer;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.util.Optional;
+import org.apache.nifi.c2.client.api.C2Serializer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class C2JacksonSerializer implements C2Serializer {
+
+    private static final Logger logger = LoggerFactory.getLogger(C2JacksonSerializer.class);
+
+    private final ObjectMapper objectMapper;
+
+    public C2JacksonSerializer() {
+        objectMapper = new ObjectMapper();
+        objectMapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
+        objectMapper.setSerializationInclusion(JsonInclude.Include.NON_NULL);
+        objectMapper.enable(DeserializationFeature.READ_ENUMS_USING_TO_STRING);
+    }
+
+    @Override
+    public <T> Optional<String> serialize(T object) {
+        if (object == null) {
+            logger.trace("C2 Object was null. Nothing to serialize. Returning empty.");
+            return Optional.empty();
+        }
+
+        String contentString = null;
+        try {
+            contentString = objectMapper.writeValueAsString(object);
+            logger.trace("Serialized C2 Object: {}", contentString);

Review Comment:
   Removed



##########
c2/c2-client-bundle/c2-client-http/src/main/java/org/apache/nifi/c2/client/http/C2HttpClient.java:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.nifi.c2.client.http;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.security.KeyStore;
+import java.security.NoSuchAlgorithmException;
+import java.util.Optional;
+import java.util.concurrent.atomic.AtomicReference;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLSocketFactory;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+import okhttp3.MediaType;
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.RequestBody;
+import okhttp3.Response;
+import okhttp3.ResponseBody;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.c2.client.C2ClientConfig;
+import org.apache.nifi.c2.client.api.C2Client;
+import org.apache.nifi.c2.client.api.C2Serializer;
+import org.apache.nifi.c2.protocol.api.C2Heartbeat;
+import org.apache.nifi.c2.protocol.api.C2HeartbeatResponse;
+import org.apache.nifi.c2.protocol.api.C2OperationAck;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class C2HttpClient implements C2Client {
+
+    private static final Logger logger = LoggerFactory.getLogger(C2HttpClient.class);
+    private static final MediaType MEDIA_TYPE_APPLICATION_JSON = MediaType.parse("application/json");
+
+    private final AtomicReference<OkHttpClient> httpClientReference = new AtomicReference<>();
+    private final C2ClientConfig clientConfig;
+    private final C2Serializer serializer;
+
+    public C2HttpClient(C2ClientConfig clientConfig, C2Serializer serializer) {
+        super();
+        this.clientConfig = clientConfig;
+        this.serializer = serializer;
+        final OkHttpClient.Builder okHttpClientBuilder = new OkHttpClient.Builder();
+
+        // Set whether to follow redirects
+        okHttpClientBuilder.followRedirects(true);
+
+        // check if the ssl path is set and add the factory if so
+        if (StringUtils.isNotBlank(clientConfig.getKeystoreFilename())) {
+            try {
+                setSslSocketFactory(okHttpClientBuilder);
+            } catch (Exception e) {
+                throw new IllegalStateException(e);

Review Comment:
   applied



##########
c2/c2-client-bundle/c2-client-base/src/main/java/org/apache/nifi/c2/client/PersistentUuidGenerator.java:
##########
@@ -0,0 +1,76 @@
+/*
+ * 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.nifi.c2.client;
+
+import org.apache.nifi.c2.client.api.IdGenerator;
+import org.apache.nifi.util.file.FileUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.Arrays;
+import java.util.List;
+import java.util.UUID;
+
+public class PersistentUuidGenerator implements IdGenerator {
+
+    private static final Logger logger = LoggerFactory.getLogger(PersistentUuidGenerator.class);
+
+    private final File persistenceLocation;
+
+    public PersistentUuidGenerator(final File persistenceLocation) {
+        this.persistenceLocation = persistenceLocation;
+    }
+
+    @Override
+    public String generate() {
+        if (this.persistenceLocation.exists()) {
+            return readFile();
+        } else {
+            return makeFile();
+        }
+    }
+
+    private String readFile() {
+        try {
+            final List<String> fileLines = Files.readAllLines(persistenceLocation.toPath());
+            if (fileLines.size() != 1) {
+                throw new IllegalStateException(String.format("The file %s for the persisted identifier has the incorrect format.", persistenceLocation));
+            }
+            final String uuid = fileLines.get(0);
+            return uuid;
+        } catch (IOException e) {
+            throw new IllegalStateException(String.format("Could not read file %s for persisted identifier.", persistenceLocation), e);
+
+        }
+    }
+
+    private String makeFile() {
+        try {
+            final File parentDirectory = persistenceLocation.getParentFile();
+            FileUtils.ensureDirectoryExistAndCanAccess(parentDirectory);
+            final String uuid = UUID.randomUUID().toString();
+            Files.write(persistenceLocation.toPath(), Arrays.asList(uuid));
+            logger.info("Created identifier {} at {}.", uuid, persistenceLocation);

Review Comment:
   applied



##########
c2/c2-client-bundle/c2-client-http/src/main/java/org/apache/nifi/c2/client/http/C2HttpClient.java:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.nifi.c2.client.http;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.security.KeyStore;
+import java.security.NoSuchAlgorithmException;
+import java.util.Optional;
+import java.util.concurrent.atomic.AtomicReference;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLSocketFactory;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+import okhttp3.MediaType;
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.RequestBody;
+import okhttp3.Response;
+import okhttp3.ResponseBody;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.c2.client.C2ClientConfig;
+import org.apache.nifi.c2.client.api.C2Client;
+import org.apache.nifi.c2.client.api.C2Serializer;
+import org.apache.nifi.c2.protocol.api.C2Heartbeat;
+import org.apache.nifi.c2.protocol.api.C2HeartbeatResponse;
+import org.apache.nifi.c2.protocol.api.C2OperationAck;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class C2HttpClient implements C2Client {
+
+    private static final Logger logger = LoggerFactory.getLogger(C2HttpClient.class);
+    private static final MediaType MEDIA_TYPE_APPLICATION_JSON = MediaType.parse("application/json");
+
+    private final AtomicReference<OkHttpClient> httpClientReference = new AtomicReference<>();
+    private final C2ClientConfig clientConfig;
+    private final C2Serializer serializer;
+
+    public C2HttpClient(C2ClientConfig clientConfig, C2Serializer serializer) {
+        super();
+        this.clientConfig = clientConfig;
+        this.serializer = serializer;
+        final OkHttpClient.Builder okHttpClientBuilder = new OkHttpClient.Builder();
+
+        // Set whether to follow redirects
+        okHttpClientBuilder.followRedirects(true);
+
+        // check if the ssl path is set and add the factory if so
+        if (StringUtils.isNotBlank(clientConfig.getKeystoreFilename())) {
+            try {
+                setSslSocketFactory(okHttpClientBuilder);
+            } catch (Exception e) {
+                throw new IllegalStateException(e);
+            }
+        }
+
+        httpClientReference.set(okHttpClientBuilder.build());
+    }
+
+    @Override
+    public Optional<C2HeartbeatResponse> publishHeartbeat(C2Heartbeat heartbeat) {
+        return serializer.serialize(heartbeat).flatMap(this::sendHeartbeat);
+    }
+
+    private Optional<C2HeartbeatResponse> sendHeartbeat(String heartbeat) {
+        logger.debug("Sending heartbeat to {}", clientConfig.getC2Url());
+
+        Optional<C2HeartbeatResponse> c2HeartbeatResponse = Optional.empty();
+        Request request = new Request.Builder()
+            .post(RequestBody.create(heartbeat, MEDIA_TYPE_APPLICATION_JSON))
+            .url(clientConfig.getC2Url())
+            .build();
+
+        try {
+            Response heartbeatResponse = httpClientReference.get().newCall(request).execute();

Review Comment:
   Good catch, updated.



##########
nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/c2/C2NiFiProperties.java:
##########
@@ -0,0 +1,62 @@
+/*
+ * 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.nifi.c2;
+
+import java.util.concurrent.TimeUnit;
+
+public class C2NiFiProperties {

Review Comment:
   Actually we needed to keep this one and removed the other.



##########
c2/c2-client-bundle/c2-client-http/src/main/java/org/apache/nifi/c2/client/http/C2HttpClient.java:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.nifi.c2.client.http;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.security.KeyStore;
+import java.security.NoSuchAlgorithmException;
+import java.util.Optional;
+import java.util.concurrent.atomic.AtomicReference;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLSocketFactory;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+import okhttp3.MediaType;
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.RequestBody;
+import okhttp3.Response;
+import okhttp3.ResponseBody;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.c2.client.C2ClientConfig;
+import org.apache.nifi.c2.client.api.C2Client;
+import org.apache.nifi.c2.client.api.C2Serializer;
+import org.apache.nifi.c2.protocol.api.C2Heartbeat;
+import org.apache.nifi.c2.protocol.api.C2HeartbeatResponse;
+import org.apache.nifi.c2.protocol.api.C2OperationAck;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class C2HttpClient implements C2Client {
+
+    private static final Logger logger = LoggerFactory.getLogger(C2HttpClient.class);
+    private static final MediaType MEDIA_TYPE_APPLICATION_JSON = MediaType.parse("application/json");
+
+    private final AtomicReference<OkHttpClient> httpClientReference = new AtomicReference<>();
+    private final C2ClientConfig clientConfig;
+    private final C2Serializer serializer;
+
+    public C2HttpClient(C2ClientConfig clientConfig, C2Serializer serializer) {
+        super();
+        this.clientConfig = clientConfig;
+        this.serializer = serializer;
+        final OkHttpClient.Builder okHttpClientBuilder = new OkHttpClient.Builder();

Review Comment:
   Added property and configuration for setting call timeout.



##########
c2/c2-client-bundle/c2-client-http/src/main/java/org/apache/nifi/c2/client/http/C2HttpClient.java:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.nifi.c2.client.http;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.security.KeyStore;
+import java.security.NoSuchAlgorithmException;
+import java.util.Optional;
+import java.util.concurrent.atomic.AtomicReference;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLSocketFactory;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+import okhttp3.MediaType;
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.RequestBody;
+import okhttp3.Response;
+import okhttp3.ResponseBody;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.c2.client.C2ClientConfig;
+import org.apache.nifi.c2.client.api.C2Client;
+import org.apache.nifi.c2.client.api.C2Serializer;
+import org.apache.nifi.c2.protocol.api.C2Heartbeat;
+import org.apache.nifi.c2.protocol.api.C2HeartbeatResponse;
+import org.apache.nifi.c2.protocol.api.C2OperationAck;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class C2HttpClient implements C2Client {
+
+    private static final Logger logger = LoggerFactory.getLogger(C2HttpClient.class);
+    private static final MediaType MEDIA_TYPE_APPLICATION_JSON = MediaType.parse("application/json");
+
+    private final AtomicReference<OkHttpClient> httpClientReference = new AtomicReference<>();
+    private final C2ClientConfig clientConfig;
+    private final C2Serializer serializer;
+
+    public C2HttpClient(C2ClientConfig clientConfig, C2Serializer serializer) {
+        super();
+        this.clientConfig = clientConfig;
+        this.serializer = serializer;
+        final OkHttpClient.Builder okHttpClientBuilder = new OkHttpClient.Builder();
+
+        // Set whether to follow redirects
+        okHttpClientBuilder.followRedirects(true);
+
+        // check if the ssl path is set and add the factory if so
+        if (StringUtils.isNotBlank(clientConfig.getKeystoreFilename())) {
+            try {
+                setSslSocketFactory(okHttpClientBuilder);
+            } catch (Exception e) {
+                throw new IllegalStateException(e);
+            }
+        }
+
+        httpClientReference.set(okHttpClientBuilder.build());
+    }
+
+    @Override
+    public Optional<C2HeartbeatResponse> publishHeartbeat(C2Heartbeat heartbeat) {
+        return serializer.serialize(heartbeat).flatMap(this::sendHeartbeat);
+    }
+
+    private Optional<C2HeartbeatResponse> sendHeartbeat(String heartbeat) {
+        logger.debug("Sending heartbeat to {}", clientConfig.getC2Url());
+
+        Optional<C2HeartbeatResponse> c2HeartbeatResponse = Optional.empty();
+        Request request = new Request.Builder()
+            .post(RequestBody.create(heartbeat, MEDIA_TYPE_APPLICATION_JSON))
+            .url(clientConfig.getC2Url())
+            .build();
+
+        try {
+            Response heartbeatResponse = httpClientReference.get().newCall(request).execute();
+            c2HeartbeatResponse = getResponseBody(heartbeatResponse).flatMap(response -> serializer.deserialize(response, C2HeartbeatResponse.class));
+        } catch (IOException ce) {
+            logger.error("Error while sending heartbeat", ce);
+        }
+
+        return c2HeartbeatResponse;
+    }
+
+    private Optional<String> getResponseBody(Response response) {
+        String responseBody = null;
+
+        try {
+            responseBody = response.body().string();
+            logger.debug("Received response (Status={}) {}", response.code(), responseBody);
+        } catch (IOException e) {
+            logger.error("Could not get response body: ", e);

Review Comment:
   Applied



##########
c2/c2-client-bundle/c2-client-http/src/main/java/org/apache/nifi/c2/client/http/C2HttpClient.java:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.nifi.c2.client.http;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.security.KeyStore;
+import java.security.NoSuchAlgorithmException;
+import java.util.Optional;
+import java.util.concurrent.atomic.AtomicReference;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLSocketFactory;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+import okhttp3.MediaType;
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.RequestBody;
+import okhttp3.Response;
+import okhttp3.ResponseBody;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.c2.client.C2ClientConfig;
+import org.apache.nifi.c2.client.api.C2Client;
+import org.apache.nifi.c2.client.api.C2Serializer;
+import org.apache.nifi.c2.protocol.api.C2Heartbeat;
+import org.apache.nifi.c2.protocol.api.C2HeartbeatResponse;
+import org.apache.nifi.c2.protocol.api.C2OperationAck;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class C2HttpClient implements C2Client {
+
+    private static final Logger logger = LoggerFactory.getLogger(C2HttpClient.class);
+    private static final MediaType MEDIA_TYPE_APPLICATION_JSON = MediaType.parse("application/json");
+
+    private final AtomicReference<OkHttpClient> httpClientReference = new AtomicReference<>();
+    private final C2ClientConfig clientConfig;
+    private final C2Serializer serializer;
+
+    public C2HttpClient(C2ClientConfig clientConfig, C2Serializer serializer) {
+        super();
+        this.clientConfig = clientConfig;
+        this.serializer = serializer;
+        final OkHttpClient.Builder okHttpClientBuilder = new OkHttpClient.Builder();
+
+        // Set whether to follow redirects
+        okHttpClientBuilder.followRedirects(true);
+
+        // check if the ssl path is set and add the factory if so
+        if (StringUtils.isNotBlank(clientConfig.getKeystoreFilename())) {
+            try {
+                setSslSocketFactory(okHttpClientBuilder);
+            } catch (Exception e) {
+                throw new IllegalStateException(e);
+            }
+        }
+
+        httpClientReference.set(okHttpClientBuilder.build());
+    }
+
+    @Override
+    public Optional<C2HeartbeatResponse> publishHeartbeat(C2Heartbeat heartbeat) {
+        return serializer.serialize(heartbeat).flatMap(this::sendHeartbeat);
+    }
+
+    private Optional<C2HeartbeatResponse> sendHeartbeat(String heartbeat) {
+        logger.debug("Sending heartbeat to {}", clientConfig.getC2Url());
+
+        Optional<C2HeartbeatResponse> c2HeartbeatResponse = Optional.empty();
+        Request request = new Request.Builder()
+            .post(RequestBody.create(heartbeat, MEDIA_TYPE_APPLICATION_JSON))
+            .url(clientConfig.getC2Url())
+            .build();
+
+        try {
+            Response heartbeatResponse = httpClientReference.get().newCall(request).execute();
+            c2HeartbeatResponse = getResponseBody(heartbeatResponse).flatMap(response -> serializer.deserialize(response, C2HeartbeatResponse.class));
+        } catch (IOException ce) {
+            logger.error("Error while sending heartbeat", ce);

Review Comment:
   Applied



##########
c2/c2-client-bundle/c2-client-http/src/main/java/org/apache/nifi/c2/client/http/C2HttpClient.java:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.nifi.c2.client.http;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.security.KeyStore;
+import java.security.NoSuchAlgorithmException;
+import java.util.Optional;
+import java.util.concurrent.atomic.AtomicReference;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLSocketFactory;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+import okhttp3.MediaType;
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.RequestBody;
+import okhttp3.Response;
+import okhttp3.ResponseBody;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.c2.client.C2ClientConfig;
+import org.apache.nifi.c2.client.api.C2Client;
+import org.apache.nifi.c2.client.api.C2Serializer;
+import org.apache.nifi.c2.protocol.api.C2Heartbeat;
+import org.apache.nifi.c2.protocol.api.C2HeartbeatResponse;
+import org.apache.nifi.c2.protocol.api.C2OperationAck;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class C2HttpClient implements C2Client {
+
+    private static final Logger logger = LoggerFactory.getLogger(C2HttpClient.class);
+    private static final MediaType MEDIA_TYPE_APPLICATION_JSON = MediaType.parse("application/json");
+
+    private final AtomicReference<OkHttpClient> httpClientReference = new AtomicReference<>();
+    private final C2ClientConfig clientConfig;
+    private final C2Serializer serializer;
+
+    public C2HttpClient(C2ClientConfig clientConfig, C2Serializer serializer) {
+        super();
+        this.clientConfig = clientConfig;
+        this.serializer = serializer;
+        final OkHttpClient.Builder okHttpClientBuilder = new OkHttpClient.Builder();
+
+        // Set whether to follow redirects
+        okHttpClientBuilder.followRedirects(true);
+
+        // check if the ssl path is set and add the factory if so
+        if (StringUtils.isNotBlank(clientConfig.getKeystoreFilename())) {
+            try {
+                setSslSocketFactory(okHttpClientBuilder);
+            } catch (Exception e) {
+                throw new IllegalStateException(e);
+            }
+        }
+
+        httpClientReference.set(okHttpClientBuilder.build());
+    }
+
+    @Override
+    public Optional<C2HeartbeatResponse> publishHeartbeat(C2Heartbeat heartbeat) {
+        return serializer.serialize(heartbeat).flatMap(this::sendHeartbeat);
+    }
+
+    private Optional<C2HeartbeatResponse> sendHeartbeat(String heartbeat) {
+        logger.debug("Sending heartbeat to {}", clientConfig.getC2Url());
+
+        Optional<C2HeartbeatResponse> c2HeartbeatResponse = Optional.empty();
+        Request request = new Request.Builder()
+            .post(RequestBody.create(heartbeat, MEDIA_TYPE_APPLICATION_JSON))
+            .url(clientConfig.getC2Url())
+            .build();
+
+        try {
+            Response heartbeatResponse = httpClientReference.get().newCall(request).execute();
+            c2HeartbeatResponse = getResponseBody(heartbeatResponse).flatMap(response -> serializer.deserialize(response, C2HeartbeatResponse.class));
+        } catch (IOException ce) {
+            logger.error("Error while sending heartbeat", ce);
+        }
+
+        return c2HeartbeatResponse;
+    }
+
+    private Optional<String> getResponseBody(Response response) {
+        String responseBody = null;
+
+        try {
+            responseBody = response.body().string();
+            logger.debug("Received response (Status={}) {}", response.code(), responseBody);
+        } catch (IOException e) {
+            logger.error("Could not get response body: ", e);
+        }
+
+        return Optional.ofNullable(responseBody);
+    }
+
+    private void setSslSocketFactory(OkHttpClient.Builder okHttpClientBuilder) throws Exception {
+        final String keystoreLocation = clientConfig.getKeystoreFilename();
+        final String keystoreType = clientConfig.getKeystoreType().getType();
+        final String keystorePass = clientConfig.getKeystorePass();
+
+        assertKeystorePropertiesSet(keystoreLocation, keystorePass, keystoreType);
+
+        // prepare the keystore
+        final KeyStore keyStore = KeyStore.getInstance(keystoreType);
+
+        try (FileInputStream keyStoreStream = new FileInputStream(keystoreLocation)) {
+            keyStore.load(keyStoreStream, keystorePass.toCharArray());
+        }
+
+        final KeyManagerFactory keyManagerFactory = KeyManagerFactory.getInstance(KeyManagerFactory.getDefaultAlgorithm());
+        keyManagerFactory.init(keyStore, keystorePass.toCharArray());
+
+        // load truststore
+        final String truststoreLocation = clientConfig.getTruststoreFilename();
+        final String truststorePass = clientConfig.getTruststorePass();
+        final String truststoreType = clientConfig.getTruststoreType().getType();
+        assertTruststorePropertiesSet(truststoreLocation, truststorePass, truststoreType);
+
+        KeyStore truststore = KeyStore.getInstance(truststoreType);
+        final TrustManagerFactory trustManagerFactory = TrustManagerFactory.getInstance("X509");
+        truststore.load(new FileInputStream(truststoreLocation), truststorePass.toCharArray());
+        trustManagerFactory.init(truststore);
+
+        final X509TrustManager x509TrustManager;
+        TrustManager[] trustManagers = trustManagerFactory.getTrustManagers();
+        if (trustManagers[0] != null) {
+            x509TrustManager = (X509TrustManager) trustManagers[0];
+        } else {
+            throw new IllegalStateException("List of trust managers is null");
+        }
+
+        SSLContext tempSslContext;
+        try {
+            tempSslContext = SSLContext.getInstance("TLS");
+        } catch (NoSuchAlgorithmException e) {
+            logger.warn("Unable to use 'TLS' for the PullHttpChangeIngestor due to NoSuchAlgorithmException. Will attempt to use the default algorithm.", e);
+            tempSslContext = SSLContext.getDefault();
+        }
+
+        final SSLContext sslContext = tempSslContext;
+        sslContext.init(keyManagerFactory.getKeyManagers(), trustManagerFactory.getTrustManagers(), null);
+
+        final SSLSocketFactory sslSocketFactory = sslContext.getSocketFactory();
+        okHttpClientBuilder.sslSocketFactory(sslSocketFactory, x509TrustManager);
+    }
+
+    private void assertKeystorePropertiesSet(String location, String password, String type) {
+        if (location == null || location.isEmpty()) {
+            throw new IllegalArgumentException(clientConfig.getKeystoreFilename() + " is null or is empty");
+        }
+
+        if (password == null || password.isEmpty()) {
+            throw new IllegalArgumentException("The client's keystore filename is set but its password is not (or is empty). If the location is set, the password must also be.");
+        }
+
+        if (type == null || type.isEmpty()) {
+            throw new IllegalArgumentException("The client's keystore filename is set but its type is not (or is empty). If the location is set, the type must also be.");
+        }
+    }
+
+    private void assertTruststorePropertiesSet(String location, String password, String type) {
+        if (location == null || location.isEmpty()) {
+            throw new IllegalArgumentException("The client's truststore filename is not set or is empty");
+        }
+
+        if (password == null || password.isEmpty()) {
+            throw new IllegalArgumentException("The client's truststore filename is set but its password is not (or is empty). If the location is set, the password must also be.");
+        }
+
+        if (type == null || type.isEmpty()) {
+            throw new IllegalArgumentException("The client's truststore filename is set but its type is not (or is empty). If the location is set, the type must also be.");
+        }
+    }
+
+    @Override
+    public ByteBuffer retrieveUpdateContent(String flowUpdateUrl) {
+        final Request.Builder requestBuilder = new Request.Builder()
+                .get()
+                .url(flowUpdateUrl);
+        final Request request = requestBuilder.build();
+
+        ResponseBody body;
+        try (final Response response = httpClientReference.get().newCall(request).execute()) {
+            logger.debug("Response received: {}", response);
+
+            int code = response.code();
+            if (code >= 400) {
+                throw new IOException("Got response code " + code + " while trying to pull configuration: " + response.body().string());

Review Comment:
   Applied the changed messaging. Regarding the extension of IOException, similarly to earlier this part of the code is inherited and we plan to do additional enhancements after this PR is merged. Would you be ok if I would look into it at that point? Added to my list.



##########
c2/c2-client-bundle/c2-client-http/src/main/java/org/apache/nifi/c2/client/http/C2HttpClient.java:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.nifi.c2.client.http;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.security.KeyStore;
+import java.security.NoSuchAlgorithmException;
+import java.util.Optional;
+import java.util.concurrent.atomic.AtomicReference;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLSocketFactory;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+import okhttp3.MediaType;
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.RequestBody;
+import okhttp3.Response;
+import okhttp3.ResponseBody;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.c2.client.C2ClientConfig;
+import org.apache.nifi.c2.client.api.C2Client;
+import org.apache.nifi.c2.client.api.C2Serializer;
+import org.apache.nifi.c2.protocol.api.C2Heartbeat;
+import org.apache.nifi.c2.protocol.api.C2HeartbeatResponse;
+import org.apache.nifi.c2.protocol.api.C2OperationAck;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class C2HttpClient implements C2Client {
+
+    private static final Logger logger = LoggerFactory.getLogger(C2HttpClient.class);
+    private static final MediaType MEDIA_TYPE_APPLICATION_JSON = MediaType.parse("application/json");
+
+    private final AtomicReference<OkHttpClient> httpClientReference = new AtomicReference<>();
+    private final C2ClientConfig clientConfig;
+    private final C2Serializer serializer;
+
+    public C2HttpClient(C2ClientConfig clientConfig, C2Serializer serializer) {
+        super();
+        this.clientConfig = clientConfig;
+        this.serializer = serializer;
+        final OkHttpClient.Builder okHttpClientBuilder = new OkHttpClient.Builder();
+
+        // Set whether to follow redirects
+        okHttpClientBuilder.followRedirects(true);
+
+        // check if the ssl path is set and add the factory if so
+        if (StringUtils.isNotBlank(clientConfig.getKeystoreFilename())) {
+            try {
+                setSslSocketFactory(okHttpClientBuilder);
+            } catch (Exception e) {
+                throw new IllegalStateException(e);
+            }
+        }
+
+        httpClientReference.set(okHttpClientBuilder.build());
+    }
+
+    @Override
+    public Optional<C2HeartbeatResponse> publishHeartbeat(C2Heartbeat heartbeat) {
+        return serializer.serialize(heartbeat).flatMap(this::sendHeartbeat);
+    }
+
+    private Optional<C2HeartbeatResponse> sendHeartbeat(String heartbeat) {
+        logger.debug("Sending heartbeat to {}", clientConfig.getC2Url());
+
+        Optional<C2HeartbeatResponse> c2HeartbeatResponse = Optional.empty();
+        Request request = new Request.Builder()
+            .post(RequestBody.create(heartbeat, MEDIA_TYPE_APPLICATION_JSON))
+            .url(clientConfig.getC2Url())
+            .build();
+
+        try {
+            Response heartbeatResponse = httpClientReference.get().newCall(request).execute();
+            c2HeartbeatResponse = getResponseBody(heartbeatResponse).flatMap(response -> serializer.deserialize(response, C2HeartbeatResponse.class));
+        } catch (IOException ce) {
+            logger.error("Error while sending heartbeat", ce);
+        }
+
+        return c2HeartbeatResponse;
+    }
+
+    private Optional<String> getResponseBody(Response response) {
+        String responseBody = null;
+
+        try {
+            responseBody = response.body().string();
+            logger.debug("Received response (Status={}) {}", response.code(), responseBody);
+        } catch (IOException e) {
+            logger.error("Could not get response body: ", e);
+        }
+
+        return Optional.ofNullable(responseBody);
+    }
+
+    private void setSslSocketFactory(OkHttpClient.Builder okHttpClientBuilder) throws Exception {
+        final String keystoreLocation = clientConfig.getKeystoreFilename();
+        final String keystoreType = clientConfig.getKeystoreType().getType();
+        final String keystorePass = clientConfig.getKeystorePass();
+
+        assertKeystorePropertiesSet(keystoreLocation, keystorePass, keystoreType);
+
+        // prepare the keystore
+        final KeyStore keyStore = KeyStore.getInstance(keystoreType);
+
+        try (FileInputStream keyStoreStream = new FileInputStream(keystoreLocation)) {
+            keyStore.load(keyStoreStream, keystorePass.toCharArray());
+        }
+
+        final KeyManagerFactory keyManagerFactory = KeyManagerFactory.getInstance(KeyManagerFactory.getDefaultAlgorithm());
+        keyManagerFactory.init(keyStore, keystorePass.toCharArray());
+
+        // load truststore
+        final String truststoreLocation = clientConfig.getTruststoreFilename();
+        final String truststorePass = clientConfig.getTruststorePass();
+        final String truststoreType = clientConfig.getTruststoreType().getType();
+        assertTruststorePropertiesSet(truststoreLocation, truststorePass, truststoreType);
+
+        KeyStore truststore = KeyStore.getInstance(truststoreType);
+        final TrustManagerFactory trustManagerFactory = TrustManagerFactory.getInstance("X509");
+        truststore.load(new FileInputStream(truststoreLocation), truststorePass.toCharArray());
+        trustManagerFactory.init(truststore);
+
+        final X509TrustManager x509TrustManager;
+        TrustManager[] trustManagers = trustManagerFactory.getTrustManagers();
+        if (trustManagers[0] != null) {
+            x509TrustManager = (X509TrustManager) trustManagers[0];
+        } else {
+            throw new IllegalStateException("List of trust managers is null");
+        }
+
+        SSLContext tempSslContext;
+        try {
+            tempSslContext = SSLContext.getInstance("TLS");
+        } catch (NoSuchAlgorithmException e) {
+            logger.warn("Unable to use 'TLS' for the PullHttpChangeIngestor due to NoSuchAlgorithmException. Will attempt to use the default algorithm.", e);
+            tempSslContext = SSLContext.getDefault();

Review Comment:
   Thanks for the details, changed to exception throwing as proposed.



##########
minifi/pom.xml:
##########
@@ -449,10 +449,22 @@ limitations under the License.
                 <version>1.17.0-SNAPSHOT</version>
             </dependency>
 
+            <!-- C2 modules -->
+            <dependency>
+                <groupId>org.apache.nifi</groupId>
+                <artifactId>c2-client-api</artifactId>
+                <version>1.17.0-SNAPSHOT</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.nifi</groupId>
+                <artifactId>c2-client-base</artifactId>
+                <version>1.17.0-SNAPSHOT</version>
+            </dependency>
+
             <dependency>
                 <groupId>com.squareup.okhttp3</groupId>
                 <artifactId>okhttp</artifactId>
-                <version>3.12.3</version>
+                <version>${okhttp.version}</version>

Review Comment:
   Removed



##########
c2/c2-client-bundle/c2-client-http/src/main/java/org/apache/nifi/c2/client/http/C2HttpClient.java:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.nifi.c2.client.http;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.security.KeyStore;
+import java.security.NoSuchAlgorithmException;
+import java.util.Optional;
+import java.util.concurrent.atomic.AtomicReference;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLSocketFactory;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+import okhttp3.MediaType;
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.RequestBody;
+import okhttp3.Response;
+import okhttp3.ResponseBody;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.c2.client.C2ClientConfig;
+import org.apache.nifi.c2.client.api.C2Client;
+import org.apache.nifi.c2.client.api.C2Serializer;
+import org.apache.nifi.c2.protocol.api.C2Heartbeat;
+import org.apache.nifi.c2.protocol.api.C2HeartbeatResponse;
+import org.apache.nifi.c2.protocol.api.C2OperationAck;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class C2HttpClient implements C2Client {
+
+    private static final Logger logger = LoggerFactory.getLogger(C2HttpClient.class);
+    private static final MediaType MEDIA_TYPE_APPLICATION_JSON = MediaType.parse("application/json");
+
+    private final AtomicReference<OkHttpClient> httpClientReference = new AtomicReference<>();
+    private final C2ClientConfig clientConfig;
+    private final C2Serializer serializer;
+
+    public C2HttpClient(C2ClientConfig clientConfig, C2Serializer serializer) {
+        super();
+        this.clientConfig = clientConfig;
+        this.serializer = serializer;
+        final OkHttpClient.Builder okHttpClientBuilder = new OkHttpClient.Builder();
+
+        // Set whether to follow redirects
+        okHttpClientBuilder.followRedirects(true);
+
+        // check if the ssl path is set and add the factory if so
+        if (StringUtils.isNotBlank(clientConfig.getKeystoreFilename())) {
+            try {
+                setSslSocketFactory(okHttpClientBuilder);
+            } catch (Exception e) {
+                throw new IllegalStateException(e);
+            }
+        }
+
+        httpClientReference.set(okHttpClientBuilder.build());
+    }
+
+    @Override
+    public Optional<C2HeartbeatResponse> publishHeartbeat(C2Heartbeat heartbeat) {
+        return serializer.serialize(heartbeat).flatMap(this::sendHeartbeat);
+    }
+
+    private Optional<C2HeartbeatResponse> sendHeartbeat(String heartbeat) {
+        logger.debug("Sending heartbeat to {}", clientConfig.getC2Url());
+
+        Optional<C2HeartbeatResponse> c2HeartbeatResponse = Optional.empty();
+        Request request = new Request.Builder()
+            .post(RequestBody.create(heartbeat, MEDIA_TYPE_APPLICATION_JSON))
+            .url(clientConfig.getC2Url())
+            .build();
+
+        try {
+            Response heartbeatResponse = httpClientReference.get().newCall(request).execute();
+            c2HeartbeatResponse = getResponseBody(heartbeatResponse).flatMap(response -> serializer.deserialize(response, C2HeartbeatResponse.class));
+        } catch (IOException ce) {
+            logger.error("Error while sending heartbeat", ce);
+        }
+
+        return c2HeartbeatResponse;
+    }
+
+    private Optional<String> getResponseBody(Response response) {
+        String responseBody = null;
+
+        try {
+            responseBody = response.body().string();
+            logger.debug("Received response (Status={}) {}", response.code(), responseBody);
+        } catch (IOException e) {
+            logger.error("Could not get response body: ", e);
+        }
+
+        return Optional.ofNullable(responseBody);
+    }
+
+    private void setSslSocketFactory(OkHttpClient.Builder okHttpClientBuilder) throws Exception {
+        final String keystoreLocation = clientConfig.getKeystoreFilename();
+        final String keystoreType = clientConfig.getKeystoreType().getType();
+        final String keystorePass = clientConfig.getKeystorePass();
+
+        assertKeystorePropertiesSet(keystoreLocation, keystorePass, keystoreType);
+
+        // prepare the keystore
+        final KeyStore keyStore = KeyStore.getInstance(keystoreType);
+
+        try (FileInputStream keyStoreStream = new FileInputStream(keystoreLocation)) {
+            keyStore.load(keyStoreStream, keystorePass.toCharArray());
+        }
+
+        final KeyManagerFactory keyManagerFactory = KeyManagerFactory.getInstance(KeyManagerFactory.getDefaultAlgorithm());
+        keyManagerFactory.init(keyStore, keystorePass.toCharArray());
+
+        // load truststore
+        final String truststoreLocation = clientConfig.getTruststoreFilename();
+        final String truststorePass = clientConfig.getTruststorePass();
+        final String truststoreType = clientConfig.getTruststoreType().getType();
+        assertTruststorePropertiesSet(truststoreLocation, truststorePass, truststoreType);
+
+        KeyStore truststore = KeyStore.getInstance(truststoreType);
+        final TrustManagerFactory trustManagerFactory = TrustManagerFactory.getInstance("X509");
+        truststore.load(new FileInputStream(truststoreLocation), truststorePass.toCharArray());
+        trustManagerFactory.init(truststore);
+
+        final X509TrustManager x509TrustManager;
+        TrustManager[] trustManagers = trustManagerFactory.getTrustManagers();
+        if (trustManagers[0] != null) {
+            x509TrustManager = (X509TrustManager) trustManagers[0];
+        } else {
+            throw new IllegalStateException("List of trust managers is null");
+        }
+
+        SSLContext tempSslContext;
+        try {
+            tempSslContext = SSLContext.getInstance("TLS");
+        } catch (NoSuchAlgorithmException e) {
+            logger.warn("Unable to use 'TLS' for the PullHttpChangeIngestor due to NoSuchAlgorithmException. Will attempt to use the default algorithm.", e);
+            tempSslContext = SSLContext.getDefault();
+        }
+
+        final SSLContext sslContext = tempSslContext;
+        sslContext.init(keyManagerFactory.getKeyManagers(), trustManagerFactory.getTrustManagers(), null);
+
+        final SSLSocketFactory sslSocketFactory = sslContext.getSocketFactory();
+        okHttpClientBuilder.sslSocketFactory(sslSocketFactory, x509TrustManager);
+    }
+
+    private void assertKeystorePropertiesSet(String location, String password, String type) {
+        if (location == null || location.isEmpty()) {
+            throw new IllegalArgumentException(clientConfig.getKeystoreFilename() + " is null or is empty");
+        }
+
+        if (password == null || password.isEmpty()) {
+            throw new IllegalArgumentException("The client's keystore filename is set but its password is not (or is empty). If the location is set, the password must also be.");
+        }
+
+        if (type == null || type.isEmpty()) {
+            throw new IllegalArgumentException("The client's keystore filename is set but its type is not (or is empty). If the location is set, the type must also be.");
+        }
+    }
+
+    private void assertTruststorePropertiesSet(String location, String password, String type) {
+        if (location == null || location.isEmpty()) {
+            throw new IllegalArgumentException("The client's truststore filename is not set or is empty");
+        }
+
+        if (password == null || password.isEmpty()) {
+            throw new IllegalArgumentException("The client's truststore filename is set but its password is not (or is empty). If the location is set, the password must also be.");
+        }
+
+        if (type == null || type.isEmpty()) {
+            throw new IllegalArgumentException("The client's truststore filename is set but its type is not (or is empty). If the location is set, the type must also be.");
+        }
+    }
+
+    @Override
+    public ByteBuffer retrieveUpdateContent(String flowUpdateUrl) {
+        final Request.Builder requestBuilder = new Request.Builder()
+                .get()
+                .url(flowUpdateUrl);
+        final Request request = requestBuilder.build();
+
+        ResponseBody body;
+        try (final Response response = httpClientReference.get().newCall(request).execute()) {
+            logger.debug("Response received: {}", response);
+
+            int code = response.code();
+            if (code >= 400) {
+                throw new IOException("Got response code " + code + " while trying to pull configuration: " + response.body().string());
+            }
+
+            body = response.body();
+
+            if (body == null) {
+                logger.warn("No body returned when pulling a new configuration");
+                return null;
+            }
+
+            final ByteBuffer bodyByteBuffer = ByteBuffer.wrap(body.bytes());
+            return bodyByteBuffer;
+
+        } catch (Exception e) {
+            logger.warn("Hit an exception while trying to pull", e);

Review Comment:
   Applied



##########
c2/c2-client-bundle/c2-client-http/src/main/java/org/apache/nifi/c2/client/http/C2HttpClient.java:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.nifi.c2.client.http;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.security.KeyStore;
+import java.security.NoSuchAlgorithmException;
+import java.util.Optional;
+import java.util.concurrent.atomic.AtomicReference;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLSocketFactory;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+import okhttp3.MediaType;
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.RequestBody;
+import okhttp3.Response;
+import okhttp3.ResponseBody;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.c2.client.C2ClientConfig;
+import org.apache.nifi.c2.client.api.C2Client;
+import org.apache.nifi.c2.client.api.C2Serializer;
+import org.apache.nifi.c2.protocol.api.C2Heartbeat;
+import org.apache.nifi.c2.protocol.api.C2HeartbeatResponse;
+import org.apache.nifi.c2.protocol.api.C2OperationAck;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class C2HttpClient implements C2Client {
+
+    private static final Logger logger = LoggerFactory.getLogger(C2HttpClient.class);
+    private static final MediaType MEDIA_TYPE_APPLICATION_JSON = MediaType.parse("application/json");
+
+    private final AtomicReference<OkHttpClient> httpClientReference = new AtomicReference<>();
+    private final C2ClientConfig clientConfig;
+    private final C2Serializer serializer;
+
+    public C2HttpClient(C2ClientConfig clientConfig, C2Serializer serializer) {
+        super();
+        this.clientConfig = clientConfig;
+        this.serializer = serializer;
+        final OkHttpClient.Builder okHttpClientBuilder = new OkHttpClient.Builder();
+
+        // Set whether to follow redirects
+        okHttpClientBuilder.followRedirects(true);
+
+        // check if the ssl path is set and add the factory if so
+        if (StringUtils.isNotBlank(clientConfig.getKeystoreFilename())) {
+            try {
+                setSslSocketFactory(okHttpClientBuilder);
+            } catch (Exception e) {
+                throw new IllegalStateException(e);
+            }
+        }
+
+        httpClientReference.set(okHttpClientBuilder.build());
+    }
+
+    @Override
+    public Optional<C2HeartbeatResponse> publishHeartbeat(C2Heartbeat heartbeat) {
+        return serializer.serialize(heartbeat).flatMap(this::sendHeartbeat);
+    }
+
+    private Optional<C2HeartbeatResponse> sendHeartbeat(String heartbeat) {
+        logger.debug("Sending heartbeat to {}", clientConfig.getC2Url());
+
+        Optional<C2HeartbeatResponse> c2HeartbeatResponse = Optional.empty();
+        Request request = new Request.Builder()
+            .post(RequestBody.create(heartbeat, MEDIA_TYPE_APPLICATION_JSON))
+            .url(clientConfig.getC2Url())
+            .build();
+
+        try {
+            Response heartbeatResponse = httpClientReference.get().newCall(request).execute();
+            c2HeartbeatResponse = getResponseBody(heartbeatResponse).flatMap(response -> serializer.deserialize(response, C2HeartbeatResponse.class));
+        } catch (IOException ce) {
+            logger.error("Error while sending heartbeat", ce);
+        }
+
+        return c2HeartbeatResponse;
+    }
+
+    private Optional<String> getResponseBody(Response response) {
+        String responseBody = null;
+
+        try {
+            responseBody = response.body().string();
+            logger.debug("Received response (Status={}) {}", response.code(), responseBody);
+        } catch (IOException e) {
+            logger.error("Could not get response body: ", e);
+        }
+
+        return Optional.ofNullable(responseBody);
+    }
+
+    private void setSslSocketFactory(OkHttpClient.Builder okHttpClientBuilder) throws Exception {
+        final String keystoreLocation = clientConfig.getKeystoreFilename();
+        final String keystoreType = clientConfig.getKeystoreType().getType();
+        final String keystorePass = clientConfig.getKeystorePass();
+
+        assertKeystorePropertiesSet(keystoreLocation, keystorePass, keystoreType);
+
+        // prepare the keystore
+        final KeyStore keyStore = KeyStore.getInstance(keystoreType);
+
+        try (FileInputStream keyStoreStream = new FileInputStream(keystoreLocation)) {
+            keyStore.load(keyStoreStream, keystorePass.toCharArray());
+        }
+
+        final KeyManagerFactory keyManagerFactory = KeyManagerFactory.getInstance(KeyManagerFactory.getDefaultAlgorithm());
+        keyManagerFactory.init(keyStore, keystorePass.toCharArray());
+
+        // load truststore
+        final String truststoreLocation = clientConfig.getTruststoreFilename();
+        final String truststorePass = clientConfig.getTruststorePass();
+        final String truststoreType = clientConfig.getTruststoreType().getType();
+        assertTruststorePropertiesSet(truststoreLocation, truststorePass, truststoreType);
+
+        KeyStore truststore = KeyStore.getInstance(truststoreType);
+        final TrustManagerFactory trustManagerFactory = TrustManagerFactory.getInstance("X509");
+        truststore.load(new FileInputStream(truststoreLocation), truststorePass.toCharArray());
+        trustManagerFactory.init(truststore);
+
+        final X509TrustManager x509TrustManager;
+        TrustManager[] trustManagers = trustManagerFactory.getTrustManagers();
+        if (trustManagers[0] != null) {
+            x509TrustManager = (X509TrustManager) trustManagers[0];
+        } else {
+            throw new IllegalStateException("List of trust managers is null");
+        }
+
+        SSLContext tempSslContext;
+        try {
+            tempSslContext = SSLContext.getInstance("TLS");
+        } catch (NoSuchAlgorithmException e) {
+            logger.warn("Unable to use 'TLS' for the PullHttpChangeIngestor due to NoSuchAlgorithmException. Will attempt to use the default algorithm.", e);
+            tempSslContext = SSLContext.getDefault();
+        }
+
+        final SSLContext sslContext = tempSslContext;
+        sslContext.init(keyManagerFactory.getKeyManagers(), trustManagerFactory.getTrustManagers(), null);
+
+        final SSLSocketFactory sslSocketFactory = sslContext.getSocketFactory();
+        okHttpClientBuilder.sslSocketFactory(sslSocketFactory, x509TrustManager);
+    }
+
+    private void assertKeystorePropertiesSet(String location, String password, String type) {
+        if (location == null || location.isEmpty()) {
+            throw new IllegalArgumentException(clientConfig.getKeystoreFilename() + " is null or is empty");
+        }
+
+        if (password == null || password.isEmpty()) {
+            throw new IllegalArgumentException("The client's keystore filename is set but its password is not (or is empty). If the location is set, the password must also be.");
+        }
+
+        if (type == null || type.isEmpty()) {
+            throw new IllegalArgumentException("The client's keystore filename is set but its type is not (or is empty). If the location is set, the type must also be.");
+        }
+    }
+
+    private void assertTruststorePropertiesSet(String location, String password, String type) {
+        if (location == null || location.isEmpty()) {
+            throw new IllegalArgumentException("The client's truststore filename is not set or is empty");
+        }
+
+        if (password == null || password.isEmpty()) {
+            throw new IllegalArgumentException("The client's truststore filename is set but its password is not (or is empty). If the location is set, the password must also be.");
+        }
+
+        if (type == null || type.isEmpty()) {
+            throw new IllegalArgumentException("The client's truststore filename is set but its type is not (or is empty). If the location is set, the type must also be.");
+        }
+    }
+
+    @Override
+    public ByteBuffer retrieveUpdateContent(String flowUpdateUrl) {
+        final Request.Builder requestBuilder = new Request.Builder()
+                .get()
+                .url(flowUpdateUrl);
+        final Request request = requestBuilder.build();
+
+        ResponseBody body;
+        try (final Response response = httpClientReference.get().newCall(request).execute()) {
+            logger.debug("Response received: {}", response);
+
+            int code = response.code();
+            if (code >= 400) {
+                throw new IOException("Got response code " + code + " while trying to pull configuration: " + response.body().string());
+            }
+
+            body = response.body();
+
+            if (body == null) {
+                logger.warn("No body returned when pulling a new configuration");
+                return null;
+            }
+
+            final ByteBuffer bodyByteBuffer = ByteBuffer.wrap(body.bytes());
+            return bodyByteBuffer;
+
+        } catch (Exception e) {
+            logger.warn("Hit an exception while trying to pull", e);
+            return null;
+        }
+    }
+
+    @Override
+    public void acknowledgeOperation(C2OperationAck operationAck) {
+        logger.info("Performing acknowledgement request to {} for operation {}", clientConfig.getC2AckUrl(), operationAck.getOperationId());
+        final ObjectMapper jacksonObjectMapper = new ObjectMapper();
+        jacksonObjectMapper.setSerializationInclusion(JsonInclude.Include.NON_NULL);

Review Comment:
   Good catch, C2Serialiser was introduced just for this purpose and tried to use it everywhere it looks like I missed this spot. Updated.



##########
c2/c2-client-bundle/c2-client-http/src/main/java/org/apache/nifi/c2/client/http/C2HttpClient.java:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.nifi.c2.client.http;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.security.KeyStore;
+import java.security.NoSuchAlgorithmException;
+import java.util.Optional;
+import java.util.concurrent.atomic.AtomicReference;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLSocketFactory;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+import okhttp3.MediaType;
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.RequestBody;
+import okhttp3.Response;
+import okhttp3.ResponseBody;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.c2.client.C2ClientConfig;
+import org.apache.nifi.c2.client.api.C2Client;
+import org.apache.nifi.c2.client.api.C2Serializer;
+import org.apache.nifi.c2.protocol.api.C2Heartbeat;
+import org.apache.nifi.c2.protocol.api.C2HeartbeatResponse;
+import org.apache.nifi.c2.protocol.api.C2OperationAck;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class C2HttpClient implements C2Client {
+
+    private static final Logger logger = LoggerFactory.getLogger(C2HttpClient.class);
+    private static final MediaType MEDIA_TYPE_APPLICATION_JSON = MediaType.parse("application/json");
+
+    private final AtomicReference<OkHttpClient> httpClientReference = new AtomicReference<>();
+    private final C2ClientConfig clientConfig;
+    private final C2Serializer serializer;
+
+    public C2HttpClient(C2ClientConfig clientConfig, C2Serializer serializer) {
+        super();
+        this.clientConfig = clientConfig;
+        this.serializer = serializer;
+        final OkHttpClient.Builder okHttpClientBuilder = new OkHttpClient.Builder();
+
+        // Set whether to follow redirects
+        okHttpClientBuilder.followRedirects(true);
+
+        // check if the ssl path is set and add the factory if so
+        if (StringUtils.isNotBlank(clientConfig.getKeystoreFilename())) {
+            try {
+                setSslSocketFactory(okHttpClientBuilder);
+            } catch (Exception e) {
+                throw new IllegalStateException(e);
+            }
+        }
+
+        httpClientReference.set(okHttpClientBuilder.build());
+    }
+
+    @Override
+    public Optional<C2HeartbeatResponse> publishHeartbeat(C2Heartbeat heartbeat) {
+        return serializer.serialize(heartbeat).flatMap(this::sendHeartbeat);
+    }
+
+    private Optional<C2HeartbeatResponse> sendHeartbeat(String heartbeat) {
+        logger.debug("Sending heartbeat to {}", clientConfig.getC2Url());
+
+        Optional<C2HeartbeatResponse> c2HeartbeatResponse = Optional.empty();
+        Request request = new Request.Builder()
+            .post(RequestBody.create(heartbeat, MEDIA_TYPE_APPLICATION_JSON))
+            .url(clientConfig.getC2Url())
+            .build();
+
+        try {
+            Response heartbeatResponse = httpClientReference.get().newCall(request).execute();
+            c2HeartbeatResponse = getResponseBody(heartbeatResponse).flatMap(response -> serializer.deserialize(response, C2HeartbeatResponse.class));
+        } catch (IOException ce) {
+            logger.error("Error while sending heartbeat", ce);
+        }
+
+        return c2HeartbeatResponse;
+    }
+
+    private Optional<String> getResponseBody(Response response) {
+        String responseBody = null;
+
+        try {
+            responseBody = response.body().string();
+            logger.debug("Received response (Status={}) {}", response.code(), responseBody);
+        } catch (IOException e) {
+            logger.error("Could not get response body: ", e);
+        }
+
+        return Optional.ofNullable(responseBody);
+    }
+
+    private void setSslSocketFactory(OkHttpClient.Builder okHttpClientBuilder) throws Exception {
+        final String keystoreLocation = clientConfig.getKeystoreFilename();
+        final String keystoreType = clientConfig.getKeystoreType().getType();
+        final String keystorePass = clientConfig.getKeystorePass();
+
+        assertKeystorePropertiesSet(keystoreLocation, keystorePass, keystoreType);
+
+        // prepare the keystore
+        final KeyStore keyStore = KeyStore.getInstance(keystoreType);
+
+        try (FileInputStream keyStoreStream = new FileInputStream(keystoreLocation)) {
+            keyStore.load(keyStoreStream, keystorePass.toCharArray());
+        }
+
+        final KeyManagerFactory keyManagerFactory = KeyManagerFactory.getInstance(KeyManagerFactory.getDefaultAlgorithm());
+        keyManagerFactory.init(keyStore, keystorePass.toCharArray());
+
+        // load truststore
+        final String truststoreLocation = clientConfig.getTruststoreFilename();
+        final String truststorePass = clientConfig.getTruststorePass();
+        final String truststoreType = clientConfig.getTruststoreType().getType();
+        assertTruststorePropertiesSet(truststoreLocation, truststorePass, truststoreType);
+
+        KeyStore truststore = KeyStore.getInstance(truststoreType);
+        final TrustManagerFactory trustManagerFactory = TrustManagerFactory.getInstance("X509");
+        truststore.load(new FileInputStream(truststoreLocation), truststorePass.toCharArray());
+        trustManagerFactory.init(truststore);
+
+        final X509TrustManager x509TrustManager;
+        TrustManager[] trustManagers = trustManagerFactory.getTrustManagers();
+        if (trustManagers[0] != null) {
+            x509TrustManager = (X509TrustManager) trustManagers[0];
+        } else {
+            throw new IllegalStateException("List of trust managers is null");
+        }
+
+        SSLContext tempSslContext;
+        try {
+            tempSslContext = SSLContext.getInstance("TLS");
+        } catch (NoSuchAlgorithmException e) {
+            logger.warn("Unable to use 'TLS' for the PullHttpChangeIngestor due to NoSuchAlgorithmException. Will attempt to use the default algorithm.", e);
+            tempSslContext = SSLContext.getDefault();
+        }
+
+        final SSLContext sslContext = tempSslContext;
+        sslContext.init(keyManagerFactory.getKeyManagers(), trustManagerFactory.getTrustManagers(), null);
+
+        final SSLSocketFactory sslSocketFactory = sslContext.getSocketFactory();
+        okHttpClientBuilder.sslSocketFactory(sslSocketFactory, x509TrustManager);
+    }
+
+    private void assertKeystorePropertiesSet(String location, String password, String type) {
+        if (location == null || location.isEmpty()) {
+            throw new IllegalArgumentException(clientConfig.getKeystoreFilename() + " is null or is empty");
+        }
+
+        if (password == null || password.isEmpty()) {
+            throw new IllegalArgumentException("The client's keystore filename is set but its password is not (or is empty). If the location is set, the password must also be.");
+        }
+
+        if (type == null || type.isEmpty()) {
+            throw new IllegalArgumentException("The client's keystore filename is set but its type is not (or is empty). If the location is set, the type must also be.");
+        }
+    }
+
+    private void assertTruststorePropertiesSet(String location, String password, String type) {
+        if (location == null || location.isEmpty()) {
+            throw new IllegalArgumentException("The client's truststore filename is not set or is empty");
+        }
+
+        if (password == null || password.isEmpty()) {
+            throw new IllegalArgumentException("The client's truststore filename is set but its password is not (or is empty). If the location is set, the password must also be.");
+        }
+
+        if (type == null || type.isEmpty()) {
+            throw new IllegalArgumentException("The client's truststore filename is set but its type is not (or is empty). If the location is set, the type must also be.");
+        }
+    }
+
+    @Override
+    public ByteBuffer retrieveUpdateContent(String flowUpdateUrl) {
+        final Request.Builder requestBuilder = new Request.Builder()
+                .get()
+                .url(flowUpdateUrl);
+        final Request request = requestBuilder.build();
+
+        ResponseBody body;
+        try (final Response response = httpClientReference.get().newCall(request).execute()) {
+            logger.debug("Response received: {}", response);
+
+            int code = response.code();
+            if (code >= 400) {
+                throw new IOException("Got response code " + code + " while trying to pull configuration: " + response.body().string());
+            }
+
+            body = response.body();
+
+            if (body == null) {
+                logger.warn("No body returned when pulling a new configuration");
+                return null;
+            }
+
+            final ByteBuffer bodyByteBuffer = ByteBuffer.wrap(body.bytes());
+            return bodyByteBuffer;
+
+        } catch (Exception e) {
+            logger.warn("Hit an exception while trying to pull", e);
+            return null;
+        }
+    }
+
+    @Override
+    public void acknowledgeOperation(C2OperationAck operationAck) {
+        logger.info("Performing acknowledgement request to {} for operation {}", clientConfig.getC2AckUrl(), operationAck.getOperationId());
+        final ObjectMapper jacksonObjectMapper = new ObjectMapper();
+        jacksonObjectMapper.setSerializationInclusion(JsonInclude.Include.NON_NULL);
+        try {
+            final String operationAckBody = jacksonObjectMapper.writeValueAsString(operationAck);
+
+            final RequestBody requestBody = RequestBody.create(operationAckBody, MediaType.parse("application/json"));
+            final Request.Builder requestBuilder = new Request.Builder()
+                    .post(requestBody)
+                    .url(clientConfig.getC2AckUrl());
+            final Response heartbeatResponse = httpClientReference.get().newCall(requestBuilder.build()).execute();
+            if (!heartbeatResponse.isSuccessful()) {
+                logger.warn("Acknowledgement was not successful.");

Review Comment:
   Updated



##########
c2/c2-client-bundle/c2-client-http/src/main/java/org/apache/nifi/c2/client/http/C2HttpClient.java:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.nifi.c2.client.http;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.security.KeyStore;
+import java.security.NoSuchAlgorithmException;
+import java.util.Optional;
+import java.util.concurrent.atomic.AtomicReference;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLSocketFactory;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+import okhttp3.MediaType;
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.RequestBody;
+import okhttp3.Response;
+import okhttp3.ResponseBody;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.c2.client.C2ClientConfig;
+import org.apache.nifi.c2.client.api.C2Client;
+import org.apache.nifi.c2.client.api.C2Serializer;
+import org.apache.nifi.c2.protocol.api.C2Heartbeat;
+import org.apache.nifi.c2.protocol.api.C2HeartbeatResponse;
+import org.apache.nifi.c2.protocol.api.C2OperationAck;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class C2HttpClient implements C2Client {
+
+    private static final Logger logger = LoggerFactory.getLogger(C2HttpClient.class);
+    private static final MediaType MEDIA_TYPE_APPLICATION_JSON = MediaType.parse("application/json");
+
+    private final AtomicReference<OkHttpClient> httpClientReference = new AtomicReference<>();
+    private final C2ClientConfig clientConfig;
+    private final C2Serializer serializer;
+
+    public C2HttpClient(C2ClientConfig clientConfig, C2Serializer serializer) {
+        super();
+        this.clientConfig = clientConfig;
+        this.serializer = serializer;
+        final OkHttpClient.Builder okHttpClientBuilder = new OkHttpClient.Builder();
+
+        // Set whether to follow redirects
+        okHttpClientBuilder.followRedirects(true);
+
+        // check if the ssl path is set and add the factory if so
+        if (StringUtils.isNotBlank(clientConfig.getKeystoreFilename())) {
+            try {
+                setSslSocketFactory(okHttpClientBuilder);
+            } catch (Exception e) {
+                throw new IllegalStateException(e);
+            }
+        }
+
+        httpClientReference.set(okHttpClientBuilder.build());
+    }
+
+    @Override
+    public Optional<C2HeartbeatResponse> publishHeartbeat(C2Heartbeat heartbeat) {
+        return serializer.serialize(heartbeat).flatMap(this::sendHeartbeat);
+    }
+
+    private Optional<C2HeartbeatResponse> sendHeartbeat(String heartbeat) {
+        logger.debug("Sending heartbeat to {}", clientConfig.getC2Url());
+
+        Optional<C2HeartbeatResponse> c2HeartbeatResponse = Optional.empty();
+        Request request = new Request.Builder()
+            .post(RequestBody.create(heartbeat, MEDIA_TYPE_APPLICATION_JSON))
+            .url(clientConfig.getC2Url())
+            .build();
+
+        try {
+            Response heartbeatResponse = httpClientReference.get().newCall(request).execute();
+            c2HeartbeatResponse = getResponseBody(heartbeatResponse).flatMap(response -> serializer.deserialize(response, C2HeartbeatResponse.class));
+        } catch (IOException ce) {
+            logger.error("Error while sending heartbeat", ce);
+        }
+
+        return c2HeartbeatResponse;
+    }
+
+    private Optional<String> getResponseBody(Response response) {
+        String responseBody = null;
+
+        try {
+            responseBody = response.body().string();
+            logger.debug("Received response (Status={}) {}", response.code(), responseBody);
+        } catch (IOException e) {
+            logger.error("Could not get response body: ", e);
+        }
+
+        return Optional.ofNullable(responseBody);
+    }
+
+    private void setSslSocketFactory(OkHttpClient.Builder okHttpClientBuilder) throws Exception {
+        final String keystoreLocation = clientConfig.getKeystoreFilename();
+        final String keystoreType = clientConfig.getKeystoreType().getType();
+        final String keystorePass = clientConfig.getKeystorePass();
+
+        assertKeystorePropertiesSet(keystoreLocation, keystorePass, keystoreType);
+
+        // prepare the keystore
+        final KeyStore keyStore = KeyStore.getInstance(keystoreType);
+
+        try (FileInputStream keyStoreStream = new FileInputStream(keystoreLocation)) {
+            keyStore.load(keyStoreStream, keystorePass.toCharArray());
+        }
+
+        final KeyManagerFactory keyManagerFactory = KeyManagerFactory.getInstance(KeyManagerFactory.getDefaultAlgorithm());
+        keyManagerFactory.init(keyStore, keystorePass.toCharArray());
+
+        // load truststore
+        final String truststoreLocation = clientConfig.getTruststoreFilename();
+        final String truststorePass = clientConfig.getTruststorePass();
+        final String truststoreType = clientConfig.getTruststoreType().getType();
+        assertTruststorePropertiesSet(truststoreLocation, truststorePass, truststoreType);
+
+        KeyStore truststore = KeyStore.getInstance(truststoreType);
+        final TrustManagerFactory trustManagerFactory = TrustManagerFactory.getInstance("X509");
+        truststore.load(new FileInputStream(truststoreLocation), truststorePass.toCharArray());
+        trustManagerFactory.init(truststore);
+
+        final X509TrustManager x509TrustManager;
+        TrustManager[] trustManagers = trustManagerFactory.getTrustManagers();
+        if (trustManagers[0] != null) {
+            x509TrustManager = (X509TrustManager) trustManagers[0];
+        } else {
+            throw new IllegalStateException("List of trust managers is null");
+        }
+
+        SSLContext tempSslContext;
+        try {
+            tempSslContext = SSLContext.getInstance("TLS");
+        } catch (NoSuchAlgorithmException e) {
+            logger.warn("Unable to use 'TLS' for the PullHttpChangeIngestor due to NoSuchAlgorithmException. Will attempt to use the default algorithm.", e);
+            tempSslContext = SSLContext.getDefault();
+        }
+
+        final SSLContext sslContext = tempSslContext;
+        sslContext.init(keyManagerFactory.getKeyManagers(), trustManagerFactory.getTrustManagers(), null);
+
+        final SSLSocketFactory sslSocketFactory = sslContext.getSocketFactory();
+        okHttpClientBuilder.sslSocketFactory(sslSocketFactory, x509TrustManager);
+    }
+
+    private void assertKeystorePropertiesSet(String location, String password, String type) {
+        if (location == null || location.isEmpty()) {
+            throw new IllegalArgumentException(clientConfig.getKeystoreFilename() + " is null or is empty");
+        }
+
+        if (password == null || password.isEmpty()) {
+            throw new IllegalArgumentException("The client's keystore filename is set but its password is not (or is empty). If the location is set, the password must also be.");
+        }
+
+        if (type == null || type.isEmpty()) {
+            throw new IllegalArgumentException("The client's keystore filename is set but its type is not (or is empty). If the location is set, the type must also be.");
+        }
+    }
+
+    private void assertTruststorePropertiesSet(String location, String password, String type) {
+        if (location == null || location.isEmpty()) {
+            throw new IllegalArgumentException("The client's truststore filename is not set or is empty");
+        }
+
+        if (password == null || password.isEmpty()) {
+            throw new IllegalArgumentException("The client's truststore filename is set but its password is not (or is empty). If the location is set, the password must also be.");
+        }
+
+        if (type == null || type.isEmpty()) {
+            throw new IllegalArgumentException("The client's truststore filename is set but its type is not (or is empty). If the location is set, the type must also be.");
+        }
+    }
+
+    @Override
+    public ByteBuffer retrieveUpdateContent(String flowUpdateUrl) {
+        final Request.Builder requestBuilder = new Request.Builder()
+                .get()
+                .url(flowUpdateUrl);
+        final Request request = requestBuilder.build();
+
+        ResponseBody body;
+        try (final Response response = httpClientReference.get().newCall(request).execute()) {
+            logger.debug("Response received: {}", response);
+
+            int code = response.code();
+            if (code >= 400) {
+                throw new IOException("Got response code " + code + " while trying to pull configuration: " + response.body().string());
+            }
+
+            body = response.body();
+
+            if (body == null) {
+                logger.warn("No body returned when pulling a new configuration");
+                return null;
+            }
+
+            final ByteBuffer bodyByteBuffer = ByteBuffer.wrap(body.bytes());
+            return bodyByteBuffer;
+
+        } catch (Exception e) {
+            logger.warn("Hit an exception while trying to pull", e);
+            return null;
+        }
+    }
+
+    @Override
+    public void acknowledgeOperation(C2OperationAck operationAck) {
+        logger.info("Performing acknowledgement request to {} for operation {}", clientConfig.getC2AckUrl(), operationAck.getOperationId());
+        final ObjectMapper jacksonObjectMapper = new ObjectMapper();
+        jacksonObjectMapper.setSerializationInclusion(JsonInclude.Include.NON_NULL);
+        try {
+            final String operationAckBody = jacksonObjectMapper.writeValueAsString(operationAck);
+
+            final RequestBody requestBody = RequestBody.create(operationAckBody, MediaType.parse("application/json"));

Review Comment:
   We actually already have a constant for this called MEDIA_TYPE_APPLICATION_JSON but it looks like this place was missed. Corrected.



##########
c2/c2-client-bundle/c2-client-service/src/main/java/org/apache/nifi/c2/client/service/C2HeartbeatFactory.java:
##########
@@ -0,0 +1,248 @@
+/*
+ * 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.nifi.c2.client.service;
+
+import static org.apache.commons.lang3.StringUtils.isNotBlank;
+
+import java.io.File;
+import java.lang.management.ManagementFactory;
+import java.lang.management.OperatingSystemMXBean;
+import java.net.InetAddress;
+import java.net.NetworkInterface;
+import java.util.Enumeration;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.nifi.c2.client.C2ClientConfig;
+import org.apache.nifi.c2.client.PersistentUuidGenerator;
+import org.apache.nifi.c2.client.service.model.RuntimeInfoWrapper;
+import org.apache.nifi.c2.protocol.api.AgentInfo;
+import org.apache.nifi.c2.protocol.api.AgentRepositories;
+import org.apache.nifi.c2.protocol.api.AgentStatus;
+import org.apache.nifi.c2.protocol.api.C2Heartbeat;
+import org.apache.nifi.c2.protocol.api.DeviceInfo;
+import org.apache.nifi.c2.protocol.api.FlowInfo;
+import org.apache.nifi.c2.protocol.api.FlowQueueStatus;
+import org.apache.nifi.c2.protocol.api.NetworkInfo;
+import org.apache.nifi.c2.protocol.api.SystemInfo;
+import org.apache.nifi.c2.protocol.component.api.RuntimeManifest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class C2HeartbeatFactory {
+
+    private static final Logger logger = LoggerFactory.getLogger(C2HeartbeatFactory.class);
+
+    private static final String AGENT_IDENTIFIER_FILENAME = "agent-identifier";
+    private static final String DEVICE_IDENTIFIER_FILENAME = "device-identifier";
+
+    private final C2ClientConfig clientConfig;
+    private final FlowIdHolder flowIdHolder;
+
+    private String agentId;
+    private String deviceId;
+    private File confDirectory;
+
+    public C2HeartbeatFactory(C2ClientConfig clientConfig, FlowIdHolder flowIdHolder) {
+        this.clientConfig = clientConfig;
+        this.flowIdHolder = flowIdHolder;
+    }
+
+    public C2Heartbeat create(RuntimeInfoWrapper runtimeInfoWrapper) {
+        C2Heartbeat heartbeat = new C2Heartbeat();
+
+        heartbeat.setAgentInfo(getAgentInfo(runtimeInfoWrapper.getAgentRepositories(), runtimeInfoWrapper.getManifest()));
+        heartbeat.setDeviceInfo(generateDeviceInfo());
+        heartbeat.setFlowInfo(getFlowInfo(runtimeInfoWrapper.getQueueStatus()));
+        heartbeat.setCreated(System.currentTimeMillis());
+
+        return heartbeat;
+    }
+
+    //////
+    // FlowInfo
+    //////

Review Comment:
   Removed



##########
c2/c2-client-bundle/c2-client-http/src/main/java/org/apache/nifi/c2/client/http/C2HttpClient.java:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.nifi.c2.client.http;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.security.KeyStore;
+import java.security.NoSuchAlgorithmException;
+import java.util.Optional;
+import java.util.concurrent.atomic.AtomicReference;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLSocketFactory;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+import okhttp3.MediaType;
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.RequestBody;
+import okhttp3.Response;
+import okhttp3.ResponseBody;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.c2.client.C2ClientConfig;
+import org.apache.nifi.c2.client.api.C2Client;
+import org.apache.nifi.c2.client.api.C2Serializer;
+import org.apache.nifi.c2.protocol.api.C2Heartbeat;
+import org.apache.nifi.c2.protocol.api.C2HeartbeatResponse;
+import org.apache.nifi.c2.protocol.api.C2OperationAck;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class C2HttpClient implements C2Client {
+
+    private static final Logger logger = LoggerFactory.getLogger(C2HttpClient.class);
+    private static final MediaType MEDIA_TYPE_APPLICATION_JSON = MediaType.parse("application/json");
+
+    private final AtomicReference<OkHttpClient> httpClientReference = new AtomicReference<>();
+    private final C2ClientConfig clientConfig;
+    private final C2Serializer serializer;
+
+    public C2HttpClient(C2ClientConfig clientConfig, C2Serializer serializer) {
+        super();
+        this.clientConfig = clientConfig;
+        this.serializer = serializer;
+        final OkHttpClient.Builder okHttpClientBuilder = new OkHttpClient.Builder();
+
+        // Set whether to follow redirects
+        okHttpClientBuilder.followRedirects(true);
+
+        // check if the ssl path is set and add the factory if so
+        if (StringUtils.isNotBlank(clientConfig.getKeystoreFilename())) {
+            try {
+                setSslSocketFactory(okHttpClientBuilder);
+            } catch (Exception e) {
+                throw new IllegalStateException(e);
+            }
+        }
+
+        httpClientReference.set(okHttpClientBuilder.build());
+    }
+
+    @Override
+    public Optional<C2HeartbeatResponse> publishHeartbeat(C2Heartbeat heartbeat) {
+        return serializer.serialize(heartbeat).flatMap(this::sendHeartbeat);
+    }
+
+    private Optional<C2HeartbeatResponse> sendHeartbeat(String heartbeat) {
+        logger.debug("Sending heartbeat to {}", clientConfig.getC2Url());
+
+        Optional<C2HeartbeatResponse> c2HeartbeatResponse = Optional.empty();
+        Request request = new Request.Builder()
+            .post(RequestBody.create(heartbeat, MEDIA_TYPE_APPLICATION_JSON))
+            .url(clientConfig.getC2Url())
+            .build();
+
+        try {
+            Response heartbeatResponse = httpClientReference.get().newCall(request).execute();
+            c2HeartbeatResponse = getResponseBody(heartbeatResponse).flatMap(response -> serializer.deserialize(response, C2HeartbeatResponse.class));
+        } catch (IOException ce) {
+            logger.error("Error while sending heartbeat", ce);
+        }
+
+        return c2HeartbeatResponse;
+    }
+
+    private Optional<String> getResponseBody(Response response) {
+        String responseBody = null;
+
+        try {
+            responseBody = response.body().string();
+            logger.debug("Received response (Status={}) {}", response.code(), responseBody);
+        } catch (IOException e) {
+            logger.error("Could not get response body: ", e);
+        }
+
+        return Optional.ofNullable(responseBody);
+    }
+
+    private void setSslSocketFactory(OkHttpClient.Builder okHttpClientBuilder) throws Exception {
+        final String keystoreLocation = clientConfig.getKeystoreFilename();
+        final String keystoreType = clientConfig.getKeystoreType().getType();
+        final String keystorePass = clientConfig.getKeystorePass();
+
+        assertKeystorePropertiesSet(keystoreLocation, keystorePass, keystoreType);
+
+        // prepare the keystore
+        final KeyStore keyStore = KeyStore.getInstance(keystoreType);
+
+        try (FileInputStream keyStoreStream = new FileInputStream(keystoreLocation)) {
+            keyStore.load(keyStoreStream, keystorePass.toCharArray());
+        }
+
+        final KeyManagerFactory keyManagerFactory = KeyManagerFactory.getInstance(KeyManagerFactory.getDefaultAlgorithm());
+        keyManagerFactory.init(keyStore, keystorePass.toCharArray());
+
+        // load truststore
+        final String truststoreLocation = clientConfig.getTruststoreFilename();
+        final String truststorePass = clientConfig.getTruststorePass();
+        final String truststoreType = clientConfig.getTruststoreType().getType();
+        assertTruststorePropertiesSet(truststoreLocation, truststorePass, truststoreType);
+
+        KeyStore truststore = KeyStore.getInstance(truststoreType);
+        final TrustManagerFactory trustManagerFactory = TrustManagerFactory.getInstance("X509");
+        truststore.load(new FileInputStream(truststoreLocation), truststorePass.toCharArray());
+        trustManagerFactory.init(truststore);
+
+        final X509TrustManager x509TrustManager;
+        TrustManager[] trustManagers = trustManagerFactory.getTrustManagers();
+        if (trustManagers[0] != null) {
+            x509TrustManager = (X509TrustManager) trustManagers[0];
+        } else {
+            throw new IllegalStateException("List of trust managers is null");
+        }
+
+        SSLContext tempSslContext;
+        try {
+            tempSslContext = SSLContext.getInstance("TLS");
+        } catch (NoSuchAlgorithmException e) {
+            logger.warn("Unable to use 'TLS' for the PullHttpChangeIngestor due to NoSuchAlgorithmException. Will attempt to use the default algorithm.", e);
+            tempSslContext = SSLContext.getDefault();
+        }
+
+        final SSLContext sslContext = tempSslContext;
+        sslContext.init(keyManagerFactory.getKeyManagers(), trustManagerFactory.getTrustManagers(), null);
+
+        final SSLSocketFactory sslSocketFactory = sslContext.getSocketFactory();
+        okHttpClientBuilder.sslSocketFactory(sslSocketFactory, x509TrustManager);
+    }
+
+    private void assertKeystorePropertiesSet(String location, String password, String type) {
+        if (location == null || location.isEmpty()) {
+            throw new IllegalArgumentException(clientConfig.getKeystoreFilename() + " is null or is empty");
+        }
+
+        if (password == null || password.isEmpty()) {
+            throw new IllegalArgumentException("The client's keystore filename is set but its password is not (or is empty). If the location is set, the password must also be.");
+        }
+
+        if (type == null || type.isEmpty()) {
+            throw new IllegalArgumentException("The client's keystore filename is set but its type is not (or is empty). If the location is set, the type must also be.");
+        }
+    }
+
+    private void assertTruststorePropertiesSet(String location, String password, String type) {
+        if (location == null || location.isEmpty()) {
+            throw new IllegalArgumentException("The client's truststore filename is not set or is empty");
+        }
+
+        if (password == null || password.isEmpty()) {
+            throw new IllegalArgumentException("The client's truststore filename is set but its password is not (or is empty). If the location is set, the password must also be.");
+        }
+
+        if (type == null || type.isEmpty()) {
+            throw new IllegalArgumentException("The client's truststore filename is set but its type is not (or is empty). If the location is set, the type must also be.");
+        }
+    }
+
+    @Override
+    public ByteBuffer retrieveUpdateContent(String flowUpdateUrl) {
+        final Request.Builder requestBuilder = new Request.Builder()
+                .get()
+                .url(flowUpdateUrl);
+        final Request request = requestBuilder.build();
+
+        ResponseBody body;
+        try (final Response response = httpClientReference.get().newCall(request).execute()) {
+            logger.debug("Response received: {}", response);
+
+            int code = response.code();
+            if (code >= 400) {
+                throw new IOException("Got response code " + code + " while trying to pull configuration: " + response.body().string());
+            }
+
+            body = response.body();
+
+            if (body == null) {
+                logger.warn("No body returned when pulling a new configuration");
+                return null;
+            }
+
+            final ByteBuffer bodyByteBuffer = ByteBuffer.wrap(body.bytes());
+            return bodyByteBuffer;
+
+        } catch (Exception e) {
+            logger.warn("Hit an exception while trying to pull", e);
+            return null;
+        }
+    }
+
+    @Override
+    public void acknowledgeOperation(C2OperationAck operationAck) {
+        logger.info("Performing acknowledgement request to {} for operation {}", clientConfig.getC2AckUrl(), operationAck.getOperationId());
+        final ObjectMapper jacksonObjectMapper = new ObjectMapper();
+        jacksonObjectMapper.setSerializationInclusion(JsonInclude.Include.NON_NULL);
+        try {
+            final String operationAckBody = jacksonObjectMapper.writeValueAsString(operationAck);
+
+            final RequestBody requestBody = RequestBody.create(operationAckBody, MediaType.parse("application/json"));
+            final Request.Builder requestBuilder = new Request.Builder()
+                    .post(requestBody)
+                    .url(clientConfig.getC2AckUrl());
+            final Response heartbeatResponse = httpClientReference.get().newCall(requestBuilder.build()).execute();
+            if (!heartbeatResponse.isSuccessful()) {
+                logger.warn("Acknowledgement was not successful.");
+            }
+            logger.trace("Status on acknowledgement was {}", heartbeatResponse.code());

Review Comment:
   I removed this particular log (but added status elsewhere) also with the okhttp logging interceptor this should be covered.



##########
c2/c2-client-bundle/c2-client-service/src/main/java/org/apache/nifi/c2/client/service/C2ClientService.java:
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.nifi.c2.client.service;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.List;
+import java.util.function.Function;
+import org.apache.nifi.c2.client.api.C2Client;
+import org.apache.nifi.c2.client.service.model.RuntimeInfoWrapper;
+import org.apache.nifi.c2.client.service.operation.C2OperationService;
+import org.apache.nifi.c2.client.service.operation.UpdateConfigurationOperationHandler;
+import org.apache.nifi.c2.protocol.api.C2Heartbeat;
+import org.apache.nifi.c2.protocol.api.C2HeartbeatResponse;
+import org.apache.nifi.c2.protocol.api.C2Operation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class C2ClientService {
+
+    private static final Logger logger = LoggerFactory.getLogger(C2ClientService.class);
+
+    private final C2Client client;
+    private final C2HeartbeatFactory c2HeartbeatFactory;
+    private final C2OperationService operationService;
+    private final UpdateConfigurationOperationHandler updateConfigurationOperationHandler;
+
+    public C2ClientService(C2Client client, C2HeartbeatFactory c2HeartbeatFactory, FlowIdHolder flowIdHolder, Function<ByteBuffer, Boolean> updateFlow) {
+        this.client = client;
+        this.c2HeartbeatFactory = c2HeartbeatFactory;
+        this.updateConfigurationOperationHandler = new UpdateConfigurationOperationHandler(client, flowIdHolder, updateFlow);
+        this.operationService = new C2OperationService(Arrays.asList(updateConfigurationOperationHandler));
+    }
+
+    public void sendHeartbeat(RuntimeInfoWrapper runtimeInfoWrapper) {
+        try {
+            // TODO exception handling for all the C2 Client interactions (IOExceptions, logger.error vs logger.warn, etc.)
+            C2Heartbeat c2Heartbeat = c2HeartbeatFactory.create(runtimeInfoWrapper);
+            client.publishHeartbeat(c2Heartbeat).ifPresent(this::processResponse);
+        } catch (IOException ioe) {
+            // TODO

Review Comment:
   These TODOs were left here from the "original approach" after the rewrites they had no point I removed them (for example as the function returns optional with the latest implementation there was no exception thrown so the try/catch became obsolete. I removed it. Thanks.



##########
c2/c2-client-bundle/c2-client-service/src/main/java/org/apache/nifi/c2/client/service/C2HeartbeatFactory.java:
##########
@@ -0,0 +1,248 @@
+/*
+ * 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.nifi.c2.client.service;
+
+import static org.apache.commons.lang3.StringUtils.isNotBlank;
+
+import java.io.File;
+import java.lang.management.ManagementFactory;
+import java.lang.management.OperatingSystemMXBean;
+import java.net.InetAddress;
+import java.net.NetworkInterface;
+import java.util.Enumeration;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.nifi.c2.client.C2ClientConfig;
+import org.apache.nifi.c2.client.PersistentUuidGenerator;
+import org.apache.nifi.c2.client.service.model.RuntimeInfoWrapper;
+import org.apache.nifi.c2.protocol.api.AgentInfo;
+import org.apache.nifi.c2.protocol.api.AgentRepositories;
+import org.apache.nifi.c2.protocol.api.AgentStatus;
+import org.apache.nifi.c2.protocol.api.C2Heartbeat;
+import org.apache.nifi.c2.protocol.api.DeviceInfo;
+import org.apache.nifi.c2.protocol.api.FlowInfo;
+import org.apache.nifi.c2.protocol.api.FlowQueueStatus;
+import org.apache.nifi.c2.protocol.api.NetworkInfo;
+import org.apache.nifi.c2.protocol.api.SystemInfo;
+import org.apache.nifi.c2.protocol.component.api.RuntimeManifest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class C2HeartbeatFactory {
+
+    private static final Logger logger = LoggerFactory.getLogger(C2HeartbeatFactory.class);
+
+    private static final String AGENT_IDENTIFIER_FILENAME = "agent-identifier";
+    private static final String DEVICE_IDENTIFIER_FILENAME = "device-identifier";
+
+    private final C2ClientConfig clientConfig;
+    private final FlowIdHolder flowIdHolder;
+
+    private String agentId;
+    private String deviceId;
+    private File confDirectory;
+
+    public C2HeartbeatFactory(C2ClientConfig clientConfig, FlowIdHolder flowIdHolder) {
+        this.clientConfig = clientConfig;
+        this.flowIdHolder = flowIdHolder;
+    }
+
+    public C2Heartbeat create(RuntimeInfoWrapper runtimeInfoWrapper) {
+        C2Heartbeat heartbeat = new C2Heartbeat();
+
+        heartbeat.setAgentInfo(getAgentInfo(runtimeInfoWrapper.getAgentRepositories(), runtimeInfoWrapper.getManifest()));
+        heartbeat.setDeviceInfo(generateDeviceInfo());
+        heartbeat.setFlowInfo(getFlowInfo(runtimeInfoWrapper.getQueueStatus()));
+        heartbeat.setCreated(System.currentTimeMillis());
+
+        return heartbeat;
+    }
+
+    //////
+    // FlowInfo
+    //////
+
+    private FlowInfo getFlowInfo(Map<String, FlowQueueStatus> queueStatus) {
+        FlowInfo flowInfo = new FlowInfo();
+        flowInfo.setQueues(queueStatus);
+        Optional.ofNullable(flowIdHolder.getFlowId()).ifPresent(flowInfo::setFlowId);
+        return flowInfo;
+    }
+
+    //////
+    // AgentInfo
+    //////
+
+    private AgentInfo getAgentInfo(AgentRepositories repos, RuntimeManifest manifest) {
+        AgentInfo agentInfo = new AgentInfo();
+        agentInfo.setAgentClass(clientConfig.getAgentClass());
+        agentInfo.setIdentifier(getAgentId());
+
+        AgentStatus agentStatus = new AgentStatus();
+        // TODO: Look into this, it doesn't look right
+        agentStatus.setUptime(System.currentTimeMillis());
+        agentStatus.setRepositories(repos);
+
+        agentInfo.setStatus(agentStatus);
+        agentInfo.setAgentManifest(manifest);
+
+        return agentInfo;
+    }
+
+    private String getAgentId() {
+        if (agentId == null) {
+            String rawAgentId = clientConfig.getAgentIdentifier();
+            if (isNotBlank(rawAgentId)) {
+                agentId = rawAgentId.trim();
+            } else {
+                File idFile = new File(getConfDirectory(), AGENT_IDENTIFIER_FILENAME);
+                agentId = new PersistentUuidGenerator(idFile).generate();
+            }
+        }
+
+        return agentId;
+    }
+
+    //////
+    // DeviceInfo
+    //////
+
+    private DeviceInfo generateDeviceInfo() {
+        // Populate DeviceInfo
+        final DeviceInfo deviceInfo = new DeviceInfo();
+        deviceInfo.setNetworkInfo(generateNetworkInfo());
+        deviceInfo.setIdentifier(getDeviceIdentifier(deviceInfo.getNetworkInfo()));
+        deviceInfo.setSystemInfo(generateSystemInfo());
+        return deviceInfo;
+    }
+
+    private NetworkInfo generateNetworkInfo() {
+        NetworkInfo networkInfo = new NetworkInfo();
+        try {
+            // Determine all interfaces
+            final Enumeration<NetworkInterface> networkInterfaces = NetworkInterface.getNetworkInterfaces();
+
+            final Set<NetworkInterface> operationIfaces = new HashSet<>();
+
+            // Determine eligible interfaces
+            while (networkInterfaces.hasMoreElements()) {
+                final NetworkInterface networkInterface = networkInterfaces.nextElement();
+                if (!networkInterface.isLoopback() && networkInterface.isUp()) {
+                    operationIfaces.add(networkInterface);
+                }
+            }
+            logger.trace("Have {} interfaces with names {}", operationIfaces.size(),
+                operationIfaces.stream()
+                    .map(NetworkInterface::getName)
+                    .collect(Collectors.toSet())
+            );
+
+            if (!operationIfaces.isEmpty()) {
+                if (operationIfaces.size() > 1) {
+                    logger.debug("Instance has multiple interfaces.  Generated information may be non-deterministic.");
+                }
+
+                NetworkInterface iface = operationIfaces.iterator().next();
+                final StringBuilder macSb = new StringBuilder();
+                byte[] hardwareAddress = iface.getHardwareAddress();
+                if (hardwareAddress != null) {
+                    for (int i = 0; i < hardwareAddress.length; i++) {
+                        macSb.append(String.format("%02X", hardwareAddress[i]));
+                    }
+                }
+                final String macString = macSb.toString();
+                Enumeration<InetAddress> inetAddresses = iface.getInetAddresses();
+                while (inetAddresses.hasMoreElements()) {
+                    InetAddress inetAddress = inetAddresses.nextElement();
+                    String hostAddress = inetAddress.getHostAddress();
+                    String hostName = inetAddress.getHostName();
+                    byte[] address = inetAddress.getAddress();
+                    String canonicalHostName = inetAddress.getCanonicalHostName();
+
+                    networkInfo.setDeviceId(iface.getName());
+                    networkInfo.setHostname(hostName);
+                    networkInfo.setIpAddress(hostAddress);
+                }
+            }
+        } catch (
+            Exception e) {
+            logger.error("Had exception determining network information", e);

Review Comment:
   Applied



##########
c2/c2-client-bundle/c2-client-service/src/main/java/org/apache/nifi/c2/client/service/FlowIdHolder.java:
##########
@@ -0,0 +1,85 @@
+/*
+ * 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.nifi.c2.client.service;
+
+import static java.util.Collections.singletonList;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.List;
+import org.apache.nifi.util.file.FileUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class FlowIdHolder {
+    private static final Logger LOGGER = LoggerFactory.getLogger(FlowIdHolder.class);
+    private static final String FLOW_IDENTIFIER_FILENAME = "flow-identifier";
+
+    private volatile String flowId;
+    private final String configDirectoryName;
+
+    public FlowIdHolder(String configDirectoryName) {
+        this.configDirectoryName = configDirectoryName;
+        this.flowId = readFlowId();
+    }
+
+    public String getFlowId() {
+        return flowId;
+    }
+
+    public void setFlowId(String flowId) {
+        this.flowId = flowId;
+        persistFlowId(flowId);
+    }
+
+    private void persistFlowId(String flowId) {
+        File flowIdFile = new File(configDirectoryName, FLOW_IDENTIFIER_FILENAME);
+        try {
+            FileUtils.ensureDirectoryExistAndCanAccess(flowIdFile.getParentFile());
+            saveFlowId(flowIdFile, flowId);
+        } catch (IOException e) {
+            LOGGER.error("Failed to save flow information due to: {}", e.getMessage());

Review Comment:
   applied



##########
c2/c2-client-bundle/c2-client-service/src/main/java/org/apache/nifi/c2/client/service/operation/C2OperationHandler.java:
##########
@@ -0,0 +1,31 @@
+/*
+ * 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.nifi.c2.client.service.operation;
+
+import org.apache.nifi.c2.protocol.api.C2Operation;
+import org.apache.nifi.c2.protocol.api.C2OperationAck;
+import org.apache.nifi.c2.protocol.api.OperandType;
+import org.apache.nifi.c2.protocol.api.OperationType;
+
+public interface C2OperationHandler {
+
+    OperationType getOperationType();
+
+    OperandType getOperandType();
+
+    C2OperationAck handle(C2Operation operation);

Review Comment:
   Added



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/util/ProcessUtils.java:
##########
@@ -0,0 +1,131 @@
+/*
+ * 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.nifi.minifi.bootstrap.util;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.Reader;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ProcessUtils {

Review Comment:
   @ferencerdei could you please take a look?



##########
c2/c2-client-bundle/c2-client-service/src/main/java/org/apache/nifi/c2/client/service/operation/UpdateConfigurationOperationHandler.java:
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.nifi.c2.client.service.operation;
+
+import static org.apache.commons.lang3.StringUtils.EMPTY;
+import static org.apache.nifi.c2.protocol.api.OperandType.CONFIGURATION;
+import static org.apache.nifi.c2.protocol.api.OperationType.UPDATE;
+
+import java.net.URI;
+import java.nio.ByteBuffer;
+import java.util.Optional;
+import java.util.function.Function;
+import org.apache.nifi.c2.client.api.C2Client;
+import org.apache.nifi.c2.client.service.FlowIdHolder;
+import org.apache.nifi.c2.protocol.api.C2Operation;
+import org.apache.nifi.c2.protocol.api.C2OperationAck;
+import org.apache.nifi.c2.protocol.api.C2OperationState;
+import org.apache.nifi.c2.protocol.api.OperandType;
+import org.apache.nifi.c2.protocol.api.OperationType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class UpdateConfigurationOperationHandler implements C2OperationHandler {
+
+    private static final Logger logger = LoggerFactory.getLogger(UpdateConfigurationOperationHandler.class);
+
+    private static final String LOCATION = "location";
+
+    private final C2Client client;
+    private final Function<ByteBuffer, Boolean> updateFlow;
+    private final FlowIdHolder flowIdHolder;
+
+    public UpdateConfigurationOperationHandler(C2Client client, FlowIdHolder flowIdHolder, Function<ByteBuffer, Boolean> updateFlow) {
+        this.client = client;
+        this.updateFlow = updateFlow;
+        this.flowIdHolder = flowIdHolder;
+    }
+
+    @Override
+    public OperationType getOperationType() {
+        return UPDATE;
+    }
+
+    @Override
+    public OperandType getOperandType() {
+        return CONFIGURATION;
+    }
+
+    @Override
+    public C2OperationAck handle(C2Operation operation) {
+        String opIdentifier = Optional.ofNullable(operation.getIdentifier())
+            .orElse(EMPTY);
+        C2OperationAck operationAck = new C2OperationAck();
+        C2OperationState state = new C2OperationState();
+        operationAck.setOperationState(state);
+        operationAck.setOperationId(opIdentifier);
+
+        String updateLocation = Optional.ofNullable(operation.getArgs())
+            .map(map -> map.get(LOCATION))
+            .orElse(EMPTY);
+
+        String newFlowId = parseFlowId(updateLocation);
+        if (flowIdHolder.getFlowId() == null || !flowIdHolder.getFlowId().equals(newFlowId)) {
+            logger.info("Will perform flow update from {} for operation #{}. Previous flow id was {}, replacing with new id {}", updateLocation, opIdentifier,
+                flowIdHolder.getFlowId() == null ? "not set" : flowIdHolder.getFlowId(), newFlowId);
+        } else {
+            logger.info("Flow is current, no update is necessary...");
+        }
+
+        flowIdHolder.setFlowId(newFlowId);
+        ByteBuffer updateContent = client.retrieveUpdateContent(updateLocation);
+        if (updateContent != null) {
+            if (updateFlow.apply(updateContent)) {
+                state.setState(C2OperationState.OperationState.FULLY_APPLIED);
+                logger.debug("Update configuration applied for operation #{}.", opIdentifier);
+            } else {
+                state.setState(C2OperationState.OperationState.NOT_APPLIED);
+                logger.error("Update resulted in error for operation #{}.", opIdentifier);
+            }
+        } else {
+            state.setState(C2OperationState.OperationState.NOT_APPLIED);
+            logger.error("Update content retrieval resulted in empty content so flow update was omitted for operation #{}.", opIdentifier);
+        }
+
+        return operationAck;
+    }
+
+    private String parseFlowId(String flowUpdateUrl) {
+        try {
+            URI flowUri = new URI(flowUpdateUrl);
+            String flowUriPath = flowUri.getPath();
+            String[] split = flowUriPath.split("/");
+            String flowId = split[4];

Review Comment:
   There is a change currently under development on the server side which will affect this so we will make it more robust when that will be ready. Till then I added the check as proposed.



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/command/EnvRunner.java:
##########
@@ -0,0 +1,107 @@
+/*
+ * 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.nifi.minifi.bootstrap.command;
+
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CMD_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.DEFAULT_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.Status.ERROR;
+import static org.apache.nifi.minifi.bootstrap.Status.MINIFI_NOT_RUNNING;
+import static org.apache.nifi.minifi.bootstrap.Status.OK;
+
+import java.lang.reflect.Method;
+import java.util.Map.Entry;
+import java.util.Properties;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.apache.nifi.minifi.bootstrap.MiNiFiStatus;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiStatusProvider;
+
+public class EnvRunner implements CommandRunner {
+    private final MiNiFiParameters miNiFiParameters;
+    private final MiNiFiStatusProvider miNiFiStatusProvider;
+
+    public EnvRunner(MiNiFiParameters miNiFiParameters, MiNiFiStatusProvider miNiFiStatusProvider) {
+        this.miNiFiParameters = miNiFiParameters;
+        this.miNiFiStatusProvider = miNiFiStatusProvider;
+    }
+
+    /**
+     * Returns information about the MiNiFi's virtual machine.
+     * @param args the input arguments
+     * @return status code
+     */
+    @Override
+    public int runCommand(String[] args) {
+        return env();
+    }
+
+    private int env() {
+        MiNiFiStatus status = miNiFiStatusProvider.getStatus(miNiFiParameters.getMiNiFiPort(), miNiFiParameters.getMinifiPid());
+        if (status.getPid() == null) {
+            CMD_LOGGER.error("Apache MiNiFi is not running");
+            return MINIFI_NOT_RUNNING.getStatusCode();
+        }
+
+        Class<?> virtualMachineClass;
+        try {
+            virtualMachineClass = Class.forName("com.sun.tools.attach.VirtualMachine");

Review Comment:
   Talked with @ferencerdei we will validate. 



##########
c2/c2-client-bundle/c2-client-service/src/main/java/org/apache/nifi/c2/client/service/C2HeartbeatFactory.java:
##########
@@ -0,0 +1,248 @@
+/*
+ * 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.nifi.c2.client.service;
+
+import static org.apache.commons.lang3.StringUtils.isNotBlank;
+
+import java.io.File;
+import java.lang.management.ManagementFactory;
+import java.lang.management.OperatingSystemMXBean;
+import java.net.InetAddress;
+import java.net.NetworkInterface;
+import java.util.Enumeration;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.nifi.c2.client.C2ClientConfig;
+import org.apache.nifi.c2.client.PersistentUuidGenerator;
+import org.apache.nifi.c2.client.service.model.RuntimeInfoWrapper;
+import org.apache.nifi.c2.protocol.api.AgentInfo;
+import org.apache.nifi.c2.protocol.api.AgentRepositories;
+import org.apache.nifi.c2.protocol.api.AgentStatus;
+import org.apache.nifi.c2.protocol.api.C2Heartbeat;
+import org.apache.nifi.c2.protocol.api.DeviceInfo;
+import org.apache.nifi.c2.protocol.api.FlowInfo;
+import org.apache.nifi.c2.protocol.api.FlowQueueStatus;
+import org.apache.nifi.c2.protocol.api.NetworkInfo;
+import org.apache.nifi.c2.protocol.api.SystemInfo;
+import org.apache.nifi.c2.protocol.component.api.RuntimeManifest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class C2HeartbeatFactory {
+
+    private static final Logger logger = LoggerFactory.getLogger(C2HeartbeatFactory.class);
+
+    private static final String AGENT_IDENTIFIER_FILENAME = "agent-identifier";
+    private static final String DEVICE_IDENTIFIER_FILENAME = "device-identifier";
+
+    private final C2ClientConfig clientConfig;
+    private final FlowIdHolder flowIdHolder;
+
+    private String agentId;
+    private String deviceId;
+    private File confDirectory;
+
+    public C2HeartbeatFactory(C2ClientConfig clientConfig, FlowIdHolder flowIdHolder) {
+        this.clientConfig = clientConfig;
+        this.flowIdHolder = flowIdHolder;
+    }
+
+    public C2Heartbeat create(RuntimeInfoWrapper runtimeInfoWrapper) {
+        C2Heartbeat heartbeat = new C2Heartbeat();
+
+        heartbeat.setAgentInfo(getAgentInfo(runtimeInfoWrapper.getAgentRepositories(), runtimeInfoWrapper.getManifest()));
+        heartbeat.setDeviceInfo(generateDeviceInfo());
+        heartbeat.setFlowInfo(getFlowInfo(runtimeInfoWrapper.getQueueStatus()));
+        heartbeat.setCreated(System.currentTimeMillis());
+
+        return heartbeat;
+    }
+
+    //////
+    // FlowInfo
+    //////
+
+    private FlowInfo getFlowInfo(Map<String, FlowQueueStatus> queueStatus) {
+        FlowInfo flowInfo = new FlowInfo();
+        flowInfo.setQueues(queueStatus);
+        Optional.ofNullable(flowIdHolder.getFlowId()).ifPresent(flowInfo::setFlowId);
+        return flowInfo;
+    }
+
+    //////
+    // AgentInfo
+    //////
+
+    private AgentInfo getAgentInfo(AgentRepositories repos, RuntimeManifest manifest) {
+        AgentInfo agentInfo = new AgentInfo();
+        agentInfo.setAgentClass(clientConfig.getAgentClass());
+        agentInfo.setIdentifier(getAgentId());
+
+        AgentStatus agentStatus = new AgentStatus();
+        // TODO: Look into this, it doesn't look right
+        agentStatus.setUptime(System.currentTimeMillis());
+        agentStatus.setRepositories(repos);
+
+        agentInfo.setStatus(agentStatus);
+        agentInfo.setAgentManifest(manifest);
+
+        return agentInfo;
+    }
+
+    private String getAgentId() {
+        if (agentId == null) {
+            String rawAgentId = clientConfig.getAgentIdentifier();
+            if (isNotBlank(rawAgentId)) {
+                agentId = rawAgentId.trim();
+            } else {
+                File idFile = new File(getConfDirectory(), AGENT_IDENTIFIER_FILENAME);
+                agentId = new PersistentUuidGenerator(idFile).generate();
+            }
+        }
+
+        return agentId;
+    }
+
+    //////
+    // DeviceInfo
+    //////
+
+    private DeviceInfo generateDeviceInfo() {
+        // Populate DeviceInfo
+        final DeviceInfo deviceInfo = new DeviceInfo();
+        deviceInfo.setNetworkInfo(generateNetworkInfo());
+        deviceInfo.setIdentifier(getDeviceIdentifier(deviceInfo.getNetworkInfo()));
+        deviceInfo.setSystemInfo(generateSystemInfo());
+        return deviceInfo;
+    }
+
+    private NetworkInfo generateNetworkInfo() {
+        NetworkInfo networkInfo = new NetworkInfo();
+        try {
+            // Determine all interfaces
+            final Enumeration<NetworkInterface> networkInterfaces = NetworkInterface.getNetworkInterfaces();
+
+            final Set<NetworkInterface> operationIfaces = new HashSet<>();
+
+            // Determine eligible interfaces
+            while (networkInterfaces.hasMoreElements()) {
+                final NetworkInterface networkInterface = networkInterfaces.nextElement();
+                if (!networkInterface.isLoopback() && networkInterface.isUp()) {
+                    operationIfaces.add(networkInterface);
+                }
+            }
+            logger.trace("Have {} interfaces with names {}", operationIfaces.size(),
+                operationIfaces.stream()
+                    .map(NetworkInterface::getName)
+                    .collect(Collectors.toSet())
+            );
+
+            if (!operationIfaces.isEmpty()) {
+                if (operationIfaces.size() > 1) {
+                    logger.debug("Instance has multiple interfaces.  Generated information may be non-deterministic.");
+                }
+
+                NetworkInterface iface = operationIfaces.iterator().next();
+                final StringBuilder macSb = new StringBuilder();
+                byte[] hardwareAddress = iface.getHardwareAddress();
+                if (hardwareAddress != null) {
+                    for (int i = 0; i < hardwareAddress.length; i++) {
+                        macSb.append(String.format("%02X", hardwareAddress[i]));
+                    }
+                }
+                final String macString = macSb.toString();
+                Enumeration<InetAddress> inetAddresses = iface.getInetAddresses();
+                while (inetAddresses.hasMoreElements()) {
+                    InetAddress inetAddress = inetAddresses.nextElement();
+                    String hostAddress = inetAddress.getHostAddress();
+                    String hostName = inetAddress.getHostName();
+                    byte[] address = inetAddress.getAddress();
+                    String canonicalHostName = inetAddress.getCanonicalHostName();
+
+                    networkInfo.setDeviceId(iface.getName());
+                    networkInfo.setHostname(hostName);
+                    networkInfo.setIpAddress(hostAddress);
+                }
+            }
+        } catch (
+            Exception e) {
+            logger.error("Had exception determining network information", e);
+        }
+        return networkInfo;
+    }
+
+    private String getDeviceIdentifier(NetworkInfo networkInfo) {
+        if (deviceId == null) {
+            if (networkInfo.getDeviceId() != null) {
+                try {
+                    final NetworkInterface netInterface = NetworkInterface.getByName(networkInfo.getDeviceId());
+                    byte[] hardwareAddress = netInterface.getHardwareAddress();
+                    final StringBuilder macBuilder = new StringBuilder();
+                    if (hardwareAddress != null) {
+                        for (byte address : hardwareAddress) {
+                            macBuilder.append(String.format("%02X", address));

Review Comment:
   Removed the code duplication as its result was not used.



##########
nifi-server-api/pom.xml:
##########
@@ -42,5 +42,10 @@
             <version>1.17.0-SNAPSHOT</version>
             <scope>provided</scope>
         </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>c2-client-api</artifactId>
+            <version>1.17.0-SNAPSHOT</version>
+        </dependency>

Review Comment:
   I think this is the result of a semi successful rebase, you are right it is not needed, removed.



##########
minifi/minifi-nar-bundles/minifi-framework-bundle/minifi-server-nar/pom.xml:
##########
@@ -43,6 +43,12 @@
             <artifactId>nifi-headless-server</artifactId>
             <version>1.17.0-SNAPSHOT</version>
         </dependency>
+        <dependency>
+            <groupId>org.apache.commons</groupId>
+            <artifactId>commons-lang3</artifactId>
+            <version>${org.apache.commons.lang3.version}</version>

Review Comment:
   Removed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] bejancsaba commented on a diff in pull request #6075: NIFI-9666 Implement base C2 client functionality

Posted by GitBox <gi...@apache.org>.
bejancsaba commented on code in PR #6075:
URL: https://github.com/apache/nifi/pull/6075#discussion_r891077674


##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/MiNiFiConfigurationChangeListener.java:
##########
@@ -0,0 +1,215 @@
+/*
+ * 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.nifi.minifi.bootstrap.service;
+
+import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CONF_DIR_KEY;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.MINIFI_CONFIG_FILE_KEY;
+import static org.apache.nifi.minifi.bootstrap.configuration.ingestors.PullHttpChangeIngestor.OVERRIDE_SECURITY;
+import static org.apache.nifi.minifi.bootstrap.configuration.ingestors.PullHttpChangeIngestor.PULL_HTTP_BASE_KEY;
+import static org.apache.nifi.minifi.bootstrap.util.ConfigTransformer.generateConfigFiles;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.util.Properties;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.commons.io.IOUtils;
+import org.apache.nifi.minifi.bootstrap.RunMiNiFi;
+import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeException;
+import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeListener;
+import org.apache.nifi.minifi.bootstrap.util.ByteBufferInputStream;
+import org.apache.nifi.minifi.bootstrap.util.ConfigTransformer;
+import org.apache.nifi.minifi.commons.schema.ConfigSchema;
+import org.apache.nifi.minifi.commons.schema.common.ConvertableSchema;
+import org.apache.nifi.minifi.commons.schema.serialization.SchemaLoader;
+import org.slf4j.Logger;
+
+public class MiNiFiConfigurationChangeListener implements ConfigurationChangeListener {
+
+    private final RunMiNiFi runner;
+    private final Logger logger;
+    private final BootstrapFileProvider bootstrapFileProvider;
+
+    private static final ReentrantLock handlingLock = new ReentrantLock();
+
+    public MiNiFiConfigurationChangeListener(RunMiNiFi runner, Logger logger, BootstrapFileProvider bootstrapFileProvider) {
+        this.runner = runner;
+        this.logger = logger;
+        this.bootstrapFileProvider = bootstrapFileProvider;
+    }
+
+    @Override
+    public void handleChange(InputStream configInputStream) throws ConfigurationChangeException {
+        logger.info("Received notification of a change");
+
+        if (!handlingLock.tryLock()) {
+            throw new ConfigurationChangeException("Instance is already handling another change");
+        }
+        // Store the incoming stream as a byte array to be shared among components that need it
+        try(ByteArrayOutputStream bufferedConfigOs = new ByteArrayOutputStream()) {
+
+            Properties bootstrapProperties = bootstrapFileProvider.getBootstrapProperties();
+            File configFile = new File(bootstrapProperties.getProperty(MINIFI_CONFIG_FILE_KEY));
+
+            byte[] copyArray = new byte[1024];
+            int available;
+            while ((available = configInputStream.read(copyArray)) > 0) {
+                bufferedConfigOs.write(copyArray, 0, available);
+            }
+
+            File swapConfigFile = bootstrapFileProvider.getSwapFile();
+            logger.info("Persisting old configuration to {}", swapConfigFile.getAbsolutePath());
+
+            try (FileInputStream configFileInputStream = new FileInputStream(configFile)) {
+                Files.copy(configFileInputStream, swapConfigFile.toPath(), REPLACE_EXISTING);
+            }
+
+            persistBackNonFlowSectionsFromOriginalSchema(bufferedConfigOs.toByteArray(), bootstrapProperties, configFile);
+
+            // Create an input stream to feed to the config transformer
+            try (ByteArrayInputStream newConfigBais = new ByteArrayInputStream(IOUtils.toByteArray(new FileInputStream(configFile)))) {
+                newConfigBais.mark(-1);
+
+                try {
+                    String confDir = bootstrapProperties.getProperty(CONF_DIR_KEY);
+
+                    try {
+                        logger.info("Performing transformation for input and saving outputs to {}", confDir);
+                        ByteBuffer tempConfigFile = generateConfigFiles(newConfigBais, confDir, bootstrapFileProvider.getBootstrapProperties());
+                        runner.getConfigFileReference().set(tempConfigFile.asReadOnlyBuffer());
+
+                        try {
+                            logger.info("Reloading instance with new configuration");
+                            restartInstance();
+                        } catch (Exception e) {
+                            logger.debug("Transformation of new config file failed after transformation into Flow.xml and nifi.properties, reverting.");
+                            try (FileInputStream swapConfigFileStream = new FileInputStream(swapConfigFile)) {
+                                ByteBuffer resetConfigFile = generateConfigFiles(swapConfigFileStream, confDir, bootstrapFileProvider.getBootstrapProperties());
+                                runner.getConfigFileReference().set(resetConfigFile.asReadOnlyBuffer());
+                            }
+                            throw e;
+                        }
+                    } catch (Exception e) {
+                        logger.debug("Transformation of new config file failed after replacing original with the swap file, reverting.");
+                        try (FileInputStream swapConfigFileStream = new FileInputStream(swapConfigFile)) {
+                            Files.copy(swapConfigFileStream, configFile.toPath(), REPLACE_EXISTING);
+                        }
+                        throw e;
+                    }
+                } catch (Exception e) {
+                    logger.debug("Transformation of new config file failed after swap file was created, deleting it.");
+                    if (!swapConfigFile.delete()) {
+                        logger.warn("The swap file failed to delete after a failed handling of a change. It should be cleaned up manually.");
+                    }
+                    throw e;
+                }
+            }
+        } catch (ConfigurationChangeException e){
+            logger.error("Unable to carry out reloading of configuration on receipt of notification event", e);
+            throw e;
+        } catch (IOException ioe) {
+            logger.error("Unable to carry out reloading of configuration on receipt of notification event", ioe);
+            throw new ConfigurationChangeException("Unable to perform reload of received configuration change", ioe);
+        } finally {
+            try {
+                if (configInputStream != null) {
+                    configInputStream.close() ;
+                }
+            } catch (IOException e) {
+                // Quietly close
+            }

Review Comment:
   Used IOUtils.closeQuitely, thanks for the suggestion, wasn't aware of that



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #6075: NIFI-9666 Implement base C2 client functionality

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on code in PR #6075:
URL: https://github.com/apache/nifi/pull/6075#discussion_r892995458


##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/command/EnvRunner.java:
##########
@@ -0,0 +1,107 @@
+/*
+ * 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.nifi.minifi.bootstrap.command;
+
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CMD_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.DEFAULT_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.Status.ERROR;
+import static org.apache.nifi.minifi.bootstrap.Status.MINIFI_NOT_RUNNING;
+import static org.apache.nifi.minifi.bootstrap.Status.OK;
+
+import java.lang.reflect.Method;
+import java.util.Map.Entry;
+import java.util.Properties;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.apache.nifi.minifi.bootstrap.MiNiFiStatus;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiStatusProvider;
+
+public class EnvRunner implements CommandRunner {
+    private final MiNiFiParameters miNiFiParameters;
+    private final MiNiFiStatusProvider miNiFiStatusProvider;
+
+    public EnvRunner(MiNiFiParameters miNiFiParameters, MiNiFiStatusProvider miNiFiStatusProvider) {
+        this.miNiFiParameters = miNiFiParameters;
+        this.miNiFiStatusProvider = miNiFiStatusProvider;
+    }
+
+    /**
+     * Returns information about the MiNiFi's virtual machine.
+     * @param args the input arguments
+     * @return status code
+     */
+    @Override
+    public int runCommand(String[] args) {
+        return env();
+    }
+
+    private int env() {
+        MiNiFiStatus status = miNiFiStatusProvider.getStatus(miNiFiParameters.getMiNiFiPort(), miNiFiParameters.getMinifiPid());
+        if (status.getPid() == null) {
+            CMD_LOGGER.error("Apache MiNiFi is not running");
+            return MINIFI_NOT_RUNNING.getStatusCode();
+        }
+
+        Class<?> virtualMachineClass;
+        try {
+            virtualMachineClass = Class.forName("com.sun.tools.attach.VirtualMachine");

Review Comment:
   NiFi and NiFi Registry support Java 17, so MiNiFi should maintain compatibility. Using this approach does not seem maintainable going forward. Is the general purpose of this class to log the System properties from the MiNiFi instance? It seems like some other approach to serialize and transmit the information would be much more maintainable and supported across Java versions.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] kevdoran commented on pull request #6075: NIFI-9666 Implement base C2 client functionality

Posted by GitBox <gi...@apache.org>.
kevdoran commented on PR #6075:
URL: https://github.com/apache/nifi/pull/6075#issuecomment-1155229599

   @ferencerdei - thank you for pointing that out. It was indeed the difference between the dockerhub and dockermaven images. No concerns there. I have not found anything else so far in functional testing, so I should finish my review of this soon. Thanks!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] bejancsaba commented on a diff in pull request #6075: NIFI-9666 Implement base C2 client functionality

Posted by GitBox <gi...@apache.org>.
bejancsaba commented on code in PR #6075:
URL: https://github.com/apache/nifi/pull/6075#discussion_r899970325


##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/PeriodicStatusReporterManager.java:
##########
@@ -0,0 +1,130 @@
+/*
+ * 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.nifi.minifi.bootstrap.service;
+
+import static org.apache.nifi.minifi.commons.schema.common.BootstrapPropertyKeys.STATUS_REPORTER_COMPONENTS_KEY;
+
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.apache.nifi.minifi.bootstrap.MiNiFiStatus;
+import org.apache.nifi.minifi.bootstrap.QueryableStatusAggregator;
+import org.apache.nifi.minifi.bootstrap.status.PeriodicStatusReporter;
+import org.apache.nifi.minifi.commons.status.FlowStatusReport;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PeriodicStatusReporterManager implements QueryableStatusAggregator {
+    private static final Logger LOGGER = LoggerFactory.getLogger(PeriodicStatusReporterManager.class);
+    private static final String FLOW_STATUS_REPORT_CMD = "FLOW_STATUS_REPORT";
+
+    private final Properties bootstrapProperties;
+    private final MiNiFiStatusProvider miNiFiStatusProvider;
+    private final MiNiFiCommandSender miNiFiCommandSender;
+    private final MiNiFiParameters miNiFiParameters;
+
+    private Set<PeriodicStatusReporter> periodicStatusReporters = Collections.emptySet();
+
+    public PeriodicStatusReporterManager(Properties bootstrapProperties, MiNiFiStatusProvider miNiFiStatusProvider, MiNiFiCommandSender miNiFiCommandSender,
+        MiNiFiParameters miNiFiParameters) {
+        this.bootstrapProperties = bootstrapProperties;
+        this.miNiFiStatusProvider = miNiFiStatusProvider;
+        this.miNiFiCommandSender = miNiFiCommandSender;
+        this.miNiFiParameters = miNiFiParameters;
+    }
+
+    public void startPeriodicNotifiers() {
+        periodicStatusReporters = initializePeriodicNotifiers();
+
+        for (PeriodicStatusReporter periodicStatusReporter: periodicStatusReporters) {
+            periodicStatusReporter.start();
+            LOGGER.debug("Started {} notifier", periodicStatusReporter.getClass().getCanonicalName());
+        }
+    }
+
+    public void shutdownPeriodicStatusReporters() {
+        LOGGER.debug("Initiating shutdown of bootstrap periodic status reporters...");
+        for (PeriodicStatusReporter periodicStatusReporter : periodicStatusReporters) {
+            try {
+                periodicStatusReporter.stop();
+            } catch (Exception exception) {
+                LOGGER.error("Could not successfully stop periodic status reporter " + periodicStatusReporter.getClass() + " due to ", exception);
+            }
+        }
+    }
+
+    public FlowStatusReport statusReport(String statusRequest) {
+        MiNiFiStatus status = miNiFiStatusProvider.getStatus(miNiFiParameters.getMiNiFiPort(), miNiFiParameters.getMinifiPid());
+
+        List<String> problemsGeneratingReport = new LinkedList<>();
+        if (!status.isProcessRunning()) {
+            problemsGeneratingReport.add("MiNiFi process is not running");
+        }
+
+        if (!status.isRespondingToPing()) {
+            problemsGeneratingReport.add("MiNiFi process is not responding to pings");
+        }
+
+        if (!problemsGeneratingReport.isEmpty()) {
+            FlowStatusReport flowStatusReport = new FlowStatusReport();
+            flowStatusReport.setErrorsGeneratingReport(problemsGeneratingReport);
+            return flowStatusReport;
+        }
+
+        return getFlowStatusReport(statusRequest, status.getPort());
+    }
+
+    private Set<PeriodicStatusReporter> initializePeriodicNotifiers() {
+        LOGGER.debug("Initiating bootstrap periodic status reporters...");
+        Set<PeriodicStatusReporter> statusReporters = new HashSet<>();
+
+        String reportersCsv = bootstrapProperties.getProperty(STATUS_REPORTER_COMPONENTS_KEY);
+
+        if (reportersCsv != null && !reportersCsv.isEmpty()) {
+            for (String reporterClassname : reportersCsv.split(",")) {
+                try {
+                    Class<?> reporterClass = Class.forName(reporterClassname);
+                    PeriodicStatusReporter reporter = (PeriodicStatusReporter) reporterClass.newInstance();
+                    reporter.initialize(bootstrapProperties, this);
+                    statusReporters.add(reporter);
+                    LOGGER.debug("Initialized {} notifier", reporterClass.getCanonicalName());
+                } catch (InstantiationException | IllegalAccessException | ClassNotFoundException e) {
+                    throw new RuntimeException("Issue instantiating notifier " + reporterClassname, e);
+                }
+            }
+        }
+        return statusReporters;
+    }
+
+    private FlowStatusReport getFlowStatusReport(String statusRequest, int port) {
+        FlowStatusReport flowStatusReport;
+        try {
+            flowStatusReport = miNiFiCommandSender.sendCommandForObject(FLOW_STATUS_REPORT_CMD, port, FlowStatusReport.class, statusRequest);
+        } catch (Exception e) {
+            flowStatusReport = new FlowStatusReport();
+            String message = "Failed to get status report from MiNiFi due to:" + e.getMessage();
+            flowStatusReport.setErrorsGeneratingReport(Collections.singletonList(message));
+            LOGGER.error(message);

Review Comment:
   Added



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] mattyb149 commented on pull request #6075: NIFI-9666 Implement base C2 client functionality

Posted by GitBox <gi...@apache.org>.
mattyb149 commented on PR #6075:
URL: https://github.com/apache/nifi/pull/6075#issuecomment-1136027696

   To set up the MiNiFi Java agent's C2 capability, add the following section to bootstrap.conf:
   
   ```
   nifi.minifi.notifier.ingestors=org.apache.nifi.minifi.bootstrap.configuration.ingestors.FileChangeIngestor
   nifi.minifi.notifier.ingestors.file.config.path=./conf/config-new.yml
   nifi.minifi.notifier.ingestors.file.polling.period.seconds=5
   c2.enable=true
   c2.config.directory=./conf
   c2.runtime.manifest.identifier=minifi
   c2.runtime.type=minifi-java
   c2.rest.url=http://localhost:10090/c2/config/api/c2-protocol/heartbeat
   c2.rest.url.ack=http://localhost:10090/c2/config/api/c2-protocol/acknowledge
   c2.agent.heartbeat.period=5000
   c2.agent.identifier=123-456-789
   c2.agent.class=java
   ```
   
   This sets up a FileChangeIngestor to watch for changes in conf/config-new.yml, which is where the agent currently persists the flow update coming from a C2 server. Under NIFI-9667 (and NIFI-9428 at a higher level) the agent will restart the flow without involving a separate file and FileChangeIngestor.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] bejancsaba commented on a diff in pull request #6075: NIFI-9666 Implement base C2 client functionality

Posted by GitBox <gi...@apache.org>.
bejancsaba commented on code in PR #6075:
URL: https://github.com/apache/nifi/pull/6075#discussion_r891124737


##########
minifi/pom.xml:
##########
@@ -44,6 +44,7 @@ limitations under the License.
         <system.rules.version>1.16.1</system.rules.version>
         <aws.sdk.version>1.11.172</aws.sdk.version>
         <yammer.metrics.version>2.2.0</yammer.metrics.version>
+        <okhttp.version>4.9.3</okhttp.version>

Review Comment:
   Removed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] bejancsaba commented on a diff in pull request #6075: NIFI-9666 Implement base C2 client functionality

Posted by GitBox <gi...@apache.org>.
bejancsaba commented on code in PR #6075:
URL: https://github.com/apache/nifi/pull/6075#discussion_r891074623


##########
c2/c2-client-bundle/c2-client-api/src/main/java/org/apache/nifi/c2/client/api/C2Client.java:
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.nifi.c2.client.api;
+
+import java.util.Optional;
+import org.apache.nifi.c2.protocol.api.C2Heartbeat;
+import org.apache.nifi.c2.protocol.api.C2HeartbeatResponse;
+import org.apache.nifi.c2.protocol.api.C2OperationAck;
+
+/**
+ * Defines interface methods used to implement a C2 Client. The controller can be application-specific but is used for such tasks as updating the flow.
+ */
+public interface C2Client {
+
+    /**
+     * Responsible for sending the C2Heartbeat to the C2 Server
+     *
+     * @param heartbeat the heartbeat to be sent
+     * @return optional response from the C2 Server if the response arrived it will be populated
+     */
+    Optional<C2HeartbeatResponse> publishHeartbeat(C2Heartbeat heartbeat);
+
+    /**
+     * Retrive the content of the new flow from the C2 Server
+     *
+     * @param flowUpdateUrl url where the content should be downloaded from
+     * @return the actual downloaded content. Will be null if no content can be downloaded
+     */
+    byte[] retrieveUpdateContent(String flowUpdateUrl);

Review Comment:
   Applied, thanks.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] ferencerdei commented on a diff in pull request #6075: NIFI-9666 Implement base C2 client functionality

Posted by GitBox <gi...@apache.org>.
ferencerdei commented on code in PR #6075:
URL: https://github.com/apache/nifi/pull/6075#discussion_r893250907


##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/command/StartRunner.java:
##########
@@ -0,0 +1,435 @@
+/*
+ * 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.nifi.minifi.bootstrap.command;
+
+import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CMD_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CONF_DIR_KEY;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.DEFAULT_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.MINIFI_CONFIG_FILE_KEY;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.STATUS_FILE_PID_KEY;
+import static org.apache.nifi.minifi.bootstrap.Status.ERROR;
+import static org.apache.nifi.minifi.bootstrap.Status.OK;
+import static org.apache.nifi.minifi.bootstrap.util.ConfigTransformer.generateConfigFiles;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.nifi.bootstrap.util.OSUtils;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.apache.nifi.minifi.bootstrap.RunMiNiFi;
+import org.apache.nifi.minifi.bootstrap.ShutdownHook;
+import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeException;
+import org.apache.nifi.minifi.bootstrap.exception.StartupFailureException;
+import org.apache.nifi.minifi.bootstrap.service.BootstrapFileProvider;
+import org.apache.nifi.minifi.bootstrap.service.CurrentPortProvider;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiListener;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiStdLogHandler;
+import org.apache.nifi.minifi.bootstrap.service.PeriodicStatusReporterManager;
+import org.apache.nifi.minifi.bootstrap.util.ProcessUtils;
+import org.apache.nifi.util.Tuple;
+
+public class StartRunner implements CommandRunner {
+    private static final String DEFAULT_JAVA_CMD = "java";
+    private static final String DEFAULT_LOG_DIR = "./logs";
+    private static final String DEFAULT_LIB_DIR = "./lib";
+    private static final String DEFAULT_CONF_DIR = "./conf";
+    private static final int STARTUP_WAIT_SECONDS = 60;
+    private static final String DEFAULT_CONFIG_FILE = DEFAULT_CONF_DIR + "/bootstrap.conf";
+
+    private final CurrentPortProvider currentPortProvider;
+    private final BootstrapFileProvider bootstrapFileProvider;
+    private final PeriodicStatusReporterManager periodicStatusReporterManager;
+    private final MiNiFiStdLogHandler miNiFiStdLogHandler;
+    private final MiNiFiParameters miNiFiParameters;
+    private final File bootstrapConfigFile;
+    private final Lock lock = new ReentrantLock();
+    private final Condition startupCondition = lock.newCondition();
+    private final RunMiNiFi runMiNiFi;
+    private volatile ShutdownHook shutdownHook;
+
+    public StartRunner(CurrentPortProvider currentPortProvider, BootstrapFileProvider bootstrapFileProvider,
+        PeriodicStatusReporterManager periodicStatusReporterManager, MiNiFiStdLogHandler miNiFiStdLogHandler, MiNiFiParameters miNiFiParameters, File bootstrapConfigFile,
+        RunMiNiFi runMiNiFi) {
+        this.currentPortProvider = currentPortProvider;
+        this.bootstrapFileProvider = bootstrapFileProvider;
+        this.periodicStatusReporterManager = periodicStatusReporterManager;
+        this.miNiFiStdLogHandler = miNiFiStdLogHandler;
+        this.miNiFiParameters = miNiFiParameters;
+        this.bootstrapConfigFile = bootstrapConfigFile;
+        this.runMiNiFi = runMiNiFi;
+    }
+
+    /**
+     * Starts (and restarts) MiNiFi process during the whole lifecycle of the bootstrap process.
+     * @param args the input arguments
+     * @return status code
+     */
+    @Override
+    public int runCommand(String[] args) {
+        try {
+            start();
+        } catch (Exception e) {
+            CMD_LOGGER.error("Exception happened during MiNiFi startup", e);
+            return ERROR.getStatusCode();
+        }
+        return OK.getStatusCode();
+    }
+
+    private void start() throws IOException, InterruptedException {
+        Integer port = currentPortProvider.getCurrentPort();
+        if (port != null) {
+            String alreadyRunningMessage = "Apache MiNiFi is already running";
+            CMD_LOGGER.info(alreadyRunningMessage + ", listening to Bootstrap on port {}", port);
+            return;
+        }
+
+        File prevLockFile = bootstrapFileProvider.getLockFile();
+        if (prevLockFile.exists() && !prevLockFile.delete()) {
+            CMD_LOGGER.warn("Failed to delete previous lock file {}; this file should be cleaned up manually", prevLockFile);
+        }
+
+        Properties bootstrapProperties = bootstrapFileProvider.getBootstrapProperties();
+        String confDir = bootstrapProperties.getProperty(CONF_DIR_KEY);
+        initConfigFiles(bootstrapProperties, confDir);
+
+        Tuple<ProcessBuilder, Process> tuple = startMiNiFi();
+        ProcessBuilder builder = tuple.getKey();
+        Process process = tuple.getValue();
+
+        try {
+            while (true) {
+                if (ProcessUtils.isAlive(process)) {
+                    handleReload();
+                } else {
+                    Runtime runtime = Runtime.getRuntime();
+                    try {
+                        runtime.removeShutdownHook(shutdownHook);
+                    } catch (IllegalStateException ise) {
+                        // happens when already shutting down
+                    }
+
+                    if (runMiNiFi.isAutoRestartNiFi() && needRestart()) {
+                        File reloadFile = bootstrapFileProvider.getReloadLockFile();
+                        if (reloadFile.exists()) {
+                            DEFAULT_LOGGER.info("Currently reloading configuration. Will wait to restart MiNiFi.");
+                            Thread.sleep(5000L);
+                            continue;
+                        }
+
+                        process = restartNifi(bootstrapProperties, confDir, builder, runtime);
+                        // failed to start process
+                        if (process == null) {
+                            return;
+                        }
+                    } else {
+                        return;
+                    }
+                }
+            }
+        } finally {
+            miNiFiStdLogHandler.shutdown();
+            runMiNiFi.shutdownChangeNotifier();
+            periodicStatusReporterManager.shutdownPeriodicStatusReporters();
+        }
+    }
+
+    private Process restartNifi(Properties bootstrapProperties, String confDir, ProcessBuilder builder, Runtime runtime) throws IOException {
+        Process process;
+        boolean previouslyStarted = runMiNiFi.isNiFiStarted();
+        if (!previouslyStarted) {
+            File swapConfigFile = bootstrapFileProvider.getSwapFile();
+            if (swapConfigFile.exists()) {
+                DEFAULT_LOGGER.info("Swap file exists, MiNiFi failed trying to change configuration. Reverting to old configuration.");
+
+                try {
+                    ByteBuffer tempConfigFile = generateConfigFiles(new FileInputStream(swapConfigFile), confDir, bootstrapProperties);
+                    runMiNiFi.getConfigFileReference().set(tempConfigFile.asReadOnlyBuffer());
+                } catch (ConfigurationChangeException e) {
+                    DEFAULT_LOGGER.error("The swap file is malformed, unable to restart from prior state. Will not attempt to restart MiNiFi. Swap File should be cleaned up manually.");
+                    return null;
+                }
+
+                Files.copy(swapConfigFile.toPath(), Paths.get(bootstrapProperties.getProperty(MINIFI_CONFIG_FILE_KEY)), REPLACE_EXISTING);
+
+                DEFAULT_LOGGER.info("Replacing config file with swap file and deleting swap file");
+                if (!swapConfigFile.delete()) {
+                    DEFAULT_LOGGER.warn("The swap file failed to delete after replacing using it to revert to the old configuration. It should be cleaned up manually.");
+                }
+                runMiNiFi.setReloading(false);
+            } else {
+                DEFAULT_LOGGER.info("MiNiFi either never started or failed to restart. Will not attempt to restart MiNiFi");
+                return null;
+            }
+        } else {
+            runMiNiFi.setNiFiStarted(false);
+        }
+
+        miNiFiParameters.setSecretKey(null);
+        process = builder.start();
+        miNiFiStdLogHandler.initLogging(process);
+
+        Long pid = OSUtils.getProcessId(process, DEFAULT_LOGGER);
+        if (pid != null) {
+            miNiFiParameters.setMinifiPid(pid);
+            Properties minifiProps = new Properties();
+            minifiProps.setProperty(STATUS_FILE_PID_KEY, String.valueOf(pid));
+            bootstrapFileProvider.saveStatusProperties(minifiProps);
+        }
+
+        shutdownHook = new ShutdownHook(runMiNiFi, miNiFiStdLogHandler);
+        runtime.addShutdownHook(shutdownHook);
+
+        boolean started = waitForStart();
+
+        if (started) {
+            DEFAULT_LOGGER.info("Successfully spawned the thread to start Apache MiNiFi{}", (pid == null ? "" : " with PID " + pid));
+        } else {
+            DEFAULT_LOGGER.error("Apache MiNiFi does not appear to have started");
+        }
+        return process;
+    }
+
+    private boolean needRestart() throws IOException {
+        boolean needRestart = true;
+        File statusFile = bootstrapFileProvider.getStatusFile();
+        if (!statusFile.exists()) {
+            DEFAULT_LOGGER.info("Status File no longer exists. Will not restart MiNiFi");
+            return false;
+        }
+
+        File lockFile = bootstrapFileProvider.getLockFile();
+        if (lockFile.exists()) {
+            DEFAULT_LOGGER.info("A shutdown was initiated. Will not restart MiNiFi");
+            return false;
+        }
+        return needRestart;
+    }
+
+    private void handleReload() {
+        try {
+            Thread.sleep(1000L);
+            if (runMiNiFi.getReloading() && runMiNiFi.isNiFiStarted()) {
+                File swapConfigFile = bootstrapFileProvider.getSwapFile();
+                if (swapConfigFile.exists()) {
+                    DEFAULT_LOGGER.info("MiNiFi has finished reloading successfully and swap file exists. Deleting old configuration.");
+
+                    if (swapConfigFile.delete()) {
+                        DEFAULT_LOGGER.info("Swap file was successfully deleted.");
+                    } else {
+                        DEFAULT_LOGGER.error("Swap file was not deleted. It should be deleted manually.");
+                    }
+                }
+                runMiNiFi.setReloading(false);
+            }
+        } catch (InterruptedException ie) {
+        }
+    }
+
+    private void initConfigFiles(Properties bootstrapProperties, String confDir) throws IOException {
+        File configFile = new File(bootstrapProperties.getProperty(MINIFI_CONFIG_FILE_KEY));
+        try (InputStream inputStream = new FileInputStream(configFile)) {
+            ByteBuffer tempConfigFile = generateConfigFiles(inputStream, confDir, bootstrapProperties);
+            runMiNiFi.getConfigFileReference().set(tempConfigFile.asReadOnlyBuffer());
+        } catch (FileNotFoundException e) {
+            String fileNotFoundMessage = "The config file defined in " + MINIFI_CONFIG_FILE_KEY + " does not exists.";
+            DEFAULT_LOGGER.error(fileNotFoundMessage, e);
+            throw new StartupFailureException(fileNotFoundMessage);
+        } catch (ConfigurationChangeException e) {
+            String malformedConfigFileMessage = "The config file is malformed, unable to start.";
+            DEFAULT_LOGGER.error(malformedConfigFileMessage, e);
+            throw new StartupFailureException(malformedConfigFileMessage);
+        }
+    }
+
+    private Tuple<ProcessBuilder, Process> startMiNiFi() throws IOException {
+        ProcessBuilder builder = new ProcessBuilder();
+
+        Properties props = bootstrapFileProvider.getBootstrapProperties();
+        File bootstrapConfigAbsoluteFile = bootstrapConfigFile.getAbsoluteFile();
+        File binDir = bootstrapConfigAbsoluteFile.getParentFile();
+
+        File workingDir = Optional.ofNullable(props.getProperty("working.dir"))
+            .map(File::new)
+            .orElse(binDir.getParentFile());
+
+        builder.directory(workingDir);
+
+        String minifiLogDir = System.getProperty("org.apache.nifi.minifi.bootstrap.config.log.dir", DEFAULT_LOG_DIR).trim();
+        File libDir = getFile(props.getProperty("lib.dir", DEFAULT_LIB_DIR).trim(), workingDir);
+        File confDir = getFile(props.getProperty(CONF_DIR_KEY, DEFAULT_CONF_DIR).trim(), workingDir);
+
+        String minifiPropsFilename = props.getProperty("props.file");
+        if (minifiPropsFilename == null) {
+            if (confDir.exists()) {
+                minifiPropsFilename = new File(confDir, "nifi.properties").getAbsolutePath();
+            } else {
+                minifiPropsFilename = DEFAULT_CONFIG_FILE;
+            }
+        }
+
+        minifiPropsFilename = minifiPropsFilename.trim();
+
+        List<String> javaAdditionalArgs = new ArrayList<>();
+        for (Entry<Object, Object> entry : props.entrySet()) {
+            String key = (String) entry.getKey();
+            String value = (String) entry.getValue();
+
+            if (key.startsWith("java.arg")) {
+                javaAdditionalArgs.add(value);
+            }
+        }
+
+        File[] libFiles = libDir.listFiles((dir, filename) -> filename.toLowerCase().endsWith(".jar"));
+
+        if (libFiles == null || libFiles.length == 0) {
+            throw new RuntimeException("Could not find lib directory at " + libDir.getAbsolutePath());
+        }
+
+        File[] confFiles = confDir.listFiles();
+        if (confFiles == null || confFiles.length == 0) {
+            throw new RuntimeException("Could not find conf directory at " + confDir.getAbsolutePath());
+        }
+
+        List<String> cpFiles = new ArrayList<>(confFiles.length + libFiles.length);
+        cpFiles.add(confDir.getAbsolutePath());
+        for (File file : libFiles) {
+            cpFiles.add(file.getAbsolutePath());
+        }
+
+        StringBuilder classPathBuilder = new StringBuilder();
+        for (int i = 0; i < cpFiles.size(); i++) {
+            String filename = cpFiles.get(i);
+            classPathBuilder.append(filename);
+            if (i < cpFiles.size() - 1) {
+                classPathBuilder.append(File.pathSeparatorChar);
+            }
+        }
+
+        String classPath = classPathBuilder.toString();
+        String javaCmd = props.getProperty("java");
+        if (javaCmd == null) {
+            javaCmd = DEFAULT_JAVA_CMD;
+        }
+        if (javaCmd.equals(DEFAULT_JAVA_CMD)) {
+            String javaHome = System.getenv("JAVA_HOME");
+            if (javaHome != null) {
+                String fileExtension = isWindows() ? ".exe" : "";

Review Comment:
   changed the code



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #6075: NIFI-9666 Implement base C2 client functionality

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on code in PR #6075:
URL: https://github.com/apache/nifi/pull/6075#discussion_r888116339


##########
c2/c2-client-bundle/c2-client-api/src/main/java/org/apache/nifi/c2/client/api/C2Client.java:
##########
@@ -0,0 +1,37 @@
+/*
+ * 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.nifi.c2.client.api;
+
+import java.util.Optional;
+import org.apache.nifi.c2.protocol.api.C2Heartbeat;
+import org.apache.nifi.c2.protocol.api.C2HeartbeatResponse;
+import org.apache.nifi.c2.protocol.api.C2OperationAck;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+/**
+ * Defines interface methods used to implement a C2 Client. The controller can be application-specific but is used for such tasks as updating the flow.
+ */
+public interface C2Client {
+
+    Optional<C2HeartbeatResponse> publishHeartbeat(C2Heartbeat heartbeat) throws IOException;

Review Comment:
   It would be helpful to provide JavaDoc comments on these interface methods.



##########
c2/c2-client-bundle/c2-client-base/pom.xml:
##########
@@ -0,0 +1,71 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+ -->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <artifactId>c2-client-bundle</artifactId>
+        <groupId>org.apache.nifi</groupId>
+        <version>1.17.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>c2-client-base</artifactId>
+    <packaging>jar</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>c2-client-api</artifactId>
+            <version>1.17.0-SNAPSHOT</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>c2-protocol-api</artifactId>
+            <version>1.17.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-security-utils-api</artifactId>
+            <version>1.17.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-security-utils</artifactId>
+            <version>1.17.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-utils</artifactId>
+            <version>1.17.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.commons</groupId>
+            <artifactId>commons-lang3</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>com.fasterxml.jackson.core</groupId>
+            <artifactId>jackson-databind</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>com.fasterxml.jackson.core</groupId>
+            <artifactId>jackson-annotations</artifactId>
+            <version>2.9.0</version>

Review Comment:
   This version property should be removed so that the version can be defined using the jackson-bom in the root Maven configuration.



##########
c2/c2-client-bundle/c2-client-api/src/main/java/org/apache/nifi/c2/client/api/C2Client.java:
##########
@@ -0,0 +1,37 @@
+/*
+ * 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.nifi.c2.client.api;
+
+import java.util.Optional;
+import org.apache.nifi.c2.protocol.api.C2Heartbeat;
+import org.apache.nifi.c2.protocol.api.C2HeartbeatResponse;
+import org.apache.nifi.c2.protocol.api.C2OperationAck;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+/**
+ * Defines interface methods used to implement a C2 Client. The controller can be application-specific but is used for such tasks as updating the flow.
+ */
+public interface C2Client {
+
+    Optional<C2HeartbeatResponse> publishHeartbeat(C2Heartbeat heartbeat) throws IOException;
+
+    ByteBuffer retrieveUpdateContent(String flowUpdateUrl);

Review Comment:
   Is there a particular reason for using `ByteBuffer` as container for a response? It seems that either an `InputStream` or `byte[]` would be a better fit for the interface contract, since `ByteBuffer` support read and write operations. Is it necessary to provide any additional information about the response? If so, it might make more sense to return some type of response object that contains the update content as well as the possibility for additional status information.



##########
c2/c2-client-bundle/c2-client-http/src/main/java/org/apache/nifi/c2/client/http/C2HttpClient.java:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.nifi.c2.client.http;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.security.KeyStore;
+import java.security.NoSuchAlgorithmException;
+import java.util.Optional;
+import java.util.concurrent.atomic.AtomicReference;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLSocketFactory;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+import okhttp3.MediaType;
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.RequestBody;
+import okhttp3.Response;
+import okhttp3.ResponseBody;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.c2.client.C2ClientConfig;
+import org.apache.nifi.c2.client.api.C2Client;
+import org.apache.nifi.c2.client.api.C2Serializer;
+import org.apache.nifi.c2.protocol.api.C2Heartbeat;
+import org.apache.nifi.c2.protocol.api.C2HeartbeatResponse;
+import org.apache.nifi.c2.protocol.api.C2OperationAck;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class C2HttpClient implements C2Client {
+
+    private static final Logger logger = LoggerFactory.getLogger(C2HttpClient.class);
+    private static final MediaType MEDIA_TYPE_APPLICATION_JSON = MediaType.parse("application/json");
+
+    private final AtomicReference<OkHttpClient> httpClientReference = new AtomicReference<>();
+    private final C2ClientConfig clientConfig;
+    private final C2Serializer serializer;
+
+    public C2HttpClient(C2ClientConfig clientConfig, C2Serializer serializer) {
+        super();
+        this.clientConfig = clientConfig;
+        this.serializer = serializer;
+        final OkHttpClient.Builder okHttpClientBuilder = new OkHttpClient.Builder();
+
+        // Set whether to follow redirects
+        okHttpClientBuilder.followRedirects(true);
+
+        // check if the ssl path is set and add the factory if so
+        if (StringUtils.isNotBlank(clientConfig.getKeystoreFilename())) {
+            try {
+                setSslSocketFactory(okHttpClientBuilder);
+            } catch (Exception e) {
+                throw new IllegalStateException(e);

Review Comment:
   This should include an error message:
   ```suggestion
                   throw new IllegalStateException("OkHttp TLS configuration failed", e);
   ```



##########
c2/c2-client-bundle/c2-client-http/src/main/java/org/apache/nifi/c2/client/http/C2HttpClient.java:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.nifi.c2.client.http;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.security.KeyStore;
+import java.security.NoSuchAlgorithmException;
+import java.util.Optional;
+import java.util.concurrent.atomic.AtomicReference;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLSocketFactory;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+import okhttp3.MediaType;
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.RequestBody;
+import okhttp3.Response;
+import okhttp3.ResponseBody;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.c2.client.C2ClientConfig;
+import org.apache.nifi.c2.client.api.C2Client;
+import org.apache.nifi.c2.client.api.C2Serializer;
+import org.apache.nifi.c2.protocol.api.C2Heartbeat;
+import org.apache.nifi.c2.protocol.api.C2HeartbeatResponse;
+import org.apache.nifi.c2.protocol.api.C2OperationAck;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class C2HttpClient implements C2Client {
+
+    private static final Logger logger = LoggerFactory.getLogger(C2HttpClient.class);
+    private static final MediaType MEDIA_TYPE_APPLICATION_JSON = MediaType.parse("application/json");
+
+    private final AtomicReference<OkHttpClient> httpClientReference = new AtomicReference<>();
+    private final C2ClientConfig clientConfig;
+    private final C2Serializer serializer;
+
+    public C2HttpClient(C2ClientConfig clientConfig, C2Serializer serializer) {
+        super();
+        this.clientConfig = clientConfig;
+        this.serializer = serializer;
+        final OkHttpClient.Builder okHttpClientBuilder = new OkHttpClient.Builder();
+
+        // Set whether to follow redirects
+        okHttpClientBuilder.followRedirects(true);
+
+        // check if the ssl path is set and add the factory if so
+        if (StringUtils.isNotBlank(clientConfig.getKeystoreFilename())) {
+            try {
+                setSslSocketFactory(okHttpClientBuilder);
+            } catch (Exception e) {
+                throw new IllegalStateException(e);
+            }
+        }
+
+        httpClientReference.set(okHttpClientBuilder.build());
+    }
+
+    @Override
+    public Optional<C2HeartbeatResponse> publishHeartbeat(C2Heartbeat heartbeat) {
+        return serializer.serialize(heartbeat).flatMap(this::sendHeartbeat);
+    }
+
+    private Optional<C2HeartbeatResponse> sendHeartbeat(String heartbeat) {
+        logger.debug("Sending heartbeat to {}", clientConfig.getC2Url());
+
+        Optional<C2HeartbeatResponse> c2HeartbeatResponse = Optional.empty();
+        Request request = new Request.Builder()
+            .post(RequestBody.create(heartbeat, MEDIA_TYPE_APPLICATION_JSON))
+            .url(clientConfig.getC2Url())
+            .build();
+
+        try {
+            Response heartbeatResponse = httpClientReference.get().newCall(request).execute();

Review Comment:
   It would be better to wrap the `Response` in a try-with-resources to ensure it is closed.



##########
c2/c2-client-bundle/c2-client-http/src/main/java/org/apache/nifi/c2/client/http/C2HttpClient.java:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.nifi.c2.client.http;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.security.KeyStore;
+import java.security.NoSuchAlgorithmException;
+import java.util.Optional;
+import java.util.concurrent.atomic.AtomicReference;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLSocketFactory;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+import okhttp3.MediaType;
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.RequestBody;
+import okhttp3.Response;
+import okhttp3.ResponseBody;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.c2.client.C2ClientConfig;
+import org.apache.nifi.c2.client.api.C2Client;
+import org.apache.nifi.c2.client.api.C2Serializer;
+import org.apache.nifi.c2.protocol.api.C2Heartbeat;
+import org.apache.nifi.c2.protocol.api.C2HeartbeatResponse;
+import org.apache.nifi.c2.protocol.api.C2OperationAck;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class C2HttpClient implements C2Client {
+
+    private static final Logger logger = LoggerFactory.getLogger(C2HttpClient.class);
+    private static final MediaType MEDIA_TYPE_APPLICATION_JSON = MediaType.parse("application/json");
+
+    private final AtomicReference<OkHttpClient> httpClientReference = new AtomicReference<>();
+    private final C2ClientConfig clientConfig;
+    private final C2Serializer serializer;
+
+    public C2HttpClient(C2ClientConfig clientConfig, C2Serializer serializer) {
+        super();
+        this.clientConfig = clientConfig;
+        this.serializer = serializer;
+        final OkHttpClient.Builder okHttpClientBuilder = new OkHttpClient.Builder();
+
+        // Set whether to follow redirects
+        okHttpClientBuilder.followRedirects(true);
+
+        // check if the ssl path is set and add the factory if so
+        if (StringUtils.isNotBlank(clientConfig.getKeystoreFilename())) {
+            try {
+                setSslSocketFactory(okHttpClientBuilder);
+            } catch (Exception e) {
+                throw new IllegalStateException(e);
+            }
+        }
+
+        httpClientReference.set(okHttpClientBuilder.build());
+    }
+
+    @Override
+    public Optional<C2HeartbeatResponse> publishHeartbeat(C2Heartbeat heartbeat) {
+        return serializer.serialize(heartbeat).flatMap(this::sendHeartbeat);
+    }
+
+    private Optional<C2HeartbeatResponse> sendHeartbeat(String heartbeat) {
+        logger.debug("Sending heartbeat to {}", clientConfig.getC2Url());
+
+        Optional<C2HeartbeatResponse> c2HeartbeatResponse = Optional.empty();
+        Request request = new Request.Builder()
+            .post(RequestBody.create(heartbeat, MEDIA_TYPE_APPLICATION_JSON))
+            .url(clientConfig.getC2Url())
+            .build();
+
+        try {
+            Response heartbeatResponse = httpClientReference.get().newCall(request).execute();
+            c2HeartbeatResponse = getResponseBody(heartbeatResponse).flatMap(response -> serializer.deserialize(response, C2HeartbeatResponse.class));
+        } catch (IOException ce) {
+            logger.error("Error while sending heartbeat", ce);
+        }
+
+        return c2HeartbeatResponse;
+    }
+
+    private Optional<String> getResponseBody(Response response) {
+        String responseBody = null;
+
+        try {
+            responseBody = response.body().string();
+            logger.debug("Received response (Status={}) {}", response.code(), responseBody);
+        } catch (IOException e) {
+            logger.error("Could not get response body: ", e);
+        }
+
+        return Optional.ofNullable(responseBody);
+    }
+
+    private void setSslSocketFactory(OkHttpClient.Builder okHttpClientBuilder) throws Exception {
+        final String keystoreLocation = clientConfig.getKeystoreFilename();
+        final String keystoreType = clientConfig.getKeystoreType().getType();
+        final String keystorePass = clientConfig.getKeystorePass();
+
+        assertKeystorePropertiesSet(keystoreLocation, keystorePass, keystoreType);
+
+        // prepare the keystore
+        final KeyStore keyStore = KeyStore.getInstance(keystoreType);
+
+        try (FileInputStream keyStoreStream = new FileInputStream(keystoreLocation)) {
+            keyStore.load(keyStoreStream, keystorePass.toCharArray());
+        }
+
+        final KeyManagerFactory keyManagerFactory = KeyManagerFactory.getInstance(KeyManagerFactory.getDefaultAlgorithm());
+        keyManagerFactory.init(keyStore, keystorePass.toCharArray());
+
+        // load truststore
+        final String truststoreLocation = clientConfig.getTruststoreFilename();
+        final String truststorePass = clientConfig.getTruststorePass();
+        final String truststoreType = clientConfig.getTruststoreType().getType();
+        assertTruststorePropertiesSet(truststoreLocation, truststorePass, truststoreType);
+
+        KeyStore truststore = KeyStore.getInstance(truststoreType);
+        final TrustManagerFactory trustManagerFactory = TrustManagerFactory.getInstance("X509");
+        truststore.load(new FileInputStream(truststoreLocation), truststorePass.toCharArray());
+        trustManagerFactory.init(truststore);
+
+        final X509TrustManager x509TrustManager;
+        TrustManager[] trustManagers = trustManagerFactory.getTrustManagers();
+        if (trustManagers[0] != null) {
+            x509TrustManager = (X509TrustManager) trustManagers[0];
+        } else {
+            throw new IllegalStateException("List of trust managers is null");
+        }
+
+        SSLContext tempSslContext;
+        try {
+            tempSslContext = SSLContext.getInstance("TLS");
+        } catch (NoSuchAlgorithmException e) {
+            logger.warn("Unable to use 'TLS' for the PullHttpChangeIngestor due to NoSuchAlgorithmException. Will attempt to use the default algorithm.", e);
+            tempSslContext = SSLContext.getDefault();
+        }
+
+        final SSLContext sslContext = tempSslContext;
+        sslContext.init(keyManagerFactory.getKeyManagers(), trustManagerFactory.getTrustManagers(), null);
+
+        final SSLSocketFactory sslSocketFactory = sslContext.getSocketFactory();
+        okHttpClientBuilder.sslSocketFactory(sslSocketFactory, x509TrustManager);
+    }
+
+    private void assertKeystorePropertiesSet(String location, String password, String type) {
+        if (location == null || location.isEmpty()) {
+            throw new IllegalArgumentException(clientConfig.getKeystoreFilename() + " is null or is empty");
+        }
+
+        if (password == null || password.isEmpty()) {
+            throw new IllegalArgumentException("The client's keystore filename is set but its password is not (or is empty). If the location is set, the password must also be.");
+        }
+
+        if (type == null || type.isEmpty()) {
+            throw new IllegalArgumentException("The client's keystore filename is set but its type is not (or is empty). If the location is set, the type must also be.");
+        }
+    }
+
+    private void assertTruststorePropertiesSet(String location, String password, String type) {
+        if (location == null || location.isEmpty()) {
+            throw new IllegalArgumentException("The client's truststore filename is not set or is empty");
+        }
+
+        if (password == null || password.isEmpty()) {
+            throw new IllegalArgumentException("The client's truststore filename is set but its password is not (or is empty). If the location is set, the password must also be.");
+        }
+
+        if (type == null || type.isEmpty()) {
+            throw new IllegalArgumentException("The client's truststore filename is set but its type is not (or is empty). If the location is set, the type must also be.");
+        }
+    }
+
+    @Override
+    public ByteBuffer retrieveUpdateContent(String flowUpdateUrl) {
+        final Request.Builder requestBuilder = new Request.Builder()
+                .get()
+                .url(flowUpdateUrl);
+        final Request request = requestBuilder.build();
+
+        ResponseBody body;
+        try (final Response response = httpClientReference.get().newCall(request).execute()) {
+            logger.debug("Response received: {}", response);
+
+            int code = response.code();
+            if (code >= 400) {
+                throw new IOException("Got response code " + code + " while trying to pull configuration: " + response.body().string());
+            }
+
+            body = response.body();
+
+            if (body == null) {
+                logger.warn("No body returned when pulling a new configuration");
+                return null;
+            }
+
+            final ByteBuffer bodyByteBuffer = ByteBuffer.wrap(body.bytes());
+            return bodyByteBuffer;
+
+        } catch (Exception e) {
+            logger.warn("Hit an exception while trying to pull", e);

Review Comment:
   Recommend adjusting the message:
   ```suggestion
               logger.warn("Configuration retrieval failed", e);
   ```



##########
c2/c2-client-bundle/c2-client-http/src/main/java/org/apache/nifi/c2/client/http/C2HttpClient.java:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.nifi.c2.client.http;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.security.KeyStore;
+import java.security.NoSuchAlgorithmException;
+import java.util.Optional;
+import java.util.concurrent.atomic.AtomicReference;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLSocketFactory;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+import okhttp3.MediaType;
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.RequestBody;
+import okhttp3.Response;
+import okhttp3.ResponseBody;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.c2.client.C2ClientConfig;
+import org.apache.nifi.c2.client.api.C2Client;
+import org.apache.nifi.c2.client.api.C2Serializer;
+import org.apache.nifi.c2.protocol.api.C2Heartbeat;
+import org.apache.nifi.c2.protocol.api.C2HeartbeatResponse;
+import org.apache.nifi.c2.protocol.api.C2OperationAck;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class C2HttpClient implements C2Client {
+
+    private static final Logger logger = LoggerFactory.getLogger(C2HttpClient.class);
+    private static final MediaType MEDIA_TYPE_APPLICATION_JSON = MediaType.parse("application/json");
+
+    private final AtomicReference<OkHttpClient> httpClientReference = new AtomicReference<>();
+    private final C2ClientConfig clientConfig;
+    private final C2Serializer serializer;
+
+    public C2HttpClient(C2ClientConfig clientConfig, C2Serializer serializer) {
+        super();
+        this.clientConfig = clientConfig;
+        this.serializer = serializer;
+        final OkHttpClient.Builder okHttpClientBuilder = new OkHttpClient.Builder();
+
+        // Set whether to follow redirects
+        okHttpClientBuilder.followRedirects(true);
+
+        // check if the ssl path is set and add the factory if so
+        if (StringUtils.isNotBlank(clientConfig.getKeystoreFilename())) {
+            try {
+                setSslSocketFactory(okHttpClientBuilder);
+            } catch (Exception e) {
+                throw new IllegalStateException(e);
+            }
+        }
+
+        httpClientReference.set(okHttpClientBuilder.build());
+    }
+
+    @Override
+    public Optional<C2HeartbeatResponse> publishHeartbeat(C2Heartbeat heartbeat) {
+        return serializer.serialize(heartbeat).flatMap(this::sendHeartbeat);
+    }
+
+    private Optional<C2HeartbeatResponse> sendHeartbeat(String heartbeat) {
+        logger.debug("Sending heartbeat to {}", clientConfig.getC2Url());
+
+        Optional<C2HeartbeatResponse> c2HeartbeatResponse = Optional.empty();
+        Request request = new Request.Builder()
+            .post(RequestBody.create(heartbeat, MEDIA_TYPE_APPLICATION_JSON))
+            .url(clientConfig.getC2Url())
+            .build();
+
+        try {
+            Response heartbeatResponse = httpClientReference.get().newCall(request).execute();
+            c2HeartbeatResponse = getResponseBody(heartbeatResponse).flatMap(response -> serializer.deserialize(response, C2HeartbeatResponse.class));
+        } catch (IOException ce) {
+            logger.error("Error while sending heartbeat", ce);
+        }
+
+        return c2HeartbeatResponse;
+    }
+
+    private Optional<String> getResponseBody(Response response) {
+        String responseBody = null;
+
+        try {
+            responseBody = response.body().string();
+            logger.debug("Received response (Status={}) {}", response.code(), responseBody);
+        } catch (IOException e) {
+            logger.error("Could not get response body: ", e);
+        }
+
+        return Optional.ofNullable(responseBody);
+    }
+
+    private void setSslSocketFactory(OkHttpClient.Builder okHttpClientBuilder) throws Exception {
+        final String keystoreLocation = clientConfig.getKeystoreFilename();
+        final String keystoreType = clientConfig.getKeystoreType().getType();
+        final String keystorePass = clientConfig.getKeystorePass();
+
+        assertKeystorePropertiesSet(keystoreLocation, keystorePass, keystoreType);
+
+        // prepare the keystore
+        final KeyStore keyStore = KeyStore.getInstance(keystoreType);
+
+        try (FileInputStream keyStoreStream = new FileInputStream(keystoreLocation)) {
+            keyStore.load(keyStoreStream, keystorePass.toCharArray());
+        }
+
+        final KeyManagerFactory keyManagerFactory = KeyManagerFactory.getInstance(KeyManagerFactory.getDefaultAlgorithm());
+        keyManagerFactory.init(keyStore, keystorePass.toCharArray());
+
+        // load truststore
+        final String truststoreLocation = clientConfig.getTruststoreFilename();
+        final String truststorePass = clientConfig.getTruststorePass();
+        final String truststoreType = clientConfig.getTruststoreType().getType();
+        assertTruststorePropertiesSet(truststoreLocation, truststorePass, truststoreType);
+
+        KeyStore truststore = KeyStore.getInstance(truststoreType);
+        final TrustManagerFactory trustManagerFactory = TrustManagerFactory.getInstance("X509");
+        truststore.load(new FileInputStream(truststoreLocation), truststorePass.toCharArray());
+        trustManagerFactory.init(truststore);
+
+        final X509TrustManager x509TrustManager;
+        TrustManager[] trustManagers = trustManagerFactory.getTrustManagers();
+        if (trustManagers[0] != null) {
+            x509TrustManager = (X509TrustManager) trustManagers[0];
+        } else {
+            throw new IllegalStateException("List of trust managers is null");
+        }
+
+        SSLContext tempSslContext;
+        try {
+            tempSslContext = SSLContext.getInstance("TLS");
+        } catch (NoSuchAlgorithmException e) {
+            logger.warn("Unable to use 'TLS' for the PullHttpChangeIngestor due to NoSuchAlgorithmException. Will attempt to use the default algorithm.", e);
+            tempSslContext = SSLContext.getDefault();
+        }
+
+        final SSLContext sslContext = tempSslContext;
+        sslContext.init(keyManagerFactory.getKeyManagers(), trustManagerFactory.getTrustManagers(), null);
+
+        final SSLSocketFactory sslSocketFactory = sslContext.getSocketFactory();
+        okHttpClientBuilder.sslSocketFactory(sslSocketFactory, x509TrustManager);
+    }
+
+    private void assertKeystorePropertiesSet(String location, String password, String type) {
+        if (location == null || location.isEmpty()) {
+            throw new IllegalArgumentException(clientConfig.getKeystoreFilename() + " is null or is empty");
+        }
+
+        if (password == null || password.isEmpty()) {
+            throw new IllegalArgumentException("The client's keystore filename is set but its password is not (or is empty). If the location is set, the password must also be.");
+        }
+
+        if (type == null || type.isEmpty()) {
+            throw new IllegalArgumentException("The client's keystore filename is set but its type is not (or is empty). If the location is set, the type must also be.");
+        }
+    }
+
+    private void assertTruststorePropertiesSet(String location, String password, String type) {
+        if (location == null || location.isEmpty()) {
+            throw new IllegalArgumentException("The client's truststore filename is not set or is empty");
+        }
+
+        if (password == null || password.isEmpty()) {
+            throw new IllegalArgumentException("The client's truststore filename is set but its password is not (or is empty). If the location is set, the password must also be.");
+        }
+
+        if (type == null || type.isEmpty()) {
+            throw new IllegalArgumentException("The client's truststore filename is set but its type is not (or is empty). If the location is set, the type must also be.");
+        }
+    }
+
+    @Override
+    public ByteBuffer retrieveUpdateContent(String flowUpdateUrl) {
+        final Request.Builder requestBuilder = new Request.Builder()
+                .get()
+                .url(flowUpdateUrl);
+        final Request request = requestBuilder.build();
+
+        ResponseBody body;
+        try (final Response response = httpClientReference.get().newCall(request).execute()) {
+            logger.debug("Response received: {}", response);
+
+            int code = response.code();
+            if (code >= 400) {
+                throw new IOException("Got response code " + code + " while trying to pull configuration: " + response.body().string());
+            }
+
+            body = response.body();
+
+            if (body == null) {
+                logger.warn("No body returned when pulling a new configuration");
+                return null;
+            }
+
+            final ByteBuffer bodyByteBuffer = ByteBuffer.wrap(body.bytes());
+            return bodyByteBuffer;
+
+        } catch (Exception e) {
+            logger.warn("Hit an exception while trying to pull", e);
+            return null;
+        }
+    }
+
+    @Override
+    public void acknowledgeOperation(C2OperationAck operationAck) {
+        logger.info("Performing acknowledgement request to {} for operation {}", clientConfig.getC2AckUrl(), operationAck.getOperationId());
+        final ObjectMapper jacksonObjectMapper = new ObjectMapper();
+        jacksonObjectMapper.setSerializationInclusion(JsonInclude.Include.NON_NULL);
+        try {
+            final String operationAckBody = jacksonObjectMapper.writeValueAsString(operationAck);
+
+            final RequestBody requestBody = RequestBody.create(operationAckBody, MediaType.parse("application/json"));

Review Comment:
   The `MediaType.parse()` call could be declared once and reused.



##########
c2/c2-client-bundle/c2-client-http/src/main/java/org/apache/nifi/c2/client/http/C2HttpClient.java:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.nifi.c2.client.http;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.security.KeyStore;
+import java.security.NoSuchAlgorithmException;
+import java.util.Optional;
+import java.util.concurrent.atomic.AtomicReference;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLSocketFactory;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+import okhttp3.MediaType;
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.RequestBody;
+import okhttp3.Response;
+import okhttp3.ResponseBody;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.c2.client.C2ClientConfig;
+import org.apache.nifi.c2.client.api.C2Client;
+import org.apache.nifi.c2.client.api.C2Serializer;
+import org.apache.nifi.c2.protocol.api.C2Heartbeat;
+import org.apache.nifi.c2.protocol.api.C2HeartbeatResponse;
+import org.apache.nifi.c2.protocol.api.C2OperationAck;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class C2HttpClient implements C2Client {
+
+    private static final Logger logger = LoggerFactory.getLogger(C2HttpClient.class);
+    private static final MediaType MEDIA_TYPE_APPLICATION_JSON = MediaType.parse("application/json");
+
+    private final AtomicReference<OkHttpClient> httpClientReference = new AtomicReference<>();
+    private final C2ClientConfig clientConfig;
+    private final C2Serializer serializer;
+
+    public C2HttpClient(C2ClientConfig clientConfig, C2Serializer serializer) {
+        super();
+        this.clientConfig = clientConfig;
+        this.serializer = serializer;
+        final OkHttpClient.Builder okHttpClientBuilder = new OkHttpClient.Builder();
+
+        // Set whether to follow redirects
+        okHttpClientBuilder.followRedirects(true);
+
+        // check if the ssl path is set and add the factory if so
+        if (StringUtils.isNotBlank(clientConfig.getKeystoreFilename())) {
+            try {
+                setSslSocketFactory(okHttpClientBuilder);
+            } catch (Exception e) {
+                throw new IllegalStateException(e);
+            }
+        }
+
+        httpClientReference.set(okHttpClientBuilder.build());
+    }
+
+    @Override
+    public Optional<C2HeartbeatResponse> publishHeartbeat(C2Heartbeat heartbeat) {
+        return serializer.serialize(heartbeat).flatMap(this::sendHeartbeat);
+    }
+
+    private Optional<C2HeartbeatResponse> sendHeartbeat(String heartbeat) {
+        logger.debug("Sending heartbeat to {}", clientConfig.getC2Url());
+
+        Optional<C2HeartbeatResponse> c2HeartbeatResponse = Optional.empty();
+        Request request = new Request.Builder()
+            .post(RequestBody.create(heartbeat, MEDIA_TYPE_APPLICATION_JSON))
+            .url(clientConfig.getC2Url())
+            .build();
+
+        try {
+            Response heartbeatResponse = httpClientReference.get().newCall(request).execute();
+            c2HeartbeatResponse = getResponseBody(heartbeatResponse).flatMap(response -> serializer.deserialize(response, C2HeartbeatResponse.class));
+        } catch (IOException ce) {
+            logger.error("Error while sending heartbeat", ce);
+        }
+
+        return c2HeartbeatResponse;
+    }
+
+    private Optional<String> getResponseBody(Response response) {
+        String responseBody = null;
+
+        try {
+            responseBody = response.body().string();
+            logger.debug("Received response (Status={}) {}", response.code(), responseBody);
+        } catch (IOException e) {
+            logger.error("Could not get response body: ", e);
+        }
+
+        return Optional.ofNullable(responseBody);
+    }
+
+    private void setSslSocketFactory(OkHttpClient.Builder okHttpClientBuilder) throws Exception {
+        final String keystoreLocation = clientConfig.getKeystoreFilename();
+        final String keystoreType = clientConfig.getKeystoreType().getType();
+        final String keystorePass = clientConfig.getKeystorePass();
+
+        assertKeystorePropertiesSet(keystoreLocation, keystorePass, keystoreType);
+
+        // prepare the keystore
+        final KeyStore keyStore = KeyStore.getInstance(keystoreType);
+
+        try (FileInputStream keyStoreStream = new FileInputStream(keystoreLocation)) {
+            keyStore.load(keyStoreStream, keystorePass.toCharArray());
+        }
+
+        final KeyManagerFactory keyManagerFactory = KeyManagerFactory.getInstance(KeyManagerFactory.getDefaultAlgorithm());
+        keyManagerFactory.init(keyStore, keystorePass.toCharArray());
+
+        // load truststore
+        final String truststoreLocation = clientConfig.getTruststoreFilename();
+        final String truststorePass = clientConfig.getTruststorePass();
+        final String truststoreType = clientConfig.getTruststoreType().getType();
+        assertTruststorePropertiesSet(truststoreLocation, truststorePass, truststoreType);
+
+        KeyStore truststore = KeyStore.getInstance(truststoreType);
+        final TrustManagerFactory trustManagerFactory = TrustManagerFactory.getInstance("X509");
+        truststore.load(new FileInputStream(truststoreLocation), truststorePass.toCharArray());
+        trustManagerFactory.init(truststore);
+
+        final X509TrustManager x509TrustManager;
+        TrustManager[] trustManagers = trustManagerFactory.getTrustManagers();
+        if (trustManagers[0] != null) {
+            x509TrustManager = (X509TrustManager) trustManagers[0];
+        } else {
+            throw new IllegalStateException("List of trust managers is null");
+        }
+
+        SSLContext tempSslContext;
+        try {
+            tempSslContext = SSLContext.getInstance("TLS");
+        } catch (NoSuchAlgorithmException e) {
+            logger.warn("Unable to use 'TLS' for the PullHttpChangeIngestor due to NoSuchAlgorithmException. Will attempt to use the default algorithm.", e);
+            tempSslContext = SSLContext.getDefault();
+        }
+
+        final SSLContext sslContext = tempSslContext;
+        sslContext.init(keyManagerFactory.getKeyManagers(), trustManagerFactory.getTrustManagers(), null);
+
+        final SSLSocketFactory sslSocketFactory = sslContext.getSocketFactory();
+        okHttpClientBuilder.sslSocketFactory(sslSocketFactory, x509TrustManager);
+    }
+
+    private void assertKeystorePropertiesSet(String location, String password, String type) {
+        if (location == null || location.isEmpty()) {
+            throw new IllegalArgumentException(clientConfig.getKeystoreFilename() + " is null or is empty");
+        }
+
+        if (password == null || password.isEmpty()) {
+            throw new IllegalArgumentException("The client's keystore filename is set but its password is not (or is empty). If the location is set, the password must also be.");
+        }
+
+        if (type == null || type.isEmpty()) {
+            throw new IllegalArgumentException("The client's keystore filename is set but its type is not (or is empty). If the location is set, the type must also be.");
+        }
+    }
+
+    private void assertTruststorePropertiesSet(String location, String password, String type) {
+        if (location == null || location.isEmpty()) {
+            throw new IllegalArgumentException("The client's truststore filename is not set or is empty");
+        }
+
+        if (password == null || password.isEmpty()) {
+            throw new IllegalArgumentException("The client's truststore filename is set but its password is not (or is empty). If the location is set, the password must also be.");
+        }
+
+        if (type == null || type.isEmpty()) {
+            throw new IllegalArgumentException("The client's truststore filename is set but its type is not (or is empty). If the location is set, the type must also be.");
+        }
+    }
+
+    @Override
+    public ByteBuffer retrieveUpdateContent(String flowUpdateUrl) {
+        final Request.Builder requestBuilder = new Request.Builder()
+                .get()
+                .url(flowUpdateUrl);
+        final Request request = requestBuilder.build();
+
+        ResponseBody body;
+        try (final Response response = httpClientReference.get().newCall(request).execute()) {
+            logger.debug("Response received: {}", response);
+
+            int code = response.code();
+            if (code >= 400) {
+                throw new IOException("Got response code " + code + " while trying to pull configuration: " + response.body().string());
+            }
+
+            body = response.body();
+
+            if (body == null) {
+                logger.warn("No body returned when pulling a new configuration");
+                return null;
+            }
+
+            final ByteBuffer bodyByteBuffer = ByteBuffer.wrap(body.bytes());
+            return bodyByteBuffer;
+
+        } catch (Exception e) {
+            logger.warn("Hit an exception while trying to pull", e);
+            return null;
+        }
+    }
+
+    @Override
+    public void acknowledgeOperation(C2OperationAck operationAck) {
+        logger.info("Performing acknowledgement request to {} for operation {}", clientConfig.getC2AckUrl(), operationAck.getOperationId());
+        final ObjectMapper jacksonObjectMapper = new ObjectMapper();
+        jacksonObjectMapper.setSerializationInclusion(JsonInclude.Include.NON_NULL);
+        try {
+            final String operationAckBody = jacksonObjectMapper.writeValueAsString(operationAck);
+
+            final RequestBody requestBody = RequestBody.create(operationAckBody, MediaType.parse("application/json"));
+            final Request.Builder requestBuilder = new Request.Builder()
+                    .post(requestBody)
+                    .url(clientConfig.getC2AckUrl());
+            final Response heartbeatResponse = httpClientReference.get().newCall(requestBuilder.build()).execute();
+            if (!heartbeatResponse.isSuccessful()) {
+                logger.warn("Acknowledgement was not successful.");

Review Comment:
   It would be helpful to log some additional details if possible, such as the C2 Server URL and HTTP response code, and perhaps other information.



##########
c2/c2-client-bundle/c2-client-service/src/main/java/org/apache/nifi/c2/client/service/FlowIdHolder.java:
##########
@@ -0,0 +1,85 @@
+/*
+ * 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.nifi.c2.client.service;
+
+import static java.util.Collections.singletonList;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.List;
+import org.apache.nifi.util.file.FileUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class FlowIdHolder {
+    private static final Logger LOGGER = LoggerFactory.getLogger(FlowIdHolder.class);
+    private static final String FLOW_IDENTIFIER_FILENAME = "flow-identifier";
+
+    private volatile String flowId;
+    private final String configDirectoryName;
+
+    public FlowIdHolder(String configDirectoryName) {
+        this.configDirectoryName = configDirectoryName;
+        this.flowId = readFlowId();
+    }
+
+    public String getFlowId() {
+        return flowId;
+    }
+
+    public void setFlowId(String flowId) {
+        this.flowId = flowId;
+        persistFlowId(flowId);
+    }
+
+    private void persistFlowId(String flowId) {
+        File flowIdFile = new File(configDirectoryName, FLOW_IDENTIFIER_FILENAME);
+        try {
+            FileUtils.ensureDirectoryExistAndCanAccess(flowIdFile.getParentFile());
+            saveFlowId(flowIdFile, flowId);
+        } catch (IOException e) {
+            LOGGER.error("Failed to save flow information due to: {}", e.getMessage());
+        }
+    }
+
+    private void saveFlowId(File flowUpdateInfoFile, String flowId) {
+        try {
+            Files.write(flowUpdateInfoFile.toPath(), singletonList(flowId));
+        } catch (IOException e) {
+            LOGGER.error("Failed to persist flow data", e);

Review Comment:
   ```suggestion
               LOGGER.error("Writing Flow [{}] failed", flowId, e);
   ```



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/command/EnvRunner.java:
##########
@@ -0,0 +1,107 @@
+/*
+ * 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.nifi.minifi.bootstrap.command;
+
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CMD_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.DEFAULT_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.Status.ERROR;
+import static org.apache.nifi.minifi.bootstrap.Status.MINIFI_NOT_RUNNING;
+import static org.apache.nifi.minifi.bootstrap.Status.OK;
+
+import java.lang.reflect.Method;
+import java.util.Map.Entry;
+import java.util.Properties;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.apache.nifi.minifi.bootstrap.MiNiFiStatus;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiStatusProvider;
+
+public class EnvRunner implements CommandRunner {
+    private final MiNiFiParameters miNiFiParameters;
+    private final MiNiFiStatusProvider miNiFiStatusProvider;
+
+    public EnvRunner(MiNiFiParameters miNiFiParameters, MiNiFiStatusProvider miNiFiStatusProvider) {
+        this.miNiFiParameters = miNiFiParameters;
+        this.miNiFiStatusProvider = miNiFiStatusProvider;
+    }
+
+    /**
+     * Returns information about the MiNiFi's virtual machine.
+     * @param args the input arguments
+     * @return status code
+     */
+    @Override
+    public int runCommand(String[] args) {
+        return env();
+    }
+
+    private int env() {
+        MiNiFiStatus status = miNiFiStatusProvider.getStatus(miNiFiParameters.getMiNiFiPort(), miNiFiParameters.getMinifiPid());
+        if (status.getPid() == null) {
+            CMD_LOGGER.error("Apache MiNiFi is not running");
+            return MINIFI_NOT_RUNNING.getStatusCode();
+        }
+
+        Class<?> virtualMachineClass;
+        try {
+            virtualMachineClass = Class.forName("com.sun.tools.attach.VirtualMachine");

Review Comment:
   What kind of limitations does this introduce in terms of Java versions? Does it work on Java 8, 11, and 17?



##########
c2/c2-client-bundle/c2-client-service/src/main/java/org/apache/nifi/c2/client/service/operation/C2OperationHandler.java:
##########
@@ -0,0 +1,31 @@
+/*
+ * 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.nifi.c2.client.service.operation;
+
+import org.apache.nifi.c2.protocol.api.C2Operation;
+import org.apache.nifi.c2.protocol.api.C2OperationAck;
+import org.apache.nifi.c2.protocol.api.OperandType;
+import org.apache.nifi.c2.protocol.api.OperationType;
+
+public interface C2OperationHandler {
+
+    OperationType getOperationType();
+
+    OperandType getOperandType();
+
+    C2OperationAck handle(C2Operation operation);

Review Comment:
   Recommend including some comments in this interface and methods.



##########
minifi/pom.xml:
##########
@@ -449,10 +449,22 @@ limitations under the License.
                 <version>1.17.0-SNAPSHOT</version>
             </dependency>
 
+            <!-- C2 modules -->
+            <dependency>
+                <groupId>org.apache.nifi</groupId>
+                <artifactId>c2-client-api</artifactId>
+                <version>1.17.0-SNAPSHOT</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.nifi</groupId>
+                <artifactId>c2-client-base</artifactId>
+                <version>1.17.0-SNAPSHOT</version>
+            </dependency>
+
             <dependency>
                 <groupId>com.squareup.okhttp3</groupId>
                 <artifactId>okhttp</artifactId>
-                <version>3.12.3</version>
+                <version>${okhttp.version}</version>

Review Comment:
   This version can be removed.



##########
c2/c2-client-bundle/c2-client-base/pom.xml:
##########
@@ -0,0 +1,71 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+ -->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <artifactId>c2-client-bundle</artifactId>
+        <groupId>org.apache.nifi</groupId>
+        <version>1.17.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>c2-client-base</artifactId>
+    <packaging>jar</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>c2-client-api</artifactId>
+            <version>1.17.0-SNAPSHOT</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>c2-protocol-api</artifactId>
+            <version>1.17.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-security-utils-api</artifactId>
+            <version>1.17.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-security-utils</artifactId>
+            <version>1.17.0-SNAPSHOT</version>
+        </dependency>

Review Comment:
   This module brings a number of additional dependencies, it would be worth evaluating whether it is necessary, or whether particular classes could be separated out to avoid including this dependency.



##########
c2/c2-client-bundle/c2-client-base/src/main/java/org/apache/nifi/c2/client/PersistentUuidGenerator.java:
##########
@@ -0,0 +1,76 @@
+/*
+ * 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.nifi.c2.client;
+
+import org.apache.nifi.c2.client.api.IdGenerator;
+import org.apache.nifi.util.file.FileUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.Arrays;
+import java.util.List;
+import java.util.UUID;
+
+public class PersistentUuidGenerator implements IdGenerator {
+
+    private static final Logger logger = LoggerFactory.getLogger(PersistentUuidGenerator.class);
+
+    private final File persistenceLocation;
+
+    public PersistentUuidGenerator(final File persistenceLocation) {
+        this.persistenceLocation = persistenceLocation;
+    }
+
+    @Override
+    public String generate() {
+        if (this.persistenceLocation.exists()) {
+            return readFile();
+        } else {
+            return makeFile();
+        }
+    }
+
+    private String readFile() {
+        try {
+            final List<String> fileLines = Files.readAllLines(persistenceLocation.toPath());
+            if (fileLines.size() != 1) {
+                throw new IllegalStateException(String.format("The file %s for the persisted identifier has the incorrect format.", persistenceLocation));
+            }
+            final String uuid = fileLines.get(0);
+            return uuid;
+        } catch (IOException e) {
+            throw new IllegalStateException(String.format("Could not read file %s for persisted identifier.", persistenceLocation), e);
+
+        }
+    }
+
+    private String makeFile() {
+        try {
+            final File parentDirectory = persistenceLocation.getParentFile();
+            FileUtils.ensureDirectoryExistAndCanAccess(parentDirectory);
+            final String uuid = UUID.randomUUID().toString();
+            Files.write(persistenceLocation.toPath(), Arrays.asList(uuid));
+            logger.info("Created identifier {} at {}.", uuid, persistenceLocation);

Review Comment:
   Recommend removing the trailing period from the message. Should this be an info level message? It seems more appropriate as a debug.
   ```suggestion
               logger.debug("Created identifier {} at {}", uuid, persistenceLocation);
   ```



##########
c2/c2-client-bundle/c2-client-base/src/main/java/org/apache/nifi/c2/serializer/C2JacksonSerializer.java:
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.nifi.c2.serializer;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.util.Optional;
+import org.apache.nifi.c2.client.api.C2Serializer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class C2JacksonSerializer implements C2Serializer {
+
+    private static final Logger logger = LoggerFactory.getLogger(C2JacksonSerializer.class);
+
+    private final ObjectMapper objectMapper;
+
+    public C2JacksonSerializer() {
+        objectMapper = new ObjectMapper();
+        objectMapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
+        objectMapper.setSerializationInclusion(JsonInclude.Include.NON_NULL);
+        objectMapper.enable(DeserializationFeature.READ_ENUMS_USING_TO_STRING);
+    }
+
+    @Override
+    public <T> Optional<String> serialize(T object) {
+        if (object == null) {
+            logger.trace("C2 Object was null. Nothing to serialize. Returning empty.");
+            return Optional.empty();
+        }
+
+        String contentString = null;
+        try {
+            contentString = objectMapper.writeValueAsString(object);
+            logger.trace("Serialized C2 Object: {}", contentString);
+        } catch (JsonProcessingException e) {
+            logger.error("Can't serialise C2 Object: ", e);
+        }
+
+        return Optional.ofNullable(contentString);
+    }
+
+    @Override
+    public <T> Optional<T> deserialize(String content, Class<T> valueType) {
+        if (content == null) {
+            logger.trace("Content for deserialization was null. Returning empty.");
+            return Optional.empty();
+        }
+
+        T responseObject = null;
+        try {
+            responseObject = objectMapper.readValue(content, valueType);
+        } catch (JsonProcessingException e) {
+            logger.error("Can't deserialize response object: ", e);

Review Comment:
   ```suggestion
               logger.error("Object deserialization from JSON failed", e);
   ```



##########
c2/c2-client-bundle/c2-client-base/src/main/java/org/apache/nifi/c2/serializer/C2JacksonSerializer.java:
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.nifi.c2.serializer;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.util.Optional;
+import org.apache.nifi.c2.client.api.C2Serializer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class C2JacksonSerializer implements C2Serializer {
+
+    private static final Logger logger = LoggerFactory.getLogger(C2JacksonSerializer.class);
+
+    private final ObjectMapper objectMapper;
+
+    public C2JacksonSerializer() {
+        objectMapper = new ObjectMapper();
+        objectMapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
+        objectMapper.setSerializationInclusion(JsonInclude.Include.NON_NULL);
+        objectMapper.enable(DeserializationFeature.READ_ENUMS_USING_TO_STRING);
+    }
+
+    @Override
+    public <T> Optional<String> serialize(T object) {
+        if (object == null) {
+            logger.trace("C2 Object was null. Nothing to serialize. Returning empty.");
+            return Optional.empty();
+        }
+
+        String contentString = null;
+        try {
+            contentString = objectMapper.writeValueAsString(object);
+            logger.trace("Serialized C2 Object: {}", contentString);

Review Comment:
   Recommend avoiding writing out the entire JSON string to a log.
   ```suggestion
   ```



##########
c2/c2-client-bundle/c2-client-http/pom.xml:
##########
@@ -0,0 +1,48 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+ -->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <artifactId>c2-client-bundle</artifactId>
+        <groupId>org.apache.nifi</groupId>
+        <version>1.17.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>c2-client-http</artifactId>
+    <packaging>jar</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>c2-client-api</artifactId>
+            <version>1.17.0-SNAPSHOT</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>c2-client-base</artifactId>
+            <version>1.17.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>com.squareup.okhttp3</groupId>
+            <artifactId>okhttp</artifactId>
+            <version>${okhttp.version}</version>

Review Comment:
   This version property can be removed since it is controlled through the okhttp-bom dependency in the root Maven configuration.



##########
c2/c2-client-bundle/c2-client-base/src/main/java/org/apache/nifi/c2/serializer/C2JacksonSerializer.java:
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.nifi.c2.serializer;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.util.Optional;
+import org.apache.nifi.c2.client.api.C2Serializer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class C2JacksonSerializer implements C2Serializer {
+
+    private static final Logger logger = LoggerFactory.getLogger(C2JacksonSerializer.class);
+
+    private final ObjectMapper objectMapper;
+
+    public C2JacksonSerializer() {
+        objectMapper = new ObjectMapper();
+        objectMapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
+        objectMapper.setSerializationInclusion(JsonInclude.Include.NON_NULL);
+        objectMapper.enable(DeserializationFeature.READ_ENUMS_USING_TO_STRING);
+    }
+
+    @Override
+    public <T> Optional<String> serialize(T object) {
+        if (object == null) {
+            logger.trace("C2 Object was null. Nothing to serialize. Returning empty.");
+            return Optional.empty();
+        }
+
+        String contentString = null;
+        try {
+            contentString = objectMapper.writeValueAsString(object);
+            logger.trace("Serialized C2 Object: {}", contentString);
+        } catch (JsonProcessingException e) {
+            logger.error("Can't serialise C2 Object: ", e);

Review Comment:
   Recommend rewording this message to avoid the colon and conjunction:
   ```suggestion
               logger.error("Object serialization to JSON failed", e);
   ```
   



##########
c2/c2-client-bundle/c2-client-http/src/main/java/org/apache/nifi/c2/client/http/C2HttpClient.java:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.nifi.c2.client.http;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.security.KeyStore;
+import java.security.NoSuchAlgorithmException;
+import java.util.Optional;
+import java.util.concurrent.atomic.AtomicReference;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLSocketFactory;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+import okhttp3.MediaType;
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.RequestBody;
+import okhttp3.Response;
+import okhttp3.ResponseBody;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.c2.client.C2ClientConfig;
+import org.apache.nifi.c2.client.api.C2Client;
+import org.apache.nifi.c2.client.api.C2Serializer;
+import org.apache.nifi.c2.protocol.api.C2Heartbeat;
+import org.apache.nifi.c2.protocol.api.C2HeartbeatResponse;
+import org.apache.nifi.c2.protocol.api.C2OperationAck;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class C2HttpClient implements C2Client {
+
+    private static final Logger logger = LoggerFactory.getLogger(C2HttpClient.class);
+    private static final MediaType MEDIA_TYPE_APPLICATION_JSON = MediaType.parse("application/json");
+
+    private final AtomicReference<OkHttpClient> httpClientReference = new AtomicReference<>();
+    private final C2ClientConfig clientConfig;
+    private final C2Serializer serializer;
+
+    public C2HttpClient(C2ClientConfig clientConfig, C2Serializer serializer) {
+        super();
+        this.clientConfig = clientConfig;
+        this.serializer = serializer;
+        final OkHttpClient.Builder okHttpClientBuilder = new OkHttpClient.Builder();

Review Comment:
   The configuration should include setting timeout values, which looks like it would require some additions to the `C2ClientConfig`.



##########
c2/c2-client-bundle/c2-client-http/src/main/java/org/apache/nifi/c2/client/http/C2HttpClient.java:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.nifi.c2.client.http;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.security.KeyStore;
+import java.security.NoSuchAlgorithmException;
+import java.util.Optional;
+import java.util.concurrent.atomic.AtomicReference;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLSocketFactory;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+import okhttp3.MediaType;
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.RequestBody;
+import okhttp3.Response;
+import okhttp3.ResponseBody;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.c2.client.C2ClientConfig;
+import org.apache.nifi.c2.client.api.C2Client;
+import org.apache.nifi.c2.client.api.C2Serializer;
+import org.apache.nifi.c2.protocol.api.C2Heartbeat;
+import org.apache.nifi.c2.protocol.api.C2HeartbeatResponse;
+import org.apache.nifi.c2.protocol.api.C2OperationAck;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class C2HttpClient implements C2Client {
+
+    private static final Logger logger = LoggerFactory.getLogger(C2HttpClient.class);
+    private static final MediaType MEDIA_TYPE_APPLICATION_JSON = MediaType.parse("application/json");
+
+    private final AtomicReference<OkHttpClient> httpClientReference = new AtomicReference<>();
+    private final C2ClientConfig clientConfig;
+    private final C2Serializer serializer;
+
+    public C2HttpClient(C2ClientConfig clientConfig, C2Serializer serializer) {
+        super();
+        this.clientConfig = clientConfig;
+        this.serializer = serializer;
+        final OkHttpClient.Builder okHttpClientBuilder = new OkHttpClient.Builder();
+
+        // Set whether to follow redirects
+        okHttpClientBuilder.followRedirects(true);
+
+        // check if the ssl path is set and add the factory if so
+        if (StringUtils.isNotBlank(clientConfig.getKeystoreFilename())) {
+            try {
+                setSslSocketFactory(okHttpClientBuilder);
+            } catch (Exception e) {
+                throw new IllegalStateException(e);
+            }
+        }
+
+        httpClientReference.set(okHttpClientBuilder.build());
+    }
+
+    @Override
+    public Optional<C2HeartbeatResponse> publishHeartbeat(C2Heartbeat heartbeat) {
+        return serializer.serialize(heartbeat).flatMap(this::sendHeartbeat);
+    }
+
+    private Optional<C2HeartbeatResponse> sendHeartbeat(String heartbeat) {
+        logger.debug("Sending heartbeat to {}", clientConfig.getC2Url());
+
+        Optional<C2HeartbeatResponse> c2HeartbeatResponse = Optional.empty();
+        Request request = new Request.Builder()
+            .post(RequestBody.create(heartbeat, MEDIA_TYPE_APPLICATION_JSON))
+            .url(clientConfig.getC2Url())
+            .build();
+
+        try {
+            Response heartbeatResponse = httpClientReference.get().newCall(request).execute();
+            c2HeartbeatResponse = getResponseBody(heartbeatResponse).flatMap(response -> serializer.deserialize(response, C2HeartbeatResponse.class));
+        } catch (IOException ce) {
+            logger.error("Error while sending heartbeat", ce);

Review Comment:
   Recommend including the URL in the error message:
   ```suggestion
               logger.error("Send Heartbeat failed [{}]", clientConfig.getC2Url(), ce);
   ```



##########
c2/c2-client-bundle/c2-client-api/src/main/java/org/apache/nifi/c2/client/api/C2Properties.java:
##########
@@ -0,0 +1,141 @@
+/*
+ * 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.nifi.c2.client.api;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Properties;
+import java.util.concurrent.TimeUnit;
+
+public class C2Properties {
+
+    private static final Logger logger = LoggerFactory.getLogger(C2Properties.class);
+
+    public static final String NIFI_PREFIX = "nifi.";
+
+    public static final String C2_PROPERTY_BASE = NIFI_PREFIX + "c2.";
+    public static final String C2_ENABLE_KEY = C2_PROPERTY_BASE + "enable";
+    public static final String C2_AGENT_PROTOCOL_KEY = C2_PROPERTY_BASE + "agent.protocol.class";
+    public static final String C2_REST_URL_KEY = C2_PROPERTY_BASE + "rest.url";
+    public static final String C2_REST_URL_ACK_KEY = C2_PROPERTY_BASE + "rest.url.ack";
+    public static final String C2_AGENT_HEARTBEAT_PERIOD_KEY = C2_PROPERTY_BASE + "agent.heartbeat.period";
+    public static final String C2_AGENT_CLASS_KEY = C2_PROPERTY_BASE + "agent.class";
+    public static final String C2_AGENT_IDENTIFIER_KEY = C2_PROPERTY_BASE + "agent.identifier";
+
+    public static final String C2_ROOT_CLASS_DEFINITIONS_KEY = C2_PROPERTY_BASE + "root.class.definitions";
+    public static final String C2_METRICS_NAME_KEY = C2_ROOT_CLASS_DEFINITIONS_KEY + ".metrics.name";
+    public static final String C2_METRICS_METRICS_KEY = C2_ROOT_CLASS_DEFINITIONS_KEY + ".metrics.metrics";
+    public static final String C2_METRICS_METRICS_TYPED_METRICS_NAME_KEY = C2_ROOT_CLASS_DEFINITIONS_KEY + ".metrics.metrics.typedmetrics.name";
+    public static final String C2_METRICS_METRICS_QUEUED_METRICS_NAME_KEY = C2_ROOT_CLASS_DEFINITIONS_KEY + ".metrics.metrics.queuemetrics.name";
+    public static final String C2_METRICS_METRICS_QUEUE_METRICS_CLASSES_KEY = C2_ROOT_CLASS_DEFINITIONS_KEY + ".metrics.metrics.queuemetrics.classes";
+    public static final String C2_METRICS_METRICS_TYPED_METRICS_CLASSES_KEY = C2_ROOT_CLASS_DEFINITIONS_KEY + ".metrics.metrics.typedmetrics.classes";
+    public static final String C2_METRICS_METRICS_PROCESSOR_METRICS_NAME_KEY = C2_ROOT_CLASS_DEFINITIONS_KEY + ".metrics.metrics.processorMetrics.name";
+    public static final String C2_METRICS_METRICS_PROCESSOR_METRICS_CLASSES_KEY = C2_ROOT_CLASS_DEFINITIONS_KEY + ".metrics.metrics.processorMetrics.classes";
+
+    /* REST Heartbeat Reporter Security Properties */
+    private static final String C2_REST_SECURITY_BASE_KEY = NIFI_PREFIX + "c2.security";
+    public static final String TRUSTSTORE_LOCATION_KEY = C2_REST_SECURITY_BASE_KEY + ".truststore.location";
+    public static final String TRUSTSTORE_PASSWORD_KEY = C2_REST_SECURITY_BASE_KEY + ".truststore.password";
+    public static final String TRUSTSTORE_TYPE_KEY = C2_REST_SECURITY_BASE_KEY + ".truststore.type";
+    public static final String KEYSTORE_LOCATION_KEY = C2_REST_SECURITY_BASE_KEY + ".keystore.location";
+    public static final String KEYSTORE_PASSWORD_KEY = C2_REST_SECURITY_BASE_KEY + ".keystore.password";
+    public static final String KEYSTORE_TYPE_KEY = C2_REST_SECURITY_BASE_KEY + ".keystore.type";
+    public static final String NEED_CLIENT_AUTH_KEY = C2_REST_SECURITY_BASE_KEY + ".need.client.auth";
+
+    // Defaults
+    // Heartbeat period of 1 second
+    public static final long C2_AGENT_DEFAULT_HEARTBEAT_PERIOD = TimeUnit.SECONDS.toMillis(1);

Review Comment:
   Although these public static variables follow the pattern of `NiFiProperties`, this seems like an opportunity to follow a different approach. Instead of public static variables, what do you think about using a separate public `enum` named something like `C2PropertyName`?



##########
c2/c2-client-bundle/c2-client-http/src/main/java/org/apache/nifi/c2/client/http/C2HttpClient.java:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.nifi.c2.client.http;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.security.KeyStore;
+import java.security.NoSuchAlgorithmException;
+import java.util.Optional;
+import java.util.concurrent.atomic.AtomicReference;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLSocketFactory;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+import okhttp3.MediaType;
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.RequestBody;
+import okhttp3.Response;
+import okhttp3.ResponseBody;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.c2.client.C2ClientConfig;
+import org.apache.nifi.c2.client.api.C2Client;
+import org.apache.nifi.c2.client.api.C2Serializer;
+import org.apache.nifi.c2.protocol.api.C2Heartbeat;
+import org.apache.nifi.c2.protocol.api.C2HeartbeatResponse;
+import org.apache.nifi.c2.protocol.api.C2OperationAck;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class C2HttpClient implements C2Client {
+
+    private static final Logger logger = LoggerFactory.getLogger(C2HttpClient.class);
+    private static final MediaType MEDIA_TYPE_APPLICATION_JSON = MediaType.parse("application/json");
+
+    private final AtomicReference<OkHttpClient> httpClientReference = new AtomicReference<>();
+    private final C2ClientConfig clientConfig;
+    private final C2Serializer serializer;
+
+    public C2HttpClient(C2ClientConfig clientConfig, C2Serializer serializer) {
+        super();
+        this.clientConfig = clientConfig;
+        this.serializer = serializer;
+        final OkHttpClient.Builder okHttpClientBuilder = new OkHttpClient.Builder();
+
+        // Set whether to follow redirects
+        okHttpClientBuilder.followRedirects(true);
+
+        // check if the ssl path is set and add the factory if so
+        if (StringUtils.isNotBlank(clientConfig.getKeystoreFilename())) {
+            try {
+                setSslSocketFactory(okHttpClientBuilder);
+            } catch (Exception e) {
+                throw new IllegalStateException(e);
+            }
+        }
+
+        httpClientReference.set(okHttpClientBuilder.build());
+    }
+
+    @Override
+    public Optional<C2HeartbeatResponse> publishHeartbeat(C2Heartbeat heartbeat) {
+        return serializer.serialize(heartbeat).flatMap(this::sendHeartbeat);
+    }
+
+    private Optional<C2HeartbeatResponse> sendHeartbeat(String heartbeat) {
+        logger.debug("Sending heartbeat to {}", clientConfig.getC2Url());
+
+        Optional<C2HeartbeatResponse> c2HeartbeatResponse = Optional.empty();
+        Request request = new Request.Builder()
+            .post(RequestBody.create(heartbeat, MEDIA_TYPE_APPLICATION_JSON))
+            .url(clientConfig.getC2Url())
+            .build();
+
+        try {
+            Response heartbeatResponse = httpClientReference.get().newCall(request).execute();
+            c2HeartbeatResponse = getResponseBody(heartbeatResponse).flatMap(response -> serializer.deserialize(response, C2HeartbeatResponse.class));
+        } catch (IOException ce) {
+            logger.error("Error while sending heartbeat", ce);
+        }
+
+        return c2HeartbeatResponse;
+    }
+
+    private Optional<String> getResponseBody(Response response) {
+        String responseBody = null;
+
+        try {
+            responseBody = response.body().string();
+            logger.debug("Received response (Status={}) {}", response.code(), responseBody);
+        } catch (IOException e) {
+            logger.error("Could not get response body: ", e);
+        }
+
+        return Optional.ofNullable(responseBody);
+    }
+
+    private void setSslSocketFactory(OkHttpClient.Builder okHttpClientBuilder) throws Exception {
+        final String keystoreLocation = clientConfig.getKeystoreFilename();
+        final String keystoreType = clientConfig.getKeystoreType().getType();
+        final String keystorePass = clientConfig.getKeystorePass();
+
+        assertKeystorePropertiesSet(keystoreLocation, keystorePass, keystoreType);
+
+        // prepare the keystore
+        final KeyStore keyStore = KeyStore.getInstance(keystoreType);
+
+        try (FileInputStream keyStoreStream = new FileInputStream(keystoreLocation)) {
+            keyStore.load(keyStoreStream, keystorePass.toCharArray());
+        }
+
+        final KeyManagerFactory keyManagerFactory = KeyManagerFactory.getInstance(KeyManagerFactory.getDefaultAlgorithm());
+        keyManagerFactory.init(keyStore, keystorePass.toCharArray());
+
+        // load truststore
+        final String truststoreLocation = clientConfig.getTruststoreFilename();
+        final String truststorePass = clientConfig.getTruststorePass();
+        final String truststoreType = clientConfig.getTruststoreType().getType();
+        assertTruststorePropertiesSet(truststoreLocation, truststorePass, truststoreType);
+
+        KeyStore truststore = KeyStore.getInstance(truststoreType);
+        final TrustManagerFactory trustManagerFactory = TrustManagerFactory.getInstance("X509");
+        truststore.load(new FileInputStream(truststoreLocation), truststorePass.toCharArray());
+        trustManagerFactory.init(truststore);
+
+        final X509TrustManager x509TrustManager;
+        TrustManager[] trustManagers = trustManagerFactory.getTrustManagers();
+        if (trustManagers[0] != null) {
+            x509TrustManager = (X509TrustManager) trustManagers[0];
+        } else {
+            throw new IllegalStateException("List of trust managers is null");
+        }
+
+        SSLContext tempSslContext;
+        try {
+            tempSslContext = SSLContext.getInstance("TLS");
+        } catch (NoSuchAlgorithmException e) {
+            logger.warn("Unable to use 'TLS' for the PullHttpChangeIngestor due to NoSuchAlgorithmException. Will attempt to use the default algorithm.", e);
+            tempSslContext = SSLContext.getDefault();
+        }
+
+        final SSLContext sslContext = tempSslContext;
+        sslContext.init(keyManagerFactory.getKeyManagers(), trustManagerFactory.getTrustManagers(), null);
+
+        final SSLSocketFactory sslSocketFactory = sslContext.getSocketFactory();
+        okHttpClientBuilder.sslSocketFactory(sslSocketFactory, x509TrustManager);
+    }
+
+    private void assertKeystorePropertiesSet(String location, String password, String type) {
+        if (location == null || location.isEmpty()) {
+            throw new IllegalArgumentException(clientConfig.getKeystoreFilename() + " is null or is empty");
+        }
+
+        if (password == null || password.isEmpty()) {
+            throw new IllegalArgumentException("The client's keystore filename is set but its password is not (or is empty). If the location is set, the password must also be.");
+        }
+
+        if (type == null || type.isEmpty()) {
+            throw new IllegalArgumentException("The client's keystore filename is set but its type is not (or is empty). If the location is set, the type must also be.");
+        }
+    }
+
+    private void assertTruststorePropertiesSet(String location, String password, String type) {
+        if (location == null || location.isEmpty()) {
+            throw new IllegalArgumentException("The client's truststore filename is not set or is empty");
+        }
+
+        if (password == null || password.isEmpty()) {
+            throw new IllegalArgumentException("The client's truststore filename is set but its password is not (or is empty). If the location is set, the password must also be.");
+        }
+
+        if (type == null || type.isEmpty()) {
+            throw new IllegalArgumentException("The client's truststore filename is set but its type is not (or is empty). If the location is set, the type must also be.");
+        }
+    }
+
+    @Override
+    public ByteBuffer retrieveUpdateContent(String flowUpdateUrl) {
+        final Request.Builder requestBuilder = new Request.Builder()
+                .get()
+                .url(flowUpdateUrl);
+        final Request request = requestBuilder.build();
+
+        ResponseBody body;
+        try (final Response response = httpClientReference.get().newCall(request).execute()) {
+            logger.debug("Response received: {}", response);
+
+            int code = response.code();
+            if (code >= 400) {
+                throw new IOException("Got response code " + code + " while trying to pull configuration: " + response.body().string());

Review Comment:
   Recommend adjusting the message. It may also be helpful to create a subclass of `IOException` that could help indicate the server returned an error versus some other kind of communication problem.
   ```suggestion
                  final String message = String.format("Configuration retrieval failed: HTTP %d %s", code, response.body().string());
                   throw new IOException(message);
   ```



##########
c2/c2-client-bundle/c2-client-http/src/main/java/org/apache/nifi/c2/client/http/C2HttpClient.java:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.nifi.c2.client.http;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.security.KeyStore;
+import java.security.NoSuchAlgorithmException;
+import java.util.Optional;
+import java.util.concurrent.atomic.AtomicReference;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLSocketFactory;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+import okhttp3.MediaType;
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.RequestBody;
+import okhttp3.Response;
+import okhttp3.ResponseBody;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.c2.client.C2ClientConfig;
+import org.apache.nifi.c2.client.api.C2Client;
+import org.apache.nifi.c2.client.api.C2Serializer;
+import org.apache.nifi.c2.protocol.api.C2Heartbeat;
+import org.apache.nifi.c2.protocol.api.C2HeartbeatResponse;
+import org.apache.nifi.c2.protocol.api.C2OperationAck;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class C2HttpClient implements C2Client {
+
+    private static final Logger logger = LoggerFactory.getLogger(C2HttpClient.class);
+    private static final MediaType MEDIA_TYPE_APPLICATION_JSON = MediaType.parse("application/json");
+
+    private final AtomicReference<OkHttpClient> httpClientReference = new AtomicReference<>();
+    private final C2ClientConfig clientConfig;
+    private final C2Serializer serializer;
+
+    public C2HttpClient(C2ClientConfig clientConfig, C2Serializer serializer) {
+        super();
+        this.clientConfig = clientConfig;
+        this.serializer = serializer;
+        final OkHttpClient.Builder okHttpClientBuilder = new OkHttpClient.Builder();
+
+        // Set whether to follow redirects
+        okHttpClientBuilder.followRedirects(true);
+
+        // check if the ssl path is set and add the factory if so
+        if (StringUtils.isNotBlank(clientConfig.getKeystoreFilename())) {
+            try {
+                setSslSocketFactory(okHttpClientBuilder);
+            } catch (Exception e) {
+                throw new IllegalStateException(e);
+            }
+        }
+
+        httpClientReference.set(okHttpClientBuilder.build());
+    }
+
+    @Override
+    public Optional<C2HeartbeatResponse> publishHeartbeat(C2Heartbeat heartbeat) {
+        return serializer.serialize(heartbeat).flatMap(this::sendHeartbeat);
+    }
+
+    private Optional<C2HeartbeatResponse> sendHeartbeat(String heartbeat) {
+        logger.debug("Sending heartbeat to {}", clientConfig.getC2Url());
+
+        Optional<C2HeartbeatResponse> c2HeartbeatResponse = Optional.empty();
+        Request request = new Request.Builder()
+            .post(RequestBody.create(heartbeat, MEDIA_TYPE_APPLICATION_JSON))
+            .url(clientConfig.getC2Url())
+            .build();
+
+        try {
+            Response heartbeatResponse = httpClientReference.get().newCall(request).execute();
+            c2HeartbeatResponse = getResponseBody(heartbeatResponse).flatMap(response -> serializer.deserialize(response, C2HeartbeatResponse.class));
+        } catch (IOException ce) {
+            logger.error("Error while sending heartbeat", ce);
+        }
+
+        return c2HeartbeatResponse;
+    }
+
+    private Optional<String> getResponseBody(Response response) {
+        String responseBody = null;
+
+        try {
+            responseBody = response.body().string();
+            logger.debug("Received response (Status={}) {}", response.code(), responseBody);
+        } catch (IOException e) {
+            logger.error("Could not get response body: ", e);
+        }
+
+        return Optional.ofNullable(responseBody);
+    }
+
+    private void setSslSocketFactory(OkHttpClient.Builder okHttpClientBuilder) throws Exception {
+        final String keystoreLocation = clientConfig.getKeystoreFilename();
+        final String keystoreType = clientConfig.getKeystoreType().getType();
+        final String keystorePass = clientConfig.getKeystorePass();
+
+        assertKeystorePropertiesSet(keystoreLocation, keystorePass, keystoreType);
+
+        // prepare the keystore
+        final KeyStore keyStore = KeyStore.getInstance(keystoreType);
+
+        try (FileInputStream keyStoreStream = new FileInputStream(keystoreLocation)) {
+            keyStore.load(keyStoreStream, keystorePass.toCharArray());
+        }
+
+        final KeyManagerFactory keyManagerFactory = KeyManagerFactory.getInstance(KeyManagerFactory.getDefaultAlgorithm());
+        keyManagerFactory.init(keyStore, keystorePass.toCharArray());
+
+        // load truststore
+        final String truststoreLocation = clientConfig.getTruststoreFilename();
+        final String truststorePass = clientConfig.getTruststorePass();
+        final String truststoreType = clientConfig.getTruststoreType().getType();
+        assertTruststorePropertiesSet(truststoreLocation, truststorePass, truststoreType);
+
+        KeyStore truststore = KeyStore.getInstance(truststoreType);
+        final TrustManagerFactory trustManagerFactory = TrustManagerFactory.getInstance("X509");
+        truststore.load(new FileInputStream(truststoreLocation), truststorePass.toCharArray());
+        trustManagerFactory.init(truststore);
+
+        final X509TrustManager x509TrustManager;
+        TrustManager[] trustManagers = trustManagerFactory.getTrustManagers();
+        if (trustManagers[0] != null) {
+            x509TrustManager = (X509TrustManager) trustManagers[0];
+        } else {
+            throw new IllegalStateException("List of trust managers is null");
+        }
+
+        SSLContext tempSslContext;
+        try {
+            tempSslContext = SSLContext.getInstance("TLS");
+        } catch (NoSuchAlgorithmException e) {
+            logger.warn("Unable to use 'TLS' for the PullHttpChangeIngestor due to NoSuchAlgorithmException. Will attempt to use the default algorithm.", e);
+            tempSslContext = SSLContext.getDefault();

Review Comment:
   As described in the [SSLContext.getDefault()](https://docs.oracle.com/javase/8/docs/api/javax/net/ssl/SSLContext.html#getDefault()) documentation, the default context does not require initialization, so it does not seem like this will work. Recommend throwing an exception instead.



##########
c2/c2-client-bundle/c2-client-http/src/main/java/org/apache/nifi/c2/client/http/C2HttpClient.java:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.nifi.c2.client.http;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.security.KeyStore;
+import java.security.NoSuchAlgorithmException;
+import java.util.Optional;
+import java.util.concurrent.atomic.AtomicReference;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLSocketFactory;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+import okhttp3.MediaType;
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.RequestBody;
+import okhttp3.Response;
+import okhttp3.ResponseBody;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.c2.client.C2ClientConfig;
+import org.apache.nifi.c2.client.api.C2Client;
+import org.apache.nifi.c2.client.api.C2Serializer;
+import org.apache.nifi.c2.protocol.api.C2Heartbeat;
+import org.apache.nifi.c2.protocol.api.C2HeartbeatResponse;
+import org.apache.nifi.c2.protocol.api.C2OperationAck;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class C2HttpClient implements C2Client {
+
+    private static final Logger logger = LoggerFactory.getLogger(C2HttpClient.class);
+    private static final MediaType MEDIA_TYPE_APPLICATION_JSON = MediaType.parse("application/json");
+
+    private final AtomicReference<OkHttpClient> httpClientReference = new AtomicReference<>();
+    private final C2ClientConfig clientConfig;
+    private final C2Serializer serializer;
+
+    public C2HttpClient(C2ClientConfig clientConfig, C2Serializer serializer) {
+        super();
+        this.clientConfig = clientConfig;
+        this.serializer = serializer;
+        final OkHttpClient.Builder okHttpClientBuilder = new OkHttpClient.Builder();
+
+        // Set whether to follow redirects
+        okHttpClientBuilder.followRedirects(true);
+
+        // check if the ssl path is set and add the factory if so
+        if (StringUtils.isNotBlank(clientConfig.getKeystoreFilename())) {
+            try {
+                setSslSocketFactory(okHttpClientBuilder);
+            } catch (Exception e) {
+                throw new IllegalStateException(e);
+            }
+        }
+
+        httpClientReference.set(okHttpClientBuilder.build());
+    }
+
+    @Override
+    public Optional<C2HeartbeatResponse> publishHeartbeat(C2Heartbeat heartbeat) {
+        return serializer.serialize(heartbeat).flatMap(this::sendHeartbeat);
+    }
+
+    private Optional<C2HeartbeatResponse> sendHeartbeat(String heartbeat) {
+        logger.debug("Sending heartbeat to {}", clientConfig.getC2Url());
+
+        Optional<C2HeartbeatResponse> c2HeartbeatResponse = Optional.empty();
+        Request request = new Request.Builder()
+            .post(RequestBody.create(heartbeat, MEDIA_TYPE_APPLICATION_JSON))
+            .url(clientConfig.getC2Url())
+            .build();
+
+        try {
+            Response heartbeatResponse = httpClientReference.get().newCall(request).execute();
+            c2HeartbeatResponse = getResponseBody(heartbeatResponse).flatMap(response -> serializer.deserialize(response, C2HeartbeatResponse.class));
+        } catch (IOException ce) {
+            logger.error("Error while sending heartbeat", ce);
+        }
+
+        return c2HeartbeatResponse;
+    }
+
+    private Optional<String> getResponseBody(Response response) {
+        String responseBody = null;
+
+        try {
+            responseBody = response.body().string();
+            logger.debug("Received response (Status={}) {}", response.code(), responseBody);
+        } catch (IOException e) {
+            logger.error("Could not get response body: ", e);
+        }
+
+        return Optional.ofNullable(responseBody);
+    }
+
+    private void setSslSocketFactory(OkHttpClient.Builder okHttpClientBuilder) throws Exception {
+        final String keystoreLocation = clientConfig.getKeystoreFilename();
+        final String keystoreType = clientConfig.getKeystoreType().getType();
+        final String keystorePass = clientConfig.getKeystorePass();
+
+        assertKeystorePropertiesSet(keystoreLocation, keystorePass, keystoreType);
+
+        // prepare the keystore
+        final KeyStore keyStore = KeyStore.getInstance(keystoreType);
+
+        try (FileInputStream keyStoreStream = new FileInputStream(keystoreLocation)) {
+            keyStore.load(keyStoreStream, keystorePass.toCharArray());
+        }
+
+        final KeyManagerFactory keyManagerFactory = KeyManagerFactory.getInstance(KeyManagerFactory.getDefaultAlgorithm());
+        keyManagerFactory.init(keyStore, keystorePass.toCharArray());
+
+        // load truststore
+        final String truststoreLocation = clientConfig.getTruststoreFilename();
+        final String truststorePass = clientConfig.getTruststorePass();
+        final String truststoreType = clientConfig.getTruststoreType().getType();
+        assertTruststorePropertiesSet(truststoreLocation, truststorePass, truststoreType);
+
+        KeyStore truststore = KeyStore.getInstance(truststoreType);
+        final TrustManagerFactory trustManagerFactory = TrustManagerFactory.getInstance("X509");
+        truststore.load(new FileInputStream(truststoreLocation), truststorePass.toCharArray());
+        trustManagerFactory.init(truststore);
+
+        final X509TrustManager x509TrustManager;
+        TrustManager[] trustManagers = trustManagerFactory.getTrustManagers();
+        if (trustManagers[0] != null) {
+            x509TrustManager = (X509TrustManager) trustManagers[0];
+        } else {
+            throw new IllegalStateException("List of trust managers is null");
+        }
+
+        SSLContext tempSslContext;
+        try {
+            tempSslContext = SSLContext.getInstance("TLS");
+        } catch (NoSuchAlgorithmException e) {
+            logger.warn("Unable to use 'TLS' for the PullHttpChangeIngestor due to NoSuchAlgorithmException. Will attempt to use the default algorithm.", e);

Review Comment:
   Recommend throwing an exception instead:
   ```suggestion
               throw new IllegalStateException("SSLContext creation failed", e);
   ```



##########
c2/c2-client-bundle/c2-client-http/src/main/java/org/apache/nifi/c2/client/http/C2HttpClient.java:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.nifi.c2.client.http;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.security.KeyStore;
+import java.security.NoSuchAlgorithmException;
+import java.util.Optional;
+import java.util.concurrent.atomic.AtomicReference;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLSocketFactory;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+import okhttp3.MediaType;
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.RequestBody;
+import okhttp3.Response;
+import okhttp3.ResponseBody;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.c2.client.C2ClientConfig;
+import org.apache.nifi.c2.client.api.C2Client;
+import org.apache.nifi.c2.client.api.C2Serializer;
+import org.apache.nifi.c2.protocol.api.C2Heartbeat;
+import org.apache.nifi.c2.protocol.api.C2HeartbeatResponse;
+import org.apache.nifi.c2.protocol.api.C2OperationAck;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class C2HttpClient implements C2Client {
+
+    private static final Logger logger = LoggerFactory.getLogger(C2HttpClient.class);
+    private static final MediaType MEDIA_TYPE_APPLICATION_JSON = MediaType.parse("application/json");
+
+    private final AtomicReference<OkHttpClient> httpClientReference = new AtomicReference<>();
+    private final C2ClientConfig clientConfig;
+    private final C2Serializer serializer;
+
+    public C2HttpClient(C2ClientConfig clientConfig, C2Serializer serializer) {
+        super();
+        this.clientConfig = clientConfig;
+        this.serializer = serializer;
+        final OkHttpClient.Builder okHttpClientBuilder = new OkHttpClient.Builder();
+
+        // Set whether to follow redirects
+        okHttpClientBuilder.followRedirects(true);
+
+        // check if the ssl path is set and add the factory if so
+        if (StringUtils.isNotBlank(clientConfig.getKeystoreFilename())) {
+            try {
+                setSslSocketFactory(okHttpClientBuilder);
+            } catch (Exception e) {
+                throw new IllegalStateException(e);
+            }
+        }
+
+        httpClientReference.set(okHttpClientBuilder.build());
+    }
+
+    @Override
+    public Optional<C2HeartbeatResponse> publishHeartbeat(C2Heartbeat heartbeat) {
+        return serializer.serialize(heartbeat).flatMap(this::sendHeartbeat);
+    }
+
+    private Optional<C2HeartbeatResponse> sendHeartbeat(String heartbeat) {
+        logger.debug("Sending heartbeat to {}", clientConfig.getC2Url());
+
+        Optional<C2HeartbeatResponse> c2HeartbeatResponse = Optional.empty();
+        Request request = new Request.Builder()
+            .post(RequestBody.create(heartbeat, MEDIA_TYPE_APPLICATION_JSON))
+            .url(clientConfig.getC2Url())
+            .build();
+
+        try {
+            Response heartbeatResponse = httpClientReference.get().newCall(request).execute();
+            c2HeartbeatResponse = getResponseBody(heartbeatResponse).flatMap(response -> serializer.deserialize(response, C2HeartbeatResponse.class));
+        } catch (IOException ce) {
+            logger.error("Error while sending heartbeat", ce);
+        }
+
+        return c2HeartbeatResponse;
+    }
+
+    private Optional<String> getResponseBody(Response response) {
+        String responseBody = null;
+
+        try {
+            responseBody = response.body().string();
+            logger.debug("Received response (Status={}) {}", response.code(), responseBody);
+        } catch (IOException e) {
+            logger.error("Could not get response body: ", e);

Review Comment:
   This error could be difficult to troubleshooting without some of the request information, it would be helpful to pass an include the request URL at minimum.
   ```suggestion
               logger.error("HTTP Request failed", e);
   ```



##########
c2/c2-client-bundle/c2-client-http/src/main/java/org/apache/nifi/c2/client/http/C2HttpClient.java:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.nifi.c2.client.http;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.security.KeyStore;
+import java.security.NoSuchAlgorithmException;
+import java.util.Optional;
+import java.util.concurrent.atomic.AtomicReference;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLSocketFactory;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+import okhttp3.MediaType;
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.RequestBody;
+import okhttp3.Response;
+import okhttp3.ResponseBody;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.c2.client.C2ClientConfig;
+import org.apache.nifi.c2.client.api.C2Client;
+import org.apache.nifi.c2.client.api.C2Serializer;
+import org.apache.nifi.c2.protocol.api.C2Heartbeat;
+import org.apache.nifi.c2.protocol.api.C2HeartbeatResponse;
+import org.apache.nifi.c2.protocol.api.C2OperationAck;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class C2HttpClient implements C2Client {
+
+    private static final Logger logger = LoggerFactory.getLogger(C2HttpClient.class);
+    private static final MediaType MEDIA_TYPE_APPLICATION_JSON = MediaType.parse("application/json");
+
+    private final AtomicReference<OkHttpClient> httpClientReference = new AtomicReference<>();
+    private final C2ClientConfig clientConfig;
+    private final C2Serializer serializer;
+
+    public C2HttpClient(C2ClientConfig clientConfig, C2Serializer serializer) {
+        super();
+        this.clientConfig = clientConfig;
+        this.serializer = serializer;
+        final OkHttpClient.Builder okHttpClientBuilder = new OkHttpClient.Builder();
+
+        // Set whether to follow redirects
+        okHttpClientBuilder.followRedirects(true);
+
+        // check if the ssl path is set and add the factory if so
+        if (StringUtils.isNotBlank(clientConfig.getKeystoreFilename())) {
+            try {
+                setSslSocketFactory(okHttpClientBuilder);
+            } catch (Exception e) {
+                throw new IllegalStateException(e);
+            }
+        }
+
+        httpClientReference.set(okHttpClientBuilder.build());
+    }
+
+    @Override
+    public Optional<C2HeartbeatResponse> publishHeartbeat(C2Heartbeat heartbeat) {
+        return serializer.serialize(heartbeat).flatMap(this::sendHeartbeat);
+    }
+
+    private Optional<C2HeartbeatResponse> sendHeartbeat(String heartbeat) {
+        logger.debug("Sending heartbeat to {}", clientConfig.getC2Url());
+
+        Optional<C2HeartbeatResponse> c2HeartbeatResponse = Optional.empty();
+        Request request = new Request.Builder()
+            .post(RequestBody.create(heartbeat, MEDIA_TYPE_APPLICATION_JSON))
+            .url(clientConfig.getC2Url())
+            .build();
+
+        try {
+            Response heartbeatResponse = httpClientReference.get().newCall(request).execute();
+            c2HeartbeatResponse = getResponseBody(heartbeatResponse).flatMap(response -> serializer.deserialize(response, C2HeartbeatResponse.class));
+        } catch (IOException ce) {
+            logger.error("Error while sending heartbeat", ce);
+        }
+
+        return c2HeartbeatResponse;
+    }
+
+    private Optional<String> getResponseBody(Response response) {
+        String responseBody = null;
+
+        try {
+            responseBody = response.body().string();
+            logger.debug("Received response (Status={}) {}", response.code(), responseBody);

Review Comment:
   OkHttp has some standard logging for HTTP requests and responses, which would be better to implement for full tracing, as opposed to debug logging of the status without having the request context information.



##########
c2/c2-client-bundle/c2-client-http/src/main/java/org/apache/nifi/c2/client/http/C2HttpClient.java:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.nifi.c2.client.http;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.security.KeyStore;
+import java.security.NoSuchAlgorithmException;
+import java.util.Optional;
+import java.util.concurrent.atomic.AtomicReference;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLSocketFactory;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+import okhttp3.MediaType;
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.RequestBody;
+import okhttp3.Response;
+import okhttp3.ResponseBody;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.c2.client.C2ClientConfig;
+import org.apache.nifi.c2.client.api.C2Client;
+import org.apache.nifi.c2.client.api.C2Serializer;
+import org.apache.nifi.c2.protocol.api.C2Heartbeat;
+import org.apache.nifi.c2.protocol.api.C2HeartbeatResponse;
+import org.apache.nifi.c2.protocol.api.C2OperationAck;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class C2HttpClient implements C2Client {
+
+    private static final Logger logger = LoggerFactory.getLogger(C2HttpClient.class);
+    private static final MediaType MEDIA_TYPE_APPLICATION_JSON = MediaType.parse("application/json");
+
+    private final AtomicReference<OkHttpClient> httpClientReference = new AtomicReference<>();
+    private final C2ClientConfig clientConfig;
+    private final C2Serializer serializer;
+
+    public C2HttpClient(C2ClientConfig clientConfig, C2Serializer serializer) {
+        super();
+        this.clientConfig = clientConfig;
+        this.serializer = serializer;
+        final OkHttpClient.Builder okHttpClientBuilder = new OkHttpClient.Builder();
+
+        // Set whether to follow redirects
+        okHttpClientBuilder.followRedirects(true);
+
+        // check if the ssl path is set and add the factory if so
+        if (StringUtils.isNotBlank(clientConfig.getKeystoreFilename())) {
+            try {
+                setSslSocketFactory(okHttpClientBuilder);
+            } catch (Exception e) {
+                throw new IllegalStateException(e);
+            }
+        }
+
+        httpClientReference.set(okHttpClientBuilder.build());
+    }
+
+    @Override
+    public Optional<C2HeartbeatResponse> publishHeartbeat(C2Heartbeat heartbeat) {
+        return serializer.serialize(heartbeat).flatMap(this::sendHeartbeat);
+    }
+
+    private Optional<C2HeartbeatResponse> sendHeartbeat(String heartbeat) {
+        logger.debug("Sending heartbeat to {}", clientConfig.getC2Url());
+
+        Optional<C2HeartbeatResponse> c2HeartbeatResponse = Optional.empty();
+        Request request = new Request.Builder()
+            .post(RequestBody.create(heartbeat, MEDIA_TYPE_APPLICATION_JSON))
+            .url(clientConfig.getC2Url())
+            .build();
+
+        try {
+            Response heartbeatResponse = httpClientReference.get().newCall(request).execute();
+            c2HeartbeatResponse = getResponseBody(heartbeatResponse).flatMap(response -> serializer.deserialize(response, C2HeartbeatResponse.class));
+        } catch (IOException ce) {
+            logger.error("Error while sending heartbeat", ce);
+        }
+
+        return c2HeartbeatResponse;
+    }
+
+    private Optional<String> getResponseBody(Response response) {
+        String responseBody = null;
+
+        try {
+            responseBody = response.body().string();
+            logger.debug("Received response (Status={}) {}", response.code(), responseBody);
+        } catch (IOException e) {
+            logger.error("Could not get response body: ", e);
+        }
+
+        return Optional.ofNullable(responseBody);
+    }
+
+    private void setSslSocketFactory(OkHttpClient.Builder okHttpClientBuilder) throws Exception {
+        final String keystoreLocation = clientConfig.getKeystoreFilename();
+        final String keystoreType = clientConfig.getKeystoreType().getType();
+        final String keystorePass = clientConfig.getKeystorePass();
+
+        assertKeystorePropertiesSet(keystoreLocation, keystorePass, keystoreType);
+
+        // prepare the keystore
+        final KeyStore keyStore = KeyStore.getInstance(keystoreType);
+
+        try (FileInputStream keyStoreStream = new FileInputStream(keystoreLocation)) {
+            keyStore.load(keyStoreStream, keystorePass.toCharArray());
+        }
+
+        final KeyManagerFactory keyManagerFactory = KeyManagerFactory.getInstance(KeyManagerFactory.getDefaultAlgorithm());
+        keyManagerFactory.init(keyStore, keystorePass.toCharArray());
+
+        // load truststore
+        final String truststoreLocation = clientConfig.getTruststoreFilename();
+        final String truststorePass = clientConfig.getTruststorePass();
+        final String truststoreType = clientConfig.getTruststoreType().getType();
+        assertTruststorePropertiesSet(truststoreLocation, truststorePass, truststoreType);
+
+        KeyStore truststore = KeyStore.getInstance(truststoreType);
+        final TrustManagerFactory trustManagerFactory = TrustManagerFactory.getInstance("X509");
+        truststore.load(new FileInputStream(truststoreLocation), truststorePass.toCharArray());
+        trustManagerFactory.init(truststore);
+
+        final X509TrustManager x509TrustManager;
+        TrustManager[] trustManagers = trustManagerFactory.getTrustManagers();
+        if (trustManagers[0] != null) {
+            x509TrustManager = (X509TrustManager) trustManagers[0];
+        } else {
+            throw new IllegalStateException("List of trust managers is null");
+        }
+
+        SSLContext tempSslContext;
+        try {
+            tempSslContext = SSLContext.getInstance("TLS");
+        } catch (NoSuchAlgorithmException e) {
+            logger.warn("Unable to use 'TLS' for the PullHttpChangeIngestor due to NoSuchAlgorithmException. Will attempt to use the default algorithm.", e);
+            tempSslContext = SSLContext.getDefault();
+        }
+
+        final SSLContext sslContext = tempSslContext;
+        sslContext.init(keyManagerFactory.getKeyManagers(), trustManagerFactory.getTrustManagers(), null);
+
+        final SSLSocketFactory sslSocketFactory = sslContext.getSocketFactory();
+        okHttpClientBuilder.sslSocketFactory(sslSocketFactory, x509TrustManager);
+    }
+
+    private void assertKeystorePropertiesSet(String location, String password, String type) {
+        if (location == null || location.isEmpty()) {
+            throw new IllegalArgumentException(clientConfig.getKeystoreFilename() + " is null or is empty");
+        }
+
+        if (password == null || password.isEmpty()) {
+            throw new IllegalArgumentException("The client's keystore filename is set but its password is not (or is empty). If the location is set, the password must also be.");
+        }
+
+        if (type == null || type.isEmpty()) {
+            throw new IllegalArgumentException("The client's keystore filename is set but its type is not (or is empty). If the location is set, the type must also be.");
+        }
+    }
+
+    private void assertTruststorePropertiesSet(String location, String password, String type) {
+        if (location == null || location.isEmpty()) {
+            throw new IllegalArgumentException("The client's truststore filename is not set or is empty");
+        }
+
+        if (password == null || password.isEmpty()) {
+            throw new IllegalArgumentException("The client's truststore filename is set but its password is not (or is empty). If the location is set, the password must also be.");
+        }
+
+        if (type == null || type.isEmpty()) {
+            throw new IllegalArgumentException("The client's truststore filename is set but its type is not (or is empty). If the location is set, the type must also be.");
+        }
+    }
+
+    @Override
+    public ByteBuffer retrieveUpdateContent(String flowUpdateUrl) {
+        final Request.Builder requestBuilder = new Request.Builder()
+                .get()
+                .url(flowUpdateUrl);
+        final Request request = requestBuilder.build();
+
+        ResponseBody body;
+        try (final Response response = httpClientReference.get().newCall(request).execute()) {
+            logger.debug("Response received: {}", response);
+
+            int code = response.code();
+            if (code >= 400) {
+                throw new IOException("Got response code " + code + " while trying to pull configuration: " + response.body().string());
+            }
+
+            body = response.body();
+
+            if (body == null) {
+                logger.warn("No body returned when pulling a new configuration");
+                return null;
+            }
+
+            final ByteBuffer bodyByteBuffer = ByteBuffer.wrap(body.bytes());
+            return bodyByteBuffer;
+
+        } catch (Exception e) {
+            logger.warn("Hit an exception while trying to pull", e);
+            return null;
+        }
+    }
+
+    @Override
+    public void acknowledgeOperation(C2OperationAck operationAck) {
+        logger.info("Performing acknowledgement request to {} for operation {}", clientConfig.getC2AckUrl(), operationAck.getOperationId());
+        final ObjectMapper jacksonObjectMapper = new ObjectMapper();
+        jacksonObjectMapper.setSerializationInclusion(JsonInclude.Include.NON_NULL);

Review Comment:
   The ObjectMapper should be instantiated once and reused, instead of being recreated in each method invocation.



##########
c2/c2-client-bundle/c2-client-service/src/main/java/org/apache/nifi/c2/client/service/C2ClientService.java:
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.nifi.c2.client.service;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.List;
+import java.util.function.Function;
+import org.apache.nifi.c2.client.api.C2Client;
+import org.apache.nifi.c2.client.service.model.RuntimeInfoWrapper;
+import org.apache.nifi.c2.client.service.operation.C2OperationService;
+import org.apache.nifi.c2.client.service.operation.UpdateConfigurationOperationHandler;
+import org.apache.nifi.c2.protocol.api.C2Heartbeat;
+import org.apache.nifi.c2.protocol.api.C2HeartbeatResponse;
+import org.apache.nifi.c2.protocol.api.C2Operation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class C2ClientService {
+
+    private static final Logger logger = LoggerFactory.getLogger(C2ClientService.class);
+
+    private final C2Client client;
+    private final C2HeartbeatFactory c2HeartbeatFactory;
+    private final C2OperationService operationService;
+    private final UpdateConfigurationOperationHandler updateConfigurationOperationHandler;
+
+    public C2ClientService(C2Client client, C2HeartbeatFactory c2HeartbeatFactory, FlowIdHolder flowIdHolder, Function<ByteBuffer, Boolean> updateFlow) {
+        this.client = client;
+        this.c2HeartbeatFactory = c2HeartbeatFactory;
+        this.updateConfigurationOperationHandler = new UpdateConfigurationOperationHandler(client, flowIdHolder, updateFlow);
+        this.operationService = new C2OperationService(Arrays.asList(updateConfigurationOperationHandler));
+    }
+
+    public void sendHeartbeat(RuntimeInfoWrapper runtimeInfoWrapper) {
+        try {
+            // TODO exception handling for all the C2 Client interactions (IOExceptions, logger.error vs logger.warn, etc.)
+            C2Heartbeat c2Heartbeat = c2HeartbeatFactory.create(runtimeInfoWrapper);
+            client.publishHeartbeat(c2Heartbeat).ifPresent(this::processResponse);
+        } catch (IOException ioe) {
+            // TODO

Review Comment:
   Is there a plan in place for addressing these TODO statements, or should they be addressed in this PR?



##########
c2/c2-client-bundle/c2-client-http/src/main/java/org/apache/nifi/c2/client/http/C2HttpClient.java:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.nifi.c2.client.http;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.security.KeyStore;
+import java.security.NoSuchAlgorithmException;
+import java.util.Optional;
+import java.util.concurrent.atomic.AtomicReference;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLSocketFactory;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+import okhttp3.MediaType;
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.RequestBody;
+import okhttp3.Response;
+import okhttp3.ResponseBody;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.c2.client.C2ClientConfig;
+import org.apache.nifi.c2.client.api.C2Client;
+import org.apache.nifi.c2.client.api.C2Serializer;
+import org.apache.nifi.c2.protocol.api.C2Heartbeat;
+import org.apache.nifi.c2.protocol.api.C2HeartbeatResponse;
+import org.apache.nifi.c2.protocol.api.C2OperationAck;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class C2HttpClient implements C2Client {
+
+    private static final Logger logger = LoggerFactory.getLogger(C2HttpClient.class);
+    private static final MediaType MEDIA_TYPE_APPLICATION_JSON = MediaType.parse("application/json");
+
+    private final AtomicReference<OkHttpClient> httpClientReference = new AtomicReference<>();
+    private final C2ClientConfig clientConfig;
+    private final C2Serializer serializer;
+
+    public C2HttpClient(C2ClientConfig clientConfig, C2Serializer serializer) {
+        super();
+        this.clientConfig = clientConfig;
+        this.serializer = serializer;
+        final OkHttpClient.Builder okHttpClientBuilder = new OkHttpClient.Builder();
+
+        // Set whether to follow redirects
+        okHttpClientBuilder.followRedirects(true);
+
+        // check if the ssl path is set and add the factory if so
+        if (StringUtils.isNotBlank(clientConfig.getKeystoreFilename())) {
+            try {
+                setSslSocketFactory(okHttpClientBuilder);
+            } catch (Exception e) {
+                throw new IllegalStateException(e);
+            }
+        }
+
+        httpClientReference.set(okHttpClientBuilder.build());
+    }
+
+    @Override
+    public Optional<C2HeartbeatResponse> publishHeartbeat(C2Heartbeat heartbeat) {
+        return serializer.serialize(heartbeat).flatMap(this::sendHeartbeat);
+    }
+
+    private Optional<C2HeartbeatResponse> sendHeartbeat(String heartbeat) {
+        logger.debug("Sending heartbeat to {}", clientConfig.getC2Url());
+
+        Optional<C2HeartbeatResponse> c2HeartbeatResponse = Optional.empty();
+        Request request = new Request.Builder()
+            .post(RequestBody.create(heartbeat, MEDIA_TYPE_APPLICATION_JSON))
+            .url(clientConfig.getC2Url())
+            .build();
+
+        try {
+            Response heartbeatResponse = httpClientReference.get().newCall(request).execute();
+            c2HeartbeatResponse = getResponseBody(heartbeatResponse).flatMap(response -> serializer.deserialize(response, C2HeartbeatResponse.class));
+        } catch (IOException ce) {
+            logger.error("Error while sending heartbeat", ce);
+        }
+
+        return c2HeartbeatResponse;
+    }
+
+    private Optional<String> getResponseBody(Response response) {
+        String responseBody = null;
+
+        try {
+            responseBody = response.body().string();
+            logger.debug("Received response (Status={}) {}", response.code(), responseBody);
+        } catch (IOException e) {
+            logger.error("Could not get response body: ", e);
+        }
+
+        return Optional.ofNullable(responseBody);
+    }
+
+    private void setSslSocketFactory(OkHttpClient.Builder okHttpClientBuilder) throws Exception {
+        final String keystoreLocation = clientConfig.getKeystoreFilename();
+        final String keystoreType = clientConfig.getKeystoreType().getType();
+        final String keystorePass = clientConfig.getKeystorePass();
+
+        assertKeystorePropertiesSet(keystoreLocation, keystorePass, keystoreType);
+
+        // prepare the keystore
+        final KeyStore keyStore = KeyStore.getInstance(keystoreType);
+
+        try (FileInputStream keyStoreStream = new FileInputStream(keystoreLocation)) {
+            keyStore.load(keyStoreStream, keystorePass.toCharArray());
+        }
+
+        final KeyManagerFactory keyManagerFactory = KeyManagerFactory.getInstance(KeyManagerFactory.getDefaultAlgorithm());
+        keyManagerFactory.init(keyStore, keystorePass.toCharArray());
+
+        // load truststore
+        final String truststoreLocation = clientConfig.getTruststoreFilename();
+        final String truststorePass = clientConfig.getTruststorePass();
+        final String truststoreType = clientConfig.getTruststoreType().getType();
+        assertTruststorePropertiesSet(truststoreLocation, truststorePass, truststoreType);
+
+        KeyStore truststore = KeyStore.getInstance(truststoreType);
+        final TrustManagerFactory trustManagerFactory = TrustManagerFactory.getInstance("X509");
+        truststore.load(new FileInputStream(truststoreLocation), truststorePass.toCharArray());
+        trustManagerFactory.init(truststore);
+
+        final X509TrustManager x509TrustManager;
+        TrustManager[] trustManagers = trustManagerFactory.getTrustManagers();
+        if (trustManagers[0] != null) {
+            x509TrustManager = (X509TrustManager) trustManagers[0];
+        } else {
+            throw new IllegalStateException("List of trust managers is null");
+        }
+
+        SSLContext tempSslContext;
+        try {
+            tempSslContext = SSLContext.getInstance("TLS");
+        } catch (NoSuchAlgorithmException e) {
+            logger.warn("Unable to use 'TLS' for the PullHttpChangeIngestor due to NoSuchAlgorithmException. Will attempt to use the default algorithm.", e);
+            tempSslContext = SSLContext.getDefault();
+        }
+
+        final SSLContext sslContext = tempSslContext;
+        sslContext.init(keyManagerFactory.getKeyManagers(), trustManagerFactory.getTrustManagers(), null);
+
+        final SSLSocketFactory sslSocketFactory = sslContext.getSocketFactory();
+        okHttpClientBuilder.sslSocketFactory(sslSocketFactory, x509TrustManager);
+    }
+
+    private void assertKeystorePropertiesSet(String location, String password, String type) {
+        if (location == null || location.isEmpty()) {
+            throw new IllegalArgumentException(clientConfig.getKeystoreFilename() + " is null or is empty");
+        }
+
+        if (password == null || password.isEmpty()) {
+            throw new IllegalArgumentException("The client's keystore filename is set but its password is not (or is empty). If the location is set, the password must also be.");
+        }
+
+        if (type == null || type.isEmpty()) {
+            throw new IllegalArgumentException("The client's keystore filename is set but its type is not (or is empty). If the location is set, the type must also be.");
+        }
+    }
+
+    private void assertTruststorePropertiesSet(String location, String password, String type) {
+        if (location == null || location.isEmpty()) {
+            throw new IllegalArgumentException("The client's truststore filename is not set or is empty");
+        }
+
+        if (password == null || password.isEmpty()) {
+            throw new IllegalArgumentException("The client's truststore filename is set but its password is not (or is empty). If the location is set, the password must also be.");
+        }
+
+        if (type == null || type.isEmpty()) {
+            throw new IllegalArgumentException("The client's truststore filename is set but its type is not (or is empty). If the location is set, the type must also be.");
+        }
+    }
+
+    @Override
+    public ByteBuffer retrieveUpdateContent(String flowUpdateUrl) {
+        final Request.Builder requestBuilder = new Request.Builder()
+                .get()
+                .url(flowUpdateUrl);
+        final Request request = requestBuilder.build();
+
+        ResponseBody body;
+        try (final Response response = httpClientReference.get().newCall(request).execute()) {
+            logger.debug("Response received: {}", response);
+
+            int code = response.code();
+            if (code >= 400) {
+                throw new IOException("Got response code " + code + " while trying to pull configuration: " + response.body().string());
+            }
+
+            body = response.body();
+
+            if (body == null) {
+                logger.warn("No body returned when pulling a new configuration");
+                return null;
+            }
+
+            final ByteBuffer bodyByteBuffer = ByteBuffer.wrap(body.bytes());
+            return bodyByteBuffer;
+
+        } catch (Exception e) {
+            logger.warn("Hit an exception while trying to pull", e);
+            return null;
+        }
+    }
+
+    @Override
+    public void acknowledgeOperation(C2OperationAck operationAck) {
+        logger.info("Performing acknowledgement request to {} for operation {}", clientConfig.getC2AckUrl(), operationAck.getOperationId());
+        final ObjectMapper jacksonObjectMapper = new ObjectMapper();
+        jacksonObjectMapper.setSerializationInclusion(JsonInclude.Include.NON_NULL);
+        try {
+            final String operationAckBody = jacksonObjectMapper.writeValueAsString(operationAck);
+
+            final RequestBody requestBody = RequestBody.create(operationAckBody, MediaType.parse("application/json"));
+            final Request.Builder requestBuilder = new Request.Builder()
+                    .post(requestBody)
+                    .url(clientConfig.getC2AckUrl());
+            final Response heartbeatResponse = httpClientReference.get().newCall(requestBuilder.build()).execute();
+            if (!heartbeatResponse.isSuccessful()) {
+                logger.warn("Acknowledgement was not successful.");
+            }
+            logger.trace("Status on acknowledgement was {}", heartbeatResponse.code());

Review Comment:
   Including the C2 server URL would be helpful for troubleshooting.



##########
c2/c2-client-bundle/c2-client-service/src/main/java/org/apache/nifi/c2/client/service/C2HeartbeatFactory.java:
##########
@@ -0,0 +1,248 @@
+/*
+ * 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.nifi.c2.client.service;
+
+import static org.apache.commons.lang3.StringUtils.isNotBlank;
+
+import java.io.File;
+import java.lang.management.ManagementFactory;
+import java.lang.management.OperatingSystemMXBean;
+import java.net.InetAddress;
+import java.net.NetworkInterface;
+import java.util.Enumeration;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.nifi.c2.client.C2ClientConfig;
+import org.apache.nifi.c2.client.PersistentUuidGenerator;
+import org.apache.nifi.c2.client.service.model.RuntimeInfoWrapper;
+import org.apache.nifi.c2.protocol.api.AgentInfo;
+import org.apache.nifi.c2.protocol.api.AgentRepositories;
+import org.apache.nifi.c2.protocol.api.AgentStatus;
+import org.apache.nifi.c2.protocol.api.C2Heartbeat;
+import org.apache.nifi.c2.protocol.api.DeviceInfo;
+import org.apache.nifi.c2.protocol.api.FlowInfo;
+import org.apache.nifi.c2.protocol.api.FlowQueueStatus;
+import org.apache.nifi.c2.protocol.api.NetworkInfo;
+import org.apache.nifi.c2.protocol.api.SystemInfo;
+import org.apache.nifi.c2.protocol.component.api.RuntimeManifest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class C2HeartbeatFactory {
+
+    private static final Logger logger = LoggerFactory.getLogger(C2HeartbeatFactory.class);
+
+    private static final String AGENT_IDENTIFIER_FILENAME = "agent-identifier";
+    private static final String DEVICE_IDENTIFIER_FILENAME = "device-identifier";
+
+    private final C2ClientConfig clientConfig;
+    private final FlowIdHolder flowIdHolder;
+
+    private String agentId;
+    private String deviceId;
+    private File confDirectory;
+
+    public C2HeartbeatFactory(C2ClientConfig clientConfig, FlowIdHolder flowIdHolder) {
+        this.clientConfig = clientConfig;
+        this.flowIdHolder = flowIdHolder;
+    }
+
+    public C2Heartbeat create(RuntimeInfoWrapper runtimeInfoWrapper) {
+        C2Heartbeat heartbeat = new C2Heartbeat();
+
+        heartbeat.setAgentInfo(getAgentInfo(runtimeInfoWrapper.getAgentRepositories(), runtimeInfoWrapper.getManifest()));
+        heartbeat.setDeviceInfo(generateDeviceInfo());
+        heartbeat.setFlowInfo(getFlowInfo(runtimeInfoWrapper.getQueueStatus()));
+        heartbeat.setCreated(System.currentTimeMillis());
+
+        return heartbeat;
+    }
+
+    //////
+    // FlowInfo
+    //////
+
+    private FlowInfo getFlowInfo(Map<String, FlowQueueStatus> queueStatus) {
+        FlowInfo flowInfo = new FlowInfo();
+        flowInfo.setQueues(queueStatus);
+        Optional.ofNullable(flowIdHolder.getFlowId()).ifPresent(flowInfo::setFlowId);
+        return flowInfo;
+    }
+
+    //////
+    // AgentInfo
+    //////
+
+    private AgentInfo getAgentInfo(AgentRepositories repos, RuntimeManifest manifest) {
+        AgentInfo agentInfo = new AgentInfo();
+        agentInfo.setAgentClass(clientConfig.getAgentClass());
+        agentInfo.setIdentifier(getAgentId());
+
+        AgentStatus agentStatus = new AgentStatus();
+        // TODO: Look into this, it doesn't look right
+        agentStatus.setUptime(System.currentTimeMillis());
+        agentStatus.setRepositories(repos);
+
+        agentInfo.setStatus(agentStatus);
+        agentInfo.setAgentManifest(manifest);
+
+        return agentInfo;
+    }
+
+    private String getAgentId() {
+        if (agentId == null) {
+            String rawAgentId = clientConfig.getAgentIdentifier();
+            if (isNotBlank(rawAgentId)) {
+                agentId = rawAgentId.trim();
+            } else {
+                File idFile = new File(getConfDirectory(), AGENT_IDENTIFIER_FILENAME);
+                agentId = new PersistentUuidGenerator(idFile).generate();
+            }
+        }
+
+        return agentId;
+    }
+
+    //////
+    // DeviceInfo
+    //////
+
+    private DeviceInfo generateDeviceInfo() {
+        // Populate DeviceInfo
+        final DeviceInfo deviceInfo = new DeviceInfo();
+        deviceInfo.setNetworkInfo(generateNetworkInfo());
+        deviceInfo.setIdentifier(getDeviceIdentifier(deviceInfo.getNetworkInfo()));
+        deviceInfo.setSystemInfo(generateSystemInfo());
+        return deviceInfo;
+    }
+
+    private NetworkInfo generateNetworkInfo() {
+        NetworkInfo networkInfo = new NetworkInfo();
+        try {
+            // Determine all interfaces
+            final Enumeration<NetworkInterface> networkInterfaces = NetworkInterface.getNetworkInterfaces();
+
+            final Set<NetworkInterface> operationIfaces = new HashSet<>();
+
+            // Determine eligible interfaces
+            while (networkInterfaces.hasMoreElements()) {
+                final NetworkInterface networkInterface = networkInterfaces.nextElement();
+                if (!networkInterface.isLoopback() && networkInterface.isUp()) {
+                    operationIfaces.add(networkInterface);
+                }
+            }
+            logger.trace("Have {} interfaces with names {}", operationIfaces.size(),
+                operationIfaces.stream()
+                    .map(NetworkInterface::getName)
+                    .collect(Collectors.toSet())
+            );
+
+            if (!operationIfaces.isEmpty()) {
+                if (operationIfaces.size() > 1) {
+                    logger.debug("Instance has multiple interfaces.  Generated information may be non-deterministic.");
+                }
+
+                NetworkInterface iface = operationIfaces.iterator().next();
+                final StringBuilder macSb = new StringBuilder();
+                byte[] hardwareAddress = iface.getHardwareAddress();
+                if (hardwareAddress != null) {
+                    for (int i = 0; i < hardwareAddress.length; i++) {
+                        macSb.append(String.format("%02X", hardwareAddress[i]));
+                    }
+                }
+                final String macString = macSb.toString();
+                Enumeration<InetAddress> inetAddresses = iface.getInetAddresses();
+                while (inetAddresses.hasMoreElements()) {
+                    InetAddress inetAddress = inetAddresses.nextElement();
+                    String hostAddress = inetAddress.getHostAddress();
+                    String hostName = inetAddress.getHostName();
+                    byte[] address = inetAddress.getAddress();
+                    String canonicalHostName = inetAddress.getCanonicalHostName();
+
+                    networkInfo.setDeviceId(iface.getName());
+                    networkInfo.setHostname(hostName);
+                    networkInfo.setIpAddress(hostAddress);
+                }
+            }
+        } catch (
+            Exception e) {
+            logger.error("Had exception determining network information", e);
+        }
+        return networkInfo;
+    }
+
+    private String getDeviceIdentifier(NetworkInfo networkInfo) {
+        if (deviceId == null) {
+            if (networkInfo.getDeviceId() != null) {
+                try {
+                    final NetworkInterface netInterface = NetworkInterface.getByName(networkInfo.getDeviceId());
+                    byte[] hardwareAddress = netInterface.getHardwareAddress();
+                    final StringBuilder macBuilder = new StringBuilder();
+                    if (hardwareAddress != null) {
+                        for (byte address : hardwareAddress) {
+                            macBuilder.append(String.format("%02X", address));

Review Comment:
   It looks like the MAC Address formatting occurs in several places, so it would be helpful to create a reusable private method.



##########
c2/c2-client-bundle/c2-client-service/src/main/java/org/apache/nifi/c2/client/service/C2HeartbeatFactory.java:
##########
@@ -0,0 +1,248 @@
+/*
+ * 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.nifi.c2.client.service;
+
+import static org.apache.commons.lang3.StringUtils.isNotBlank;
+
+import java.io.File;
+import java.lang.management.ManagementFactory;
+import java.lang.management.OperatingSystemMXBean;
+import java.net.InetAddress;
+import java.net.NetworkInterface;
+import java.util.Enumeration;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.nifi.c2.client.C2ClientConfig;
+import org.apache.nifi.c2.client.PersistentUuidGenerator;
+import org.apache.nifi.c2.client.service.model.RuntimeInfoWrapper;
+import org.apache.nifi.c2.protocol.api.AgentInfo;
+import org.apache.nifi.c2.protocol.api.AgentRepositories;
+import org.apache.nifi.c2.protocol.api.AgentStatus;
+import org.apache.nifi.c2.protocol.api.C2Heartbeat;
+import org.apache.nifi.c2.protocol.api.DeviceInfo;
+import org.apache.nifi.c2.protocol.api.FlowInfo;
+import org.apache.nifi.c2.protocol.api.FlowQueueStatus;
+import org.apache.nifi.c2.protocol.api.NetworkInfo;
+import org.apache.nifi.c2.protocol.api.SystemInfo;
+import org.apache.nifi.c2.protocol.component.api.RuntimeManifest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class C2HeartbeatFactory {
+
+    private static final Logger logger = LoggerFactory.getLogger(C2HeartbeatFactory.class);
+
+    private static final String AGENT_IDENTIFIER_FILENAME = "agent-identifier";
+    private static final String DEVICE_IDENTIFIER_FILENAME = "device-identifier";
+
+    private final C2ClientConfig clientConfig;
+    private final FlowIdHolder flowIdHolder;
+
+    private String agentId;
+    private String deviceId;
+    private File confDirectory;
+
+    public C2HeartbeatFactory(C2ClientConfig clientConfig, FlowIdHolder flowIdHolder) {
+        this.clientConfig = clientConfig;
+        this.flowIdHolder = flowIdHolder;
+    }
+
+    public C2Heartbeat create(RuntimeInfoWrapper runtimeInfoWrapper) {
+        C2Heartbeat heartbeat = new C2Heartbeat();
+
+        heartbeat.setAgentInfo(getAgentInfo(runtimeInfoWrapper.getAgentRepositories(), runtimeInfoWrapper.getManifest()));
+        heartbeat.setDeviceInfo(generateDeviceInfo());
+        heartbeat.setFlowInfo(getFlowInfo(runtimeInfoWrapper.getQueueStatus()));
+        heartbeat.setCreated(System.currentTimeMillis());
+
+        return heartbeat;
+    }
+
+    //////
+    // FlowInfo
+    //////
+
+    private FlowInfo getFlowInfo(Map<String, FlowQueueStatus> queueStatus) {
+        FlowInfo flowInfo = new FlowInfo();
+        flowInfo.setQueues(queueStatus);
+        Optional.ofNullable(flowIdHolder.getFlowId()).ifPresent(flowInfo::setFlowId);
+        return flowInfo;
+    }
+
+    //////
+    // AgentInfo
+    //////
+
+    private AgentInfo getAgentInfo(AgentRepositories repos, RuntimeManifest manifest) {
+        AgentInfo agentInfo = new AgentInfo();
+        agentInfo.setAgentClass(clientConfig.getAgentClass());
+        agentInfo.setIdentifier(getAgentId());
+
+        AgentStatus agentStatus = new AgentStatus();
+        // TODO: Look into this, it doesn't look right
+        agentStatus.setUptime(System.currentTimeMillis());
+        agentStatus.setRepositories(repos);
+
+        agentInfo.setStatus(agentStatus);
+        agentInfo.setAgentManifest(manifest);
+
+        return agentInfo;
+    }
+
+    private String getAgentId() {
+        if (agentId == null) {
+            String rawAgentId = clientConfig.getAgentIdentifier();
+            if (isNotBlank(rawAgentId)) {
+                agentId = rawAgentId.trim();
+            } else {
+                File idFile = new File(getConfDirectory(), AGENT_IDENTIFIER_FILENAME);
+                agentId = new PersistentUuidGenerator(idFile).generate();
+            }
+        }
+
+        return agentId;
+    }
+
+    //////
+    // DeviceInfo
+    //////
+
+    private DeviceInfo generateDeviceInfo() {
+        // Populate DeviceInfo
+        final DeviceInfo deviceInfo = new DeviceInfo();
+        deviceInfo.setNetworkInfo(generateNetworkInfo());
+        deviceInfo.setIdentifier(getDeviceIdentifier(deviceInfo.getNetworkInfo()));
+        deviceInfo.setSystemInfo(generateSystemInfo());
+        return deviceInfo;
+    }
+
+    private NetworkInfo generateNetworkInfo() {
+        NetworkInfo networkInfo = new NetworkInfo();
+        try {
+            // Determine all interfaces
+            final Enumeration<NetworkInterface> networkInterfaces = NetworkInterface.getNetworkInterfaces();
+
+            final Set<NetworkInterface> operationIfaces = new HashSet<>();
+
+            // Determine eligible interfaces
+            while (networkInterfaces.hasMoreElements()) {
+                final NetworkInterface networkInterface = networkInterfaces.nextElement();
+                if (!networkInterface.isLoopback() && networkInterface.isUp()) {
+                    operationIfaces.add(networkInterface);
+                }
+            }
+            logger.trace("Have {} interfaces with names {}", operationIfaces.size(),
+                operationIfaces.stream()
+                    .map(NetworkInterface::getName)
+                    .collect(Collectors.toSet())
+            );
+
+            if (!operationIfaces.isEmpty()) {
+                if (operationIfaces.size() > 1) {
+                    logger.debug("Instance has multiple interfaces.  Generated information may be non-deterministic.");
+                }
+
+                NetworkInterface iface = operationIfaces.iterator().next();
+                final StringBuilder macSb = new StringBuilder();
+                byte[] hardwareAddress = iface.getHardwareAddress();
+                if (hardwareAddress != null) {
+                    for (int i = 0; i < hardwareAddress.length; i++) {
+                        macSb.append(String.format("%02X", hardwareAddress[i]));
+                    }
+                }
+                final String macString = macSb.toString();
+                Enumeration<InetAddress> inetAddresses = iface.getInetAddresses();
+                while (inetAddresses.hasMoreElements()) {
+                    InetAddress inetAddress = inetAddresses.nextElement();
+                    String hostAddress = inetAddress.getHostAddress();
+                    String hostName = inetAddress.getHostName();
+                    byte[] address = inetAddress.getAddress();
+                    String canonicalHostName = inetAddress.getCanonicalHostName();
+
+                    networkInfo.setDeviceId(iface.getName());
+                    networkInfo.setHostname(hostName);
+                    networkInfo.setIpAddress(hostAddress);
+                }
+            }
+        } catch (
+            Exception e) {
+            logger.error("Had exception determining network information", e);

Review Comment:
   Recommend adjusting the wording:
   ```suggestion
           } catch (
               Exception e) {
               logger.error("Network Interface processing failed", e);
   ```



##########
c2/c2-client-bundle/c2-client-service/src/main/java/org/apache/nifi/c2/client/service/C2HeartbeatFactory.java:
##########
@@ -0,0 +1,248 @@
+/*
+ * 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.nifi.c2.client.service;
+
+import static org.apache.commons.lang3.StringUtils.isNotBlank;
+
+import java.io.File;
+import java.lang.management.ManagementFactory;
+import java.lang.management.OperatingSystemMXBean;
+import java.net.InetAddress;
+import java.net.NetworkInterface;
+import java.util.Enumeration;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.nifi.c2.client.C2ClientConfig;
+import org.apache.nifi.c2.client.PersistentUuidGenerator;
+import org.apache.nifi.c2.client.service.model.RuntimeInfoWrapper;
+import org.apache.nifi.c2.protocol.api.AgentInfo;
+import org.apache.nifi.c2.protocol.api.AgentRepositories;
+import org.apache.nifi.c2.protocol.api.AgentStatus;
+import org.apache.nifi.c2.protocol.api.C2Heartbeat;
+import org.apache.nifi.c2.protocol.api.DeviceInfo;
+import org.apache.nifi.c2.protocol.api.FlowInfo;
+import org.apache.nifi.c2.protocol.api.FlowQueueStatus;
+import org.apache.nifi.c2.protocol.api.NetworkInfo;
+import org.apache.nifi.c2.protocol.api.SystemInfo;
+import org.apache.nifi.c2.protocol.component.api.RuntimeManifest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class C2HeartbeatFactory {
+
+    private static final Logger logger = LoggerFactory.getLogger(C2HeartbeatFactory.class);
+
+    private static final String AGENT_IDENTIFIER_FILENAME = "agent-identifier";
+    private static final String DEVICE_IDENTIFIER_FILENAME = "device-identifier";
+
+    private final C2ClientConfig clientConfig;
+    private final FlowIdHolder flowIdHolder;
+
+    private String agentId;
+    private String deviceId;
+    private File confDirectory;
+
+    public C2HeartbeatFactory(C2ClientConfig clientConfig, FlowIdHolder flowIdHolder) {
+        this.clientConfig = clientConfig;
+        this.flowIdHolder = flowIdHolder;
+    }
+
+    public C2Heartbeat create(RuntimeInfoWrapper runtimeInfoWrapper) {
+        C2Heartbeat heartbeat = new C2Heartbeat();
+
+        heartbeat.setAgentInfo(getAgentInfo(runtimeInfoWrapper.getAgentRepositories(), runtimeInfoWrapper.getManifest()));
+        heartbeat.setDeviceInfo(generateDeviceInfo());
+        heartbeat.setFlowInfo(getFlowInfo(runtimeInfoWrapper.getQueueStatus()));
+        heartbeat.setCreated(System.currentTimeMillis());
+
+        return heartbeat;
+    }
+
+    //////
+    // FlowInfo
+    //////
+
+    private FlowInfo getFlowInfo(Map<String, FlowQueueStatus> queueStatus) {
+        FlowInfo flowInfo = new FlowInfo();
+        flowInfo.setQueues(queueStatus);
+        Optional.ofNullable(flowIdHolder.getFlowId()).ifPresent(flowInfo::setFlowId);
+        return flowInfo;
+    }
+
+    //////
+    // AgentInfo
+    //////
+
+    private AgentInfo getAgentInfo(AgentRepositories repos, RuntimeManifest manifest) {
+        AgentInfo agentInfo = new AgentInfo();
+        agentInfo.setAgentClass(clientConfig.getAgentClass());
+        agentInfo.setIdentifier(getAgentId());
+
+        AgentStatus agentStatus = new AgentStatus();
+        // TODO: Look into this, it doesn't look right
+        agentStatus.setUptime(System.currentTimeMillis());

Review Comment:
   It sounds like this needs to be addressed?



##########
c2/c2-client-bundle/c2-client-service/src/main/java/org/apache/nifi/c2/client/service/operation/UpdateConfigurationOperationHandler.java:
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.nifi.c2.client.service.operation;
+
+import static org.apache.commons.lang3.StringUtils.EMPTY;
+import static org.apache.nifi.c2.protocol.api.OperandType.CONFIGURATION;
+import static org.apache.nifi.c2.protocol.api.OperationType.UPDATE;
+
+import java.net.URI;
+import java.nio.ByteBuffer;
+import java.util.Optional;
+import java.util.function.Function;
+import org.apache.nifi.c2.client.api.C2Client;
+import org.apache.nifi.c2.client.service.FlowIdHolder;
+import org.apache.nifi.c2.protocol.api.C2Operation;
+import org.apache.nifi.c2.protocol.api.C2OperationAck;
+import org.apache.nifi.c2.protocol.api.C2OperationState;
+import org.apache.nifi.c2.protocol.api.OperandType;
+import org.apache.nifi.c2.protocol.api.OperationType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class UpdateConfigurationOperationHandler implements C2OperationHandler {
+
+    private static final Logger logger = LoggerFactory.getLogger(UpdateConfigurationOperationHandler.class);
+
+    private static final String LOCATION = "location";
+
+    private final C2Client client;
+    private final Function<ByteBuffer, Boolean> updateFlow;
+    private final FlowIdHolder flowIdHolder;
+
+    public UpdateConfigurationOperationHandler(C2Client client, FlowIdHolder flowIdHolder, Function<ByteBuffer, Boolean> updateFlow) {
+        this.client = client;
+        this.updateFlow = updateFlow;
+        this.flowIdHolder = flowIdHolder;
+    }
+
+    @Override
+    public OperationType getOperationType() {
+        return UPDATE;
+    }
+
+    @Override
+    public OperandType getOperandType() {
+        return CONFIGURATION;
+    }
+
+    @Override
+    public C2OperationAck handle(C2Operation operation) {
+        String opIdentifier = Optional.ofNullable(operation.getIdentifier())
+            .orElse(EMPTY);
+        C2OperationAck operationAck = new C2OperationAck();
+        C2OperationState state = new C2OperationState();
+        operationAck.setOperationState(state);
+        operationAck.setOperationId(opIdentifier);
+
+        String updateLocation = Optional.ofNullable(operation.getArgs())
+            .map(map -> map.get(LOCATION))
+            .orElse(EMPTY);
+
+        String newFlowId = parseFlowId(updateLocation);
+        if (flowIdHolder.getFlowId() == null || !flowIdHolder.getFlowId().equals(newFlowId)) {
+            logger.info("Will perform flow update from {} for operation #{}. Previous flow id was {}, replacing with new id {}", updateLocation, opIdentifier,
+                flowIdHolder.getFlowId() == null ? "not set" : flowIdHolder.getFlowId(), newFlowId);
+        } else {
+            logger.info("Flow is current, no update is necessary...");
+        }
+
+        flowIdHolder.setFlowId(newFlowId);
+        ByteBuffer updateContent = client.retrieveUpdateContent(updateLocation);
+        if (updateContent != null) {
+            if (updateFlow.apply(updateContent)) {
+                state.setState(C2OperationState.OperationState.FULLY_APPLIED);
+                logger.debug("Update configuration applied for operation #{}.", opIdentifier);
+            } else {
+                state.setState(C2OperationState.OperationState.NOT_APPLIED);
+                logger.error("Update resulted in error for operation #{}.", opIdentifier);
+            }
+        } else {
+            state.setState(C2OperationState.OperationState.NOT_APPLIED);
+            logger.error("Update content retrieval resulted in empty content so flow update was omitted for operation #{}.", opIdentifier);
+        }
+
+        return operationAck;
+    }
+
+    private String parseFlowId(String flowUpdateUrl) {
+        try {
+            URI flowUri = new URI(flowUpdateUrl);
+            String flowUriPath = flowUri.getPath();
+            String[] split = flowUriPath.split("/");
+            String flowId = split[4];

Review Comment:
   It would be helpful to check the split length to avoid unexpected exceptions.



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/util/ProcessUtils.java:
##########
@@ -0,0 +1,131 @@
+/*
+ * 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.nifi.minifi.bootstrap.util;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.Reader;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ProcessUtils {

Review Comment:
   It might be worthwhile to name this something like `UnixProcessUtils` to help indicate that these commands will not work on Windows.



##########
c2/c2-client-bundle/c2-client-http/src/main/java/org/apache/nifi/c2/client/http/C2HttpClient.java:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.nifi.c2.client.http;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.security.KeyStore;
+import java.security.NoSuchAlgorithmException;
+import java.util.Optional;
+import java.util.concurrent.atomic.AtomicReference;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLSocketFactory;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+import okhttp3.MediaType;
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.RequestBody;
+import okhttp3.Response;
+import okhttp3.ResponseBody;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.c2.client.C2ClientConfig;
+import org.apache.nifi.c2.client.api.C2Client;
+import org.apache.nifi.c2.client.api.C2Serializer;
+import org.apache.nifi.c2.protocol.api.C2Heartbeat;
+import org.apache.nifi.c2.protocol.api.C2HeartbeatResponse;
+import org.apache.nifi.c2.protocol.api.C2OperationAck;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class C2HttpClient implements C2Client {
+
+    private static final Logger logger = LoggerFactory.getLogger(C2HttpClient.class);
+    private static final MediaType MEDIA_TYPE_APPLICATION_JSON = MediaType.parse("application/json");
+
+    private final AtomicReference<OkHttpClient> httpClientReference = new AtomicReference<>();
+    private final C2ClientConfig clientConfig;
+    private final C2Serializer serializer;
+
+    public C2HttpClient(C2ClientConfig clientConfig, C2Serializer serializer) {
+        super();
+        this.clientConfig = clientConfig;
+        this.serializer = serializer;
+        final OkHttpClient.Builder okHttpClientBuilder = new OkHttpClient.Builder();
+
+        // Set whether to follow redirects
+        okHttpClientBuilder.followRedirects(true);
+
+        // check if the ssl path is set and add the factory if so
+        if (StringUtils.isNotBlank(clientConfig.getKeystoreFilename())) {
+            try {
+                setSslSocketFactory(okHttpClientBuilder);
+            } catch (Exception e) {
+                throw new IllegalStateException(e);
+            }
+        }
+
+        httpClientReference.set(okHttpClientBuilder.build());
+    }
+
+    @Override
+    public Optional<C2HeartbeatResponse> publishHeartbeat(C2Heartbeat heartbeat) {
+        return serializer.serialize(heartbeat).flatMap(this::sendHeartbeat);
+    }
+
+    private Optional<C2HeartbeatResponse> sendHeartbeat(String heartbeat) {
+        logger.debug("Sending heartbeat to {}", clientConfig.getC2Url());
+
+        Optional<C2HeartbeatResponse> c2HeartbeatResponse = Optional.empty();
+        Request request = new Request.Builder()
+            .post(RequestBody.create(heartbeat, MEDIA_TYPE_APPLICATION_JSON))
+            .url(clientConfig.getC2Url())
+            .build();
+
+        try {
+            Response heartbeatResponse = httpClientReference.get().newCall(request).execute();
+            c2HeartbeatResponse = getResponseBody(heartbeatResponse).flatMap(response -> serializer.deserialize(response, C2HeartbeatResponse.class));
+        } catch (IOException ce) {
+            logger.error("Error while sending heartbeat", ce);
+        }
+
+        return c2HeartbeatResponse;
+    }
+
+    private Optional<String> getResponseBody(Response response) {
+        String responseBody = null;
+
+        try {
+            responseBody = response.body().string();
+            logger.debug("Received response (Status={}) {}", response.code(), responseBody);
+        } catch (IOException e) {
+            logger.error("Could not get response body: ", e);
+        }
+
+        return Optional.ofNullable(responseBody);
+    }
+
+    private void setSslSocketFactory(OkHttpClient.Builder okHttpClientBuilder) throws Exception {
+        final String keystoreLocation = clientConfig.getKeystoreFilename();
+        final String keystoreType = clientConfig.getKeystoreType().getType();
+        final String keystorePass = clientConfig.getKeystorePass();
+
+        assertKeystorePropertiesSet(keystoreLocation, keystorePass, keystoreType);
+
+        // prepare the keystore
+        final KeyStore keyStore = KeyStore.getInstance(keystoreType);
+
+        try (FileInputStream keyStoreStream = new FileInputStream(keystoreLocation)) {
+            keyStore.load(keyStoreStream, keystorePass.toCharArray());
+        }
+
+        final KeyManagerFactory keyManagerFactory = KeyManagerFactory.getInstance(KeyManagerFactory.getDefaultAlgorithm());
+        keyManagerFactory.init(keyStore, keystorePass.toCharArray());
+
+        // load truststore
+        final String truststoreLocation = clientConfig.getTruststoreFilename();
+        final String truststorePass = clientConfig.getTruststorePass();
+        final String truststoreType = clientConfig.getTruststoreType().getType();
+        assertTruststorePropertiesSet(truststoreLocation, truststorePass, truststoreType);
+
+        KeyStore truststore = KeyStore.getInstance(truststoreType);
+        final TrustManagerFactory trustManagerFactory = TrustManagerFactory.getInstance("X509");
+        truststore.load(new FileInputStream(truststoreLocation), truststorePass.toCharArray());
+        trustManagerFactory.init(truststore);
+
+        final X509TrustManager x509TrustManager;
+        TrustManager[] trustManagers = trustManagerFactory.getTrustManagers();
+        if (trustManagers[0] != null) {
+            x509TrustManager = (X509TrustManager) trustManagers[0];
+        } else {
+            throw new IllegalStateException("List of trust managers is null");
+        }
+
+        SSLContext tempSslContext;
+        try {
+            tempSslContext = SSLContext.getInstance("TLS");
+        } catch (NoSuchAlgorithmException e) {
+            logger.warn("Unable to use 'TLS' for the PullHttpChangeIngestor due to NoSuchAlgorithmException. Will attempt to use the default algorithm.", e);
+            tempSslContext = SSLContext.getDefault();
+        }
+
+        final SSLContext sslContext = tempSslContext;
+        sslContext.init(keyManagerFactory.getKeyManagers(), trustManagerFactory.getTrustManagers(), null);
+
+        final SSLSocketFactory sslSocketFactory = sslContext.getSocketFactory();
+        okHttpClientBuilder.sslSocketFactory(sslSocketFactory, x509TrustManager);
+    }
+
+    private void assertKeystorePropertiesSet(String location, String password, String type) {
+        if (location == null || location.isEmpty()) {
+            throw new IllegalArgumentException(clientConfig.getKeystoreFilename() + " is null or is empty");
+        }
+
+        if (password == null || password.isEmpty()) {
+            throw new IllegalArgumentException("The client's keystore filename is set but its password is not (or is empty). If the location is set, the password must also be.");
+        }
+
+        if (type == null || type.isEmpty()) {
+            throw new IllegalArgumentException("The client's keystore filename is set but its type is not (or is empty). If the location is set, the type must also be.");
+        }
+    }
+
+    private void assertTruststorePropertiesSet(String location, String password, String type) {
+        if (location == null || location.isEmpty()) {
+            throw new IllegalArgumentException("The client's truststore filename is not set or is empty");
+        }
+
+        if (password == null || password.isEmpty()) {
+            throw new IllegalArgumentException("The client's truststore filename is set but its password is not (or is empty). If the location is set, the password must also be.");
+        }
+
+        if (type == null || type.isEmpty()) {
+            throw new IllegalArgumentException("The client's truststore filename is set but its type is not (or is empty). If the location is set, the type must also be.");
+        }
+    }
+
+    @Override
+    public ByteBuffer retrieveUpdateContent(String flowUpdateUrl) {
+        final Request.Builder requestBuilder = new Request.Builder()
+                .get()
+                .url(flowUpdateUrl);
+        final Request request = requestBuilder.build();
+
+        ResponseBody body;
+        try (final Response response = httpClientReference.get().newCall(request).execute()) {
+            logger.debug("Response received: {}", response);
+
+            int code = response.code();
+            if (code >= 400) {
+                throw new IOException("Got response code " + code + " while trying to pull configuration: " + response.body().string());
+            }
+
+            body = response.body();
+
+            if (body == null) {
+                logger.warn("No body returned when pulling a new configuration");
+                return null;
+            }
+
+            final ByteBuffer bodyByteBuffer = ByteBuffer.wrap(body.bytes());
+            return bodyByteBuffer;
+
+        } catch (Exception e) {
+            logger.warn("Hit an exception while trying to pull", e);
+            return null;
+        }
+    }
+
+    @Override
+    public void acknowledgeOperation(C2OperationAck operationAck) {
+        logger.info("Performing acknowledgement request to {} for operation {}", clientConfig.getC2AckUrl(), operationAck.getOperationId());
+        final ObjectMapper jacksonObjectMapper = new ObjectMapper();
+        jacksonObjectMapper.setSerializationInclusion(JsonInclude.Include.NON_NULL);
+        try {
+            final String operationAckBody = jacksonObjectMapper.writeValueAsString(operationAck);
+
+            final RequestBody requestBody = RequestBody.create(operationAckBody, MediaType.parse("application/json"));
+            final Request.Builder requestBuilder = new Request.Builder()
+                    .post(requestBody)
+                    .url(clientConfig.getC2AckUrl());
+            final Response heartbeatResponse = httpClientReference.get().newCall(requestBuilder.build()).execute();

Review Comment:
   The Response handling should be wrapped in a try-with-resources.



##########
minifi/minifi-nar-bundles/minifi-framework-bundle/minifi-server-nar/pom.xml:
##########
@@ -43,6 +43,12 @@
             <artifactId>nifi-headless-server</artifactId>
             <version>1.17.0-SNAPSHOT</version>
         </dependency>
+        <dependency>
+            <groupId>org.apache.commons</groupId>
+            <artifactId>commons-lang3</artifactId>
+            <version>${org.apache.commons.lang3.version}</version>

Review Comment:
   This version can be removed since it is defined in the root Maven configuration.



##########
c2/c2-client-bundle/c2-client-service/src/main/java/org/apache/nifi/c2/client/service/FlowIdHolder.java:
##########
@@ -0,0 +1,85 @@
+/*
+ * 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.nifi.c2.client.service;
+
+import static java.util.Collections.singletonList;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.List;
+import org.apache.nifi.util.file.FileUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class FlowIdHolder {
+    private static final Logger LOGGER = LoggerFactory.getLogger(FlowIdHolder.class);
+    private static final String FLOW_IDENTIFIER_FILENAME = "flow-identifier";
+
+    private volatile String flowId;
+    private final String configDirectoryName;
+
+    public FlowIdHolder(String configDirectoryName) {
+        this.configDirectoryName = configDirectoryName;
+        this.flowId = readFlowId();
+    }
+
+    public String getFlowId() {
+        return flowId;
+    }
+
+    public void setFlowId(String flowId) {
+        this.flowId = flowId;
+        persistFlowId(flowId);
+    }
+
+    private void persistFlowId(String flowId) {
+        File flowIdFile = new File(configDirectoryName, FLOW_IDENTIFIER_FILENAME);
+        try {
+            FileUtils.ensureDirectoryExistAndCanAccess(flowIdFile.getParentFile());
+            saveFlowId(flowIdFile, flowId);
+        } catch (IOException e) {
+            LOGGER.error("Failed to save flow information due to: {}", e.getMessage());

Review Comment:
   The entire exception should be logged for troubleshooting instead of just the message:
   ```suggestion
               LOGGER.error("Persisting Flow [{}] failed", flowId, e);
   ```



##########
c2/c2-client-bundle/c2-client-service/src/main/java/org/apache/nifi/c2/client/service/operation/UpdateConfigurationOperationHandler.java:
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.nifi.c2.client.service.operation;
+
+import static org.apache.commons.lang3.StringUtils.EMPTY;
+import static org.apache.nifi.c2.protocol.api.OperandType.CONFIGURATION;
+import static org.apache.nifi.c2.protocol.api.OperationType.UPDATE;
+
+import java.net.URI;
+import java.nio.ByteBuffer;
+import java.util.Optional;
+import java.util.function.Function;
+import org.apache.nifi.c2.client.api.C2Client;
+import org.apache.nifi.c2.client.service.FlowIdHolder;
+import org.apache.nifi.c2.protocol.api.C2Operation;
+import org.apache.nifi.c2.protocol.api.C2OperationAck;
+import org.apache.nifi.c2.protocol.api.C2OperationState;
+import org.apache.nifi.c2.protocol.api.OperandType;
+import org.apache.nifi.c2.protocol.api.OperationType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class UpdateConfigurationOperationHandler implements C2OperationHandler {
+
+    private static final Logger logger = LoggerFactory.getLogger(UpdateConfigurationOperationHandler.class);
+
+    private static final String LOCATION = "location";
+
+    private final C2Client client;
+    private final Function<ByteBuffer, Boolean> updateFlow;
+    private final FlowIdHolder flowIdHolder;
+
+    public UpdateConfigurationOperationHandler(C2Client client, FlowIdHolder flowIdHolder, Function<ByteBuffer, Boolean> updateFlow) {
+        this.client = client;
+        this.updateFlow = updateFlow;
+        this.flowIdHolder = flowIdHolder;
+    }
+
+    @Override
+    public OperationType getOperationType() {
+        return UPDATE;
+    }
+
+    @Override
+    public OperandType getOperandType() {
+        return CONFIGURATION;
+    }
+
+    @Override
+    public C2OperationAck handle(C2Operation operation) {
+        String opIdentifier = Optional.ofNullable(operation.getIdentifier())
+            .orElse(EMPTY);
+        C2OperationAck operationAck = new C2OperationAck();
+        C2OperationState state = new C2OperationState();
+        operationAck.setOperationState(state);
+        operationAck.setOperationId(opIdentifier);
+
+        String updateLocation = Optional.ofNullable(operation.getArgs())
+            .map(map -> map.get(LOCATION))
+            .orElse(EMPTY);
+
+        String newFlowId = parseFlowId(updateLocation);
+        if (flowIdHolder.getFlowId() == null || !flowIdHolder.getFlowId().equals(newFlowId)) {
+            logger.info("Will perform flow update from {} for operation #{}. Previous flow id was {}, replacing with new id {}", updateLocation, opIdentifier,
+                flowIdHolder.getFlowId() == null ? "not set" : flowIdHolder.getFlowId(), newFlowId);
+        } else {
+            logger.info("Flow is current, no update is necessary...");

Review Comment:
   Should this be a debug instead of an info message?



##########
nifi-server-api/pom.xml:
##########
@@ -42,5 +42,10 @@
             <version>1.17.0-SNAPSHOT</version>
             <scope>provided</scope>
         </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>c2-client-api</artifactId>
+            <version>1.17.0-SNAPSHOT</version>
+        </dependency>

Review Comment:
   Is this dependency necessary?



##########
c2/c2-client-bundle/c2-client-service/src/main/java/org/apache/nifi/c2/client/service/C2HeartbeatFactory.java:
##########
@@ -0,0 +1,248 @@
+/*
+ * 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.nifi.c2.client.service;
+
+import static org.apache.commons.lang3.StringUtils.isNotBlank;
+
+import java.io.File;
+import java.lang.management.ManagementFactory;
+import java.lang.management.OperatingSystemMXBean;
+import java.net.InetAddress;
+import java.net.NetworkInterface;
+import java.util.Enumeration;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.nifi.c2.client.C2ClientConfig;
+import org.apache.nifi.c2.client.PersistentUuidGenerator;
+import org.apache.nifi.c2.client.service.model.RuntimeInfoWrapper;
+import org.apache.nifi.c2.protocol.api.AgentInfo;
+import org.apache.nifi.c2.protocol.api.AgentRepositories;
+import org.apache.nifi.c2.protocol.api.AgentStatus;
+import org.apache.nifi.c2.protocol.api.C2Heartbeat;
+import org.apache.nifi.c2.protocol.api.DeviceInfo;
+import org.apache.nifi.c2.protocol.api.FlowInfo;
+import org.apache.nifi.c2.protocol.api.FlowQueueStatus;
+import org.apache.nifi.c2.protocol.api.NetworkInfo;
+import org.apache.nifi.c2.protocol.api.SystemInfo;
+import org.apache.nifi.c2.protocol.component.api.RuntimeManifest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class C2HeartbeatFactory {
+
+    private static final Logger logger = LoggerFactory.getLogger(C2HeartbeatFactory.class);
+
+    private static final String AGENT_IDENTIFIER_FILENAME = "agent-identifier";
+    private static final String DEVICE_IDENTIFIER_FILENAME = "device-identifier";
+
+    private final C2ClientConfig clientConfig;
+    private final FlowIdHolder flowIdHolder;
+
+    private String agentId;
+    private String deviceId;
+    private File confDirectory;
+
+    public C2HeartbeatFactory(C2ClientConfig clientConfig, FlowIdHolder flowIdHolder) {
+        this.clientConfig = clientConfig;
+        this.flowIdHolder = flowIdHolder;
+    }
+
+    public C2Heartbeat create(RuntimeInfoWrapper runtimeInfoWrapper) {
+        C2Heartbeat heartbeat = new C2Heartbeat();
+
+        heartbeat.setAgentInfo(getAgentInfo(runtimeInfoWrapper.getAgentRepositories(), runtimeInfoWrapper.getManifest()));
+        heartbeat.setDeviceInfo(generateDeviceInfo());
+        heartbeat.setFlowInfo(getFlowInfo(runtimeInfoWrapper.getQueueStatus()));
+        heartbeat.setCreated(System.currentTimeMillis());
+
+        return heartbeat;
+    }
+
+    //////
+    // FlowInfo
+    //////

Review Comment:
   These comments do not follow standard JavaDoc conventions, recommend removing them, and including method-level comments if necessary.



##########
nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/c2/C2NiFiProperties.java:
##########
@@ -0,0 +1,62 @@
+/*
+ * 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.nifi.c2;
+
+import java.util.concurrent.TimeUnit;
+
+public class C2NiFiProperties {

Review Comment:
   Is there a reason for this class in addition to the `C2Properties` class in `c2-client-api`, or can this be removed?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] bejancsaba commented on a diff in pull request #6075: NIFI-9666 Implement base C2 client functionality

Posted by GitBox <gi...@apache.org>.
bejancsaba commented on code in PR #6075:
URL: https://github.com/apache/nifi/pull/6075#discussion_r899966573


##########
c2/c2-client-bundle/c2-client-base/src/main/java/org/apache/nifi/c2/client/C2ClientConfig.java:
##########
@@ -0,0 +1,269 @@
+/*
+ * 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.nifi.c2.client;
+
+import javax.net.ssl.HostnameVerifier;
+
+/**
+ * Configuration for a C2 Client.
+ */
+public class C2ClientConfig {
+
+    private final String c2Url;
+    private final String c2AckUrl;
+    private final String agentClass;
+    private final String agentIdentifier;
+    private final String confDirectory;
+    private final String runtimeManifestIdentifier;
+    private final String runtimeType;
+    private final long heartbeatPeriod;
+    private final String keystoreFilename;
+    private final String keystorePass;
+    private final String keyPass;
+    private final String keystoreType;
+    private final String truststoreFilename;
+    private final String truststorePass;
+    private final String truststoreType;
+    private final HostnameVerifier hostnameVerifier;

Review Comment:
   Removed it was not used, thanks for raising.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #6075: NIFI-9666 Implement base C2 client functionality

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on code in PR #6075:
URL: https://github.com/apache/nifi/pull/6075#discussion_r890256810


##########
c2/c2-client-bundle/c2-client-service/src/main/java/org/apache/nifi/c2/client/service/operation/UpdateConfigurationOperationHandler.java:
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.nifi.c2.client.service.operation;
+
+import static org.apache.commons.lang3.StringUtils.EMPTY;
+import static org.apache.nifi.c2.protocol.api.OperandType.CONFIGURATION;
+import static org.apache.nifi.c2.protocol.api.OperationType.UPDATE;
+
+import java.net.URI;
+import java.nio.ByteBuffer;
+import java.util.Optional;
+import java.util.function.Function;
+import org.apache.nifi.c2.client.api.C2Client;
+import org.apache.nifi.c2.client.service.FlowIdHolder;
+import org.apache.nifi.c2.protocol.api.C2Operation;
+import org.apache.nifi.c2.protocol.api.C2OperationAck;
+import org.apache.nifi.c2.protocol.api.C2OperationState;
+import org.apache.nifi.c2.protocol.api.OperandType;
+import org.apache.nifi.c2.protocol.api.OperationType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class UpdateConfigurationOperationHandler implements C2OperationHandler {
+
+    private static final Logger logger = LoggerFactory.getLogger(UpdateConfigurationOperationHandler.class);
+
+    private static final String LOCATION = "location";
+
+    private final C2Client client;
+    private final Function<ByteBuffer, Boolean> updateFlow;
+    private final FlowIdHolder flowIdHolder;
+
+    public UpdateConfigurationOperationHandler(C2Client client, FlowIdHolder flowIdHolder, Function<ByteBuffer, Boolean> updateFlow) {
+        this.client = client;
+        this.updateFlow = updateFlow;
+        this.flowIdHolder = flowIdHolder;
+    }
+
+    @Override
+    public OperationType getOperationType() {
+        return UPDATE;
+    }
+
+    @Override
+    public OperandType getOperandType() {
+        return CONFIGURATION;
+    }
+
+    @Override
+    public C2OperationAck handle(C2Operation operation) {
+        String opIdentifier = Optional.ofNullable(operation.getIdentifier())
+            .orElse(EMPTY);
+        C2OperationAck operationAck = new C2OperationAck();
+        C2OperationState state = new C2OperationState();
+        operationAck.setOperationState(state);
+        operationAck.setOperationId(opIdentifier);
+
+        String updateLocation = Optional.ofNullable(operation.getArgs())
+            .map(map -> map.get(LOCATION))
+            .orElse(EMPTY);
+
+        String newFlowId = parseFlowId(updateLocation);
+        if (flowIdHolder.getFlowId() == null || !flowIdHolder.getFlowId().equals(newFlowId)) {
+            logger.info("Will perform flow update from {} for operation #{}. Previous flow id was {}, replacing with new id {}", updateLocation, opIdentifier,
+                flowIdHolder.getFlowId() == null ? "not set" : flowIdHolder.getFlowId(), newFlowId);
+        } else {
+            logger.info("Flow is current, no update is necessary...");

Review Comment:
   Thanks for the additional background, if it is infrequent, leaving it as an info message sounds good.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #6075: NIFI-9666 Implement base C2 client functionality

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on code in PR #6075:
URL: https://github.com/apache/nifi/pull/6075#discussion_r890262671


##########
c2/c2-client-bundle/c2-client-api/src/main/java/org/apache/nifi/c2/client/api/C2Client.java:
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.nifi.c2.client.api;
+
+import java.util.Optional;
+import org.apache.nifi.c2.protocol.api.C2Heartbeat;
+import org.apache.nifi.c2.protocol.api.C2HeartbeatResponse;
+import org.apache.nifi.c2.protocol.api.C2OperationAck;
+
+/**
+ * Defines interface methods used to implement a C2 Client. The controller can be application-specific but is used for such tasks as updating the flow.
+ */
+public interface C2Client {
+
+    /**
+     * Responsible for sending the C2Heartbeat to the C2 Server
+     *
+     * @param heartbeat the heartbeat to be sent
+     * @return optional response from the C2 Server if the response arrived it will be populated
+     */
+    Optional<C2HeartbeatResponse> publishHeartbeat(C2Heartbeat heartbeat);
+
+    /**
+     * Retrive the content of the new flow from the C2 Server
+     *
+     * @param flowUpdateUrl url where the content should be downloaded from
+     * @return the actual downloaded content. Will be null if no content can be downloaded
+     */
+    byte[] retrieveUpdateContent(String flowUpdateUrl);

Review Comment:
   Thanks for adjusting the return object. If the response can be `null`, recommend changing the interface to return `Optional<byte[]>` to clarify the interface contract.



##########
c2/c2-client-bundle/c2-client-http/src/main/java/org/apache/nifi/c2/client/http/C2HttpClient.java:
##########
@@ -0,0 +1,250 @@
+/*
+ * 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.nifi.c2.client.http;
+
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.security.KeyStore;
+import java.security.NoSuchAlgorithmException;
+import java.util.Optional;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLSocketFactory;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+import okhttp3.MediaType;
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.RequestBody;
+import okhttp3.Response;
+import okhttp3.ResponseBody;
+import okhttp3.logging.HttpLoggingInterceptor;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.c2.client.C2ClientConfig;
+import org.apache.nifi.c2.client.api.C2Client;
+import org.apache.nifi.c2.client.api.C2Serializer;
+import org.apache.nifi.c2.protocol.api.C2Heartbeat;
+import org.apache.nifi.c2.protocol.api.C2HeartbeatResponse;
+import org.apache.nifi.c2.protocol.api.C2OperationAck;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class C2HttpClient implements C2Client {
+
+    private static final Logger logger = LoggerFactory.getLogger(C2HttpClient.class);
+    private static final MediaType MEDIA_TYPE_APPLICATION_JSON = MediaType.parse("application/json");
+
+    private final AtomicReference<OkHttpClient> httpClientReference = new AtomicReference<>();
+    private final C2ClientConfig clientConfig;
+    private final C2Serializer serializer;
+
+    public C2HttpClient(C2ClientConfig clientConfig, C2Serializer serializer) {
+        super();
+        this.clientConfig = clientConfig;
+        this.serializer = serializer;
+        final OkHttpClient.Builder okHttpClientBuilder = new OkHttpClient.Builder();
+
+        // Configure request and response logging
+        HttpLoggingInterceptor logging = new HttpLoggingInterceptor();
+        logging.setLevel(HttpLoggingInterceptor.Level.BASIC);
+        okHttpClientBuilder.addInterceptor(logging);
+
+        // Set whether to follow redirects
+        okHttpClientBuilder.followRedirects(true);
+
+        // Timeout for calls made to the server
+        okHttpClientBuilder.callTimeout(clientConfig.getCallTimeout(), TimeUnit.MILLISECONDS);
+
+        // check if the ssl path is set and add the factory if so
+        if (StringUtils.isNotBlank(clientConfig.getKeystoreFilename())) {
+            try {
+                setSslSocketFactory(okHttpClientBuilder);
+            } catch (Exception e) {
+                throw new IllegalStateException("OkHttp TLS configuration failed", e);
+            }
+        }
+
+        httpClientReference.set(okHttpClientBuilder.build());
+    }
+
+    @Override
+    public Optional<C2HeartbeatResponse> publishHeartbeat(C2Heartbeat heartbeat) {
+        return serializer.serialize(heartbeat).flatMap(this::sendHeartbeat);
+    }
+
+    private Optional<C2HeartbeatResponse> sendHeartbeat(String heartbeat) {
+        logger.debug("Sending heartbeat to {}", clientConfig.getC2Url());
+
+        Optional<C2HeartbeatResponse> c2HeartbeatResponse = Optional.empty();
+        Request request = new Request.Builder()
+            .post(RequestBody.create(heartbeat, MEDIA_TYPE_APPLICATION_JSON))
+            .url(clientConfig.getC2Url())
+            .build();
+
+        try (Response heartbeatResponse = httpClientReference.get().newCall(request).execute()) {
+            c2HeartbeatResponse = getResponseBody(heartbeatResponse).flatMap(response -> serializer.deserialize(response, C2HeartbeatResponse.class));
+        } catch (IOException ce) {
+            logger.error("Send Heartbeat failed [{}]", clientConfig.getC2Url(), ce);
+        }
+
+        return c2HeartbeatResponse;
+    }
+
+    private Optional<String> getResponseBody(Response response) {
+        String responseBody = null;
+
+        try {
+            responseBody = response.body().string();
+            logger.debug("Received response body {}", responseBody);
+        } catch (IOException e) {
+            logger.error("HTTP Request failed", e);
+        }
+
+        return Optional.ofNullable(responseBody);
+    }
+
+    private void setSslSocketFactory(OkHttpClient.Builder okHttpClientBuilder) throws Exception {
+        final String keystoreLocation = clientConfig.getKeystoreFilename();
+        final String keystoreType = clientConfig.getKeystoreType();
+        final String keystorePass = clientConfig.getKeystorePass();
+
+        assertKeystorePropertiesSet(keystoreLocation, keystorePass, keystoreType);
+
+        // prepare the keystore
+        final KeyStore keyStore = KeyStore.getInstance(keystoreType);
+
+        try (FileInputStream keyStoreStream = new FileInputStream(keystoreLocation)) {
+            keyStore.load(keyStoreStream, keystorePass.toCharArray());
+        }
+
+        final KeyManagerFactory keyManagerFactory = KeyManagerFactory.getInstance(KeyManagerFactory.getDefaultAlgorithm());
+        keyManagerFactory.init(keyStore, keystorePass.toCharArray());
+
+        // load truststore
+        final String truststoreLocation = clientConfig.getTruststoreFilename();
+        final String truststorePass = clientConfig.getTruststorePass();
+        final String truststoreType = clientConfig.getTruststoreType();
+        assertTruststorePropertiesSet(truststoreLocation, truststorePass, truststoreType);
+
+        KeyStore truststore = KeyStore.getInstance(truststoreType);
+        final TrustManagerFactory trustManagerFactory = TrustManagerFactory.getInstance("X509");
+        truststore.load(new FileInputStream(truststoreLocation), truststorePass.toCharArray());
+        trustManagerFactory.init(truststore);
+
+        final X509TrustManager x509TrustManager;
+        TrustManager[] trustManagers = trustManagerFactory.getTrustManagers();
+        if (trustManagers[0] != null) {
+            x509TrustManager = (X509TrustManager) trustManagers[0];
+        } else {
+            throw new IllegalStateException("List of trust managers is null");
+        }
+
+        SSLContext tempSslContext;
+        try {
+            tempSslContext = SSLContext.getInstance("TLS");
+        } catch (NoSuchAlgorithmException e) {
+            throw new IllegalStateException("SSLContext creation failed", e);
+        }
+
+        final SSLContext sslContext = tempSslContext;
+        sslContext.init(keyManagerFactory.getKeyManagers(), trustManagerFactory.getTrustManagers(), null);
+
+        final SSLSocketFactory sslSocketFactory = sslContext.getSocketFactory();
+        okHttpClientBuilder.sslSocketFactory(sslSocketFactory, x509TrustManager);
+    }
+
+    private void assertKeystorePropertiesSet(String location, String password, String type) {
+        if (location == null || location.isEmpty()) {
+            throw new IllegalArgumentException(clientConfig.getKeystoreFilename() + " is null or is empty");
+        }
+
+        if (password == null || password.isEmpty()) {
+            throw new IllegalArgumentException("The client's keystore filename is set but its password is not (or is empty). If the location is set, the password must also be.");
+        }
+
+        if (type == null || type.isEmpty()) {
+            throw new IllegalArgumentException("The client's keystore filename is set but its type is not (or is empty). If the location is set, the type must also be.");
+        }
+    }
+
+    private void assertTruststorePropertiesSet(String location, String password, String type) {
+        if (location == null || location.isEmpty()) {
+            throw new IllegalArgumentException("The client's truststore filename is not set or is empty");
+        }
+
+        if (password == null || password.isEmpty()) {
+            throw new IllegalArgumentException("The client's truststore filename is set but its password is not (or is empty). If the location is set, the password must also be.");
+        }
+
+        if (type == null || type.isEmpty()) {
+            throw new IllegalArgumentException("The client's truststore filename is set but its type is not (or is empty). If the location is set, the type must also be.");
+        }
+    }
+
+    @Override
+    public byte[] retrieveUpdateContent(String flowUpdateUrl) {
+        final Request.Builder requestBuilder = new Request.Builder()
+                .get()
+                .url(flowUpdateUrl);
+        final Request request = requestBuilder.build();
+
+        ResponseBody body;
+        try (final Response response = httpClientReference.get().newCall(request).execute()) {
+            logger.debug("Response received: {}", response);

Review Comment:
   Is this log still necessary with the addition of the logging interceptor?



##########
c2/c2-client-bundle/c2-client-service/src/main/java/org/apache/nifi/c2/client/service/operation/UpdateConfigurationOperationHandler.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.nifi.c2.client.service.operation;
+
+import static org.apache.commons.lang3.StringUtils.EMPTY;
+import static org.apache.nifi.c2.protocol.api.OperandType.CONFIGURATION;
+import static org.apache.nifi.c2.protocol.api.OperationType.UPDATE;
+
+import java.net.URI;
+import java.util.Optional;
+import java.util.function.Function;
+import org.apache.nifi.c2.client.api.C2Client;
+import org.apache.nifi.c2.client.service.FlowIdHolder;
+import org.apache.nifi.c2.protocol.api.C2Operation;
+import org.apache.nifi.c2.protocol.api.C2OperationAck;
+import org.apache.nifi.c2.protocol.api.C2OperationState;
+import org.apache.nifi.c2.protocol.api.OperandType;
+import org.apache.nifi.c2.protocol.api.OperationType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class UpdateConfigurationOperationHandler implements C2OperationHandler {
+
+    private static final Logger logger = LoggerFactory.getLogger(UpdateConfigurationOperationHandler.class);
+
+    private static final String LOCATION = "location";
+
+    private final C2Client client;
+    private final Function<byte[], Boolean> updateFlow;
+    private final FlowIdHolder flowIdHolder;
+
+    public UpdateConfigurationOperationHandler(C2Client client, FlowIdHolder flowIdHolder, Function<byte[], Boolean> updateFlow) {
+        this.client = client;
+        this.updateFlow = updateFlow;
+        this.flowIdHolder = flowIdHolder;
+    }
+
+    @Override
+    public OperationType getOperationType() {
+        return UPDATE;
+    }
+
+    @Override
+    public OperandType getOperandType() {
+        return CONFIGURATION;
+    }
+
+    @Override
+    public C2OperationAck handle(C2Operation operation) {
+        String opIdentifier = Optional.ofNullable(operation.getIdentifier())
+            .orElse(EMPTY);
+        C2OperationAck operationAck = new C2OperationAck();
+        C2OperationState state = new C2OperationState();
+        operationAck.setOperationState(state);
+        operationAck.setOperationId(opIdentifier);
+
+        String updateLocation = Optional.ofNullable(operation.getArgs())
+            .map(map -> map.get(LOCATION))
+            .orElse(EMPTY);
+
+        String newFlowId = parseFlowId(updateLocation);
+        if (flowIdHolder.getFlowId() == null || !flowIdHolder.getFlowId().equals(newFlowId)) {
+            logger.info("Will perform flow update from {} for operation #{}. Previous flow id was {}, replacing with new id {}", updateLocation, opIdentifier,
+                flowIdHolder.getFlowId() == null ? "not set" : flowIdHolder.getFlowId(), newFlowId);
+        } else {
+            logger.info("Flow is current, no update is necessary...");
+        }
+
+        flowIdHolder.setFlowId(newFlowId);
+        byte[] updateContent = client.retrieveUpdateContent(updateLocation);
+        if (updateContent != null) {
+            if (updateFlow.apply(updateContent)) {
+                state.setState(C2OperationState.OperationState.FULLY_APPLIED);
+                logger.debug("Update configuration applied for operation #{}.", opIdentifier);
+            } else {
+                state.setState(C2OperationState.OperationState.NOT_APPLIED);
+                logger.error("Update resulted in error for operation #{}.", opIdentifier);
+            }
+        } else {
+            state.setState(C2OperationState.OperationState.NOT_APPLIED);
+            logger.error("Update content retrieval resulted in empty content so flow update was omitted for operation #{}.", opIdentifier);
+        }
+
+        return operationAck;
+    }
+
+    private String parseFlowId(String flowUpdateUrl) {
+        try {
+            URI flowUri = new URI(flowUpdateUrl);
+            String flowUriPath = flowUri.getPath();
+            String[] split = flowUriPath.split("/");
+            if (split.length > 4) {
+                return split[4];
+            } else {
+                throw new Exception();

Review Comment:
   Recommend adding a message:
   ```suggestion
                   throw new IllegalArgumentException(String.format("Flow Update URL format unexpected [%s]", flowUpdateUrl));
   ```



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/MiNiFiStatus.java:
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.nifi.minifi.bootstrap;
+
+public class MiNiFiStatus {
+
+    private final Integer port;

Review Comment:
   Should `port` be nullable, or can a primitive `int` be used?



##########
c2/c2-client-bundle/pom.xml:
##########
@@ -0,0 +1,41 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+ -->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <artifactId>c2</artifactId>
+        <groupId>org.apache.nifi</groupId>
+        <version>1.17.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>c2-client-bundle</artifactId>
+    <packaging>pom</packaging>
+
+    <modules>
+        <module>c2-client-api</module>
+        <module>c2-client-base</module>
+        <module>c2-client-http</module>
+        <module>c2-client-service</module>
+    </modules>
+
+    <dependencyManagement>
+        <dependencies>
+        </dependencies>
+    </dependencyManagement>

Review Comment:
   This empty block can be removed.



##########
c2/c2-client-bundle/c2-client-base/src/main/java/org/apache/nifi/c2/client/C2ClientConfig.java:
##########
@@ -0,0 +1,269 @@
+/*
+ * 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.nifi.c2.client;
+
+import javax.net.ssl.HostnameVerifier;
+
+/**
+ * Configuration for a C2 Client.
+ */
+public class C2ClientConfig {
+
+    private final String c2Url;
+    private final String c2AckUrl;
+    private final String agentClass;
+    private final String agentIdentifier;
+    private final String confDirectory;
+    private final String runtimeManifestIdentifier;
+    private final String runtimeType;
+    private final Long heartbeatPeriod;
+    private final Long callTimeout;
+    private final String keystoreFilename;
+    private final String keystorePass;
+    private final String keyPass;
+    private final String keystoreType;
+    private final String truststoreFilename;
+    private final String truststorePass;
+    private final String truststoreType;
+    private final HostnameVerifier hostnameVerifier;
+    private final Integer readTimeout;
+    private final Integer connectTimeout;

Review Comment:
   Is there a reason for making these values nullable using the Integer type, as opposed to the primitive int type?



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/MiNiFiStatus.java:
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.nifi.minifi.bootstrap;
+
+public class MiNiFiStatus {
+
+    private final Integer port;
+    private final String pid;

Review Comment:
   Is there a reason for using `String` as opposed to `int` for the PID?



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/WindowsService.java:
##########
@@ -17,26 +17,20 @@
 package org.apache.nifi.minifi.bootstrap;
 
 import java.io.IOException;
-import java.io.File;
+import org.apache.nifi.minifi.bootstrap.service.BootstrapFileProvider;
 
 public class WindowsService {

Review Comment:
   Is the name of this class still accurate? It does not seem to be specific to Windows.



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/command/CommandRunner.java:
##########
@@ -0,0 +1,28 @@
+/*
+ * 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.nifi.minifi.bootstrap.command;
+
+public interface CommandRunner {
+
+    /**
+     * Executes a command.
+     * @param args the input arguments
+     * @return status code
+     */
+    int runCommand(String[] args);

Review Comment:
   Would it make more sense to change this to varargs `String...` to avoid having to pass empty String arrays for commands that do not take any arguments?



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/command/EnvRunner.java:
##########
@@ -0,0 +1,107 @@
+/*
+ * 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.nifi.minifi.bootstrap.command;
+
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CMD_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.DEFAULT_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.Status.ERROR;
+import static org.apache.nifi.minifi.bootstrap.Status.MINIFI_NOT_RUNNING;
+import static org.apache.nifi.minifi.bootstrap.Status.OK;
+
+import java.lang.reflect.Method;
+import java.util.Map.Entry;
+import java.util.Properties;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.apache.nifi.minifi.bootstrap.MiNiFiStatus;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiStatusProvider;
+
+public class EnvRunner implements CommandRunner {
+    private final MiNiFiParameters miNiFiParameters;
+    private final MiNiFiStatusProvider miNiFiStatusProvider;
+
+    public EnvRunner(MiNiFiParameters miNiFiParameters, MiNiFiStatusProvider miNiFiStatusProvider) {
+        this.miNiFiParameters = miNiFiParameters;
+        this.miNiFiStatusProvider = miNiFiStatusProvider;
+    }
+
+    /**
+     * Returns information about the MiNiFi's virtual machine.
+     * @param args the input arguments
+     * @return status code
+     */
+    @Override
+    public int runCommand(String[] args) {
+        return env();
+    }
+
+    private int env() {
+        MiNiFiStatus status = miNiFiStatusProvider.getStatus(miNiFiParameters.getMiNiFiPort(), miNiFiParameters.getMinifiPid());
+        if (status.getPid() == null) {
+            CMD_LOGGER.error("Apache MiNiFi is not running");
+            return MINIFI_NOT_RUNNING.getStatusCode();
+        }
+
+        Class<?> virtualMachineClass;
+        try {
+            virtualMachineClass = Class.forName("com.sun.tools.attach.VirtualMachine");
+        } catch (ClassNotFoundException cnfe) {
+            CMD_LOGGER.error("Seems tools.jar (Linux / Windows JDK) or classes.jar (Mac OS) is not available in classpath");
+            return ERROR.getStatusCode();
+        }
+
+        Method attachMethod;
+        Method detachMethod;
+        try {
+            attachMethod = virtualMachineClass.getMethod("attach", String.class);
+            detachMethod = virtualMachineClass.getDeclaredMethod("detach");
+        } catch (Exception e) {
+            CMD_LOGGER.error("Methods required for getting environment not available");
+            DEFAULT_LOGGER.error("Exception:", e);
+            return ERROR.getStatusCode();
+        }
+
+        Object virtualMachine;
+        try {
+            virtualMachine = attachMethod.invoke(null, status.getPid());
+        } catch (Exception e) {
+            CMD_LOGGER.error("Problem attaching to MiNiFi");
+            DEFAULT_LOGGER.error("Exception:", e);

Review Comment:
   ```suggestion
               DEFAULT_LOGGER.error("Virtual Machine attachment failed", e);
   ```



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/command/EnvRunner.java:
##########
@@ -0,0 +1,107 @@
+/*
+ * 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.nifi.minifi.bootstrap.command;
+
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CMD_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.DEFAULT_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.Status.ERROR;
+import static org.apache.nifi.minifi.bootstrap.Status.MINIFI_NOT_RUNNING;
+import static org.apache.nifi.minifi.bootstrap.Status.OK;
+
+import java.lang.reflect.Method;
+import java.util.Map.Entry;
+import java.util.Properties;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.apache.nifi.minifi.bootstrap.MiNiFiStatus;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiStatusProvider;
+
+public class EnvRunner implements CommandRunner {
+    private final MiNiFiParameters miNiFiParameters;
+    private final MiNiFiStatusProvider miNiFiStatusProvider;
+
+    public EnvRunner(MiNiFiParameters miNiFiParameters, MiNiFiStatusProvider miNiFiStatusProvider) {
+        this.miNiFiParameters = miNiFiParameters;
+        this.miNiFiStatusProvider = miNiFiStatusProvider;
+    }
+
+    /**
+     * Returns information about the MiNiFi's virtual machine.
+     * @param args the input arguments
+     * @return status code
+     */
+    @Override
+    public int runCommand(String[] args) {
+        return env();
+    }
+
+    private int env() {
+        MiNiFiStatus status = miNiFiStatusProvider.getStatus(miNiFiParameters.getMiNiFiPort(), miNiFiParameters.getMinifiPid());
+        if (status.getPid() == null) {
+            CMD_LOGGER.error("Apache MiNiFi is not running");
+            return MINIFI_NOT_RUNNING.getStatusCode();
+        }
+
+        Class<?> virtualMachineClass;
+        try {
+            virtualMachineClass = Class.forName("com.sun.tools.attach.VirtualMachine");
+        } catch (ClassNotFoundException cnfe) {
+            CMD_LOGGER.error("Seems tools.jar (Linux / Windows JDK) or classes.jar (Mac OS) is not available in classpath");
+            return ERROR.getStatusCode();
+        }
+
+        Method attachMethod;
+        Method detachMethod;
+        try {
+            attachMethod = virtualMachineClass.getMethod("attach", String.class);
+            detachMethod = virtualMachineClass.getDeclaredMethod("detach");
+        } catch (Exception e) {
+            CMD_LOGGER.error("Methods required for getting environment not available");
+            DEFAULT_LOGGER.error("Exception:", e);

Review Comment:
   ```suggestion
               DEFAULT_LOGGER.error("Virtual Machine method lookup failed", e);
   ```



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/RunMiNiFi.java:
##########
@@ -200,1289 +150,38 @@ private static void printUsage() {
         System.out.println();
     }
 
-    public static void main(String[] args) throws IOException, InterruptedException {
-        if (args.length < 1 || args.length > 3) {
-            printUsage();
-            return;
-        }
-
-        File dumpFile = null;
-
-        final String cmd = args[0];
-        if (cmd.equals("dump")) {
-            if (args.length > 1) {
-                dumpFile = new File(args[1]);
-            } else {
-                dumpFile = null;
-            }
-        }
-
-        switch (cmd.toLowerCase()) {
-            case "start":
-            case "run":
-            case "stop":
-            case "status":
-            case "dump":
-            case "restart":
-            case "env":
-            case "flowstatus":
-                break;
-            default:
-                printUsage();
-                return;
-        }
-
-        final File configFile = getBootstrapConfFile();
-        final RunMiNiFi runMiNiFi = new RunMiNiFi(configFile);
-
-        Integer exitStatus = null;
-        switch (cmd.toLowerCase()) {
-            case "start":
-                runMiNiFi.start();
-                break;
-            case "run":
-                runMiNiFi.start();
-                break;
-            case "stop":
-                runMiNiFi.stop();
-                break;
-            case "status":
-                exitStatus = runMiNiFi.status();
-                break;
-            case "restart":
-                runMiNiFi.stop();
-                runMiNiFi.start();
-                break;
-            case "dump":
-                runMiNiFi.dump(dumpFile);
-                break;
-            case "env":
-                runMiNiFi.env();
-                break;
-            case "flowstatus":
-                if(args.length == 2) {
-                    System.out.println(runMiNiFi.statusReport(args[1]));
-                } else {
-                    System.out.println("The 'flowStatus' command requires an input query. See the System Admin Guide 'FlowStatus Script Query' section for complete details.");
-                }
-                break;
-        }
-        if (exitStatus != null) {
-            System.exit(exitStatus);
-        }
-    }
-
-    public static File getBootstrapConfFile() {
-        String configFilename = System.getProperty("org.apache.nifi.minifi.bootstrap.config.file");
-
-        if (configFilename == null) {
-            final String nifiHome = System.getenv("MINIFI_HOME");
-            if (nifiHome != null) {
-                final File nifiHomeFile = new File(nifiHome.trim());
-                final File configFile = new File(nifiHomeFile, DEFAULT_CONFIG_FILE);
-                configFilename = configFile.getAbsolutePath();
-            }
-        }
-
-        if (configFilename == null) {
-            configFilename = DEFAULT_CONFIG_FILE;
-        }
-
-        final File configFile = new File(configFilename);
-        return configFile;
-    }
-
-    private File getBootstrapFile(final Logger logger, String directory, String defaultDirectory, String fileName) throws IOException {
-
-        final File confDir = bootstrapConfigFile.getParentFile();
-        final File nifiHome = confDir.getParentFile();
-
-        String confFileDir = System.getProperty(directory);
-
-        final File fileDir;
-
-        if (confFileDir != null) {
-            fileDir = new File(confFileDir.trim());
-        } else {
-            fileDir = new File(nifiHome, defaultDirectory);
-        }
-
-        FileUtils.ensureDirectoryExistAndCanAccess(fileDir);
-        final File statusFile = new File(fileDir, fileName);
-        logger.debug("Status File: {}", statusFile);
-        return statusFile;
-    }
-
-    File getPidFile(final Logger logger) throws IOException {
-        return getBootstrapFile(logger, MINIFI_PID_DIR_PROP, DEFAULT_PID_DIR, MINIFI_PID_FILE_NAME);
-    }
-
-    File getStatusFile(final Logger logger) throws IOException {
-        return getBootstrapFile(logger, MINIFI_PID_DIR_PROP, DEFAULT_PID_DIR, MINIFI_STATUS_FILE_NAME);
-    }
-
-    File getLockFile(final Logger logger) throws IOException {
-        return getBootstrapFile(logger, MINIFI_PID_DIR_PROP, DEFAULT_PID_DIR, MINIFI_LOCK_FILE_NAME);
-    }
-
-    File getStatusFile() throws IOException{
-        return getStatusFile(defaultLogger);
-    }
-
-    public File getReloadFile(final Logger logger) {
-        final File confDir = bootstrapConfigFile.getParentFile();
-        final File nifiHome = confDir.getParentFile();
-        final File bin = new File(nifiHome, "bin");
-        final File reloadFile = new File(bin, "minifi.reload.lock");
-
-        logger.debug("Reload File: {}", reloadFile);
-        return reloadFile;
-    }
-
-    public File getSwapFile(final Logger logger) {
-        final File confDir = bootstrapConfigFile.getParentFile();
-        final File swapFile = new File(confDir, "swap.yml");
-
-        logger.debug("Swap File: {}", swapFile);
-        return swapFile;
-    }
-
-
-    private Properties loadProperties(final Logger logger) throws IOException {
-        final Properties props = new Properties();
-        final File statusFile = getStatusFile(logger);
-        if (statusFile == null || !statusFile.exists()) {
-            logger.debug("No status file to load properties from");
-            return props;
-        }
-
-        try (final FileInputStream fis = new FileInputStream(getStatusFile(logger))) {
-            props.load(fis);
-        }
-
-        final Map<Object, Object> modified = new HashMap<>(props);
-        modified.remove("secret.key");
-        logger.debug("Properties: {}", modified);
-
-        return props;
-    }
-
-    private synchronized void saveProperties(final Properties minifiProps, final Logger logger) throws IOException {
-        final String pid = minifiProps.getProperty(PID_KEY);
-        if (!StringUtils.isBlank(pid)) {
-            writePidFile(pid, logger);
-        }
-
-        final File statusFile = getStatusFile(logger);
-        if (statusFile.exists() && !statusFile.delete()) {
-            logger.warn("Failed to delete {}", statusFile);
-        }
-
-        if (!statusFile.createNewFile()) {
-            throw new IOException("Failed to create file " + statusFile);
-        }
-
-        try {
-            final Set<PosixFilePermission> perms = new HashSet<>();
-            perms.add(PosixFilePermission.OWNER_WRITE);
-            perms.add(PosixFilePermission.OWNER_READ);
-            perms.add(PosixFilePermission.GROUP_READ);
-            perms.add(PosixFilePermission.OTHERS_READ);
-            Files.setPosixFilePermissions(statusFile.toPath(), perms);
-        } catch (final Exception e) {
-            logger.warn("Failed to set permissions so that only the owner can read status file {}; "
-                + "this may allows others to have access to the key needed to communicate with MiNiFi. "
-                + "Permissions should be changed so that only the owner can read this file", statusFile);
-        }
-
-        try (final FileOutputStream fos = new FileOutputStream(statusFile)) {
-            minifiProps.store(fos, null);
-            fos.getFD().sync();
-        }
-
-        logger.debug("Saved Properties {} to {}", new Object[]{minifiProps, statusFile});
-    }
-
-    private synchronized void writePidFile(final String pid, final Logger logger) throws IOException {
-        final File pidFile = getPidFile(logger);
-        if (pidFile.exists() && !pidFile.delete()) {
-           logger.warn("Failed to delete {}", pidFile);
-        }
-
-        if (!pidFile.createNewFile()) {
-            throw new IOException("Failed to create file " + pidFile);
-        }
-
-        try {
-            final Set<PosixFilePermission> perms = new HashSet<>();
-            perms.add(PosixFilePermission.OWNER_READ);
-            perms.add(PosixFilePermission.OWNER_WRITE);
-            Files.setPosixFilePermissions(pidFile.toPath(), perms);
-        } catch (final Exception e) {
-            logger.warn("Failed to set permissions so that only the owner can read pid file {}; "
-                    + "this may allows others to have access to the key needed to communicate with MiNiFi. "
-                    + "Permissions should be changed so that only the owner can read this file", pidFile);
-        }
-
-        try (final FileOutputStream fos = new FileOutputStream(pidFile)) {
-            fos.write(pid.getBytes(StandardCharsets.UTF_8));
-            fos.getFD().sync();
-        }
-
-        logger.debug("Saved Pid {} to {}", new Object[]{pid, pidFile});
-    }
-
-    private boolean isPingSuccessful(final int port, final String secretKey, final Logger logger) {
-        logger.debug("Pinging {}", port);
-
-        try (final Socket socket = new Socket("localhost", port)) {
-            final OutputStream out = socket.getOutputStream();
-            out.write((PING_CMD + " " + secretKey + "\n").getBytes(StandardCharsets.UTF_8));
-            out.flush();
-
-            logger.debug("Sent PING command");
-            socket.setSoTimeout(5000);
-            final InputStream in = socket.getInputStream();
-            final BufferedReader reader = new BufferedReader(new InputStreamReader(in));
-            final String response = reader.readLine();
-            logger.debug("PING response: {}", response);
-            out.close();
-            reader.close();
-
-            return PING_CMD.equals(response);
-        } catch (final IOException ioe) {
-            return false;
-        }
-    }
-
-    private Integer getCurrentPort(final Logger logger) throws IOException {
-        final Properties props = loadProperties(logger);
-        final String portVal = props.getProperty("port");
-        if (portVal == null) {
-            logger.debug("No Port found in status file");
-            return null;
-        } else {
-            logger.debug("Port defined in status file: {}", portVal);
-        }
-
-        final int port = Integer.parseInt(portVal);
-        final boolean success = isPingSuccessful(port, props.getProperty("secret.key"), logger);
-        if (success) {
-            logger.debug("Successful PING on port {}", port);
-            return port;
-        }
-
-        final String pid = props.getProperty(PID_KEY);
-        logger.debug("PID in status file is {}", pid);
-        if (pid != null) {
-            final boolean procRunning = isProcessRunning(pid, logger);
-            if (procRunning) {
-                return port;
-            } else {
-                return null;
-            }
-        }
-
-        return null;
-    }
-
-    private boolean isProcessRunning(final String pid, final Logger logger) {
-        try {
-            // We use the "ps" command to check if the process is still running.
-            final ProcessBuilder builder = new ProcessBuilder();
-
-            builder.command("ps", "-p", pid);
-            final Process proc = builder.start();
-
-            // Look for the pid in the output of the 'ps' command.
-            boolean running = false;
-            String line;
-            try (final InputStream in = proc.getInputStream();
-                 final Reader streamReader = new InputStreamReader(in);
-                 final BufferedReader reader = new BufferedReader(streamReader)) {
-
-                while ((line = reader.readLine()) != null) {
-                    if (line.trim().startsWith(pid)) {
-                        running = true;
-                    }
-                }
-            }
-
-            // If output of the ps command had our PID, the process is running.
-            if (running) {
-                logger.debug("Process with PID {} is running", pid);
-            } else {
-                logger.debug("Process with PID {} is not running", pid);
-            }
-
-            return running;
-        } catch (final IOException ioe) {
-            System.err.println("Failed to determine if Process " + pid + " is running; assuming that it is not");
-            return false;
-        }
-    }
-
-    private Status getStatus(final Logger logger) {
-        final Properties props;
-        try {
-            props = loadProperties(logger);
-        } catch (final IOException ioe) {
-            return new Status(null, null, false, false);
-        }
-
-        if (props == null) {
-            return new Status(null, null, false, false);
-        }
-
-        final String portValue = props.getProperty("port");
-        final String pid = props.getProperty(PID_KEY);
-        final String secretKey = props.getProperty("secret.key");
-
-        if (portValue == null && pid == null) {
-            return new Status(null, null, false, false);
-        }
-
-        Integer port = null;
-        boolean pingSuccess = false;
-        if (portValue != null) {
-            try {
-                port = Integer.parseInt(portValue);
-                pingSuccess = isPingSuccessful(port, secretKey, logger);
-            } catch (final NumberFormatException nfe) {
-                return new Status(null, null, false, false);
-            }
-        }
-
-        if (pingSuccess) {
-            return new Status(port, pid, true, true);
-        }
-
-        final boolean alive = (pid != null) && isProcessRunning(pid, logger);
-        return new Status(port, pid, pingSuccess, alive);
-    }
-
-    public int status() throws IOException {
-        final Logger logger = cmdLogger;
-        final Status status = getStatus(logger);
-        if (status.isRespondingToPing()) {
-            logger.info("Apache MiNiFi is currently running, listening to Bootstrap on port {}, PID={}",
-                new Object[]{status.getPort(), status.getPid() == null ? "unknown" : status.getPid()});
-            return 0;
-        }
-
-        if (status.isProcessRunning()) {
-            logger.info("Apache MiNiFi is running at PID {} but is not responding to ping requests", status.getPid());
-            return 4;
-        }
-
-        if (status.getPort() == null) {
-            logger.info("Apache MiNiFi is not running");
-            return 3;
-        }
-
-        if (status.getPid() == null) {
-            logger.info("Apache MiNiFi is not responding to Ping requests. The process may have died or may be hung");
-        } else {
-            logger.info("Apache MiNiFi is not running");
-        }
-        return 3;
-    }
-
-    public FlowStatusReport statusReport(String statusRequest) throws IOException {
-        final Logger logger = cmdLogger;
-        final Status status = getStatus(logger);
-        final Properties props = loadProperties(logger);
-
-        List<String> problemsGeneratingReport = new LinkedList<>();
-        if (!status.isProcessRunning()) {
-            problemsGeneratingReport.add("MiNiFi process is not running");
-        }
-
-        if (!status.isRespondingToPing()) {
-            problemsGeneratingReport.add("MiNiFi process is not responding to pings");
-        }
-
-        if (!problemsGeneratingReport.isEmpty()) {
-            FlowStatusReport flowStatusReport = new FlowStatusReport();
-            flowStatusReport.setErrorsGeneratingReport(problemsGeneratingReport);
-            return flowStatusReport;
-        }
-
-        return getFlowStatusReport(statusRequest, status.getPort(), props.getProperty("secret.key"), logger);
-    }
-
-    public void env() {
-        final Logger logger = cmdLogger;
-        final Status status = getStatus(logger);
-        if (status.getPid() == null) {
-            logger.info("Apache MiNiFi is not running");
+    public void setMiNiFiParameters(int port, String secretKey) throws IOException {
+        if (Optional.ofNullable(secretKey).filter(key -> key.equals(miNiFiParameters.getSecretKey())).isPresent() && miNiFiParameters.getMiNiFiPort() == port) {
+            DEFAULT_LOGGER.debug("secretKey and port match with the known one, nothing to update");
             return;
         }
-        final Class<?> virtualMachineClass;
-        try {
-            virtualMachineClass = Class.forName("com.sun.tools.attach.VirtualMachine");
-        } catch (final ClassNotFoundException cnfe) {
-            logger.error("Seems tools.jar (Linux / Windows JDK) or classes.jar (Mac OS) is not available in classpath");
-            return;
-        }
-        final Method attachMethod;
-        final Method detachMethod;
 
-        try {
-            attachMethod = virtualMachineClass.getMethod("attach", String.class);
-            detachMethod = virtualMachineClass.getDeclaredMethod("detach");
-        } catch (final Exception e) {
-            logger.error("Methods required for getting environment not available", e);
-            return;
-        }
+        miNiFiParameters.setMiNiFiPort(port);
+        miNiFiParameters.setSecretKey(secretKey);
 
-        final Object virtualMachine;
-        try {
-            virtualMachine = attachMethod.invoke(null, status.getPid());
-        } catch (final Throwable t) {
-            logger.error("Problem attaching to MiNiFi", t);
-            return;
+        Properties minifiProps = new Properties();
+        long minifiPid = miNiFiParameters.getMinifiPid();
+        if (minifiPid != UNINITIALIZED) {
+            minifiProps.setProperty(STATUS_FILE_PID_KEY, String.valueOf(minifiPid));
         }
+        minifiProps.setProperty(STATUS_FILE_PORT_KEY, String.valueOf(port));
+        minifiProps.setProperty(STATUS_FILE_SECRET_KEY, secretKey);
 
+        File statusFile = bootstrapFileProvider.getStatusFile();
         try {
-            final Method getSystemPropertiesMethod = virtualMachine.getClass().getMethod("getSystemProperties");
-
-            final Properties sysProps = (Properties) getSystemPropertiesMethod.invoke(virtualMachine);
-            for (Entry<Object, Object> syspropEntry : sysProps.entrySet()) {
-                logger.info(syspropEntry.getKey().toString() + " = " + syspropEntry.getValue().toString());
-            }
-        } catch (Throwable t) {
-            throw new RuntimeException(t);
-        } finally {
-            try {
-                detachMethod.invoke(virtualMachine);
-            } catch (final Exception e) {
-                logger.warn("Caught exception detaching from process", e);
-            }
-        }
-    }
-
-    /**
-     * Writes a MiNiFi thread dump to the given file; if file is null, logs at
-     * INFO level instead.
-     *
-     * @param dumpFile the file to write the dump content to
-     * @throws IOException if any issues occur while writing the dump file
-     */
-    public void dump(final File dumpFile) throws IOException {
-        final Logger logger = defaultLogger;    // dump to bootstrap log file by default
-        final Integer port = getCurrentPort(logger);
-        if (port == null) {
-            logger.info("Apache MiNiFi is not currently running");
-            return;
+            bootstrapFileProvider.saveStatusProperties(minifiProps);
+        } catch (IOException ioe) {
+            DEFAULT_LOGGER.warn("Apache MiNiFi has started but failed to persist MiNiFi Port information to {} due to {}", statusFile.getAbsolutePath(), ioe);

Review Comment:
   Recommend remove the placeholder for the exception since the stack trace will be logged.
   ```suggestion
               DEFAULT_LOGGER.warn("Apache MiNiFi has started but failed to persist MiNiFi Port information to {}", statusFile.getAbsolutePath(), ioe);
   ```



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/command/EnvRunner.java:
##########
@@ -0,0 +1,107 @@
+/*
+ * 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.nifi.minifi.bootstrap.command;
+
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CMD_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.DEFAULT_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.Status.ERROR;
+import static org.apache.nifi.minifi.bootstrap.Status.MINIFI_NOT_RUNNING;
+import static org.apache.nifi.minifi.bootstrap.Status.OK;
+
+import java.lang.reflect.Method;
+import java.util.Map.Entry;
+import java.util.Properties;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.apache.nifi.minifi.bootstrap.MiNiFiStatus;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiStatusProvider;
+
+public class EnvRunner implements CommandRunner {
+    private final MiNiFiParameters miNiFiParameters;
+    private final MiNiFiStatusProvider miNiFiStatusProvider;
+
+    public EnvRunner(MiNiFiParameters miNiFiParameters, MiNiFiStatusProvider miNiFiStatusProvider) {
+        this.miNiFiParameters = miNiFiParameters;
+        this.miNiFiStatusProvider = miNiFiStatusProvider;
+    }
+
+    /**
+     * Returns information about the MiNiFi's virtual machine.
+     * @param args the input arguments
+     * @return status code
+     */
+    @Override
+    public int runCommand(String[] args) {
+        return env();
+    }
+
+    private int env() {
+        MiNiFiStatus status = miNiFiStatusProvider.getStatus(miNiFiParameters.getMiNiFiPort(), miNiFiParameters.getMinifiPid());
+        if (status.getPid() == null) {
+            CMD_LOGGER.error("Apache MiNiFi is not running");
+            return MINIFI_NOT_RUNNING.getStatusCode();
+        }
+
+        Class<?> virtualMachineClass;
+        try {
+            virtualMachineClass = Class.forName("com.sun.tools.attach.VirtualMachine");
+        } catch (ClassNotFoundException cnfe) {
+            CMD_LOGGER.error("Seems tools.jar (Linux / Windows JDK) or classes.jar (Mac OS) is not available in classpath");
+            return ERROR.getStatusCode();
+        }
+
+        Method attachMethod;
+        Method detachMethod;
+        try {
+            attachMethod = virtualMachineClass.getMethod("attach", String.class);
+            detachMethod = virtualMachineClass.getDeclaredMethod("detach");
+        } catch (Exception e) {
+            CMD_LOGGER.error("Methods required for getting environment not available");
+            DEFAULT_LOGGER.error("Exception:", e);
+            return ERROR.getStatusCode();
+        }
+
+        Object virtualMachine;
+        try {
+            virtualMachine = attachMethod.invoke(null, status.getPid());
+        } catch (Exception e) {
+            CMD_LOGGER.error("Problem attaching to MiNiFi");
+            DEFAULT_LOGGER.error("Exception:", e);
+            return ERROR.getStatusCode();
+        }
+
+        try {
+            Method getSystemPropertiesMethod = virtualMachine.getClass().getMethod("getSystemProperties");
+
+            Properties sysProps = (Properties) getSystemPropertiesMethod.invoke(virtualMachine);

Review Comment:
   Is there a reason for using this approach as opposed to `System.getProperties()`?



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/command/StartRunner.java:
##########
@@ -0,0 +1,435 @@
+/*
+ * 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.nifi.minifi.bootstrap.command;
+
+import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CMD_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CONF_DIR_KEY;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.DEFAULT_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.MINIFI_CONFIG_FILE_KEY;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.STATUS_FILE_PID_KEY;
+import static org.apache.nifi.minifi.bootstrap.Status.ERROR;
+import static org.apache.nifi.minifi.bootstrap.Status.OK;
+import static org.apache.nifi.minifi.bootstrap.util.ConfigTransformer.generateConfigFiles;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.nifi.bootstrap.util.OSUtils;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.apache.nifi.minifi.bootstrap.RunMiNiFi;
+import org.apache.nifi.minifi.bootstrap.ShutdownHook;
+import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeException;
+import org.apache.nifi.minifi.bootstrap.exception.StartupFailureException;
+import org.apache.nifi.minifi.bootstrap.service.BootstrapFileProvider;
+import org.apache.nifi.minifi.bootstrap.service.CurrentPortProvider;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiListener;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiStdLogHandler;
+import org.apache.nifi.minifi.bootstrap.service.PeriodicStatusReporterManager;
+import org.apache.nifi.minifi.bootstrap.util.ProcessUtils;
+import org.apache.nifi.util.Tuple;
+
+public class StartRunner implements CommandRunner {
+    private static final String DEFAULT_JAVA_CMD = "java";
+    private static final String DEFAULT_LOG_DIR = "./logs";
+    private static final String DEFAULT_LIB_DIR = "./lib";
+    private static final String DEFAULT_CONF_DIR = "./conf";
+    private static final int STARTUP_WAIT_SECONDS = 60;
+    private static final String DEFAULT_CONFIG_FILE = DEFAULT_CONF_DIR + "/bootstrap.conf";
+
+    private final CurrentPortProvider currentPortProvider;
+    private final BootstrapFileProvider bootstrapFileProvider;
+    private final PeriodicStatusReporterManager periodicStatusReporterManager;
+    private final MiNiFiStdLogHandler miNiFiStdLogHandler;
+    private final MiNiFiParameters miNiFiParameters;
+    private final File bootstrapConfigFile;
+    private final Lock lock = new ReentrantLock();
+    private final Condition startupCondition = lock.newCondition();
+    private final RunMiNiFi runMiNiFi;
+    private volatile ShutdownHook shutdownHook;
+
+    public StartRunner(CurrentPortProvider currentPortProvider, BootstrapFileProvider bootstrapFileProvider,
+        PeriodicStatusReporterManager periodicStatusReporterManager, MiNiFiStdLogHandler miNiFiStdLogHandler, MiNiFiParameters miNiFiParameters, File bootstrapConfigFile,
+        RunMiNiFi runMiNiFi) {
+        this.currentPortProvider = currentPortProvider;
+        this.bootstrapFileProvider = bootstrapFileProvider;
+        this.periodicStatusReporterManager = periodicStatusReporterManager;
+        this.miNiFiStdLogHandler = miNiFiStdLogHandler;
+        this.miNiFiParameters = miNiFiParameters;
+        this.bootstrapConfigFile = bootstrapConfigFile;
+        this.runMiNiFi = runMiNiFi;
+    }
+
+    /**
+     * Starts (and restarts) MiNiFi process during the whole lifecycle of the bootstrap process.
+     * @param args the input arguments
+     * @return status code
+     */
+    @Override
+    public int runCommand(String[] args) {
+        try {
+            start();
+        } catch (Exception e) {
+            CMD_LOGGER.error("Exception happened during MiNiFi startup", e);
+            return ERROR.getStatusCode();
+        }
+        return OK.getStatusCode();
+    }
+
+    private void start() throws IOException, InterruptedException {
+        Integer port = currentPortProvider.getCurrentPort();
+        if (port != null) {
+            String alreadyRunningMessage = "Apache MiNiFi is already running";
+            CMD_LOGGER.info(alreadyRunningMessage + ", listening to Bootstrap on port {}", port);
+            return;
+        }
+
+        File prevLockFile = bootstrapFileProvider.getLockFile();
+        if (prevLockFile.exists() && !prevLockFile.delete()) {
+            CMD_LOGGER.warn("Failed to delete previous lock file {}; this file should be cleaned up manually", prevLockFile);
+        }
+
+        Properties bootstrapProperties = bootstrapFileProvider.getBootstrapProperties();
+        String confDir = bootstrapProperties.getProperty(CONF_DIR_KEY);
+        initConfigFiles(bootstrapProperties, confDir);
+
+        Tuple<ProcessBuilder, Process> tuple = startMiNiFi();
+        ProcessBuilder builder = tuple.getKey();
+        Process process = tuple.getValue();
+
+        try {
+            while (true) {
+                if (ProcessUtils.isAlive(process)) {
+                    handleReload();
+                } else {
+                    Runtime runtime = Runtime.getRuntime();
+                    try {
+                        runtime.removeShutdownHook(shutdownHook);
+                    } catch (IllegalStateException ise) {
+                        // happens when already shutting down

Review Comment:
   Perhaps this should be logged at the trace level instead of having an empty block?



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/command/EnvRunner.java:
##########
@@ -0,0 +1,107 @@
+/*
+ * 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.nifi.minifi.bootstrap.command;
+
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CMD_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.DEFAULT_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.Status.ERROR;
+import static org.apache.nifi.minifi.bootstrap.Status.MINIFI_NOT_RUNNING;
+import static org.apache.nifi.minifi.bootstrap.Status.OK;
+
+import java.lang.reflect.Method;
+import java.util.Map.Entry;
+import java.util.Properties;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.apache.nifi.minifi.bootstrap.MiNiFiStatus;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiStatusProvider;
+
+public class EnvRunner implements CommandRunner {
+    private final MiNiFiParameters miNiFiParameters;
+    private final MiNiFiStatusProvider miNiFiStatusProvider;
+
+    public EnvRunner(MiNiFiParameters miNiFiParameters, MiNiFiStatusProvider miNiFiStatusProvider) {
+        this.miNiFiParameters = miNiFiParameters;
+        this.miNiFiStatusProvider = miNiFiStatusProvider;
+    }
+
+    /**
+     * Returns information about the MiNiFi's virtual machine.
+     * @param args the input arguments
+     * @return status code
+     */
+    @Override
+    public int runCommand(String[] args) {
+        return env();
+    }
+
+    private int env() {
+        MiNiFiStatus status = miNiFiStatusProvider.getStatus(miNiFiParameters.getMiNiFiPort(), miNiFiParameters.getMinifiPid());
+        if (status.getPid() == null) {
+            CMD_LOGGER.error("Apache MiNiFi is not running");
+            return MINIFI_NOT_RUNNING.getStatusCode();
+        }
+
+        Class<?> virtualMachineClass;
+        try {
+            virtualMachineClass = Class.forName("com.sun.tools.attach.VirtualMachine");
+        } catch (ClassNotFoundException cnfe) {
+            CMD_LOGGER.error("Seems tools.jar (Linux / Windows JDK) or classes.jar (Mac OS) is not available in classpath");
+            return ERROR.getStatusCode();
+        }
+
+        Method attachMethod;
+        Method detachMethod;
+        try {
+            attachMethod = virtualMachineClass.getMethod("attach", String.class);
+            detachMethod = virtualMachineClass.getDeclaredMethod("detach");
+        } catch (Exception e) {
+            CMD_LOGGER.error("Methods required for getting environment not available");
+            DEFAULT_LOGGER.error("Exception:", e);
+            return ERROR.getStatusCode();
+        }
+
+        Object virtualMachine;
+        try {
+            virtualMachine = attachMethod.invoke(null, status.getPid());
+        } catch (Exception e) {
+            CMD_LOGGER.error("Problem attaching to MiNiFi");
+            DEFAULT_LOGGER.error("Exception:", e);
+            return ERROR.getStatusCode();
+        }
+
+        try {
+            Method getSystemPropertiesMethod = virtualMachine.getClass().getMethod("getSystemProperties");
+
+            Properties sysProps = (Properties) getSystemPropertiesMethod.invoke(virtualMachine);
+            for (Entry<Object, Object> syspropEntry : sysProps.entrySet()) {
+                CMD_LOGGER.info(syspropEntry.getKey().toString() + " = " + syspropEntry.getValue().toString());
+            }
+        } catch (Exception e) {
+            CMD_LOGGER.error("Exception happened during the systemproperties call");
+            DEFAULT_LOGGER.error("Exception:", e);
+            return ERROR.getStatusCode();
+        } finally {
+            try {
+                detachMethod.invoke(virtualMachine);
+            } catch (final Exception e) {
+                CMD_LOGGER.warn("Caught exception detaching from process", e);

Review Comment:
   ```suggestion
                   CMD_LOGGER.warn("Virtual Machine detachment failed", e);
   ```



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/command/EnvRunner.java:
##########
@@ -0,0 +1,107 @@
+/*
+ * 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.nifi.minifi.bootstrap.command;
+
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CMD_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.DEFAULT_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.Status.ERROR;
+import static org.apache.nifi.minifi.bootstrap.Status.MINIFI_NOT_RUNNING;
+import static org.apache.nifi.minifi.bootstrap.Status.OK;
+
+import java.lang.reflect.Method;
+import java.util.Map.Entry;
+import java.util.Properties;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.apache.nifi.minifi.bootstrap.MiNiFiStatus;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiStatusProvider;
+
+public class EnvRunner implements CommandRunner {
+    private final MiNiFiParameters miNiFiParameters;
+    private final MiNiFiStatusProvider miNiFiStatusProvider;
+
+    public EnvRunner(MiNiFiParameters miNiFiParameters, MiNiFiStatusProvider miNiFiStatusProvider) {
+        this.miNiFiParameters = miNiFiParameters;
+        this.miNiFiStatusProvider = miNiFiStatusProvider;
+    }
+
+    /**
+     * Returns information about the MiNiFi's virtual machine.
+     * @param args the input arguments
+     * @return status code
+     */
+    @Override
+    public int runCommand(String[] args) {
+        return env();
+    }
+
+    private int env() {
+        MiNiFiStatus status = miNiFiStatusProvider.getStatus(miNiFiParameters.getMiNiFiPort(), miNiFiParameters.getMinifiPid());
+        if (status.getPid() == null) {
+            CMD_LOGGER.error("Apache MiNiFi is not running");
+            return MINIFI_NOT_RUNNING.getStatusCode();
+        }
+
+        Class<?> virtualMachineClass;
+        try {
+            virtualMachineClass = Class.forName("com.sun.tools.attach.VirtualMachine");
+        } catch (ClassNotFoundException cnfe) {
+            CMD_LOGGER.error("Seems tools.jar (Linux / Windows JDK) or classes.jar (Mac OS) is not available in classpath");
+            return ERROR.getStatusCode();
+        }
+
+        Method attachMethod;
+        Method detachMethod;
+        try {
+            attachMethod = virtualMachineClass.getMethod("attach", String.class);
+            detachMethod = virtualMachineClass.getDeclaredMethod("detach");
+        } catch (Exception e) {
+            CMD_LOGGER.error("Methods required for getting environment not available");
+            DEFAULT_LOGGER.error("Exception:", e);
+            return ERROR.getStatusCode();
+        }
+
+        Object virtualMachine;
+        try {
+            virtualMachine = attachMethod.invoke(null, status.getPid());
+        } catch (Exception e) {
+            CMD_LOGGER.error("Problem attaching to MiNiFi");
+            DEFAULT_LOGGER.error("Exception:", e);
+            return ERROR.getStatusCode();
+        }
+
+        try {
+            Method getSystemPropertiesMethod = virtualMachine.getClass().getMethod("getSystemProperties");
+
+            Properties sysProps = (Properties) getSystemPropertiesMethod.invoke(virtualMachine);
+            for (Entry<Object, Object> syspropEntry : sysProps.entrySet()) {
+                CMD_LOGGER.info(syspropEntry.getKey().toString() + " = " + syspropEntry.getValue().toString());
+            }
+        } catch (Exception e) {
+            CMD_LOGGER.error("Exception happened during the systemproperties call");
+            DEFAULT_LOGGER.error("Exception:", e);

Review Comment:
   This could probably be removed if the above approach could be refactored to using System.getProperties.



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/command/StartRunner.java:
##########
@@ -0,0 +1,435 @@
+/*
+ * 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.nifi.minifi.bootstrap.command;
+
+import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CMD_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CONF_DIR_KEY;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.DEFAULT_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.MINIFI_CONFIG_FILE_KEY;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.STATUS_FILE_PID_KEY;
+import static org.apache.nifi.minifi.bootstrap.Status.ERROR;
+import static org.apache.nifi.minifi.bootstrap.Status.OK;
+import static org.apache.nifi.minifi.bootstrap.util.ConfigTransformer.generateConfigFiles;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.nifi.bootstrap.util.OSUtils;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.apache.nifi.minifi.bootstrap.RunMiNiFi;
+import org.apache.nifi.minifi.bootstrap.ShutdownHook;
+import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeException;
+import org.apache.nifi.minifi.bootstrap.exception.StartupFailureException;
+import org.apache.nifi.minifi.bootstrap.service.BootstrapFileProvider;
+import org.apache.nifi.minifi.bootstrap.service.CurrentPortProvider;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiListener;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiStdLogHandler;
+import org.apache.nifi.minifi.bootstrap.service.PeriodicStatusReporterManager;
+import org.apache.nifi.minifi.bootstrap.util.ProcessUtils;
+import org.apache.nifi.util.Tuple;
+
+public class StartRunner implements CommandRunner {
+    private static final String DEFAULT_JAVA_CMD = "java";
+    private static final String DEFAULT_LOG_DIR = "./logs";
+    private static final String DEFAULT_LIB_DIR = "./lib";
+    private static final String DEFAULT_CONF_DIR = "./conf";
+    private static final int STARTUP_WAIT_SECONDS = 60;
+    private static final String DEFAULT_CONFIG_FILE = DEFAULT_CONF_DIR + "/bootstrap.conf";
+
+    private final CurrentPortProvider currentPortProvider;
+    private final BootstrapFileProvider bootstrapFileProvider;
+    private final PeriodicStatusReporterManager periodicStatusReporterManager;
+    private final MiNiFiStdLogHandler miNiFiStdLogHandler;
+    private final MiNiFiParameters miNiFiParameters;
+    private final File bootstrapConfigFile;
+    private final Lock lock = new ReentrantLock();
+    private final Condition startupCondition = lock.newCondition();
+    private final RunMiNiFi runMiNiFi;
+    private volatile ShutdownHook shutdownHook;
+
+    public StartRunner(CurrentPortProvider currentPortProvider, BootstrapFileProvider bootstrapFileProvider,
+        PeriodicStatusReporterManager periodicStatusReporterManager, MiNiFiStdLogHandler miNiFiStdLogHandler, MiNiFiParameters miNiFiParameters, File bootstrapConfigFile,
+        RunMiNiFi runMiNiFi) {
+        this.currentPortProvider = currentPortProvider;
+        this.bootstrapFileProvider = bootstrapFileProvider;
+        this.periodicStatusReporterManager = periodicStatusReporterManager;
+        this.miNiFiStdLogHandler = miNiFiStdLogHandler;
+        this.miNiFiParameters = miNiFiParameters;
+        this.bootstrapConfigFile = bootstrapConfigFile;
+        this.runMiNiFi = runMiNiFi;
+    }
+
+    /**
+     * Starts (and restarts) MiNiFi process during the whole lifecycle of the bootstrap process.
+     * @param args the input arguments
+     * @return status code
+     */
+    @Override
+    public int runCommand(String[] args) {
+        try {
+            start();
+        } catch (Exception e) {
+            CMD_LOGGER.error("Exception happened during MiNiFi startup", e);
+            return ERROR.getStatusCode();
+        }
+        return OK.getStatusCode();
+    }
+
+    private void start() throws IOException, InterruptedException {
+        Integer port = currentPortProvider.getCurrentPort();
+        if (port != null) {
+            String alreadyRunningMessage = "Apache MiNiFi is already running";
+            CMD_LOGGER.info(alreadyRunningMessage + ", listening to Bootstrap on port {}", port);
+            return;
+        }
+
+        File prevLockFile = bootstrapFileProvider.getLockFile();
+        if (prevLockFile.exists() && !prevLockFile.delete()) {
+            CMD_LOGGER.warn("Failed to delete previous lock file {}; this file should be cleaned up manually", prevLockFile);
+        }
+
+        Properties bootstrapProperties = bootstrapFileProvider.getBootstrapProperties();
+        String confDir = bootstrapProperties.getProperty(CONF_DIR_KEY);
+        initConfigFiles(bootstrapProperties, confDir);
+
+        Tuple<ProcessBuilder, Process> tuple = startMiNiFi();
+        ProcessBuilder builder = tuple.getKey();
+        Process process = tuple.getValue();
+
+        try {
+            while (true) {
+                if (ProcessUtils.isAlive(process)) {
+                    handleReload();
+                } else {
+                    Runtime runtime = Runtime.getRuntime();
+                    try {
+                        runtime.removeShutdownHook(shutdownHook);
+                    } catch (IllegalStateException ise) {
+                        // happens when already shutting down
+                    }
+
+                    if (runMiNiFi.isAutoRestartNiFi() && needRestart()) {
+                        File reloadFile = bootstrapFileProvider.getReloadLockFile();
+                        if (reloadFile.exists()) {
+                            DEFAULT_LOGGER.info("Currently reloading configuration. Will wait to restart MiNiFi.");
+                            Thread.sleep(5000L);
+                            continue;
+                        }

Review Comment:
   Instead of having this while loop and sleep, what about refactoring to having a Runnable command that executes on a scheduled basis of every 5 seconds?



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/command/StartRunner.java:
##########
@@ -0,0 +1,435 @@
+/*
+ * 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.nifi.minifi.bootstrap.command;
+
+import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CMD_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CONF_DIR_KEY;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.DEFAULT_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.MINIFI_CONFIG_FILE_KEY;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.STATUS_FILE_PID_KEY;
+import static org.apache.nifi.minifi.bootstrap.Status.ERROR;
+import static org.apache.nifi.minifi.bootstrap.Status.OK;
+import static org.apache.nifi.minifi.bootstrap.util.ConfigTransformer.generateConfigFiles;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.nifi.bootstrap.util.OSUtils;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.apache.nifi.minifi.bootstrap.RunMiNiFi;
+import org.apache.nifi.minifi.bootstrap.ShutdownHook;
+import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeException;
+import org.apache.nifi.minifi.bootstrap.exception.StartupFailureException;
+import org.apache.nifi.minifi.bootstrap.service.BootstrapFileProvider;
+import org.apache.nifi.minifi.bootstrap.service.CurrentPortProvider;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiListener;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiStdLogHandler;
+import org.apache.nifi.minifi.bootstrap.service.PeriodicStatusReporterManager;
+import org.apache.nifi.minifi.bootstrap.util.ProcessUtils;
+import org.apache.nifi.util.Tuple;
+
+public class StartRunner implements CommandRunner {
+    private static final String DEFAULT_JAVA_CMD = "java";
+    private static final String DEFAULT_LOG_DIR = "./logs";
+    private static final String DEFAULT_LIB_DIR = "./lib";
+    private static final String DEFAULT_CONF_DIR = "./conf";
+    private static final int STARTUP_WAIT_SECONDS = 60;
+    private static final String DEFAULT_CONFIG_FILE = DEFAULT_CONF_DIR + "/bootstrap.conf";
+
+    private final CurrentPortProvider currentPortProvider;
+    private final BootstrapFileProvider bootstrapFileProvider;
+    private final PeriodicStatusReporterManager periodicStatusReporterManager;
+    private final MiNiFiStdLogHandler miNiFiStdLogHandler;
+    private final MiNiFiParameters miNiFiParameters;
+    private final File bootstrapConfigFile;
+    private final Lock lock = new ReentrantLock();
+    private final Condition startupCondition = lock.newCondition();
+    private final RunMiNiFi runMiNiFi;
+    private volatile ShutdownHook shutdownHook;
+
+    public StartRunner(CurrentPortProvider currentPortProvider, BootstrapFileProvider bootstrapFileProvider,
+        PeriodicStatusReporterManager periodicStatusReporterManager, MiNiFiStdLogHandler miNiFiStdLogHandler, MiNiFiParameters miNiFiParameters, File bootstrapConfigFile,
+        RunMiNiFi runMiNiFi) {
+        this.currentPortProvider = currentPortProvider;
+        this.bootstrapFileProvider = bootstrapFileProvider;
+        this.periodicStatusReporterManager = periodicStatusReporterManager;
+        this.miNiFiStdLogHandler = miNiFiStdLogHandler;
+        this.miNiFiParameters = miNiFiParameters;
+        this.bootstrapConfigFile = bootstrapConfigFile;
+        this.runMiNiFi = runMiNiFi;
+    }
+
+    /**
+     * Starts (and restarts) MiNiFi process during the whole lifecycle of the bootstrap process.
+     * @param args the input arguments
+     * @return status code
+     */
+    @Override
+    public int runCommand(String[] args) {
+        try {
+            start();
+        } catch (Exception e) {
+            CMD_LOGGER.error("Exception happened during MiNiFi startup", e);
+            return ERROR.getStatusCode();
+        }
+        return OK.getStatusCode();
+    }
+
+    private void start() throws IOException, InterruptedException {
+        Integer port = currentPortProvider.getCurrentPort();
+        if (port != null) {
+            String alreadyRunningMessage = "Apache MiNiFi is already running";
+            CMD_LOGGER.info(alreadyRunningMessage + ", listening to Bootstrap on port {}", port);
+            return;
+        }
+
+        File prevLockFile = bootstrapFileProvider.getLockFile();
+        if (prevLockFile.exists() && !prevLockFile.delete()) {
+            CMD_LOGGER.warn("Failed to delete previous lock file {}; this file should be cleaned up manually", prevLockFile);
+        }
+
+        Properties bootstrapProperties = bootstrapFileProvider.getBootstrapProperties();
+        String confDir = bootstrapProperties.getProperty(CONF_DIR_KEY);
+        initConfigFiles(bootstrapProperties, confDir);
+
+        Tuple<ProcessBuilder, Process> tuple = startMiNiFi();
+        ProcessBuilder builder = tuple.getKey();
+        Process process = tuple.getValue();
+
+        try {
+            while (true) {
+                if (ProcessUtils.isAlive(process)) {
+                    handleReload();
+                } else {
+                    Runtime runtime = Runtime.getRuntime();
+                    try {
+                        runtime.removeShutdownHook(shutdownHook);
+                    } catch (IllegalStateException ise) {
+                        // happens when already shutting down
+                    }
+
+                    if (runMiNiFi.isAutoRestartNiFi() && needRestart()) {
+                        File reloadFile = bootstrapFileProvider.getReloadLockFile();
+                        if (reloadFile.exists()) {
+                            DEFAULT_LOGGER.info("Currently reloading configuration. Will wait to restart MiNiFi.");
+                            Thread.sleep(5000L);
+                            continue;
+                        }
+
+                        process = restartNifi(bootstrapProperties, confDir, builder, runtime);
+                        // failed to start process
+                        if (process == null) {
+                            return;
+                        }
+                    } else {
+                        return;
+                    }
+                }
+            }
+        } finally {
+            miNiFiStdLogHandler.shutdown();
+            runMiNiFi.shutdownChangeNotifier();
+            periodicStatusReporterManager.shutdownPeriodicStatusReporters();
+        }
+    }
+
+    private Process restartNifi(Properties bootstrapProperties, String confDir, ProcessBuilder builder, Runtime runtime) throws IOException {
+        Process process;
+        boolean previouslyStarted = runMiNiFi.isNiFiStarted();
+        if (!previouslyStarted) {
+            File swapConfigFile = bootstrapFileProvider.getSwapFile();
+            if (swapConfigFile.exists()) {
+                DEFAULT_LOGGER.info("Swap file exists, MiNiFi failed trying to change configuration. Reverting to old configuration.");
+
+                try {
+                    ByteBuffer tempConfigFile = generateConfigFiles(new FileInputStream(swapConfigFile), confDir, bootstrapProperties);
+                    runMiNiFi.getConfigFileReference().set(tempConfigFile.asReadOnlyBuffer());
+                } catch (ConfigurationChangeException e) {
+                    DEFAULT_LOGGER.error("The swap file is malformed, unable to restart from prior state. Will not attempt to restart MiNiFi. Swap File should be cleaned up manually.");
+                    return null;
+                }
+
+                Files.copy(swapConfigFile.toPath(), Paths.get(bootstrapProperties.getProperty(MINIFI_CONFIG_FILE_KEY)), REPLACE_EXISTING);
+
+                DEFAULT_LOGGER.info("Replacing config file with swap file and deleting swap file");
+                if (!swapConfigFile.delete()) {
+                    DEFAULT_LOGGER.warn("The swap file failed to delete after replacing using it to revert to the old configuration. It should be cleaned up manually.");
+                }
+                runMiNiFi.setReloading(false);
+            } else {
+                DEFAULT_LOGGER.info("MiNiFi either never started or failed to restart. Will not attempt to restart MiNiFi");
+                return null;
+            }
+        } else {
+            runMiNiFi.setNiFiStarted(false);
+        }
+
+        miNiFiParameters.setSecretKey(null);
+        process = builder.start();
+        miNiFiStdLogHandler.initLogging(process);
+
+        Long pid = OSUtils.getProcessId(process, DEFAULT_LOGGER);
+        if (pid != null) {
+            miNiFiParameters.setMinifiPid(pid);
+            Properties minifiProps = new Properties();
+            minifiProps.setProperty(STATUS_FILE_PID_KEY, String.valueOf(pid));
+            bootstrapFileProvider.saveStatusProperties(minifiProps);
+        }
+
+        shutdownHook = new ShutdownHook(runMiNiFi, miNiFiStdLogHandler);
+        runtime.addShutdownHook(shutdownHook);
+
+        boolean started = waitForStart();
+
+        if (started) {
+            DEFAULT_LOGGER.info("Successfully spawned the thread to start Apache MiNiFi{}", (pid == null ? "" : " with PID " + pid));
+        } else {
+            DEFAULT_LOGGER.error("Apache MiNiFi does not appear to have started");
+        }
+        return process;
+    }
+
+    private boolean needRestart() throws IOException {
+        boolean needRestart = true;
+        File statusFile = bootstrapFileProvider.getStatusFile();
+        if (!statusFile.exists()) {
+            DEFAULT_LOGGER.info("Status File no longer exists. Will not restart MiNiFi");
+            return false;
+        }
+
+        File lockFile = bootstrapFileProvider.getLockFile();
+        if (lockFile.exists()) {
+            DEFAULT_LOGGER.info("A shutdown was initiated. Will not restart MiNiFi");
+            return false;
+        }
+        return needRestart;
+    }
+
+    private void handleReload() {
+        try {
+            Thread.sleep(1000L);
+            if (runMiNiFi.getReloading() && runMiNiFi.isNiFiStarted()) {
+                File swapConfigFile = bootstrapFileProvider.getSwapFile();
+                if (swapConfigFile.exists()) {
+                    DEFAULT_LOGGER.info("MiNiFi has finished reloading successfully and swap file exists. Deleting old configuration.");
+
+                    if (swapConfigFile.delete()) {
+                        DEFAULT_LOGGER.info("Swap file was successfully deleted.");
+                    } else {
+                        DEFAULT_LOGGER.error("Swap file was not deleted. It should be deleted manually.");
+                    }
+                }
+                runMiNiFi.setReloading(false);
+            }
+        } catch (InterruptedException ie) {
+        }
+    }
+
+    private void initConfigFiles(Properties bootstrapProperties, String confDir) throws IOException {
+        File configFile = new File(bootstrapProperties.getProperty(MINIFI_CONFIG_FILE_KEY));
+        try (InputStream inputStream = new FileInputStream(configFile)) {
+            ByteBuffer tempConfigFile = generateConfigFiles(inputStream, confDir, bootstrapProperties);
+            runMiNiFi.getConfigFileReference().set(tempConfigFile.asReadOnlyBuffer());
+        } catch (FileNotFoundException e) {
+            String fileNotFoundMessage = "The config file defined in " + MINIFI_CONFIG_FILE_KEY + " does not exists.";
+            DEFAULT_LOGGER.error(fileNotFoundMessage, e);
+            throw new StartupFailureException(fileNotFoundMessage);
+        } catch (ConfigurationChangeException e) {
+            String malformedConfigFileMessage = "The config file is malformed, unable to start.";
+            DEFAULT_LOGGER.error(malformedConfigFileMessage, e);
+            throw new StartupFailureException(malformedConfigFileMessage);
+        }
+    }
+
+    private Tuple<ProcessBuilder, Process> startMiNiFi() throws IOException {
+        ProcessBuilder builder = new ProcessBuilder();
+
+        Properties props = bootstrapFileProvider.getBootstrapProperties();
+        File bootstrapConfigAbsoluteFile = bootstrapConfigFile.getAbsoluteFile();
+        File binDir = bootstrapConfigAbsoluteFile.getParentFile();
+
+        File workingDir = Optional.ofNullable(props.getProperty("working.dir"))
+            .map(File::new)
+            .orElse(binDir.getParentFile());
+
+        builder.directory(workingDir);
+
+        String minifiLogDir = System.getProperty("org.apache.nifi.minifi.bootstrap.config.log.dir", DEFAULT_LOG_DIR).trim();
+        File libDir = getFile(props.getProperty("lib.dir", DEFAULT_LIB_DIR).trim(), workingDir);
+        File confDir = getFile(props.getProperty(CONF_DIR_KEY, DEFAULT_CONF_DIR).trim(), workingDir);
+
+        String minifiPropsFilename = props.getProperty("props.file");
+        if (minifiPropsFilename == null) {
+            if (confDir.exists()) {
+                minifiPropsFilename = new File(confDir, "nifi.properties").getAbsolutePath();
+            } else {
+                minifiPropsFilename = DEFAULT_CONFIG_FILE;
+            }
+        }
+
+        minifiPropsFilename = minifiPropsFilename.trim();
+
+        List<String> javaAdditionalArgs = new ArrayList<>();
+        for (Entry<Object, Object> entry : props.entrySet()) {
+            String key = (String) entry.getKey();
+            String value = (String) entry.getValue();
+
+            if (key.startsWith("java.arg")) {
+                javaAdditionalArgs.add(value);
+            }
+        }
+
+        File[] libFiles = libDir.listFiles((dir, filename) -> filename.toLowerCase().endsWith(".jar"));
+
+        if (libFiles == null || libFiles.length == 0) {
+            throw new RuntimeException("Could not find lib directory at " + libDir.getAbsolutePath());
+        }
+
+        File[] confFiles = confDir.listFiles();
+        if (confFiles == null || confFiles.length == 0) {
+            throw new RuntimeException("Could not find conf directory at " + confDir.getAbsolutePath());
+        }
+
+        List<String> cpFiles = new ArrayList<>(confFiles.length + libFiles.length);
+        cpFiles.add(confDir.getAbsolutePath());
+        for (File file : libFiles) {
+            cpFiles.add(file.getAbsolutePath());
+        }
+
+        StringBuilder classPathBuilder = new StringBuilder();
+        for (int i = 0; i < cpFiles.size(); i++) {
+            String filename = cpFiles.get(i);
+            classPathBuilder.append(filename);
+            if (i < cpFiles.size() - 1) {
+                classPathBuilder.append(File.pathSeparatorChar);
+            }
+        }
+
+        String classPath = classPathBuilder.toString();
+        String javaCmd = props.getProperty("java");
+        if (javaCmd == null) {
+            javaCmd = DEFAULT_JAVA_CMD;
+        }
+        if (javaCmd.equals(DEFAULT_JAVA_CMD)) {
+            String javaHome = System.getenv("JAVA_HOME");
+            if (javaHome != null) {
+                String fileExtension = isWindows() ? ".exe" : "";

Review Comment:
   Instead of reading the System property, what about just checking for the existence of the `java.exe` file?



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/BootstrapCodec.java:
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.nifi.minifi.bootstrap.service;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.util.Arrays;
+import org.apache.nifi.minifi.bootstrap.RunMiNiFi;
+import org.apache.nifi.minifi.bootstrap.exception.InvalidCommandException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class BootstrapCodec {
+
+    private static final String TRUE = "true";
+    private static final String FALSE = "false";

Review Comment:
   These values could be replaced with direct references to Boolean.TRUE.toString() and Boolean.FALSE.toString()



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/BootstrapCodec.java:
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.nifi.minifi.bootstrap.service;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.util.Arrays;
+import org.apache.nifi.minifi.bootstrap.RunMiNiFi;
+import org.apache.nifi.minifi.bootstrap.exception.InvalidCommandException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class BootstrapCodec {
+
+    private static final String TRUE = "true";
+    private static final String FALSE = "false";
+
+    private final RunMiNiFi runner;
+    private final BufferedReader reader;
+    private final BufferedWriter writer;
+    private final Logger logger = LoggerFactory.getLogger(BootstrapCodec.class);
+
+    public BootstrapCodec(RunMiNiFi runner, InputStream in, OutputStream out) {
+        this.runner = runner;
+        this.reader = new BufferedReader(new InputStreamReader(in));
+        this.writer = new BufferedWriter(new OutputStreamWriter(out));
+    }
+
+    public void communicate() throws IOException {
+        String line = reader.readLine();
+        String[] splits = line.split(" ");
+        if (splits.length == 0) {
+            throw new IOException("Received invalid command from MiNiFi: " + line);
+        }
+
+        String cmd = splits[0];
+        String[] args;
+        if (splits.length == 1) {
+            args = new String[0];
+        } else {
+            args = Arrays.copyOfRange(splits, 1, splits.length);
+        }
+
+        try {
+            processRequest(cmd, args);
+        } catch (InvalidCommandException exception) {
+            throw new IOException("Received invalid command from MiNiFi: " + line + " : " + (exception.getMessage() == null ? "" : "Details: " + exception));

Review Comment:
   Recommend including the exception cause as a separate argument:
   ```suggestion
               throw new IOException("Received invalid command from MiNiFi: " + line, exception));
   ```



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/BootstrapFileProvider.java:
##########
@@ -0,0 +1,234 @@
+/*
+ * 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.nifi.minifi.bootstrap.service;
+
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.STATUS_FILE_PID_KEY;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.attribute.PosixFilePermission;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.util.file.FileUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class BootstrapFileProvider {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(BootstrapFileProvider.class);
+
+    private static final String MINIFI_PID_FILE_NAME = "minifi.pid";
+    private static final String MINIFI_STATUS_FILE_NAME = "minifi.status";
+    private static final String MINIFI_LOCK_FILE_NAME = "minifi.lock";
+    private static final String DEFAULT_CONFIG_FILE = "./conf/bootstrap.conf";
+    private static final String BOOTSTRAP_CONFIG_FILE_SYSTEM_PROPERTY_KEY = "org.apache.nifi.minifi.bootstrap.config.file";
+    private static final String MINIFI_HOME_ENV_VARIABLE_KEY = "MINIFI_HOME";
+    private static final String MINIFI_PID_DIR_PROP = "org.apache.nifi.minifi.bootstrap.config.pid.dir";
+    private static final String DEFAULT_PID_DIR = "bin";
+
+    private final File bootstrapConfigFile;
+
+    public BootstrapFileProvider(File bootstrapConfigFile) {
+        if (bootstrapConfigFile == null || !bootstrapConfigFile.exists()) {
+            throw new IllegalArgumentException("The specified bootstrap file doesn't exists: " + bootstrapConfigFile);
+        }
+        this.bootstrapConfigFile = bootstrapConfigFile;
+    }
+
+    public static File getBootstrapConfFile() {
+        File bootstrapConfigFile = Optional.ofNullable(System.getProperty(BOOTSTRAP_CONFIG_FILE_SYSTEM_PROPERTY_KEY))
+            .map(File::new)
+            .orElseGet(() -> Optional.ofNullable(System.getenv(MINIFI_HOME_ENV_VARIABLE_KEY))
+                .map(File::new)
+                .map(nifiHomeFile -> new File(nifiHomeFile, DEFAULT_CONFIG_FILE))
+                .orElseGet(() -> new File(DEFAULT_CONFIG_FILE)));
+        LOGGER.debug("Bootstrap config file: {}", bootstrapConfigFile);
+        return bootstrapConfigFile;
+    }
+
+    public File getPidFile() throws IOException {
+        return getBootstrapFile(MINIFI_PID_FILE_NAME);
+    }
+
+    public File getStatusFile() throws IOException {
+        return getBootstrapFile(MINIFI_STATUS_FILE_NAME);
+    }
+
+    public File getLockFile() throws IOException {
+        return getBootstrapFile(MINIFI_LOCK_FILE_NAME);
+    }
+
+    public File getReloadLockFile() {
+        File confDir = bootstrapConfigFile.getParentFile();
+        File nifiHome = confDir.getParentFile();
+        File bin = new File(nifiHome, "bin");
+        File reloadFile = new File(bin, "minifi.reload.lock");
+
+        LOGGER.debug("Reload File: {}", reloadFile);
+        return reloadFile;
+    }
+
+    public File getSwapFile() {
+        File confDir = bootstrapConfigFile.getParentFile();
+        File swapFile = new File(confDir, "swap.yml");
+
+        LOGGER.debug("Swap File: {}", swapFile);
+        return swapFile;
+    }
+
+    public Properties getBootstrapProperties() throws IOException {
+        if (!bootstrapConfigFile.exists()) {
+            throw new FileNotFoundException(bootstrapConfigFile.getAbsolutePath());
+        }
+
+        Properties bootstrapProperties = new Properties();
+        try (FileInputStream fis = new FileInputStream(bootstrapConfigFile)) {
+            bootstrapProperties.load(fis);
+        }
+
+        logProperties("Bootstrap", bootstrapProperties);

Review Comment:
   Can any of these bootstrap properties contain sensitive values? If so, they should not be logged.



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/command/StartRunner.java:
##########
@@ -0,0 +1,435 @@
+/*
+ * 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.nifi.minifi.bootstrap.command;
+
+import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CMD_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CONF_DIR_KEY;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.DEFAULT_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.MINIFI_CONFIG_FILE_KEY;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.STATUS_FILE_PID_KEY;
+import static org.apache.nifi.minifi.bootstrap.Status.ERROR;
+import static org.apache.nifi.minifi.bootstrap.Status.OK;
+import static org.apache.nifi.minifi.bootstrap.util.ConfigTransformer.generateConfigFiles;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.nifi.bootstrap.util.OSUtils;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.apache.nifi.minifi.bootstrap.RunMiNiFi;
+import org.apache.nifi.minifi.bootstrap.ShutdownHook;
+import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeException;
+import org.apache.nifi.minifi.bootstrap.exception.StartupFailureException;
+import org.apache.nifi.minifi.bootstrap.service.BootstrapFileProvider;
+import org.apache.nifi.minifi.bootstrap.service.CurrentPortProvider;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiListener;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiStdLogHandler;
+import org.apache.nifi.minifi.bootstrap.service.PeriodicStatusReporterManager;
+import org.apache.nifi.minifi.bootstrap.util.ProcessUtils;
+import org.apache.nifi.util.Tuple;
+
+public class StartRunner implements CommandRunner {
+    private static final String DEFAULT_JAVA_CMD = "java";
+    private static final String DEFAULT_LOG_DIR = "./logs";
+    private static final String DEFAULT_LIB_DIR = "./lib";
+    private static final String DEFAULT_CONF_DIR = "./conf";
+    private static final int STARTUP_WAIT_SECONDS = 60;
+    private static final String DEFAULT_CONFIG_FILE = DEFAULT_CONF_DIR + "/bootstrap.conf";
+
+    private final CurrentPortProvider currentPortProvider;
+    private final BootstrapFileProvider bootstrapFileProvider;
+    private final PeriodicStatusReporterManager periodicStatusReporterManager;
+    private final MiNiFiStdLogHandler miNiFiStdLogHandler;
+    private final MiNiFiParameters miNiFiParameters;
+    private final File bootstrapConfigFile;
+    private final Lock lock = new ReentrantLock();
+    private final Condition startupCondition = lock.newCondition();
+    private final RunMiNiFi runMiNiFi;
+    private volatile ShutdownHook shutdownHook;
+
+    public StartRunner(CurrentPortProvider currentPortProvider, BootstrapFileProvider bootstrapFileProvider,
+        PeriodicStatusReporterManager periodicStatusReporterManager, MiNiFiStdLogHandler miNiFiStdLogHandler, MiNiFiParameters miNiFiParameters, File bootstrapConfigFile,
+        RunMiNiFi runMiNiFi) {
+        this.currentPortProvider = currentPortProvider;
+        this.bootstrapFileProvider = bootstrapFileProvider;
+        this.periodicStatusReporterManager = periodicStatusReporterManager;
+        this.miNiFiStdLogHandler = miNiFiStdLogHandler;
+        this.miNiFiParameters = miNiFiParameters;
+        this.bootstrapConfigFile = bootstrapConfigFile;
+        this.runMiNiFi = runMiNiFi;
+    }
+
+    /**
+     * Starts (and restarts) MiNiFi process during the whole lifecycle of the bootstrap process.
+     * @param args the input arguments
+     * @return status code
+     */
+    @Override
+    public int runCommand(String[] args) {
+        try {
+            start();
+        } catch (Exception e) {
+            CMD_LOGGER.error("Exception happened during MiNiFi startup", e);
+            return ERROR.getStatusCode();
+        }
+        return OK.getStatusCode();
+    }
+
+    private void start() throws IOException, InterruptedException {
+        Integer port = currentPortProvider.getCurrentPort();
+        if (port != null) {
+            String alreadyRunningMessage = "Apache MiNiFi is already running";
+            CMD_LOGGER.info(alreadyRunningMessage + ", listening to Bootstrap on port {}", port);
+            return;
+        }
+
+        File prevLockFile = bootstrapFileProvider.getLockFile();
+        if (prevLockFile.exists() && !prevLockFile.delete()) {
+            CMD_LOGGER.warn("Failed to delete previous lock file {}; this file should be cleaned up manually", prevLockFile);
+        }
+
+        Properties bootstrapProperties = bootstrapFileProvider.getBootstrapProperties();
+        String confDir = bootstrapProperties.getProperty(CONF_DIR_KEY);
+        initConfigFiles(bootstrapProperties, confDir);
+
+        Tuple<ProcessBuilder, Process> tuple = startMiNiFi();
+        ProcessBuilder builder = tuple.getKey();
+        Process process = tuple.getValue();
+
+        try {
+            while (true) {
+                if (ProcessUtils.isAlive(process)) {
+                    handleReload();
+                } else {
+                    Runtime runtime = Runtime.getRuntime();
+                    try {
+                        runtime.removeShutdownHook(shutdownHook);
+                    } catch (IllegalStateException ise) {
+                        // happens when already shutting down
+                    }
+
+                    if (runMiNiFi.isAutoRestartNiFi() && needRestart()) {
+                        File reloadFile = bootstrapFileProvider.getReloadLockFile();
+                        if (reloadFile.exists()) {
+                            DEFAULT_LOGGER.info("Currently reloading configuration. Will wait to restart MiNiFi.");
+                            Thread.sleep(5000L);
+                            continue;
+                        }
+
+                        process = restartNifi(bootstrapProperties, confDir, builder, runtime);
+                        // failed to start process
+                        if (process == null) {
+                            return;
+                        }
+                    } else {
+                        return;
+                    }
+                }
+            }
+        } finally {
+            miNiFiStdLogHandler.shutdown();
+            runMiNiFi.shutdownChangeNotifier();
+            periodicStatusReporterManager.shutdownPeriodicStatusReporters();
+        }
+    }
+
+    private Process restartNifi(Properties bootstrapProperties, String confDir, ProcessBuilder builder, Runtime runtime) throws IOException {
+        Process process;
+        boolean previouslyStarted = runMiNiFi.isNiFiStarted();
+        if (!previouslyStarted) {
+            File swapConfigFile = bootstrapFileProvider.getSwapFile();
+            if (swapConfigFile.exists()) {
+                DEFAULT_LOGGER.info("Swap file exists, MiNiFi failed trying to change configuration. Reverting to old configuration.");
+
+                try {
+                    ByteBuffer tempConfigFile = generateConfigFiles(new FileInputStream(swapConfigFile), confDir, bootstrapProperties);
+                    runMiNiFi.getConfigFileReference().set(tempConfigFile.asReadOnlyBuffer());
+                } catch (ConfigurationChangeException e) {
+                    DEFAULT_LOGGER.error("The swap file is malformed, unable to restart from prior state. Will not attempt to restart MiNiFi. Swap File should be cleaned up manually.");
+                    return null;
+                }
+
+                Files.copy(swapConfigFile.toPath(), Paths.get(bootstrapProperties.getProperty(MINIFI_CONFIG_FILE_KEY)), REPLACE_EXISTING);
+
+                DEFAULT_LOGGER.info("Replacing config file with swap file and deleting swap file");
+                if (!swapConfigFile.delete()) {
+                    DEFAULT_LOGGER.warn("The swap file failed to delete after replacing using it to revert to the old configuration. It should be cleaned up manually.");
+                }
+                runMiNiFi.setReloading(false);
+            } else {
+                DEFAULT_LOGGER.info("MiNiFi either never started or failed to restart. Will not attempt to restart MiNiFi");
+                return null;
+            }
+        } else {
+            runMiNiFi.setNiFiStarted(false);
+        }
+
+        miNiFiParameters.setSecretKey(null);
+        process = builder.start();
+        miNiFiStdLogHandler.initLogging(process);
+
+        Long pid = OSUtils.getProcessId(process, DEFAULT_LOGGER);
+        if (pid != null) {
+            miNiFiParameters.setMinifiPid(pid);
+            Properties minifiProps = new Properties();
+            minifiProps.setProperty(STATUS_FILE_PID_KEY, String.valueOf(pid));
+            bootstrapFileProvider.saveStatusProperties(minifiProps);
+        }
+
+        shutdownHook = new ShutdownHook(runMiNiFi, miNiFiStdLogHandler);
+        runtime.addShutdownHook(shutdownHook);
+
+        boolean started = waitForStart();
+
+        if (started) {
+            DEFAULT_LOGGER.info("Successfully spawned the thread to start Apache MiNiFi{}", (pid == null ? "" : " with PID " + pid));
+        } else {
+            DEFAULT_LOGGER.error("Apache MiNiFi does not appear to have started");
+        }
+        return process;
+    }
+
+    private boolean needRestart() throws IOException {
+        boolean needRestart = true;
+        File statusFile = bootstrapFileProvider.getStatusFile();
+        if (!statusFile.exists()) {
+            DEFAULT_LOGGER.info("Status File no longer exists. Will not restart MiNiFi");
+            return false;
+        }
+
+        File lockFile = bootstrapFileProvider.getLockFile();
+        if (lockFile.exists()) {
+            DEFAULT_LOGGER.info("A shutdown was initiated. Will not restart MiNiFi");
+            return false;
+        }
+        return needRestart;
+    }
+
+    private void handleReload() {
+        try {
+            Thread.sleep(1000L);
+            if (runMiNiFi.getReloading() && runMiNiFi.isNiFiStarted()) {
+                File swapConfigFile = bootstrapFileProvider.getSwapFile();
+                if (swapConfigFile.exists()) {
+                    DEFAULT_LOGGER.info("MiNiFi has finished reloading successfully and swap file exists. Deleting old configuration.");
+
+                    if (swapConfigFile.delete()) {
+                        DEFAULT_LOGGER.info("Swap file was successfully deleted.");
+                    } else {
+                        DEFAULT_LOGGER.error("Swap file was not deleted. It should be deleted manually.");
+                    }
+                }
+                runMiNiFi.setReloading(false);
+            }
+        } catch (InterruptedException ie) {
+        }
+    }
+
+    private void initConfigFiles(Properties bootstrapProperties, String confDir) throws IOException {
+        File configFile = new File(bootstrapProperties.getProperty(MINIFI_CONFIG_FILE_KEY));
+        try (InputStream inputStream = new FileInputStream(configFile)) {
+            ByteBuffer tempConfigFile = generateConfigFiles(inputStream, confDir, bootstrapProperties);
+            runMiNiFi.getConfigFileReference().set(tempConfigFile.asReadOnlyBuffer());
+        } catch (FileNotFoundException e) {
+            String fileNotFoundMessage = "The config file defined in " + MINIFI_CONFIG_FILE_KEY + " does not exists.";
+            DEFAULT_LOGGER.error(fileNotFoundMessage, e);
+            throw new StartupFailureException(fileNotFoundMessage);
+        } catch (ConfigurationChangeException e) {
+            String malformedConfigFileMessage = "The config file is malformed, unable to start.";
+            DEFAULT_LOGGER.error(malformedConfigFileMessage, e);
+            throw new StartupFailureException(malformedConfigFileMessage);
+        }
+    }
+
+    private Tuple<ProcessBuilder, Process> startMiNiFi() throws IOException {

Review Comment:
   This is a fairly large method. It looks like the command building could be refactored out to a separate class, which would make the approach easier to maintain.



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/configuration/ConfigurationChangeCoordinator.java:
##########
@@ -99,16 +84,43 @@ public Collection<ListenerHandleResult> notifyListeners(ByteBuffer newConfig) {
                 result = new ListenerHandleResult(listener, ex);
             }
             listenerHandleResults.add(result);
-            logger.info("Listener notification result:" + result.toString());
+            LOGGER.info("Listener notification result: {}", result);
         }
         return listenerHandleResults;
     }
 
 
     @Override
-    public void close() throws IOException {
-        for (ChangeIngestor changeIngestor : changeIngestors) {
-            changeIngestor.close();
+    public void close() {
+        try {
+            for (ChangeIngestor changeIngestor : changeIngestors) {
+                changeIngestor.close();
+            }
+            changeIngestors.clear();
+        } catch (IOException e) {
+            LOGGER.warn("Could not successfully stop notifiers", e);
+        }
+    }
+
+    private void initialize() {
+        close();
+        // cleanup previously initialized ingestors
+        String ingestorsCsv = bootstrapProperties.getProperty(NOTIFIER_INGESTORS_KEY);
+
+        if (ingestorsCsv != null && !ingestorsCsv.isEmpty()) {
+            for (String ingestorClassname : ingestorsCsv.split(",")) {
+                ingestorClassname = ingestorClassname.trim();
+                try {
+                    Class<?> ingestorClass = Class.forName(ingestorClassname);
+                    ChangeIngestor changeIngestor = (ChangeIngestor) ingestorClass.newInstance();
+                    changeIngestor.initialize(bootstrapProperties, runMiNiFi, this);
+                    changeIngestors.add(changeIngestor);
+                    LOGGER.info("Initialized ingestor: {}", ingestorClassname);
+                } catch (Exception e) {
+                    LOGGER.error("Issue instantiating {} ingestor", ingestorClassname);
+                    LOGGER.error("Exception", e);

Review Comment:
   Recommend consolidating these logs to a single invocation.
   ```suggestion
                       LOGGER.error("Instantiating [{}] ingestor failed", ingestorClassname, e);
   ```



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/command/StartRunner.java:
##########
@@ -0,0 +1,435 @@
+/*
+ * 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.nifi.minifi.bootstrap.command;
+
+import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CMD_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CONF_DIR_KEY;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.DEFAULT_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.MINIFI_CONFIG_FILE_KEY;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.STATUS_FILE_PID_KEY;
+import static org.apache.nifi.minifi.bootstrap.Status.ERROR;
+import static org.apache.nifi.minifi.bootstrap.Status.OK;
+import static org.apache.nifi.minifi.bootstrap.util.ConfigTransformer.generateConfigFiles;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.nifi.bootstrap.util.OSUtils;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.apache.nifi.minifi.bootstrap.RunMiNiFi;
+import org.apache.nifi.minifi.bootstrap.ShutdownHook;
+import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeException;
+import org.apache.nifi.minifi.bootstrap.exception.StartupFailureException;
+import org.apache.nifi.minifi.bootstrap.service.BootstrapFileProvider;
+import org.apache.nifi.minifi.bootstrap.service.CurrentPortProvider;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiListener;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiStdLogHandler;
+import org.apache.nifi.minifi.bootstrap.service.PeriodicStatusReporterManager;
+import org.apache.nifi.minifi.bootstrap.util.ProcessUtils;
+import org.apache.nifi.util.Tuple;
+
+public class StartRunner implements CommandRunner {
+    private static final String DEFAULT_JAVA_CMD = "java";
+    private static final String DEFAULT_LOG_DIR = "./logs";
+    private static final String DEFAULT_LIB_DIR = "./lib";
+    private static final String DEFAULT_CONF_DIR = "./conf";
+    private static final int STARTUP_WAIT_SECONDS = 60;
+    private static final String DEFAULT_CONFIG_FILE = DEFAULT_CONF_DIR + "/bootstrap.conf";
+
+    private final CurrentPortProvider currentPortProvider;
+    private final BootstrapFileProvider bootstrapFileProvider;
+    private final PeriodicStatusReporterManager periodicStatusReporterManager;
+    private final MiNiFiStdLogHandler miNiFiStdLogHandler;
+    private final MiNiFiParameters miNiFiParameters;
+    private final File bootstrapConfigFile;
+    private final Lock lock = new ReentrantLock();
+    private final Condition startupCondition = lock.newCondition();
+    private final RunMiNiFi runMiNiFi;
+    private volatile ShutdownHook shutdownHook;
+
+    public StartRunner(CurrentPortProvider currentPortProvider, BootstrapFileProvider bootstrapFileProvider,
+        PeriodicStatusReporterManager periodicStatusReporterManager, MiNiFiStdLogHandler miNiFiStdLogHandler, MiNiFiParameters miNiFiParameters, File bootstrapConfigFile,
+        RunMiNiFi runMiNiFi) {
+        this.currentPortProvider = currentPortProvider;
+        this.bootstrapFileProvider = bootstrapFileProvider;
+        this.periodicStatusReporterManager = periodicStatusReporterManager;
+        this.miNiFiStdLogHandler = miNiFiStdLogHandler;
+        this.miNiFiParameters = miNiFiParameters;
+        this.bootstrapConfigFile = bootstrapConfigFile;
+        this.runMiNiFi = runMiNiFi;
+    }
+
+    /**
+     * Starts (and restarts) MiNiFi process during the whole lifecycle of the bootstrap process.
+     * @param args the input arguments
+     * @return status code
+     */
+    @Override
+    public int runCommand(String[] args) {
+        try {
+            start();
+        } catch (Exception e) {
+            CMD_LOGGER.error("Exception happened during MiNiFi startup", e);
+            return ERROR.getStatusCode();
+        }
+        return OK.getStatusCode();
+    }
+
+    private void start() throws IOException, InterruptedException {
+        Integer port = currentPortProvider.getCurrentPort();
+        if (port != null) {
+            String alreadyRunningMessage = "Apache MiNiFi is already running";
+            CMD_LOGGER.info(alreadyRunningMessage + ", listening to Bootstrap on port {}", port);
+            return;
+        }
+
+        File prevLockFile = bootstrapFileProvider.getLockFile();
+        if (prevLockFile.exists() && !prevLockFile.delete()) {
+            CMD_LOGGER.warn("Failed to delete previous lock file {}; this file should be cleaned up manually", prevLockFile);
+        }
+
+        Properties bootstrapProperties = bootstrapFileProvider.getBootstrapProperties();
+        String confDir = bootstrapProperties.getProperty(CONF_DIR_KEY);
+        initConfigFiles(bootstrapProperties, confDir);
+
+        Tuple<ProcessBuilder, Process> tuple = startMiNiFi();
+        ProcessBuilder builder = tuple.getKey();
+        Process process = tuple.getValue();
+
+        try {
+            while (true) {
+                if (ProcessUtils.isAlive(process)) {
+                    handleReload();
+                } else {
+                    Runtime runtime = Runtime.getRuntime();
+                    try {
+                        runtime.removeShutdownHook(shutdownHook);
+                    } catch (IllegalStateException ise) {
+                        // happens when already shutting down
+                    }
+
+                    if (runMiNiFi.isAutoRestartNiFi() && needRestart()) {
+                        File reloadFile = bootstrapFileProvider.getReloadLockFile();
+                        if (reloadFile.exists()) {
+                            DEFAULT_LOGGER.info("Currently reloading configuration. Will wait to restart MiNiFi.");
+                            Thread.sleep(5000L);
+                            continue;
+                        }
+
+                        process = restartNifi(bootstrapProperties, confDir, builder, runtime);
+                        // failed to start process
+                        if (process == null) {
+                            return;
+                        }
+                    } else {
+                        return;
+                    }
+                }
+            }
+        } finally {
+            miNiFiStdLogHandler.shutdown();
+            runMiNiFi.shutdownChangeNotifier();
+            periodicStatusReporterManager.shutdownPeriodicStatusReporters();
+        }
+    }
+
+    private Process restartNifi(Properties bootstrapProperties, String confDir, ProcessBuilder builder, Runtime runtime) throws IOException {
+        Process process;
+        boolean previouslyStarted = runMiNiFi.isNiFiStarted();
+        if (!previouslyStarted) {
+            File swapConfigFile = bootstrapFileProvider.getSwapFile();
+            if (swapConfigFile.exists()) {
+                DEFAULT_LOGGER.info("Swap file exists, MiNiFi failed trying to change configuration. Reverting to old configuration.");
+
+                try {
+                    ByteBuffer tempConfigFile = generateConfigFiles(new FileInputStream(swapConfigFile), confDir, bootstrapProperties);
+                    runMiNiFi.getConfigFileReference().set(tempConfigFile.asReadOnlyBuffer());
+                } catch (ConfigurationChangeException e) {
+                    DEFAULT_LOGGER.error("The swap file is malformed, unable to restart from prior state. Will not attempt to restart MiNiFi. Swap File should be cleaned up manually.");
+                    return null;
+                }
+
+                Files.copy(swapConfigFile.toPath(), Paths.get(bootstrapProperties.getProperty(MINIFI_CONFIG_FILE_KEY)), REPLACE_EXISTING);
+
+                DEFAULT_LOGGER.info("Replacing config file with swap file and deleting swap file");
+                if (!swapConfigFile.delete()) {
+                    DEFAULT_LOGGER.warn("The swap file failed to delete after replacing using it to revert to the old configuration. It should be cleaned up manually.");
+                }
+                runMiNiFi.setReloading(false);
+            } else {
+                DEFAULT_LOGGER.info("MiNiFi either never started or failed to restart. Will not attempt to restart MiNiFi");
+                return null;
+            }
+        } else {
+            runMiNiFi.setNiFiStarted(false);
+        }
+
+        miNiFiParameters.setSecretKey(null);
+        process = builder.start();
+        miNiFiStdLogHandler.initLogging(process);
+
+        Long pid = OSUtils.getProcessId(process, DEFAULT_LOGGER);
+        if (pid != null) {
+            miNiFiParameters.setMinifiPid(pid);
+            Properties minifiProps = new Properties();
+            minifiProps.setProperty(STATUS_FILE_PID_KEY, String.valueOf(pid));
+            bootstrapFileProvider.saveStatusProperties(minifiProps);
+        }
+
+        shutdownHook = new ShutdownHook(runMiNiFi, miNiFiStdLogHandler);
+        runtime.addShutdownHook(shutdownHook);
+
+        boolean started = waitForStart();
+
+        if (started) {
+            DEFAULT_LOGGER.info("Successfully spawned the thread to start Apache MiNiFi{}", (pid == null ? "" : " with PID " + pid));
+        } else {
+            DEFAULT_LOGGER.error("Apache MiNiFi does not appear to have started");
+        }
+        return process;
+    }
+
+    private boolean needRestart() throws IOException {
+        boolean needRestart = true;
+        File statusFile = bootstrapFileProvider.getStatusFile();
+        if (!statusFile.exists()) {
+            DEFAULT_LOGGER.info("Status File no longer exists. Will not restart MiNiFi");
+            return false;
+        }
+
+        File lockFile = bootstrapFileProvider.getLockFile();
+        if (lockFile.exists()) {
+            DEFAULT_LOGGER.info("A shutdown was initiated. Will not restart MiNiFi");
+            return false;
+        }
+        return needRestart;
+    }
+
+    private void handleReload() {
+        try {
+            Thread.sleep(1000L);
+            if (runMiNiFi.getReloading() && runMiNiFi.isNiFiStarted()) {
+                File swapConfigFile = bootstrapFileProvider.getSwapFile();
+                if (swapConfigFile.exists()) {
+                    DEFAULT_LOGGER.info("MiNiFi has finished reloading successfully and swap file exists. Deleting old configuration.");
+
+                    if (swapConfigFile.delete()) {
+                        DEFAULT_LOGGER.info("Swap file was successfully deleted.");
+                    } else {
+                        DEFAULT_LOGGER.error("Swap file was not deleted. It should be deleted manually.");
+                    }
+                }
+                runMiNiFi.setReloading(false);
+            }
+        } catch (InterruptedException ie) {
+        }
+    }
+
+    private void initConfigFiles(Properties bootstrapProperties, String confDir) throws IOException {
+        File configFile = new File(bootstrapProperties.getProperty(MINIFI_CONFIG_FILE_KEY));
+        try (InputStream inputStream = new FileInputStream(configFile)) {
+            ByteBuffer tempConfigFile = generateConfigFiles(inputStream, confDir, bootstrapProperties);
+            runMiNiFi.getConfigFileReference().set(tempConfigFile.asReadOnlyBuffer());
+        } catch (FileNotFoundException e) {
+            String fileNotFoundMessage = "The config file defined in " + MINIFI_CONFIG_FILE_KEY + " does not exists.";
+            DEFAULT_LOGGER.error(fileNotFoundMessage, e);
+            throw new StartupFailureException(fileNotFoundMessage);
+        } catch (ConfigurationChangeException e) {
+            String malformedConfigFileMessage = "The config file is malformed, unable to start.";
+            DEFAULT_LOGGER.error(malformedConfigFileMessage, e);
+            throw new StartupFailureException(malformedConfigFileMessage);
+        }
+    }
+
+    private Tuple<ProcessBuilder, Process> startMiNiFi() throws IOException {
+        ProcessBuilder builder = new ProcessBuilder();
+
+        Properties props = bootstrapFileProvider.getBootstrapProperties();
+        File bootstrapConfigAbsoluteFile = bootstrapConfigFile.getAbsoluteFile();
+        File binDir = bootstrapConfigAbsoluteFile.getParentFile();
+
+        File workingDir = Optional.ofNullable(props.getProperty("working.dir"))
+            .map(File::new)
+            .orElse(binDir.getParentFile());
+
+        builder.directory(workingDir);
+
+        String minifiLogDir = System.getProperty("org.apache.nifi.minifi.bootstrap.config.log.dir", DEFAULT_LOG_DIR).trim();
+        File libDir = getFile(props.getProperty("lib.dir", DEFAULT_LIB_DIR).trim(), workingDir);
+        File confDir = getFile(props.getProperty(CONF_DIR_KEY, DEFAULT_CONF_DIR).trim(), workingDir);
+
+        String minifiPropsFilename = props.getProperty("props.file");
+        if (minifiPropsFilename == null) {
+            if (confDir.exists()) {
+                minifiPropsFilename = new File(confDir, "nifi.properties").getAbsolutePath();
+            } else {
+                minifiPropsFilename = DEFAULT_CONFIG_FILE;
+            }
+        }
+
+        minifiPropsFilename = minifiPropsFilename.trim();
+
+        List<String> javaAdditionalArgs = new ArrayList<>();
+        for (Entry<Object, Object> entry : props.entrySet()) {
+            String key = (String) entry.getKey();
+            String value = (String) entry.getValue();
+
+            if (key.startsWith("java.arg")) {
+                javaAdditionalArgs.add(value);
+            }
+        }
+
+        File[] libFiles = libDir.listFiles((dir, filename) -> filename.toLowerCase().endsWith(".jar"));
+
+        if (libFiles == null || libFiles.length == 0) {
+            throw new RuntimeException("Could not find lib directory at " + libDir.getAbsolutePath());
+        }
+
+        File[] confFiles = confDir.listFiles();
+        if (confFiles == null || confFiles.length == 0) {
+            throw new RuntimeException("Could not find conf directory at " + confDir.getAbsolutePath());
+        }
+
+        List<String> cpFiles = new ArrayList<>(confFiles.length + libFiles.length);
+        cpFiles.add(confDir.getAbsolutePath());
+        for (File file : libFiles) {
+            cpFiles.add(file.getAbsolutePath());
+        }
+
+        StringBuilder classPathBuilder = new StringBuilder();
+        for (int i = 0; i < cpFiles.size(); i++) {
+            String filename = cpFiles.get(i);
+            classPathBuilder.append(filename);
+            if (i < cpFiles.size() - 1) {
+                classPathBuilder.append(File.pathSeparatorChar);
+            }
+        }
+
+        String classPath = classPathBuilder.toString();
+        String javaCmd = props.getProperty("java");
+        if (javaCmd == null) {
+            javaCmd = DEFAULT_JAVA_CMD;
+        }
+        if (javaCmd.equals(DEFAULT_JAVA_CMD)) {
+            String javaHome = System.getenv("JAVA_HOME");
+            if (javaHome != null) {
+                String fileExtension = isWindows() ? ".exe" : "";
+                File javaFile = new File(javaHome + File.separatorChar + "bin"
+                    + File.separatorChar + "java" + fileExtension);
+                if (javaFile.exists() && javaFile.canExecute()) {
+                    javaCmd = javaFile.getAbsolutePath();
+                }
+            }
+        }
+
+        MiNiFiListener listener = new MiNiFiListener();
+        int listenPort = listener.start(runMiNiFi);
+
+        List<String> cmd = new ArrayList<>();
+
+        cmd.add(javaCmd);
+        cmd.add("-classpath");
+        cmd.add(classPath);
+        cmd.addAll(javaAdditionalArgs);
+        cmd.add("-Dnifi.properties.file.path=" + minifiPropsFilename);
+        cmd.add("-Dnifi.bootstrap.listen.port=" + listenPort);
+        cmd.add("-Dapp=MiNiFi");
+        cmd.add("-Dorg.apache.nifi.minifi.bootstrap.config.log.dir="+minifiLogDir);

Review Comment:
   ```suggestion
           cmd.add("-Dorg.apache.nifi.minifi.bootstrap.config.log.dir=" + minifiLogDir);
   ```



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/BootstrapFileProvider.java:
##########
@@ -0,0 +1,234 @@
+/*
+ * 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.nifi.minifi.bootstrap.service;
+
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.STATUS_FILE_PID_KEY;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.attribute.PosixFilePermission;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.util.file.FileUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class BootstrapFileProvider {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(BootstrapFileProvider.class);
+
+    private static final String MINIFI_PID_FILE_NAME = "minifi.pid";
+    private static final String MINIFI_STATUS_FILE_NAME = "minifi.status";
+    private static final String MINIFI_LOCK_FILE_NAME = "minifi.lock";
+    private static final String DEFAULT_CONFIG_FILE = "./conf/bootstrap.conf";
+    private static final String BOOTSTRAP_CONFIG_FILE_SYSTEM_PROPERTY_KEY = "org.apache.nifi.minifi.bootstrap.config.file";
+    private static final String MINIFI_HOME_ENV_VARIABLE_KEY = "MINIFI_HOME";
+    private static final String MINIFI_PID_DIR_PROP = "org.apache.nifi.minifi.bootstrap.config.pid.dir";
+    private static final String DEFAULT_PID_DIR = "bin";
+
+    private final File bootstrapConfigFile;
+
+    public BootstrapFileProvider(File bootstrapConfigFile) {
+        if (bootstrapConfigFile == null || !bootstrapConfigFile.exists()) {
+            throw new IllegalArgumentException("The specified bootstrap file doesn't exists: " + bootstrapConfigFile);
+        }
+        this.bootstrapConfigFile = bootstrapConfigFile;
+    }
+
+    public static File getBootstrapConfFile() {
+        File bootstrapConfigFile = Optional.ofNullable(System.getProperty(BOOTSTRAP_CONFIG_FILE_SYSTEM_PROPERTY_KEY))
+            .map(File::new)
+            .orElseGet(() -> Optional.ofNullable(System.getenv(MINIFI_HOME_ENV_VARIABLE_KEY))
+                .map(File::new)
+                .map(nifiHomeFile -> new File(nifiHomeFile, DEFAULT_CONFIG_FILE))
+                .orElseGet(() -> new File(DEFAULT_CONFIG_FILE)));
+        LOGGER.debug("Bootstrap config file: {}", bootstrapConfigFile);
+        return bootstrapConfigFile;
+    }
+
+    public File getPidFile() throws IOException {
+        return getBootstrapFile(MINIFI_PID_FILE_NAME);
+    }
+
+    public File getStatusFile() throws IOException {
+        return getBootstrapFile(MINIFI_STATUS_FILE_NAME);
+    }
+
+    public File getLockFile() throws IOException {
+        return getBootstrapFile(MINIFI_LOCK_FILE_NAME);
+    }
+
+    public File getReloadLockFile() {
+        File confDir = bootstrapConfigFile.getParentFile();
+        File nifiHome = confDir.getParentFile();
+        File bin = new File(nifiHome, "bin");
+        File reloadFile = new File(bin, "minifi.reload.lock");
+
+        LOGGER.debug("Reload File: {}", reloadFile);
+        return reloadFile;
+    }
+
+    public File getSwapFile() {
+        File confDir = bootstrapConfigFile.getParentFile();
+        File swapFile = new File(confDir, "swap.yml");
+
+        LOGGER.debug("Swap File: {}", swapFile);
+        return swapFile;
+    }
+
+    public Properties getBootstrapProperties() throws IOException {
+        if (!bootstrapConfigFile.exists()) {
+            throw new FileNotFoundException(bootstrapConfigFile.getAbsolutePath());
+        }
+
+        Properties bootstrapProperties = new Properties();
+        try (FileInputStream fis = new FileInputStream(bootstrapConfigFile)) {
+            bootstrapProperties.load(fis);
+        }
+
+        logProperties("Bootstrap", bootstrapProperties);
+
+        return bootstrapProperties;
+    }
+
+    public Properties getStatusProperties() {
+        Properties props = new Properties();
+
+        try {
+            File statusFile = getStatusFile();
+            if (statusFile == null || !statusFile.exists()) {
+                LOGGER.debug("No status file to load properties from");
+                return props;
+            }
+
+            try (FileInputStream fis = new FileInputStream(statusFile)) {
+                props.load(fis);
+            }
+        } catch (IOException exception) {
+            LOGGER.error("Failed to load MiNiFi status properties");
+        }
+
+        logProperties("MiNiFi status", props);

Review Comment:
   It looks like this can include the secret key, so logging should be avoided.



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/MiNiFiCommandSender.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 org.apache.nifi.minifi.bootstrap.service;
+
+import java.io.BufferedReader;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.ObjectInputStream;
+import java.io.OutputStream;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.net.SocketTimeoutException;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MiNiFiCommandSender {
+    private static final Logger LOGGER = LoggerFactory.getLogger(MiNiFiCommandSender.class);
+    private static final String PING_CMD = "PING";
+
+    private final MiNiFiParameters miNiFiParameters;
+
+    public MiNiFiCommandSender(MiNiFiParameters miNiFiParameters) {
+        this.miNiFiParameters = miNiFiParameters;
+    }
+
+    public Optional<String> sendCommand(String cmd, Integer port) throws IOException {
+        Optional<String> response = Optional.empty();
+
+        if (port == null) {
+            LOGGER.info("Apache MiNiFi is not currently running");
+            return response;
+        }
+
+        try (Socket socket = new Socket()) {
+            LOGGER.debug("Connecting to MiNiFi instance");
+            socket.setSoTimeout(10000);
+            socket.connect(new InetSocketAddress("localhost", port));
+            LOGGER.debug("Established connection to MiNiFi instance.");
+
+            LOGGER.debug("Sending {} Command to port {}", cmd, port);
+            OutputStream out = socket.getOutputStream();
+            out.write((cmd + " " + miNiFiParameters.getSecretKey() + "\n").getBytes(StandardCharsets.UTF_8));

Review Comment:
   With several uses of this approach for command formatting, recommend creating a separate method that encapsulates the string formatting and serialization to a byte array to ensure a consistent approach.



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/MiNiFiCommandSender.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 org.apache.nifi.minifi.bootstrap.service;
+
+import java.io.BufferedReader;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.ObjectInputStream;
+import java.io.OutputStream;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.net.SocketTimeoutException;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MiNiFiCommandSender {
+    private static final Logger LOGGER = LoggerFactory.getLogger(MiNiFiCommandSender.class);
+    private static final String PING_CMD = "PING";
+
+    private final MiNiFiParameters miNiFiParameters;
+
+    public MiNiFiCommandSender(MiNiFiParameters miNiFiParameters) {
+        this.miNiFiParameters = miNiFiParameters;
+    }
+
+    public Optional<String> sendCommand(String cmd, Integer port) throws IOException {
+        Optional<String> response = Optional.empty();
+
+        if (port == null) {
+            LOGGER.info("Apache MiNiFi is not currently running");
+            return response;
+        }
+
+        try (Socket socket = new Socket()) {
+            LOGGER.debug("Connecting to MiNiFi instance");
+            socket.setSoTimeout(10000);
+            socket.connect(new InetSocketAddress("localhost", port));
+            LOGGER.debug("Established connection to MiNiFi instance.");
+
+            LOGGER.debug("Sending {} Command to port {}", cmd, port);
+            OutputStream out = socket.getOutputStream();
+            out.write((cmd + " " + miNiFiParameters.getSecretKey() + "\n").getBytes(StandardCharsets.UTF_8));
+            out.flush();
+            socket.shutdownOutput();

Review Comment:
   It looks like this use of the OutputStream could be wrapped in a try-with-resources.



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/BootstrapFileProvider.java:
##########
@@ -0,0 +1,234 @@
+/*
+ * 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.nifi.minifi.bootstrap.service;
+
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.STATUS_FILE_PID_KEY;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.attribute.PosixFilePermission;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.util.file.FileUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class BootstrapFileProvider {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(BootstrapFileProvider.class);
+
+    private static final String MINIFI_PID_FILE_NAME = "minifi.pid";
+    private static final String MINIFI_STATUS_FILE_NAME = "minifi.status";
+    private static final String MINIFI_LOCK_FILE_NAME = "minifi.lock";
+    private static final String DEFAULT_CONFIG_FILE = "./conf/bootstrap.conf";
+    private static final String BOOTSTRAP_CONFIG_FILE_SYSTEM_PROPERTY_KEY = "org.apache.nifi.minifi.bootstrap.config.file";
+    private static final String MINIFI_HOME_ENV_VARIABLE_KEY = "MINIFI_HOME";
+    private static final String MINIFI_PID_DIR_PROP = "org.apache.nifi.minifi.bootstrap.config.pid.dir";
+    private static final String DEFAULT_PID_DIR = "bin";
+
+    private final File bootstrapConfigFile;
+
+    public BootstrapFileProvider(File bootstrapConfigFile) {
+        if (bootstrapConfigFile == null || !bootstrapConfigFile.exists()) {
+            throw new IllegalArgumentException("The specified bootstrap file doesn't exists: " + bootstrapConfigFile);
+        }
+        this.bootstrapConfigFile = bootstrapConfigFile;
+    }
+
+    public static File getBootstrapConfFile() {
+        File bootstrapConfigFile = Optional.ofNullable(System.getProperty(BOOTSTRAP_CONFIG_FILE_SYSTEM_PROPERTY_KEY))
+            .map(File::new)
+            .orElseGet(() -> Optional.ofNullable(System.getenv(MINIFI_HOME_ENV_VARIABLE_KEY))
+                .map(File::new)
+                .map(nifiHomeFile -> new File(nifiHomeFile, DEFAULT_CONFIG_FILE))
+                .orElseGet(() -> new File(DEFAULT_CONFIG_FILE)));
+        LOGGER.debug("Bootstrap config file: {}", bootstrapConfigFile);
+        return bootstrapConfigFile;
+    }
+
+    public File getPidFile() throws IOException {
+        return getBootstrapFile(MINIFI_PID_FILE_NAME);
+    }
+
+    public File getStatusFile() throws IOException {
+        return getBootstrapFile(MINIFI_STATUS_FILE_NAME);
+    }
+
+    public File getLockFile() throws IOException {
+        return getBootstrapFile(MINIFI_LOCK_FILE_NAME);
+    }
+
+    public File getReloadLockFile() {
+        File confDir = bootstrapConfigFile.getParentFile();
+        File nifiHome = confDir.getParentFile();
+        File bin = new File(nifiHome, "bin");
+        File reloadFile = new File(bin, "minifi.reload.lock");
+
+        LOGGER.debug("Reload File: {}", reloadFile);
+        return reloadFile;
+    }
+
+    public File getSwapFile() {
+        File confDir = bootstrapConfigFile.getParentFile();
+        File swapFile = new File(confDir, "swap.yml");
+
+        LOGGER.debug("Swap File: {}", swapFile);
+        return swapFile;
+    }
+
+    public Properties getBootstrapProperties() throws IOException {
+        if (!bootstrapConfigFile.exists()) {
+            throw new FileNotFoundException(bootstrapConfigFile.getAbsolutePath());
+        }
+
+        Properties bootstrapProperties = new Properties();
+        try (FileInputStream fis = new FileInputStream(bootstrapConfigFile)) {
+            bootstrapProperties.load(fis);
+        }
+
+        logProperties("Bootstrap", bootstrapProperties);
+
+        return bootstrapProperties;
+    }
+
+    public Properties getStatusProperties() {
+        Properties props = new Properties();
+
+        try {
+            File statusFile = getStatusFile();
+            if (statusFile == null || !statusFile.exists()) {
+                LOGGER.debug("No status file to load properties from");
+                return props;
+            }
+
+            try (FileInputStream fis = new FileInputStream(statusFile)) {
+                props.load(fis);
+            }
+        } catch (IOException exception) {
+            LOGGER.error("Failed to load MiNiFi status properties");
+        }
+
+        logProperties("MiNiFi status", props);
+
+        return props;
+    }
+
+    public synchronized void saveStatusProperties(Properties minifiProps) throws IOException {
+        String pid = minifiProps.getProperty(STATUS_FILE_PID_KEY);
+        if (!StringUtils.isBlank(pid)) {
+            writePidFile(pid);
+        }
+
+        File statusFile = getStatusFile();
+        if (statusFile.exists() && !statusFile.delete()) {
+            LOGGER.warn("Failed to delete {}", statusFile);
+        }
+
+        if (!statusFile.createNewFile()) {
+            throw new IOException("Failed to create file " + statusFile);
+        }
+
+        try {
+            Set<PosixFilePermission> perms = new HashSet<>();
+            perms.add(PosixFilePermission.OWNER_WRITE);
+            perms.add(PosixFilePermission.OWNER_READ);
+            perms.add(PosixFilePermission.GROUP_READ);
+            perms.add(PosixFilePermission.OTHERS_READ);
+            Files.setPosixFilePermissions(statusFile.toPath(), perms);
+        } catch (Exception e) {
+            LOGGER.warn("Failed to set permissions so that only the owner can read status file {}; "
+                + "this may allows others to have access to the key needed to communicate with MiNiFi. "
+                + "Permissions should be changed so that only the owner can read this file", statusFile);
+        }
+
+        try (FileOutputStream fos = new FileOutputStream(statusFile)) {
+            minifiProps.store(fos, null);
+            fos.getFD().sync();
+        }
+
+        LOGGER.debug("Saving MiNiFi properties to {}", statusFile);
+        logProperties("Saved MiNiFi", minifiProps);
+    }
+
+    private void writePidFile(String pid) throws IOException {
+        File pidFile = getPidFile();
+        if (pidFile.exists() && !pidFile.delete()) {
+            LOGGER.warn("Failed to delete {}", pidFile);
+        }
+
+        if (!pidFile.createNewFile()) {
+            throw new IOException("Failed to create file " + pidFile);
+        }
+
+        try {
+            Set<PosixFilePermission> perms = new HashSet<>();
+            perms.add(PosixFilePermission.OWNER_READ);
+            perms.add(PosixFilePermission.OWNER_WRITE);
+            Files.setPosixFilePermissions(pidFile.toPath(), perms);
+        } catch (Exception e) {
+            LOGGER.warn("Failed to set permissions so that only the owner can read pid file {}; "
+                + "this may allows others to have access to the key needed to communicate with MiNiFi. "
+                + "Permissions should be changed so that only the owner can read this file", pidFile);
+        }
+
+        try (FileOutputStream fos = new FileOutputStream(pidFile)) {
+            fos.write(pid.getBytes(StandardCharsets.UTF_8));
+            fos.getFD().sync();
+        }
+
+        LOGGER.debug("Saved Pid {} to {}", pid, pidFile);
+    }
+
+    private File getBootstrapFile(String fileName) throws IOException {
+        File configFileDir = Optional.ofNullable(System.getProperty(MINIFI_PID_DIR_PROP))
+            .map(String::trim)
+            .map(File::new)
+            .orElseGet(() -> {
+                File confDir = bootstrapConfigFile.getParentFile();
+                File nifiHome = confDir.getParentFile();
+                return new File(nifiHome, DEFAULT_PID_DIR);
+            });
+
+        FileUtils.ensureDirectoryExistAndCanAccess(configFileDir);
+        File statusFile = new File(configFileDir, fileName);
+        LOGGER.debug("Run File: {}", statusFile);
+
+        return statusFile;
+    }
+
+    private void logProperties(String type, Properties props) {
+        if (LOGGER.isDebugEnabled()) {
+            LOGGER.debug("{} properties: {}", type, props.entrySet()
+                .stream()
+                .filter(e -> {
+                    String key = ((String) e.getKey()).toLowerCase();
+                    return !"secret.key".equals(key) && !key.contains("password") && !key.contains("passwd");

Review Comment:
   This check is helpful, but it seems prone to missing new values. It seems better to avoid logging properties altogether, or incorporating something with a property name enum that indicates sensitive status.



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/MiNiFiCommandSender.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 org.apache.nifi.minifi.bootstrap.service;
+
+import java.io.BufferedReader;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.ObjectInputStream;
+import java.io.OutputStream;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.net.SocketTimeoutException;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MiNiFiCommandSender {
+    private static final Logger LOGGER = LoggerFactory.getLogger(MiNiFiCommandSender.class);
+    private static final String PING_CMD = "PING";
+
+    private final MiNiFiParameters miNiFiParameters;
+
+    public MiNiFiCommandSender(MiNiFiParameters miNiFiParameters) {
+        this.miNiFiParameters = miNiFiParameters;
+    }
+
+    public Optional<String> sendCommand(String cmd, Integer port) throws IOException {
+        Optional<String> response = Optional.empty();
+
+        if (port == null) {
+            LOGGER.info("Apache MiNiFi is not currently running");
+            return response;
+        }
+
+        try (Socket socket = new Socket()) {
+            LOGGER.debug("Connecting to MiNiFi instance");
+            socket.setSoTimeout(10000);
+            socket.connect(new InetSocketAddress("localhost", port));
+            LOGGER.debug("Established connection to MiNiFi instance.");
+
+            LOGGER.debug("Sending {} Command to port {}", cmd, port);
+            OutputStream out = socket.getOutputStream();
+            out.write((cmd + " " + miNiFiParameters.getSecretKey() + "\n").getBytes(StandardCharsets.UTF_8));
+            out.flush();
+            socket.shutdownOutput();
+
+            InputStream in = socket.getInputStream();
+            StringBuilder sb = new StringBuilder();
+            int numLines = 0;
+            try (BufferedReader reader = new BufferedReader(new InputStreamReader(in))) {

Review Comment:
   Recommend moving `socket.getInputStream()` inside the try-with-resources call and passed directly to InputStreamReader
   ```suggestion
               StringBuilder sb = new StringBuilder();
               int numLines = 0;
               try (BufferedReader reader = new BufferedReader(new InputStreamReader(socket.getInputStream()))) {
   ```



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/MiNiFiCommandSender.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 org.apache.nifi.minifi.bootstrap.service;
+
+import java.io.BufferedReader;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.ObjectInputStream;
+import java.io.OutputStream;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.net.SocketTimeoutException;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MiNiFiCommandSender {
+    private static final Logger LOGGER = LoggerFactory.getLogger(MiNiFiCommandSender.class);
+    private static final String PING_CMD = "PING";
+
+    private final MiNiFiParameters miNiFiParameters;
+
+    public MiNiFiCommandSender(MiNiFiParameters miNiFiParameters) {
+        this.miNiFiParameters = miNiFiParameters;
+    }
+
+    public Optional<String> sendCommand(String cmd, Integer port) throws IOException {
+        Optional<String> response = Optional.empty();
+
+        if (port == null) {
+            LOGGER.info("Apache MiNiFi is not currently running");
+            return response;
+        }
+
+        try (Socket socket = new Socket()) {
+            LOGGER.debug("Connecting to MiNiFi instance");
+            socket.setSoTimeout(10000);
+            socket.connect(new InetSocketAddress("localhost", port));
+            LOGGER.debug("Established connection to MiNiFi instance.");
+
+            LOGGER.debug("Sending {} Command to port {}", cmd, port);
+            OutputStream out = socket.getOutputStream();
+            out.write((cmd + " " + miNiFiParameters.getSecretKey() + "\n").getBytes(StandardCharsets.UTF_8));
+            out.flush();
+            socket.shutdownOutput();
+
+            InputStream in = socket.getInputStream();
+            StringBuilder sb = new StringBuilder();
+            int numLines = 0;
+            try (BufferedReader reader = new BufferedReader(new InputStreamReader(in))) {
+                String line;
+                while ((line = reader.readLine()) != null) {
+                    if (numLines++ > 0) {
+                        sb.append("\n");
+                    }
+                    sb.append(line);
+                }
+            }
+
+            String responseString = sb.toString().trim();
+
+            LOGGER.debug("Received response to {} command: {}", cmd, responseString);
+            response = Optional.of(responseString);
+        }
+        return response;
+    }
+
+    <T> T sendCommandForObject(String cmd, Integer port, String... extraParams) throws IOException {
+        T response;
+        try (Socket socket = new Socket("localhost", port)) {
+            OutputStream out = socket.getOutputStream();
+            String argsString = Arrays.stream(extraParams).collect(Collectors.joining(" ", " ", ""));
+            String commandWithArgs = cmd + " " + miNiFiParameters.getSecretKey() + argsString + "\n";
+            out.write((commandWithArgs).getBytes(StandardCharsets.UTF_8));
+            LOGGER.debug("Sending {} command to MiNiFi with the following args: [{}]", cmd, argsString);
+            out.flush();
+
+            socket.setSoTimeout(5000);

Review Comment:
   Recommend setting a static variable and reusing for consistent timeout. This reference is 5 seconds, whereas the above method has 10 seconds.



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/MiNiFiCommandSender.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 org.apache.nifi.minifi.bootstrap.service;
+
+import java.io.BufferedReader;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.ObjectInputStream;
+import java.io.OutputStream;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.net.SocketTimeoutException;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MiNiFiCommandSender {
+    private static final Logger LOGGER = LoggerFactory.getLogger(MiNiFiCommandSender.class);
+    private static final String PING_CMD = "PING";
+
+    private final MiNiFiParameters miNiFiParameters;
+
+    public MiNiFiCommandSender(MiNiFiParameters miNiFiParameters) {
+        this.miNiFiParameters = miNiFiParameters;
+    }
+
+    public Optional<String> sendCommand(String cmd, Integer port) throws IOException {
+        Optional<String> response = Optional.empty();
+
+        if (port == null) {
+            LOGGER.info("Apache MiNiFi is not currently running");
+            return response;
+        }
+
+        try (Socket socket = new Socket()) {
+            LOGGER.debug("Connecting to MiNiFi instance");
+            socket.setSoTimeout(10000);
+            socket.connect(new InetSocketAddress("localhost", port));
+            LOGGER.debug("Established connection to MiNiFi instance.");
+
+            LOGGER.debug("Sending {} Command to port {}", cmd, port);
+            OutputStream out = socket.getOutputStream();
+            out.write((cmd + " " + miNiFiParameters.getSecretKey() + "\n").getBytes(StandardCharsets.UTF_8));
+            out.flush();
+            socket.shutdownOutput();
+
+            InputStream in = socket.getInputStream();
+            StringBuilder sb = new StringBuilder();
+            int numLines = 0;
+            try (BufferedReader reader = new BufferedReader(new InputStreamReader(in))) {
+                String line;
+                while ((line = reader.readLine()) != null) {
+                    if (numLines++ > 0) {
+                        sb.append("\n");
+                    }
+                    sb.append(line);
+                }
+            }
+
+            String responseString = sb.toString().trim();
+
+            LOGGER.debug("Received response to {} command: {}", cmd, responseString);
+            response = Optional.of(responseString);
+        }
+        return response;
+    }
+
+    <T> T sendCommandForObject(String cmd, Integer port, String... extraParams) throws IOException {
+        T response;
+        try (Socket socket = new Socket("localhost", port)) {
+            OutputStream out = socket.getOutputStream();
+            String argsString = Arrays.stream(extraParams).collect(Collectors.joining(" ", " ", ""));
+            String commandWithArgs = cmd + " " + miNiFiParameters.getSecretKey() + argsString + "\n";
+            out.write((commandWithArgs).getBytes(StandardCharsets.UTF_8));
+            LOGGER.debug("Sending {} command to MiNiFi with the following args: [{}]", cmd, argsString);
+            out.flush();
+
+            socket.setSoTimeout(5000);
+            InputStream in = socket.getInputStream();
+
+            ObjectInputStream ois = new ObjectInputStream(in);
+            Object o = ois.readObject();

Review Comment:
   Use of the ObjectInputStream presents security concerns in some environments. Although the previous implementation used the ObjectInputStream, this approach should now be refactored. Using JSON would provide a safer alternative.



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/MiNiFiCommandSender.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 org.apache.nifi.minifi.bootstrap.service;
+
+import java.io.BufferedReader;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.ObjectInputStream;
+import java.io.OutputStream;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.net.SocketTimeoutException;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MiNiFiCommandSender {
+    private static final Logger LOGGER = LoggerFactory.getLogger(MiNiFiCommandSender.class);
+    private static final String PING_CMD = "PING";
+
+    private final MiNiFiParameters miNiFiParameters;
+
+    public MiNiFiCommandSender(MiNiFiParameters miNiFiParameters) {
+        this.miNiFiParameters = miNiFiParameters;
+    }
+
+    public Optional<String> sendCommand(String cmd, Integer port) throws IOException {
+        Optional<String> response = Optional.empty();
+
+        if (port == null) {
+            LOGGER.info("Apache MiNiFi is not currently running");
+            return response;
+        }
+
+        try (Socket socket = new Socket()) {
+            LOGGER.debug("Connecting to MiNiFi instance");
+            socket.setSoTimeout(10000);
+            socket.connect(new InetSocketAddress("localhost", port));
+            LOGGER.debug("Established connection to MiNiFi instance.");
+
+            LOGGER.debug("Sending {} Command to port {}", cmd, port);
+            OutputStream out = socket.getOutputStream();
+            out.write((cmd + " " + miNiFiParameters.getSecretKey() + "\n").getBytes(StandardCharsets.UTF_8));
+            out.flush();
+            socket.shutdownOutput();
+
+            InputStream in = socket.getInputStream();
+            StringBuilder sb = new StringBuilder();
+            int numLines = 0;
+            try (BufferedReader reader = new BufferedReader(new InputStreamReader(in))) {
+                String line;
+                while ((line = reader.readLine()) != null) {
+                    if (numLines++ > 0) {
+                        sb.append("\n");
+                    }
+                    sb.append(line);
+                }
+            }
+
+            String responseString = sb.toString().trim();
+
+            LOGGER.debug("Received response to {} command: {}", cmd, responseString);
+            response = Optional.of(responseString);
+        }
+        return response;
+    }
+
+    <T> T sendCommandForObject(String cmd, Integer port, String... extraParams) throws IOException {
+        T response;
+        try (Socket socket = new Socket("localhost", port)) {
+            OutputStream out = socket.getOutputStream();
+            String argsString = Arrays.stream(extraParams).collect(Collectors.joining(" ", " ", ""));
+            String commandWithArgs = cmd + " " + miNiFiParameters.getSecretKey() + argsString + "\n";
+            out.write((commandWithArgs).getBytes(StandardCharsets.UTF_8));
+            LOGGER.debug("Sending {} command to MiNiFi with the following args: [{}]", cmd, argsString);
+            out.flush();
+
+            socket.setSoTimeout(5000);
+            InputStream in = socket.getInputStream();
+
+            ObjectInputStream ois = new ObjectInputStream(in);
+            Object o = ois.readObject();
+            ois.close();
+            out.close();
+            response = castResponse(cmd, o);
+        } catch (EOFException | ClassNotFoundException | SocketTimeoutException e) {
+            String message = "Failed to get response for " + cmd + " Potentially due to the process currently being down (restarting or otherwise). Exception:" + e;
+            LOGGER.error(message);
+            throw new RuntimeException(message);
+        }
+        return response;
+    }
+
+    private <T> T castResponse(String cmd, Object o) {
+        T response;
+        try {
+            response = (T) o;
+        } catch (ClassCastException e) {
+            String message = "Failed to cast " + cmd + " response to the requested type";
+            LOGGER.error(message);
+            throw new RuntimeException(message);

Review Comment:
   Is it necessary to both log and error and throw an exception?



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/MiNiFiConfigurationChangeListener.java:
##########
@@ -0,0 +1,215 @@
+/*
+ * 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.nifi.minifi.bootstrap.service;
+
+import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CONF_DIR_KEY;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.MINIFI_CONFIG_FILE_KEY;
+import static org.apache.nifi.minifi.bootstrap.configuration.ingestors.PullHttpChangeIngestor.OVERRIDE_SECURITY;
+import static org.apache.nifi.minifi.bootstrap.configuration.ingestors.PullHttpChangeIngestor.PULL_HTTP_BASE_KEY;
+import static org.apache.nifi.minifi.bootstrap.util.ConfigTransformer.generateConfigFiles;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.util.Properties;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.commons.io.IOUtils;
+import org.apache.nifi.minifi.bootstrap.RunMiNiFi;
+import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeException;
+import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeListener;
+import org.apache.nifi.minifi.bootstrap.util.ByteBufferInputStream;
+import org.apache.nifi.minifi.bootstrap.util.ConfigTransformer;
+import org.apache.nifi.minifi.commons.schema.ConfigSchema;
+import org.apache.nifi.minifi.commons.schema.common.ConvertableSchema;
+import org.apache.nifi.minifi.commons.schema.serialization.SchemaLoader;
+import org.slf4j.Logger;
+
+public class MiNiFiConfigurationChangeListener implements ConfigurationChangeListener {
+
+    private final RunMiNiFi runner;
+    private final Logger logger;
+    private final BootstrapFileProvider bootstrapFileProvider;
+
+    private static final ReentrantLock handlingLock = new ReentrantLock();
+
+    public MiNiFiConfigurationChangeListener(RunMiNiFi runner, Logger logger, BootstrapFileProvider bootstrapFileProvider) {
+        this.runner = runner;
+        this.logger = logger;
+        this.bootstrapFileProvider = bootstrapFileProvider;
+    }
+
+    @Override
+    public void handleChange(InputStream configInputStream) throws ConfigurationChangeException {
+        logger.info("Received notification of a change");
+
+        if (!handlingLock.tryLock()) {
+            throw new ConfigurationChangeException("Instance is already handling another change");
+        }
+        // Store the incoming stream as a byte array to be shared among components that need it
+        try(ByteArrayOutputStream bufferedConfigOs = new ByteArrayOutputStream()) {
+
+            Properties bootstrapProperties = bootstrapFileProvider.getBootstrapProperties();
+            File configFile = new File(bootstrapProperties.getProperty(MINIFI_CONFIG_FILE_KEY));
+
+            byte[] copyArray = new byte[1024];
+            int available;
+            while ((available = configInputStream.read(copyArray)) > 0) {
+                bufferedConfigOs.write(copyArray, 0, available);
+            }
+
+            File swapConfigFile = bootstrapFileProvider.getSwapFile();
+            logger.info("Persisting old configuration to {}", swapConfigFile.getAbsolutePath());
+
+            try (FileInputStream configFileInputStream = new FileInputStream(configFile)) {
+                Files.copy(configFileInputStream, swapConfigFile.toPath(), REPLACE_EXISTING);
+            }
+
+            persistBackNonFlowSectionsFromOriginalSchema(bufferedConfigOs.toByteArray(), bootstrapProperties, configFile);
+
+            // Create an input stream to feed to the config transformer
+            try (ByteArrayInputStream newConfigBais = new ByteArrayInputStream(IOUtils.toByteArray(new FileInputStream(configFile)))) {
+                newConfigBais.mark(-1);
+
+                try {
+                    String confDir = bootstrapProperties.getProperty(CONF_DIR_KEY);
+
+                    try {
+                        logger.info("Performing transformation for input and saving outputs to {}", confDir);
+                        ByteBuffer tempConfigFile = generateConfigFiles(newConfigBais, confDir, bootstrapFileProvider.getBootstrapProperties());
+                        runner.getConfigFileReference().set(tempConfigFile.asReadOnlyBuffer());
+
+                        try {
+                            logger.info("Reloading instance with new configuration");
+                            restartInstance();
+                        } catch (Exception e) {
+                            logger.debug("Transformation of new config file failed after transformation into Flow.xml and nifi.properties, reverting.");
+                            try (FileInputStream swapConfigFileStream = new FileInputStream(swapConfigFile)) {
+                                ByteBuffer resetConfigFile = generateConfigFiles(swapConfigFileStream, confDir, bootstrapFileProvider.getBootstrapProperties());
+                                runner.getConfigFileReference().set(resetConfigFile.asReadOnlyBuffer());
+                            }
+                            throw e;
+                        }
+                    } catch (Exception e) {
+                        logger.debug("Transformation of new config file failed after replacing original with the swap file, reverting.");
+                        try (FileInputStream swapConfigFileStream = new FileInputStream(swapConfigFile)) {
+                            Files.copy(swapConfigFileStream, configFile.toPath(), REPLACE_EXISTING);
+                        }
+                        throw e;
+                    }
+                } catch (Exception e) {
+                    logger.debug("Transformation of new config file failed after swap file was created, deleting it.");
+                    if (!swapConfigFile.delete()) {
+                        logger.warn("The swap file failed to delete after a failed handling of a change. It should be cleaned up manually.");
+                    }
+                    throw e;
+                }
+            }
+        } catch (ConfigurationChangeException e){
+            logger.error("Unable to carry out reloading of configuration on receipt of notification event", e);
+            throw e;
+        } catch (IOException ioe) {
+            logger.error("Unable to carry out reloading of configuration on receipt of notification event", ioe);
+            throw new ConfigurationChangeException("Unable to perform reload of received configuration change", ioe);
+        } finally {
+            try {
+                if (configInputStream != null) {
+                    configInputStream.close() ;
+                }
+            } catch (IOException e) {
+                // Quietly close
+            }

Review Comment:
   Using one of the commons-lang3 utilities should provide support for a quite close without the empty catch.



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/util/ConfigTransformer.java:
##########
@@ -88,42 +91,68 @@ public final class ConfigTransformer {
     private ConfigTransformer() {
     }
 
+    public static ByteBuffer generateConfigFiles(InputStream configIs, String configDestinationPath, Properties bootstrapProperties) throws ConfigurationChangeException, IOException {

Review Comment:
   Is there a reason for returning `ByteBuffer` as opposed to `byte[]`?



##########
nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowService.java:
##########
@@ -294,6 +299,17 @@ public void start() throws LifeCycleStartException {
 
             if (configuredForClustering) {
                 senderListener.start();
+            } else {
+                // If standalone and C2 is enabled, create a C2 client
+                final boolean c2Enabled = Boolean.parseBoolean(nifiProperties.getProperty(C2NiFiProperties.C2_ENABLE_KEY, "false"));
+                if (c2Enabled) {
+                    logger.info("C2 enabled, creating a C2 client instance");
+                    c2NifiClientService = new C2NifiClientService(nifiProperties, this, this.controller);
+                    c2NifiClientService.start();
+                } else {
+                    logger.info("The '" + C2NiFiProperties.C2_ENABLE_KEY + "' property is missing/false, not creating a C2 client instance");

Review Comment:
   ```suggestion
                       logger.info("C2 Property [{}] missing or disabled: C2 client not created", C2NiFiProperties.C2_ENABLE_KEY);
   ```



##########
nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/c2/C2NifiClientService.java:
##########
@@ -0,0 +1,208 @@
+/*
+ * 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.nifi.c2;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Optional;
+import org.apache.nifi.c2.client.C2ClientConfig;
+import org.apache.nifi.c2.client.http.C2HttpClient;
+import org.apache.nifi.c2.client.service.C2ClientService;
+import org.apache.nifi.c2.client.service.C2HeartbeatFactory;
+import org.apache.nifi.c2.client.service.FlowIdHolder;
+import org.apache.nifi.c2.client.service.model.RuntimeInfoWrapper;
+import org.apache.nifi.c2.protocol.api.AgentRepositories;
+import org.apache.nifi.c2.protocol.api.AgentRepositoryStatus;
+import org.apache.nifi.c2.protocol.api.FlowQueueStatus;
+import org.apache.nifi.c2.serializer.C2JacksonSerializer;
+import org.apache.nifi.controller.FlowController;;
+import org.apache.nifi.controller.status.ConnectionStatus;
+import org.apache.nifi.controller.status.ProcessGroupStatus;
+import org.apache.nifi.diagnostics.StorageUsage;
+import org.apache.nifi.diagnostics.SystemDiagnostics;
+import org.apache.nifi.extension.manifest.parser.ExtensionManifestParser;
+import org.apache.nifi.extension.manifest.parser.jaxb.JAXBExtensionManifestParser;
+import org.apache.nifi.manifest.RuntimeManifestService;
+import org.apache.nifi.manifest.StandardRuntimeManifestService;
+import org.apache.nifi.nar.ExtensionManagerHolder;
+import org.apache.nifi.services.FlowService;
+import org.apache.nifi.util.FormatUtils;
+import org.apache.nifi.util.NiFiProperties;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+public class C2NifiClientService {
+
+    private static final Logger logger = LoggerFactory.getLogger(C2NifiClientService.class);
+    private static final String DEFAULT_CONF_DIR = "./conf";
+    private static final String TARGET_CONFIG_FILE = "/config-new.yml";
+    private static final String ROOT_GROUP_ID = "root";
+    private static final Long INITIAL_DELAY = 0L;
+    private static final Integer TERMINATION_WAIT = 5000;
+
+    private final C2ClientService c2ClientService;
+
+    private final FlowService flowService;
+    private final FlowController flowController;
+    private final String propertiesDir;
+    private final ScheduledThreadPoolExecutor scheduledExecutorService = new ScheduledThreadPoolExecutor(1);
+    private final ExtensionManifestParser extensionManifestParser = new JAXBExtensionManifestParser();
+
+    private final RuntimeManifestService runtimeManifestService;
+    private final long heartbeatPeriod;
+
+    public C2NifiClientService(final NiFiProperties niFiProperties, final FlowService flowService, final FlowController flowController) {
+        C2ClientConfig clientConfig = generateClientConfig(niFiProperties);
+        FlowIdHolder flowIdHolder = new FlowIdHolder(clientConfig.getConfDirectory());
+        this.propertiesDir = niFiProperties.getProperty(NiFiProperties.PROPERTIES_FILE_PATH, null);
+        this.runtimeManifestService = new StandardRuntimeManifestService(
+            ExtensionManagerHolder.getExtensionManager(),
+            extensionManifestParser,
+            clientConfig.getRuntimeManifestIdentifier(),
+            clientConfig.getRuntimeType()
+        );
+        this.heartbeatPeriod = clientConfig.getHeartbeatPeriod();
+        this.flowService = flowService;
+        this.flowController = flowController;
+        this.c2ClientService = new C2ClientService(
+            new C2HttpClient(clientConfig, new C2JacksonSerializer()),
+            new C2HeartbeatFactory(clientConfig, flowIdHolder),
+            flowIdHolder,
+            this::updateFlowContent
+        );
+    }
+
+    private C2ClientConfig generateClientConfig(NiFiProperties properties) {
+        return new C2ClientConfig.Builder()
+                .agentClass(properties.getProperty(C2NiFiProperties.C2_AGENT_CLASS_KEY, ""))
+                .agentIdentifier(properties.getProperty(C2NiFiProperties.C2_AGENT_IDENTIFIER_KEY))
+                .heartbeatPeriod(Long.valueOf(properties.getProperty(C2NiFiProperties.C2_AGENT_HEARTBEAT_PERIOD_KEY,
+                    String.valueOf(C2NiFiProperties.C2_AGENT_DEFAULT_HEARTBEAT_PERIOD))))
+                .callTimeout((long) FormatUtils.getPreciseTimeDuration(properties.getProperty(C2NiFiProperties.C2_CALL_TIMEOUT,
+                    C2NiFiProperties.C2_DEFAULT_CALL_TIMEOUT), TimeUnit.MILLISECONDS))
+                .c2Url(properties.getProperty(C2NiFiProperties.C2_REST_URL_KEY, ""))
+                .confDirectory(properties.getProperty(C2NiFiProperties.C2_CONFIG_DIRECTORY_KEY, DEFAULT_CONF_DIR))
+                .runtimeManifestIdentifier(properties.getProperty(C2NiFiProperties.C2_RUNTIME_MANIFEST_IDENTIFIER_KEY, ""))
+                .runtimeType(properties.getProperty(C2NiFiProperties.C2_RUNTIME_TYPE_KEY, ""))
+                .c2AckUrl(properties.getProperty(C2NiFiProperties.C2_REST_URL_ACK_KEY, ""))
+                .truststoreFilename(properties.getProperty(C2NiFiProperties.TRUSTSTORE_LOCATION_KEY, ""))
+                .truststorePassword(properties.getProperty(C2NiFiProperties.TRUSTSTORE_PASSWORD_KEY, ""))
+                .truststoreType(properties.getProperty(C2NiFiProperties.TRUSTSTORE_TYPE_KEY, "JKS"))
+                .keystoreFilename(properties.getProperty(C2NiFiProperties.KEYSTORE_LOCATION_KEY, ""))
+                .keystorePassword(properties.getProperty(C2NiFiProperties.KEYSTORE_PASSWORD_KEY, ""))
+                .keystoreType(properties.getProperty(C2NiFiProperties.KEYSTORE_TYPE_KEY, "JKS"))
+                .build();
+    }
+
+    public void start() {
+        try {
+            scheduledExecutorService.scheduleAtFixedRate(() -> c2ClientService.sendHeartbeat(generateRuntimeInfo()), INITIAL_DELAY, heartbeatPeriod, TimeUnit.MILLISECONDS);
+        } catch (Exception e) {
+            logger.error("Could not start C2 Client Heartbeat Reporting", e);
+            throw new RuntimeException(e);
+        }
+    }
+
+    public void stop() {
+        try {
+            scheduledExecutorService.shutdown();
+            scheduledExecutorService.awaitTermination(TERMINATION_WAIT, TimeUnit.MILLISECONDS);
+        } catch (InterruptedException ignore) {
+            logger.info("Stopping C2 Client's thread was interrupted but shutting down anyway the C2NifiClientService");
+        }
+    }
+
+    private RuntimeInfoWrapper generateRuntimeInfo() {
+        return new RuntimeInfoWrapper(getAgentRepositories(), runtimeManifestService.getManifest(), getQueueStatus());
+    }
+
+    private AgentRepositories getAgentRepositories() {
+        final SystemDiagnostics systemDiagnostics = flowController.getSystemDiagnostics();
+
+        final AgentRepositories repos = new AgentRepositories();
+        final AgentRepositoryStatus flowFileRepoStatus = new AgentRepositoryStatus();
+        final StorageUsage ffRepoStorageUsage = systemDiagnostics.getFlowFileRepositoryStorageUsage();
+        flowFileRepoStatus.setDataSize(ffRepoStorageUsage.getUsedSpace());
+        flowFileRepoStatus.setDataSizeMax(ffRepoStorageUsage.getTotalSpace());
+        repos.setFlowFile(flowFileRepoStatus);
+
+        final AgentRepositoryStatus provRepoStatus = new AgentRepositoryStatus();
+        final Iterator<Map.Entry<String, StorageUsage>> provRepoStorageUsages = systemDiagnostics.getProvenanceRepositoryStorageUsage().entrySet().iterator();
+        if (provRepoStorageUsages.hasNext()) {
+            final StorageUsage provRepoStorageUsage = provRepoStorageUsages.next().getValue();
+            provRepoStatus.setDataSize(provRepoStorageUsage.getUsedSpace());
+            provRepoStatus.setDataSizeMax(provRepoStorageUsage.getTotalSpace());
+        }
+
+        repos.setProvenance(provRepoStatus);
+
+        return repos;
+    }
+
+    private Map<String, FlowQueueStatus> getQueueStatus() {
+        ProcessGroupStatus rootProcessGroupStatus = flowController.getEventAccess().getGroupStatus(ROOT_GROUP_ID);
+
+        final Collection<ConnectionStatus> connectionStatuses = rootProcessGroupStatus.getConnectionStatus();
+
+        final Map<String, FlowQueueStatus> processGroupStatus = new HashMap<>();
+        for (ConnectionStatus connectionStatus : connectionStatuses) {
+            final FlowQueueStatus flowQueueStatus = new FlowQueueStatus();
+
+            flowQueueStatus.setSize((long) connectionStatus.getQueuedCount());
+            flowQueueStatus.setSizeMax(connectionStatus.getBackPressureObjectThreshold());
+
+            flowQueueStatus.setDataSize(connectionStatus.getQueuedBytes());
+            flowQueueStatus.setDataSizeMax(connectionStatus.getBackPressureBytesThreshold());
+
+            processGroupStatus.put(connectionStatus.getId(), flowQueueStatus);
+        }
+
+        return processGroupStatus;
+    }
+
+    private boolean updateFlowContent(byte[] updateContent) {
+        logger.debug("Update content: \n{}", new String(updateContent, StandardCharsets.UTF_8));
+        Path path = getTargetConfigFile().toPath();
+        try {
+            Files.write(getTargetConfigFile().toPath(), updateContent);
+            logger.info("Updated configuration was written to: {}", path);
+            return true;
+        } catch (IOException e) {
+            logger.error("Configuration update failed. File creation was not successful targeting: {}", path);
+            logger.error("Exception: ", e);

Review Comment:
   ```suggestion
               logger.error("Configuration update failed. File creation was not successful targeting: {}", path, e);
   ```



##########
nifi-server-api/src/main/java/org/apache/nifi/NiFiServer.java:
##########
@@ -26,13 +26,13 @@
 import java.util.Set;
 
 /**
- *
+ * The main interface for declaring a NiFi-based server application
  */
 public interface NiFiServer {
 
     void start();
 
-    void initialize(NiFiProperties properties, Bundle systemBundle, Set<Bundle> bundles, ExtensionMapping extensionMapping);
+    void initialize(final NiFiProperties properties, final Bundle systemBundle, final Set<Bundle> bundles, final ExtensionMapping extensionMapping);

Review Comment:
   The `final` keyword is implied, and not necessary on interface method arguments.



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/command/StartRunner.java:
##########
@@ -0,0 +1,435 @@
+/*
+ * 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.nifi.minifi.bootstrap.command;
+
+import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CMD_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CONF_DIR_KEY;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.DEFAULT_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.MINIFI_CONFIG_FILE_KEY;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.STATUS_FILE_PID_KEY;
+import static org.apache.nifi.minifi.bootstrap.Status.ERROR;
+import static org.apache.nifi.minifi.bootstrap.Status.OK;
+import static org.apache.nifi.minifi.bootstrap.util.ConfigTransformer.generateConfigFiles;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.nifi.bootstrap.util.OSUtils;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.apache.nifi.minifi.bootstrap.RunMiNiFi;
+import org.apache.nifi.minifi.bootstrap.ShutdownHook;
+import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeException;
+import org.apache.nifi.minifi.bootstrap.exception.StartupFailureException;
+import org.apache.nifi.minifi.bootstrap.service.BootstrapFileProvider;
+import org.apache.nifi.minifi.bootstrap.service.CurrentPortProvider;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiListener;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiStdLogHandler;
+import org.apache.nifi.minifi.bootstrap.service.PeriodicStatusReporterManager;
+import org.apache.nifi.minifi.bootstrap.util.ProcessUtils;
+import org.apache.nifi.util.Tuple;
+
+public class StartRunner implements CommandRunner {
+    private static final String DEFAULT_JAVA_CMD = "java";
+    private static final String DEFAULT_LOG_DIR = "./logs";
+    private static final String DEFAULT_LIB_DIR = "./lib";
+    private static final String DEFAULT_CONF_DIR = "./conf";
+    private static final int STARTUP_WAIT_SECONDS = 60;
+    private static final String DEFAULT_CONFIG_FILE = DEFAULT_CONF_DIR + "/bootstrap.conf";
+
+    private final CurrentPortProvider currentPortProvider;
+    private final BootstrapFileProvider bootstrapFileProvider;
+    private final PeriodicStatusReporterManager periodicStatusReporterManager;
+    private final MiNiFiStdLogHandler miNiFiStdLogHandler;
+    private final MiNiFiParameters miNiFiParameters;
+    private final File bootstrapConfigFile;
+    private final Lock lock = new ReentrantLock();
+    private final Condition startupCondition = lock.newCondition();
+    private final RunMiNiFi runMiNiFi;
+    private volatile ShutdownHook shutdownHook;
+
+    public StartRunner(CurrentPortProvider currentPortProvider, BootstrapFileProvider bootstrapFileProvider,
+        PeriodicStatusReporterManager periodicStatusReporterManager, MiNiFiStdLogHandler miNiFiStdLogHandler, MiNiFiParameters miNiFiParameters, File bootstrapConfigFile,
+        RunMiNiFi runMiNiFi) {
+        this.currentPortProvider = currentPortProvider;
+        this.bootstrapFileProvider = bootstrapFileProvider;
+        this.periodicStatusReporterManager = periodicStatusReporterManager;
+        this.miNiFiStdLogHandler = miNiFiStdLogHandler;
+        this.miNiFiParameters = miNiFiParameters;
+        this.bootstrapConfigFile = bootstrapConfigFile;
+        this.runMiNiFi = runMiNiFi;
+    }
+
+    /**
+     * Starts (and restarts) MiNiFi process during the whole lifecycle of the bootstrap process.
+     * @param args the input arguments
+     * @return status code
+     */
+    @Override
+    public int runCommand(String[] args) {
+        try {
+            start();
+        } catch (Exception e) {
+            CMD_LOGGER.error("Exception happened during MiNiFi startup", e);
+            return ERROR.getStatusCode();
+        }
+        return OK.getStatusCode();
+    }
+
+    private void start() throws IOException, InterruptedException {
+        Integer port = currentPortProvider.getCurrentPort();
+        if (port != null) {
+            String alreadyRunningMessage = "Apache MiNiFi is already running";
+            CMD_LOGGER.info(alreadyRunningMessage + ", listening to Bootstrap on port {}", port);

Review Comment:
   This could be consolidated:
   ```suggestion
               CMD_LOGGER.info("Apache MiNiFi is already running, listening to Bootstrap on port {}", port);
   ```



##########
minifi/pom.xml:
##########
@@ -449,10 +449,22 @@ limitations under the License.
                 <version>1.17.0-SNAPSHOT</version>
             </dependency>
 
+            <!-- C2 modules -->
+            <dependency>
+                <groupId>org.apache.nifi</groupId>
+                <artifactId>c2-client-api</artifactId>
+                <version>1.17.0-SNAPSHOT</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.nifi</groupId>
+                <artifactId>c2-client-base</artifactId>
+                <version>1.17.0-SNAPSHOT</version>
+            </dependency>
+
             <dependency>
                 <groupId>com.squareup.okhttp3</groupId>
                 <artifactId>okhttp</artifactId>
-                <version>3.12.3</version>
+                <version>${okhttp.version}</version>

Review Comment:
   It looks like this line still needs to be removed.



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/MiNiFiCommandSender.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 org.apache.nifi.minifi.bootstrap.service;
+
+import java.io.BufferedReader;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.ObjectInputStream;
+import java.io.OutputStream;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.net.SocketTimeoutException;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MiNiFiCommandSender {
+    private static final Logger LOGGER = LoggerFactory.getLogger(MiNiFiCommandSender.class);
+    private static final String PING_CMD = "PING";
+
+    private final MiNiFiParameters miNiFiParameters;
+
+    public MiNiFiCommandSender(MiNiFiParameters miNiFiParameters) {
+        this.miNiFiParameters = miNiFiParameters;
+    }
+
+    public Optional<String> sendCommand(String cmd, Integer port) throws IOException {
+        Optional<String> response = Optional.empty();
+
+        if (port == null) {
+            LOGGER.info("Apache MiNiFi is not currently running");
+            return response;
+        }
+
+        try (Socket socket = new Socket()) {
+            LOGGER.debug("Connecting to MiNiFi instance");
+            socket.setSoTimeout(10000);
+            socket.connect(new InetSocketAddress("localhost", port));

Review Comment:
   The timeout argument should be added:
   ```suggestion
               socket.connect(new InetSocketAddress("localhost", port), 10000);
   ```



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/BootstrapFileProvider.java:
##########
@@ -0,0 +1,234 @@
+/*
+ * 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.nifi.minifi.bootstrap.service;
+
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.STATUS_FILE_PID_KEY;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.attribute.PosixFilePermission;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.util.file.FileUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class BootstrapFileProvider {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(BootstrapFileProvider.class);
+
+    private static final String MINIFI_PID_FILE_NAME = "minifi.pid";
+    private static final String MINIFI_STATUS_FILE_NAME = "minifi.status";
+    private static final String MINIFI_LOCK_FILE_NAME = "minifi.lock";
+    private static final String DEFAULT_CONFIG_FILE = "./conf/bootstrap.conf";
+    private static final String BOOTSTRAP_CONFIG_FILE_SYSTEM_PROPERTY_KEY = "org.apache.nifi.minifi.bootstrap.config.file";
+    private static final String MINIFI_HOME_ENV_VARIABLE_KEY = "MINIFI_HOME";
+    private static final String MINIFI_PID_DIR_PROP = "org.apache.nifi.minifi.bootstrap.config.pid.dir";
+    private static final String DEFAULT_PID_DIR = "bin";
+
+    private final File bootstrapConfigFile;
+
+    public BootstrapFileProvider(File bootstrapConfigFile) {
+        if (bootstrapConfigFile == null || !bootstrapConfigFile.exists()) {
+            throw new IllegalArgumentException("The specified bootstrap file doesn't exists: " + bootstrapConfigFile);
+        }
+        this.bootstrapConfigFile = bootstrapConfigFile;
+    }
+
+    public static File getBootstrapConfFile() {
+        File bootstrapConfigFile = Optional.ofNullable(System.getProperty(BOOTSTRAP_CONFIG_FILE_SYSTEM_PROPERTY_KEY))
+            .map(File::new)
+            .orElseGet(() -> Optional.ofNullable(System.getenv(MINIFI_HOME_ENV_VARIABLE_KEY))
+                .map(File::new)
+                .map(nifiHomeFile -> new File(nifiHomeFile, DEFAULT_CONFIG_FILE))
+                .orElseGet(() -> new File(DEFAULT_CONFIG_FILE)));
+        LOGGER.debug("Bootstrap config file: {}", bootstrapConfigFile);
+        return bootstrapConfigFile;
+    }
+
+    public File getPidFile() throws IOException {
+        return getBootstrapFile(MINIFI_PID_FILE_NAME);
+    }
+
+    public File getStatusFile() throws IOException {
+        return getBootstrapFile(MINIFI_STATUS_FILE_NAME);
+    }
+
+    public File getLockFile() throws IOException {
+        return getBootstrapFile(MINIFI_LOCK_FILE_NAME);
+    }
+
+    public File getReloadLockFile() {
+        File confDir = bootstrapConfigFile.getParentFile();
+        File nifiHome = confDir.getParentFile();
+        File bin = new File(nifiHome, "bin");
+        File reloadFile = new File(bin, "minifi.reload.lock");
+
+        LOGGER.debug("Reload File: {}", reloadFile);
+        return reloadFile;
+    }
+
+    public File getSwapFile() {
+        File confDir = bootstrapConfigFile.getParentFile();
+        File swapFile = new File(confDir, "swap.yml");
+
+        LOGGER.debug("Swap File: {}", swapFile);
+        return swapFile;
+    }
+
+    public Properties getBootstrapProperties() throws IOException {
+        if (!bootstrapConfigFile.exists()) {
+            throw new FileNotFoundException(bootstrapConfigFile.getAbsolutePath());
+        }
+
+        Properties bootstrapProperties = new Properties();
+        try (FileInputStream fis = new FileInputStream(bootstrapConfigFile)) {
+            bootstrapProperties.load(fis);
+        }
+
+        logProperties("Bootstrap", bootstrapProperties);
+
+        return bootstrapProperties;
+    }
+
+    public Properties getStatusProperties() {
+        Properties props = new Properties();
+
+        try {
+            File statusFile = getStatusFile();
+            if (statusFile == null || !statusFile.exists()) {
+                LOGGER.debug("No status file to load properties from");
+                return props;
+            }
+
+            try (FileInputStream fis = new FileInputStream(statusFile)) {
+                props.load(fis);
+            }
+        } catch (IOException exception) {
+            LOGGER.error("Failed to load MiNiFi status properties");
+        }
+
+        logProperties("MiNiFi status", props);
+
+        return props;
+    }
+
+    public synchronized void saveStatusProperties(Properties minifiProps) throws IOException {
+        String pid = minifiProps.getProperty(STATUS_FILE_PID_KEY);
+        if (!StringUtils.isBlank(pid)) {
+            writePidFile(pid);
+        }
+
+        File statusFile = getStatusFile();
+        if (statusFile.exists() && !statusFile.delete()) {
+            LOGGER.warn("Failed to delete {}", statusFile);
+        }
+
+        if (!statusFile.createNewFile()) {
+            throw new IOException("Failed to create file " + statusFile);
+        }
+
+        try {
+            Set<PosixFilePermission> perms = new HashSet<>();
+            perms.add(PosixFilePermission.OWNER_WRITE);
+            perms.add(PosixFilePermission.OWNER_READ);
+            perms.add(PosixFilePermission.GROUP_READ);
+            perms.add(PosixFilePermission.OTHERS_READ);
+            Files.setPosixFilePermissions(statusFile.toPath(), perms);
+        } catch (Exception e) {
+            LOGGER.warn("Failed to set permissions so that only the owner can read status file {}; "
+                + "this may allows others to have access to the key needed to communicate with MiNiFi. "
+                + "Permissions should be changed so that only the owner can read this file", statusFile);
+        }
+
+        try (FileOutputStream fos = new FileOutputStream(statusFile)) {
+            minifiProps.store(fos, null);
+            fos.getFD().sync();
+        }
+
+        LOGGER.debug("Saving MiNiFi properties to {}", statusFile);
+        logProperties("Saved MiNiFi", minifiProps);

Review Comment:
   See other notes on logging properties



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/MiNiFiConfigurationChangeListener.java:
##########
@@ -0,0 +1,215 @@
+/*
+ * 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.nifi.minifi.bootstrap.service;
+
+import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CONF_DIR_KEY;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.MINIFI_CONFIG_FILE_KEY;
+import static org.apache.nifi.minifi.bootstrap.configuration.ingestors.PullHttpChangeIngestor.OVERRIDE_SECURITY;
+import static org.apache.nifi.minifi.bootstrap.configuration.ingestors.PullHttpChangeIngestor.PULL_HTTP_BASE_KEY;
+import static org.apache.nifi.minifi.bootstrap.util.ConfigTransformer.generateConfigFiles;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.util.Properties;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.commons.io.IOUtils;
+import org.apache.nifi.minifi.bootstrap.RunMiNiFi;
+import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeException;
+import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeListener;
+import org.apache.nifi.minifi.bootstrap.util.ByteBufferInputStream;
+import org.apache.nifi.minifi.bootstrap.util.ConfigTransformer;
+import org.apache.nifi.minifi.commons.schema.ConfigSchema;
+import org.apache.nifi.minifi.commons.schema.common.ConvertableSchema;
+import org.apache.nifi.minifi.commons.schema.serialization.SchemaLoader;
+import org.slf4j.Logger;
+
+public class MiNiFiConfigurationChangeListener implements ConfigurationChangeListener {
+
+    private final RunMiNiFi runner;
+    private final Logger logger;
+    private final BootstrapFileProvider bootstrapFileProvider;
+
+    private static final ReentrantLock handlingLock = new ReentrantLock();
+
+    public MiNiFiConfigurationChangeListener(RunMiNiFi runner, Logger logger, BootstrapFileProvider bootstrapFileProvider) {
+        this.runner = runner;
+        this.logger = logger;
+        this.bootstrapFileProvider = bootstrapFileProvider;
+    }
+
+    @Override
+    public void handleChange(InputStream configInputStream) throws ConfigurationChangeException {
+        logger.info("Received notification of a change");
+
+        if (!handlingLock.tryLock()) {
+            throw new ConfigurationChangeException("Instance is already handling another change");
+        }
+        // Store the incoming stream as a byte array to be shared among components that need it
+        try(ByteArrayOutputStream bufferedConfigOs = new ByteArrayOutputStream()) {
+
+            Properties bootstrapProperties = bootstrapFileProvider.getBootstrapProperties();
+            File configFile = new File(bootstrapProperties.getProperty(MINIFI_CONFIG_FILE_KEY));
+
+            byte[] copyArray = new byte[1024];
+            int available;
+            while ((available = configInputStream.read(copyArray)) > 0) {
+                bufferedConfigOs.write(copyArray, 0, available);
+            }
+
+            File swapConfigFile = bootstrapFileProvider.getSwapFile();
+            logger.info("Persisting old configuration to {}", swapConfigFile.getAbsolutePath());
+
+            try (FileInputStream configFileInputStream = new FileInputStream(configFile)) {
+                Files.copy(configFileInputStream, swapConfigFile.toPath(), REPLACE_EXISTING);
+            }
+
+            persistBackNonFlowSectionsFromOriginalSchema(bufferedConfigOs.toByteArray(), bootstrapProperties, configFile);
+
+            // Create an input stream to feed to the config transformer
+            try (ByteArrayInputStream newConfigBais = new ByteArrayInputStream(IOUtils.toByteArray(new FileInputStream(configFile)))) {
+                newConfigBais.mark(-1);

Review Comment:
   Is it necessary to read the entire file and wrap it in a ByteArrayInputStream? Is there a reason for not using the FileInputStream directly?



##########
minifi/pom.xml:
##########
@@ -44,6 +44,7 @@ limitations under the License.
         <system.rules.version>1.16.1</system.rules.version>
         <aws.sdk.version>1.11.172</aws.sdk.version>
         <yammer.metrics.version>2.2.0</yammer.metrics.version>
+        <okhttp.version>4.9.3</okhttp.version>

Review Comment:
   This property can be removed



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/MiNiFiCommandSender.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 org.apache.nifi.minifi.bootstrap.service;
+
+import java.io.BufferedReader;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.ObjectInputStream;
+import java.io.OutputStream;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.net.SocketTimeoutException;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MiNiFiCommandSender {
+    private static final Logger LOGGER = LoggerFactory.getLogger(MiNiFiCommandSender.class);
+    private static final String PING_CMD = "PING";
+
+    private final MiNiFiParameters miNiFiParameters;
+
+    public MiNiFiCommandSender(MiNiFiParameters miNiFiParameters) {
+        this.miNiFiParameters = miNiFiParameters;
+    }
+
+    public Optional<String> sendCommand(String cmd, Integer port) throws IOException {
+        Optional<String> response = Optional.empty();
+
+        if (port == null) {
+            LOGGER.info("Apache MiNiFi is not currently running");
+            return response;
+        }
+
+        try (Socket socket = new Socket()) {
+            LOGGER.debug("Connecting to MiNiFi instance");
+            socket.setSoTimeout(10000);
+            socket.connect(new InetSocketAddress("localhost", port));
+            LOGGER.debug("Established connection to MiNiFi instance.");
+
+            LOGGER.debug("Sending {} Command to port {}", cmd, port);
+            OutputStream out = socket.getOutputStream();
+            out.write((cmd + " " + miNiFiParameters.getSecretKey() + "\n").getBytes(StandardCharsets.UTF_8));
+            out.flush();
+            socket.shutdownOutput();
+
+            InputStream in = socket.getInputStream();
+            StringBuilder sb = new StringBuilder();
+            int numLines = 0;
+            try (BufferedReader reader = new BufferedReader(new InputStreamReader(in))) {
+                String line;
+                while ((line = reader.readLine()) != null) {
+                    if (numLines++ > 0) {
+                        sb.append("\n");
+                    }
+                    sb.append(line);
+                }
+            }
+
+            String responseString = sb.toString().trim();
+
+            LOGGER.debug("Received response to {} command: {}", cmd, responseString);
+            response = Optional.of(responseString);
+        }
+        return response;
+    }
+
+    <T> T sendCommandForObject(String cmd, Integer port, String... extraParams) throws IOException {
+        T response;
+        try (Socket socket = new Socket("localhost", port)) {
+            OutputStream out = socket.getOutputStream();
+            String argsString = Arrays.stream(extraParams).collect(Collectors.joining(" ", " ", ""));
+            String commandWithArgs = cmd + " " + miNiFiParameters.getSecretKey() + argsString + "\n";
+            out.write((commandWithArgs).getBytes(StandardCharsets.UTF_8));
+            LOGGER.debug("Sending {} command to MiNiFi with the following args: [{}]", cmd, argsString);
+            out.flush();
+
+            socket.setSoTimeout(5000);
+            InputStream in = socket.getInputStream();
+
+            ObjectInputStream ois = new ObjectInputStream(in);
+            Object o = ois.readObject();
+            ois.close();
+            out.close();
+            response = castResponse(cmd, o);
+        } catch (EOFException | ClassNotFoundException | SocketTimeoutException e) {
+            String message = "Failed to get response for " + cmd + " Potentially due to the process currently being down (restarting or otherwise). Exception:" + e;
+            LOGGER.error(message);

Review Comment:
   The exception should be passed as a separate argument instead of being concatenated to the message string:
   ```suggestion
               String message = "Failed to get response for " + cmd + " Potentially due to the process currently being down (restarting or otherwise)";
               LOGGER.error(message, e);
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #6075: NIFI-9666 Implement base C2 client functionality

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on code in PR #6075:
URL: https://github.com/apache/nifi/pull/6075#discussion_r892997968


##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/command/StartRunner.java:
##########
@@ -0,0 +1,435 @@
+/*
+ * 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.nifi.minifi.bootstrap.command;
+
+import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CMD_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CONF_DIR_KEY;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.DEFAULT_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.MINIFI_CONFIG_FILE_KEY;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.STATUS_FILE_PID_KEY;
+import static org.apache.nifi.minifi.bootstrap.Status.ERROR;
+import static org.apache.nifi.minifi.bootstrap.Status.OK;
+import static org.apache.nifi.minifi.bootstrap.util.ConfigTransformer.generateConfigFiles;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.nifi.bootstrap.util.OSUtils;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.apache.nifi.minifi.bootstrap.RunMiNiFi;
+import org.apache.nifi.minifi.bootstrap.ShutdownHook;
+import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeException;
+import org.apache.nifi.minifi.bootstrap.exception.StartupFailureException;
+import org.apache.nifi.minifi.bootstrap.service.BootstrapFileProvider;
+import org.apache.nifi.minifi.bootstrap.service.CurrentPortProvider;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiListener;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiStdLogHandler;
+import org.apache.nifi.minifi.bootstrap.service.PeriodicStatusReporterManager;
+import org.apache.nifi.minifi.bootstrap.util.ProcessUtils;
+import org.apache.nifi.util.Tuple;
+
+public class StartRunner implements CommandRunner {
+    private static final String DEFAULT_JAVA_CMD = "java";
+    private static final String DEFAULT_LOG_DIR = "./logs";
+    private static final String DEFAULT_LIB_DIR = "./lib";
+    private static final String DEFAULT_CONF_DIR = "./conf";
+    private static final int STARTUP_WAIT_SECONDS = 60;
+    private static final String DEFAULT_CONFIG_FILE = DEFAULT_CONF_DIR + "/bootstrap.conf";
+
+    private final CurrentPortProvider currentPortProvider;
+    private final BootstrapFileProvider bootstrapFileProvider;
+    private final PeriodicStatusReporterManager periodicStatusReporterManager;
+    private final MiNiFiStdLogHandler miNiFiStdLogHandler;
+    private final MiNiFiParameters miNiFiParameters;
+    private final File bootstrapConfigFile;
+    private final Lock lock = new ReentrantLock();
+    private final Condition startupCondition = lock.newCondition();
+    private final RunMiNiFi runMiNiFi;
+    private volatile ShutdownHook shutdownHook;
+
+    public StartRunner(CurrentPortProvider currentPortProvider, BootstrapFileProvider bootstrapFileProvider,
+        PeriodicStatusReporterManager periodicStatusReporterManager, MiNiFiStdLogHandler miNiFiStdLogHandler, MiNiFiParameters miNiFiParameters, File bootstrapConfigFile,
+        RunMiNiFi runMiNiFi) {
+        this.currentPortProvider = currentPortProvider;
+        this.bootstrapFileProvider = bootstrapFileProvider;
+        this.periodicStatusReporterManager = periodicStatusReporterManager;
+        this.miNiFiStdLogHandler = miNiFiStdLogHandler;
+        this.miNiFiParameters = miNiFiParameters;
+        this.bootstrapConfigFile = bootstrapConfigFile;
+        this.runMiNiFi = runMiNiFi;
+    }
+
+    /**
+     * Starts (and restarts) MiNiFi process during the whole lifecycle of the bootstrap process.
+     * @param args the input arguments
+     * @return status code
+     */
+    @Override
+    public int runCommand(String[] args) {
+        try {
+            start();
+        } catch (Exception e) {
+            CMD_LOGGER.error("Exception happened during MiNiFi startup", e);
+            return ERROR.getStatusCode();
+        }
+        return OK.getStatusCode();
+    }
+
+    private void start() throws IOException, InterruptedException {
+        Integer port = currentPortProvider.getCurrentPort();
+        if (port != null) {
+            String alreadyRunningMessage = "Apache MiNiFi is already running";
+            CMD_LOGGER.info(alreadyRunningMessage + ", listening to Bootstrap on port {}", port);
+            return;
+        }
+
+        File prevLockFile = bootstrapFileProvider.getLockFile();
+        if (prevLockFile.exists() && !prevLockFile.delete()) {
+            CMD_LOGGER.warn("Failed to delete previous lock file {}; this file should be cleaned up manually", prevLockFile);
+        }
+
+        Properties bootstrapProperties = bootstrapFileProvider.getBootstrapProperties();
+        String confDir = bootstrapProperties.getProperty(CONF_DIR_KEY);
+        initConfigFiles(bootstrapProperties, confDir);
+
+        Tuple<ProcessBuilder, Process> tuple = startMiNiFi();
+        ProcessBuilder builder = tuple.getKey();
+        Process process = tuple.getValue();
+
+        try {
+            while (true) {
+                if (ProcessUtils.isAlive(process)) {
+                    handleReload();
+                } else {
+                    Runtime runtime = Runtime.getRuntime();
+                    try {
+                        runtime.removeShutdownHook(shutdownHook);
+                    } catch (IllegalStateException ise) {
+                        // happens when already shutting down
+                    }
+
+                    if (runMiNiFi.isAutoRestartNiFi() && needRestart()) {
+                        File reloadFile = bootstrapFileProvider.getReloadLockFile();
+                        if (reloadFile.exists()) {
+                            DEFAULT_LOGGER.info("Currently reloading configuration. Will wait to restart MiNiFi.");
+                            Thread.sleep(5000L);
+                            continue;
+                        }
+
+                        process = restartNifi(bootstrapProperties, confDir, builder, runtime);
+                        // failed to start process
+                        if (process == null) {
+                            return;
+                        }
+                    } else {
+                        return;
+                    }
+                }
+            }
+        } finally {
+            miNiFiStdLogHandler.shutdown();
+            runMiNiFi.shutdownChangeNotifier();
+            periodicStatusReporterManager.shutdownPeriodicStatusReporters();
+        }
+    }
+
+    private Process restartNifi(Properties bootstrapProperties, String confDir, ProcessBuilder builder, Runtime runtime) throws IOException {
+        Process process;
+        boolean previouslyStarted = runMiNiFi.isNiFiStarted();
+        if (!previouslyStarted) {
+            File swapConfigFile = bootstrapFileProvider.getSwapFile();
+            if (swapConfigFile.exists()) {
+                DEFAULT_LOGGER.info("Swap file exists, MiNiFi failed trying to change configuration. Reverting to old configuration.");
+
+                try {
+                    ByteBuffer tempConfigFile = generateConfigFiles(new FileInputStream(swapConfigFile), confDir, bootstrapProperties);
+                    runMiNiFi.getConfigFileReference().set(tempConfigFile.asReadOnlyBuffer());
+                } catch (ConfigurationChangeException e) {
+                    DEFAULT_LOGGER.error("The swap file is malformed, unable to restart from prior state. Will not attempt to restart MiNiFi. Swap File should be cleaned up manually.");
+                    return null;
+                }
+
+                Files.copy(swapConfigFile.toPath(), Paths.get(bootstrapProperties.getProperty(MINIFI_CONFIG_FILE_KEY)), REPLACE_EXISTING);
+
+                DEFAULT_LOGGER.info("Replacing config file with swap file and deleting swap file");
+                if (!swapConfigFile.delete()) {
+                    DEFAULT_LOGGER.warn("The swap file failed to delete after replacing using it to revert to the old configuration. It should be cleaned up manually.");
+                }
+                runMiNiFi.setReloading(false);
+            } else {
+                DEFAULT_LOGGER.info("MiNiFi either never started or failed to restart. Will not attempt to restart MiNiFi");
+                return null;
+            }
+        } else {
+            runMiNiFi.setNiFiStarted(false);
+        }
+
+        miNiFiParameters.setSecretKey(null);
+        process = builder.start();
+        miNiFiStdLogHandler.initLogging(process);
+
+        Long pid = OSUtils.getProcessId(process, DEFAULT_LOGGER);
+        if (pid != null) {
+            miNiFiParameters.setMinifiPid(pid);
+            Properties minifiProps = new Properties();
+            minifiProps.setProperty(STATUS_FILE_PID_KEY, String.valueOf(pid));
+            bootstrapFileProvider.saveStatusProperties(minifiProps);
+        }
+
+        shutdownHook = new ShutdownHook(runMiNiFi, miNiFiStdLogHandler);
+        runtime.addShutdownHook(shutdownHook);
+
+        boolean started = waitForStart();
+
+        if (started) {
+            DEFAULT_LOGGER.info("Successfully spawned the thread to start Apache MiNiFi{}", (pid == null ? "" : " with PID " + pid));
+        } else {
+            DEFAULT_LOGGER.error("Apache MiNiFi does not appear to have started");
+        }
+        return process;
+    }
+
+    private boolean needRestart() throws IOException {
+        boolean needRestart = true;
+        File statusFile = bootstrapFileProvider.getStatusFile();
+        if (!statusFile.exists()) {
+            DEFAULT_LOGGER.info("Status File no longer exists. Will not restart MiNiFi");
+            return false;
+        }
+
+        File lockFile = bootstrapFileProvider.getLockFile();
+        if (lockFile.exists()) {
+            DEFAULT_LOGGER.info("A shutdown was initiated. Will not restart MiNiFi");
+            return false;
+        }
+        return needRestart;
+    }
+
+    private void handleReload() {
+        try {
+            Thread.sleep(1000L);
+            if (runMiNiFi.getReloading() && runMiNiFi.isNiFiStarted()) {
+                File swapConfigFile = bootstrapFileProvider.getSwapFile();
+                if (swapConfigFile.exists()) {
+                    DEFAULT_LOGGER.info("MiNiFi has finished reloading successfully and swap file exists. Deleting old configuration.");
+
+                    if (swapConfigFile.delete()) {
+                        DEFAULT_LOGGER.info("Swap file was successfully deleted.");
+                    } else {
+                        DEFAULT_LOGGER.error("Swap file was not deleted. It should be deleted manually.");
+                    }
+                }
+                runMiNiFi.setReloading(false);
+            }
+        } catch (InterruptedException ie) {
+        }
+    }
+
+    private void initConfigFiles(Properties bootstrapProperties, String confDir) throws IOException {
+        File configFile = new File(bootstrapProperties.getProperty(MINIFI_CONFIG_FILE_KEY));
+        try (InputStream inputStream = new FileInputStream(configFile)) {
+            ByteBuffer tempConfigFile = generateConfigFiles(inputStream, confDir, bootstrapProperties);
+            runMiNiFi.getConfigFileReference().set(tempConfigFile.asReadOnlyBuffer());
+        } catch (FileNotFoundException e) {
+            String fileNotFoundMessage = "The config file defined in " + MINIFI_CONFIG_FILE_KEY + " does not exists.";
+            DEFAULT_LOGGER.error(fileNotFoundMessage, e);
+            throw new StartupFailureException(fileNotFoundMessage);
+        } catch (ConfigurationChangeException e) {
+            String malformedConfigFileMessage = "The config file is malformed, unable to start.";
+            DEFAULT_LOGGER.error(malformedConfigFileMessage, e);
+            throw new StartupFailureException(malformedConfigFileMessage);
+        }
+    }
+
+    private Tuple<ProcessBuilder, Process> startMiNiFi() throws IOException {
+        ProcessBuilder builder = new ProcessBuilder();
+
+        Properties props = bootstrapFileProvider.getBootstrapProperties();
+        File bootstrapConfigAbsoluteFile = bootstrapConfigFile.getAbsoluteFile();
+        File binDir = bootstrapConfigAbsoluteFile.getParentFile();
+
+        File workingDir = Optional.ofNullable(props.getProperty("working.dir"))
+            .map(File::new)
+            .orElse(binDir.getParentFile());
+
+        builder.directory(workingDir);
+
+        String minifiLogDir = System.getProperty("org.apache.nifi.minifi.bootstrap.config.log.dir", DEFAULT_LOG_DIR).trim();
+        File libDir = getFile(props.getProperty("lib.dir", DEFAULT_LIB_DIR).trim(), workingDir);
+        File confDir = getFile(props.getProperty(CONF_DIR_KEY, DEFAULT_CONF_DIR).trim(), workingDir);
+
+        String minifiPropsFilename = props.getProperty("props.file");
+        if (minifiPropsFilename == null) {
+            if (confDir.exists()) {
+                minifiPropsFilename = new File(confDir, "nifi.properties").getAbsolutePath();
+            } else {
+                minifiPropsFilename = DEFAULT_CONFIG_FILE;
+            }
+        }
+
+        minifiPropsFilename = minifiPropsFilename.trim();
+
+        List<String> javaAdditionalArgs = new ArrayList<>();
+        for (Entry<Object, Object> entry : props.entrySet()) {
+            String key = (String) entry.getKey();
+            String value = (String) entry.getValue();
+
+            if (key.startsWith("java.arg")) {
+                javaAdditionalArgs.add(value);
+            }
+        }
+
+        File[] libFiles = libDir.listFiles((dir, filename) -> filename.toLowerCase().endsWith(".jar"));
+
+        if (libFiles == null || libFiles.length == 0) {
+            throw new RuntimeException("Could not find lib directory at " + libDir.getAbsolutePath());
+        }
+
+        File[] confFiles = confDir.listFiles();
+        if (confFiles == null || confFiles.length == 0) {
+            throw new RuntimeException("Could not find conf directory at " + confDir.getAbsolutePath());
+        }
+
+        List<String> cpFiles = new ArrayList<>(confFiles.length + libFiles.length);
+        cpFiles.add(confDir.getAbsolutePath());
+        for (File file : libFiles) {
+            cpFiles.add(file.getAbsolutePath());
+        }
+
+        StringBuilder classPathBuilder = new StringBuilder();
+        for (int i = 0; i < cpFiles.size(); i++) {
+            String filename = cpFiles.get(i);
+            classPathBuilder.append(filename);
+            if (i < cpFiles.size() - 1) {
+                classPathBuilder.append(File.pathSeparatorChar);
+            }
+        }
+
+        String classPath = classPathBuilder.toString();
+        String javaCmd = props.getProperty("java");
+        if (javaCmd == null) {
+            javaCmd = DEFAULT_JAVA_CMD;
+        }
+        if (javaCmd.equals(DEFAULT_JAVA_CMD)) {
+            String javaHome = System.getenv("JAVA_HOME");
+            if (javaHome != null) {
+                String fileExtension = isWindows() ? ".exe" : "";

Review Comment:
   I was thinking of checking for the existence of the `java` or `java.exe` based on the full path to `JAVA_HOME`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #6075: NIFI-9666 Implement base C2 client functionality

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on code in PR #6075:
URL: https://github.com/apache/nifi/pull/6075#discussion_r892997077


##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/WindowsService.java:
##########
@@ -17,26 +17,20 @@
 package org.apache.nifi.minifi.bootstrap;
 
 import java.io.IOException;
-import java.io.File;
+import org.apache.nifi.minifi.bootstrap.service.BootstrapFileProvider;
 
 public class WindowsService {

Review Comment:
   Thanks, it might be helpful to add a comment about the purpose, since the implementation is not platform specific.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #6075: NIFI-9666 Implement base C2 client functionality

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on code in PR #6075:
URL: https://github.com/apache/nifi/pull/6075#discussion_r892995803


##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/util/ProcessUtils.java:
##########
@@ -0,0 +1,131 @@
+/*
+ * 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.nifi.minifi.bootstrap.util;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.Reader;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ProcessUtils {

Review Comment:
   Thanks for the reply @ferencerdei. Renaming it for now seems acceptable, but I recommend opening a Jira issue to implement a Windows-based solution if Windows is intended to be supported.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] ferencerdei commented on pull request #6075: NIFI-9666 Implement base C2 client functionality

Posted by GitBox <gi...@apache.org>.
ferencerdei commented on PR #6075:
URL: https://github.com/apache/nifi/pull/6075#issuecomment-1152091415

   > It appears something has changed with the dockermaven Docker image for MiNIFi that makes it so it does not respond to interrupt signals.
   > 
   > When running the docker image in the foreground, ctrl-c (^C), should send an interrupt to stop the container and have it shut down, like so:
   > 
   > ```
   > Received trapped signal, beginning shutdown...
   > ```
   > 
   > This does not work for me on this PR branch (it does for the latest stable release version of MiNiFi Java).
   > 
   > Also, possibly relatedly, but I only see the bootstrap logs in the Docker output for this branch, whereas we should see bootstrap and app log output, correct? Again, my expectation is that this PR should not change that behavior for the minifi Docker image. I think there may have been some unintentional regressions introduced.
   
   Thanks for checking it. I've fixed the startup shell script so now the bootstrap process receives the SIGTERM signal.
   
   Regarding the logging: the output is configurable via the logback.xml, if you want to display both bootstrap and minifi process logs, you need to add the console appender to both and mount the file to the config dir. There was no change around this, it works the same with the legacy minifi.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #6075: NIFI-9666 Implement base C2 client functionality

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on code in PR #6075:
URL: https://github.com/apache/nifi/pull/6075#discussion_r890244649


##########
c2/c2-client-bundle/c2-client-api/src/main/java/org/apache/nifi/c2/client/api/C2Properties.java:
##########
@@ -0,0 +1,141 @@
+/*
+ * 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.nifi.c2.client.api;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Properties;
+import java.util.concurrent.TimeUnit;
+
+public class C2Properties {
+
+    private static final Logger logger = LoggerFactory.getLogger(C2Properties.class);
+
+    public static final String NIFI_PREFIX = "nifi.";
+
+    public static final String C2_PROPERTY_BASE = NIFI_PREFIX + "c2.";
+    public static final String C2_ENABLE_KEY = C2_PROPERTY_BASE + "enable";
+    public static final String C2_AGENT_PROTOCOL_KEY = C2_PROPERTY_BASE + "agent.protocol.class";
+    public static final String C2_REST_URL_KEY = C2_PROPERTY_BASE + "rest.url";
+    public static final String C2_REST_URL_ACK_KEY = C2_PROPERTY_BASE + "rest.url.ack";
+    public static final String C2_AGENT_HEARTBEAT_PERIOD_KEY = C2_PROPERTY_BASE + "agent.heartbeat.period";
+    public static final String C2_AGENT_CLASS_KEY = C2_PROPERTY_BASE + "agent.class";
+    public static final String C2_AGENT_IDENTIFIER_KEY = C2_PROPERTY_BASE + "agent.identifier";
+
+    public static final String C2_ROOT_CLASS_DEFINITIONS_KEY = C2_PROPERTY_BASE + "root.class.definitions";
+    public static final String C2_METRICS_NAME_KEY = C2_ROOT_CLASS_DEFINITIONS_KEY + ".metrics.name";
+    public static final String C2_METRICS_METRICS_KEY = C2_ROOT_CLASS_DEFINITIONS_KEY + ".metrics.metrics";
+    public static final String C2_METRICS_METRICS_TYPED_METRICS_NAME_KEY = C2_ROOT_CLASS_DEFINITIONS_KEY + ".metrics.metrics.typedmetrics.name";
+    public static final String C2_METRICS_METRICS_QUEUED_METRICS_NAME_KEY = C2_ROOT_CLASS_DEFINITIONS_KEY + ".metrics.metrics.queuemetrics.name";
+    public static final String C2_METRICS_METRICS_QUEUE_METRICS_CLASSES_KEY = C2_ROOT_CLASS_DEFINITIONS_KEY + ".metrics.metrics.queuemetrics.classes";
+    public static final String C2_METRICS_METRICS_TYPED_METRICS_CLASSES_KEY = C2_ROOT_CLASS_DEFINITIONS_KEY + ".metrics.metrics.typedmetrics.classes";
+    public static final String C2_METRICS_METRICS_PROCESSOR_METRICS_NAME_KEY = C2_ROOT_CLASS_DEFINITIONS_KEY + ".metrics.metrics.processorMetrics.name";
+    public static final String C2_METRICS_METRICS_PROCESSOR_METRICS_CLASSES_KEY = C2_ROOT_CLASS_DEFINITIONS_KEY + ".metrics.metrics.processorMetrics.classes";
+
+    /* REST Heartbeat Reporter Security Properties */
+    private static final String C2_REST_SECURITY_BASE_KEY = NIFI_PREFIX + "c2.security";
+    public static final String TRUSTSTORE_LOCATION_KEY = C2_REST_SECURITY_BASE_KEY + ".truststore.location";
+    public static final String TRUSTSTORE_PASSWORD_KEY = C2_REST_SECURITY_BASE_KEY + ".truststore.password";
+    public static final String TRUSTSTORE_TYPE_KEY = C2_REST_SECURITY_BASE_KEY + ".truststore.type";
+    public static final String KEYSTORE_LOCATION_KEY = C2_REST_SECURITY_BASE_KEY + ".keystore.location";
+    public static final String KEYSTORE_PASSWORD_KEY = C2_REST_SECURITY_BASE_KEY + ".keystore.password";
+    public static final String KEYSTORE_TYPE_KEY = C2_REST_SECURITY_BASE_KEY + ".keystore.type";
+    public static final String NEED_CLIENT_AUTH_KEY = C2_REST_SECURITY_BASE_KEY + ".need.client.auth";
+
+    // Defaults
+    // Heartbeat period of 1 second
+    public static final long C2_AGENT_DEFAULT_HEARTBEAT_PERIOD = TimeUnit.SECONDS.toMillis(1);

Review Comment:
   Thanks for the reply and consideration, refactoring to an enum-based approach in a separate issue sounds good.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #6075: NIFI-9666 Implement base C2 client functionality

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on code in PR #6075:
URL: https://github.com/apache/nifi/pull/6075#discussion_r892998570


##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/MiNiFiCommandSender.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 org.apache.nifi.minifi.bootstrap.service;
+
+import java.io.BufferedReader;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.ObjectInputStream;
+import java.io.OutputStream;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.net.SocketTimeoutException;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MiNiFiCommandSender {
+    private static final Logger LOGGER = LoggerFactory.getLogger(MiNiFiCommandSender.class);
+    private static final String PING_CMD = "PING";
+
+    private final MiNiFiParameters miNiFiParameters;
+
+    public MiNiFiCommandSender(MiNiFiParameters miNiFiParameters) {
+        this.miNiFiParameters = miNiFiParameters;
+    }
+
+    public Optional<String> sendCommand(String cmd, Integer port) throws IOException {
+        Optional<String> response = Optional.empty();
+
+        if (port == null) {
+            LOGGER.info("Apache MiNiFi is not currently running");
+            return response;
+        }
+
+        try (Socket socket = new Socket()) {
+            LOGGER.debug("Connecting to MiNiFi instance");
+            socket.setSoTimeout(10000);
+            socket.connect(new InetSocketAddress("localhost", port));
+            LOGGER.debug("Established connection to MiNiFi instance.");
+
+            LOGGER.debug("Sending {} Command to port {}", cmd, port);
+            OutputStream out = socket.getOutputStream();
+            out.write((cmd + " " + miNiFiParameters.getSecretKey() + "\n").getBytes(StandardCharsets.UTF_8));
+            out.flush();
+            socket.shutdownOutput();
+
+            InputStream in = socket.getInputStream();
+            StringBuilder sb = new StringBuilder();
+            int numLines = 0;
+            try (BufferedReader reader = new BufferedReader(new InputStreamReader(in))) {
+                String line;
+                while ((line = reader.readLine()) != null) {
+                    if (numLines++ > 0) {
+                        sb.append("\n");
+                    }
+                    sb.append(line);
+                }
+            }
+
+            String responseString = sb.toString().trim();
+
+            LOGGER.debug("Received response to {} command: {}", cmd, responseString);
+            response = Optional.of(responseString);
+        }
+        return response;
+    }
+
+    <T> T sendCommandForObject(String cmd, Integer port, String... extraParams) throws IOException {
+        T response;
+        try (Socket socket = new Socket("localhost", port)) {
+            OutputStream out = socket.getOutputStream();
+            String argsString = Arrays.stream(extraParams).collect(Collectors.joining(" ", " ", ""));
+            String commandWithArgs = cmd + " " + miNiFiParameters.getSecretKey() + argsString + "\n";
+            out.write((commandWithArgs).getBytes(StandardCharsets.UTF_8));
+            LOGGER.debug("Sending {} command to MiNiFi with the following args: [{}]", cmd, argsString);
+            out.flush();
+
+            socket.setSoTimeout(5000);
+            InputStream in = socket.getInputStream();
+
+            ObjectInputStream ois = new ObjectInputStream(in);
+            Object o = ois.readObject();

Review Comment:
   Thanks for addressing this issue!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] bejancsaba commented on a diff in pull request #6075: NIFI-9666 Implement base C2 client functionality

Posted by GitBox <gi...@apache.org>.
bejancsaba commented on code in PR #6075:
URL: https://github.com/apache/nifi/pull/6075#discussion_r899970475


##########
minifi/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/bootstrap.conf:
##########
@@ -129,3 +128,36 @@ java.arg.7=-Djava.security.egd=file:/dev/urandom
 
 #Set headless mode by default
 java.arg.14=-Djava.awt.headless=true
+
+# MiNiFi Command & Control Configuration
+# C2 Properties
+# Enabling C2 Uncomment each of the following options
+#c2.enable=true
+## define protocol parameters
+#c2.rest.url=
+#c2.rest.url.ack=
+## c2 timeouts
+#c2.rest.connectionTimeout=5 sec
+#c2.rest.readTimeout=5 sec
+#c2.rest.callTimeout=10 sec
+## heartbeat in milliseconds
+#c2.agent.heartbeat.period=5000
+## define parameters about your agent
+#c2.agent.class=
+#c2.config.directory=./conf
+#c2.runtime.manifest.identifier=minifi
+#c2.runtime.type=minifi-java
+# Optional.  Defaults to a hardware based unique identifier
+#c2.agent.identifier=
+## Define TLS security properties for C2 communications
+#c2.security.truststore.location=
+#c2.security.truststore.password=
+#c2.security.truststore.type=JKS
+#c2.security.keystore.location=
+#c2.security.keystore.password=
+#c2.security.keystore.type=JKS
+#c2.security.need.client.auth=true

Review Comment:
   Removed, was not used



##########
nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/c2/C2NiFiProperties.java:
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.nifi.c2;
+
+import java.util.concurrent.TimeUnit;
+
+public class C2NiFiProperties {
+
+    public static final String C2_PREFIX = "c2.";
+
+    public static final String C2_ENABLE_KEY = C2_PREFIX + "enable";
+    public static final String C2_AGENT_PROTOCOL_KEY = C2_PREFIX + "agent.protocol.class";
+    public static final String C2_COAP_HOST_KEY = C2_PREFIX + "agent.coap.host";
+    public static final String C2_COAP_PORT_KEY = C2_PREFIX + "agent.coap.port";
+    public static final String C2_CONFIG_DIRECTORY_KEY = C2_PREFIX + "config.directory";
+    public static final String C2_RUNTIME_MANIFEST_IDENTIFIER_KEY = C2_PREFIX + "runtime.manifest.identifier";
+    public static final String C2_RUNTIME_TYPE_KEY = C2_PREFIX + "runtime.type";
+    public static final String C2_REST_URL_KEY = C2_PREFIX + "rest.url";
+    public static final String C2_REST_URL_ACK_KEY = C2_PREFIX + "rest.url.ack";
+    public static final String C2_ROOT_CLASSES_KEY = C2_PREFIX + "root.classes";
+    public static final String C2_AGENT_HEARTBEAT_PERIOD_KEY = C2_PREFIX + "agent.heartbeat.period";
+    public static final String C2_CONNECTION_TIMEOUT = C2_PREFIX + "rest.connectionTimeout";
+    public static final String C2_READ_TIMEOUT = C2_PREFIX + "rest.readTimeout";
+    public static final String C2_CALL_TIMEOUT = C2_PREFIX + "rest.callTimeout";
+    public static final String C2_AGENT_CLASS_KEY = C2_PREFIX + "agent.class";
+    public static final String C2_AGENT_IDENTIFIER_KEY = C2_PREFIX + "agent.identifier";
+
+    public static final String C2_ROOT_CLASS_DEFINITIONS_KEY = C2_PREFIX + "root.class.definitions";
+    public static final String C2_METRICS_NAME_KEY = C2_ROOT_CLASS_DEFINITIONS_KEY + ".metrics.name";
+    public static final String C2_METRICS_METRICS_KEY = C2_ROOT_CLASS_DEFINITIONS_KEY + ".metrics.metrics";
+    public static final String C2_METRICS_METRICS_TYPED_METRICS_NAME_KEY = C2_ROOT_CLASS_DEFINITIONS_KEY + ".metrics.metrics.typedmetrics.name";
+    public static final String C2_METRICS_METRICS_QUEUED_METRICS_NAME_KEY = C2_ROOT_CLASS_DEFINITIONS_KEY + ".metrics.metrics.queuemetrics.name";
+    public static final String C2_METRICS_METRICS_QUEUE_METRICS_CLASSES_KEY = C2_ROOT_CLASS_DEFINITIONS_KEY + ".metrics.metrics.queuemetrics.classes";
+    public static final String C2_METRICS_METRICS_TYPED_METRICS_CLASSES_KEY = C2_ROOT_CLASS_DEFINITIONS_KEY + ".metrics.metrics.typedmetrics.classes";
+    public static final String C2_METRICS_METRICS_PROCESSOR_METRICS_NAME_KEY = C2_ROOT_CLASS_DEFINITIONS_KEY + ".metrics.metrics.processorMetrics.name";
+    public static final String C2_METRICS_METRICS_PROCESSOR_METRICS_CLASSES_KEY = C2_ROOT_CLASS_DEFINITIONS_KEY + ".metrics.metrics.processorMetrics.classes";
+
+    /* C2 Client Security Properties */
+    private static final String C2_REST_SECURITY_BASE_KEY = C2_PREFIX + "security";
+    public static final String TRUSTSTORE_LOCATION_KEY = C2_REST_SECURITY_BASE_KEY + ".truststore.location";
+    public static final String TRUSTSTORE_PASSWORD_KEY = C2_REST_SECURITY_BASE_KEY + ".truststore.password";
+    public static final String TRUSTSTORE_TYPE_KEY = C2_REST_SECURITY_BASE_KEY + ".truststore.type";
+    public static final String KEYSTORE_LOCATION_KEY = C2_REST_SECURITY_BASE_KEY + ".keystore.location";
+    public static final String KEYSTORE_PASSWORD_KEY = C2_REST_SECURITY_BASE_KEY + ".keystore.password";
+    public static final String KEYSTORE_TYPE_KEY = C2_REST_SECURITY_BASE_KEY + ".keystore.type";
+    public static final String NEED_CLIENT_AUTH_KEY = C2_REST_SECURITY_BASE_KEY + ".need.client.auth";

Review Comment:
   Removed, was not used



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] kevdoran commented on a diff in pull request #6075: NIFI-9666 Implement base C2 client functionality

Posted by GitBox <gi...@apache.org>.
kevdoran commented on code in PR #6075:
URL: https://github.com/apache/nifi/pull/6075#discussion_r898375453


##########
c2/c2-client-bundle/c2-client-service/src/main/java/org/apache/nifi/c2/client/service/operation/UpdateConfigurationOperationHandler.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.nifi.c2.client.service.operation;
+
+import static org.apache.commons.lang3.StringUtils.EMPTY;
+import static org.apache.nifi.c2.protocol.api.OperandType.CONFIGURATION;
+import static org.apache.nifi.c2.protocol.api.OperationType.UPDATE;
+
+import java.net.URI;
+import java.util.Optional;
+import java.util.function.Function;
+import org.apache.nifi.c2.client.api.C2Client;
+import org.apache.nifi.c2.client.service.FlowIdHolder;
+import org.apache.nifi.c2.protocol.api.C2Operation;
+import org.apache.nifi.c2.protocol.api.C2OperationAck;
+import org.apache.nifi.c2.protocol.api.C2OperationState;
+import org.apache.nifi.c2.protocol.api.OperandType;
+import org.apache.nifi.c2.protocol.api.OperationType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class UpdateConfigurationOperationHandler implements C2OperationHandler {
+
+    private static final Logger logger = LoggerFactory.getLogger(UpdateConfigurationOperationHandler.class);
+
+    private static final String LOCATION = "location";
+
+    private final C2Client client;
+    private final Function<byte[], Boolean> updateFlow;
+    private final FlowIdHolder flowIdHolder;
+
+    public UpdateConfigurationOperationHandler(C2Client client, FlowIdHolder flowIdHolder, Function<byte[], Boolean> updateFlow) {
+        this.client = client;
+        this.updateFlow = updateFlow;
+        this.flowIdHolder = flowIdHolder;
+    }
+
+    @Override
+    public OperationType getOperationType() {
+        return UPDATE;
+    }
+
+    @Override
+    public OperandType getOperandType() {
+        return CONFIGURATION;
+    }
+
+    @Override
+    public C2OperationAck handle(C2Operation operation) {
+        String opIdentifier = Optional.ofNullable(operation.getIdentifier())
+            .orElse(EMPTY);
+        C2OperationAck operationAck = new C2OperationAck();
+        C2OperationState state = new C2OperationState();
+        operationAck.setOperationState(state);
+        operationAck.setOperationId(opIdentifier);
+
+        String updateLocation = Optional.ofNullable(operation.getArgs())
+            .map(map -> map.get(LOCATION))
+            .orElse(EMPTY);
+
+        String newFlowId = parseFlowId(updateLocation);
+        if (flowIdHolder.getFlowId() == null || !flowIdHolder.getFlowId().equals(newFlowId)) {
+            logger.info("Will perform flow update from {} for operation #{}. Previous flow id was {}, replacing with new id {}", updateLocation, opIdentifier,
+                flowIdHolder.getFlowId() == null ? "not set" : flowIdHolder.getFlowId(), newFlowId);
+        } else {
+            logger.info("Flow is current, no update is necessary...");
+        }
+
+        flowIdHolder.setFlowId(newFlowId);
+        byte[] updateContent = client.retrieveUpdateContent(updateLocation);
+        if (updateContent != null) {
+            if (updateFlow.apply(updateContent)) {
+                state.setState(C2OperationState.OperationState.FULLY_APPLIED);
+                logger.debug("Update configuration applied for operation #{}.", opIdentifier);
+            } else {
+                state.setState(C2OperationState.OperationState.NOT_APPLIED);
+                logger.error("Update resulted in error for operation #{}.", opIdentifier);
+            }
+        } else {
+            state.setState(C2OperationState.OperationState.NOT_APPLIED);
+            logger.error("Update content retrieval resulted in empty content so flow update was omitted for operation #{}.", opIdentifier);
+        }
+
+        return operationAck;
+    }
+
+    private String parseFlowId(String flowUpdateUrl) {
+        try {
+            URI flowUri = new URI(flowUpdateUrl);
+            String flowUriPath = flowUri.getPath();
+            String[] split = flowUriPath.split("/");
+            if (split.length > 4) {
+                return split[4];

Review Comment:
   @bejancsaba @adamdebreceni - we could probably extend the c2 protocol so that the flow metadata, such as ID, is passed as a key/value pair in the update operation from the server, so that getting it from the URL is no necessary at all.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] bejancsaba commented on pull request #6075: NIFI-9666 Implement base C2 client functionality

Posted by GitBox <gi...@apache.org>.
bejancsaba commented on PR #6075:
URL: https://github.com/apache/nifi/pull/6075#issuecomment-1157415064

   > I have finished my functional review of the c2 client capability added in this PR, and found it to be working as expected. Really awesome to see it in action, it performed wonderfully. Nice work all!
   > 
   > I only have one observation, which is that heartbeats are being logged at the info level by the OkHttpClient library being used:
   > 
   > ```
   > 2022-06-15 19:10:06,666 INFO [pool-23-thread-1] okhttp3.OkHttpClient --> POST http://host:port/contextPath/heartbeat (875608-byte body)
   > 2022-06-15 19:10:09,047 INFO [pool-23-thread-1] okhttp3.OkHttpClient <-- 200 OK http://host:port/contextPath/heartbeat (2380ms, 2-byte body)
   > ```
   > 
   > I feel that this is too noisy, and should be moved to the debug level. Perhaps error code responses should be an elevated level (if they are not already).
   
   Moved it to debug level, we have dedicated logging for covering errors in most cases. If that is ok from your side we will enhance that as needed in a follow up PR focusing more on testing and potential refactors.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] ferencerdei commented on a diff in pull request #6075: NIFI-9666 Implement base C2 client functionality

Posted by GitBox <gi...@apache.org>.
ferencerdei commented on code in PR #6075:
URL: https://github.com/apache/nifi/pull/6075#discussion_r891005901


##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/configuration/ConfigurationChangeCoordinator.java:
##########
@@ -99,16 +84,43 @@ public Collection<ListenerHandleResult> notifyListeners(ByteBuffer newConfig) {
                 result = new ListenerHandleResult(listener, ex);
             }
             listenerHandleResults.add(result);
-            logger.info("Listener notification result:" + result.toString());
+            LOGGER.info("Listener notification result: {}", result);
         }
         return listenerHandleResults;
     }
 
 
     @Override
-    public void close() throws IOException {
-        for (ChangeIngestor changeIngestor : changeIngestors) {
-            changeIngestor.close();
+    public void close() {
+        try {
+            for (ChangeIngestor changeIngestor : changeIngestors) {
+                changeIngestor.close();
+            }
+            changeIngestors.clear();
+        } catch (IOException e) {
+            LOGGER.warn("Could not successfully stop notifiers", e);
+        }
+    }
+
+    private void initialize() {
+        close();
+        // cleanup previously initialized ingestors
+        String ingestorsCsv = bootstrapProperties.getProperty(NOTIFIER_INGESTORS_KEY);
+
+        if (ingestorsCsv != null && !ingestorsCsv.isEmpty()) {
+            for (String ingestorClassname : ingestorsCsv.split(",")) {
+                ingestorClassname = ingestorClassname.trim();
+                try {
+                    Class<?> ingestorClass = Class.forName(ingestorClassname);
+                    ChangeIngestor changeIngestor = (ChangeIngestor) ingestorClass.newInstance();
+                    changeIngestor.initialize(bootstrapProperties, runMiNiFi, this);
+                    changeIngestors.add(changeIngestor);
+                    LOGGER.info("Initialized ingestor: {}", ingestorClassname);
+                } catch (Exception e) {
+                    LOGGER.error("Issue instantiating {} ingestor", ingestorClassname);
+                    LOGGER.error("Exception", e);

Review Comment:
   done



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/command/EnvRunner.java:
##########
@@ -0,0 +1,107 @@
+/*
+ * 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.nifi.minifi.bootstrap.command;
+
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CMD_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.DEFAULT_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.Status.ERROR;
+import static org.apache.nifi.minifi.bootstrap.Status.MINIFI_NOT_RUNNING;
+import static org.apache.nifi.minifi.bootstrap.Status.OK;
+
+import java.lang.reflect.Method;
+import java.util.Map.Entry;
+import java.util.Properties;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.apache.nifi.minifi.bootstrap.MiNiFiStatus;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiStatusProvider;
+
+public class EnvRunner implements CommandRunner {
+    private final MiNiFiParameters miNiFiParameters;
+    private final MiNiFiStatusProvider miNiFiStatusProvider;
+
+    public EnvRunner(MiNiFiParameters miNiFiParameters, MiNiFiStatusProvider miNiFiStatusProvider) {
+        this.miNiFiParameters = miNiFiParameters;
+        this.miNiFiStatusProvider = miNiFiStatusProvider;
+    }
+
+    /**
+     * Returns information about the MiNiFi's virtual machine.
+     * @param args the input arguments
+     * @return status code
+     */
+    @Override
+    public int runCommand(String[] args) {
+        return env();
+    }
+
+    private int env() {
+        MiNiFiStatus status = miNiFiStatusProvider.getStatus(miNiFiParameters.getMiNiFiPort(), miNiFiParameters.getMinifiPid());
+        if (status.getPid() == null) {
+            CMD_LOGGER.error("Apache MiNiFi is not running");
+            return MINIFI_NOT_RUNNING.getStatusCode();
+        }
+
+        Class<?> virtualMachineClass;
+        try {
+            virtualMachineClass = Class.forName("com.sun.tools.attach.VirtualMachine");
+        } catch (ClassNotFoundException cnfe) {
+            CMD_LOGGER.error("Seems tools.jar (Linux / Windows JDK) or classes.jar (Mac OS) is not available in classpath");
+            return ERROR.getStatusCode();
+        }
+
+        Method attachMethod;
+        Method detachMethod;
+        try {
+            attachMethod = virtualMachineClass.getMethod("attach", String.class);
+            detachMethod = virtualMachineClass.getDeclaredMethod("detach");
+        } catch (Exception e) {
+            CMD_LOGGER.error("Methods required for getting environment not available");
+            DEFAULT_LOGGER.error("Exception:", e);
+            return ERROR.getStatusCode();
+        }
+
+        Object virtualMachine;
+        try {
+            virtualMachine = attachMethod.invoke(null, status.getPid());
+        } catch (Exception e) {
+            CMD_LOGGER.error("Problem attaching to MiNiFi");
+            DEFAULT_LOGGER.error("Exception:", e);
+            return ERROR.getStatusCode();
+        }
+
+        try {
+            Method getSystemPropertiesMethod = virtualMachine.getClass().getMethod("getSystemProperties");
+
+            Properties sysProps = (Properties) getSystemPropertiesMethod.invoke(virtualMachine);

Review Comment:
   The bootstrap(commands) and the MiNiFi can run on different JVM, so with this method we return the MiNiFi's properties and not the bootstrap process. (This code is a port of RunNiFi)



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/command/StartRunner.java:
##########
@@ -0,0 +1,435 @@
+/*
+ * 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.nifi.minifi.bootstrap.command;
+
+import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CMD_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CONF_DIR_KEY;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.DEFAULT_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.MINIFI_CONFIG_FILE_KEY;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.STATUS_FILE_PID_KEY;
+import static org.apache.nifi.minifi.bootstrap.Status.ERROR;
+import static org.apache.nifi.minifi.bootstrap.Status.OK;
+import static org.apache.nifi.minifi.bootstrap.util.ConfigTransformer.generateConfigFiles;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.nifi.bootstrap.util.OSUtils;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.apache.nifi.minifi.bootstrap.RunMiNiFi;
+import org.apache.nifi.minifi.bootstrap.ShutdownHook;
+import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeException;
+import org.apache.nifi.minifi.bootstrap.exception.StartupFailureException;
+import org.apache.nifi.minifi.bootstrap.service.BootstrapFileProvider;
+import org.apache.nifi.minifi.bootstrap.service.CurrentPortProvider;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiListener;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiStdLogHandler;
+import org.apache.nifi.minifi.bootstrap.service.PeriodicStatusReporterManager;
+import org.apache.nifi.minifi.bootstrap.util.ProcessUtils;
+import org.apache.nifi.util.Tuple;
+
+public class StartRunner implements CommandRunner {
+    private static final String DEFAULT_JAVA_CMD = "java";
+    private static final String DEFAULT_LOG_DIR = "./logs";
+    private static final String DEFAULT_LIB_DIR = "./lib";
+    private static final String DEFAULT_CONF_DIR = "./conf";
+    private static final int STARTUP_WAIT_SECONDS = 60;
+    private static final String DEFAULT_CONFIG_FILE = DEFAULT_CONF_DIR + "/bootstrap.conf";
+
+    private final CurrentPortProvider currentPortProvider;
+    private final BootstrapFileProvider bootstrapFileProvider;
+    private final PeriodicStatusReporterManager periodicStatusReporterManager;
+    private final MiNiFiStdLogHandler miNiFiStdLogHandler;
+    private final MiNiFiParameters miNiFiParameters;
+    private final File bootstrapConfigFile;
+    private final Lock lock = new ReentrantLock();
+    private final Condition startupCondition = lock.newCondition();
+    private final RunMiNiFi runMiNiFi;
+    private volatile ShutdownHook shutdownHook;
+
+    public StartRunner(CurrentPortProvider currentPortProvider, BootstrapFileProvider bootstrapFileProvider,
+        PeriodicStatusReporterManager periodicStatusReporterManager, MiNiFiStdLogHandler miNiFiStdLogHandler, MiNiFiParameters miNiFiParameters, File bootstrapConfigFile,
+        RunMiNiFi runMiNiFi) {
+        this.currentPortProvider = currentPortProvider;
+        this.bootstrapFileProvider = bootstrapFileProvider;
+        this.periodicStatusReporterManager = periodicStatusReporterManager;
+        this.miNiFiStdLogHandler = miNiFiStdLogHandler;
+        this.miNiFiParameters = miNiFiParameters;
+        this.bootstrapConfigFile = bootstrapConfigFile;
+        this.runMiNiFi = runMiNiFi;
+    }
+
+    /**
+     * Starts (and restarts) MiNiFi process during the whole lifecycle of the bootstrap process.
+     * @param args the input arguments
+     * @return status code
+     */
+    @Override
+    public int runCommand(String[] args) {
+        try {
+            start();
+        } catch (Exception e) {
+            CMD_LOGGER.error("Exception happened during MiNiFi startup", e);
+            return ERROR.getStatusCode();
+        }
+        return OK.getStatusCode();
+    }
+
+    private void start() throws IOException, InterruptedException {
+        Integer port = currentPortProvider.getCurrentPort();
+        if (port != null) {
+            String alreadyRunningMessage = "Apache MiNiFi is already running";
+            CMD_LOGGER.info(alreadyRunningMessage + ", listening to Bootstrap on port {}", port);
+            return;
+        }
+
+        File prevLockFile = bootstrapFileProvider.getLockFile();
+        if (prevLockFile.exists() && !prevLockFile.delete()) {
+            CMD_LOGGER.warn("Failed to delete previous lock file {}; this file should be cleaned up manually", prevLockFile);
+        }
+
+        Properties bootstrapProperties = bootstrapFileProvider.getBootstrapProperties();
+        String confDir = bootstrapProperties.getProperty(CONF_DIR_KEY);
+        initConfigFiles(bootstrapProperties, confDir);
+
+        Tuple<ProcessBuilder, Process> tuple = startMiNiFi();
+        ProcessBuilder builder = tuple.getKey();
+        Process process = tuple.getValue();
+
+        try {
+            while (true) {
+                if (ProcessUtils.isAlive(process)) {
+                    handleReload();
+                } else {
+                    Runtime runtime = Runtime.getRuntime();
+                    try {
+                        runtime.removeShutdownHook(shutdownHook);
+                    } catch (IllegalStateException ise) {
+                        // happens when already shutting down
+                    }
+
+                    if (runMiNiFi.isAutoRestartNiFi() && needRestart()) {
+                        File reloadFile = bootstrapFileProvider.getReloadLockFile();
+                        if (reloadFile.exists()) {
+                            DEFAULT_LOGGER.info("Currently reloading configuration. Will wait to restart MiNiFi.");
+                            Thread.sleep(5000L);
+                            continue;
+                        }
+
+                        process = restartNifi(bootstrapProperties, confDir, builder, runtime);
+                        // failed to start process
+                        if (process == null) {
+                            return;
+                        }
+                    } else {
+                        return;
+                    }
+                }
+            }
+        } finally {
+            miNiFiStdLogHandler.shutdown();
+            runMiNiFi.shutdownChangeNotifier();
+            periodicStatusReporterManager.shutdownPeriodicStatusReporters();
+        }
+    }
+
+    private Process restartNifi(Properties bootstrapProperties, String confDir, ProcessBuilder builder, Runtime runtime) throws IOException {
+        Process process;
+        boolean previouslyStarted = runMiNiFi.isNiFiStarted();
+        if (!previouslyStarted) {
+            File swapConfigFile = bootstrapFileProvider.getSwapFile();
+            if (swapConfigFile.exists()) {
+                DEFAULT_LOGGER.info("Swap file exists, MiNiFi failed trying to change configuration. Reverting to old configuration.");
+
+                try {
+                    ByteBuffer tempConfigFile = generateConfigFiles(new FileInputStream(swapConfigFile), confDir, bootstrapProperties);
+                    runMiNiFi.getConfigFileReference().set(tempConfigFile.asReadOnlyBuffer());
+                } catch (ConfigurationChangeException e) {
+                    DEFAULT_LOGGER.error("The swap file is malformed, unable to restart from prior state. Will not attempt to restart MiNiFi. Swap File should be cleaned up manually.");
+                    return null;
+                }
+
+                Files.copy(swapConfigFile.toPath(), Paths.get(bootstrapProperties.getProperty(MINIFI_CONFIG_FILE_KEY)), REPLACE_EXISTING);
+
+                DEFAULT_LOGGER.info("Replacing config file with swap file and deleting swap file");
+                if (!swapConfigFile.delete()) {
+                    DEFAULT_LOGGER.warn("The swap file failed to delete after replacing using it to revert to the old configuration. It should be cleaned up manually.");
+                }
+                runMiNiFi.setReloading(false);
+            } else {
+                DEFAULT_LOGGER.info("MiNiFi either never started or failed to restart. Will not attempt to restart MiNiFi");
+                return null;
+            }
+        } else {
+            runMiNiFi.setNiFiStarted(false);
+        }
+
+        miNiFiParameters.setSecretKey(null);
+        process = builder.start();
+        miNiFiStdLogHandler.initLogging(process);
+
+        Long pid = OSUtils.getProcessId(process, DEFAULT_LOGGER);
+        if (pid != null) {
+            miNiFiParameters.setMinifiPid(pid);
+            Properties minifiProps = new Properties();
+            minifiProps.setProperty(STATUS_FILE_PID_KEY, String.valueOf(pid));
+            bootstrapFileProvider.saveStatusProperties(minifiProps);
+        }
+
+        shutdownHook = new ShutdownHook(runMiNiFi, miNiFiStdLogHandler);
+        runtime.addShutdownHook(shutdownHook);
+
+        boolean started = waitForStart();
+
+        if (started) {
+            DEFAULT_LOGGER.info("Successfully spawned the thread to start Apache MiNiFi{}", (pid == null ? "" : " with PID " + pid));
+        } else {
+            DEFAULT_LOGGER.error("Apache MiNiFi does not appear to have started");
+        }
+        return process;
+    }
+
+    private boolean needRestart() throws IOException {
+        boolean needRestart = true;
+        File statusFile = bootstrapFileProvider.getStatusFile();
+        if (!statusFile.exists()) {
+            DEFAULT_LOGGER.info("Status File no longer exists. Will not restart MiNiFi");
+            return false;
+        }
+
+        File lockFile = bootstrapFileProvider.getLockFile();
+        if (lockFile.exists()) {
+            DEFAULT_LOGGER.info("A shutdown was initiated. Will not restart MiNiFi");
+            return false;
+        }
+        return needRestart;
+    }
+
+    private void handleReload() {
+        try {
+            Thread.sleep(1000L);
+            if (runMiNiFi.getReloading() && runMiNiFi.isNiFiStarted()) {
+                File swapConfigFile = bootstrapFileProvider.getSwapFile();
+                if (swapConfigFile.exists()) {
+                    DEFAULT_LOGGER.info("MiNiFi has finished reloading successfully and swap file exists. Deleting old configuration.");
+
+                    if (swapConfigFile.delete()) {
+                        DEFAULT_LOGGER.info("Swap file was successfully deleted.");
+                    } else {
+                        DEFAULT_LOGGER.error("Swap file was not deleted. It should be deleted manually.");
+                    }
+                }
+                runMiNiFi.setReloading(false);
+            }
+        } catch (InterruptedException ie) {
+        }
+    }
+
+    private void initConfigFiles(Properties bootstrapProperties, String confDir) throws IOException {
+        File configFile = new File(bootstrapProperties.getProperty(MINIFI_CONFIG_FILE_KEY));
+        try (InputStream inputStream = new FileInputStream(configFile)) {
+            ByteBuffer tempConfigFile = generateConfigFiles(inputStream, confDir, bootstrapProperties);
+            runMiNiFi.getConfigFileReference().set(tempConfigFile.asReadOnlyBuffer());
+        } catch (FileNotFoundException e) {
+            String fileNotFoundMessage = "The config file defined in " + MINIFI_CONFIG_FILE_KEY + " does not exists.";
+            DEFAULT_LOGGER.error(fileNotFoundMessage, e);
+            throw new StartupFailureException(fileNotFoundMessage);
+        } catch (ConfigurationChangeException e) {
+            String malformedConfigFileMessage = "The config file is malformed, unable to start.";
+            DEFAULT_LOGGER.error(malformedConfigFileMessage, e);
+            throw new StartupFailureException(malformedConfigFileMessage);
+        }
+    }
+
+    private Tuple<ProcessBuilder, Process> startMiNiFi() throws IOException {
+        ProcessBuilder builder = new ProcessBuilder();
+
+        Properties props = bootstrapFileProvider.getBootstrapProperties();
+        File bootstrapConfigAbsoluteFile = bootstrapConfigFile.getAbsoluteFile();
+        File binDir = bootstrapConfigAbsoluteFile.getParentFile();
+
+        File workingDir = Optional.ofNullable(props.getProperty("working.dir"))
+            .map(File::new)
+            .orElse(binDir.getParentFile());
+
+        builder.directory(workingDir);
+
+        String minifiLogDir = System.getProperty("org.apache.nifi.minifi.bootstrap.config.log.dir", DEFAULT_LOG_DIR).trim();
+        File libDir = getFile(props.getProperty("lib.dir", DEFAULT_LIB_DIR).trim(), workingDir);
+        File confDir = getFile(props.getProperty(CONF_DIR_KEY, DEFAULT_CONF_DIR).trim(), workingDir);
+
+        String minifiPropsFilename = props.getProperty("props.file");
+        if (minifiPropsFilename == null) {
+            if (confDir.exists()) {
+                minifiPropsFilename = new File(confDir, "nifi.properties").getAbsolutePath();
+            } else {
+                minifiPropsFilename = DEFAULT_CONFIG_FILE;
+            }
+        }
+
+        minifiPropsFilename = minifiPropsFilename.trim();
+
+        List<String> javaAdditionalArgs = new ArrayList<>();
+        for (Entry<Object, Object> entry : props.entrySet()) {
+            String key = (String) entry.getKey();
+            String value = (String) entry.getValue();
+
+            if (key.startsWith("java.arg")) {
+                javaAdditionalArgs.add(value);
+            }
+        }
+
+        File[] libFiles = libDir.listFiles((dir, filename) -> filename.toLowerCase().endsWith(".jar"));
+
+        if (libFiles == null || libFiles.length == 0) {
+            throw new RuntimeException("Could not find lib directory at " + libDir.getAbsolutePath());
+        }
+
+        File[] confFiles = confDir.listFiles();
+        if (confFiles == null || confFiles.length == 0) {
+            throw new RuntimeException("Could not find conf directory at " + confDir.getAbsolutePath());
+        }
+
+        List<String> cpFiles = new ArrayList<>(confFiles.length + libFiles.length);
+        cpFiles.add(confDir.getAbsolutePath());
+        for (File file : libFiles) {
+            cpFiles.add(file.getAbsolutePath());
+        }
+
+        StringBuilder classPathBuilder = new StringBuilder();
+        for (int i = 0; i < cpFiles.size(); i++) {
+            String filename = cpFiles.get(i);
+            classPathBuilder.append(filename);
+            if (i < cpFiles.size() - 1) {
+                classPathBuilder.append(File.pathSeparatorChar);
+            }
+        }
+
+        String classPath = classPathBuilder.toString();
+        String javaCmd = props.getProperty("java");
+        if (javaCmd == null) {
+            javaCmd = DEFAULT_JAVA_CMD;
+        }
+        if (javaCmd.equals(DEFAULT_JAVA_CMD)) {
+            String javaHome = System.getenv("JAVA_HOME");
+            if (javaHome != null) {
+                String fileExtension = isWindows() ? ".exe" : "";
+                File javaFile = new File(javaHome + File.separatorChar + "bin"
+                    + File.separatorChar + "java" + fileExtension);
+                if (javaFile.exists() && javaFile.canExecute()) {
+                    javaCmd = javaFile.getAbsolutePath();
+                }
+            }
+        }
+
+        MiNiFiListener listener = new MiNiFiListener();
+        int listenPort = listener.start(runMiNiFi);
+
+        List<String> cmd = new ArrayList<>();
+
+        cmd.add(javaCmd);
+        cmd.add("-classpath");
+        cmd.add(classPath);
+        cmd.addAll(javaAdditionalArgs);
+        cmd.add("-Dnifi.properties.file.path=" + minifiPropsFilename);
+        cmd.add("-Dnifi.bootstrap.listen.port=" + listenPort);
+        cmd.add("-Dapp=MiNiFi");
+        cmd.add("-Dorg.apache.nifi.minifi.bootstrap.config.log.dir="+minifiLogDir);

Review Comment:
   done



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/WindowsService.java:
##########
@@ -17,26 +17,20 @@
 package org.apache.nifi.minifi.bootstrap;
 
 import java.io.IOException;
-import java.io.File;
+import org.apache.nifi.minifi.bootstrap.service.BootstrapFileProvider;
 
 public class WindowsService {

Review Comment:
   This class is used only in windows when the MiNiFi is installed through the install-service.bat.
   It uses the apache commons daemon which requires a class with a start and a stop method.



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/MiNiFiCommandSender.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 org.apache.nifi.minifi.bootstrap.service;
+
+import java.io.BufferedReader;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.ObjectInputStream;
+import java.io.OutputStream;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.net.SocketTimeoutException;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MiNiFiCommandSender {
+    private static final Logger LOGGER = LoggerFactory.getLogger(MiNiFiCommandSender.class);
+    private static final String PING_CMD = "PING";
+
+    private final MiNiFiParameters miNiFiParameters;
+
+    public MiNiFiCommandSender(MiNiFiParameters miNiFiParameters) {
+        this.miNiFiParameters = miNiFiParameters;
+    }
+
+    public Optional<String> sendCommand(String cmd, Integer port) throws IOException {
+        Optional<String> response = Optional.empty();
+
+        if (port == null) {
+            LOGGER.info("Apache MiNiFi is not currently running");
+            return response;
+        }
+
+        try (Socket socket = new Socket()) {
+            LOGGER.debug("Connecting to MiNiFi instance");
+            socket.setSoTimeout(10000);
+            socket.connect(new InetSocketAddress("localhost", port));
+            LOGGER.debug("Established connection to MiNiFi instance.");
+
+            LOGGER.debug("Sending {} Command to port {}", cmd, port);
+            OutputStream out = socket.getOutputStream();
+            out.write((cmd + " " + miNiFiParameters.getSecretKey() + "\n").getBytes(StandardCharsets.UTF_8));

Review Comment:
   done



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/BootstrapCodec.java:
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.nifi.minifi.bootstrap.service;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.util.Arrays;
+import org.apache.nifi.minifi.bootstrap.RunMiNiFi;
+import org.apache.nifi.minifi.bootstrap.exception.InvalidCommandException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class BootstrapCodec {
+
+    private static final String TRUE = "true";
+    private static final String FALSE = "false";

Review Comment:
   done



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/command/EnvRunner.java:
##########
@@ -0,0 +1,107 @@
+/*
+ * 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.nifi.minifi.bootstrap.command;
+
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CMD_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.DEFAULT_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.Status.ERROR;
+import static org.apache.nifi.minifi.bootstrap.Status.MINIFI_NOT_RUNNING;
+import static org.apache.nifi.minifi.bootstrap.Status.OK;
+
+import java.lang.reflect.Method;
+import java.util.Map.Entry;
+import java.util.Properties;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.apache.nifi.minifi.bootstrap.MiNiFiStatus;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiStatusProvider;
+
+public class EnvRunner implements CommandRunner {
+    private final MiNiFiParameters miNiFiParameters;
+    private final MiNiFiStatusProvider miNiFiStatusProvider;
+
+    public EnvRunner(MiNiFiParameters miNiFiParameters, MiNiFiStatusProvider miNiFiStatusProvider) {
+        this.miNiFiParameters = miNiFiParameters;
+        this.miNiFiStatusProvider = miNiFiStatusProvider;
+    }
+
+    /**
+     * Returns information about the MiNiFi's virtual machine.
+     * @param args the input arguments
+     * @return status code
+     */
+    @Override
+    public int runCommand(String[] args) {
+        return env();
+    }
+
+    private int env() {
+        MiNiFiStatus status = miNiFiStatusProvider.getStatus(miNiFiParameters.getMiNiFiPort(), miNiFiParameters.getMinifiPid());
+        if (status.getPid() == null) {
+            CMD_LOGGER.error("Apache MiNiFi is not running");
+            return MINIFI_NOT_RUNNING.getStatusCode();
+        }
+
+        Class<?> virtualMachineClass;
+        try {
+            virtualMachineClass = Class.forName("com.sun.tools.attach.VirtualMachine");
+        } catch (ClassNotFoundException cnfe) {
+            CMD_LOGGER.error("Seems tools.jar (Linux / Windows JDK) or classes.jar (Mac OS) is not available in classpath");
+            return ERROR.getStatusCode();
+        }
+
+        Method attachMethod;
+        Method detachMethod;
+        try {
+            attachMethod = virtualMachineClass.getMethod("attach", String.class);
+            detachMethod = virtualMachineClass.getDeclaredMethod("detach");
+        } catch (Exception e) {
+            CMD_LOGGER.error("Methods required for getting environment not available");
+            DEFAULT_LOGGER.error("Exception:", e);

Review Comment:
   done



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/command/StartRunner.java:
##########
@@ -0,0 +1,435 @@
+/*
+ * 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.nifi.minifi.bootstrap.command;
+
+import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CMD_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CONF_DIR_KEY;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.DEFAULT_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.MINIFI_CONFIG_FILE_KEY;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.STATUS_FILE_PID_KEY;
+import static org.apache.nifi.minifi.bootstrap.Status.ERROR;
+import static org.apache.nifi.minifi.bootstrap.Status.OK;
+import static org.apache.nifi.minifi.bootstrap.util.ConfigTransformer.generateConfigFiles;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.nifi.bootstrap.util.OSUtils;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.apache.nifi.minifi.bootstrap.RunMiNiFi;
+import org.apache.nifi.minifi.bootstrap.ShutdownHook;
+import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeException;
+import org.apache.nifi.minifi.bootstrap.exception.StartupFailureException;
+import org.apache.nifi.minifi.bootstrap.service.BootstrapFileProvider;
+import org.apache.nifi.minifi.bootstrap.service.CurrentPortProvider;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiListener;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiStdLogHandler;
+import org.apache.nifi.minifi.bootstrap.service.PeriodicStatusReporterManager;
+import org.apache.nifi.minifi.bootstrap.util.ProcessUtils;
+import org.apache.nifi.util.Tuple;
+
+public class StartRunner implements CommandRunner {
+    private static final String DEFAULT_JAVA_CMD = "java";
+    private static final String DEFAULT_LOG_DIR = "./logs";
+    private static final String DEFAULT_LIB_DIR = "./lib";
+    private static final String DEFAULT_CONF_DIR = "./conf";
+    private static final int STARTUP_WAIT_SECONDS = 60;
+    private static final String DEFAULT_CONFIG_FILE = DEFAULT_CONF_DIR + "/bootstrap.conf";
+
+    private final CurrentPortProvider currentPortProvider;
+    private final BootstrapFileProvider bootstrapFileProvider;
+    private final PeriodicStatusReporterManager periodicStatusReporterManager;
+    private final MiNiFiStdLogHandler miNiFiStdLogHandler;
+    private final MiNiFiParameters miNiFiParameters;
+    private final File bootstrapConfigFile;
+    private final Lock lock = new ReentrantLock();
+    private final Condition startupCondition = lock.newCondition();
+    private final RunMiNiFi runMiNiFi;
+    private volatile ShutdownHook shutdownHook;
+
+    public StartRunner(CurrentPortProvider currentPortProvider, BootstrapFileProvider bootstrapFileProvider,
+        PeriodicStatusReporterManager periodicStatusReporterManager, MiNiFiStdLogHandler miNiFiStdLogHandler, MiNiFiParameters miNiFiParameters, File bootstrapConfigFile,
+        RunMiNiFi runMiNiFi) {
+        this.currentPortProvider = currentPortProvider;
+        this.bootstrapFileProvider = bootstrapFileProvider;
+        this.periodicStatusReporterManager = periodicStatusReporterManager;
+        this.miNiFiStdLogHandler = miNiFiStdLogHandler;
+        this.miNiFiParameters = miNiFiParameters;
+        this.bootstrapConfigFile = bootstrapConfigFile;
+        this.runMiNiFi = runMiNiFi;
+    }
+
+    /**
+     * Starts (and restarts) MiNiFi process during the whole lifecycle of the bootstrap process.
+     * @param args the input arguments
+     * @return status code
+     */
+    @Override
+    public int runCommand(String[] args) {
+        try {
+            start();
+        } catch (Exception e) {
+            CMD_LOGGER.error("Exception happened during MiNiFi startup", e);
+            return ERROR.getStatusCode();
+        }
+        return OK.getStatusCode();
+    }
+
+    private void start() throws IOException, InterruptedException {
+        Integer port = currentPortProvider.getCurrentPort();
+        if (port != null) {
+            String alreadyRunningMessage = "Apache MiNiFi is already running";
+            CMD_LOGGER.info(alreadyRunningMessage + ", listening to Bootstrap on port {}", port);

Review Comment:
   done



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/BootstrapCodec.java:
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.nifi.minifi.bootstrap.service;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.util.Arrays;
+import org.apache.nifi.minifi.bootstrap.RunMiNiFi;
+import org.apache.nifi.minifi.bootstrap.exception.InvalidCommandException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class BootstrapCodec {
+
+    private static final String TRUE = "true";
+    private static final String FALSE = "false";
+
+    private final RunMiNiFi runner;
+    private final BufferedReader reader;
+    private final BufferedWriter writer;
+    private final Logger logger = LoggerFactory.getLogger(BootstrapCodec.class);
+
+    public BootstrapCodec(RunMiNiFi runner, InputStream in, OutputStream out) {
+        this.runner = runner;
+        this.reader = new BufferedReader(new InputStreamReader(in));
+        this.writer = new BufferedWriter(new OutputStreamWriter(out));
+    }
+
+    public void communicate() throws IOException {
+        String line = reader.readLine();
+        String[] splits = line.split(" ");
+        if (splits.length == 0) {
+            throw new IOException("Received invalid command from MiNiFi: " + line);
+        }
+
+        String cmd = splits[0];
+        String[] args;
+        if (splits.length == 1) {
+            args = new String[0];
+        } else {
+            args = Arrays.copyOfRange(splits, 1, splits.length);
+        }
+
+        try {
+            processRequest(cmd, args);
+        } catch (InvalidCommandException exception) {
+            throw new IOException("Received invalid command from MiNiFi: " + line + " : " + (exception.getMessage() == null ? "" : "Details: " + exception));

Review Comment:
   done



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/BootstrapFileProvider.java:
##########
@@ -0,0 +1,234 @@
+/*
+ * 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.nifi.minifi.bootstrap.service;
+
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.STATUS_FILE_PID_KEY;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.attribute.PosixFilePermission;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.util.file.FileUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class BootstrapFileProvider {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(BootstrapFileProvider.class);
+
+    private static final String MINIFI_PID_FILE_NAME = "minifi.pid";
+    private static final String MINIFI_STATUS_FILE_NAME = "minifi.status";
+    private static final String MINIFI_LOCK_FILE_NAME = "minifi.lock";
+    private static final String DEFAULT_CONFIG_FILE = "./conf/bootstrap.conf";
+    private static final String BOOTSTRAP_CONFIG_FILE_SYSTEM_PROPERTY_KEY = "org.apache.nifi.minifi.bootstrap.config.file";
+    private static final String MINIFI_HOME_ENV_VARIABLE_KEY = "MINIFI_HOME";
+    private static final String MINIFI_PID_DIR_PROP = "org.apache.nifi.minifi.bootstrap.config.pid.dir";
+    private static final String DEFAULT_PID_DIR = "bin";
+
+    private final File bootstrapConfigFile;
+
+    public BootstrapFileProvider(File bootstrapConfigFile) {
+        if (bootstrapConfigFile == null || !bootstrapConfigFile.exists()) {
+            throw new IllegalArgumentException("The specified bootstrap file doesn't exists: " + bootstrapConfigFile);
+        }
+        this.bootstrapConfigFile = bootstrapConfigFile;
+    }
+
+    public static File getBootstrapConfFile() {
+        File bootstrapConfigFile = Optional.ofNullable(System.getProperty(BOOTSTRAP_CONFIG_FILE_SYSTEM_PROPERTY_KEY))
+            .map(File::new)
+            .orElseGet(() -> Optional.ofNullable(System.getenv(MINIFI_HOME_ENV_VARIABLE_KEY))
+                .map(File::new)
+                .map(nifiHomeFile -> new File(nifiHomeFile, DEFAULT_CONFIG_FILE))
+                .orElseGet(() -> new File(DEFAULT_CONFIG_FILE)));
+        LOGGER.debug("Bootstrap config file: {}", bootstrapConfigFile);
+        return bootstrapConfigFile;
+    }
+
+    public File getPidFile() throws IOException {
+        return getBootstrapFile(MINIFI_PID_FILE_NAME);
+    }
+
+    public File getStatusFile() throws IOException {
+        return getBootstrapFile(MINIFI_STATUS_FILE_NAME);
+    }
+
+    public File getLockFile() throws IOException {
+        return getBootstrapFile(MINIFI_LOCK_FILE_NAME);
+    }
+
+    public File getReloadLockFile() {
+        File confDir = bootstrapConfigFile.getParentFile();
+        File nifiHome = confDir.getParentFile();
+        File bin = new File(nifiHome, "bin");
+        File reloadFile = new File(bin, "minifi.reload.lock");
+
+        LOGGER.debug("Reload File: {}", reloadFile);
+        return reloadFile;
+    }
+
+    public File getSwapFile() {
+        File confDir = bootstrapConfigFile.getParentFile();
+        File swapFile = new File(confDir, "swap.yml");
+
+        LOGGER.debug("Swap File: {}", swapFile);
+        return swapFile;
+    }
+
+    public Properties getBootstrapProperties() throws IOException {
+        if (!bootstrapConfigFile.exists()) {
+            throw new FileNotFoundException(bootstrapConfigFile.getAbsolutePath());
+        }
+
+        Properties bootstrapProperties = new Properties();
+        try (FileInputStream fis = new FileInputStream(bootstrapConfigFile)) {
+            bootstrapProperties.load(fis);
+        }
+
+        logProperties("Bootstrap", bootstrapProperties);
+
+        return bootstrapProperties;
+    }
+
+    public Properties getStatusProperties() {
+        Properties props = new Properties();
+
+        try {
+            File statusFile = getStatusFile();
+            if (statusFile == null || !statusFile.exists()) {
+                LOGGER.debug("No status file to load properties from");
+                return props;
+            }
+
+            try (FileInputStream fis = new FileInputStream(statusFile)) {
+                props.load(fis);
+            }
+        } catch (IOException exception) {
+            LOGGER.error("Failed to load MiNiFi status properties");
+        }
+
+        logProperties("MiNiFi status", props);

Review Comment:
   Added the secret.key to the secret keys enum so it will be filtered out.



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/MiNiFiStatus.java:
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.nifi.minifi.bootstrap;
+
+public class MiNiFiStatus {
+
+    private final Integer port;

Review Comment:
   Can be null until the MiNiFi process startup is in progress.



##########
nifi-server-api/src/main/java/org/apache/nifi/NiFiServer.java:
##########
@@ -26,13 +26,13 @@
 import java.util.Set;
 
 /**
- *
+ * The main interface for declaring a NiFi-based server application
  */
 public interface NiFiServer {
 
     void start();
 
-    void initialize(NiFiProperties properties, Bundle systemBundle, Set<Bundle> bundles, ExtensionMapping extensionMapping);
+    void initialize(final NiFiProperties properties, final Bundle systemBundle, final Set<Bundle> bundles, final ExtensionMapping extensionMapping);

Review Comment:
   done



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/command/StartRunner.java:
##########
@@ -0,0 +1,435 @@
+/*
+ * 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.nifi.minifi.bootstrap.command;
+
+import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CMD_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CONF_DIR_KEY;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.DEFAULT_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.MINIFI_CONFIG_FILE_KEY;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.STATUS_FILE_PID_KEY;
+import static org.apache.nifi.minifi.bootstrap.Status.ERROR;
+import static org.apache.nifi.minifi.bootstrap.Status.OK;
+import static org.apache.nifi.minifi.bootstrap.util.ConfigTransformer.generateConfigFiles;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.nifi.bootstrap.util.OSUtils;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.apache.nifi.minifi.bootstrap.RunMiNiFi;
+import org.apache.nifi.minifi.bootstrap.ShutdownHook;
+import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeException;
+import org.apache.nifi.minifi.bootstrap.exception.StartupFailureException;
+import org.apache.nifi.minifi.bootstrap.service.BootstrapFileProvider;
+import org.apache.nifi.minifi.bootstrap.service.CurrentPortProvider;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiListener;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiStdLogHandler;
+import org.apache.nifi.minifi.bootstrap.service.PeriodicStatusReporterManager;
+import org.apache.nifi.minifi.bootstrap.util.ProcessUtils;
+import org.apache.nifi.util.Tuple;
+
+public class StartRunner implements CommandRunner {
+    private static final String DEFAULT_JAVA_CMD = "java";
+    private static final String DEFAULT_LOG_DIR = "./logs";
+    private static final String DEFAULT_LIB_DIR = "./lib";
+    private static final String DEFAULT_CONF_DIR = "./conf";
+    private static final int STARTUP_WAIT_SECONDS = 60;
+    private static final String DEFAULT_CONFIG_FILE = DEFAULT_CONF_DIR + "/bootstrap.conf";
+
+    private final CurrentPortProvider currentPortProvider;
+    private final BootstrapFileProvider bootstrapFileProvider;
+    private final PeriodicStatusReporterManager periodicStatusReporterManager;
+    private final MiNiFiStdLogHandler miNiFiStdLogHandler;
+    private final MiNiFiParameters miNiFiParameters;
+    private final File bootstrapConfigFile;
+    private final Lock lock = new ReentrantLock();
+    private final Condition startupCondition = lock.newCondition();
+    private final RunMiNiFi runMiNiFi;
+    private volatile ShutdownHook shutdownHook;
+
+    public StartRunner(CurrentPortProvider currentPortProvider, BootstrapFileProvider bootstrapFileProvider,
+        PeriodicStatusReporterManager periodicStatusReporterManager, MiNiFiStdLogHandler miNiFiStdLogHandler, MiNiFiParameters miNiFiParameters, File bootstrapConfigFile,
+        RunMiNiFi runMiNiFi) {
+        this.currentPortProvider = currentPortProvider;
+        this.bootstrapFileProvider = bootstrapFileProvider;
+        this.periodicStatusReporterManager = periodicStatusReporterManager;
+        this.miNiFiStdLogHandler = miNiFiStdLogHandler;
+        this.miNiFiParameters = miNiFiParameters;
+        this.bootstrapConfigFile = bootstrapConfigFile;
+        this.runMiNiFi = runMiNiFi;
+    }
+
+    /**
+     * Starts (and restarts) MiNiFi process during the whole lifecycle of the bootstrap process.
+     * @param args the input arguments
+     * @return status code
+     */
+    @Override
+    public int runCommand(String[] args) {
+        try {
+            start();
+        } catch (Exception e) {
+            CMD_LOGGER.error("Exception happened during MiNiFi startup", e);
+            return ERROR.getStatusCode();
+        }
+        return OK.getStatusCode();
+    }
+
+    private void start() throws IOException, InterruptedException {
+        Integer port = currentPortProvider.getCurrentPort();
+        if (port != null) {
+            String alreadyRunningMessage = "Apache MiNiFi is already running";
+            CMD_LOGGER.info(alreadyRunningMessage + ", listening to Bootstrap on port {}", port);
+            return;
+        }
+
+        File prevLockFile = bootstrapFileProvider.getLockFile();
+        if (prevLockFile.exists() && !prevLockFile.delete()) {
+            CMD_LOGGER.warn("Failed to delete previous lock file {}; this file should be cleaned up manually", prevLockFile);
+        }
+
+        Properties bootstrapProperties = bootstrapFileProvider.getBootstrapProperties();
+        String confDir = bootstrapProperties.getProperty(CONF_DIR_KEY);
+        initConfigFiles(bootstrapProperties, confDir);
+
+        Tuple<ProcessBuilder, Process> tuple = startMiNiFi();
+        ProcessBuilder builder = tuple.getKey();
+        Process process = tuple.getValue();
+
+        try {
+            while (true) {
+                if (ProcessUtils.isAlive(process)) {
+                    handleReload();
+                } else {
+                    Runtime runtime = Runtime.getRuntime();
+                    try {
+                        runtime.removeShutdownHook(shutdownHook);
+                    } catch (IllegalStateException ise) {
+                        // happens when already shutting down
+                    }
+
+                    if (runMiNiFi.isAutoRestartNiFi() && needRestart()) {
+                        File reloadFile = bootstrapFileProvider.getReloadLockFile();
+                        if (reloadFile.exists()) {
+                            DEFAULT_LOGGER.info("Currently reloading configuration. Will wait to restart MiNiFi.");
+                            Thread.sleep(5000L);
+                            continue;
+                        }

Review Comment:
   I'm not sure if that would work. This while loop guarantees that the bootstrap doesn't exit before the MiNiFi process. This is the same approach as in NiFi bootstrap.



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/MiNiFiCommandSender.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 org.apache.nifi.minifi.bootstrap.service;
+
+import java.io.BufferedReader;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.ObjectInputStream;
+import java.io.OutputStream;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.net.SocketTimeoutException;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MiNiFiCommandSender {
+    private static final Logger LOGGER = LoggerFactory.getLogger(MiNiFiCommandSender.class);
+    private static final String PING_CMD = "PING";
+
+    private final MiNiFiParameters miNiFiParameters;
+
+    public MiNiFiCommandSender(MiNiFiParameters miNiFiParameters) {
+        this.miNiFiParameters = miNiFiParameters;
+    }
+
+    public Optional<String> sendCommand(String cmd, Integer port) throws IOException {
+        Optional<String> response = Optional.empty();
+
+        if (port == null) {
+            LOGGER.info("Apache MiNiFi is not currently running");
+            return response;
+        }
+
+        try (Socket socket = new Socket()) {
+            LOGGER.debug("Connecting to MiNiFi instance");
+            socket.setSoTimeout(10000);
+            socket.connect(new InetSocketAddress("localhost", port));
+            LOGGER.debug("Established connection to MiNiFi instance.");
+
+            LOGGER.debug("Sending {} Command to port {}", cmd, port);
+            OutputStream out = socket.getOutputStream();
+            out.write((cmd + " " + miNiFiParameters.getSecretKey() + "\n").getBytes(StandardCharsets.UTF_8));
+            out.flush();
+            socket.shutdownOutput();
+
+            InputStream in = socket.getInputStream();
+            StringBuilder sb = new StringBuilder();
+            int numLines = 0;
+            try (BufferedReader reader = new BufferedReader(new InputStreamReader(in))) {

Review Comment:
   done



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/MiNiFiCommandSender.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 org.apache.nifi.minifi.bootstrap.service;
+
+import java.io.BufferedReader;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.ObjectInputStream;
+import java.io.OutputStream;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.net.SocketTimeoutException;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MiNiFiCommandSender {
+    private static final Logger LOGGER = LoggerFactory.getLogger(MiNiFiCommandSender.class);
+    private static final String PING_CMD = "PING";
+
+    private final MiNiFiParameters miNiFiParameters;
+
+    public MiNiFiCommandSender(MiNiFiParameters miNiFiParameters) {
+        this.miNiFiParameters = miNiFiParameters;
+    }
+
+    public Optional<String> sendCommand(String cmd, Integer port) throws IOException {
+        Optional<String> response = Optional.empty();
+
+        if (port == null) {
+            LOGGER.info("Apache MiNiFi is not currently running");
+            return response;
+        }
+
+        try (Socket socket = new Socket()) {
+            LOGGER.debug("Connecting to MiNiFi instance");
+            socket.setSoTimeout(10000);
+            socket.connect(new InetSocketAddress("localhost", port));

Review Comment:
   done



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/MiNiFiCommandSender.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 org.apache.nifi.minifi.bootstrap.service;
+
+import java.io.BufferedReader;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.ObjectInputStream;
+import java.io.OutputStream;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.net.SocketTimeoutException;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MiNiFiCommandSender {
+    private static final Logger LOGGER = LoggerFactory.getLogger(MiNiFiCommandSender.class);
+    private static final String PING_CMD = "PING";
+
+    private final MiNiFiParameters miNiFiParameters;
+
+    public MiNiFiCommandSender(MiNiFiParameters miNiFiParameters) {
+        this.miNiFiParameters = miNiFiParameters;
+    }
+
+    public Optional<String> sendCommand(String cmd, Integer port) throws IOException {
+        Optional<String> response = Optional.empty();
+
+        if (port == null) {
+            LOGGER.info("Apache MiNiFi is not currently running");
+            return response;
+        }
+
+        try (Socket socket = new Socket()) {
+            LOGGER.debug("Connecting to MiNiFi instance");
+            socket.setSoTimeout(10000);
+            socket.connect(new InetSocketAddress("localhost", port));
+            LOGGER.debug("Established connection to MiNiFi instance.");
+
+            LOGGER.debug("Sending {} Command to port {}", cmd, port);
+            OutputStream out = socket.getOutputStream();
+            out.write((cmd + " " + miNiFiParameters.getSecretKey() + "\n").getBytes(StandardCharsets.UTF_8));
+            out.flush();
+            socket.shutdownOutput();

Review Comment:
   done



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/RunMiNiFi.java:
##########
@@ -200,1289 +150,38 @@ private static void printUsage() {
         System.out.println();
     }
 
-    public static void main(String[] args) throws IOException, InterruptedException {
-        if (args.length < 1 || args.length > 3) {
-            printUsage();
-            return;
-        }
-
-        File dumpFile = null;
-
-        final String cmd = args[0];
-        if (cmd.equals("dump")) {
-            if (args.length > 1) {
-                dumpFile = new File(args[1]);
-            } else {
-                dumpFile = null;
-            }
-        }
-
-        switch (cmd.toLowerCase()) {
-            case "start":
-            case "run":
-            case "stop":
-            case "status":
-            case "dump":
-            case "restart":
-            case "env":
-            case "flowstatus":
-                break;
-            default:
-                printUsage();
-                return;
-        }
-
-        final File configFile = getBootstrapConfFile();
-        final RunMiNiFi runMiNiFi = new RunMiNiFi(configFile);
-
-        Integer exitStatus = null;
-        switch (cmd.toLowerCase()) {
-            case "start":
-                runMiNiFi.start();
-                break;
-            case "run":
-                runMiNiFi.start();
-                break;
-            case "stop":
-                runMiNiFi.stop();
-                break;
-            case "status":
-                exitStatus = runMiNiFi.status();
-                break;
-            case "restart":
-                runMiNiFi.stop();
-                runMiNiFi.start();
-                break;
-            case "dump":
-                runMiNiFi.dump(dumpFile);
-                break;
-            case "env":
-                runMiNiFi.env();
-                break;
-            case "flowstatus":
-                if(args.length == 2) {
-                    System.out.println(runMiNiFi.statusReport(args[1]));
-                } else {
-                    System.out.println("The 'flowStatus' command requires an input query. See the System Admin Guide 'FlowStatus Script Query' section for complete details.");
-                }
-                break;
-        }
-        if (exitStatus != null) {
-            System.exit(exitStatus);
-        }
-    }
-
-    public static File getBootstrapConfFile() {
-        String configFilename = System.getProperty("org.apache.nifi.minifi.bootstrap.config.file");
-
-        if (configFilename == null) {
-            final String nifiHome = System.getenv("MINIFI_HOME");
-            if (nifiHome != null) {
-                final File nifiHomeFile = new File(nifiHome.trim());
-                final File configFile = new File(nifiHomeFile, DEFAULT_CONFIG_FILE);
-                configFilename = configFile.getAbsolutePath();
-            }
-        }
-
-        if (configFilename == null) {
-            configFilename = DEFAULT_CONFIG_FILE;
-        }
-
-        final File configFile = new File(configFilename);
-        return configFile;
-    }
-
-    private File getBootstrapFile(final Logger logger, String directory, String defaultDirectory, String fileName) throws IOException {
-
-        final File confDir = bootstrapConfigFile.getParentFile();
-        final File nifiHome = confDir.getParentFile();
-
-        String confFileDir = System.getProperty(directory);
-
-        final File fileDir;
-
-        if (confFileDir != null) {
-            fileDir = new File(confFileDir.trim());
-        } else {
-            fileDir = new File(nifiHome, defaultDirectory);
-        }
-
-        FileUtils.ensureDirectoryExistAndCanAccess(fileDir);
-        final File statusFile = new File(fileDir, fileName);
-        logger.debug("Status File: {}", statusFile);
-        return statusFile;
-    }
-
-    File getPidFile(final Logger logger) throws IOException {
-        return getBootstrapFile(logger, MINIFI_PID_DIR_PROP, DEFAULT_PID_DIR, MINIFI_PID_FILE_NAME);
-    }
-
-    File getStatusFile(final Logger logger) throws IOException {
-        return getBootstrapFile(logger, MINIFI_PID_DIR_PROP, DEFAULT_PID_DIR, MINIFI_STATUS_FILE_NAME);
-    }
-
-    File getLockFile(final Logger logger) throws IOException {
-        return getBootstrapFile(logger, MINIFI_PID_DIR_PROP, DEFAULT_PID_DIR, MINIFI_LOCK_FILE_NAME);
-    }
-
-    File getStatusFile() throws IOException{
-        return getStatusFile(defaultLogger);
-    }
-
-    public File getReloadFile(final Logger logger) {
-        final File confDir = bootstrapConfigFile.getParentFile();
-        final File nifiHome = confDir.getParentFile();
-        final File bin = new File(nifiHome, "bin");
-        final File reloadFile = new File(bin, "minifi.reload.lock");
-
-        logger.debug("Reload File: {}", reloadFile);
-        return reloadFile;
-    }
-
-    public File getSwapFile(final Logger logger) {
-        final File confDir = bootstrapConfigFile.getParentFile();
-        final File swapFile = new File(confDir, "swap.yml");
-
-        logger.debug("Swap File: {}", swapFile);
-        return swapFile;
-    }
-
-
-    private Properties loadProperties(final Logger logger) throws IOException {
-        final Properties props = new Properties();
-        final File statusFile = getStatusFile(logger);
-        if (statusFile == null || !statusFile.exists()) {
-            logger.debug("No status file to load properties from");
-            return props;
-        }
-
-        try (final FileInputStream fis = new FileInputStream(getStatusFile(logger))) {
-            props.load(fis);
-        }
-
-        final Map<Object, Object> modified = new HashMap<>(props);
-        modified.remove("secret.key");
-        logger.debug("Properties: {}", modified);
-
-        return props;
-    }
-
-    private synchronized void saveProperties(final Properties minifiProps, final Logger logger) throws IOException {
-        final String pid = minifiProps.getProperty(PID_KEY);
-        if (!StringUtils.isBlank(pid)) {
-            writePidFile(pid, logger);
-        }
-
-        final File statusFile = getStatusFile(logger);
-        if (statusFile.exists() && !statusFile.delete()) {
-            logger.warn("Failed to delete {}", statusFile);
-        }
-
-        if (!statusFile.createNewFile()) {
-            throw new IOException("Failed to create file " + statusFile);
-        }
-
-        try {
-            final Set<PosixFilePermission> perms = new HashSet<>();
-            perms.add(PosixFilePermission.OWNER_WRITE);
-            perms.add(PosixFilePermission.OWNER_READ);
-            perms.add(PosixFilePermission.GROUP_READ);
-            perms.add(PosixFilePermission.OTHERS_READ);
-            Files.setPosixFilePermissions(statusFile.toPath(), perms);
-        } catch (final Exception e) {
-            logger.warn("Failed to set permissions so that only the owner can read status file {}; "
-                + "this may allows others to have access to the key needed to communicate with MiNiFi. "
-                + "Permissions should be changed so that only the owner can read this file", statusFile);
-        }
-
-        try (final FileOutputStream fos = new FileOutputStream(statusFile)) {
-            minifiProps.store(fos, null);
-            fos.getFD().sync();
-        }
-
-        logger.debug("Saved Properties {} to {}", new Object[]{minifiProps, statusFile});
-    }
-
-    private synchronized void writePidFile(final String pid, final Logger logger) throws IOException {
-        final File pidFile = getPidFile(logger);
-        if (pidFile.exists() && !pidFile.delete()) {
-           logger.warn("Failed to delete {}", pidFile);
-        }
-
-        if (!pidFile.createNewFile()) {
-            throw new IOException("Failed to create file " + pidFile);
-        }
-
-        try {
-            final Set<PosixFilePermission> perms = new HashSet<>();
-            perms.add(PosixFilePermission.OWNER_READ);
-            perms.add(PosixFilePermission.OWNER_WRITE);
-            Files.setPosixFilePermissions(pidFile.toPath(), perms);
-        } catch (final Exception e) {
-            logger.warn("Failed to set permissions so that only the owner can read pid file {}; "
-                    + "this may allows others to have access to the key needed to communicate with MiNiFi. "
-                    + "Permissions should be changed so that only the owner can read this file", pidFile);
-        }
-
-        try (final FileOutputStream fos = new FileOutputStream(pidFile)) {
-            fos.write(pid.getBytes(StandardCharsets.UTF_8));
-            fos.getFD().sync();
-        }
-
-        logger.debug("Saved Pid {} to {}", new Object[]{pid, pidFile});
-    }
-
-    private boolean isPingSuccessful(final int port, final String secretKey, final Logger logger) {
-        logger.debug("Pinging {}", port);
-
-        try (final Socket socket = new Socket("localhost", port)) {
-            final OutputStream out = socket.getOutputStream();
-            out.write((PING_CMD + " " + secretKey + "\n").getBytes(StandardCharsets.UTF_8));
-            out.flush();
-
-            logger.debug("Sent PING command");
-            socket.setSoTimeout(5000);
-            final InputStream in = socket.getInputStream();
-            final BufferedReader reader = new BufferedReader(new InputStreamReader(in));
-            final String response = reader.readLine();
-            logger.debug("PING response: {}", response);
-            out.close();
-            reader.close();
-
-            return PING_CMD.equals(response);
-        } catch (final IOException ioe) {
-            return false;
-        }
-    }
-
-    private Integer getCurrentPort(final Logger logger) throws IOException {
-        final Properties props = loadProperties(logger);
-        final String portVal = props.getProperty("port");
-        if (portVal == null) {
-            logger.debug("No Port found in status file");
-            return null;
-        } else {
-            logger.debug("Port defined in status file: {}", portVal);
-        }
-
-        final int port = Integer.parseInt(portVal);
-        final boolean success = isPingSuccessful(port, props.getProperty("secret.key"), logger);
-        if (success) {
-            logger.debug("Successful PING on port {}", port);
-            return port;
-        }
-
-        final String pid = props.getProperty(PID_KEY);
-        logger.debug("PID in status file is {}", pid);
-        if (pid != null) {
-            final boolean procRunning = isProcessRunning(pid, logger);
-            if (procRunning) {
-                return port;
-            } else {
-                return null;
-            }
-        }
-
-        return null;
-    }
-
-    private boolean isProcessRunning(final String pid, final Logger logger) {
-        try {
-            // We use the "ps" command to check if the process is still running.
-            final ProcessBuilder builder = new ProcessBuilder();
-
-            builder.command("ps", "-p", pid);
-            final Process proc = builder.start();
-
-            // Look for the pid in the output of the 'ps' command.
-            boolean running = false;
-            String line;
-            try (final InputStream in = proc.getInputStream();
-                 final Reader streamReader = new InputStreamReader(in);
-                 final BufferedReader reader = new BufferedReader(streamReader)) {
-
-                while ((line = reader.readLine()) != null) {
-                    if (line.trim().startsWith(pid)) {
-                        running = true;
-                    }
-                }
-            }
-
-            // If output of the ps command had our PID, the process is running.
-            if (running) {
-                logger.debug("Process with PID {} is running", pid);
-            } else {
-                logger.debug("Process with PID {} is not running", pid);
-            }
-
-            return running;
-        } catch (final IOException ioe) {
-            System.err.println("Failed to determine if Process " + pid + " is running; assuming that it is not");
-            return false;
-        }
-    }
-
-    private Status getStatus(final Logger logger) {
-        final Properties props;
-        try {
-            props = loadProperties(logger);
-        } catch (final IOException ioe) {
-            return new Status(null, null, false, false);
-        }
-
-        if (props == null) {
-            return new Status(null, null, false, false);
-        }
-
-        final String portValue = props.getProperty("port");
-        final String pid = props.getProperty(PID_KEY);
-        final String secretKey = props.getProperty("secret.key");
-
-        if (portValue == null && pid == null) {
-            return new Status(null, null, false, false);
-        }
-
-        Integer port = null;
-        boolean pingSuccess = false;
-        if (portValue != null) {
-            try {
-                port = Integer.parseInt(portValue);
-                pingSuccess = isPingSuccessful(port, secretKey, logger);
-            } catch (final NumberFormatException nfe) {
-                return new Status(null, null, false, false);
-            }
-        }
-
-        if (pingSuccess) {
-            return new Status(port, pid, true, true);
-        }
-
-        final boolean alive = (pid != null) && isProcessRunning(pid, logger);
-        return new Status(port, pid, pingSuccess, alive);
-    }
-
-    public int status() throws IOException {
-        final Logger logger = cmdLogger;
-        final Status status = getStatus(logger);
-        if (status.isRespondingToPing()) {
-            logger.info("Apache MiNiFi is currently running, listening to Bootstrap on port {}, PID={}",
-                new Object[]{status.getPort(), status.getPid() == null ? "unknown" : status.getPid()});
-            return 0;
-        }
-
-        if (status.isProcessRunning()) {
-            logger.info("Apache MiNiFi is running at PID {} but is not responding to ping requests", status.getPid());
-            return 4;
-        }
-
-        if (status.getPort() == null) {
-            logger.info("Apache MiNiFi is not running");
-            return 3;
-        }
-
-        if (status.getPid() == null) {
-            logger.info("Apache MiNiFi is not responding to Ping requests. The process may have died or may be hung");
-        } else {
-            logger.info("Apache MiNiFi is not running");
-        }
-        return 3;
-    }
-
-    public FlowStatusReport statusReport(String statusRequest) throws IOException {
-        final Logger logger = cmdLogger;
-        final Status status = getStatus(logger);
-        final Properties props = loadProperties(logger);
-
-        List<String> problemsGeneratingReport = new LinkedList<>();
-        if (!status.isProcessRunning()) {
-            problemsGeneratingReport.add("MiNiFi process is not running");
-        }
-
-        if (!status.isRespondingToPing()) {
-            problemsGeneratingReport.add("MiNiFi process is not responding to pings");
-        }
-
-        if (!problemsGeneratingReport.isEmpty()) {
-            FlowStatusReport flowStatusReport = new FlowStatusReport();
-            flowStatusReport.setErrorsGeneratingReport(problemsGeneratingReport);
-            return flowStatusReport;
-        }
-
-        return getFlowStatusReport(statusRequest, status.getPort(), props.getProperty("secret.key"), logger);
-    }
-
-    public void env() {
-        final Logger logger = cmdLogger;
-        final Status status = getStatus(logger);
-        if (status.getPid() == null) {
-            logger.info("Apache MiNiFi is not running");
+    public void setMiNiFiParameters(int port, String secretKey) throws IOException {
+        if (Optional.ofNullable(secretKey).filter(key -> key.equals(miNiFiParameters.getSecretKey())).isPresent() && miNiFiParameters.getMiNiFiPort() == port) {
+            DEFAULT_LOGGER.debug("secretKey and port match with the known one, nothing to update");
             return;
         }
-        final Class<?> virtualMachineClass;
-        try {
-            virtualMachineClass = Class.forName("com.sun.tools.attach.VirtualMachine");
-        } catch (final ClassNotFoundException cnfe) {
-            logger.error("Seems tools.jar (Linux / Windows JDK) or classes.jar (Mac OS) is not available in classpath");
-            return;
-        }
-        final Method attachMethod;
-        final Method detachMethod;
 
-        try {
-            attachMethod = virtualMachineClass.getMethod("attach", String.class);
-            detachMethod = virtualMachineClass.getDeclaredMethod("detach");
-        } catch (final Exception e) {
-            logger.error("Methods required for getting environment not available", e);
-            return;
-        }
+        miNiFiParameters.setMiNiFiPort(port);
+        miNiFiParameters.setSecretKey(secretKey);
 
-        final Object virtualMachine;
-        try {
-            virtualMachine = attachMethod.invoke(null, status.getPid());
-        } catch (final Throwable t) {
-            logger.error("Problem attaching to MiNiFi", t);
-            return;
+        Properties minifiProps = new Properties();
+        long minifiPid = miNiFiParameters.getMinifiPid();
+        if (minifiPid != UNINITIALIZED) {
+            minifiProps.setProperty(STATUS_FILE_PID_KEY, String.valueOf(minifiPid));
         }
+        minifiProps.setProperty(STATUS_FILE_PORT_KEY, String.valueOf(port));
+        minifiProps.setProperty(STATUS_FILE_SECRET_KEY, secretKey);
 
+        File statusFile = bootstrapFileProvider.getStatusFile();
         try {
-            final Method getSystemPropertiesMethod = virtualMachine.getClass().getMethod("getSystemProperties");
-
-            final Properties sysProps = (Properties) getSystemPropertiesMethod.invoke(virtualMachine);
-            for (Entry<Object, Object> syspropEntry : sysProps.entrySet()) {
-                logger.info(syspropEntry.getKey().toString() + " = " + syspropEntry.getValue().toString());
-            }
-        } catch (Throwable t) {
-            throw new RuntimeException(t);
-        } finally {
-            try {
-                detachMethod.invoke(virtualMachine);
-            } catch (final Exception e) {
-                logger.warn("Caught exception detaching from process", e);
-            }
-        }
-    }
-
-    /**
-     * Writes a MiNiFi thread dump to the given file; if file is null, logs at
-     * INFO level instead.
-     *
-     * @param dumpFile the file to write the dump content to
-     * @throws IOException if any issues occur while writing the dump file
-     */
-    public void dump(final File dumpFile) throws IOException {
-        final Logger logger = defaultLogger;    // dump to bootstrap log file by default
-        final Integer port = getCurrentPort(logger);
-        if (port == null) {
-            logger.info("Apache MiNiFi is not currently running");
-            return;
+            bootstrapFileProvider.saveStatusProperties(minifiProps);
+        } catch (IOException ioe) {
+            DEFAULT_LOGGER.warn("Apache MiNiFi has started but failed to persist MiNiFi Port information to {} due to {}", statusFile.getAbsolutePath(), ioe);

Review Comment:
   done



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/MiNiFiStatus.java:
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.nifi.minifi.bootstrap;
+
+public class MiNiFiStatus {
+
+    private final Integer port;
+    private final String pid;

Review Comment:
   There is not, I can change it to Long. (it can be also null / same reason as for the port / , and doesn't fit to Int)
   We use OsUtils from NiFi commons, but according to the java 9+ documentation it's a long value here as well: https://docs.oracle.com/javase/9/docs/api/java/lang/ProcessHandle.html#pid--



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/util/ProcessUtils.java:
##########
@@ -0,0 +1,131 @@
+/*
+ * 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.nifi.minifi.bootstrap.util;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.Reader;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ProcessUtils {

Review Comment:
   You are totally right, naively I thought somehow these are "translated" to the right system command within the JVM, so I simply reused the existing code from NiFi bootstrap. I can rename it for now, but we might need to investigate if we need to implement an OS agnostic solution.



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/command/StartRunner.java:
##########
@@ -0,0 +1,435 @@
+/*
+ * 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.nifi.minifi.bootstrap.command;
+
+import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CMD_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CONF_DIR_KEY;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.DEFAULT_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.MINIFI_CONFIG_FILE_KEY;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.STATUS_FILE_PID_KEY;
+import static org.apache.nifi.minifi.bootstrap.Status.ERROR;
+import static org.apache.nifi.minifi.bootstrap.Status.OK;
+import static org.apache.nifi.minifi.bootstrap.util.ConfigTransformer.generateConfigFiles;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.nifi.bootstrap.util.OSUtils;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.apache.nifi.minifi.bootstrap.RunMiNiFi;
+import org.apache.nifi.minifi.bootstrap.ShutdownHook;
+import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeException;
+import org.apache.nifi.minifi.bootstrap.exception.StartupFailureException;
+import org.apache.nifi.minifi.bootstrap.service.BootstrapFileProvider;
+import org.apache.nifi.minifi.bootstrap.service.CurrentPortProvider;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiListener;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiStdLogHandler;
+import org.apache.nifi.minifi.bootstrap.service.PeriodicStatusReporterManager;
+import org.apache.nifi.minifi.bootstrap.util.ProcessUtils;
+import org.apache.nifi.util.Tuple;
+
+public class StartRunner implements CommandRunner {
+    private static final String DEFAULT_JAVA_CMD = "java";
+    private static final String DEFAULT_LOG_DIR = "./logs";
+    private static final String DEFAULT_LIB_DIR = "./lib";
+    private static final String DEFAULT_CONF_DIR = "./conf";
+    private static final int STARTUP_WAIT_SECONDS = 60;
+    private static final String DEFAULT_CONFIG_FILE = DEFAULT_CONF_DIR + "/bootstrap.conf";
+
+    private final CurrentPortProvider currentPortProvider;
+    private final BootstrapFileProvider bootstrapFileProvider;
+    private final PeriodicStatusReporterManager periodicStatusReporterManager;
+    private final MiNiFiStdLogHandler miNiFiStdLogHandler;
+    private final MiNiFiParameters miNiFiParameters;
+    private final File bootstrapConfigFile;
+    private final Lock lock = new ReentrantLock();
+    private final Condition startupCondition = lock.newCondition();
+    private final RunMiNiFi runMiNiFi;
+    private volatile ShutdownHook shutdownHook;
+
+    public StartRunner(CurrentPortProvider currentPortProvider, BootstrapFileProvider bootstrapFileProvider,
+        PeriodicStatusReporterManager periodicStatusReporterManager, MiNiFiStdLogHandler miNiFiStdLogHandler, MiNiFiParameters miNiFiParameters, File bootstrapConfigFile,
+        RunMiNiFi runMiNiFi) {
+        this.currentPortProvider = currentPortProvider;
+        this.bootstrapFileProvider = bootstrapFileProvider;
+        this.periodicStatusReporterManager = periodicStatusReporterManager;
+        this.miNiFiStdLogHandler = miNiFiStdLogHandler;
+        this.miNiFiParameters = miNiFiParameters;
+        this.bootstrapConfigFile = bootstrapConfigFile;
+        this.runMiNiFi = runMiNiFi;
+    }
+
+    /**
+     * Starts (and restarts) MiNiFi process during the whole lifecycle of the bootstrap process.
+     * @param args the input arguments
+     * @return status code
+     */
+    @Override
+    public int runCommand(String[] args) {
+        try {
+            start();
+        } catch (Exception e) {
+            CMD_LOGGER.error("Exception happened during MiNiFi startup", e);
+            return ERROR.getStatusCode();
+        }
+        return OK.getStatusCode();
+    }
+
+    private void start() throws IOException, InterruptedException {
+        Integer port = currentPortProvider.getCurrentPort();
+        if (port != null) {
+            String alreadyRunningMessage = "Apache MiNiFi is already running";
+            CMD_LOGGER.info(alreadyRunningMessage + ", listening to Bootstrap on port {}", port);
+            return;
+        }
+
+        File prevLockFile = bootstrapFileProvider.getLockFile();
+        if (prevLockFile.exists() && !prevLockFile.delete()) {
+            CMD_LOGGER.warn("Failed to delete previous lock file {}; this file should be cleaned up manually", prevLockFile);
+        }
+
+        Properties bootstrapProperties = bootstrapFileProvider.getBootstrapProperties();
+        String confDir = bootstrapProperties.getProperty(CONF_DIR_KEY);
+        initConfigFiles(bootstrapProperties, confDir);
+
+        Tuple<ProcessBuilder, Process> tuple = startMiNiFi();
+        ProcessBuilder builder = tuple.getKey();
+        Process process = tuple.getValue();
+
+        try {
+            while (true) {
+                if (ProcessUtils.isAlive(process)) {
+                    handleReload();
+                } else {
+                    Runtime runtime = Runtime.getRuntime();
+                    try {
+                        runtime.removeShutdownHook(shutdownHook);
+                    } catch (IllegalStateException ise) {
+                        // happens when already shutting down
+                    }
+
+                    if (runMiNiFi.isAutoRestartNiFi() && needRestart()) {
+                        File reloadFile = bootstrapFileProvider.getReloadLockFile();
+                        if (reloadFile.exists()) {
+                            DEFAULT_LOGGER.info("Currently reloading configuration. Will wait to restart MiNiFi.");
+                            Thread.sleep(5000L);
+                            continue;
+                        }
+
+                        process = restartNifi(bootstrapProperties, confDir, builder, runtime);
+                        // failed to start process
+                        if (process == null) {
+                            return;
+                        }
+                    } else {
+                        return;
+                    }
+                }
+            }
+        } finally {
+            miNiFiStdLogHandler.shutdown();
+            runMiNiFi.shutdownChangeNotifier();
+            periodicStatusReporterManager.shutdownPeriodicStatusReporters();
+        }
+    }
+
+    private Process restartNifi(Properties bootstrapProperties, String confDir, ProcessBuilder builder, Runtime runtime) throws IOException {
+        Process process;
+        boolean previouslyStarted = runMiNiFi.isNiFiStarted();
+        if (!previouslyStarted) {
+            File swapConfigFile = bootstrapFileProvider.getSwapFile();
+            if (swapConfigFile.exists()) {
+                DEFAULT_LOGGER.info("Swap file exists, MiNiFi failed trying to change configuration. Reverting to old configuration.");
+
+                try {
+                    ByteBuffer tempConfigFile = generateConfigFiles(new FileInputStream(swapConfigFile), confDir, bootstrapProperties);
+                    runMiNiFi.getConfigFileReference().set(tempConfigFile.asReadOnlyBuffer());
+                } catch (ConfigurationChangeException e) {
+                    DEFAULT_LOGGER.error("The swap file is malformed, unable to restart from prior state. Will not attempt to restart MiNiFi. Swap File should be cleaned up manually.");
+                    return null;
+                }
+
+                Files.copy(swapConfigFile.toPath(), Paths.get(bootstrapProperties.getProperty(MINIFI_CONFIG_FILE_KEY)), REPLACE_EXISTING);
+
+                DEFAULT_LOGGER.info("Replacing config file with swap file and deleting swap file");
+                if (!swapConfigFile.delete()) {
+                    DEFAULT_LOGGER.warn("The swap file failed to delete after replacing using it to revert to the old configuration. It should be cleaned up manually.");
+                }
+                runMiNiFi.setReloading(false);
+            } else {
+                DEFAULT_LOGGER.info("MiNiFi either never started or failed to restart. Will not attempt to restart MiNiFi");
+                return null;
+            }
+        } else {
+            runMiNiFi.setNiFiStarted(false);
+        }
+
+        miNiFiParameters.setSecretKey(null);
+        process = builder.start();
+        miNiFiStdLogHandler.initLogging(process);
+
+        Long pid = OSUtils.getProcessId(process, DEFAULT_LOGGER);
+        if (pid != null) {
+            miNiFiParameters.setMinifiPid(pid);
+            Properties minifiProps = new Properties();
+            minifiProps.setProperty(STATUS_FILE_PID_KEY, String.valueOf(pid));
+            bootstrapFileProvider.saveStatusProperties(minifiProps);
+        }
+
+        shutdownHook = new ShutdownHook(runMiNiFi, miNiFiStdLogHandler);
+        runtime.addShutdownHook(shutdownHook);
+
+        boolean started = waitForStart();
+
+        if (started) {
+            DEFAULT_LOGGER.info("Successfully spawned the thread to start Apache MiNiFi{}", (pid == null ? "" : " with PID " + pid));
+        } else {
+            DEFAULT_LOGGER.error("Apache MiNiFi does not appear to have started");
+        }
+        return process;
+    }
+
+    private boolean needRestart() throws IOException {
+        boolean needRestart = true;
+        File statusFile = bootstrapFileProvider.getStatusFile();
+        if (!statusFile.exists()) {
+            DEFAULT_LOGGER.info("Status File no longer exists. Will not restart MiNiFi");
+            return false;
+        }
+
+        File lockFile = bootstrapFileProvider.getLockFile();
+        if (lockFile.exists()) {
+            DEFAULT_LOGGER.info("A shutdown was initiated. Will not restart MiNiFi");
+            return false;
+        }
+        return needRestart;
+    }
+
+    private void handleReload() {
+        try {
+            Thread.sleep(1000L);
+            if (runMiNiFi.getReloading() && runMiNiFi.isNiFiStarted()) {
+                File swapConfigFile = bootstrapFileProvider.getSwapFile();
+                if (swapConfigFile.exists()) {
+                    DEFAULT_LOGGER.info("MiNiFi has finished reloading successfully and swap file exists. Deleting old configuration.");
+
+                    if (swapConfigFile.delete()) {
+                        DEFAULT_LOGGER.info("Swap file was successfully deleted.");
+                    } else {
+                        DEFAULT_LOGGER.error("Swap file was not deleted. It should be deleted manually.");
+                    }
+                }
+                runMiNiFi.setReloading(false);
+            }
+        } catch (InterruptedException ie) {
+        }
+    }
+
+    private void initConfigFiles(Properties bootstrapProperties, String confDir) throws IOException {
+        File configFile = new File(bootstrapProperties.getProperty(MINIFI_CONFIG_FILE_KEY));
+        try (InputStream inputStream = new FileInputStream(configFile)) {
+            ByteBuffer tempConfigFile = generateConfigFiles(inputStream, confDir, bootstrapProperties);
+            runMiNiFi.getConfigFileReference().set(tempConfigFile.asReadOnlyBuffer());
+        } catch (FileNotFoundException e) {
+            String fileNotFoundMessage = "The config file defined in " + MINIFI_CONFIG_FILE_KEY + " does not exists.";
+            DEFAULT_LOGGER.error(fileNotFoundMessage, e);
+            throw new StartupFailureException(fileNotFoundMessage);
+        } catch (ConfigurationChangeException e) {
+            String malformedConfigFileMessage = "The config file is malformed, unable to start.";
+            DEFAULT_LOGGER.error(malformedConfigFileMessage, e);
+            throw new StartupFailureException(malformedConfigFileMessage);
+        }
+    }
+
+    private Tuple<ProcessBuilder, Process> startMiNiFi() throws IOException {

Review Comment:
   extracted the command building logics to MiNiFiExecCommandProvider



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/BootstrapFileProvider.java:
##########
@@ -0,0 +1,234 @@
+/*
+ * 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.nifi.minifi.bootstrap.service;
+
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.STATUS_FILE_PID_KEY;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.attribute.PosixFilePermission;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.util.file.FileUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class BootstrapFileProvider {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(BootstrapFileProvider.class);
+
+    private static final String MINIFI_PID_FILE_NAME = "minifi.pid";
+    private static final String MINIFI_STATUS_FILE_NAME = "minifi.status";
+    private static final String MINIFI_LOCK_FILE_NAME = "minifi.lock";
+    private static final String DEFAULT_CONFIG_FILE = "./conf/bootstrap.conf";
+    private static final String BOOTSTRAP_CONFIG_FILE_SYSTEM_PROPERTY_KEY = "org.apache.nifi.minifi.bootstrap.config.file";
+    private static final String MINIFI_HOME_ENV_VARIABLE_KEY = "MINIFI_HOME";
+    private static final String MINIFI_PID_DIR_PROP = "org.apache.nifi.minifi.bootstrap.config.pid.dir";
+    private static final String DEFAULT_PID_DIR = "bin";
+
+    private final File bootstrapConfigFile;
+
+    public BootstrapFileProvider(File bootstrapConfigFile) {
+        if (bootstrapConfigFile == null || !bootstrapConfigFile.exists()) {
+            throw new IllegalArgumentException("The specified bootstrap file doesn't exists: " + bootstrapConfigFile);
+        }
+        this.bootstrapConfigFile = bootstrapConfigFile;
+    }
+
+    public static File getBootstrapConfFile() {
+        File bootstrapConfigFile = Optional.ofNullable(System.getProperty(BOOTSTRAP_CONFIG_FILE_SYSTEM_PROPERTY_KEY))
+            .map(File::new)
+            .orElseGet(() -> Optional.ofNullable(System.getenv(MINIFI_HOME_ENV_VARIABLE_KEY))
+                .map(File::new)
+                .map(nifiHomeFile -> new File(nifiHomeFile, DEFAULT_CONFIG_FILE))
+                .orElseGet(() -> new File(DEFAULT_CONFIG_FILE)));
+        LOGGER.debug("Bootstrap config file: {}", bootstrapConfigFile);
+        return bootstrapConfigFile;
+    }
+
+    public File getPidFile() throws IOException {
+        return getBootstrapFile(MINIFI_PID_FILE_NAME);
+    }
+
+    public File getStatusFile() throws IOException {
+        return getBootstrapFile(MINIFI_STATUS_FILE_NAME);
+    }
+
+    public File getLockFile() throws IOException {
+        return getBootstrapFile(MINIFI_LOCK_FILE_NAME);
+    }
+
+    public File getReloadLockFile() {
+        File confDir = bootstrapConfigFile.getParentFile();
+        File nifiHome = confDir.getParentFile();
+        File bin = new File(nifiHome, "bin");
+        File reloadFile = new File(bin, "minifi.reload.lock");
+
+        LOGGER.debug("Reload File: {}", reloadFile);
+        return reloadFile;
+    }
+
+    public File getSwapFile() {
+        File confDir = bootstrapConfigFile.getParentFile();
+        File swapFile = new File(confDir, "swap.yml");
+
+        LOGGER.debug("Swap File: {}", swapFile);
+        return swapFile;
+    }
+
+    public Properties getBootstrapProperties() throws IOException {
+        if (!bootstrapConfigFile.exists()) {
+            throw new FileNotFoundException(bootstrapConfigFile.getAbsolutePath());
+        }
+
+        Properties bootstrapProperties = new Properties();
+        try (FileInputStream fis = new FileInputStream(bootstrapConfigFile)) {
+            bootstrapProperties.load(fis);
+        }
+
+        logProperties("Bootstrap", bootstrapProperties);

Review Comment:
   Yes, it can contain the keystore / truststore keys for example. I'll introduce an enum with the sensitive keys, so we can filter based on this.



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/BootstrapFileProvider.java:
##########
@@ -0,0 +1,234 @@
+/*
+ * 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.nifi.minifi.bootstrap.service;
+
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.STATUS_FILE_PID_KEY;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.attribute.PosixFilePermission;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.util.file.FileUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class BootstrapFileProvider {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(BootstrapFileProvider.class);
+
+    private static final String MINIFI_PID_FILE_NAME = "minifi.pid";
+    private static final String MINIFI_STATUS_FILE_NAME = "minifi.status";
+    private static final String MINIFI_LOCK_FILE_NAME = "minifi.lock";
+    private static final String DEFAULT_CONFIG_FILE = "./conf/bootstrap.conf";
+    private static final String BOOTSTRAP_CONFIG_FILE_SYSTEM_PROPERTY_KEY = "org.apache.nifi.minifi.bootstrap.config.file";
+    private static final String MINIFI_HOME_ENV_VARIABLE_KEY = "MINIFI_HOME";
+    private static final String MINIFI_PID_DIR_PROP = "org.apache.nifi.minifi.bootstrap.config.pid.dir";
+    private static final String DEFAULT_PID_DIR = "bin";
+
+    private final File bootstrapConfigFile;
+
+    public BootstrapFileProvider(File bootstrapConfigFile) {
+        if (bootstrapConfigFile == null || !bootstrapConfigFile.exists()) {
+            throw new IllegalArgumentException("The specified bootstrap file doesn't exists: " + bootstrapConfigFile);
+        }
+        this.bootstrapConfigFile = bootstrapConfigFile;
+    }
+
+    public static File getBootstrapConfFile() {
+        File bootstrapConfigFile = Optional.ofNullable(System.getProperty(BOOTSTRAP_CONFIG_FILE_SYSTEM_PROPERTY_KEY))
+            .map(File::new)
+            .orElseGet(() -> Optional.ofNullable(System.getenv(MINIFI_HOME_ENV_VARIABLE_KEY))
+                .map(File::new)
+                .map(nifiHomeFile -> new File(nifiHomeFile, DEFAULT_CONFIG_FILE))
+                .orElseGet(() -> new File(DEFAULT_CONFIG_FILE)));
+        LOGGER.debug("Bootstrap config file: {}", bootstrapConfigFile);
+        return bootstrapConfigFile;
+    }
+
+    public File getPidFile() throws IOException {
+        return getBootstrapFile(MINIFI_PID_FILE_NAME);
+    }
+
+    public File getStatusFile() throws IOException {
+        return getBootstrapFile(MINIFI_STATUS_FILE_NAME);
+    }
+
+    public File getLockFile() throws IOException {
+        return getBootstrapFile(MINIFI_LOCK_FILE_NAME);
+    }
+
+    public File getReloadLockFile() {
+        File confDir = bootstrapConfigFile.getParentFile();
+        File nifiHome = confDir.getParentFile();
+        File bin = new File(nifiHome, "bin");
+        File reloadFile = new File(bin, "minifi.reload.lock");
+
+        LOGGER.debug("Reload File: {}", reloadFile);
+        return reloadFile;
+    }
+
+    public File getSwapFile() {
+        File confDir = bootstrapConfigFile.getParentFile();
+        File swapFile = new File(confDir, "swap.yml");
+
+        LOGGER.debug("Swap File: {}", swapFile);
+        return swapFile;
+    }
+
+    public Properties getBootstrapProperties() throws IOException {
+        if (!bootstrapConfigFile.exists()) {
+            throw new FileNotFoundException(bootstrapConfigFile.getAbsolutePath());
+        }
+
+        Properties bootstrapProperties = new Properties();
+        try (FileInputStream fis = new FileInputStream(bootstrapConfigFile)) {
+            bootstrapProperties.load(fis);
+        }
+
+        logProperties("Bootstrap", bootstrapProperties);
+
+        return bootstrapProperties;
+    }
+
+    public Properties getStatusProperties() {
+        Properties props = new Properties();
+
+        try {
+            File statusFile = getStatusFile();
+            if (statusFile == null || !statusFile.exists()) {
+                LOGGER.debug("No status file to load properties from");
+                return props;
+            }
+
+            try (FileInputStream fis = new FileInputStream(statusFile)) {
+                props.load(fis);
+            }
+        } catch (IOException exception) {
+            LOGGER.error("Failed to load MiNiFi status properties");
+        }
+
+        logProperties("MiNiFi status", props);
+
+        return props;
+    }
+
+    public synchronized void saveStatusProperties(Properties minifiProps) throws IOException {
+        String pid = minifiProps.getProperty(STATUS_FILE_PID_KEY);
+        if (!StringUtils.isBlank(pid)) {
+            writePidFile(pid);
+        }
+
+        File statusFile = getStatusFile();
+        if (statusFile.exists() && !statusFile.delete()) {
+            LOGGER.warn("Failed to delete {}", statusFile);
+        }
+
+        if (!statusFile.createNewFile()) {
+            throw new IOException("Failed to create file " + statusFile);
+        }
+
+        try {
+            Set<PosixFilePermission> perms = new HashSet<>();
+            perms.add(PosixFilePermission.OWNER_WRITE);
+            perms.add(PosixFilePermission.OWNER_READ);
+            perms.add(PosixFilePermission.GROUP_READ);
+            perms.add(PosixFilePermission.OTHERS_READ);
+            Files.setPosixFilePermissions(statusFile.toPath(), perms);
+        } catch (Exception e) {
+            LOGGER.warn("Failed to set permissions so that only the owner can read status file {}; "
+                + "this may allows others to have access to the key needed to communicate with MiNiFi. "
+                + "Permissions should be changed so that only the owner can read this file", statusFile);
+        }
+
+        try (FileOutputStream fos = new FileOutputStream(statusFile)) {
+            minifiProps.store(fos, null);
+            fos.getFD().sync();
+        }
+
+        LOGGER.debug("Saving MiNiFi properties to {}", statusFile);
+        logProperties("Saved MiNiFi", minifiProps);
+    }
+
+    private void writePidFile(String pid) throws IOException {
+        File pidFile = getPidFile();
+        if (pidFile.exists() && !pidFile.delete()) {
+            LOGGER.warn("Failed to delete {}", pidFile);
+        }
+
+        if (!pidFile.createNewFile()) {
+            throw new IOException("Failed to create file " + pidFile);
+        }
+
+        try {
+            Set<PosixFilePermission> perms = new HashSet<>();
+            perms.add(PosixFilePermission.OWNER_READ);
+            perms.add(PosixFilePermission.OWNER_WRITE);
+            Files.setPosixFilePermissions(pidFile.toPath(), perms);
+        } catch (Exception e) {
+            LOGGER.warn("Failed to set permissions so that only the owner can read pid file {}; "
+                + "this may allows others to have access to the key needed to communicate with MiNiFi. "
+                + "Permissions should be changed so that only the owner can read this file", pidFile);
+        }
+
+        try (FileOutputStream fos = new FileOutputStream(pidFile)) {
+            fos.write(pid.getBytes(StandardCharsets.UTF_8));
+            fos.getFD().sync();
+        }
+
+        LOGGER.debug("Saved Pid {} to {}", pid, pidFile);
+    }
+
+    private File getBootstrapFile(String fileName) throws IOException {
+        File configFileDir = Optional.ofNullable(System.getProperty(MINIFI_PID_DIR_PROP))
+            .map(String::trim)
+            .map(File::new)
+            .orElseGet(() -> {
+                File confDir = bootstrapConfigFile.getParentFile();
+                File nifiHome = confDir.getParentFile();
+                return new File(nifiHome, DEFAULT_PID_DIR);
+            });
+
+        FileUtils.ensureDirectoryExistAndCanAccess(configFileDir);
+        File statusFile = new File(configFileDir, fileName);
+        LOGGER.debug("Run File: {}", statusFile);
+
+        return statusFile;
+    }
+
+    private void logProperties(String type, Properties props) {
+        if (LOGGER.isDebugEnabled()) {
+            LOGGER.debug("{} properties: {}", type, props.entrySet()
+                .stream()
+                .filter(e -> {
+                    String key = ((String) e.getKey()).toLowerCase();
+                    return !"secret.key".equals(key) && !key.contains("password") && !key.contains("passwd");

Review Comment:
   I believe logging properties can be useful when debugging so introduced an enum to be able to filter out the sensitive props.



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/MiNiFiCommandSender.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 org.apache.nifi.minifi.bootstrap.service;
+
+import java.io.BufferedReader;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.ObjectInputStream;
+import java.io.OutputStream;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.net.SocketTimeoutException;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MiNiFiCommandSender {
+    private static final Logger LOGGER = LoggerFactory.getLogger(MiNiFiCommandSender.class);
+    private static final String PING_CMD = "PING";
+
+    private final MiNiFiParameters miNiFiParameters;
+
+    public MiNiFiCommandSender(MiNiFiParameters miNiFiParameters) {
+        this.miNiFiParameters = miNiFiParameters;
+    }
+
+    public Optional<String> sendCommand(String cmd, Integer port) throws IOException {
+        Optional<String> response = Optional.empty();
+
+        if (port == null) {
+            LOGGER.info("Apache MiNiFi is not currently running");
+            return response;
+        }
+
+        try (Socket socket = new Socket()) {
+            LOGGER.debug("Connecting to MiNiFi instance");
+            socket.setSoTimeout(10000);
+            socket.connect(new InetSocketAddress("localhost", port));
+            LOGGER.debug("Established connection to MiNiFi instance.");
+
+            LOGGER.debug("Sending {} Command to port {}", cmd, port);
+            OutputStream out = socket.getOutputStream();
+            out.write((cmd + " " + miNiFiParameters.getSecretKey() + "\n").getBytes(StandardCharsets.UTF_8));
+            out.flush();
+            socket.shutdownOutput();
+
+            InputStream in = socket.getInputStream();
+            StringBuilder sb = new StringBuilder();
+            int numLines = 0;
+            try (BufferedReader reader = new BufferedReader(new InputStreamReader(in))) {
+                String line;
+                while ((line = reader.readLine()) != null) {
+                    if (numLines++ > 0) {
+                        sb.append("\n");
+                    }
+                    sb.append(line);
+                }
+            }
+
+            String responseString = sb.toString().trim();
+
+            LOGGER.debug("Received response to {} command: {}", cmd, responseString);
+            response = Optional.of(responseString);
+        }
+        return response;
+    }
+
+    <T> T sendCommandForObject(String cmd, Integer port, String... extraParams) throws IOException {
+        T response;
+        try (Socket socket = new Socket("localhost", port)) {
+            OutputStream out = socket.getOutputStream();
+            String argsString = Arrays.stream(extraParams).collect(Collectors.joining(" ", " ", ""));
+            String commandWithArgs = cmd + " " + miNiFiParameters.getSecretKey() + argsString + "\n";
+            out.write((commandWithArgs).getBytes(StandardCharsets.UTF_8));
+            LOGGER.debug("Sending {} command to MiNiFi with the following args: [{}]", cmd, argsString);
+            out.flush();
+
+            socket.setSoTimeout(5000);
+            InputStream in = socket.getInputStream();
+
+            ObjectInputStream ois = new ObjectInputStream(in);
+            Object o = ois.readObject();

Review Comment:
   done



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/MiNiFiCommandSender.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 org.apache.nifi.minifi.bootstrap.service;
+
+import java.io.BufferedReader;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.ObjectInputStream;
+import java.io.OutputStream;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.net.SocketTimeoutException;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MiNiFiCommandSender {
+    private static final Logger LOGGER = LoggerFactory.getLogger(MiNiFiCommandSender.class);
+    private static final String PING_CMD = "PING";
+
+    private final MiNiFiParameters miNiFiParameters;
+
+    public MiNiFiCommandSender(MiNiFiParameters miNiFiParameters) {
+        this.miNiFiParameters = miNiFiParameters;
+    }
+
+    public Optional<String> sendCommand(String cmd, Integer port) throws IOException {
+        Optional<String> response = Optional.empty();
+
+        if (port == null) {
+            LOGGER.info("Apache MiNiFi is not currently running");
+            return response;
+        }
+
+        try (Socket socket = new Socket()) {
+            LOGGER.debug("Connecting to MiNiFi instance");
+            socket.setSoTimeout(10000);
+            socket.connect(new InetSocketAddress("localhost", port));
+            LOGGER.debug("Established connection to MiNiFi instance.");
+
+            LOGGER.debug("Sending {} Command to port {}", cmd, port);
+            OutputStream out = socket.getOutputStream();
+            out.write((cmd + " " + miNiFiParameters.getSecretKey() + "\n").getBytes(StandardCharsets.UTF_8));
+            out.flush();
+            socket.shutdownOutput();
+
+            InputStream in = socket.getInputStream();
+            StringBuilder sb = new StringBuilder();
+            int numLines = 0;
+            try (BufferedReader reader = new BufferedReader(new InputStreamReader(in))) {
+                String line;
+                while ((line = reader.readLine()) != null) {
+                    if (numLines++ > 0) {
+                        sb.append("\n");
+                    }
+                    sb.append(line);
+                }
+            }
+
+            String responseString = sb.toString().trim();
+
+            LOGGER.debug("Received response to {} command: {}", cmd, responseString);
+            response = Optional.of(responseString);
+        }
+        return response;
+    }
+
+    <T> T sendCommandForObject(String cmd, Integer port, String... extraParams) throws IOException {
+        T response;
+        try (Socket socket = new Socket("localhost", port)) {
+            OutputStream out = socket.getOutputStream();
+            String argsString = Arrays.stream(extraParams).collect(Collectors.joining(" ", " ", ""));
+            String commandWithArgs = cmd + " " + miNiFiParameters.getSecretKey() + argsString + "\n";
+            out.write((commandWithArgs).getBytes(StandardCharsets.UTF_8));
+            LOGGER.debug("Sending {} command to MiNiFi with the following args: [{}]", cmd, argsString);
+            out.flush();
+
+            socket.setSoTimeout(5000);
+            InputStream in = socket.getInputStream();
+
+            ObjectInputStream ois = new ObjectInputStream(in);
+            Object o = ois.readObject();
+            ois.close();
+            out.close();
+            response = castResponse(cmd, o);
+        } catch (EOFException | ClassNotFoundException | SocketTimeoutException e) {
+            String message = "Failed to get response for " + cmd + " Potentially due to the process currently being down (restarting or otherwise). Exception:" + e;
+            LOGGER.error(message);
+            throw new RuntimeException(message);
+        }
+        return response;
+    }
+
+    private <T> T castResponse(String cmd, Object o) {
+        T response;
+        try {
+            response = (T) o;
+        } catch (ClassCastException e) {
+            String message = "Failed to cast " + cmd + " response to the requested type";
+            LOGGER.error(message);
+            throw new RuntimeException(message);

Review Comment:
   Removed the logging as it's done in different level as well.



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/MiNiFiCommandSender.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 org.apache.nifi.minifi.bootstrap.service;
+
+import java.io.BufferedReader;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.ObjectInputStream;
+import java.io.OutputStream;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.net.SocketTimeoutException;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MiNiFiCommandSender {
+    private static final Logger LOGGER = LoggerFactory.getLogger(MiNiFiCommandSender.class);
+    private static final String PING_CMD = "PING";
+
+    private final MiNiFiParameters miNiFiParameters;
+
+    public MiNiFiCommandSender(MiNiFiParameters miNiFiParameters) {
+        this.miNiFiParameters = miNiFiParameters;
+    }
+
+    public Optional<String> sendCommand(String cmd, Integer port) throws IOException {
+        Optional<String> response = Optional.empty();
+
+        if (port == null) {
+            LOGGER.info("Apache MiNiFi is not currently running");
+            return response;
+        }
+
+        try (Socket socket = new Socket()) {
+            LOGGER.debug("Connecting to MiNiFi instance");
+            socket.setSoTimeout(10000);
+            socket.connect(new InetSocketAddress("localhost", port));
+            LOGGER.debug("Established connection to MiNiFi instance.");
+
+            LOGGER.debug("Sending {} Command to port {}", cmd, port);
+            OutputStream out = socket.getOutputStream();
+            out.write((cmd + " " + miNiFiParameters.getSecretKey() + "\n").getBytes(StandardCharsets.UTF_8));
+            out.flush();
+            socket.shutdownOutput();
+
+            InputStream in = socket.getInputStream();
+            StringBuilder sb = new StringBuilder();
+            int numLines = 0;
+            try (BufferedReader reader = new BufferedReader(new InputStreamReader(in))) {
+                String line;
+                while ((line = reader.readLine()) != null) {
+                    if (numLines++ > 0) {
+                        sb.append("\n");
+                    }
+                    sb.append(line);
+                }
+            }
+
+            String responseString = sb.toString().trim();
+
+            LOGGER.debug("Received response to {} command: {}", cmd, responseString);
+            response = Optional.of(responseString);
+        }
+        return response;
+    }
+
+    <T> T sendCommandForObject(String cmd, Integer port, String... extraParams) throws IOException {
+        T response;
+        try (Socket socket = new Socket("localhost", port)) {
+            OutputStream out = socket.getOutputStream();
+            String argsString = Arrays.stream(extraParams).collect(Collectors.joining(" ", " ", ""));
+            String commandWithArgs = cmd + " " + miNiFiParameters.getSecretKey() + argsString + "\n";
+            out.write((commandWithArgs).getBytes(StandardCharsets.UTF_8));
+            LOGGER.debug("Sending {} command to MiNiFi with the following args: [{}]", cmd, argsString);
+            out.flush();
+
+            socket.setSoTimeout(5000);
+            InputStream in = socket.getInputStream();
+
+            ObjectInputStream ois = new ObjectInputStream(in);
+            Object o = ois.readObject();
+            ois.close();
+            out.close();
+            response = castResponse(cmd, o);
+        } catch (EOFException | ClassNotFoundException | SocketTimeoutException e) {
+            String message = "Failed to get response for " + cmd + " Potentially due to the process currently being down (restarting or otherwise). Exception:" + e;
+            LOGGER.error(message);

Review Comment:
   done



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/BootstrapFileProvider.java:
##########
@@ -0,0 +1,234 @@
+/*
+ * 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.nifi.minifi.bootstrap.service;
+
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.STATUS_FILE_PID_KEY;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.attribute.PosixFilePermission;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.util.file.FileUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class BootstrapFileProvider {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(BootstrapFileProvider.class);
+
+    private static final String MINIFI_PID_FILE_NAME = "minifi.pid";
+    private static final String MINIFI_STATUS_FILE_NAME = "minifi.status";
+    private static final String MINIFI_LOCK_FILE_NAME = "minifi.lock";
+    private static final String DEFAULT_CONFIG_FILE = "./conf/bootstrap.conf";
+    private static final String BOOTSTRAP_CONFIG_FILE_SYSTEM_PROPERTY_KEY = "org.apache.nifi.minifi.bootstrap.config.file";
+    private static final String MINIFI_HOME_ENV_VARIABLE_KEY = "MINIFI_HOME";
+    private static final String MINIFI_PID_DIR_PROP = "org.apache.nifi.minifi.bootstrap.config.pid.dir";
+    private static final String DEFAULT_PID_DIR = "bin";
+
+    private final File bootstrapConfigFile;
+
+    public BootstrapFileProvider(File bootstrapConfigFile) {
+        if (bootstrapConfigFile == null || !bootstrapConfigFile.exists()) {
+            throw new IllegalArgumentException("The specified bootstrap file doesn't exists: " + bootstrapConfigFile);
+        }
+        this.bootstrapConfigFile = bootstrapConfigFile;
+    }
+
+    public static File getBootstrapConfFile() {
+        File bootstrapConfigFile = Optional.ofNullable(System.getProperty(BOOTSTRAP_CONFIG_FILE_SYSTEM_PROPERTY_KEY))
+            .map(File::new)
+            .orElseGet(() -> Optional.ofNullable(System.getenv(MINIFI_HOME_ENV_VARIABLE_KEY))
+                .map(File::new)
+                .map(nifiHomeFile -> new File(nifiHomeFile, DEFAULT_CONFIG_FILE))
+                .orElseGet(() -> new File(DEFAULT_CONFIG_FILE)));
+        LOGGER.debug("Bootstrap config file: {}", bootstrapConfigFile);
+        return bootstrapConfigFile;
+    }
+
+    public File getPidFile() throws IOException {
+        return getBootstrapFile(MINIFI_PID_FILE_NAME);
+    }
+
+    public File getStatusFile() throws IOException {
+        return getBootstrapFile(MINIFI_STATUS_FILE_NAME);
+    }
+
+    public File getLockFile() throws IOException {
+        return getBootstrapFile(MINIFI_LOCK_FILE_NAME);
+    }
+
+    public File getReloadLockFile() {
+        File confDir = bootstrapConfigFile.getParentFile();
+        File nifiHome = confDir.getParentFile();
+        File bin = new File(nifiHome, "bin");
+        File reloadFile = new File(bin, "minifi.reload.lock");
+
+        LOGGER.debug("Reload File: {}", reloadFile);
+        return reloadFile;
+    }
+
+    public File getSwapFile() {
+        File confDir = bootstrapConfigFile.getParentFile();
+        File swapFile = new File(confDir, "swap.yml");
+
+        LOGGER.debug("Swap File: {}", swapFile);
+        return swapFile;
+    }
+
+    public Properties getBootstrapProperties() throws IOException {
+        if (!bootstrapConfigFile.exists()) {
+            throw new FileNotFoundException(bootstrapConfigFile.getAbsolutePath());
+        }
+
+        Properties bootstrapProperties = new Properties();
+        try (FileInputStream fis = new FileInputStream(bootstrapConfigFile)) {
+            bootstrapProperties.load(fis);
+        }
+
+        logProperties("Bootstrap", bootstrapProperties);
+
+        return bootstrapProperties;
+    }
+
+    public Properties getStatusProperties() {
+        Properties props = new Properties();
+
+        try {
+            File statusFile = getStatusFile();
+            if (statusFile == null || !statusFile.exists()) {
+                LOGGER.debug("No status file to load properties from");
+                return props;
+            }
+
+            try (FileInputStream fis = new FileInputStream(statusFile)) {
+                props.load(fis);
+            }
+        } catch (IOException exception) {
+            LOGGER.error("Failed to load MiNiFi status properties");
+        }
+
+        logProperties("MiNiFi status", props);
+
+        return props;
+    }
+
+    public synchronized void saveStatusProperties(Properties minifiProps) throws IOException {
+        String pid = minifiProps.getProperty(STATUS_FILE_PID_KEY);
+        if (!StringUtils.isBlank(pid)) {
+            writePidFile(pid);
+        }
+
+        File statusFile = getStatusFile();
+        if (statusFile.exists() && !statusFile.delete()) {
+            LOGGER.warn("Failed to delete {}", statusFile);
+        }
+
+        if (!statusFile.createNewFile()) {
+            throw new IOException("Failed to create file " + statusFile);
+        }
+
+        try {
+            Set<PosixFilePermission> perms = new HashSet<>();
+            perms.add(PosixFilePermission.OWNER_WRITE);
+            perms.add(PosixFilePermission.OWNER_READ);
+            perms.add(PosixFilePermission.GROUP_READ);
+            perms.add(PosixFilePermission.OTHERS_READ);
+            Files.setPosixFilePermissions(statusFile.toPath(), perms);
+        } catch (Exception e) {
+            LOGGER.warn("Failed to set permissions so that only the owner can read status file {}; "
+                + "this may allows others to have access to the key needed to communicate with MiNiFi. "
+                + "Permissions should be changed so that only the owner can read this file", statusFile);
+        }
+
+        try (FileOutputStream fos = new FileOutputStream(statusFile)) {
+            minifiProps.store(fos, null);
+            fos.getFD().sync();
+        }
+
+        LOGGER.debug("Saving MiNiFi properties to {}", statusFile);
+        logProperties("Saved MiNiFi", minifiProps);

Review Comment:
   Introduced a filter based on the secret keys enum



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/command/EnvRunner.java:
##########
@@ -0,0 +1,107 @@
+/*
+ * 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.nifi.minifi.bootstrap.command;
+
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CMD_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.DEFAULT_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.Status.ERROR;
+import static org.apache.nifi.minifi.bootstrap.Status.MINIFI_NOT_RUNNING;
+import static org.apache.nifi.minifi.bootstrap.Status.OK;
+
+import java.lang.reflect.Method;
+import java.util.Map.Entry;
+import java.util.Properties;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.apache.nifi.minifi.bootstrap.MiNiFiStatus;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiStatusProvider;
+
+public class EnvRunner implements CommandRunner {
+    private final MiNiFiParameters miNiFiParameters;
+    private final MiNiFiStatusProvider miNiFiStatusProvider;
+
+    public EnvRunner(MiNiFiParameters miNiFiParameters, MiNiFiStatusProvider miNiFiStatusProvider) {
+        this.miNiFiParameters = miNiFiParameters;
+        this.miNiFiStatusProvider = miNiFiStatusProvider;
+    }
+
+    /**
+     * Returns information about the MiNiFi's virtual machine.
+     * @param args the input arguments
+     * @return status code
+     */
+    @Override
+    public int runCommand(String[] args) {
+        return env();
+    }
+
+    private int env() {
+        MiNiFiStatus status = miNiFiStatusProvider.getStatus(miNiFiParameters.getMiNiFiPort(), miNiFiParameters.getMinifiPid());
+        if (status.getPid() == null) {
+            CMD_LOGGER.error("Apache MiNiFi is not running");
+            return MINIFI_NOT_RUNNING.getStatusCode();
+        }
+
+        Class<?> virtualMachineClass;
+        try {
+            virtualMachineClass = Class.forName("com.sun.tools.attach.VirtualMachine");
+        } catch (ClassNotFoundException cnfe) {
+            CMD_LOGGER.error("Seems tools.jar (Linux / Windows JDK) or classes.jar (Mac OS) is not available in classpath");
+            return ERROR.getStatusCode();
+        }
+
+        Method attachMethod;
+        Method detachMethod;
+        try {
+            attachMethod = virtualMachineClass.getMethod("attach", String.class);
+            detachMethod = virtualMachineClass.getDeclaredMethod("detach");
+        } catch (Exception e) {
+            CMD_LOGGER.error("Methods required for getting environment not available");
+            DEFAULT_LOGGER.error("Exception:", e);
+            return ERROR.getStatusCode();
+        }
+
+        Object virtualMachine;
+        try {
+            virtualMachine = attachMethod.invoke(null, status.getPid());
+        } catch (Exception e) {
+            CMD_LOGGER.error("Problem attaching to MiNiFi");
+            DEFAULT_LOGGER.error("Exception:", e);

Review Comment:
   done



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/command/StartRunner.java:
##########
@@ -0,0 +1,435 @@
+/*
+ * 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.nifi.minifi.bootstrap.command;
+
+import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CMD_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CONF_DIR_KEY;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.DEFAULT_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.MINIFI_CONFIG_FILE_KEY;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.STATUS_FILE_PID_KEY;
+import static org.apache.nifi.minifi.bootstrap.Status.ERROR;
+import static org.apache.nifi.minifi.bootstrap.Status.OK;
+import static org.apache.nifi.minifi.bootstrap.util.ConfigTransformer.generateConfigFiles;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.nifi.bootstrap.util.OSUtils;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.apache.nifi.minifi.bootstrap.RunMiNiFi;
+import org.apache.nifi.minifi.bootstrap.ShutdownHook;
+import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeException;
+import org.apache.nifi.minifi.bootstrap.exception.StartupFailureException;
+import org.apache.nifi.minifi.bootstrap.service.BootstrapFileProvider;
+import org.apache.nifi.minifi.bootstrap.service.CurrentPortProvider;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiListener;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiStdLogHandler;
+import org.apache.nifi.minifi.bootstrap.service.PeriodicStatusReporterManager;
+import org.apache.nifi.minifi.bootstrap.util.ProcessUtils;
+import org.apache.nifi.util.Tuple;
+
+public class StartRunner implements CommandRunner {
+    private static final String DEFAULT_JAVA_CMD = "java";
+    private static final String DEFAULT_LOG_DIR = "./logs";
+    private static final String DEFAULT_LIB_DIR = "./lib";
+    private static final String DEFAULT_CONF_DIR = "./conf";
+    private static final int STARTUP_WAIT_SECONDS = 60;
+    private static final String DEFAULT_CONFIG_FILE = DEFAULT_CONF_DIR + "/bootstrap.conf";
+
+    private final CurrentPortProvider currentPortProvider;
+    private final BootstrapFileProvider bootstrapFileProvider;
+    private final PeriodicStatusReporterManager periodicStatusReporterManager;
+    private final MiNiFiStdLogHandler miNiFiStdLogHandler;
+    private final MiNiFiParameters miNiFiParameters;
+    private final File bootstrapConfigFile;
+    private final Lock lock = new ReentrantLock();
+    private final Condition startupCondition = lock.newCondition();
+    private final RunMiNiFi runMiNiFi;
+    private volatile ShutdownHook shutdownHook;
+
+    public StartRunner(CurrentPortProvider currentPortProvider, BootstrapFileProvider bootstrapFileProvider,
+        PeriodicStatusReporterManager periodicStatusReporterManager, MiNiFiStdLogHandler miNiFiStdLogHandler, MiNiFiParameters miNiFiParameters, File bootstrapConfigFile,
+        RunMiNiFi runMiNiFi) {
+        this.currentPortProvider = currentPortProvider;
+        this.bootstrapFileProvider = bootstrapFileProvider;
+        this.periodicStatusReporterManager = periodicStatusReporterManager;
+        this.miNiFiStdLogHandler = miNiFiStdLogHandler;
+        this.miNiFiParameters = miNiFiParameters;
+        this.bootstrapConfigFile = bootstrapConfigFile;
+        this.runMiNiFi = runMiNiFi;
+    }
+
+    /**
+     * Starts (and restarts) MiNiFi process during the whole lifecycle of the bootstrap process.
+     * @param args the input arguments
+     * @return status code
+     */
+    @Override
+    public int runCommand(String[] args) {
+        try {
+            start();
+        } catch (Exception e) {
+            CMD_LOGGER.error("Exception happened during MiNiFi startup", e);
+            return ERROR.getStatusCode();
+        }
+        return OK.getStatusCode();
+    }
+
+    private void start() throws IOException, InterruptedException {
+        Integer port = currentPortProvider.getCurrentPort();
+        if (port != null) {
+            String alreadyRunningMessage = "Apache MiNiFi is already running";
+            CMD_LOGGER.info(alreadyRunningMessage + ", listening to Bootstrap on port {}", port);
+            return;
+        }
+
+        File prevLockFile = bootstrapFileProvider.getLockFile();
+        if (prevLockFile.exists() && !prevLockFile.delete()) {
+            CMD_LOGGER.warn("Failed to delete previous lock file {}; this file should be cleaned up manually", prevLockFile);
+        }
+
+        Properties bootstrapProperties = bootstrapFileProvider.getBootstrapProperties();
+        String confDir = bootstrapProperties.getProperty(CONF_DIR_KEY);
+        initConfigFiles(bootstrapProperties, confDir);
+
+        Tuple<ProcessBuilder, Process> tuple = startMiNiFi();
+        ProcessBuilder builder = tuple.getKey();
+        Process process = tuple.getValue();
+
+        try {
+            while (true) {
+                if (ProcessUtils.isAlive(process)) {
+                    handleReload();
+                } else {
+                    Runtime runtime = Runtime.getRuntime();
+                    try {
+                        runtime.removeShutdownHook(shutdownHook);
+                    } catch (IllegalStateException ise) {
+                        // happens when already shutting down
+                    }
+
+                    if (runMiNiFi.isAutoRestartNiFi() && needRestart()) {
+                        File reloadFile = bootstrapFileProvider.getReloadLockFile();
+                        if (reloadFile.exists()) {
+                            DEFAULT_LOGGER.info("Currently reloading configuration. Will wait to restart MiNiFi.");
+                            Thread.sleep(5000L);
+                            continue;
+                        }
+
+                        process = restartNifi(bootstrapProperties, confDir, builder, runtime);
+                        // failed to start process
+                        if (process == null) {
+                            return;
+                        }
+                    } else {
+                        return;
+                    }
+                }
+            }
+        } finally {
+            miNiFiStdLogHandler.shutdown();
+            runMiNiFi.shutdownChangeNotifier();
+            periodicStatusReporterManager.shutdownPeriodicStatusReporters();
+        }
+    }
+
+    private Process restartNifi(Properties bootstrapProperties, String confDir, ProcessBuilder builder, Runtime runtime) throws IOException {
+        Process process;
+        boolean previouslyStarted = runMiNiFi.isNiFiStarted();
+        if (!previouslyStarted) {
+            File swapConfigFile = bootstrapFileProvider.getSwapFile();
+            if (swapConfigFile.exists()) {
+                DEFAULT_LOGGER.info("Swap file exists, MiNiFi failed trying to change configuration. Reverting to old configuration.");
+
+                try {
+                    ByteBuffer tempConfigFile = generateConfigFiles(new FileInputStream(swapConfigFile), confDir, bootstrapProperties);
+                    runMiNiFi.getConfigFileReference().set(tempConfigFile.asReadOnlyBuffer());
+                } catch (ConfigurationChangeException e) {
+                    DEFAULT_LOGGER.error("The swap file is malformed, unable to restart from prior state. Will not attempt to restart MiNiFi. Swap File should be cleaned up manually.");
+                    return null;
+                }
+
+                Files.copy(swapConfigFile.toPath(), Paths.get(bootstrapProperties.getProperty(MINIFI_CONFIG_FILE_KEY)), REPLACE_EXISTING);
+
+                DEFAULT_LOGGER.info("Replacing config file with swap file and deleting swap file");
+                if (!swapConfigFile.delete()) {
+                    DEFAULT_LOGGER.warn("The swap file failed to delete after replacing using it to revert to the old configuration. It should be cleaned up manually.");
+                }
+                runMiNiFi.setReloading(false);
+            } else {
+                DEFAULT_LOGGER.info("MiNiFi either never started or failed to restart. Will not attempt to restart MiNiFi");
+                return null;
+            }
+        } else {
+            runMiNiFi.setNiFiStarted(false);
+        }
+
+        miNiFiParameters.setSecretKey(null);
+        process = builder.start();
+        miNiFiStdLogHandler.initLogging(process);
+
+        Long pid = OSUtils.getProcessId(process, DEFAULT_LOGGER);
+        if (pid != null) {
+            miNiFiParameters.setMinifiPid(pid);
+            Properties minifiProps = new Properties();
+            minifiProps.setProperty(STATUS_FILE_PID_KEY, String.valueOf(pid));
+            bootstrapFileProvider.saveStatusProperties(minifiProps);
+        }
+
+        shutdownHook = new ShutdownHook(runMiNiFi, miNiFiStdLogHandler);
+        runtime.addShutdownHook(shutdownHook);
+
+        boolean started = waitForStart();
+
+        if (started) {
+            DEFAULT_LOGGER.info("Successfully spawned the thread to start Apache MiNiFi{}", (pid == null ? "" : " with PID " + pid));
+        } else {
+            DEFAULT_LOGGER.error("Apache MiNiFi does not appear to have started");
+        }
+        return process;
+    }
+
+    private boolean needRestart() throws IOException {
+        boolean needRestart = true;
+        File statusFile = bootstrapFileProvider.getStatusFile();
+        if (!statusFile.exists()) {
+            DEFAULT_LOGGER.info("Status File no longer exists. Will not restart MiNiFi");
+            return false;
+        }
+
+        File lockFile = bootstrapFileProvider.getLockFile();
+        if (lockFile.exists()) {
+            DEFAULT_LOGGER.info("A shutdown was initiated. Will not restart MiNiFi");
+            return false;
+        }
+        return needRestart;
+    }
+
+    private void handleReload() {
+        try {
+            Thread.sleep(1000L);
+            if (runMiNiFi.getReloading() && runMiNiFi.isNiFiStarted()) {
+                File swapConfigFile = bootstrapFileProvider.getSwapFile();
+                if (swapConfigFile.exists()) {
+                    DEFAULT_LOGGER.info("MiNiFi has finished reloading successfully and swap file exists. Deleting old configuration.");
+
+                    if (swapConfigFile.delete()) {
+                        DEFAULT_LOGGER.info("Swap file was successfully deleted.");
+                    } else {
+                        DEFAULT_LOGGER.error("Swap file was not deleted. It should be deleted manually.");
+                    }
+                }
+                runMiNiFi.setReloading(false);
+            }
+        } catch (InterruptedException ie) {
+        }
+    }
+
+    private void initConfigFiles(Properties bootstrapProperties, String confDir) throws IOException {
+        File configFile = new File(bootstrapProperties.getProperty(MINIFI_CONFIG_FILE_KEY));
+        try (InputStream inputStream = new FileInputStream(configFile)) {
+            ByteBuffer tempConfigFile = generateConfigFiles(inputStream, confDir, bootstrapProperties);
+            runMiNiFi.getConfigFileReference().set(tempConfigFile.asReadOnlyBuffer());
+        } catch (FileNotFoundException e) {
+            String fileNotFoundMessage = "The config file defined in " + MINIFI_CONFIG_FILE_KEY + " does not exists.";
+            DEFAULT_LOGGER.error(fileNotFoundMessage, e);
+            throw new StartupFailureException(fileNotFoundMessage);
+        } catch (ConfigurationChangeException e) {
+            String malformedConfigFileMessage = "The config file is malformed, unable to start.";
+            DEFAULT_LOGGER.error(malformedConfigFileMessage, e);
+            throw new StartupFailureException(malformedConfigFileMessage);
+        }
+    }
+
+    private Tuple<ProcessBuilder, Process> startMiNiFi() throws IOException {
+        ProcessBuilder builder = new ProcessBuilder();
+
+        Properties props = bootstrapFileProvider.getBootstrapProperties();
+        File bootstrapConfigAbsoluteFile = bootstrapConfigFile.getAbsoluteFile();
+        File binDir = bootstrapConfigAbsoluteFile.getParentFile();
+
+        File workingDir = Optional.ofNullable(props.getProperty("working.dir"))
+            .map(File::new)
+            .orElse(binDir.getParentFile());
+
+        builder.directory(workingDir);
+
+        String minifiLogDir = System.getProperty("org.apache.nifi.minifi.bootstrap.config.log.dir", DEFAULT_LOG_DIR).trim();
+        File libDir = getFile(props.getProperty("lib.dir", DEFAULT_LIB_DIR).trim(), workingDir);
+        File confDir = getFile(props.getProperty(CONF_DIR_KEY, DEFAULT_CONF_DIR).trim(), workingDir);
+
+        String minifiPropsFilename = props.getProperty("props.file");
+        if (minifiPropsFilename == null) {
+            if (confDir.exists()) {
+                minifiPropsFilename = new File(confDir, "nifi.properties").getAbsolutePath();
+            } else {
+                minifiPropsFilename = DEFAULT_CONFIG_FILE;
+            }
+        }
+
+        minifiPropsFilename = minifiPropsFilename.trim();
+
+        List<String> javaAdditionalArgs = new ArrayList<>();
+        for (Entry<Object, Object> entry : props.entrySet()) {
+            String key = (String) entry.getKey();
+            String value = (String) entry.getValue();
+
+            if (key.startsWith("java.arg")) {
+                javaAdditionalArgs.add(value);
+            }
+        }
+
+        File[] libFiles = libDir.listFiles((dir, filename) -> filename.toLowerCase().endsWith(".jar"));
+
+        if (libFiles == null || libFiles.length == 0) {
+            throw new RuntimeException("Could not find lib directory at " + libDir.getAbsolutePath());
+        }
+
+        File[] confFiles = confDir.listFiles();
+        if (confFiles == null || confFiles.length == 0) {
+            throw new RuntimeException("Could not find conf directory at " + confDir.getAbsolutePath());
+        }
+
+        List<String> cpFiles = new ArrayList<>(confFiles.length + libFiles.length);
+        cpFiles.add(confDir.getAbsolutePath());
+        for (File file : libFiles) {
+            cpFiles.add(file.getAbsolutePath());
+        }
+
+        StringBuilder classPathBuilder = new StringBuilder();
+        for (int i = 0; i < cpFiles.size(); i++) {
+            String filename = cpFiles.get(i);
+            classPathBuilder.append(filename);
+            if (i < cpFiles.size() - 1) {
+                classPathBuilder.append(File.pathSeparatorChar);
+            }
+        }
+
+        String classPath = classPathBuilder.toString();
+        String javaCmd = props.getProperty("java");
+        if (javaCmd == null) {
+            javaCmd = DEFAULT_JAVA_CMD;
+        }
+        if (javaCmd.equals(DEFAULT_JAVA_CMD)) {
+            String javaHome = System.getenv("JAVA_HOME");
+            if (javaHome != null) {
+                String fileExtension = isWindows() ? ".exe" : "";

Review Comment:
   Do you mean by executing the "where java" command ?



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/util/ConfigTransformer.java:
##########
@@ -88,42 +91,68 @@ public final class ConfigTransformer {
     private ConfigTransformer() {
     }
 
+    public static ByteBuffer generateConfigFiles(InputStream configIs, String configDestinationPath, Properties bootstrapProperties) throws ConfigurationChangeException, IOException {

Review Comment:
   No from my side, it's just an existing code movement. If you don't mind I would leave it as it is for now because I'm unsure about the impact of the change. We'll move to the flow json format soon, so this whole config transformer will be removed / refactored.



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/command/CommandRunner.java:
##########
@@ -0,0 +1,28 @@
+/*
+ * 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.nifi.minifi.bootstrap.command;
+
+public interface CommandRunner {
+
+    /**
+     * Executes a command.
+     * @param args the input arguments
+     * @return status code
+     */
+    int runCommand(String[] args);

Review Comment:
   sure, changed it



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/MiNiFiCommandSender.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 org.apache.nifi.minifi.bootstrap.service;
+
+import java.io.BufferedReader;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.ObjectInputStream;
+import java.io.OutputStream;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.net.SocketTimeoutException;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MiNiFiCommandSender {
+    private static final Logger LOGGER = LoggerFactory.getLogger(MiNiFiCommandSender.class);
+    private static final String PING_CMD = "PING";
+
+    private final MiNiFiParameters miNiFiParameters;
+
+    public MiNiFiCommandSender(MiNiFiParameters miNiFiParameters) {
+        this.miNiFiParameters = miNiFiParameters;
+    }
+
+    public Optional<String> sendCommand(String cmd, Integer port) throws IOException {
+        Optional<String> response = Optional.empty();
+
+        if (port == null) {
+            LOGGER.info("Apache MiNiFi is not currently running");
+            return response;
+        }
+
+        try (Socket socket = new Socket()) {
+            LOGGER.debug("Connecting to MiNiFi instance");
+            socket.setSoTimeout(10000);
+            socket.connect(new InetSocketAddress("localhost", port));
+            LOGGER.debug("Established connection to MiNiFi instance.");
+
+            LOGGER.debug("Sending {} Command to port {}", cmd, port);
+            OutputStream out = socket.getOutputStream();
+            out.write((cmd + " " + miNiFiParameters.getSecretKey() + "\n").getBytes(StandardCharsets.UTF_8));
+            out.flush();
+            socket.shutdownOutput();
+
+            InputStream in = socket.getInputStream();
+            StringBuilder sb = new StringBuilder();
+            int numLines = 0;
+            try (BufferedReader reader = new BufferedReader(new InputStreamReader(in))) {
+                String line;
+                while ((line = reader.readLine()) != null) {
+                    if (numLines++ > 0) {
+                        sb.append("\n");
+                    }
+                    sb.append(line);
+                }
+            }
+
+            String responseString = sb.toString().trim();
+
+            LOGGER.debug("Received response to {} command: {}", cmd, responseString);
+            response = Optional.of(responseString);
+        }
+        return response;
+    }
+
+    <T> T sendCommandForObject(String cmd, Integer port, String... extraParams) throws IOException {
+        T response;
+        try (Socket socket = new Socket("localhost", port)) {
+            OutputStream out = socket.getOutputStream();
+            String argsString = Arrays.stream(extraParams).collect(Collectors.joining(" ", " ", ""));
+            String commandWithArgs = cmd + " " + miNiFiParameters.getSecretKey() + argsString + "\n";
+            out.write((commandWithArgs).getBytes(StandardCharsets.UTF_8));
+            LOGGER.debug("Sending {} command to MiNiFi with the following args: [{}]", cmd, argsString);
+            out.flush();
+
+            socket.setSoTimeout(5000);

Review Comment:
   done



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/command/EnvRunner.java:
##########
@@ -0,0 +1,107 @@
+/*
+ * 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.nifi.minifi.bootstrap.command;
+
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CMD_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.DEFAULT_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.Status.ERROR;
+import static org.apache.nifi.minifi.bootstrap.Status.MINIFI_NOT_RUNNING;
+import static org.apache.nifi.minifi.bootstrap.Status.OK;
+
+import java.lang.reflect.Method;
+import java.util.Map.Entry;
+import java.util.Properties;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.apache.nifi.minifi.bootstrap.MiNiFiStatus;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiStatusProvider;
+
+public class EnvRunner implements CommandRunner {
+    private final MiNiFiParameters miNiFiParameters;
+    private final MiNiFiStatusProvider miNiFiStatusProvider;
+
+    public EnvRunner(MiNiFiParameters miNiFiParameters, MiNiFiStatusProvider miNiFiStatusProvider) {
+        this.miNiFiParameters = miNiFiParameters;
+        this.miNiFiStatusProvider = miNiFiStatusProvider;
+    }
+
+    /**
+     * Returns information about the MiNiFi's virtual machine.
+     * @param args the input arguments
+     * @return status code
+     */
+    @Override
+    public int runCommand(String[] args) {
+        return env();
+    }
+
+    private int env() {
+        MiNiFiStatus status = miNiFiStatusProvider.getStatus(miNiFiParameters.getMiNiFiPort(), miNiFiParameters.getMinifiPid());
+        if (status.getPid() == null) {
+            CMD_LOGGER.error("Apache MiNiFi is not running");
+            return MINIFI_NOT_RUNNING.getStatusCode();
+        }
+
+        Class<?> virtualMachineClass;
+        try {
+            virtualMachineClass = Class.forName("com.sun.tools.attach.VirtualMachine");
+        } catch (ClassNotFoundException cnfe) {
+            CMD_LOGGER.error("Seems tools.jar (Linux / Windows JDK) or classes.jar (Mac OS) is not available in classpath");
+            return ERROR.getStatusCode();
+        }
+
+        Method attachMethod;
+        Method detachMethod;
+        try {
+            attachMethod = virtualMachineClass.getMethod("attach", String.class);
+            detachMethod = virtualMachineClass.getDeclaredMethod("detach");
+        } catch (Exception e) {
+            CMD_LOGGER.error("Methods required for getting environment not available");
+            DEFAULT_LOGGER.error("Exception:", e);
+            return ERROR.getStatusCode();
+        }
+
+        Object virtualMachine;
+        try {
+            virtualMachine = attachMethod.invoke(null, status.getPid());
+        } catch (Exception e) {
+            CMD_LOGGER.error("Problem attaching to MiNiFi");
+            DEFAULT_LOGGER.error("Exception:", e);
+            return ERROR.getStatusCode();
+        }
+
+        try {
+            Method getSystemPropertiesMethod = virtualMachine.getClass().getMethod("getSystemProperties");
+
+            Properties sysProps = (Properties) getSystemPropertiesMethod.invoke(virtualMachine);
+            for (Entry<Object, Object> syspropEntry : sysProps.entrySet()) {
+                CMD_LOGGER.info(syspropEntry.getKey().toString() + " = " + syspropEntry.getValue().toString());
+            }
+        } catch (Exception e) {
+            CMD_LOGGER.error("Exception happened during the systemproperties call");
+            DEFAULT_LOGGER.error("Exception:", e);
+            return ERROR.getStatusCode();
+        } finally {
+            try {
+                detachMethod.invoke(virtualMachine);
+            } catch (final Exception e) {
+                CMD_LOGGER.warn("Caught exception detaching from process", e);

Review Comment:
   done



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/command/EnvRunner.java:
##########
@@ -0,0 +1,107 @@
+/*
+ * 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.nifi.minifi.bootstrap.command;
+
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CMD_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.DEFAULT_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.Status.ERROR;
+import static org.apache.nifi.minifi.bootstrap.Status.MINIFI_NOT_RUNNING;
+import static org.apache.nifi.minifi.bootstrap.Status.OK;
+
+import java.lang.reflect.Method;
+import java.util.Map.Entry;
+import java.util.Properties;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.apache.nifi.minifi.bootstrap.MiNiFiStatus;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiStatusProvider;
+
+public class EnvRunner implements CommandRunner {
+    private final MiNiFiParameters miNiFiParameters;
+    private final MiNiFiStatusProvider miNiFiStatusProvider;
+
+    public EnvRunner(MiNiFiParameters miNiFiParameters, MiNiFiStatusProvider miNiFiStatusProvider) {
+        this.miNiFiParameters = miNiFiParameters;
+        this.miNiFiStatusProvider = miNiFiStatusProvider;
+    }
+
+    /**
+     * Returns information about the MiNiFi's virtual machine.
+     * @param args the input arguments
+     * @return status code
+     */
+    @Override
+    public int runCommand(String[] args) {
+        return env();
+    }
+
+    private int env() {
+        MiNiFiStatus status = miNiFiStatusProvider.getStatus(miNiFiParameters.getMiNiFiPort(), miNiFiParameters.getMinifiPid());
+        if (status.getPid() == null) {
+            CMD_LOGGER.error("Apache MiNiFi is not running");
+            return MINIFI_NOT_RUNNING.getStatusCode();
+        }
+
+        Class<?> virtualMachineClass;
+        try {
+            virtualMachineClass = Class.forName("com.sun.tools.attach.VirtualMachine");

Review Comment:
   Works with java8 and 11 but the "virtualMachine.getClass().getMethod("getSystemProperties");" throws exception on java 17.
   `java.lang.IllegalAccessException: class org.apache.nifi.minifi.bootstrap.command.EnvRunner cannot access class sun.tools.attach.HotSpotVirtualMachine (in module jdk.attach) because module jdk.attach does not export sun.tools.attach to unnamed module @41a962cf
   `
   Do we need to support java 17?



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/command/EnvRunner.java:
##########
@@ -0,0 +1,107 @@
+/*
+ * 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.nifi.minifi.bootstrap.command;
+
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CMD_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.DEFAULT_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.Status.ERROR;
+import static org.apache.nifi.minifi.bootstrap.Status.MINIFI_NOT_RUNNING;
+import static org.apache.nifi.minifi.bootstrap.Status.OK;
+
+import java.lang.reflect.Method;
+import java.util.Map.Entry;
+import java.util.Properties;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.apache.nifi.minifi.bootstrap.MiNiFiStatus;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiStatusProvider;
+
+public class EnvRunner implements CommandRunner {
+    private final MiNiFiParameters miNiFiParameters;
+    private final MiNiFiStatusProvider miNiFiStatusProvider;
+
+    public EnvRunner(MiNiFiParameters miNiFiParameters, MiNiFiStatusProvider miNiFiStatusProvider) {
+        this.miNiFiParameters = miNiFiParameters;
+        this.miNiFiStatusProvider = miNiFiStatusProvider;
+    }
+
+    /**
+     * Returns information about the MiNiFi's virtual machine.
+     * @param args the input arguments
+     * @return status code
+     */
+    @Override
+    public int runCommand(String[] args) {
+        return env();
+    }
+
+    private int env() {
+        MiNiFiStatus status = miNiFiStatusProvider.getStatus(miNiFiParameters.getMiNiFiPort(), miNiFiParameters.getMinifiPid());
+        if (status.getPid() == null) {
+            CMD_LOGGER.error("Apache MiNiFi is not running");
+            return MINIFI_NOT_RUNNING.getStatusCode();
+        }
+
+        Class<?> virtualMachineClass;
+        try {
+            virtualMachineClass = Class.forName("com.sun.tools.attach.VirtualMachine");
+        } catch (ClassNotFoundException cnfe) {
+            CMD_LOGGER.error("Seems tools.jar (Linux / Windows JDK) or classes.jar (Mac OS) is not available in classpath");
+            return ERROR.getStatusCode();
+        }
+
+        Method attachMethod;
+        Method detachMethod;
+        try {
+            attachMethod = virtualMachineClass.getMethod("attach", String.class);
+            detachMethod = virtualMachineClass.getDeclaredMethod("detach");
+        } catch (Exception e) {
+            CMD_LOGGER.error("Methods required for getting environment not available");
+            DEFAULT_LOGGER.error("Exception:", e);
+            return ERROR.getStatusCode();
+        }
+
+        Object virtualMachine;
+        try {
+            virtualMachine = attachMethod.invoke(null, status.getPid());
+        } catch (Exception e) {
+            CMD_LOGGER.error("Problem attaching to MiNiFi");
+            DEFAULT_LOGGER.error("Exception:", e);
+            return ERROR.getStatusCode();
+        }
+
+        try {
+            Method getSystemPropertiesMethod = virtualMachine.getClass().getMethod("getSystemProperties");
+
+            Properties sysProps = (Properties) getSystemPropertiesMethod.invoke(virtualMachine);
+            for (Entry<Object, Object> syspropEntry : sysProps.entrySet()) {
+                CMD_LOGGER.info(syspropEntry.getKey().toString() + " = " + syspropEntry.getValue().toString());
+            }
+        } catch (Exception e) {
+            CMD_LOGGER.error("Exception happened during the systemproperties call");
+            DEFAULT_LOGGER.error("Exception:", e);

Review Comment:
   Please see my answer from the previous comment.



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/command/StartRunner.java:
##########
@@ -0,0 +1,435 @@
+/*
+ * 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.nifi.minifi.bootstrap.command;
+
+import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CMD_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CONF_DIR_KEY;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.DEFAULT_LOGGER;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.MINIFI_CONFIG_FILE_KEY;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.STATUS_FILE_PID_KEY;
+import static org.apache.nifi.minifi.bootstrap.Status.ERROR;
+import static org.apache.nifi.minifi.bootstrap.Status.OK;
+import static org.apache.nifi.minifi.bootstrap.util.ConfigTransformer.generateConfigFiles;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.nifi.bootstrap.util.OSUtils;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.apache.nifi.minifi.bootstrap.RunMiNiFi;
+import org.apache.nifi.minifi.bootstrap.ShutdownHook;
+import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeException;
+import org.apache.nifi.minifi.bootstrap.exception.StartupFailureException;
+import org.apache.nifi.minifi.bootstrap.service.BootstrapFileProvider;
+import org.apache.nifi.minifi.bootstrap.service.CurrentPortProvider;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiListener;
+import org.apache.nifi.minifi.bootstrap.service.MiNiFiStdLogHandler;
+import org.apache.nifi.minifi.bootstrap.service.PeriodicStatusReporterManager;
+import org.apache.nifi.minifi.bootstrap.util.ProcessUtils;
+import org.apache.nifi.util.Tuple;
+
+public class StartRunner implements CommandRunner {
+    private static final String DEFAULT_JAVA_CMD = "java";
+    private static final String DEFAULT_LOG_DIR = "./logs";
+    private static final String DEFAULT_LIB_DIR = "./lib";
+    private static final String DEFAULT_CONF_DIR = "./conf";
+    private static final int STARTUP_WAIT_SECONDS = 60;
+    private static final String DEFAULT_CONFIG_FILE = DEFAULT_CONF_DIR + "/bootstrap.conf";
+
+    private final CurrentPortProvider currentPortProvider;
+    private final BootstrapFileProvider bootstrapFileProvider;
+    private final PeriodicStatusReporterManager periodicStatusReporterManager;
+    private final MiNiFiStdLogHandler miNiFiStdLogHandler;
+    private final MiNiFiParameters miNiFiParameters;
+    private final File bootstrapConfigFile;
+    private final Lock lock = new ReentrantLock();
+    private final Condition startupCondition = lock.newCondition();
+    private final RunMiNiFi runMiNiFi;
+    private volatile ShutdownHook shutdownHook;
+
+    public StartRunner(CurrentPortProvider currentPortProvider, BootstrapFileProvider bootstrapFileProvider,
+        PeriodicStatusReporterManager periodicStatusReporterManager, MiNiFiStdLogHandler miNiFiStdLogHandler, MiNiFiParameters miNiFiParameters, File bootstrapConfigFile,
+        RunMiNiFi runMiNiFi) {
+        this.currentPortProvider = currentPortProvider;
+        this.bootstrapFileProvider = bootstrapFileProvider;
+        this.periodicStatusReporterManager = periodicStatusReporterManager;
+        this.miNiFiStdLogHandler = miNiFiStdLogHandler;
+        this.miNiFiParameters = miNiFiParameters;
+        this.bootstrapConfigFile = bootstrapConfigFile;
+        this.runMiNiFi = runMiNiFi;
+    }
+
+    /**
+     * Starts (and restarts) MiNiFi process during the whole lifecycle of the bootstrap process.
+     * @param args the input arguments
+     * @return status code
+     */
+    @Override
+    public int runCommand(String[] args) {
+        try {
+            start();
+        } catch (Exception e) {
+            CMD_LOGGER.error("Exception happened during MiNiFi startup", e);
+            return ERROR.getStatusCode();
+        }
+        return OK.getStatusCode();
+    }
+
+    private void start() throws IOException, InterruptedException {
+        Integer port = currentPortProvider.getCurrentPort();
+        if (port != null) {
+            String alreadyRunningMessage = "Apache MiNiFi is already running";
+            CMD_LOGGER.info(alreadyRunningMessage + ", listening to Bootstrap on port {}", port);
+            return;
+        }
+
+        File prevLockFile = bootstrapFileProvider.getLockFile();
+        if (prevLockFile.exists() && !prevLockFile.delete()) {
+            CMD_LOGGER.warn("Failed to delete previous lock file {}; this file should be cleaned up manually", prevLockFile);
+        }
+
+        Properties bootstrapProperties = bootstrapFileProvider.getBootstrapProperties();
+        String confDir = bootstrapProperties.getProperty(CONF_DIR_KEY);
+        initConfigFiles(bootstrapProperties, confDir);
+
+        Tuple<ProcessBuilder, Process> tuple = startMiNiFi();
+        ProcessBuilder builder = tuple.getKey();
+        Process process = tuple.getValue();
+
+        try {
+            while (true) {
+                if (ProcessUtils.isAlive(process)) {
+                    handleReload();
+                } else {
+                    Runtime runtime = Runtime.getRuntime();
+                    try {
+                        runtime.removeShutdownHook(shutdownHook);
+                    } catch (IllegalStateException ise) {
+                        // happens when already shutting down

Review Comment:
   done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] bejancsaba commented on a diff in pull request #6075: NIFI-9666 Implement base C2 client functionality

Posted by GitBox <gi...@apache.org>.
bejancsaba commented on code in PR #6075:
URL: https://github.com/apache/nifi/pull/6075#discussion_r891129633


##########
nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowService.java:
##########
@@ -294,6 +299,17 @@ public void start() throws LifeCycleStartException {
 
             if (configuredForClustering) {
                 senderListener.start();
+            } else {
+                // If standalone and C2 is enabled, create a C2 client
+                final boolean c2Enabled = Boolean.parseBoolean(nifiProperties.getProperty(C2NiFiProperties.C2_ENABLE_KEY, "false"));
+                if (c2Enabled) {
+                    logger.info("C2 enabled, creating a C2 client instance");
+                    c2NifiClientService = new C2NifiClientService(nifiProperties, this, this.controller);
+                    c2NifiClientService.start();
+                } else {
+                    logger.info("The '" + C2NiFiProperties.C2_ENABLE_KEY + "' property is missing/false, not creating a C2 client instance");

Review Comment:
   Applied



##########
nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/c2/C2NifiClientService.java:
##########
@@ -0,0 +1,208 @@
+/*
+ * 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.nifi.c2;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Optional;
+import org.apache.nifi.c2.client.C2ClientConfig;
+import org.apache.nifi.c2.client.http.C2HttpClient;
+import org.apache.nifi.c2.client.service.C2ClientService;
+import org.apache.nifi.c2.client.service.C2HeartbeatFactory;
+import org.apache.nifi.c2.client.service.FlowIdHolder;
+import org.apache.nifi.c2.client.service.model.RuntimeInfoWrapper;
+import org.apache.nifi.c2.protocol.api.AgentRepositories;
+import org.apache.nifi.c2.protocol.api.AgentRepositoryStatus;
+import org.apache.nifi.c2.protocol.api.FlowQueueStatus;
+import org.apache.nifi.c2.serializer.C2JacksonSerializer;
+import org.apache.nifi.controller.FlowController;;
+import org.apache.nifi.controller.status.ConnectionStatus;
+import org.apache.nifi.controller.status.ProcessGroupStatus;
+import org.apache.nifi.diagnostics.StorageUsage;
+import org.apache.nifi.diagnostics.SystemDiagnostics;
+import org.apache.nifi.extension.manifest.parser.ExtensionManifestParser;
+import org.apache.nifi.extension.manifest.parser.jaxb.JAXBExtensionManifestParser;
+import org.apache.nifi.manifest.RuntimeManifestService;
+import org.apache.nifi.manifest.StandardRuntimeManifestService;
+import org.apache.nifi.nar.ExtensionManagerHolder;
+import org.apache.nifi.services.FlowService;
+import org.apache.nifi.util.FormatUtils;
+import org.apache.nifi.util.NiFiProperties;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+public class C2NifiClientService {
+
+    private static final Logger logger = LoggerFactory.getLogger(C2NifiClientService.class);
+    private static final String DEFAULT_CONF_DIR = "./conf";
+    private static final String TARGET_CONFIG_FILE = "/config-new.yml";
+    private static final String ROOT_GROUP_ID = "root";
+    private static final Long INITIAL_DELAY = 0L;
+    private static final Integer TERMINATION_WAIT = 5000;
+
+    private final C2ClientService c2ClientService;
+
+    private final FlowService flowService;
+    private final FlowController flowController;
+    private final String propertiesDir;
+    private final ScheduledThreadPoolExecutor scheduledExecutorService = new ScheduledThreadPoolExecutor(1);
+    private final ExtensionManifestParser extensionManifestParser = new JAXBExtensionManifestParser();
+
+    private final RuntimeManifestService runtimeManifestService;
+    private final long heartbeatPeriod;
+
+    public C2NifiClientService(final NiFiProperties niFiProperties, final FlowService flowService, final FlowController flowController) {
+        C2ClientConfig clientConfig = generateClientConfig(niFiProperties);
+        FlowIdHolder flowIdHolder = new FlowIdHolder(clientConfig.getConfDirectory());
+        this.propertiesDir = niFiProperties.getProperty(NiFiProperties.PROPERTIES_FILE_PATH, null);
+        this.runtimeManifestService = new StandardRuntimeManifestService(
+            ExtensionManagerHolder.getExtensionManager(),
+            extensionManifestParser,
+            clientConfig.getRuntimeManifestIdentifier(),
+            clientConfig.getRuntimeType()
+        );
+        this.heartbeatPeriod = clientConfig.getHeartbeatPeriod();
+        this.flowService = flowService;
+        this.flowController = flowController;
+        this.c2ClientService = new C2ClientService(
+            new C2HttpClient(clientConfig, new C2JacksonSerializer()),
+            new C2HeartbeatFactory(clientConfig, flowIdHolder),
+            flowIdHolder,
+            this::updateFlowContent
+        );
+    }
+
+    private C2ClientConfig generateClientConfig(NiFiProperties properties) {
+        return new C2ClientConfig.Builder()
+                .agentClass(properties.getProperty(C2NiFiProperties.C2_AGENT_CLASS_KEY, ""))
+                .agentIdentifier(properties.getProperty(C2NiFiProperties.C2_AGENT_IDENTIFIER_KEY))
+                .heartbeatPeriod(Long.valueOf(properties.getProperty(C2NiFiProperties.C2_AGENT_HEARTBEAT_PERIOD_KEY,
+                    String.valueOf(C2NiFiProperties.C2_AGENT_DEFAULT_HEARTBEAT_PERIOD))))
+                .callTimeout((long) FormatUtils.getPreciseTimeDuration(properties.getProperty(C2NiFiProperties.C2_CALL_TIMEOUT,
+                    C2NiFiProperties.C2_DEFAULT_CALL_TIMEOUT), TimeUnit.MILLISECONDS))
+                .c2Url(properties.getProperty(C2NiFiProperties.C2_REST_URL_KEY, ""))
+                .confDirectory(properties.getProperty(C2NiFiProperties.C2_CONFIG_DIRECTORY_KEY, DEFAULT_CONF_DIR))
+                .runtimeManifestIdentifier(properties.getProperty(C2NiFiProperties.C2_RUNTIME_MANIFEST_IDENTIFIER_KEY, ""))
+                .runtimeType(properties.getProperty(C2NiFiProperties.C2_RUNTIME_TYPE_KEY, ""))
+                .c2AckUrl(properties.getProperty(C2NiFiProperties.C2_REST_URL_ACK_KEY, ""))
+                .truststoreFilename(properties.getProperty(C2NiFiProperties.TRUSTSTORE_LOCATION_KEY, ""))
+                .truststorePassword(properties.getProperty(C2NiFiProperties.TRUSTSTORE_PASSWORD_KEY, ""))
+                .truststoreType(properties.getProperty(C2NiFiProperties.TRUSTSTORE_TYPE_KEY, "JKS"))
+                .keystoreFilename(properties.getProperty(C2NiFiProperties.KEYSTORE_LOCATION_KEY, ""))
+                .keystorePassword(properties.getProperty(C2NiFiProperties.KEYSTORE_PASSWORD_KEY, ""))
+                .keystoreType(properties.getProperty(C2NiFiProperties.KEYSTORE_TYPE_KEY, "JKS"))
+                .build();
+    }
+
+    public void start() {
+        try {
+            scheduledExecutorService.scheduleAtFixedRate(() -> c2ClientService.sendHeartbeat(generateRuntimeInfo()), INITIAL_DELAY, heartbeatPeriod, TimeUnit.MILLISECONDS);
+        } catch (Exception e) {
+            logger.error("Could not start C2 Client Heartbeat Reporting", e);
+            throw new RuntimeException(e);
+        }
+    }
+
+    public void stop() {
+        try {
+            scheduledExecutorService.shutdown();
+            scheduledExecutorService.awaitTermination(TERMINATION_WAIT, TimeUnit.MILLISECONDS);
+        } catch (InterruptedException ignore) {
+            logger.info("Stopping C2 Client's thread was interrupted but shutting down anyway the C2NifiClientService");
+        }
+    }
+
+    private RuntimeInfoWrapper generateRuntimeInfo() {
+        return new RuntimeInfoWrapper(getAgentRepositories(), runtimeManifestService.getManifest(), getQueueStatus());
+    }
+
+    private AgentRepositories getAgentRepositories() {
+        final SystemDiagnostics systemDiagnostics = flowController.getSystemDiagnostics();
+
+        final AgentRepositories repos = new AgentRepositories();
+        final AgentRepositoryStatus flowFileRepoStatus = new AgentRepositoryStatus();
+        final StorageUsage ffRepoStorageUsage = systemDiagnostics.getFlowFileRepositoryStorageUsage();
+        flowFileRepoStatus.setDataSize(ffRepoStorageUsage.getUsedSpace());
+        flowFileRepoStatus.setDataSizeMax(ffRepoStorageUsage.getTotalSpace());
+        repos.setFlowFile(flowFileRepoStatus);
+
+        final AgentRepositoryStatus provRepoStatus = new AgentRepositoryStatus();
+        final Iterator<Map.Entry<String, StorageUsage>> provRepoStorageUsages = systemDiagnostics.getProvenanceRepositoryStorageUsage().entrySet().iterator();
+        if (provRepoStorageUsages.hasNext()) {
+            final StorageUsage provRepoStorageUsage = provRepoStorageUsages.next().getValue();
+            provRepoStatus.setDataSize(provRepoStorageUsage.getUsedSpace());
+            provRepoStatus.setDataSizeMax(provRepoStorageUsage.getTotalSpace());
+        }
+
+        repos.setProvenance(provRepoStatus);
+
+        return repos;
+    }
+
+    private Map<String, FlowQueueStatus> getQueueStatus() {
+        ProcessGroupStatus rootProcessGroupStatus = flowController.getEventAccess().getGroupStatus(ROOT_GROUP_ID);
+
+        final Collection<ConnectionStatus> connectionStatuses = rootProcessGroupStatus.getConnectionStatus();
+
+        final Map<String, FlowQueueStatus> processGroupStatus = new HashMap<>();
+        for (ConnectionStatus connectionStatus : connectionStatuses) {
+            final FlowQueueStatus flowQueueStatus = new FlowQueueStatus();
+
+            flowQueueStatus.setSize((long) connectionStatus.getQueuedCount());
+            flowQueueStatus.setSizeMax(connectionStatus.getBackPressureObjectThreshold());
+
+            flowQueueStatus.setDataSize(connectionStatus.getQueuedBytes());
+            flowQueueStatus.setDataSizeMax(connectionStatus.getBackPressureBytesThreshold());
+
+            processGroupStatus.put(connectionStatus.getId(), flowQueueStatus);
+        }
+
+        return processGroupStatus;
+    }
+
+    private boolean updateFlowContent(byte[] updateContent) {
+        logger.debug("Update content: \n{}", new String(updateContent, StandardCharsets.UTF_8));
+        Path path = getTargetConfigFile().toPath();
+        try {
+            Files.write(getTargetConfigFile().toPath(), updateContent);
+            logger.info("Updated configuration was written to: {}", path);
+            return true;
+        } catch (IOException e) {
+            logger.error("Configuration update failed. File creation was not successful targeting: {}", path);
+            logger.error("Exception: ", e);

Review Comment:
   Applied



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] adamdebreceni commented on a diff in pull request #6075: NIFI-9666 Implement base C2 client functionality

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #6075:
URL: https://github.com/apache/nifi/pull/6075#discussion_r891024533


##########
c2/c2-client-bundle/c2-client-http/src/main/java/org/apache/nifi/c2/client/http/C2HttpClient.java:
##########
@@ -0,0 +1,250 @@
+/*
+ * 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.nifi.c2.client.http;
+
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.security.KeyStore;
+import java.security.NoSuchAlgorithmException;
+import java.util.Optional;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLSocketFactory;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+import okhttp3.MediaType;
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.RequestBody;
+import okhttp3.Response;
+import okhttp3.ResponseBody;
+import okhttp3.logging.HttpLoggingInterceptor;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.c2.client.C2ClientConfig;
+import org.apache.nifi.c2.client.api.C2Client;
+import org.apache.nifi.c2.client.api.C2Serializer;
+import org.apache.nifi.c2.protocol.api.C2Heartbeat;
+import org.apache.nifi.c2.protocol.api.C2HeartbeatResponse;
+import org.apache.nifi.c2.protocol.api.C2OperationAck;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class C2HttpClient implements C2Client {
+
+    private static final Logger logger = LoggerFactory.getLogger(C2HttpClient.class);
+    private static final MediaType MEDIA_TYPE_APPLICATION_JSON = MediaType.parse("application/json");
+
+    private final AtomicReference<OkHttpClient> httpClientReference = new AtomicReference<>();
+    private final C2ClientConfig clientConfig;
+    private final C2Serializer serializer;
+
+    public C2HttpClient(C2ClientConfig clientConfig, C2Serializer serializer) {
+        super();
+        this.clientConfig = clientConfig;
+        this.serializer = serializer;
+        final OkHttpClient.Builder okHttpClientBuilder = new OkHttpClient.Builder();
+
+        // Configure request and response logging
+        HttpLoggingInterceptor logging = new HttpLoggingInterceptor();
+        logging.setLevel(HttpLoggingInterceptor.Level.BASIC);
+        okHttpClientBuilder.addInterceptor(logging);
+
+        // Set whether to follow redirects
+        okHttpClientBuilder.followRedirects(true);
+
+        // Timeout for calls made to the server
+        okHttpClientBuilder.callTimeout(clientConfig.getCallTimeout(), TimeUnit.MILLISECONDS);
+
+        // check if the ssl path is set and add the factory if so
+        if (StringUtils.isNotBlank(clientConfig.getKeystoreFilename())) {
+            try {
+                setSslSocketFactory(okHttpClientBuilder);
+            } catch (Exception e) {
+                throw new IllegalStateException("OkHttp TLS configuration failed", e);
+            }
+        }
+
+        httpClientReference.set(okHttpClientBuilder.build());
+    }
+
+    @Override
+    public Optional<C2HeartbeatResponse> publishHeartbeat(C2Heartbeat heartbeat) {
+        return serializer.serialize(heartbeat).flatMap(this::sendHeartbeat);
+    }
+
+    private Optional<C2HeartbeatResponse> sendHeartbeat(String heartbeat) {
+        logger.debug("Sending heartbeat to {}", clientConfig.getC2Url());
+
+        Optional<C2HeartbeatResponse> c2HeartbeatResponse = Optional.empty();
+        Request request = new Request.Builder()
+            .post(RequestBody.create(heartbeat, MEDIA_TYPE_APPLICATION_JSON))
+            .url(clientConfig.getC2Url())
+            .build();
+
+        try (Response heartbeatResponse = httpClientReference.get().newCall(request).execute()) {
+            c2HeartbeatResponse = getResponseBody(heartbeatResponse).flatMap(response -> serializer.deserialize(response, C2HeartbeatResponse.class));

Review Comment:
   added a comment in the ticket [NIFI-9667](https://issues.apache.org/jira/browse/NIFI-9667) for this to be possibly addressed later



##########
c2/c2-client-bundle/c2-client-service/src/main/java/org/apache/nifi/c2/client/service/operation/UpdateConfigurationOperationHandler.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.nifi.c2.client.service.operation;
+
+import static org.apache.commons.lang3.StringUtils.EMPTY;
+import static org.apache.nifi.c2.protocol.api.OperandType.CONFIGURATION;
+import static org.apache.nifi.c2.protocol.api.OperationType.UPDATE;
+
+import java.net.URI;
+import java.util.Optional;
+import java.util.function.Function;
+import org.apache.nifi.c2.client.api.C2Client;
+import org.apache.nifi.c2.client.service.FlowIdHolder;
+import org.apache.nifi.c2.protocol.api.C2Operation;
+import org.apache.nifi.c2.protocol.api.C2OperationAck;
+import org.apache.nifi.c2.protocol.api.C2OperationState;
+import org.apache.nifi.c2.protocol.api.OperandType;
+import org.apache.nifi.c2.protocol.api.OperationType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class UpdateConfigurationOperationHandler implements C2OperationHandler {
+
+    private static final Logger logger = LoggerFactory.getLogger(UpdateConfigurationOperationHandler.class);
+
+    private static final String LOCATION = "location";
+
+    private final C2Client client;
+    private final Function<byte[], Boolean> updateFlow;
+    private final FlowIdHolder flowIdHolder;
+
+    public UpdateConfigurationOperationHandler(C2Client client, FlowIdHolder flowIdHolder, Function<byte[], Boolean> updateFlow) {
+        this.client = client;
+        this.updateFlow = updateFlow;
+        this.flowIdHolder = flowIdHolder;
+    }
+
+    @Override
+    public OperationType getOperationType() {
+        return UPDATE;
+    }
+
+    @Override
+    public OperandType getOperandType() {
+        return CONFIGURATION;
+    }
+
+    @Override
+    public C2OperationAck handle(C2Operation operation) {
+        String opIdentifier = Optional.ofNullable(operation.getIdentifier())
+            .orElse(EMPTY);
+        C2OperationAck operationAck = new C2OperationAck();
+        C2OperationState state = new C2OperationState();
+        operationAck.setOperationState(state);
+        operationAck.setOperationId(opIdentifier);
+
+        String updateLocation = Optional.ofNullable(operation.getArgs())
+            .map(map -> map.get(LOCATION))
+            .orElse(EMPTY);
+
+        String newFlowId = parseFlowId(updateLocation);
+        if (flowIdHolder.getFlowId() == null || !flowIdHolder.getFlowId().equals(newFlowId)) {
+            logger.info("Will perform flow update from {} for operation #{}. Previous flow id was {}, replacing with new id {}", updateLocation, opIdentifier,
+                flowIdHolder.getFlowId() == null ? "not set" : flowIdHolder.getFlowId(), newFlowId);
+        } else {
+            logger.info("Flow is current, no update is necessary...");
+        }
+
+        flowIdHolder.setFlowId(newFlowId);
+        byte[] updateContent = client.retrieveUpdateContent(updateLocation);
+        if (updateContent != null) {
+            if (updateFlow.apply(updateContent)) {
+                state.setState(C2OperationState.OperationState.FULLY_APPLIED);
+                logger.debug("Update configuration applied for operation #{}.", opIdentifier);
+            } else {
+                state.setState(C2OperationState.OperationState.NOT_APPLIED);
+                logger.error("Update resulted in error for operation #{}.", opIdentifier);
+            }
+        } else {
+            state.setState(C2OperationState.OperationState.NOT_APPLIED);
+            logger.error("Update content retrieval resulted in empty content so flow update was omitted for operation #{}.", opIdentifier);
+        }
+
+        return operationAck;
+    }
+
+    private String parseFlowId(String flowUpdateUrl) {
+        try {
+            URI flowUri = new URI(flowUpdateUrl);
+            String flowUriPath = flowUri.getPath();
+            String[] split = flowUriPath.split("/");
+            if (split.length > 4) {
+                return split[4];

Review Comment:
   added a comment in the ticket [NIFI-9667](https://issues.apache.org/jira/browse/NIFI-9667) for this to be possibly addressed later



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] bejancsaba commented on a diff in pull request #6075: NIFI-9666 Implement base C2 client functionality

Posted by GitBox <gi...@apache.org>.
bejancsaba commented on code in PR #6075:
URL: https://github.com/apache/nifi/pull/6075#discussion_r899966740


##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/MiNiFiCommandSender.java:
##########
@@ -0,0 +1,131 @@
+/*
+ * 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.nifi.minifi.bootstrap.service;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.BufferedReader;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.net.SocketTimeoutException;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MiNiFiCommandSender {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(MiNiFiCommandSender.class);
+    private static final String PING_CMD = "PING";
+    private static final int SOCKET_TIMEOUT = 10000;
+    private static final int CONNECTION_TIMEOUT = 10000;
+
+    private final MiNiFiParameters miNiFiParameters;
+    private final ObjectMapper objectMapper;
+
+    public MiNiFiCommandSender(MiNiFiParameters miNiFiParameters, ObjectMapper objectMapper) {
+        this.miNiFiParameters = miNiFiParameters;
+        this.objectMapper = objectMapper;
+    }
+
+    public Optional<String> sendCommand(String cmd, Integer port, String... extraParams) throws IOException {
+        Optional<String> response = Optional.empty();
+
+        if (port == null) {
+            LOGGER.info("Apache MiNiFi is not currently running");
+            return response;
+        }
+
+        try (Socket socket = new Socket()) {
+            LOGGER.debug("Connecting to MiNiFi instance");
+            socket.setSoTimeout(SOCKET_TIMEOUT);
+            socket.connect(new InetSocketAddress("localhost", port), CONNECTION_TIMEOUT);
+            LOGGER.debug("Established connection to MiNiFi instance.");
+
+            LOGGER.debug("Sending {} Command to port {}", cmd, port);
+
+            String responseString;
+            try (OutputStream out = socket.getOutputStream()) {
+                out.write(getCommand(cmd, extraParams));
+                out.flush();
+                responseString = readResponse(socket);
+            }
+
+            LOGGER.debug("Received response to {} command: {}", cmd, responseString);
+            response = Optional.of(responseString);
+        } catch (EOFException | SocketTimeoutException e) {
+            String message = "Failed to get response for " + cmd + " Potentially due to the process currently being down (restarting or otherwise)";
+            throw new RuntimeException(message);
+        }
+        return response;
+    }
+
+    <T> T sendCommandForObject(String cmd, Integer port, Class<T> clazz, String... extraParams) throws IOException {
+        return sendCommand(cmd, port, extraParams)
+            .map(response -> deserialize(cmd, response, clazz))
+            .orElse(null);
+    }
+
+    private String readResponse(Socket socket) throws IOException {
+        StringBuilder sb = new StringBuilder();
+        int numLines = 0;
+        try (BufferedReader reader = new BufferedReader(new InputStreamReader(socket.getInputStream()))) {
+            String line;
+            while ((line = reader.readLine()) != null) {
+                if (numLines++ > 0) {
+                    sb.append("\n");
+                }
+                sb.append(line);
+            }
+        }
+
+        return sb.toString().trim();
+    }
+
+    private byte[] getCommand(String cmd, String... args) {
+        String argsString = Arrays.stream(args).collect(Collectors.joining(" "));
+        String commandWithArgs = cmd + " " + miNiFiParameters.getSecretKey() + (args.length > 0 ? " " : "") + argsString + "\n";
+        return commandWithArgs.getBytes(StandardCharsets.UTF_8);
+    }
+
+    private <T> T deserialize(String cmd, String obj, Class<T> clazz) {
+        T response;
+        try {
+            response = objectMapper.readValue(obj, clazz);
+        } catch (JsonProcessingException e) {
+            String message = "Failed to deserialize " + cmd + " response";
+            LOGGER.error(message);
+            throw new RuntimeException(message);

Review Comment:
   Added



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] bejancsaba commented on a diff in pull request #6075: NIFI-9666 Implement base C2 client functionality

Posted by GitBox <gi...@apache.org>.
bejancsaba commented on code in PR #6075:
URL: https://github.com/apache/nifi/pull/6075#discussion_r899970143


##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/MiNiFiConfigurationChangeListener.java:
##########
@@ -0,0 +1,208 @@
+/*
+ * 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.nifi.minifi.bootstrap.service;
+
+import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CONF_DIR_KEY;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.MINIFI_CONFIG_FILE_KEY;
+import static org.apache.nifi.minifi.bootstrap.configuration.ingestors.PullHttpChangeIngestor.OVERRIDE_SECURITY;
+import static org.apache.nifi.minifi.bootstrap.configuration.ingestors.PullHttpChangeIngestor.PULL_HTTP_BASE_KEY;
+import static org.apache.nifi.minifi.bootstrap.util.ConfigTransformer.generateConfigFiles;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.util.Properties;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.commons.io.IOUtils;
+import org.apache.nifi.minifi.bootstrap.RunMiNiFi;
+import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeException;
+import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeListener;
+import org.apache.nifi.minifi.bootstrap.util.ByteBufferInputStream;
+import org.apache.nifi.minifi.bootstrap.util.ConfigTransformer;
+import org.apache.nifi.minifi.commons.schema.ConfigSchema;
+import org.apache.nifi.minifi.commons.schema.common.ConvertableSchema;
+import org.apache.nifi.minifi.commons.schema.serialization.SchemaLoader;
+import org.slf4j.Logger;
+
+public class MiNiFiConfigurationChangeListener implements ConfigurationChangeListener {
+
+    private final RunMiNiFi runner;
+    private final Logger logger;
+    private final BootstrapFileProvider bootstrapFileProvider;
+
+    private static final ReentrantLock handlingLock = new ReentrantLock();
+
+    public MiNiFiConfigurationChangeListener(RunMiNiFi runner, Logger logger, BootstrapFileProvider bootstrapFileProvider) {
+        this.runner = runner;
+        this.logger = logger;
+        this.bootstrapFileProvider = bootstrapFileProvider;
+    }
+
+    @Override
+    public void handleChange(InputStream configInputStream) throws ConfigurationChangeException {
+        logger.info("Received notification of a change");
+
+        if (!handlingLock.tryLock()) {
+            throw new ConfigurationChangeException("Instance is already handling another change");
+        }
+        // Store the incoming stream as a byte array to be shared among components that need it
+        try(ByteArrayOutputStream bufferedConfigOs = new ByteArrayOutputStream()) {
+
+            Properties bootstrapProperties = bootstrapFileProvider.getBootstrapProperties();
+            File configFile = new File(bootstrapProperties.getProperty(MINIFI_CONFIG_FILE_KEY));
+
+            byte[] copyArray = new byte[1024];
+            int available;
+            while ((available = configInputStream.read(copyArray)) > 0) {
+                bufferedConfigOs.write(copyArray, 0, available);
+            }
+
+            File swapConfigFile = bootstrapFileProvider.getSwapFile();
+            logger.info("Persisting old configuration to {}", swapConfigFile.getAbsolutePath());
+
+            try (FileInputStream configFileInputStream = new FileInputStream(configFile)) {
+                Files.copy(configFileInputStream, swapConfigFile.toPath(), REPLACE_EXISTING);
+            }
+
+            persistBackNonFlowSectionsFromOriginalSchema(bufferedConfigOs.toByteArray(), bootstrapProperties, configFile);
+
+            // Create an input stream to feed to the config transformer
+            try (FileInputStream newConfigIs = new FileInputStream(configFile)) {
+
+                try {
+                    String confDir = bootstrapProperties.getProperty(CONF_DIR_KEY);
+
+                    try {
+                        logger.info("Performing transformation for input and saving outputs to {}", confDir);
+                        ByteBuffer tempConfigFile = generateConfigFiles(newConfigIs, confDir, bootstrapFileProvider.getBootstrapProperties());
+                        runner.getConfigFileReference().set(tempConfigFile.asReadOnlyBuffer());
+
+                        try {
+                            logger.info("Reloading instance with new configuration");
+                            restartInstance();
+                        } catch (Exception e) {
+                            logger.debug("Transformation of new config file failed after transformation into Flow.xml and nifi.properties, reverting.");
+                            try (FileInputStream swapConfigFileStream = new FileInputStream(swapConfigFile)) {
+                                ByteBuffer resetConfigFile = generateConfigFiles(swapConfigFileStream, confDir, bootstrapFileProvider.getBootstrapProperties());
+                                runner.getConfigFileReference().set(resetConfigFile.asReadOnlyBuffer());
+                            }
+                            throw e;
+                        }
+                    } catch (Exception e) {
+                        logger.debug("Transformation of new config file failed after replacing original with the swap file, reverting.");
+                        try (FileInputStream swapConfigFileStream = new FileInputStream(swapConfigFile)) {
+                            Files.copy(swapConfigFileStream, configFile.toPath(), REPLACE_EXISTING);
+                        }
+                        throw e;
+                    }
+                } catch (Exception e) {
+                    logger.debug("Transformation of new config file failed after swap file was created, deleting it.");
+                    if (!swapConfigFile.delete()) {
+                        logger.warn("The swap file failed to delete after a failed handling of a change. It should be cleaned up manually.");
+                    }
+                    throw e;
+                }
+            }
+        } catch (ConfigurationChangeException e){
+            logger.error("Unable to carry out reloading of configuration on receipt of notification event", e);
+            throw e;
+        } catch (IOException ioe) {
+            logger.error("Unable to carry out reloading of configuration on receipt of notification event", ioe);
+            throw new ConfigurationChangeException("Unable to perform reload of received configuration change", ioe);

Review Comment:
   The ConfigurationChangeCoordinator logs the ListenerHandlerResult so the result here will be logged. I removed the error logging to avoid duplication.



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/MiNiFiConfigurationChangeListener.java:
##########
@@ -0,0 +1,208 @@
+/*
+ * 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.nifi.minifi.bootstrap.service;
+
+import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CONF_DIR_KEY;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.MINIFI_CONFIG_FILE_KEY;
+import static org.apache.nifi.minifi.bootstrap.configuration.ingestors.PullHttpChangeIngestor.OVERRIDE_SECURITY;
+import static org.apache.nifi.minifi.bootstrap.configuration.ingestors.PullHttpChangeIngestor.PULL_HTTP_BASE_KEY;
+import static org.apache.nifi.minifi.bootstrap.util.ConfigTransformer.generateConfigFiles;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.util.Properties;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.commons.io.IOUtils;
+import org.apache.nifi.minifi.bootstrap.RunMiNiFi;
+import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeException;
+import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeListener;
+import org.apache.nifi.minifi.bootstrap.util.ByteBufferInputStream;
+import org.apache.nifi.minifi.bootstrap.util.ConfigTransformer;
+import org.apache.nifi.minifi.commons.schema.ConfigSchema;
+import org.apache.nifi.minifi.commons.schema.common.ConvertableSchema;
+import org.apache.nifi.minifi.commons.schema.serialization.SchemaLoader;
+import org.slf4j.Logger;
+
+public class MiNiFiConfigurationChangeListener implements ConfigurationChangeListener {
+
+    private final RunMiNiFi runner;
+    private final Logger logger;
+    private final BootstrapFileProvider bootstrapFileProvider;
+
+    private static final ReentrantLock handlingLock = new ReentrantLock();
+
+    public MiNiFiConfigurationChangeListener(RunMiNiFi runner, Logger logger, BootstrapFileProvider bootstrapFileProvider) {
+        this.runner = runner;
+        this.logger = logger;
+        this.bootstrapFileProvider = bootstrapFileProvider;
+    }
+
+    @Override
+    public void handleChange(InputStream configInputStream) throws ConfigurationChangeException {
+        logger.info("Received notification of a change");
+
+        if (!handlingLock.tryLock()) {
+            throw new ConfigurationChangeException("Instance is already handling another change");
+        }
+        // Store the incoming stream as a byte array to be shared among components that need it
+        try(ByteArrayOutputStream bufferedConfigOs = new ByteArrayOutputStream()) {
+
+            Properties bootstrapProperties = bootstrapFileProvider.getBootstrapProperties();
+            File configFile = new File(bootstrapProperties.getProperty(MINIFI_CONFIG_FILE_KEY));
+
+            byte[] copyArray = new byte[1024];
+            int available;
+            while ((available = configInputStream.read(copyArray)) > 0) {
+                bufferedConfigOs.write(copyArray, 0, available);
+            }
+
+            File swapConfigFile = bootstrapFileProvider.getSwapFile();
+            logger.info("Persisting old configuration to {}", swapConfigFile.getAbsolutePath());
+
+            try (FileInputStream configFileInputStream = new FileInputStream(configFile)) {
+                Files.copy(configFileInputStream, swapConfigFile.toPath(), REPLACE_EXISTING);
+            }
+
+            persistBackNonFlowSectionsFromOriginalSchema(bufferedConfigOs.toByteArray(), bootstrapProperties, configFile);
+
+            // Create an input stream to feed to the config transformer
+            try (FileInputStream newConfigIs = new FileInputStream(configFile)) {
+
+                try {
+                    String confDir = bootstrapProperties.getProperty(CONF_DIR_KEY);
+
+                    try {
+                        logger.info("Performing transformation for input and saving outputs to {}", confDir);
+                        ByteBuffer tempConfigFile = generateConfigFiles(newConfigIs, confDir, bootstrapFileProvider.getBootstrapProperties());
+                        runner.getConfigFileReference().set(tempConfigFile.asReadOnlyBuffer());
+
+                        try {
+                            logger.info("Reloading instance with new configuration");
+                            restartInstance();
+                        } catch (Exception e) {
+                            logger.debug("Transformation of new config file failed after transformation into Flow.xml and nifi.properties, reverting.");
+                            try (FileInputStream swapConfigFileStream = new FileInputStream(swapConfigFile)) {
+                                ByteBuffer resetConfigFile = generateConfigFiles(swapConfigFileStream, confDir, bootstrapFileProvider.getBootstrapProperties());
+                                runner.getConfigFileReference().set(resetConfigFile.asReadOnlyBuffer());
+                            }
+                            throw e;
+                        }
+                    } catch (Exception e) {
+                        logger.debug("Transformation of new config file failed after replacing original with the swap file, reverting.");
+                        try (FileInputStream swapConfigFileStream = new FileInputStream(swapConfigFile)) {
+                            Files.copy(swapConfigFileStream, configFile.toPath(), REPLACE_EXISTING);
+                        }
+                        throw e;
+                    }
+                } catch (Exception e) {
+                    logger.debug("Transformation of new config file failed after swap file was created, deleting it.");
+                    if (!swapConfigFile.delete()) {
+                        logger.warn("The swap file failed to delete after a failed handling of a change. It should be cleaned up manually.");
+                    }
+                    throw e;
+                }
+            }
+        } catch (ConfigurationChangeException e){
+            logger.error("Unable to carry out reloading of configuration on receipt of notification event", e);
+            throw e;
+        } catch (IOException ioe) {
+            logger.error("Unable to carry out reloading of configuration on receipt of notification event", ioe);
+            throw new ConfigurationChangeException("Unable to perform reload of received configuration change", ioe);
+        } finally {
+            IOUtils.closeQuietly(configInputStream);
+            handlingLock.unlock();
+        }
+    }
+
+    @Override
+    public String getDescriptor() {
+        return "MiNiFiConfigurationChangeListener";
+    }
+
+    private void restartInstance() throws IOException {
+        try {
+            runner.reload();
+        } catch (IOException e) {
+            throw new IOException("Unable to successfully restart MiNiFi instance after configuration change.", e);
+        }
+    }
+
+    private void persistBackNonFlowSectionsFromOriginalSchema(byte[] newSchema, Properties bootstrapProperties, File configFile) {
+        try {
+            ConvertableSchema<ConfigSchema> schemaNew = ConfigTransformer
+                .throwIfInvalid(SchemaLoader.loadConvertableSchemaFromYaml(new ByteArrayInputStream(newSchema)));
+            ConfigSchema configSchemaNew = ConfigTransformer.throwIfInvalid(schemaNew.convert());
+            ConvertableSchema<ConfigSchema> schemaOld = ConfigTransformer
+                .throwIfInvalid(SchemaLoader.loadConvertableSchemaFromYaml(new ByteBufferInputStream(runner.getConfigFileReference().get().duplicate())));
+            ConfigSchema configSchemaOld = ConfigTransformer.throwIfInvalid(schemaOld.convert());
+
+            configSchemaNew.setNifiPropertiesOverrides(configSchemaOld.getNifiPropertiesOverrides());
+
+            if (!overrideCoreProperties(bootstrapProperties)) {
+                logger.debug("Preserving previous core properties...");
+                configSchemaNew.setCoreProperties(configSchemaOld.getCoreProperties());
+            }
+
+            if (!overrideSecurityProperties(bootstrapProperties)) {
+                logger.debug("Preserving previous security properties...");
+                configSchemaNew.setSecurityProperties(configSchemaOld.getSecurityProperties());
+            }
+
+            logger.debug("Persisting changes to {}", configFile.getAbsolutePath());
+            SchemaLoader.toYaml(configSchemaNew, new FileWriter(configFile));
+        } catch (Exception e) {
+            logger.error("Loading the old and the new schema for merging was not successful");

Review Comment:
   Added



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] bejancsaba commented on a diff in pull request #6075: NIFI-9666 Implement base C2 client functionality

Posted by GitBox <gi...@apache.org>.
bejancsaba commented on code in PR #6075:
URL: https://github.com/apache/nifi/pull/6075#discussion_r899968537


##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/MiNiFiConfigurationChangeListener.java:
##########
@@ -0,0 +1,208 @@
+/*
+ * 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.nifi.minifi.bootstrap.service;
+
+import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CONF_DIR_KEY;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.MINIFI_CONFIG_FILE_KEY;
+import static org.apache.nifi.minifi.bootstrap.configuration.ingestors.PullHttpChangeIngestor.OVERRIDE_SECURITY;
+import static org.apache.nifi.minifi.bootstrap.configuration.ingestors.PullHttpChangeIngestor.PULL_HTTP_BASE_KEY;
+import static org.apache.nifi.minifi.bootstrap.util.ConfigTransformer.generateConfigFiles;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.util.Properties;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.commons.io.IOUtils;
+import org.apache.nifi.minifi.bootstrap.RunMiNiFi;
+import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeException;
+import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeListener;
+import org.apache.nifi.minifi.bootstrap.util.ByteBufferInputStream;
+import org.apache.nifi.minifi.bootstrap.util.ConfigTransformer;
+import org.apache.nifi.minifi.commons.schema.ConfigSchema;
+import org.apache.nifi.minifi.commons.schema.common.ConvertableSchema;
+import org.apache.nifi.minifi.commons.schema.serialization.SchemaLoader;
+import org.slf4j.Logger;
+
+public class MiNiFiConfigurationChangeListener implements ConfigurationChangeListener {
+
+    private final RunMiNiFi runner;
+    private final Logger logger;
+    private final BootstrapFileProvider bootstrapFileProvider;
+
+    private static final ReentrantLock handlingLock = new ReentrantLock();
+
+    public MiNiFiConfigurationChangeListener(RunMiNiFi runner, Logger logger, BootstrapFileProvider bootstrapFileProvider) {
+        this.runner = runner;
+        this.logger = logger;
+        this.bootstrapFileProvider = bootstrapFileProvider;
+    }
+
+    @Override
+    public void handleChange(InputStream configInputStream) throws ConfigurationChangeException {
+        logger.info("Received notification of a change");
+
+        if (!handlingLock.tryLock()) {
+            throw new ConfigurationChangeException("Instance is already handling another change");
+        }
+        // Store the incoming stream as a byte array to be shared among components that need it
+        try(ByteArrayOutputStream bufferedConfigOs = new ByteArrayOutputStream()) {
+
+            Properties bootstrapProperties = bootstrapFileProvider.getBootstrapProperties();
+            File configFile = new File(bootstrapProperties.getProperty(MINIFI_CONFIG_FILE_KEY));
+
+            byte[] copyArray = new byte[1024];
+            int available;
+            while ((available = configInputStream.read(copyArray)) > 0) {
+                bufferedConfigOs.write(copyArray, 0, available);
+            }
+
+            File swapConfigFile = bootstrapFileProvider.getSwapFile();
+            logger.info("Persisting old configuration to {}", swapConfigFile.getAbsolutePath());
+
+            try (FileInputStream configFileInputStream = new FileInputStream(configFile)) {
+                Files.copy(configFileInputStream, swapConfigFile.toPath(), REPLACE_EXISTING);
+            }
+
+            persistBackNonFlowSectionsFromOriginalSchema(bufferedConfigOs.toByteArray(), bootstrapProperties, configFile);
+
+            // Create an input stream to feed to the config transformer
+            try (FileInputStream newConfigIs = new FileInputStream(configFile)) {
+
+                try {
+                    String confDir = bootstrapProperties.getProperty(CONF_DIR_KEY);
+
+                    try {
+                        logger.info("Performing transformation for input and saving outputs to {}", confDir);
+                        ByteBuffer tempConfigFile = generateConfigFiles(newConfigIs, confDir, bootstrapFileProvider.getBootstrapProperties());
+                        runner.getConfigFileReference().set(tempConfigFile.asReadOnlyBuffer());
+
+                        try {
+                            logger.info("Reloading instance with new configuration");
+                            restartInstance();
+                        } catch (Exception e) {
+                            logger.debug("Transformation of new config file failed after transformation into Flow.xml and nifi.properties, reverting.");
+                            try (FileInputStream swapConfigFileStream = new FileInputStream(swapConfigFile)) {
+                                ByteBuffer resetConfigFile = generateConfigFiles(swapConfigFileStream, confDir, bootstrapFileProvider.getBootstrapProperties());
+                                runner.getConfigFileReference().set(resetConfigFile.asReadOnlyBuffer());
+                            }
+                            throw e;
+                        }
+                    } catch (Exception e) {
+                        logger.debug("Transformation of new config file failed after replacing original with the swap file, reverting.");
+                        try (FileInputStream swapConfigFileStream = new FileInputStream(swapConfigFile)) {
+                            Files.copy(swapConfigFileStream, configFile.toPath(), REPLACE_EXISTING);
+                        }
+                        throw e;
+                    }
+                } catch (Exception e) {
+                    logger.debug("Transformation of new config file failed after swap file was created, deleting it.");
+                    if (!swapConfigFile.delete()) {
+                        logger.warn("The swap file failed to delete after a failed handling of a change. It should be cleaned up manually.");
+                    }
+                    throw e;
+                }
+            }

Review Comment:
   Restructured and moved some to private functions. It could be refactored even further but didn't wanted to "overdo" it as there is upcoming work where we would pass the flow content to flowservice directly which will render most of the code here obsolete (and all of this will be removed).



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #6075: NIFI-9666 Implement base C2 client functionality

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on code in PR #6075:
URL: https://github.com/apache/nifi/pull/6075#discussion_r893724461


##########
c2/c2-client-bundle/c2-client-base/src/main/java/org/apache/nifi/c2/client/C2ClientConfig.java:
##########
@@ -0,0 +1,269 @@
+/*
+ * 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.nifi.c2.client;
+
+import javax.net.ssl.HostnameVerifier;
+
+/**
+ * Configuration for a C2 Client.
+ */
+public class C2ClientConfig {
+
+    private final String c2Url;
+    private final String c2AckUrl;
+    private final String agentClass;
+    private final String agentIdentifier;
+    private final String confDirectory;
+    private final String runtimeManifestIdentifier;
+    private final String runtimeType;
+    private final long heartbeatPeriod;
+    private final String keystoreFilename;
+    private final String keystorePass;
+    private final String keyPass;
+    private final String keystoreType;
+    private final String truststoreFilename;
+    private final String truststorePass;
+    private final String truststoreType;
+    private final HostnameVerifier hostnameVerifier;

Review Comment:
   The `HostnameVerifier` property does not appear to be used. Overriding the `HostnameVerifier` for TLS connections is generally against good security practices, so it should be removed from the configuration.



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/MiNiFiConfigurationChangeListener.java:
##########
@@ -0,0 +1,208 @@
+/*
+ * 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.nifi.minifi.bootstrap.service;
+
+import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CONF_DIR_KEY;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.MINIFI_CONFIG_FILE_KEY;
+import static org.apache.nifi.minifi.bootstrap.configuration.ingestors.PullHttpChangeIngestor.OVERRIDE_SECURITY;
+import static org.apache.nifi.minifi.bootstrap.configuration.ingestors.PullHttpChangeIngestor.PULL_HTTP_BASE_KEY;
+import static org.apache.nifi.minifi.bootstrap.util.ConfigTransformer.generateConfigFiles;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.util.Properties;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.commons.io.IOUtils;
+import org.apache.nifi.minifi.bootstrap.RunMiNiFi;
+import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeException;
+import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeListener;
+import org.apache.nifi.minifi.bootstrap.util.ByteBufferInputStream;
+import org.apache.nifi.minifi.bootstrap.util.ConfigTransformer;
+import org.apache.nifi.minifi.commons.schema.ConfigSchema;
+import org.apache.nifi.minifi.commons.schema.common.ConvertableSchema;
+import org.apache.nifi.minifi.commons.schema.serialization.SchemaLoader;
+import org.slf4j.Logger;
+
+public class MiNiFiConfigurationChangeListener implements ConfigurationChangeListener {
+
+    private final RunMiNiFi runner;
+    private final Logger logger;
+    private final BootstrapFileProvider bootstrapFileProvider;
+
+    private static final ReentrantLock handlingLock = new ReentrantLock();
+
+    public MiNiFiConfigurationChangeListener(RunMiNiFi runner, Logger logger, BootstrapFileProvider bootstrapFileProvider) {
+        this.runner = runner;
+        this.logger = logger;
+        this.bootstrapFileProvider = bootstrapFileProvider;
+    }
+
+    @Override
+    public void handleChange(InputStream configInputStream) throws ConfigurationChangeException {
+        logger.info("Received notification of a change");
+
+        if (!handlingLock.tryLock()) {
+            throw new ConfigurationChangeException("Instance is already handling another change");
+        }
+        // Store the incoming stream as a byte array to be shared among components that need it
+        try(ByteArrayOutputStream bufferedConfigOs = new ByteArrayOutputStream()) {
+
+            Properties bootstrapProperties = bootstrapFileProvider.getBootstrapProperties();
+            File configFile = new File(bootstrapProperties.getProperty(MINIFI_CONFIG_FILE_KEY));
+
+            byte[] copyArray = new byte[1024];
+            int available;
+            while ((available = configInputStream.read(copyArray)) > 0) {
+                bufferedConfigOs.write(copyArray, 0, available);
+            }
+
+            File swapConfigFile = bootstrapFileProvider.getSwapFile();
+            logger.info("Persisting old configuration to {}", swapConfigFile.getAbsolutePath());
+
+            try (FileInputStream configFileInputStream = new FileInputStream(configFile)) {
+                Files.copy(configFileInputStream, swapConfigFile.toPath(), REPLACE_EXISTING);
+            }
+
+            persistBackNonFlowSectionsFromOriginalSchema(bufferedConfigOs.toByteArray(), bootstrapProperties, configFile);
+
+            // Create an input stream to feed to the config transformer
+            try (FileInputStream newConfigIs = new FileInputStream(configFile)) {
+
+                try {
+                    String confDir = bootstrapProperties.getProperty(CONF_DIR_KEY);
+
+                    try {
+                        logger.info("Performing transformation for input and saving outputs to {}", confDir);
+                        ByteBuffer tempConfigFile = generateConfigFiles(newConfigIs, confDir, bootstrapFileProvider.getBootstrapProperties());
+                        runner.getConfigFileReference().set(tempConfigFile.asReadOnlyBuffer());
+
+                        try {
+                            logger.info("Reloading instance with new configuration");
+                            restartInstance();
+                        } catch (Exception e) {
+                            logger.debug("Transformation of new config file failed after transformation into Flow.xml and nifi.properties, reverting.");
+                            try (FileInputStream swapConfigFileStream = new FileInputStream(swapConfigFile)) {
+                                ByteBuffer resetConfigFile = generateConfigFiles(swapConfigFileStream, confDir, bootstrapFileProvider.getBootstrapProperties());
+                                runner.getConfigFileReference().set(resetConfigFile.asReadOnlyBuffer());
+                            }
+                            throw e;
+                        }
+                    } catch (Exception e) {
+                        logger.debug("Transformation of new config file failed after replacing original with the swap file, reverting.");
+                        try (FileInputStream swapConfigFileStream = new FileInputStream(swapConfigFile)) {
+                            Files.copy(swapConfigFileStream, configFile.toPath(), REPLACE_EXISTING);
+                        }
+                        throw e;
+                    }
+                } catch (Exception e) {
+                    logger.debug("Transformation of new config file failed after swap file was created, deleting it.");
+                    if (!swapConfigFile.delete()) {
+                        logger.warn("The swap file failed to delete after a failed handling of a change. It should be cleaned up manually.");
+                    }
+                    throw e;
+                }
+            }

Review Comment:
   This set of try-catch blocks is nested deeply, it would be helpful to refactor to separate methods.



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/MiNiFiCommandSender.java:
##########
@@ -0,0 +1,131 @@
+/*
+ * 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.nifi.minifi.bootstrap.service;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.BufferedReader;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.net.SocketTimeoutException;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MiNiFiCommandSender {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(MiNiFiCommandSender.class);
+    private static final String PING_CMD = "PING";
+    private static final int SOCKET_TIMEOUT = 10000;
+    private static final int CONNECTION_TIMEOUT = 10000;
+
+    private final MiNiFiParameters miNiFiParameters;
+    private final ObjectMapper objectMapper;
+
+    public MiNiFiCommandSender(MiNiFiParameters miNiFiParameters, ObjectMapper objectMapper) {
+        this.miNiFiParameters = miNiFiParameters;
+        this.objectMapper = objectMapper;
+    }
+
+    public Optional<String> sendCommand(String cmd, Integer port, String... extraParams) throws IOException {
+        Optional<String> response = Optional.empty();
+
+        if (port == null) {
+            LOGGER.info("Apache MiNiFi is not currently running");
+            return response;
+        }
+
+        try (Socket socket = new Socket()) {
+            LOGGER.debug("Connecting to MiNiFi instance");
+            socket.setSoTimeout(SOCKET_TIMEOUT);
+            socket.connect(new InetSocketAddress("localhost", port), CONNECTION_TIMEOUT);
+            LOGGER.debug("Established connection to MiNiFi instance.");
+
+            LOGGER.debug("Sending {} Command to port {}", cmd, port);
+
+            String responseString;
+            try (OutputStream out = socket.getOutputStream()) {
+                out.write(getCommand(cmd, extraParams));
+                out.flush();
+                responseString = readResponse(socket);
+            }
+
+            LOGGER.debug("Received response to {} command: {}", cmd, responseString);
+            response = Optional.of(responseString);
+        } catch (EOFException | SocketTimeoutException e) {
+            String message = "Failed to get response for " + cmd + " Potentially due to the process currently being down (restarting or otherwise)";
+            throw new RuntimeException(message);
+        }
+        return response;
+    }
+
+    <T> T sendCommandForObject(String cmd, Integer port, Class<T> clazz, String... extraParams) throws IOException {
+        return sendCommand(cmd, port, extraParams)
+            .map(response -> deserialize(cmd, response, clazz))
+            .orElse(null);
+    }
+
+    private String readResponse(Socket socket) throws IOException {
+        StringBuilder sb = new StringBuilder();
+        int numLines = 0;
+        try (BufferedReader reader = new BufferedReader(new InputStreamReader(socket.getInputStream()))) {
+            String line;
+            while ((line = reader.readLine()) != null) {
+                if (numLines++ > 0) {
+                    sb.append("\n");
+                }
+                sb.append(line);
+            }
+        }
+
+        return sb.toString().trim();
+    }
+
+    private byte[] getCommand(String cmd, String... args) {
+        String argsString = Arrays.stream(args).collect(Collectors.joining(" "));
+        String commandWithArgs = cmd + " " + miNiFiParameters.getSecretKey() + (args.length > 0 ? " " : "") + argsString + "\n";
+        return commandWithArgs.getBytes(StandardCharsets.UTF_8);
+    }
+
+    private <T> T deserialize(String cmd, String obj, Class<T> clazz) {
+        T response;
+        try {
+            response = objectMapper.readValue(obj, clazz);
+        } catch (JsonProcessingException e) {
+            String message = "Failed to deserialize " + cmd + " response";
+            LOGGER.error(message);
+            throw new RuntimeException(message);

Review Comment:
   The `JsonProcessingException` should be passed as the cause.
   ```suggestion
               throw new RuntimeException(message, e);
   ```



##########
minifi/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/bootstrap.conf:
##########
@@ -129,3 +128,36 @@ java.arg.7=-Djava.security.egd=file:/dev/urandom
 
 #Set headless mode by default
 java.arg.14=-Djava.awt.headless=true
+
+# MiNiFi Command & Control Configuration
+# C2 Properties
+# Enabling C2 Uncomment each of the following options
+#c2.enable=true
+## define protocol parameters
+#c2.rest.url=
+#c2.rest.url.ack=
+## c2 timeouts
+#c2.rest.connectionTimeout=5 sec
+#c2.rest.readTimeout=5 sec
+#c2.rest.callTimeout=10 sec
+## heartbeat in milliseconds
+#c2.agent.heartbeat.period=5000
+## define parameters about your agent
+#c2.agent.class=
+#c2.config.directory=./conf
+#c2.runtime.manifest.identifier=minifi
+#c2.runtime.type=minifi-java
+# Optional.  Defaults to a hardware based unique identifier
+#c2.agent.identifier=
+## Define TLS security properties for C2 communications
+#c2.security.truststore.location=
+#c2.security.truststore.password=
+#c2.security.truststore.type=JKS
+#c2.security.keystore.location=
+#c2.security.keystore.password=
+#c2.security.keystore.type=JKS
+#c2.security.need.client.auth=true

Review Comment:
   Is this property used?



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/MiNiFiConfigurationChangeListener.java:
##########
@@ -0,0 +1,208 @@
+/*
+ * 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.nifi.minifi.bootstrap.service;
+
+import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CONF_DIR_KEY;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.MINIFI_CONFIG_FILE_KEY;
+import static org.apache.nifi.minifi.bootstrap.configuration.ingestors.PullHttpChangeIngestor.OVERRIDE_SECURITY;
+import static org.apache.nifi.minifi.bootstrap.configuration.ingestors.PullHttpChangeIngestor.PULL_HTTP_BASE_KEY;
+import static org.apache.nifi.minifi.bootstrap.util.ConfigTransformer.generateConfigFiles;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.util.Properties;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.commons.io.IOUtils;
+import org.apache.nifi.minifi.bootstrap.RunMiNiFi;
+import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeException;
+import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeListener;
+import org.apache.nifi.minifi.bootstrap.util.ByteBufferInputStream;
+import org.apache.nifi.minifi.bootstrap.util.ConfigTransformer;
+import org.apache.nifi.minifi.commons.schema.ConfigSchema;
+import org.apache.nifi.minifi.commons.schema.common.ConvertableSchema;
+import org.apache.nifi.minifi.commons.schema.serialization.SchemaLoader;
+import org.slf4j.Logger;
+
+public class MiNiFiConfigurationChangeListener implements ConfigurationChangeListener {
+
+    private final RunMiNiFi runner;
+    private final Logger logger;
+    private final BootstrapFileProvider bootstrapFileProvider;
+
+    private static final ReentrantLock handlingLock = new ReentrantLock();
+
+    public MiNiFiConfigurationChangeListener(RunMiNiFi runner, Logger logger, BootstrapFileProvider bootstrapFileProvider) {
+        this.runner = runner;
+        this.logger = logger;
+        this.bootstrapFileProvider = bootstrapFileProvider;
+    }
+
+    @Override
+    public void handleChange(InputStream configInputStream) throws ConfigurationChangeException {
+        logger.info("Received notification of a change");
+
+        if (!handlingLock.tryLock()) {
+            throw new ConfigurationChangeException("Instance is already handling another change");
+        }
+        // Store the incoming stream as a byte array to be shared among components that need it
+        try(ByteArrayOutputStream bufferedConfigOs = new ByteArrayOutputStream()) {
+
+            Properties bootstrapProperties = bootstrapFileProvider.getBootstrapProperties();
+            File configFile = new File(bootstrapProperties.getProperty(MINIFI_CONFIG_FILE_KEY));
+
+            byte[] copyArray = new byte[1024];
+            int available;
+            while ((available = configInputStream.read(copyArray)) > 0) {
+                bufferedConfigOs.write(copyArray, 0, available);
+            }

Review Comment:
   It looks like this approach could be replaced with `IOUtils.copy()`



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/MiNiFiConfigurationChangeListener.java:
##########
@@ -0,0 +1,208 @@
+/*
+ * 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.nifi.minifi.bootstrap.service;
+
+import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CONF_DIR_KEY;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.MINIFI_CONFIG_FILE_KEY;
+import static org.apache.nifi.minifi.bootstrap.configuration.ingestors.PullHttpChangeIngestor.OVERRIDE_SECURITY;
+import static org.apache.nifi.minifi.bootstrap.configuration.ingestors.PullHttpChangeIngestor.PULL_HTTP_BASE_KEY;
+import static org.apache.nifi.minifi.bootstrap.util.ConfigTransformer.generateConfigFiles;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.util.Properties;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.commons.io.IOUtils;
+import org.apache.nifi.minifi.bootstrap.RunMiNiFi;
+import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeException;
+import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeListener;
+import org.apache.nifi.minifi.bootstrap.util.ByteBufferInputStream;
+import org.apache.nifi.minifi.bootstrap.util.ConfigTransformer;
+import org.apache.nifi.minifi.commons.schema.ConfigSchema;
+import org.apache.nifi.minifi.commons.schema.common.ConvertableSchema;
+import org.apache.nifi.minifi.commons.schema.serialization.SchemaLoader;
+import org.slf4j.Logger;
+
+public class MiNiFiConfigurationChangeListener implements ConfigurationChangeListener {
+
+    private final RunMiNiFi runner;
+    private final Logger logger;
+    private final BootstrapFileProvider bootstrapFileProvider;
+
+    private static final ReentrantLock handlingLock = new ReentrantLock();
+
+    public MiNiFiConfigurationChangeListener(RunMiNiFi runner, Logger logger, BootstrapFileProvider bootstrapFileProvider) {
+        this.runner = runner;
+        this.logger = logger;
+        this.bootstrapFileProvider = bootstrapFileProvider;
+    }
+
+    @Override
+    public void handleChange(InputStream configInputStream) throws ConfigurationChangeException {
+        logger.info("Received notification of a change");
+
+        if (!handlingLock.tryLock()) {
+            throw new ConfigurationChangeException("Instance is already handling another change");
+        }
+        // Store the incoming stream as a byte array to be shared among components that need it
+        try(ByteArrayOutputStream bufferedConfigOs = new ByteArrayOutputStream()) {
+
+            Properties bootstrapProperties = bootstrapFileProvider.getBootstrapProperties();
+            File configFile = new File(bootstrapProperties.getProperty(MINIFI_CONFIG_FILE_KEY));
+
+            byte[] copyArray = new byte[1024];
+            int available;
+            while ((available = configInputStream.read(copyArray)) > 0) {
+                bufferedConfigOs.write(copyArray, 0, available);
+            }
+
+            File swapConfigFile = bootstrapFileProvider.getSwapFile();
+            logger.info("Persisting old configuration to {}", swapConfigFile.getAbsolutePath());
+
+            try (FileInputStream configFileInputStream = new FileInputStream(configFile)) {
+                Files.copy(configFileInputStream, swapConfigFile.toPath(), REPLACE_EXISTING);
+            }
+
+            persistBackNonFlowSectionsFromOriginalSchema(bufferedConfigOs.toByteArray(), bootstrapProperties, configFile);
+
+            // Create an input stream to feed to the config transformer
+            try (FileInputStream newConfigIs = new FileInputStream(configFile)) {
+
+                try {
+                    String confDir = bootstrapProperties.getProperty(CONF_DIR_KEY);
+
+                    try {
+                        logger.info("Performing transformation for input and saving outputs to {}", confDir);
+                        ByteBuffer tempConfigFile = generateConfigFiles(newConfigIs, confDir, bootstrapFileProvider.getBootstrapProperties());
+                        runner.getConfigFileReference().set(tempConfigFile.asReadOnlyBuffer());
+
+                        try {
+                            logger.info("Reloading instance with new configuration");
+                            restartInstance();
+                        } catch (Exception e) {
+                            logger.debug("Transformation of new config file failed after transformation into Flow.xml and nifi.properties, reverting.");
+                            try (FileInputStream swapConfigFileStream = new FileInputStream(swapConfigFile)) {
+                                ByteBuffer resetConfigFile = generateConfigFiles(swapConfigFileStream, confDir, bootstrapFileProvider.getBootstrapProperties());
+                                runner.getConfigFileReference().set(resetConfigFile.asReadOnlyBuffer());
+                            }
+                            throw e;
+                        }
+                    } catch (Exception e) {
+                        logger.debug("Transformation of new config file failed after replacing original with the swap file, reverting.");
+                        try (FileInputStream swapConfigFileStream = new FileInputStream(swapConfigFile)) {
+                            Files.copy(swapConfigFileStream, configFile.toPath(), REPLACE_EXISTING);
+                        }
+                        throw e;
+                    }
+                } catch (Exception e) {
+                    logger.debug("Transformation of new config file failed after swap file was created, deleting it.");
+                    if (!swapConfigFile.delete()) {
+                        logger.warn("The swap file failed to delete after a failed handling of a change. It should be cleaned up manually.");
+                    }
+                    throw e;
+                }
+            }
+        } catch (ConfigurationChangeException e){
+            logger.error("Unable to carry out reloading of configuration on receipt of notification event", e);
+            throw e;
+        } catch (IOException ioe) {
+            logger.error("Unable to carry out reloading of configuration on receipt of notification event", ioe);
+            throw new ConfigurationChangeException("Unable to perform reload of received configuration change", ioe);

Review Comment:
   Is there a reason for logging the error and throw the exception? Shouldn't the exception be caught and logged by the caller? This approach seems like it would generate multiple log messages for the same problem.



##########
c2/c2-client-bundle/c2-client-base/src/main/java/org/apache/nifi/c2/client/C2ClientConfig.java:
##########
@@ -0,0 +1,269 @@
+/*
+ * 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.nifi.c2.client;
+
+import javax.net.ssl.HostnameVerifier;
+
+/**
+ * Configuration for a C2 Client.
+ */
+public class C2ClientConfig {
+
+    private final String c2Url;
+    private final String c2AckUrl;
+    private final String agentClass;
+    private final String agentIdentifier;
+    private final String confDirectory;
+    private final String runtimeManifestIdentifier;
+    private final String runtimeType;
+    private final long heartbeatPeriod;
+    private final String keystoreFilename;
+    private final String keystorePass;
+    private final String keyPass;
+    private final String keystoreType;
+    private final String truststoreFilename;
+    private final String truststorePass;
+    private final String truststoreType;
+    private final HostnameVerifier hostnameVerifier;

Review Comment:
   @mattyb149 or @bejancsaba Can this property be removed?



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/MiNiFiConfigurationChangeListener.java:
##########
@@ -0,0 +1,208 @@
+/*
+ * 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.nifi.minifi.bootstrap.service;
+
+import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CONF_DIR_KEY;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.MINIFI_CONFIG_FILE_KEY;
+import static org.apache.nifi.minifi.bootstrap.configuration.ingestors.PullHttpChangeIngestor.OVERRIDE_SECURITY;
+import static org.apache.nifi.minifi.bootstrap.configuration.ingestors.PullHttpChangeIngestor.PULL_HTTP_BASE_KEY;
+import static org.apache.nifi.minifi.bootstrap.util.ConfigTransformer.generateConfigFiles;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.util.Properties;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.commons.io.IOUtils;
+import org.apache.nifi.minifi.bootstrap.RunMiNiFi;
+import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeException;
+import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeListener;
+import org.apache.nifi.minifi.bootstrap.util.ByteBufferInputStream;
+import org.apache.nifi.minifi.bootstrap.util.ConfigTransformer;
+import org.apache.nifi.minifi.commons.schema.ConfigSchema;
+import org.apache.nifi.minifi.commons.schema.common.ConvertableSchema;
+import org.apache.nifi.minifi.commons.schema.serialization.SchemaLoader;
+import org.slf4j.Logger;
+
+public class MiNiFiConfigurationChangeListener implements ConfigurationChangeListener {
+
+    private final RunMiNiFi runner;
+    private final Logger logger;
+    private final BootstrapFileProvider bootstrapFileProvider;
+
+    private static final ReentrantLock handlingLock = new ReentrantLock();
+
+    public MiNiFiConfigurationChangeListener(RunMiNiFi runner, Logger logger, BootstrapFileProvider bootstrapFileProvider) {
+        this.runner = runner;
+        this.logger = logger;
+        this.bootstrapFileProvider = bootstrapFileProvider;
+    }
+
+    @Override
+    public void handleChange(InputStream configInputStream) throws ConfigurationChangeException {
+        logger.info("Received notification of a change");
+
+        if (!handlingLock.tryLock()) {
+            throw new ConfigurationChangeException("Instance is already handling another change");
+        }
+        // Store the incoming stream as a byte array to be shared among components that need it
+        try(ByteArrayOutputStream bufferedConfigOs = new ByteArrayOutputStream()) {
+
+            Properties bootstrapProperties = bootstrapFileProvider.getBootstrapProperties();
+            File configFile = new File(bootstrapProperties.getProperty(MINIFI_CONFIG_FILE_KEY));
+
+            byte[] copyArray = new byte[1024];
+            int available;
+            while ((available = configInputStream.read(copyArray)) > 0) {
+                bufferedConfigOs.write(copyArray, 0, available);
+            }
+
+            File swapConfigFile = bootstrapFileProvider.getSwapFile();
+            logger.info("Persisting old configuration to {}", swapConfigFile.getAbsolutePath());
+
+            try (FileInputStream configFileInputStream = new FileInputStream(configFile)) {
+                Files.copy(configFileInputStream, swapConfigFile.toPath(), REPLACE_EXISTING);
+            }
+
+            persistBackNonFlowSectionsFromOriginalSchema(bufferedConfigOs.toByteArray(), bootstrapProperties, configFile);
+
+            // Create an input stream to feed to the config transformer
+            try (FileInputStream newConfigIs = new FileInputStream(configFile)) {
+
+                try {
+                    String confDir = bootstrapProperties.getProperty(CONF_DIR_KEY);
+
+                    try {
+                        logger.info("Performing transformation for input and saving outputs to {}", confDir);
+                        ByteBuffer tempConfigFile = generateConfigFiles(newConfigIs, confDir, bootstrapFileProvider.getBootstrapProperties());
+                        runner.getConfigFileReference().set(tempConfigFile.asReadOnlyBuffer());
+
+                        try {
+                            logger.info("Reloading instance with new configuration");
+                            restartInstance();
+                        } catch (Exception e) {
+                            logger.debug("Transformation of new config file failed after transformation into Flow.xml and nifi.properties, reverting.");
+                            try (FileInputStream swapConfigFileStream = new FileInputStream(swapConfigFile)) {
+                                ByteBuffer resetConfigFile = generateConfigFiles(swapConfigFileStream, confDir, bootstrapFileProvider.getBootstrapProperties());
+                                runner.getConfigFileReference().set(resetConfigFile.asReadOnlyBuffer());
+                            }
+                            throw e;
+                        }
+                    } catch (Exception e) {
+                        logger.debug("Transformation of new config file failed after replacing original with the swap file, reverting.");
+                        try (FileInputStream swapConfigFileStream = new FileInputStream(swapConfigFile)) {
+                            Files.copy(swapConfigFileStream, configFile.toPath(), REPLACE_EXISTING);
+                        }
+                        throw e;
+                    }
+                } catch (Exception e) {
+                    logger.debug("Transformation of new config file failed after swap file was created, deleting it.");
+                    if (!swapConfigFile.delete()) {
+                        logger.warn("The swap file failed to delete after a failed handling of a change. It should be cleaned up manually.");
+                    }
+                    throw e;
+                }
+            }
+        } catch (ConfigurationChangeException e){
+            logger.error("Unable to carry out reloading of configuration on receipt of notification event", e);
+            throw e;
+        } catch (IOException ioe) {
+            logger.error("Unable to carry out reloading of configuration on receipt of notification event", ioe);
+            throw new ConfigurationChangeException("Unable to perform reload of received configuration change", ioe);
+        } finally {
+            IOUtils.closeQuietly(configInputStream);
+            handlingLock.unlock();
+        }
+    }
+
+    @Override
+    public String getDescriptor() {
+        return "MiNiFiConfigurationChangeListener";
+    }
+
+    private void restartInstance() throws IOException {
+        try {
+            runner.reload();
+        } catch (IOException e) {
+            throw new IOException("Unable to successfully restart MiNiFi instance after configuration change.", e);
+        }
+    }
+
+    private void persistBackNonFlowSectionsFromOriginalSchema(byte[] newSchema, Properties bootstrapProperties, File configFile) {
+        try {
+            ConvertableSchema<ConfigSchema> schemaNew = ConfigTransformer
+                .throwIfInvalid(SchemaLoader.loadConvertableSchemaFromYaml(new ByteArrayInputStream(newSchema)));
+            ConfigSchema configSchemaNew = ConfigTransformer.throwIfInvalid(schemaNew.convert());
+            ConvertableSchema<ConfigSchema> schemaOld = ConfigTransformer
+                .throwIfInvalid(SchemaLoader.loadConvertableSchemaFromYaml(new ByteBufferInputStream(runner.getConfigFileReference().get().duplicate())));
+            ConfigSchema configSchemaOld = ConfigTransformer.throwIfInvalid(schemaOld.convert());
+
+            configSchemaNew.setNifiPropertiesOverrides(configSchemaOld.getNifiPropertiesOverrides());
+
+            if (!overrideCoreProperties(bootstrapProperties)) {
+                logger.debug("Preserving previous core properties...");
+                configSchemaNew.setCoreProperties(configSchemaOld.getCoreProperties());
+            }
+
+            if (!overrideSecurityProperties(bootstrapProperties)) {
+                logger.debug("Preserving previous security properties...");
+                configSchemaNew.setSecurityProperties(configSchemaOld.getSecurityProperties());
+            }
+
+            logger.debug("Persisting changes to {}", configFile.getAbsolutePath());
+            SchemaLoader.toYaml(configSchemaNew, new FileWriter(configFile));
+        } catch (Exception e) {
+            logger.error("Loading the old and the new schema for merging was not successful");

Review Comment:
   The exception should be logged:
   ```suggestion
               logger.error("Loading the old and the new schema for merging was not successful", e);
   ```



##########
nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/c2/C2NiFiProperties.java:
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.nifi.c2;
+
+import java.util.concurrent.TimeUnit;
+
+public class C2NiFiProperties {
+
+    public static final String C2_PREFIX = "c2.";
+
+    public static final String C2_ENABLE_KEY = C2_PREFIX + "enable";
+    public static final String C2_AGENT_PROTOCOL_KEY = C2_PREFIX + "agent.protocol.class";
+    public static final String C2_COAP_HOST_KEY = C2_PREFIX + "agent.coap.host";
+    public static final String C2_COAP_PORT_KEY = C2_PREFIX + "agent.coap.port";
+    public static final String C2_CONFIG_DIRECTORY_KEY = C2_PREFIX + "config.directory";
+    public static final String C2_RUNTIME_MANIFEST_IDENTIFIER_KEY = C2_PREFIX + "runtime.manifest.identifier";
+    public static final String C2_RUNTIME_TYPE_KEY = C2_PREFIX + "runtime.type";
+    public static final String C2_REST_URL_KEY = C2_PREFIX + "rest.url";
+    public static final String C2_REST_URL_ACK_KEY = C2_PREFIX + "rest.url.ack";
+    public static final String C2_ROOT_CLASSES_KEY = C2_PREFIX + "root.classes";
+    public static final String C2_AGENT_HEARTBEAT_PERIOD_KEY = C2_PREFIX + "agent.heartbeat.period";
+    public static final String C2_CONNECTION_TIMEOUT = C2_PREFIX + "rest.connectionTimeout";
+    public static final String C2_READ_TIMEOUT = C2_PREFIX + "rest.readTimeout";
+    public static final String C2_CALL_TIMEOUT = C2_PREFIX + "rest.callTimeout";
+    public static final String C2_AGENT_CLASS_KEY = C2_PREFIX + "agent.class";
+    public static final String C2_AGENT_IDENTIFIER_KEY = C2_PREFIX + "agent.identifier";
+
+    public static final String C2_ROOT_CLASS_DEFINITIONS_KEY = C2_PREFIX + "root.class.definitions";
+    public static final String C2_METRICS_NAME_KEY = C2_ROOT_CLASS_DEFINITIONS_KEY + ".metrics.name";
+    public static final String C2_METRICS_METRICS_KEY = C2_ROOT_CLASS_DEFINITIONS_KEY + ".metrics.metrics";
+    public static final String C2_METRICS_METRICS_TYPED_METRICS_NAME_KEY = C2_ROOT_CLASS_DEFINITIONS_KEY + ".metrics.metrics.typedmetrics.name";
+    public static final String C2_METRICS_METRICS_QUEUED_METRICS_NAME_KEY = C2_ROOT_CLASS_DEFINITIONS_KEY + ".metrics.metrics.queuemetrics.name";
+    public static final String C2_METRICS_METRICS_QUEUE_METRICS_CLASSES_KEY = C2_ROOT_CLASS_DEFINITIONS_KEY + ".metrics.metrics.queuemetrics.classes";
+    public static final String C2_METRICS_METRICS_TYPED_METRICS_CLASSES_KEY = C2_ROOT_CLASS_DEFINITIONS_KEY + ".metrics.metrics.typedmetrics.classes";
+    public static final String C2_METRICS_METRICS_PROCESSOR_METRICS_NAME_KEY = C2_ROOT_CLASS_DEFINITIONS_KEY + ".metrics.metrics.processorMetrics.name";
+    public static final String C2_METRICS_METRICS_PROCESSOR_METRICS_CLASSES_KEY = C2_ROOT_CLASS_DEFINITIONS_KEY + ".metrics.metrics.processorMetrics.classes";
+
+    /* C2 Client Security Properties */
+    private static final String C2_REST_SECURITY_BASE_KEY = C2_PREFIX + "security";
+    public static final String TRUSTSTORE_LOCATION_KEY = C2_REST_SECURITY_BASE_KEY + ".truststore.location";
+    public static final String TRUSTSTORE_PASSWORD_KEY = C2_REST_SECURITY_BASE_KEY + ".truststore.password";
+    public static final String TRUSTSTORE_TYPE_KEY = C2_REST_SECURITY_BASE_KEY + ".truststore.type";
+    public static final String KEYSTORE_LOCATION_KEY = C2_REST_SECURITY_BASE_KEY + ".keystore.location";
+    public static final String KEYSTORE_PASSWORD_KEY = C2_REST_SECURITY_BASE_KEY + ".keystore.password";
+    public static final String KEYSTORE_TYPE_KEY = C2_REST_SECURITY_BASE_KEY + ".keystore.type";
+    public static final String NEED_CLIENT_AUTH_KEY = C2_REST_SECURITY_BASE_KEY + ".need.client.auth";

Review Comment:
   Is this property still applicable?



##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/PeriodicStatusReporterManager.java:
##########
@@ -0,0 +1,130 @@
+/*
+ * 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.nifi.minifi.bootstrap.service;
+
+import static org.apache.nifi.minifi.commons.schema.common.BootstrapPropertyKeys.STATUS_REPORTER_COMPONENTS_KEY;
+
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
+import org.apache.nifi.minifi.bootstrap.MiNiFiStatus;
+import org.apache.nifi.minifi.bootstrap.QueryableStatusAggregator;
+import org.apache.nifi.minifi.bootstrap.status.PeriodicStatusReporter;
+import org.apache.nifi.minifi.commons.status.FlowStatusReport;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PeriodicStatusReporterManager implements QueryableStatusAggregator {
+    private static final Logger LOGGER = LoggerFactory.getLogger(PeriodicStatusReporterManager.class);
+    private static final String FLOW_STATUS_REPORT_CMD = "FLOW_STATUS_REPORT";
+
+    private final Properties bootstrapProperties;
+    private final MiNiFiStatusProvider miNiFiStatusProvider;
+    private final MiNiFiCommandSender miNiFiCommandSender;
+    private final MiNiFiParameters miNiFiParameters;
+
+    private Set<PeriodicStatusReporter> periodicStatusReporters = Collections.emptySet();
+
+    public PeriodicStatusReporterManager(Properties bootstrapProperties, MiNiFiStatusProvider miNiFiStatusProvider, MiNiFiCommandSender miNiFiCommandSender,
+        MiNiFiParameters miNiFiParameters) {
+        this.bootstrapProperties = bootstrapProperties;
+        this.miNiFiStatusProvider = miNiFiStatusProvider;
+        this.miNiFiCommandSender = miNiFiCommandSender;
+        this.miNiFiParameters = miNiFiParameters;
+    }
+
+    public void startPeriodicNotifiers() {
+        periodicStatusReporters = initializePeriodicNotifiers();
+
+        for (PeriodicStatusReporter periodicStatusReporter: periodicStatusReporters) {
+            periodicStatusReporter.start();
+            LOGGER.debug("Started {} notifier", periodicStatusReporter.getClass().getCanonicalName());
+        }
+    }
+
+    public void shutdownPeriodicStatusReporters() {
+        LOGGER.debug("Initiating shutdown of bootstrap periodic status reporters...");
+        for (PeriodicStatusReporter periodicStatusReporter : periodicStatusReporters) {
+            try {
+                periodicStatusReporter.stop();
+            } catch (Exception exception) {
+                LOGGER.error("Could not successfully stop periodic status reporter " + periodicStatusReporter.getClass() + " due to ", exception);
+            }
+        }
+    }
+
+    public FlowStatusReport statusReport(String statusRequest) {
+        MiNiFiStatus status = miNiFiStatusProvider.getStatus(miNiFiParameters.getMiNiFiPort(), miNiFiParameters.getMinifiPid());
+
+        List<String> problemsGeneratingReport = new LinkedList<>();
+        if (!status.isProcessRunning()) {
+            problemsGeneratingReport.add("MiNiFi process is not running");
+        }
+
+        if (!status.isRespondingToPing()) {
+            problemsGeneratingReport.add("MiNiFi process is not responding to pings");
+        }
+
+        if (!problemsGeneratingReport.isEmpty()) {
+            FlowStatusReport flowStatusReport = new FlowStatusReport();
+            flowStatusReport.setErrorsGeneratingReport(problemsGeneratingReport);
+            return flowStatusReport;
+        }
+
+        return getFlowStatusReport(statusRequest, status.getPort());
+    }
+
+    private Set<PeriodicStatusReporter> initializePeriodicNotifiers() {
+        LOGGER.debug("Initiating bootstrap periodic status reporters...");
+        Set<PeriodicStatusReporter> statusReporters = new HashSet<>();
+
+        String reportersCsv = bootstrapProperties.getProperty(STATUS_REPORTER_COMPONENTS_KEY);
+
+        if (reportersCsv != null && !reportersCsv.isEmpty()) {
+            for (String reporterClassname : reportersCsv.split(",")) {
+                try {
+                    Class<?> reporterClass = Class.forName(reporterClassname);
+                    PeriodicStatusReporter reporter = (PeriodicStatusReporter) reporterClass.newInstance();
+                    reporter.initialize(bootstrapProperties, this);
+                    statusReporters.add(reporter);
+                    LOGGER.debug("Initialized {} notifier", reporterClass.getCanonicalName());
+                } catch (InstantiationException | IllegalAccessException | ClassNotFoundException e) {
+                    throw new RuntimeException("Issue instantiating notifier " + reporterClassname, e);
+                }
+            }
+        }
+        return statusReporters;
+    }
+
+    private FlowStatusReport getFlowStatusReport(String statusRequest, int port) {
+        FlowStatusReport flowStatusReport;
+        try {
+            flowStatusReport = miNiFiCommandSender.sendCommandForObject(FLOW_STATUS_REPORT_CMD, port, FlowStatusReport.class, statusRequest);
+        } catch (Exception e) {
+            flowStatusReport = new FlowStatusReport();
+            String message = "Failed to get status report from MiNiFi due to:" + e.getMessage();
+            flowStatusReport.setErrorsGeneratingReport(Collections.singletonList(message));
+            LOGGER.error(message);

Review Comment:
   Recommend logging exception for tracking:
   ```suggestion
               LOGGER.error(message, e);
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] joewitt commented on pull request #6075: NIFI-9666 Implement base C2 client functionality

Posted by GitBox <gi...@apache.org>.
joewitt commented on PR #6075:
URL: https://github.com/apache/nifi/pull/6075#issuecomment-1157816191

   generally we should squash.  but given these are newer committers and a PMCer and a good collaboration I agree it would be ideal to keep the history.  Since it is a feature and not a bug this is also more ok.
   
   I am supportive of this 'many commits' merge but we should ideally avoid it generally.
   
   Not sure if others will have different opinions but +1 from me


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] adamdebreceni commented on a diff in pull request #6075: NIFI-9666 Implement base C2 client functionality

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #6075:
URL: https://github.com/apache/nifi/pull/6075#discussion_r891005803


##########
c2/c2-client-bundle/c2-client-service/src/main/java/org/apache/nifi/c2/client/service/operation/UpdateConfigurationOperationHandler.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.nifi.c2.client.service.operation;
+
+import static org.apache.commons.lang3.StringUtils.EMPTY;
+import static org.apache.nifi.c2.protocol.api.OperandType.CONFIGURATION;
+import static org.apache.nifi.c2.protocol.api.OperationType.UPDATE;
+
+import java.net.URI;
+import java.util.Optional;
+import java.util.function.Function;
+import org.apache.nifi.c2.client.api.C2Client;
+import org.apache.nifi.c2.client.service.FlowIdHolder;
+import org.apache.nifi.c2.protocol.api.C2Operation;
+import org.apache.nifi.c2.protocol.api.C2OperationAck;
+import org.apache.nifi.c2.protocol.api.C2OperationState;
+import org.apache.nifi.c2.protocol.api.OperandType;
+import org.apache.nifi.c2.protocol.api.OperationType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class UpdateConfigurationOperationHandler implements C2OperationHandler {
+
+    private static final Logger logger = LoggerFactory.getLogger(UpdateConfigurationOperationHandler.class);
+
+    private static final String LOCATION = "location";
+
+    private final C2Client client;
+    private final Function<byte[], Boolean> updateFlow;
+    private final FlowIdHolder flowIdHolder;
+
+    public UpdateConfigurationOperationHandler(C2Client client, FlowIdHolder flowIdHolder, Function<byte[], Boolean> updateFlow) {
+        this.client = client;
+        this.updateFlow = updateFlow;
+        this.flowIdHolder = flowIdHolder;
+    }
+
+    @Override
+    public OperationType getOperationType() {
+        return UPDATE;
+    }
+
+    @Override
+    public OperandType getOperandType() {
+        return CONFIGURATION;
+    }
+
+    @Override
+    public C2OperationAck handle(C2Operation operation) {
+        String opIdentifier = Optional.ofNullable(operation.getIdentifier())
+            .orElse(EMPTY);
+        C2OperationAck operationAck = new C2OperationAck();
+        C2OperationState state = new C2OperationState();
+        operationAck.setOperationState(state);
+        operationAck.setOperationId(opIdentifier);
+
+        String updateLocation = Optional.ofNullable(operation.getArgs())
+            .map(map -> map.get(LOCATION))
+            .orElse(EMPTY);
+
+        String newFlowId = parseFlowId(updateLocation);
+        if (flowIdHolder.getFlowId() == null || !flowIdHolder.getFlowId().equals(newFlowId)) {
+            logger.info("Will perform flow update from {} for operation #{}. Previous flow id was {}, replacing with new id {}", updateLocation, opIdentifier,
+                flowIdHolder.getFlowId() == null ? "not set" : flowIdHolder.getFlowId(), newFlowId);
+        } else {
+            logger.info("Flow is current, no update is necessary...");
+        }
+
+        flowIdHolder.setFlowId(newFlowId);
+        byte[] updateContent = client.retrieveUpdateContent(updateLocation);
+        if (updateContent != null) {
+            if (updateFlow.apply(updateContent)) {
+                state.setState(C2OperationState.OperationState.FULLY_APPLIED);
+                logger.debug("Update configuration applied for operation #{}.", opIdentifier);
+            } else {
+                state.setState(C2OperationState.OperationState.NOT_APPLIED);
+                logger.error("Update resulted in error for operation #{}.", opIdentifier);
+            }
+        } else {
+            state.setState(C2OperationState.OperationState.NOT_APPLIED);
+            logger.error("Update content retrieval resulted in empty content so flow update was omitted for operation #{}.", opIdentifier);
+        }
+
+        return operationAck;
+    }
+
+    private String parseFlowId(String flowUpdateUrl) {
+        try {
+            URI flowUri = new URI(flowUpdateUrl);
+            String flowUriPath = flowUri.getPath();
+            String[] split = flowUriPath.split("/");
+            if (split.length > 4) {
+                return split[4];

Review Comment:
   this seems to be a pretty strict flow url requirement, could we rely on the last segment of the path being the flow id instead?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] ferencerdei commented on pull request #6075: NIFI-9666 Implement base C2 client functionality

Posted by GitBox <gi...@apache.org>.
ferencerdei commented on PR #6075:
URL: https://github.com/apache/nifi/pull/6075#issuecomment-1154777327

   > Thansk @ferencerdei. I've verified the SIGTERM fix works as expected.
   > 
   > Regarding the logging output: I'm comparing the output of the docker image from this PR to the [latest release of nifi-minifi ](https://hub.docker.com/r/apache/nifi-minifi/tags), and I get the full output (more than just the bootstrap process) when I run this:
   > 
   > ```
   > docker run apache/nifi-minifi:latest
   > ```
   > 
   > compared to this (after building this PR branch), which only has the bootstrap output:
   > 
   > ```
   > docker run apacheminifi:latest
   > ```
   > 
   > I could be missing something, but I would expect those to more or less be the same in docker container output behavior?
   
   Did you build the new version with the dockerhub dockerfile version? It has an additional "tail -F "${MINIFI_HOME}/logs/minifi-app.log" &" part in the startup script compared to the dockermaven. We didn't touch that part, so it should work the same.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] adamdebreceni commented on a diff in pull request #6075: NIFI-9666 Implement base C2 client functionality

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #6075:
URL: https://github.com/apache/nifi/pull/6075#discussion_r890952285


##########
c2/c2-client-bundle/c2-client-http/src/main/java/org/apache/nifi/c2/client/http/C2HttpClient.java:
##########
@@ -0,0 +1,250 @@
+/*
+ * 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.nifi.c2.client.http;
+
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.security.KeyStore;
+import java.security.NoSuchAlgorithmException;
+import java.util.Optional;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLSocketFactory;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+import okhttp3.MediaType;
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.RequestBody;
+import okhttp3.Response;
+import okhttp3.ResponseBody;
+import okhttp3.logging.HttpLoggingInterceptor;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.c2.client.C2ClientConfig;
+import org.apache.nifi.c2.client.api.C2Client;
+import org.apache.nifi.c2.client.api.C2Serializer;
+import org.apache.nifi.c2.protocol.api.C2Heartbeat;
+import org.apache.nifi.c2.protocol.api.C2HeartbeatResponse;
+import org.apache.nifi.c2.protocol.api.C2OperationAck;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class C2HttpClient implements C2Client {
+
+    private static final Logger logger = LoggerFactory.getLogger(C2HttpClient.class);
+    private static final MediaType MEDIA_TYPE_APPLICATION_JSON = MediaType.parse("application/json");
+
+    private final AtomicReference<OkHttpClient> httpClientReference = new AtomicReference<>();
+    private final C2ClientConfig clientConfig;
+    private final C2Serializer serializer;
+
+    public C2HttpClient(C2ClientConfig clientConfig, C2Serializer serializer) {
+        super();
+        this.clientConfig = clientConfig;
+        this.serializer = serializer;
+        final OkHttpClient.Builder okHttpClientBuilder = new OkHttpClient.Builder();
+
+        // Configure request and response logging
+        HttpLoggingInterceptor logging = new HttpLoggingInterceptor();
+        logging.setLevel(HttpLoggingInterceptor.Level.BASIC);
+        okHttpClientBuilder.addInterceptor(logging);
+
+        // Set whether to follow redirects
+        okHttpClientBuilder.followRedirects(true);
+
+        // Timeout for calls made to the server
+        okHttpClientBuilder.callTimeout(clientConfig.getCallTimeout(), TimeUnit.MILLISECONDS);
+
+        // check if the ssl path is set and add the factory if so
+        if (StringUtils.isNotBlank(clientConfig.getKeystoreFilename())) {
+            try {
+                setSslSocketFactory(okHttpClientBuilder);
+            } catch (Exception e) {
+                throw new IllegalStateException("OkHttp TLS configuration failed", e);
+            }
+        }
+
+        httpClientReference.set(okHttpClientBuilder.build());
+    }
+
+    @Override
+    public Optional<C2HeartbeatResponse> publishHeartbeat(C2Heartbeat heartbeat) {
+        return serializer.serialize(heartbeat).flatMap(this::sendHeartbeat);
+    }
+
+    private Optional<C2HeartbeatResponse> sendHeartbeat(String heartbeat) {
+        logger.debug("Sending heartbeat to {}", clientConfig.getC2Url());
+
+        Optional<C2HeartbeatResponse> c2HeartbeatResponse = Optional.empty();
+        Request request = new Request.Builder()
+            .post(RequestBody.create(heartbeat, MEDIA_TYPE_APPLICATION_JSON))
+            .url(clientConfig.getC2Url())
+            .build();
+
+        try (Response heartbeatResponse = httpClientReference.get().newCall(request).execute()) {
+            c2HeartbeatResponse = getResponseBody(heartbeatResponse).flatMap(response -> serializer.deserialize(response, C2HeartbeatResponse.class));

Review Comment:
   do we get some feedback on malformed heartbeat responses?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] bejancsaba commented on a diff in pull request #6075: NIFI-9666 Implement base C2 client functionality

Posted by GitBox <gi...@apache.org>.
bejancsaba commented on code in PR #6075:
URL: https://github.com/apache/nifi/pull/6075#discussion_r891075399


##########
c2/c2-client-bundle/pom.xml:
##########
@@ -0,0 +1,41 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+ -->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <artifactId>c2</artifactId>
+        <groupId>org.apache.nifi</groupId>
+        <version>1.17.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>c2-client-bundle</artifactId>
+    <packaging>pom</packaging>
+
+    <modules>
+        <module>c2-client-api</module>
+        <module>c2-client-base</module>
+        <module>c2-client-http</module>
+        <module>c2-client-service</module>
+    </modules>
+
+    <dependencyManagement>
+        <dependencies>
+        </dependencies>
+    </dependencyManagement>

Review Comment:
   Removed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] bejancsaba commented on a diff in pull request #6075: NIFI-9666 Implement base C2 client functionality

Posted by GitBox <gi...@apache.org>.
bejancsaba commented on code in PR #6075:
URL: https://github.com/apache/nifi/pull/6075#discussion_r891074306


##########
minifi/pom.xml:
##########
@@ -449,10 +449,22 @@ limitations under the License.
                 <version>1.17.0-SNAPSHOT</version>
             </dependency>
 
+            <!-- C2 modules -->
+            <dependency>
+                <groupId>org.apache.nifi</groupId>
+                <artifactId>c2-client-api</artifactId>
+                <version>1.17.0-SNAPSHOT</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.nifi</groupId>
+                <artifactId>c2-client-base</artifactId>
+                <version>1.17.0-SNAPSHOT</version>
+            </dependency>
+
             <dependency>
                 <groupId>com.squareup.okhttp3</groupId>
                 <artifactId>okhttp</artifactId>
-                <version>3.12.3</version>
+                <version>${okhttp.version}</version>

Review Comment:
   My bad, I removed the version but it caused build issues and added back and forgat to comment back. So I removed the whole dependency as is comes from parent, without the version but keeping the dependency here overrides the one coming from nifi and looks for the missing version.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #6075: NIFI-9666 Implement base C2 client functionality

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on code in PR #6075:
URL: https://github.com/apache/nifi/pull/6075#discussion_r892998854


##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/util/ConfigTransformer.java:
##########
@@ -88,42 +91,68 @@ public final class ConfigTransformer {
     private ConfigTransformer() {
     }
 
+    public static ByteBuffer generateConfigFiles(InputStream configIs, String configDestinationPath, Properties bootstrapProperties) throws ConfigurationChangeException, IOException {

Review Comment:
   That sounds fine for now.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] bejancsaba commented on a diff in pull request #6075: NIFI-9666 Implement base C2 client functionality

Posted by GitBox <gi...@apache.org>.
bejancsaba commented on code in PR #6075:
URL: https://github.com/apache/nifi/pull/6075#discussion_r898863549


##########
c2/c2-client-bundle/c2-client-service/src/main/java/org/apache/nifi/c2/client/service/operation/UpdateConfigurationOperationHandler.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.nifi.c2.client.service.operation;
+
+import static org.apache.commons.lang3.StringUtils.EMPTY;
+import static org.apache.nifi.c2.protocol.api.OperandType.CONFIGURATION;
+import static org.apache.nifi.c2.protocol.api.OperationType.UPDATE;
+
+import java.net.URI;
+import java.util.Optional;
+import java.util.function.Function;
+import org.apache.nifi.c2.client.api.C2Client;
+import org.apache.nifi.c2.client.service.FlowIdHolder;
+import org.apache.nifi.c2.protocol.api.C2Operation;
+import org.apache.nifi.c2.protocol.api.C2OperationAck;
+import org.apache.nifi.c2.protocol.api.C2OperationState;
+import org.apache.nifi.c2.protocol.api.OperandType;
+import org.apache.nifi.c2.protocol.api.OperationType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class UpdateConfigurationOperationHandler implements C2OperationHandler {
+
+    private static final Logger logger = LoggerFactory.getLogger(UpdateConfigurationOperationHandler.class);
+
+    private static final String LOCATION = "location";
+
+    private final C2Client client;
+    private final Function<byte[], Boolean> updateFlow;
+    private final FlowIdHolder flowIdHolder;
+
+    public UpdateConfigurationOperationHandler(C2Client client, FlowIdHolder flowIdHolder, Function<byte[], Boolean> updateFlow) {
+        this.client = client;
+        this.updateFlow = updateFlow;
+        this.flowIdHolder = flowIdHolder;
+    }
+
+    @Override
+    public OperationType getOperationType() {
+        return UPDATE;
+    }
+
+    @Override
+    public OperandType getOperandType() {
+        return CONFIGURATION;
+    }
+
+    @Override
+    public C2OperationAck handle(C2Operation operation) {
+        String opIdentifier = Optional.ofNullable(operation.getIdentifier())
+            .orElse(EMPTY);
+        C2OperationAck operationAck = new C2OperationAck();
+        C2OperationState state = new C2OperationState();
+        operationAck.setOperationState(state);
+        operationAck.setOperationId(opIdentifier);
+
+        String updateLocation = Optional.ofNullable(operation.getArgs())
+            .map(map -> map.get(LOCATION))
+            .orElse(EMPTY);
+
+        String newFlowId = parseFlowId(updateLocation);
+        if (flowIdHolder.getFlowId() == null || !flowIdHolder.getFlowId().equals(newFlowId)) {
+            logger.info("Will perform flow update from {} for operation #{}. Previous flow id was {}, replacing with new id {}", updateLocation, opIdentifier,
+                flowIdHolder.getFlowId() == null ? "not set" : flowIdHolder.getFlowId(), newFlowId);
+        } else {
+            logger.info("Flow is current, no update is necessary...");
+        }
+
+        flowIdHolder.setFlowId(newFlowId);
+        byte[] updateContent = client.retrieveUpdateContent(updateLocation);
+        if (updateContent != null) {
+            if (updateFlow.apply(updateContent)) {
+                state.setState(C2OperationState.OperationState.FULLY_APPLIED);
+                logger.debug("Update configuration applied for operation #{}.", opIdentifier);
+            } else {
+                state.setState(C2OperationState.OperationState.NOT_APPLIED);
+                logger.error("Update resulted in error for operation #{}.", opIdentifier);
+            }
+        } else {
+            state.setState(C2OperationState.OperationState.NOT_APPLIED);
+            logger.error("Update content retrieval resulted in empty content so flow update was omitted for operation #{}.", opIdentifier);
+        }
+
+        return operationAck;
+    }
+
+    private String parseFlowId(String flowUpdateUrl) {
+        try {
+            URI flowUri = new URI(flowUpdateUrl);
+            String flowUriPath = flowUri.getPath();
+            String[] split = flowUriPath.split("/");
+            if (split.length > 4) {
+                return split[4];

Review Comment:
   Yes we are in agreement, this is our plan moving forward. As we will start to put a little more focus on the minifi-c2-service we will keep this in mind as well but that change will come later so we would keep this as is for now if that is ok.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] kevdoran commented on a diff in pull request #6075: NIFI-9666 Implement base C2 client functionality

Posted by GitBox <gi...@apache.org>.
kevdoran commented on code in PR #6075:
URL: https://github.com/apache/nifi/pull/6075#discussion_r899229301


##########
nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/c2/C2NiFiProperties.java:
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.nifi.c2;
+
+import java.util.concurrent.TimeUnit;
+
+public class C2NiFiProperties {
+
+    public static final String C2_PREFIX = "c2.";
+
+    public static final String C2_ENABLE_KEY = C2_PREFIX + "enable";
+    public static final String C2_AGENT_PROTOCOL_KEY = C2_PREFIX + "agent.protocol.class";
+    public static final String C2_COAP_HOST_KEY = C2_PREFIX + "agent.coap.host";
+    public static final String C2_COAP_PORT_KEY = C2_PREFIX + "agent.coap.port";
+    public static final String C2_CONFIG_DIRECTORY_KEY = C2_PREFIX + "config.directory";
+    public static final String C2_RUNTIME_MANIFEST_IDENTIFIER_KEY = C2_PREFIX + "runtime.manifest.identifier";
+    public static final String C2_RUNTIME_TYPE_KEY = C2_PREFIX + "runtime.type";
+    public static final String C2_REST_URL_KEY = C2_PREFIX + "rest.url";
+    public static final String C2_REST_URL_ACK_KEY = C2_PREFIX + "rest.url.ack";
+    public static final String C2_ROOT_CLASSES_KEY = C2_PREFIX + "root.classes";
+    public static final String C2_AGENT_HEARTBEAT_PERIOD_KEY = C2_PREFIX + "agent.heartbeat.period";
+    public static final String C2_CONNECTION_TIMEOUT = C2_PREFIX + "rest.connectionTimeout";
+    public static final String C2_READ_TIMEOUT = C2_PREFIX + "rest.readTimeout";
+    public static final String C2_CALL_TIMEOUT = C2_PREFIX + "rest.callTimeout";
+    public static final String C2_AGENT_CLASS_KEY = C2_PREFIX + "agent.class";
+    public static final String C2_AGENT_IDENTIFIER_KEY = C2_PREFIX + "agent.identifier";
+
+    public static final String C2_ROOT_CLASS_DEFINITIONS_KEY = C2_PREFIX + "root.class.definitions";
+    public static final String C2_METRICS_NAME_KEY = C2_ROOT_CLASS_DEFINITIONS_KEY + ".metrics.name";
+    public static final String C2_METRICS_METRICS_KEY = C2_ROOT_CLASS_DEFINITIONS_KEY + ".metrics.metrics";
+    public static final String C2_METRICS_METRICS_TYPED_METRICS_NAME_KEY = C2_ROOT_CLASS_DEFINITIONS_KEY + ".metrics.metrics.typedmetrics.name";
+    public static final String C2_METRICS_METRICS_QUEUED_METRICS_NAME_KEY = C2_ROOT_CLASS_DEFINITIONS_KEY + ".metrics.metrics.queuemetrics.name";
+    public static final String C2_METRICS_METRICS_QUEUE_METRICS_CLASSES_KEY = C2_ROOT_CLASS_DEFINITIONS_KEY + ".metrics.metrics.queuemetrics.classes";
+    public static final String C2_METRICS_METRICS_TYPED_METRICS_CLASSES_KEY = C2_ROOT_CLASS_DEFINITIONS_KEY + ".metrics.metrics.typedmetrics.classes";
+    public static final String C2_METRICS_METRICS_PROCESSOR_METRICS_NAME_KEY = C2_ROOT_CLASS_DEFINITIONS_KEY + ".metrics.metrics.processorMetrics.name";
+    public static final String C2_METRICS_METRICS_PROCESSOR_METRICS_CLASSES_KEY = C2_ROOT_CLASS_DEFINITIONS_KEY + ".metrics.metrics.processorMetrics.classes";
+
+    /* C2 Client Security Properties */
+    private static final String C2_REST_SECURITY_BASE_KEY = C2_PREFIX + "security";
+    public static final String TRUSTSTORE_LOCATION_KEY = C2_REST_SECURITY_BASE_KEY + ".truststore.location";
+    public static final String TRUSTSTORE_PASSWORD_KEY = C2_REST_SECURITY_BASE_KEY + ".truststore.password";
+    public static final String TRUSTSTORE_TYPE_KEY = C2_REST_SECURITY_BASE_KEY + ".truststore.type";
+    public static final String KEYSTORE_LOCATION_KEY = C2_REST_SECURITY_BASE_KEY + ".keystore.location";
+    public static final String KEYSTORE_PASSWORD_KEY = C2_REST_SECURITY_BASE_KEY + ".keystore.password";
+    public static final String KEYSTORE_TYPE_KEY = C2_REST_SECURITY_BASE_KEY + ".keystore.type";
+    public static final String NEED_CLIENT_AUTH_KEY = C2_REST_SECURITY_BASE_KEY + ".need.client.auth";

Review Comment:
   Agreed I don't think this is needed, same as above



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] bejancsaba commented on a diff in pull request #6075: NIFI-9666 Implement base C2 client functionality

Posted by GitBox <gi...@apache.org>.
bejancsaba commented on code in PR #6075:
URL: https://github.com/apache/nifi/pull/6075#discussion_r899968756


##########
minifi/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/service/MiNiFiConfigurationChangeListener.java:
##########
@@ -0,0 +1,208 @@
+/*
+ * 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.nifi.minifi.bootstrap.service;
+
+import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CONF_DIR_KEY;
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.MINIFI_CONFIG_FILE_KEY;
+import static org.apache.nifi.minifi.bootstrap.configuration.ingestors.PullHttpChangeIngestor.OVERRIDE_SECURITY;
+import static org.apache.nifi.minifi.bootstrap.configuration.ingestors.PullHttpChangeIngestor.PULL_HTTP_BASE_KEY;
+import static org.apache.nifi.minifi.bootstrap.util.ConfigTransformer.generateConfigFiles;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.util.Properties;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.commons.io.IOUtils;
+import org.apache.nifi.minifi.bootstrap.RunMiNiFi;
+import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeException;
+import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeListener;
+import org.apache.nifi.minifi.bootstrap.util.ByteBufferInputStream;
+import org.apache.nifi.minifi.bootstrap.util.ConfigTransformer;
+import org.apache.nifi.minifi.commons.schema.ConfigSchema;
+import org.apache.nifi.minifi.commons.schema.common.ConvertableSchema;
+import org.apache.nifi.minifi.commons.schema.serialization.SchemaLoader;
+import org.slf4j.Logger;
+
+public class MiNiFiConfigurationChangeListener implements ConfigurationChangeListener {
+
+    private final RunMiNiFi runner;
+    private final Logger logger;
+    private final BootstrapFileProvider bootstrapFileProvider;
+
+    private static final ReentrantLock handlingLock = new ReentrantLock();
+
+    public MiNiFiConfigurationChangeListener(RunMiNiFi runner, Logger logger, BootstrapFileProvider bootstrapFileProvider) {
+        this.runner = runner;
+        this.logger = logger;
+        this.bootstrapFileProvider = bootstrapFileProvider;
+    }
+
+    @Override
+    public void handleChange(InputStream configInputStream) throws ConfigurationChangeException {
+        logger.info("Received notification of a change");
+
+        if (!handlingLock.tryLock()) {
+            throw new ConfigurationChangeException("Instance is already handling another change");
+        }
+        // Store the incoming stream as a byte array to be shared among components that need it
+        try(ByteArrayOutputStream bufferedConfigOs = new ByteArrayOutputStream()) {
+
+            Properties bootstrapProperties = bootstrapFileProvider.getBootstrapProperties();
+            File configFile = new File(bootstrapProperties.getProperty(MINIFI_CONFIG_FILE_KEY));
+
+            byte[] copyArray = new byte[1024];
+            int available;
+            while ((available = configInputStream.read(copyArray)) > 0) {
+                bufferedConfigOs.write(copyArray, 0, available);
+            }

Review Comment:
   You are right, replaced



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org