mirror of https://github.com/apache/nifi.git
NIFI-9666 Implemented MiNiFi C2 client and refactored bootstrap
- Implemented MiNiFi C2 client using OkHttp - Refactored MiNiFi bootstrap command handling and socket communication - Added C2 Client Service to nifi-framework-core This closes #6075 Co-authored-by: Matthew Burgess <mattyb149@apache.org> Co-authored-by: Csaba Bejan <bejan.csaba@gmail.com> Co-authored-by: Ferenc Erdei <ferdei@cloudera.com> Signed-off-by: David Handermann <exceptionfactory@apache.org>
This commit is contained in:
parent
bd45eb4995
commit
d89f98f5b4
|
@ -0,0 +1,61 @@
|
|||
<!--
|
||||
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.
|
||||
-->
|
||||
## Apache NiFi MiNiFi Command and Control (C2) Client
|
||||
The c2-client-bundle provides implementation for the client aspect of the [C2 Protocol](https://cwiki.apache.org/confluence/display/MINIFI/C2+Design). The essence of the implementation is the heartbeat construction and the communication with the [C2 server](../../../../minifi-c2/README.md) via the C2HttpClient.
|
||||
|
||||
Currently, relying on the [C2 Protocol API](../c2-protocol) is limited to sending heartbeats and processing/acknowledging UPDATE configuration operation in the response (if any). When exposed the new configuration will be downloaded and passed back to the system using the C2 protocol.
|
||||
|
||||
When C2 is enabled, C2ClientService will be scheduled to send heartbeats periodically, so the C2 Server can notify the client about any operations that is defined by the protocol and needs to be executed on the client side.
|
||||
|
||||
Using the client means that configuration changes and other operations can be triggered and controlled centrally via the C2 server making the management of clients more simple and configuring them more flexible. The client supports bidirectional TLS authentication.
|
||||
|
||||
### Configuration
|
||||
To use the client, the parameters coming from `C2ClientConfig` need to be properly set (this configuration class is also used for instantiating `C2HeartbeatFactory` and `C2HttpClient`)
|
||||
|
||||
```
|
||||
# The C2 Server endpoint where the heartbeat is sent
|
||||
private final String c2Url;
|
||||
|
||||
# The C2 Server endpoint where the acknowledge is sent
|
||||
private final String c2AckUrl;
|
||||
|
||||
# The class the agent belongs to (flow definition is tied to agent class on the server side)
|
||||
private final String agentClass;
|
||||
|
||||
# Unique identifier for the agent if not provided it will be generated
|
||||
private final String agentIdentifier;
|
||||
|
||||
# Directory where persistent configuration (e.g.: generated agent and device id will be persisted)
|
||||
private final String confDirectory;
|
||||
|
||||
# Property of RuntimeManifest defined in c2-protocol. A unique identifier for the manifest
|
||||
private final String runtimeManifestIdentifier;
|
||||
|
||||
# Property of RuntimeManifest defined in c2-protocol. The type of the runtime binary. Usually set when the runtime is built
|
||||
private final String runtimeType;
|
||||
|
||||
# The frequency of sending the heartbeats. This property is used by the c2-client-bundle user who should schedule the client
|
||||
private final Long heartbeatPeriod;
|
||||
|
||||
# Security properties for communication with the C2 Server
|
||||
private final String keystoreFilename;
|
||||
private final String keystorePass;
|
||||
private final String keyPass;
|
||||
private final KeystoreType keystoreType;
|
||||
private final String truststoreFilename;
|
||||
private final String truststorePass;
|
||||
private final KeystoreType truststoreType;
|
||||
private final HostnameVerifier hostnameVerifier;
|
||||
```
|
|
@ -0,0 +1,37 @@
|
|||
<?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-api</artifactId>
|
||||
<packaging>jar</packaging>
|
||||
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>c2-protocol-api</artifactId>
|
||||
<version>1.17.0-SNAPSHOT</version>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
</project>
|
|
@ -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 empty if no content can be downloaded
|
||||
*/
|
||||
Optional<byte[]> retrieveUpdateContent(String flowUpdateUrl);
|
||||
|
||||
/**
|
||||
* After operation completed the acknowledge to be sent to the C2 Server
|
||||
*
|
||||
* @param operationAck the acknowledge details to be sent
|
||||
*/
|
||||
void acknowledgeOperation(C2OperationAck operationAck);
|
||||
}
|
|
@ -0,0 +1,44 @@
|
|||
/*
|
||||
* 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;
|
||||
|
||||
/**
|
||||
* Helper class to support central configuration and functionality for serialisation / deserialisation
|
||||
*/
|
||||
public interface C2Serializer {
|
||||
|
||||
/**
|
||||
* Helper to serialise object
|
||||
*
|
||||
* @param content object to be serialised
|
||||
* @param <T> the type of the object
|
||||
* @return the serialised string representation of the parameter object if it was successful empty otherwise
|
||||
*/
|
||||
<T> Optional<String> serialize(T content);
|
||||
|
||||
/**
|
||||
* Helper to deserialise an object
|
||||
*
|
||||
* @param content the string representation of the object to be deserialsed
|
||||
* @param valueType the class of the target object
|
||||
* @param <T> the type of the target object
|
||||
* @return the deserialised object if successful empty otherwise
|
||||
*/
|
||||
<T> Optional<T> deserialize(String content, Class<T> valueType);
|
||||
}
|
|
@ -15,12 +15,20 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.nifi.minifi.bootstrap;
|
||||
package org.apache.nifi.c2.client.api;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
|
||||
/**
|
||||
* Should be implemented by the class which bootstraps the agent.
|
||||
*/
|
||||
public interface ConfigurationFileHolder {
|
||||
|
||||
/**
|
||||
* Retrieve the reference to the config file
|
||||
*
|
||||
* @return config file reference
|
||||
*/
|
||||
AtomicReference<ByteBuffer> getConfigFileReference();
|
||||
}
|
|
@ -0,0 +1,46 @@
|
|||
/*
|
||||
* 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.io.IOException;
|
||||
import java.util.Properties;
|
||||
|
||||
/**
|
||||
* Helper to support differentiating between config files to recognise changes
|
||||
*
|
||||
* @param <T> the type of the config files
|
||||
*/
|
||||
public interface Differentiator <T> {
|
||||
|
||||
/**
|
||||
* Initialise the differentiator with the initial configuration
|
||||
*
|
||||
* @param properties the properties to be used
|
||||
* @param configurationFileHolder holder for the config file
|
||||
*/
|
||||
void initialize(Properties properties, ConfigurationFileHolder configurationFileHolder);
|
||||
|
||||
/**
|
||||
* Determine whether the config file changed
|
||||
*
|
||||
* @param input the conetnt of the new config file
|
||||
* @return true if changed and false if not
|
||||
* @throws IOException when there is a config file reading related error
|
||||
*/
|
||||
boolean isNew(T input) throws IOException;
|
||||
}
|
|
@ -0,0 +1,30 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.c2.client.api;
|
||||
|
||||
/**
|
||||
* Id generator to be used in case user is not providing an optional id
|
||||
*/
|
||||
public interface IdGenerator {
|
||||
|
||||
/**
|
||||
* Generate a random id
|
||||
*
|
||||
* @return the generated id
|
||||
*/
|
||||
String generate();
|
||||
}
|
|
@ -0,0 +1,55 @@
|
|||
<?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.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>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
</project>
|
|
@ -0,0 +1,255 @@
|
|||
/*
|
||||
* 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;
|
||||
|
||||
/**
|
||||
* 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 long callTimeout;
|
||||
private final long readTimeout;
|
||||
private final long connectTimeout;
|
||||
|
||||
|
||||
private C2ClientConfig(final Builder builder) {
|
||||
this.c2Url = builder.c2Url;
|
||||
this.c2AckUrl = builder.c2AckUrl;
|
||||
this.agentClass = builder.agentClass;
|
||||
this.agentIdentifier = builder.agentIdentifier;
|
||||
this.confDirectory = builder.confDirectory;
|
||||
this.runtimeManifestIdentifier = builder.runtimeManifestIdentifier;
|
||||
this.runtimeType = builder.runtimeType;
|
||||
this.heartbeatPeriod = builder.heartbeatPeriod;
|
||||
this.callTimeout = builder.callTimeout;
|
||||
this.keystoreFilename = builder.keystoreFilename;
|
||||
this.keystorePass = builder.keystorePass;
|
||||
this.keyPass = builder.keyPass;
|
||||
this.keystoreType = builder.keystoreType;
|
||||
this.truststoreFilename = builder.truststoreFilename;
|
||||
this.truststorePass = builder.truststorePass;
|
||||
this.truststoreType = builder.truststoreType;
|
||||
this.readTimeout = builder.readTimeout;
|
||||
this.connectTimeout = builder.connectTimeout;
|
||||
}
|
||||
|
||||
public String getC2Url() {
|
||||
return c2Url;
|
||||
}
|
||||
|
||||
public String getC2AckUrl() {
|
||||
return c2AckUrl;
|
||||
}
|
||||
|
||||
public String getAgentClass() {
|
||||
return agentClass;
|
||||
}
|
||||
|
||||
public String getAgentIdentifier() {
|
||||
return agentIdentifier;
|
||||
}
|
||||
|
||||
public String getConfDirectory() {
|
||||
return confDirectory;
|
||||
}
|
||||
|
||||
public String getRuntimeManifestIdentifier() {
|
||||
return runtimeManifestIdentifier;
|
||||
}
|
||||
|
||||
public String getRuntimeType() {
|
||||
return runtimeType;
|
||||
}
|
||||
|
||||
public long getHeartbeatPeriod() {
|
||||
return heartbeatPeriod;
|
||||
}
|
||||
|
||||
public long getCallTimeout() {
|
||||
return callTimeout;
|
||||
}
|
||||
|
||||
public String getKeystoreFilename() {
|
||||
return keystoreFilename;
|
||||
}
|
||||
|
||||
public String getKeystorePass() {
|
||||
return keystorePass;
|
||||
}
|
||||
|
||||
public String getKeyPass() {
|
||||
return keyPass;
|
||||
}
|
||||
|
||||
public String getKeystoreType() {
|
||||
return keystoreType;
|
||||
}
|
||||
|
||||
public String getTruststoreFilename() {
|
||||
return truststoreFilename;
|
||||
}
|
||||
|
||||
public String getTruststorePass() {
|
||||
return truststorePass;
|
||||
}
|
||||
|
||||
public String getTruststoreType() {
|
||||
return truststoreType;
|
||||
}
|
||||
|
||||
public long getReadTimeout() {
|
||||
return readTimeout;
|
||||
}
|
||||
|
||||
public long getConnectTimeout() {
|
||||
return connectTimeout;
|
||||
}
|
||||
|
||||
/**
|
||||
* Builder for client configuration.
|
||||
*/
|
||||
public static class Builder {
|
||||
|
||||
private String c2Url;
|
||||
private String c2AckUrl;
|
||||
private String agentClass;
|
||||
private String agentIdentifier;
|
||||
private String confDirectory;
|
||||
private String runtimeManifestIdentifier;
|
||||
private String runtimeType;
|
||||
private long heartbeatPeriod;
|
||||
private long callTimeout;
|
||||
private String keystoreFilename;
|
||||
private String keystorePass;
|
||||
private String keyPass;
|
||||
private String keystoreType;
|
||||
private String truststoreFilename;
|
||||
private String truststorePass;
|
||||
private String truststoreType;
|
||||
private long readTimeout;
|
||||
private long connectTimeout;
|
||||
|
||||
public Builder c2Url(final String c2Url) {
|
||||
this.c2Url = c2Url;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder c2AckUrl(final String c2AckUrl) {
|
||||
this.c2AckUrl = c2AckUrl;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder agentClass(final String agentClass) {
|
||||
this.agentClass = agentClass;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder agentIdentifier(final String agentIdentifier) {
|
||||
this.agentIdentifier = agentIdentifier;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder confDirectory(final String confDirectory) {
|
||||
this.confDirectory = confDirectory;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder runtimeManifestIdentifier(final String runtimeManifestIdentifier) {
|
||||
this.runtimeManifestIdentifier = runtimeManifestIdentifier;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder runtimeType(final String runtimeType) {
|
||||
this.runtimeType = runtimeType;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder heartbeatPeriod(final long heartbeatPeriod) {
|
||||
this.heartbeatPeriod = heartbeatPeriod;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder callTimeout(final long callTimeout) {
|
||||
this.callTimeout = callTimeout;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder keystoreFilename(final String keystoreFilename) {
|
||||
this.keystoreFilename = keystoreFilename;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder keystorePassword(final String keystorePass) {
|
||||
this.keystorePass = keystorePass;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder keyPassword(final String keyPass) {
|
||||
this.keyPass = keyPass;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder keystoreType(final String keystoreType) {
|
||||
this.keystoreType = keystoreType;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder truststoreFilename(final String truststoreFilename) {
|
||||
this.truststoreFilename = truststoreFilename;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder truststorePassword(final String truststorePass) {
|
||||
this.truststorePass = truststorePass;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder truststoreType(final String truststoreType) {
|
||||
this.truststoreType = truststoreType;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder readTimeout(final long readTimeout) {
|
||||
this.readTimeout = readTimeout;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder connectTimeout(final long connectTimeout) {
|
||||
this.connectTimeout = connectTimeout;
|
||||
return this;
|
||||
}
|
||||
|
||||
public C2ClientConfig build() {
|
||||
return new C2ClientConfig(this);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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.c2.util.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.debug("Created identifier {} at {}", uuid, persistenceLocation);
|
||||
return uuid;
|
||||
} catch (IOException e) {
|
||||
throw new IllegalStateException(String.format("Could not create file %s as persistence file.", persistenceLocation), e);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,74 @@
|
|||
/*
|
||||
* 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);
|
||||
} catch (JsonProcessingException e) {
|
||||
logger.error("Object serialization to JSON failed", 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("Object deserialization from JSON failed", e);
|
||||
}
|
||||
|
||||
return Optional.ofNullable(responseObject);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,38 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.nifi.c2.util;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
|
||||
public class FileUtils {
|
||||
|
||||
public static void ensureDirectoryExistAndCanAccess(final File dir) throws IOException {
|
||||
if (dir.exists() && !dir.isDirectory()) {
|
||||
throw new IOException(dir.getAbsolutePath() + " is not a directory");
|
||||
} else if (!dir.exists()) {
|
||||
final boolean made = dir.mkdirs();
|
||||
if (!made) {
|
||||
throw new IOException(dir.getAbsolutePath() + " could not be created");
|
||||
}
|
||||
}
|
||||
if (!(dir.canRead() && dir.canWrite())) {
|
||||
throw new IOException(dir.getAbsolutePath() + " directory does not have read/write privilege");
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,51 @@
|
|||
<?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>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.squareup.okhttp3</groupId>
|
||||
<artifactId>logging-interceptor</artifactId>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
</project>
|
|
@ -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.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(logger::debug);
|
||||
logging.setLevel(HttpLoggingInterceptor.Level.BASIC);
|
||||
okHttpClientBuilder.addInterceptor(logging);
|
||||
|
||||
// Set whether to follow redirects
|
||||
okHttpClientBuilder.followRedirects(true);
|
||||
|
||||
// Timeouts
|
||||
okHttpClientBuilder.connectTimeout(clientConfig.getConnectTimeout(), TimeUnit.MILLISECONDS);
|
||||
okHttpClientBuilder.readTimeout(clientConfig.getReadTimeout(), TimeUnit.MILLISECONDS);
|
||||
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) {
|
||||
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 Optional<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()) {
|
||||
int code = response.code();
|
||||
if (code >= 400) {
|
||||
final String message = String.format("Configuration retrieval failed: HTTP %d %s", code, response.body().string());
|
||||
throw new IOException(message);
|
||||
}
|
||||
|
||||
body = response.body();
|
||||
|
||||
if (body == null) {
|
||||
logger.warn("No body returned when pulling a new configuration");
|
||||
return Optional.empty();
|
||||
}
|
||||
|
||||
return Optional.of(body.bytes());
|
||||
} catch (Exception e) {
|
||||
logger.warn("Configuration retrieval failed", e);
|
||||
return Optional.empty();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void acknowledgeOperation(C2OperationAck operationAck) {
|
||||
logger.info("Performing acknowledgement request to {} for operation {}", clientConfig.getC2AckUrl(), operationAck.getOperationId());
|
||||
serializer.serialize(operationAck)
|
||||
.map(operationAckBody -> RequestBody.create(operationAckBody, MEDIA_TYPE_APPLICATION_JSON))
|
||||
.map(requestBody -> new Request.Builder().post(requestBody).url(clientConfig.getC2AckUrl()).build())
|
||||
.ifPresent(this::sendAck);
|
||||
}
|
||||
|
||||
private void sendAck(Request request) {
|
||||
try(Response heartbeatResponse = httpClientReference.get().newCall(request).execute()) {
|
||||
if (!heartbeatResponse.isSuccessful()) {
|
||||
logger.warn("Acknowledgement was not successful with c2 server [{}] with status code {}", clientConfig.getC2AckUrl(), heartbeatResponse.code());
|
||||
}
|
||||
} catch (IOException e) {
|
||||
logger.error("Could not transmit ack to c2 server [{}]", clientConfig.getC2AckUrl(), e);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,49 @@
|
|||
<?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-service</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>org.apache.nifi</groupId>
|
||||
<artifactId>c2-client-http</artifactId>
|
||||
<version>1.17.0-SNAPSHOT</version>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
|
||||
</project>
|
|
@ -0,0 +1,70 @@
|
|||
/*
|
||||
* 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.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<byte[], 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) {
|
||||
C2Heartbeat c2Heartbeat = c2HeartbeatFactory.create(runtimeInfoWrapper);
|
||||
client.publishHeartbeat(c2Heartbeat).ifPresent(this::processResponse);
|
||||
}
|
||||
|
||||
private void processResponse(C2HeartbeatResponse response) {
|
||||
List<C2Operation> requestedOperations = response.getRequestedOperations();
|
||||
if (requestedOperations != null && !requestedOperations.isEmpty()) {
|
||||
logger.info("Received {} operations from the C2 server", requestedOperations.size());
|
||||
handleRequestedOperations(requestedOperations);
|
||||
} else {
|
||||
logger.trace("No operations received from the C2 server in the server. Nothing to do.");
|
||||
}
|
||||
}
|
||||
|
||||
private void handleRequestedOperations(List<C2Operation> requestedOperations) {
|
||||
for (C2Operation requestedOperation : requestedOperations) {
|
||||
operationService.handleOperation(requestedOperation)
|
||||
.ifPresent(client::acknowledgeOperation);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,227 @@
|
|||
/*
|
||||
* 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;
|
||||
}
|
||||
|
||||
private FlowInfo getFlowInfo(Map<String, FlowQueueStatus> queueStatus) {
|
||||
FlowInfo flowInfo = new FlowInfo();
|
||||
flowInfo.setQueues(queueStatus);
|
||||
Optional.ofNullable(flowIdHolder.getFlowId()).ifPresent(flowInfo::setFlowId);
|
||||
return flowInfo;
|
||||
}
|
||||
|
||||
private AgentInfo getAgentInfo(AgentRepositories repos, RuntimeManifest manifest) {
|
||||
AgentInfo agentInfo = new AgentInfo();
|
||||
agentInfo.setAgentClass(clientConfig.getAgentClass());
|
||||
agentInfo.setIdentifier(getAgentId());
|
||||
|
||||
AgentStatus agentStatus = new AgentStatus();
|
||||
agentStatus.setUptime(ManagementFactory.getRuntimeMXBean().getUptime());
|
||||
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;
|
||||
}
|
||||
|
||||
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();
|
||||
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("Network Interface processing failed", 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));
|
||||
}
|
||||
}
|
||||
deviceId = macBuilder.toString();
|
||||
} catch (Exception e) {
|
||||
logger.warn("Could not determine device identifier. Generating a unique ID", e);
|
||||
deviceId = getConfiguredDeviceId();
|
||||
}
|
||||
} else {
|
||||
deviceId = getConfiguredDeviceId();
|
||||
}
|
||||
}
|
||||
|
||||
return deviceId;
|
||||
}
|
||||
|
||||
private String getConfiguredDeviceId() {
|
||||
File idFile = new File(getConfDirectory(), DEVICE_IDENTIFIER_FILENAME);
|
||||
return new PersistentUuidGenerator(idFile).generate();
|
||||
}
|
||||
|
||||
private SystemInfo generateSystemInfo() {
|
||||
SystemInfo systemInfo = new SystemInfo();
|
||||
systemInfo.setPhysicalMem(Runtime.getRuntime().maxMemory());
|
||||
systemInfo.setMemoryUsage(Runtime.getRuntime().maxMemory() - Runtime.getRuntime().freeMemory());
|
||||
systemInfo.setvCores(Runtime.getRuntime().availableProcessors());
|
||||
|
||||
OperatingSystemMXBean osMXBean = ManagementFactory.getOperatingSystemMXBean();
|
||||
systemInfo.setMachineArch(osMXBean.getArch());
|
||||
systemInfo.setOperatingSystem(osMXBean.getName());
|
||||
systemInfo.setCpuUtilization(osMXBean.getSystemLoadAverage() / (double) osMXBean.getAvailableProcessors());
|
||||
|
||||
return systemInfo;
|
||||
}
|
||||
|
||||
private File getConfDirectory() {
|
||||
if (confDirectory == null) {
|
||||
String configDirectoryName = clientConfig.getConfDirectory();
|
||||
File configDirectory = new File(configDirectoryName);
|
||||
if (!configDirectory.exists() || !configDirectory.isDirectory()) {
|
||||
throw new IllegalStateException("Specified conf directory " + configDirectoryName + " does not exist or is not a directory.");
|
||||
}
|
||||
|
||||
confDirectory = configDirectory;
|
||||
}
|
||||
|
||||
return confDirectory;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,84 @@
|
|||
/*
|
||||
* 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.c2.util.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("Persisting Flow [{}] failed", flowId, e);
|
||||
}
|
||||
}
|
||||
|
||||
private void saveFlowId(File flowUpdateInfoFile, String flowId) {
|
||||
try {
|
||||
Files.write(flowUpdateInfoFile.toPath(), singletonList(flowId));
|
||||
} catch (IOException e) {
|
||||
LOGGER.error("Writing Flow [{}] failed", flowId, e);
|
||||
}
|
||||
}
|
||||
|
||||
private String readFlowId() {
|
||||
File flowUpdateInfoFile = new File(configDirectoryName, FLOW_IDENTIFIER_FILENAME);
|
||||
String flowId = null;
|
||||
if (flowUpdateInfoFile.exists()) {
|
||||
try {
|
||||
List<String> fileLines = Files.readAllLines(flowUpdateInfoFile.toPath());
|
||||
if (fileLines.size() != 1) {
|
||||
throw new IllegalStateException(String.format("The file %s for the persisted flow id has the incorrect format.", flowUpdateInfoFile));
|
||||
}
|
||||
flowId = fileLines.get(0);
|
||||
} catch (IOException e) {
|
||||
throw new IllegalStateException(String.format("Could not read file %s for persisted flow id.", flowUpdateInfoFile), e);
|
||||
}
|
||||
}
|
||||
return flowId;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,46 @@
|
|||
/*
|
||||
* 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.model;
|
||||
|
||||
import java.util.Map;
|
||||
import org.apache.nifi.c2.protocol.api.AgentRepositories;
|
||||
import org.apache.nifi.c2.protocol.api.FlowQueueStatus;
|
||||
import org.apache.nifi.c2.protocol.component.api.RuntimeManifest;
|
||||
|
||||
public class RuntimeInfoWrapper {
|
||||
final AgentRepositories repos;
|
||||
final RuntimeManifest manifest;
|
||||
final Map<String, FlowQueueStatus> queueStatus;
|
||||
|
||||
public RuntimeInfoWrapper(AgentRepositories repos, RuntimeManifest manifest, Map<String, FlowQueueStatus> queueStatus) {
|
||||
this.repos = repos;
|
||||
this.manifest = manifest;
|
||||
this.queueStatus = queueStatus;
|
||||
}
|
||||
|
||||
public AgentRepositories getAgentRepositories() {
|
||||
return repos;
|
||||
}
|
||||
|
||||
public RuntimeManifest getManifest() {
|
||||
return manifest;
|
||||
}
|
||||
|
||||
public Map<String, FlowQueueStatus> getQueueStatus() {
|
||||
return queueStatus;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,50 @@
|
|||
/*
|
||||
* 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;
|
||||
|
||||
/**
|
||||
* Handler interface for the different operation types
|
||||
*/
|
||||
public interface C2OperationHandler {
|
||||
|
||||
/**
|
||||
* Returns the supported OperationType by the handler
|
||||
*
|
||||
* @return the type of the operation
|
||||
*/
|
||||
OperationType getOperationType();
|
||||
|
||||
/**
|
||||
* Returns the supported OperandType by the handler
|
||||
*
|
||||
* @return the type of the operand
|
||||
*/
|
||||
OperandType getOperandType();
|
||||
|
||||
/**
|
||||
* Handler logic for the specific C2Operation
|
||||
*
|
||||
* @param operation the C2Operation to be handled
|
||||
* @return the result of the operation handling
|
||||
*/
|
||||
C2OperationAck handle(C2Operation operation);
|
||||
}
|
|
@ -0,0 +1,47 @@
|
|||
/*
|
||||
* 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 java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
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 class C2OperationService {
|
||||
|
||||
private final Map<OperationType, Map<OperandType, C2OperationHandler>> handlerMap = new HashMap<>();
|
||||
|
||||
public C2OperationService(List<C2OperationHandler> handlers) {
|
||||
for (C2OperationHandler handler : handlers) {
|
||||
handlerMap.computeIfAbsent(handler.getOperationType(), x -> new HashMap<>()).put(handler.getOperandType(), handler);
|
||||
}
|
||||
}
|
||||
|
||||
public Optional<C2OperationAck> handleOperation(C2Operation operation) {
|
||||
return getHandlerForOperation(operation)
|
||||
.map(handler -> handler.handle(operation));
|
||||
}
|
||||
|
||||
private Optional<C2OperationHandler> getHandlerForOperation(C2Operation operation) {
|
||||
return Optional.ofNullable(handlerMap.get(operation.getOperation()))
|
||||
.map(operandMap -> operandMap.get(operation.getOperand()));
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
Optional<byte[]> updateContent = client.retrieveUpdateContent(updateLocation);
|
||||
if (updateContent.isPresent()) {
|
||||
if (updateFlow.apply(updateContent.get())) {
|
||||
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 IllegalArgumentException(String.format("Flow Update URL format unexpected [%s]", flowUpdateUrl));
|
||||
}
|
||||
} catch (Exception e) {
|
||||
throw new IllegalStateException("Could not get flow id from the provided URL", e);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,36 @@
|
|||
<?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>
|
||||
</project>
|
|
@ -33,5 +33,9 @@ limitations under the License.
|
|||
<artifactId>c2-protocol-component-api</artifactId>
|
||||
<version>1.17.0-SNAPSHOT</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.commons</groupId>
|
||||
<artifactId>commons-lang3</artifactId>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
</project>
|
||||
|
|
|
@ -33,5 +33,10 @@ public enum OperandType {
|
|||
.filter(operandType -> operandType.name().equalsIgnoreCase(value))
|
||||
.findAny();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return super.toString().toLowerCase();
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -29,6 +29,7 @@ limitations under the License.
|
|||
|
||||
<modules>
|
||||
<module>c2-protocol</module>
|
||||
<module>c2-client-bundle</module>
|
||||
</modules>
|
||||
|
||||
<dependencyManagement>
|
||||
|
|
|
@ -122,6 +122,10 @@ limitations under the License.
|
|||
<groupId>org.apache.nifi.minifi</groupId>
|
||||
<artifactId>minifi-framework-api</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>c2-client-api</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi.minifi</groupId>
|
||||
<artifactId>minifi-framework-nar</artifactId>
|
||||
|
@ -210,6 +214,14 @@ limitations under the License.
|
|||
<groupId>commons-io</groupId>
|
||||
<artifactId>commons-io</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.fasterxml.jackson.core</groupId>
|
||||
<artifactId>jackson-core</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.fasterxml.jackson.core</groupId>
|
||||
<artifactId>jackson-databind</artifactId>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
|
||||
<properties>
|
||||
|
|
|
@ -40,6 +40,11 @@ limitations under the License.
|
|||
<artifactId>nifi-api</artifactId>
|
||||
<scope>compile</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>c2-client-api</artifactId>
|
||||
<scope>compile</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-bootstrap-utils</artifactId>
|
||||
|
@ -95,6 +100,15 @@ limitations under the License.
|
|||
<artifactId>commons-io</artifactId>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>com.fasterxml.jackson.core</groupId>
|
||||
<artifactId>jackson-core</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.fasterxml.jackson.core</groupId>
|
||||
<artifactId>jackson-databind</artifactId>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
|
||||
|
||||
|
|
|
@ -1,122 +0,0 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.minifi.bootstrap;
|
||||
|
||||
import org.apache.nifi.minifi.bootstrap.exception.InvalidCommandException;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
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;
|
||||
|
||||
public class BootstrapCodec {
|
||||
|
||||
private final RunMiNiFi runner;
|
||||
private final BufferedReader reader;
|
||||
private final BufferedWriter writer;
|
||||
private final Logger logger = LoggerFactory.getLogger(BootstrapCodec.class);
|
||||
|
||||
public BootstrapCodec(final RunMiNiFi runner, final InputStream in, final OutputStream out) {
|
||||
this.runner = runner;
|
||||
this.reader = new BufferedReader(new InputStreamReader(in));
|
||||
this.writer = new BufferedWriter(new OutputStreamWriter(out));
|
||||
}
|
||||
|
||||
public void communicate() throws IOException {
|
||||
final String line = reader.readLine();
|
||||
final String[] splits = line.split(" ");
|
||||
if (splits.length < 0) {
|
||||
throw new IOException("Received invalid command from MiNiFi: " + line);
|
||||
}
|
||||
|
||||
final String cmd = splits[0];
|
||||
final String[] args;
|
||||
if (splits.length == 1) {
|
||||
args = new String[0];
|
||||
} else {
|
||||
args = Arrays.copyOfRange(splits, 1, splits.length);
|
||||
}
|
||||
|
||||
try {
|
||||
processRequest(cmd, args);
|
||||
} catch (final InvalidCommandException ice) {
|
||||
throw new IOException("Received invalid command from MiNiFi: " + line + " : " + (ice.getMessage() == null ? "" : "Details: " + ice.toString()));
|
||||
}
|
||||
}
|
||||
|
||||
private void processRequest(final String cmd, final String[] args) throws InvalidCommandException, IOException {
|
||||
switch (cmd) {
|
||||
case "PORT": {
|
||||
logger.debug("Received 'PORT' command from MINIFI");
|
||||
if (args.length != 2) {
|
||||
throw new InvalidCommandException();
|
||||
}
|
||||
|
||||
final int port;
|
||||
try {
|
||||
port = Integer.parseInt(args[0]);
|
||||
} catch (final NumberFormatException nfe) {
|
||||
throw new InvalidCommandException("Invalid Port number; should be integer between 1 and 65535");
|
||||
}
|
||||
|
||||
if (port < 1 || port > 65535) {
|
||||
throw new InvalidCommandException("Invalid Port number; should be integer between 1 and 65535");
|
||||
}
|
||||
|
||||
final String secretKey = args[1];
|
||||
|
||||
runner.setMiNiFiCommandControlPort(port, secretKey);
|
||||
writer.write("OK");
|
||||
writer.newLine();
|
||||
writer.flush();
|
||||
}
|
||||
break;
|
||||
case "STARTED": {
|
||||
logger.debug("Received 'STARTED' command from MINIFI");
|
||||
if (args.length != 1) {
|
||||
throw new InvalidCommandException("STARTED command must contain a status argument");
|
||||
}
|
||||
|
||||
if (!"true".equals(args[0]) && !"false".equals(args[0])) {
|
||||
throw new InvalidCommandException("Invalid status for STARTED command; should be true or false, but was '" + args[0] + "'");
|
||||
}
|
||||
|
||||
final boolean started = Boolean.parseBoolean(args[0]);
|
||||
runner.setNiFiStarted(started);
|
||||
writer.write("OK");
|
||||
writer.newLine();
|
||||
writer.flush();
|
||||
}
|
||||
break;
|
||||
case "SHUTDOWN": {
|
||||
logger.debug("Received 'SHUTDOWN' command from MINIFI");
|
||||
runner.shutdownChangeNotifier();
|
||||
runner.shutdownPeriodicStatusReporters();
|
||||
writer.write("OK");
|
||||
writer.newLine();
|
||||
writer.flush();
|
||||
}
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,27 @@
|
|||
/*
|
||||
* 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;
|
||||
|
||||
import java.util.Optional;
|
||||
|
||||
public enum BootstrapCommand {
|
||||
START, RUN, STOP, STATUS, DUMP, RESTART, ENV, FLOWSTATUS, UNKNOWN;
|
||||
|
||||
public static Optional<BootstrapCommand> fromString(String val) {
|
||||
return Optional.ofNullable(val).map(String::toUpperCase).map(BootstrapCommand::valueOf).filter(command -> command != UNKNOWN);
|
||||
}
|
||||
}
|
|
@ -1,141 +0,0 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.minifi.bootstrap;
|
||||
|
||||
import org.apache.nifi.minifi.bootstrap.util.LimitingInputStream;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.net.ServerSocket;
|
||||
import java.net.Socket;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.ThreadFactory;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
public class MiNiFiListener {
|
||||
|
||||
private ServerSocket serverSocket;
|
||||
private volatile Listener listener;
|
||||
|
||||
int start(final RunMiNiFi runner) throws IOException {
|
||||
serverSocket = new ServerSocket();
|
||||
serverSocket.bind(new InetSocketAddress("localhost", 0));
|
||||
|
||||
final int localPort = serverSocket.getLocalPort();
|
||||
listener = new Listener(serverSocket, runner);
|
||||
final Thread listenThread = new Thread(listener);
|
||||
listenThread.setName("Listen to MiNiFi");
|
||||
listenThread.setDaemon(true);
|
||||
listenThread.start();
|
||||
return localPort;
|
||||
}
|
||||
|
||||
public void stop() throws IOException {
|
||||
final Listener listener = this.listener;
|
||||
if (listener == null) {
|
||||
return;
|
||||
}
|
||||
|
||||
listener.stop();
|
||||
}
|
||||
|
||||
private class Listener implements Runnable {
|
||||
|
||||
private final ServerSocket serverSocket;
|
||||
private final ExecutorService executor;
|
||||
private final RunMiNiFi runner;
|
||||
private volatile boolean stopped = false;
|
||||
|
||||
public Listener(final ServerSocket serverSocket, final RunMiNiFi runner) {
|
||||
this.serverSocket = serverSocket;
|
||||
this.executor = Executors.newFixedThreadPool(2, new ThreadFactory() {
|
||||
@Override
|
||||
public Thread newThread(final Runnable runnable) {
|
||||
final Thread t = Executors.defaultThreadFactory().newThread(runnable);
|
||||
t.setDaemon(true);
|
||||
t.setName("MiNiFi Bootstrap Command Listener");
|
||||
return t;
|
||||
}
|
||||
});
|
||||
|
||||
this.runner = runner;
|
||||
}
|
||||
|
||||
public void stop() throws IOException {
|
||||
stopped = true;
|
||||
|
||||
executor.shutdown();
|
||||
try {
|
||||
executor.awaitTermination(3, TimeUnit.SECONDS);
|
||||
} catch (final InterruptedException ie) {
|
||||
}
|
||||
|
||||
serverSocket.close();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
while (!serverSocket.isClosed()) {
|
||||
try {
|
||||
if (stopped) {
|
||||
return;
|
||||
}
|
||||
|
||||
final Socket socket;
|
||||
try {
|
||||
socket = serverSocket.accept();
|
||||
} catch (final IOException ioe) {
|
||||
if (stopped) {
|
||||
return;
|
||||
}
|
||||
|
||||
throw ioe;
|
||||
}
|
||||
|
||||
executor.submit(new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
try {
|
||||
// we want to ensure that we don't try to read data from an InputStream directly
|
||||
// by a BufferedReader because any user on the system could open a socket and send
|
||||
// a multi-gigabyte file without any new lines in order to crash the Bootstrap,
|
||||
// which in turn may cause the Shutdown Hook to shutdown MiNiFi.
|
||||
// So we will limit the amount of data to read to 4 KB
|
||||
final InputStream limitingIn = new LimitingInputStream(socket.getInputStream(), 4096);
|
||||
final BootstrapCodec codec = new BootstrapCodec(runner, limitingIn, socket.getOutputStream());
|
||||
codec.communicate();
|
||||
} catch (final Throwable t) {
|
||||
System.out.println("Failed to communicate with MiNiFi due to " + t);
|
||||
t.printStackTrace();
|
||||
} finally {
|
||||
try {
|
||||
socket.close();
|
||||
} catch (final IOException ioe) {
|
||||
}
|
||||
}
|
||||
}
|
||||
});
|
||||
} catch (final Throwable t) {
|
||||
System.err.println("Failed to receive information from MiNiFi due to " + t);
|
||||
t.printStackTrace();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,81 @@
|
|||
/*
|
||||
* 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;
|
||||
|
||||
import java.util.Objects;
|
||||
|
||||
public class MiNiFiParameters {
|
||||
|
||||
private volatile int ccPort;
|
||||
private volatile long minifiPid;
|
||||
private volatile String secretKey;
|
||||
|
||||
public MiNiFiParameters(int ccPort, long minifiPid, String secretKey) {
|
||||
this.ccPort = ccPort;
|
||||
this.minifiPid = minifiPid;
|
||||
this.secretKey = secretKey;
|
||||
}
|
||||
|
||||
public int getMiNiFiPort() {
|
||||
return ccPort;
|
||||
}
|
||||
|
||||
public void setMiNiFiPort(int ccPort) {
|
||||
this.ccPort = ccPort;
|
||||
}
|
||||
|
||||
public long getMinifiPid() {
|
||||
return minifiPid;
|
||||
}
|
||||
|
||||
public void setMinifiPid(long minifiPid) {
|
||||
this.minifiPid = minifiPid;
|
||||
}
|
||||
|
||||
public String getSecretKey() {
|
||||
return secretKey;
|
||||
}
|
||||
|
||||
public void setSecretKey(String secretKey) {
|
||||
this.secretKey = secretKey;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
MiNiFiParameters that = (MiNiFiParameters) o;
|
||||
return ccPort == that.ccPort && minifiPid == that.minifiPid && Objects.equals(secretKey, that.secretKey);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(ccPort, minifiPid, secretKey);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "MiNiFiParameters{" +
|
||||
"ccPort=" + ccPort +
|
||||
", minifiPid=" + minifiPid +
|
||||
'}';
|
||||
}
|
||||
}
|
|
@ -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 Long pid;
|
||||
private final boolean respondingToPing;
|
||||
private final boolean processRunning;
|
||||
|
||||
public MiNiFiStatus() {
|
||||
this.port = null;
|
||||
this.pid = null;
|
||||
this.respondingToPing = false;
|
||||
this.processRunning = false;
|
||||
}
|
||||
|
||||
public MiNiFiStatus(Integer port, Long pid, boolean respondingToPing, boolean processRunning) {
|
||||
this.port = port;
|
||||
this.pid = pid;
|
||||
this.respondingToPing = respondingToPing;
|
||||
this.processRunning = processRunning;
|
||||
}
|
||||
|
||||
public Long getPid() {
|
||||
return pid;
|
||||
}
|
||||
|
||||
public Integer getPort() {
|
||||
return port;
|
||||
}
|
||||
|
||||
public boolean isRespondingToPing() {
|
||||
return respondingToPing;
|
||||
}
|
||||
|
||||
public boolean isProcessRunning() {
|
||||
return processRunning;
|
||||
}
|
||||
}
|
File diff suppressed because it is too large
Load Diff
|
@ -0,0 +1,47 @@
|
|||
/*
|
||||
* 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;
|
||||
|
||||
import static java.util.stream.Collectors.collectingAndThen;
|
||||
import static java.util.stream.Collectors.toSet;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.Set;
|
||||
|
||||
public enum SensitiveProperty {
|
||||
SECRET_KEY("secret.key"),
|
||||
C2_SECURITY_TRUSTSTORE_PASSWORD("c2.security.truststore.password"),
|
||||
C2_SECURITY_KEYSTORE_PASSWORD("c2.security.keystore.password"),
|
||||
NIFI_MINIFI_SECURITY_KEYSTORE_PASSWORD("nifi.minifi.security.keystorePasswd"),
|
||||
NIFI_MINIFI_SECURITY_TRUSTSTORE_PASSWORD("nifi.minifi.security.truststorePasswd"),
|
||||
NIFI_MINIFI_SENSITIVE_PROPS_KEY("nifi.minifi.sensitive.props.key");
|
||||
|
||||
public static final Set<String> SENSITIVE_PROPERTIES = Arrays.stream(SensitiveProperty.values()).map(SensitiveProperty::getKey)
|
||||
.collect(collectingAndThen(toSet(), Collections::unmodifiableSet));
|
||||
|
||||
private final String key;
|
||||
|
||||
SensitiveProperty(String key) {
|
||||
this.key = key;
|
||||
}
|
||||
|
||||
public String getKey() {
|
||||
return key;
|
||||
}
|
||||
}
|
|
@ -17,107 +17,30 @@
|
|||
package org.apache.nifi.minifi.bootstrap;
|
||||
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.net.Socket;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.apache.nifi.minifi.bootstrap.status.PeriodicStatusReporter;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeCoordinator;
|
||||
import org.apache.nifi.minifi.bootstrap.service.MiNiFiStdLogHandler;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
public class ShutdownHook extends Thread {
|
||||
|
||||
private final Process nifiProcess;
|
||||
private static final Logger LOGGER = LoggerFactory.getLogger("org.apache.nifi.minifi.bootstrap.Command");
|
||||
|
||||
private final RunMiNiFi runner;
|
||||
private final int gracefulShutdownSeconds;
|
||||
private final ExecutorService executor;
|
||||
private final MiNiFiStdLogHandler miNiFiStdLogHandler;
|
||||
|
||||
private volatile String secretKey;
|
||||
|
||||
public ShutdownHook(final Process nifiProcess, final RunMiNiFi runner, final String secretKey, final int gracefulShutdownSeconds, final ExecutorService executor) {
|
||||
this.nifiProcess = nifiProcess;
|
||||
public ShutdownHook(RunMiNiFi runner, MiNiFiStdLogHandler miNiFiStdLogHandler) {
|
||||
this.runner = runner;
|
||||
this.secretKey = secretKey;
|
||||
this.gracefulShutdownSeconds = gracefulShutdownSeconds;
|
||||
this.executor = executor;
|
||||
}
|
||||
|
||||
void setSecretKey(final String secretKey) {
|
||||
this.secretKey = secretKey;
|
||||
this.miNiFiStdLogHandler = miNiFiStdLogHandler;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
executor.shutdown();
|
||||
|
||||
System.out.println("Initiating shutdown of bootstrap change ingestors...");
|
||||
ConfigurationChangeCoordinator notifier = runner.getChangeCoordinator();
|
||||
if (notifier != null) {
|
||||
try {
|
||||
notifier.close();
|
||||
} catch (IOException ioe) {
|
||||
System.out.println("Could not successfully stop notifier due to " + ioe);
|
||||
}
|
||||
}
|
||||
|
||||
System.out.println("Initiating shutdown of bootstrap periodic status reporters...");
|
||||
for (PeriodicStatusReporter periodicStatusReporter : runner.getPeriodicStatusReporters()) {
|
||||
try {
|
||||
periodicStatusReporter.stop();
|
||||
} catch (Exception exception) {
|
||||
System.out.println("Could not successfully stop periodic status reporter " + periodicStatusReporter.getClass() + " due to " + exception);
|
||||
}
|
||||
}
|
||||
LOGGER.info("Initiating Shutdown of MiNiFi...");
|
||||
|
||||
miNiFiStdLogHandler.shutdown();
|
||||
runner.shutdownChangeNotifier();
|
||||
runner.getPeriodicStatusReporterManager().shutdownPeriodicStatusReporters();
|
||||
runner.setAutoRestartNiFi(false);
|
||||
final int ccPort = runner.getNiFiCommandControlPort();
|
||||
if (ccPort > 0) {
|
||||
System.out.println("Initiating Shutdown of MiNiFi...");
|
||||
|
||||
try {
|
||||
final Socket socket = new Socket("localhost", ccPort);
|
||||
final OutputStream out = socket.getOutputStream();
|
||||
out.write(("SHUTDOWN " + secretKey + "\n").getBytes(StandardCharsets.UTF_8));
|
||||
out.flush();
|
||||
|
||||
socket.close();
|
||||
} catch (final IOException ioe) {
|
||||
System.out.println("Failed to Shutdown MiNiFi due to " + ioe);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
System.out.println("Waiting for Apache MiNiFi to finish shutting down...");
|
||||
final long startWait = System.nanoTime();
|
||||
while (RunMiNiFi.isAlive(nifiProcess)) {
|
||||
final long waitNanos = System.nanoTime() - startWait;
|
||||
final long waitSeconds = TimeUnit.NANOSECONDS.toSeconds(waitNanos);
|
||||
if (waitSeconds >= gracefulShutdownSeconds && gracefulShutdownSeconds > 0) {
|
||||
if (RunMiNiFi.isAlive(nifiProcess)) {
|
||||
System.out.println("MiNiFi has not finished shutting down after " + gracefulShutdownSeconds + " seconds. Killing process.");
|
||||
nifiProcess.destroy();
|
||||
}
|
||||
break;
|
||||
} else {
|
||||
try {
|
||||
Thread.sleep(1000L);
|
||||
} catch (final InterruptedException ie) {
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
try {
|
||||
final File statusFile = runner.getStatusFile();
|
||||
if (!statusFile.delete()) {
|
||||
System.err.println("Failed to delete status file " + statusFile.getAbsolutePath() + "; this file should be cleaned up manually");
|
||||
}
|
||||
}catch (IOException ex){
|
||||
System.err.println("Failed to retrieve status file " + ex);
|
||||
}
|
||||
|
||||
System.out.println("MiNiFi is done shutting down");
|
||||
runner.run(BootstrapCommand.STOP);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -15,15 +15,21 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.nifi.minifi.bootstrap.configuration.differentiators.interfaces;
|
||||
package org.apache.nifi.minifi.bootstrap;
|
||||
|
||||
import org.apache.nifi.minifi.bootstrap.ConfigurationFileHolder;
|
||||
public enum Status {
|
||||
OK(0),
|
||||
ERROR(1),
|
||||
MINIFI_NOT_RESPONDING(4),
|
||||
MINIFI_NOT_RUNNING(3);
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Properties;
|
||||
private final int statusCode;
|
||||
|
||||
public interface Differentiator <T> {
|
||||
void initialize(Properties properties, ConfigurationFileHolder configurationFileHolder);
|
||||
Status(int statusCode) {
|
||||
this.statusCode = statusCode;
|
||||
}
|
||||
|
||||
boolean isNew(T input) throws IOException;
|
||||
public int getStatusCode() {
|
||||
return statusCode;
|
||||
}
|
||||
}
|
|
@ -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 {
|
||||
|
||||
private static RunMiNiFi bootstrap;
|
||||
|
||||
public static void start(String[] args) throws IOException, InterruptedException {
|
||||
|
||||
final File bootstrapConfigFile = RunMiNiFi.getBootstrapConfFile();
|
||||
|
||||
bootstrap = new RunMiNiFi(bootstrapConfigFile);
|
||||
bootstrap.start();
|
||||
|
||||
public static void start(String[] args) throws IOException {
|
||||
bootstrap = new RunMiNiFi(BootstrapFileProvider.getBootstrapConfFile());
|
||||
bootstrap.run(BootstrapCommand.START);
|
||||
}
|
||||
|
||||
public static void stop(String[] args) throws IOException, InterruptedException {
|
||||
|
||||
public static void stop(String[] args) {
|
||||
bootstrap.setAutoRestartNiFi(false);
|
||||
bootstrap.stop();
|
||||
|
||||
bootstrap.run(BootstrapCommand.STOP);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
|
@ -0,0 +1,110 @@
|
|||
/*
|
||||
* 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 java.io.File;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import org.apache.nifi.minifi.bootstrap.BootstrapCommand;
|
||||
import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
|
||||
import org.apache.nifi.minifi.bootstrap.RunMiNiFi;
|
||||
import org.apache.nifi.minifi.bootstrap.service.BootstrapFileProvider;
|
||||
import org.apache.nifi.minifi.bootstrap.service.CurrentPortProvider;
|
||||
import org.apache.nifi.minifi.bootstrap.service.GracefulShutdownParameterProvider;
|
||||
import org.apache.nifi.minifi.bootstrap.service.MiNiFiCommandSender;
|
||||
import org.apache.nifi.minifi.bootstrap.service.MiNiFiExecCommandProvider;
|
||||
import org.apache.nifi.minifi.bootstrap.service.MiNiFiStatusProvider;
|
||||
import org.apache.nifi.minifi.bootstrap.service.MiNiFiStdLogHandler;
|
||||
import org.apache.nifi.minifi.bootstrap.service.PeriodicStatusReporterManager;
|
||||
|
||||
public class CommandRunnerFactory {
|
||||
|
||||
private final MiNiFiCommandSender miNiFiCommandSender;
|
||||
private final CurrentPortProvider currentPortProvider;
|
||||
private final MiNiFiParameters miNiFiParameters;
|
||||
private final MiNiFiStatusProvider miNiFiStatusProvider;
|
||||
private final PeriodicStatusReporterManager periodicStatusReporterManager;
|
||||
private final BootstrapFileProvider bootstrapFileProvider;
|
||||
private final MiNiFiStdLogHandler miNiFiStdLogHandler;
|
||||
private final File bootstrapConfigFile;
|
||||
private final RunMiNiFi runMiNiFi;
|
||||
private final GracefulShutdownParameterProvider gracefulShutdownParameterProvider;
|
||||
private final MiNiFiExecCommandProvider miNiFiExecCommandProvider;
|
||||
|
||||
public CommandRunnerFactory(MiNiFiCommandSender miNiFiCommandSender, CurrentPortProvider currentPortProvider, MiNiFiParameters miNiFiParameters,
|
||||
MiNiFiStatusProvider miNiFiStatusProvider, PeriodicStatusReporterManager periodicStatusReporterManager,
|
||||
BootstrapFileProvider bootstrapFileProvider, MiNiFiStdLogHandler miNiFiStdLogHandler, File bootstrapConfigFile, RunMiNiFi runMiNiFi,
|
||||
GracefulShutdownParameterProvider gracefulShutdownParameterProvider, MiNiFiExecCommandProvider miNiFiExecCommandProvider) {
|
||||
this.miNiFiCommandSender = miNiFiCommandSender;
|
||||
this.currentPortProvider = currentPortProvider;
|
||||
this.miNiFiParameters = miNiFiParameters;
|
||||
this.miNiFiStatusProvider = miNiFiStatusProvider;
|
||||
this.periodicStatusReporterManager = periodicStatusReporterManager;
|
||||
this.bootstrapFileProvider = bootstrapFileProvider;
|
||||
this.miNiFiStdLogHandler = miNiFiStdLogHandler;
|
||||
this.bootstrapConfigFile = bootstrapConfigFile;
|
||||
this.runMiNiFi = runMiNiFi;
|
||||
this.gracefulShutdownParameterProvider = gracefulShutdownParameterProvider;
|
||||
this.miNiFiExecCommandProvider = miNiFiExecCommandProvider;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a runner associated with the given command.
|
||||
* @param command the bootstrap command
|
||||
* @return the runner
|
||||
*/
|
||||
public CommandRunner getRunner(BootstrapCommand command) {
|
||||
CommandRunner commandRunner;
|
||||
switch (command) {
|
||||
case START:
|
||||
case RUN:
|
||||
commandRunner = new StartRunner(currentPortProvider, bootstrapFileProvider, periodicStatusReporterManager, miNiFiStdLogHandler, miNiFiParameters,
|
||||
bootstrapConfigFile, runMiNiFi, miNiFiExecCommandProvider);
|
||||
break;
|
||||
case STOP:
|
||||
commandRunner = new StopRunner(bootstrapFileProvider, miNiFiParameters, miNiFiCommandSender, currentPortProvider, gracefulShutdownParameterProvider);
|
||||
break;
|
||||
case STATUS:
|
||||
commandRunner = new StatusRunner(miNiFiParameters, miNiFiStatusProvider);
|
||||
break;
|
||||
case RESTART:
|
||||
commandRunner = new CompositeCommandRunner(getRestartServices());
|
||||
break;
|
||||
case DUMP:
|
||||
commandRunner = new DumpRunner(miNiFiCommandSender, currentPortProvider);
|
||||
break;
|
||||
case ENV:
|
||||
commandRunner = new EnvRunner(miNiFiCommandSender, currentPortProvider);
|
||||
break;
|
||||
case FLOWSTATUS:
|
||||
commandRunner = new FlowStatusRunner(periodicStatusReporterManager);
|
||||
break;
|
||||
default:
|
||||
throw new IllegalArgumentException("Unknown MiNiFi bootstrap command");
|
||||
}
|
||||
return commandRunner;
|
||||
}
|
||||
|
||||
private List<CommandRunner> getRestartServices() {
|
||||
List<CommandRunner> compositeList = new LinkedList<>();
|
||||
compositeList.add(new StopRunner(bootstrapFileProvider, miNiFiParameters, miNiFiCommandSender, currentPortProvider, gracefulShutdownParameterProvider));
|
||||
compositeList.add(new StartRunner(currentPortProvider, bootstrapFileProvider, periodicStatusReporterManager, miNiFiStdLogHandler, miNiFiParameters,
|
||||
bootstrapConfigFile, runMiNiFi, miNiFiExecCommandProvider));
|
||||
return compositeList;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,49 @@
|
|||
/*
|
||||
* 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.Status.OK;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
|
||||
/**
|
||||
* Composite runner which can execute multiple commands in a sequential order.
|
||||
*/
|
||||
public class CompositeCommandRunner implements CommandRunner {
|
||||
final List<CommandRunner> services;
|
||||
|
||||
public CompositeCommandRunner(List<CommandRunner> services) {
|
||||
this.services = Optional.ofNullable(services).map(Collections::unmodifiableList).orElse(Collections.emptyList());
|
||||
}
|
||||
|
||||
/**
|
||||
* Executes the runners in sequential order. Stops on first failure.
|
||||
* @param args the input arguments
|
||||
* @return the first failed command status code or OK if there was no failure
|
||||
*/
|
||||
@Override
|
||||
public int runCommand(String[] args) {
|
||||
return services.stream()
|
||||
.map(service -> service.runCommand(args))
|
||||
.filter(code -> code != OK.getStatusCode())
|
||||
.findFirst()
|
||||
.orElse(OK.getStatusCode());
|
||||
}
|
||||
}
|
|
@ -0,0 +1,96 @@
|
|||
/*
|
||||
* 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.io.File;
|
||||
import java.io.FileOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.Optional;
|
||||
import org.apache.nifi.minifi.bootstrap.service.CurrentPortProvider;
|
||||
import org.apache.nifi.minifi.bootstrap.service.MiNiFiCommandSender;
|
||||
|
||||
public class DumpRunner implements CommandRunner {
|
||||
private static final String DUMP_CMD = "DUMP";
|
||||
|
||||
private final MiNiFiCommandSender miNiFiCommandSender;
|
||||
private final CurrentPortProvider currentPortProvider;
|
||||
|
||||
public DumpRunner(MiNiFiCommandSender miNiFiCommandSender, CurrentPortProvider currentPortProvider) {
|
||||
this.miNiFiCommandSender = miNiFiCommandSender;
|
||||
this.currentPortProvider = currentPortProvider;
|
||||
}
|
||||
|
||||
/**
|
||||
* Writes a MiNiFi thread dump to the given file; if file is null, logs at
|
||||
* INFO level instead.
|
||||
*
|
||||
* @param args the second parameter is the file to write the dump content to
|
||||
*/
|
||||
@Override
|
||||
public int runCommand(String[] args) {
|
||||
return dump(getArg(args, 1).map(File::new).orElse(null));
|
||||
}
|
||||
|
||||
private int dump(File dumpFile) {
|
||||
Integer port = currentPortProvider.getCurrentPort();
|
||||
if (port == null) {
|
||||
CMD_LOGGER.error("Apache MiNiFi is not currently running");
|
||||
return MINIFI_NOT_RUNNING.getStatusCode();
|
||||
}
|
||||
|
||||
Optional<String> dump;
|
||||
try {
|
||||
dump = miNiFiCommandSender.sendCommand(DUMP_CMD, port);
|
||||
} catch (IOException e) {
|
||||
CMD_LOGGER.error("Failed to get DUMP response from MiNiFi");
|
||||
DEFAULT_LOGGER.error("Exception:", e);
|
||||
return ERROR.getStatusCode();
|
||||
}
|
||||
|
||||
return Optional.ofNullable(dumpFile)
|
||||
.map(dmp -> writeDumpToFile(dmp, dump))
|
||||
.orElseGet(() -> {
|
||||
dump.ifPresent(CMD_LOGGER::info);
|
||||
return OK.getStatusCode();
|
||||
});
|
||||
}
|
||||
|
||||
private Integer writeDumpToFile(File dumpFile, Optional<String> dump) {
|
||||
try (FileOutputStream fos = new FileOutputStream(dumpFile)) {
|
||||
fos.write(dump.orElse("Dump has empty response").getBytes(StandardCharsets.UTF_8));
|
||||
} catch (IOException e) {
|
||||
CMD_LOGGER.error("Failed to write DUMP response to file");
|
||||
DEFAULT_LOGGER.error("Exception:", e);
|
||||
return ERROR.getStatusCode();
|
||||
}
|
||||
// we want to log to the console (by default) that we wrote the thread dump to the specified file
|
||||
CMD_LOGGER.info("Successfully wrote thread dump to {}", dumpFile.getAbsolutePath());
|
||||
return OK.getStatusCode();
|
||||
}
|
||||
|
||||
private Optional<String> getArg(String[] args, int index) {
|
||||
return Optional.ofNullable(args.length > index ? args[index] : null);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,68 @@
|
|||
/*
|
||||
* 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.io.IOException;
|
||||
import org.apache.nifi.minifi.bootstrap.service.CurrentPortProvider;
|
||||
import org.apache.nifi.minifi.bootstrap.service.MiNiFiCommandSender;
|
||||
|
||||
public class EnvRunner implements CommandRunner {
|
||||
private static final String ENV_CMD = "ENV";
|
||||
|
||||
private final MiNiFiCommandSender miNiFiCommandSender;
|
||||
private final CurrentPortProvider currentPortProvider;
|
||||
|
||||
public EnvRunner(MiNiFiCommandSender miNiFiCommandSender, CurrentPortProvider currentPortProvider) {
|
||||
this.miNiFiCommandSender = miNiFiCommandSender;
|
||||
this.currentPortProvider = currentPortProvider;
|
||||
}
|
||||
|
||||
/**
|
||||
* 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() {
|
||||
Integer port = currentPortProvider.getCurrentPort();
|
||||
if (port == null) {
|
||||
CMD_LOGGER.error("Apache MiNiFi is not currently running");
|
||||
return MINIFI_NOT_RUNNING.getStatusCode();
|
||||
}
|
||||
|
||||
try {
|
||||
miNiFiCommandSender.sendCommand(ENV_CMD, port).ifPresent(CMD_LOGGER::info);
|
||||
} catch (IOException e) {
|
||||
CMD_LOGGER.error("Failed to get ENV response from MiNiFi");
|
||||
DEFAULT_LOGGER.error("Exception:", e);
|
||||
return ERROR.getStatusCode();
|
||||
}
|
||||
|
||||
return OK.getStatusCode();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,49 @@
|
|||
/*
|
||||
* 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.Status.ERROR;
|
||||
import static org.apache.nifi.minifi.bootstrap.Status.OK;
|
||||
|
||||
import org.apache.nifi.minifi.bootstrap.service.PeriodicStatusReporterManager;
|
||||
|
||||
public class FlowStatusRunner implements CommandRunner {
|
||||
private final PeriodicStatusReporterManager periodicStatusReporterManager;
|
||||
|
||||
public FlowStatusRunner(PeriodicStatusReporterManager periodicStatusReporterManager) {
|
||||
this.periodicStatusReporterManager = periodicStatusReporterManager;
|
||||
}
|
||||
|
||||
/**
|
||||
* Receive and print detailed flow information from MiNiFi.
|
||||
* Example query: processor:TailFile:health,stats,bulletins
|
||||
* @param args the input arguments
|
||||
* @return status code
|
||||
*/
|
||||
@Override
|
||||
public int runCommand(String[] args) {
|
||||
if(args.length == 2) {
|
||||
CMD_LOGGER.info(periodicStatusReporterManager.statusReport(args[1]).toString());
|
||||
return OK.getStatusCode();
|
||||
} else {
|
||||
CMD_LOGGER.error("The 'flowStatus' command requires an input query. See the System Admin Guide 'FlowStatus Script Query' section for complete details.");
|
||||
return ERROR.getStatusCode();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,329 @@
|
|||
/*
|
||||
* 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.List;
|
||||
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 java.util.stream.Collectors;
|
||||
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.MiNiFiExecCommandProvider;
|
||||
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.UnixProcessUtils;
|
||||
import org.apache.nifi.util.Tuple;
|
||||
|
||||
public class StartRunner implements CommandRunner {
|
||||
private static final int STARTUP_WAIT_SECONDS = 60;
|
||||
|
||||
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;
|
||||
private final MiNiFiExecCommandProvider miNiFiExecCommandProvider;
|
||||
|
||||
public StartRunner(CurrentPortProvider currentPortProvider, BootstrapFileProvider bootstrapFileProvider,
|
||||
PeriodicStatusReporterManager periodicStatusReporterManager, MiNiFiStdLogHandler miNiFiStdLogHandler, MiNiFiParameters miNiFiParameters, File bootstrapConfigFile,
|
||||
RunMiNiFi runMiNiFi, MiNiFiExecCommandProvider miNiFiExecCommandProvider) {
|
||||
this.currentPortProvider = currentPortProvider;
|
||||
this.bootstrapFileProvider = bootstrapFileProvider;
|
||||
this.periodicStatusReporterManager = periodicStatusReporterManager;
|
||||
this.miNiFiStdLogHandler = miNiFiStdLogHandler;
|
||||
this.miNiFiParameters = miNiFiParameters;
|
||||
this.bootstrapConfigFile = bootstrapConfigFile;
|
||||
this.runMiNiFi = runMiNiFi;
|
||||
this.miNiFiExecCommandProvider = miNiFiExecCommandProvider;
|
||||
}
|
||||
|
||||
/**
|
||||
* 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) {
|
||||
CMD_LOGGER.info("Apache MiNiFi is already running, 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 (UnixProcessUtils.isAlive(process)) {
|
||||
handleReload();
|
||||
} else {
|
||||
Runtime runtime = Runtime.getRuntime();
|
||||
try {
|
||||
runtime.removeShutdownHook(shutdownHook);
|
||||
} catch (IllegalStateException ise) {
|
||||
DEFAULT_LOGGER.trace("The virtual machine is already in the process of shutting down", ise);
|
||||
}
|
||||
|
||||
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 = startMiNiFiProcess(builder);
|
||||
|
||||
boolean started = waitForStart();
|
||||
|
||||
if (started) {
|
||||
Long pid = OSUtils.getProcessId(process, DEFAULT_LOGGER);
|
||||
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();
|
||||
|
||||
File workingDir = getWorkingDir();
|
||||
MiNiFiListener listener = new MiNiFiListener();
|
||||
int listenPort = listener.start(runMiNiFi);
|
||||
List<String> cmd = miNiFiExecCommandProvider.getMiNiFiExecCommand(listenPort, workingDir);
|
||||
|
||||
builder.command(cmd);
|
||||
builder.directory(workingDir);
|
||||
|
||||
CMD_LOGGER.info("Starting Apache MiNiFi...");
|
||||
CMD_LOGGER.info("Working Directory: {}", workingDir.getAbsolutePath());
|
||||
CMD_LOGGER.info("Command: {}", cmd.stream().collect(Collectors.joining(" ")));
|
||||
|
||||
return new Tuple<>(builder, startMiNiFiProcess(builder));
|
||||
}
|
||||
|
||||
private Process startMiNiFiProcess(ProcessBuilder builder) throws IOException {
|
||||
Process process = builder.start();
|
||||
miNiFiStdLogHandler.initLogging(process);
|
||||
Long pid = OSUtils.getProcessId(process, CMD_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.getRuntime().addShutdownHook(shutdownHook);
|
||||
return process;
|
||||
}
|
||||
|
||||
private File getWorkingDir() throws IOException {
|
||||
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());
|
||||
return workingDir;
|
||||
}
|
||||
|
||||
private boolean waitForStart() {
|
||||
lock.lock();
|
||||
try {
|
||||
long startTime = System.nanoTime();
|
||||
|
||||
while (miNiFiParameters.getMinifiPid() < 1 && miNiFiParameters.getMiNiFiPort() < 1) {
|
||||
try {
|
||||
startupCondition.await(1, TimeUnit.SECONDS);
|
||||
} catch (InterruptedException ie) {
|
||||
return false;
|
||||
}
|
||||
|
||||
long waitNanos = System.nanoTime() - startTime;
|
||||
long waitSeconds = TimeUnit.NANOSECONDS.toSeconds(waitNanos);
|
||||
if (waitSeconds > STARTUP_WAIT_SECONDS) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
lock.unlock();
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,73 @@
|
|||
/*
|
||||
* 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.Status.MINIFI_NOT_RESPONDING;
|
||||
import static org.apache.nifi.minifi.bootstrap.Status.MINIFI_NOT_RUNNING;
|
||||
import static org.apache.nifi.minifi.bootstrap.Status.OK;
|
||||
|
||||
import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
|
||||
import org.apache.nifi.minifi.bootstrap.MiNiFiStatus;
|
||||
import org.apache.nifi.minifi.bootstrap.service.MiNiFiStatusProvider;
|
||||
|
||||
public class StatusRunner implements CommandRunner {
|
||||
private final MiNiFiParameters miNiFiParameters;
|
||||
private final MiNiFiStatusProvider miNiFiStatusProvider;
|
||||
|
||||
public StatusRunner(MiNiFiParameters miNiFiParameters, MiNiFiStatusProvider miNiFiStatusProvider) {
|
||||
this.miNiFiParameters = miNiFiParameters;
|
||||
this.miNiFiStatusProvider = miNiFiStatusProvider;
|
||||
}
|
||||
|
||||
/**
|
||||
* Prints the current status of the MiNiFi process.
|
||||
* @param args the input arguments
|
||||
* @return status code
|
||||
*/
|
||||
@Override
|
||||
public int runCommand(String[] args) {
|
||||
return status();
|
||||
}
|
||||
|
||||
private int status() {
|
||||
MiNiFiStatus status = miNiFiStatusProvider.getStatus(miNiFiParameters.getMiNiFiPort(), miNiFiParameters.getMinifiPid());
|
||||
if (status.isRespondingToPing()) {
|
||||
CMD_LOGGER.info("Apache MiNiFi is currently running, listening to Bootstrap on port {}, PID={}",
|
||||
status.getPort(), status.getPid() == null ? "unknown" : status.getPid());
|
||||
return OK.getStatusCode();
|
||||
}
|
||||
|
||||
if (status.isProcessRunning()) {
|
||||
CMD_LOGGER.info("Apache MiNiFi is running at PID {} but is not responding to ping requests", status.getPid());
|
||||
return MINIFI_NOT_RESPONDING.getStatusCode();
|
||||
}
|
||||
|
||||
if (status.getPort() == null) {
|
||||
CMD_LOGGER.info("Apache MiNiFi is not running");
|
||||
return MINIFI_NOT_RUNNING.getStatusCode();
|
||||
}
|
||||
|
||||
if (status.getPid() == null) {
|
||||
CMD_LOGGER.info("Apache MiNiFi is not responding to Ping requests. The process may have died or may be hung");
|
||||
} else {
|
||||
CMD_LOGGER.info("Apache MiNiFi is not running");
|
||||
}
|
||||
return MINIFI_NOT_RUNNING.getStatusCode();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,126 @@
|
|||
/*
|
||||
* 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.RunMiNiFi.UNINITIALIZED;
|
||||
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.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.Optional;
|
||||
import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
|
||||
import org.apache.nifi.minifi.bootstrap.service.BootstrapFileProvider;
|
||||
import org.apache.nifi.minifi.bootstrap.service.CurrentPortProvider;
|
||||
import org.apache.nifi.minifi.bootstrap.service.GracefulShutdownParameterProvider;
|
||||
import org.apache.nifi.minifi.bootstrap.service.MiNiFiCommandSender;
|
||||
import org.apache.nifi.minifi.bootstrap.util.UnixProcessUtils;
|
||||
|
||||
public class StopRunner implements CommandRunner {
|
||||
private static final String SHUTDOWN_CMD = "SHUTDOWN";
|
||||
|
||||
private final BootstrapFileProvider bootstrapFileProvider;
|
||||
private final MiNiFiParameters miNiFiParameters;
|
||||
private final MiNiFiCommandSender miNiFiCommandSender;
|
||||
private final CurrentPortProvider currentPortProvider;
|
||||
private final GracefulShutdownParameterProvider gracefulShutdownParameterProvider;
|
||||
|
||||
public StopRunner(BootstrapFileProvider bootstrapFileProvider, MiNiFiParameters miNiFiParameters, MiNiFiCommandSender miNiFiCommandSender,
|
||||
CurrentPortProvider currentPortProvider, GracefulShutdownParameterProvider gracefulShutdownParameterProvider) {
|
||||
this.bootstrapFileProvider = bootstrapFileProvider;
|
||||
this.miNiFiParameters = miNiFiParameters;
|
||||
this.miNiFiCommandSender = miNiFiCommandSender;
|
||||
this.currentPortProvider = currentPortProvider;
|
||||
this.gracefulShutdownParameterProvider = gracefulShutdownParameterProvider;
|
||||
}
|
||||
|
||||
/**
|
||||
* Shutdown the MiNiFi and the managing bootstrap process as well.
|
||||
* @param args the input arguments
|
||||
* @return status code
|
||||
*/
|
||||
@Override
|
||||
public int runCommand(String[] args) {
|
||||
try {
|
||||
return stop();
|
||||
} catch (Exception e) {
|
||||
DEFAULT_LOGGER.error("Exception happened during stopping MiNiFi", e);
|
||||
return ERROR.getStatusCode();
|
||||
}
|
||||
}
|
||||
|
||||
private int stop() throws IOException {
|
||||
Integer currentPort = currentPortProvider.getCurrentPort();
|
||||
if (currentPort == null) {
|
||||
CMD_LOGGER.error("Apache MiNiFi is not currently running");
|
||||
return MINIFI_NOT_RUNNING.getStatusCode();
|
||||
}
|
||||
|
||||
int status = OK.getStatusCode();
|
||||
// indicate that a stop command is in progress
|
||||
File lockFile = bootstrapFileProvider.getLockFile();
|
||||
if (!lockFile.exists()) {
|
||||
lockFile.createNewFile();
|
||||
}
|
||||
|
||||
File statusFile = bootstrapFileProvider.getStatusFile();
|
||||
File pidFile = bootstrapFileProvider.getPidFile();
|
||||
long minifiPid = miNiFiParameters.getMinifiPid();
|
||||
|
||||
try {
|
||||
Optional<String> commandResponse = miNiFiCommandSender.sendCommand(SHUTDOWN_CMD, currentPort);
|
||||
if (commandResponse.filter(SHUTDOWN_CMD::equals).isPresent()) {
|
||||
CMD_LOGGER.info("Apache MiNiFi has accepted the Shutdown Command and is shutting down now");
|
||||
|
||||
if (minifiPid != UNINITIALIZED) {
|
||||
UnixProcessUtils.gracefulShutDownMiNiFiProcess(minifiPid, "MiNiFi has not finished shutting down after {} seconds. Killing process.",
|
||||
gracefulShutdownParameterProvider.getGracefulShutdownSeconds());
|
||||
|
||||
if (statusFile.exists() && !statusFile.delete()) {
|
||||
CMD_LOGGER.error("Failed to delete status file {}; this file should be cleaned up manually", statusFile);
|
||||
}
|
||||
|
||||
if (pidFile.exists() && !pidFile.delete()) {
|
||||
CMD_LOGGER.error("Failed to delete pid file {}; this file should be cleaned up manually", pidFile);
|
||||
}
|
||||
|
||||
CMD_LOGGER.info("MiNiFi has finished shutting down.");
|
||||
}
|
||||
} else {
|
||||
CMD_LOGGER.error("When sending SHUTDOWN command to MiNiFi, got unexpected response {}", commandResponse.orElse(null));
|
||||
status = ERROR.getStatusCode();
|
||||
}
|
||||
} catch (IOException e) {
|
||||
if (minifiPid == UNINITIALIZED) {
|
||||
DEFAULT_LOGGER.error("No PID found for the MiNiFi process, so unable to kill process; The process should be killed manually.");
|
||||
} else {
|
||||
DEFAULT_LOGGER.error("Will kill the MiNiFi Process with PID {}", minifiPid);
|
||||
UnixProcessUtils.killProcessTree(minifiPid);
|
||||
}
|
||||
} finally {
|
||||
if (lockFile.exists() && !lockFile.delete()) {
|
||||
CMD_LOGGER.error("Failed to delete lock file {}; this file should be cleaned up manually", lockFile);
|
||||
}
|
||||
}
|
||||
|
||||
return status;
|
||||
}
|
||||
}
|
|
@ -1,4 +1,4 @@
|
|||
/**
|
||||
/*
|
||||
* 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.
|
||||
|
@ -16,61 +16,46 @@
|
|||
*/
|
||||
package org.apache.nifi.minifi.bootstrap.configuration;
|
||||
|
||||
import org.apache.nifi.minifi.bootstrap.ConfigurationFileHolder;
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
import java.util.Optional;
|
||||
import java.util.Properties;
|
||||
import java.util.Set;
|
||||
import org.apache.nifi.minifi.bootstrap.RunMiNiFi;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.ingestors.interfaces.ChangeIngestor;
|
||||
import org.apache.nifi.minifi.bootstrap.util.ByteBufferInputStream;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
import java.util.Properties;
|
||||
import java.util.Set;
|
||||
|
||||
public class ConfigurationChangeCoordinator implements Closeable, ConfigurationChangeNotifier {
|
||||
|
||||
public static final String NOTIFIER_PROPERTY_PREFIX = "nifi.minifi.notifier";
|
||||
public static final String NOTIFIER_INGESTORS_KEY = NOTIFIER_PROPERTY_PREFIX + ".ingestors";
|
||||
private final static Logger logger = LoggerFactory.getLogger(ConfigurationChangeCoordinator.class);
|
||||
private final Set<ConfigurationChangeListener> configurationChangeListeners = new HashSet<>();
|
||||
private static final Logger LOGGER = LoggerFactory.getLogger(ConfigurationChangeCoordinator.class);
|
||||
|
||||
private final Set<ConfigurationChangeListener> configurationChangeListeners;
|
||||
private final Set<ChangeIngestor> changeIngestors = new HashSet<>();
|
||||
|
||||
/**
|
||||
* Provides an opportunity for the implementation to perform configuration and initialization based on properties received from the bootstrapping configuration
|
||||
*
|
||||
* @param properties from the bootstrap configuration
|
||||
*/
|
||||
public void initialize(Properties properties, ConfigurationFileHolder configurationFileHolder, Collection<ConfigurationChangeListener> changeListenerSet) {
|
||||
final String ingestorsCsv = properties.getProperty(NOTIFIER_INGESTORS_KEY);
|
||||
private final Properties bootstrapProperties;
|
||||
private final RunMiNiFi runMiNiFi;
|
||||
|
||||
if (ingestorsCsv != null && !ingestorsCsv.isEmpty()) {
|
||||
for (String ingestorClassname : Arrays.asList(ingestorsCsv.split(","))) {
|
||||
ingestorClassname = ingestorClassname.trim();
|
||||
try {
|
||||
Class<?> ingestorClass = Class.forName(ingestorClassname);
|
||||
ChangeIngestor changeIngestor = (ChangeIngestor) ingestorClass.newInstance();
|
||||
changeIngestor.initialize(properties, configurationFileHolder, this);
|
||||
changeIngestors.add(changeIngestor);
|
||||
logger.info("Initialized ");
|
||||
} catch (InstantiationException | IllegalAccessException | ClassNotFoundException e) {
|
||||
throw new RuntimeException("Issue instantiating ingestor " + ingestorClassname, e);
|
||||
}
|
||||
}
|
||||
}
|
||||
configurationChangeListeners.clear();
|
||||
configurationChangeListeners.addAll(changeListenerSet);
|
||||
public ConfigurationChangeCoordinator(Properties bootstrapProperties, RunMiNiFi runMiNiFi,
|
||||
Set<ConfigurationChangeListener> miNiFiConfigurationChangeListeners) {
|
||||
this.bootstrapProperties = bootstrapProperties;
|
||||
this.runMiNiFi = runMiNiFi;
|
||||
this.configurationChangeListeners = Optional.ofNullable(miNiFiConfigurationChangeListeners).map(Collections::unmodifiableSet).orElse(Collections.emptySet());
|
||||
}
|
||||
|
||||
/**
|
||||
* Begins the associated notification service provided by the given implementation. In most implementations, no action will occur until this method is invoked.
|
||||
*/
|
||||
public void start() {
|
||||
initialize();
|
||||
changeIngestors.forEach(ChangeIngestor::start);
|
||||
}
|
||||
|
||||
|
@ -87,7 +72,7 @@ public class ConfigurationChangeCoordinator implements Closeable, ConfigurationC
|
|||
* Provide the mechanism by which listeners are notified
|
||||
*/
|
||||
public Collection<ListenerHandleResult> notifyListeners(ByteBuffer newConfig) {
|
||||
logger.info("Notifying Listeners of a change");
|
||||
LOGGER.info("Notifying Listeners of a change");
|
||||
|
||||
Collection<ListenerHandleResult> listenerHandleResults = new ArrayList<>(configurationChangeListeners.size());
|
||||
for (final ConfigurationChangeListener listener : getChangeListeners()) {
|
||||
|
@ -99,16 +84,42 @@ public class ConfigurationChangeCoordinator implements Closeable, ConfigurationC
|
|||
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("Instantiating [{}] ingestor failed", ingestorClassname, e);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -17,8 +17,8 @@
|
|||
|
||||
package org.apache.nifi.minifi.bootstrap.configuration.differentiators;
|
||||
|
||||
import org.apache.nifi.minifi.bootstrap.ConfigurationFileHolder;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.differentiators.interfaces.Differentiator;
|
||||
import org.apache.nifi.c2.client.api.ConfigurationFileHolder;
|
||||
import org.apache.nifi.c2.client.api.Differentiator;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
|
|
|
@ -17,7 +17,7 @@
|
|||
|
||||
package org.apache.nifi.minifi.bootstrap.configuration.ingestors;
|
||||
|
||||
import org.apache.nifi.minifi.bootstrap.ConfigurationFileHolder;
|
||||
import org.apache.nifi.c2.client.api.ConfigurationFileHolder;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeNotifier;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.ingestors.interfaces.ChangeIngestor;
|
||||
import org.slf4j.Logger;
|
||||
|
|
|
@ -1,4 +1,4 @@
|
|||
/**
|
||||
/*
|
||||
* 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.
|
||||
|
@ -16,15 +16,10 @@
|
|||
*/
|
||||
package org.apache.nifi.minifi.bootstrap.configuration.ingestors;
|
||||
|
||||
import org.apache.commons.io.input.TeeInputStream;
|
||||
import org.apache.commons.io.output.ByteArrayOutputStream;
|
||||
import org.apache.nifi.minifi.bootstrap.ConfigurationFileHolder;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeNotifier;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.differentiators.WholeConfigDifferentiator;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.differentiators.interfaces.Differentiator;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.ingestors.interfaces.ChangeIngestor;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
import static java.nio.file.StandardWatchEventKinds.ENTRY_MODIFY;
|
||||
import static java.util.Collections.emptyList;
|
||||
import static org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeCoordinator.NOTIFIER_INGESTORS_KEY;
|
||||
import static org.apache.nifi.minifi.bootstrap.configuration.differentiators.WholeConfigDifferentiator.WHOLE_CONFIG_KEY;
|
||||
|
||||
import java.io.FileInputStream;
|
||||
import java.io.IOException;
|
||||
|
@ -39,16 +34,21 @@ import java.nio.file.WatchService;
|
|||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.Properties;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.ScheduledExecutorService;
|
||||
import java.util.concurrent.ThreadFactory;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.function.Supplier;
|
||||
|
||||
import static java.nio.file.StandardWatchEventKinds.ENTRY_MODIFY;
|
||||
import static org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeCoordinator.NOTIFIER_INGESTORS_KEY;
|
||||
import static org.apache.nifi.minifi.bootstrap.configuration.differentiators.WholeConfigDifferentiator.WHOLE_CONFIG_KEY;
|
||||
import org.apache.commons.io.input.TeeInputStream;
|
||||
import org.apache.commons.io.output.ByteArrayOutputStream;
|
||||
import org.apache.nifi.c2.client.api.ConfigurationFileHolder;
|
||||
import org.apache.nifi.c2.client.api.Differentiator;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeNotifier;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.differentiators.WholeConfigDifferentiator;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.ingestors.interfaces.ChangeIngestor;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
/**
|
||||
* FileChangeIngestor provides a simple FileSystem monitor for detecting changes for a specified file as generated from its corresponding {@link Path}. Upon modifications to the associated file,
|
||||
|
@ -98,29 +98,23 @@ public class FileChangeIngestor implements Runnable, ChangeIngestor {
|
|||
}
|
||||
|
||||
protected boolean targetChanged() {
|
||||
boolean targetChanged = false;
|
||||
boolean targetChanged;
|
||||
|
||||
final WatchKey watchKey = this.watchService.poll();
|
||||
Optional<WatchKey> watchKey = Optional.ofNullable(watchService.poll());
|
||||
|
||||
if (watchKey == null) {
|
||||
return targetChanged;
|
||||
}
|
||||
|
||||
for (WatchEvent<?> watchEvt : watchKey.pollEvents()) {
|
||||
final WatchEvent.Kind<?> evtKind = watchEvt.kind();
|
||||
|
||||
final WatchEvent<Path> pathEvent = (WatchEvent<Path>) watchEvt;
|
||||
final Path changedFile = pathEvent.context();
|
||||
|
||||
// determine target change by verifying if the changed file corresponds to the config file monitored for this path
|
||||
targetChanged = (evtKind == ENTRY_MODIFY && changedFile.equals(configFilePath.getName(configFilePath.getNameCount() - 1)));
|
||||
}
|
||||
targetChanged = watchKey
|
||||
.map(WatchKey::pollEvents)
|
||||
.orElse(emptyList())
|
||||
.stream()
|
||||
.anyMatch(watchEvent -> ENTRY_MODIFY == watchEvent.kind()
|
||||
&& ((WatchEvent<Path>) watchEvent).context().equals(configFilePath.getName(configFilePath.getNameCount() - 1)));
|
||||
|
||||
// After completing inspection, reset for detection of subsequent change events
|
||||
boolean valid = watchKey.reset();
|
||||
if (!valid) {
|
||||
throw new IllegalStateException("Unable to reinitialize file system watcher.");
|
||||
}
|
||||
watchKey.map(WatchKey::reset)
|
||||
.filter(valid -> !valid)
|
||||
.ifPresent(valid -> {
|
||||
throw new IllegalStateException("Unable to reinitialize file system watcher.");
|
||||
});
|
||||
|
||||
return targetChanged;
|
||||
}
|
||||
|
@ -212,14 +206,11 @@ public class FileChangeIngestor implements Runnable, ChangeIngestor {
|
|||
|
||||
@Override
|
||||
public void start() {
|
||||
executorService = Executors.newScheduledThreadPool(1, new ThreadFactory() {
|
||||
@Override
|
||||
public Thread newThread(final Runnable r) {
|
||||
final Thread t = Executors.defaultThreadFactory().newThread(r);
|
||||
t.setName("File Change Notifier Thread");
|
||||
t.setDaemon(true);
|
||||
return t;
|
||||
}
|
||||
executorService = Executors.newScheduledThreadPool(1, r -> {
|
||||
Thread t = Executors.defaultThreadFactory().newThread(r);
|
||||
t.setName("File Change Notifier Thread");
|
||||
t.setDaemon(true);
|
||||
return t;
|
||||
});
|
||||
this.executorService.scheduleWithFixedDelay(this, 0, pollingSeconds, DEFAULT_POLLING_PERIOD_UNIT);
|
||||
}
|
||||
|
|
|
@ -23,11 +23,11 @@ import okhttp3.OkHttpClient;
|
|||
import okhttp3.Request;
|
||||
import okhttp3.Response;
|
||||
import okhttp3.ResponseBody;
|
||||
import org.apache.nifi.minifi.bootstrap.ConfigurationFileHolder;
|
||||
import org.apache.nifi.c2.client.api.ConfigurationFileHolder;
|
||||
import org.apache.nifi.c2.client.api.Differentiator;
|
||||
import org.apache.nifi.minifi.bootstrap.RunMiNiFi;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeNotifier;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.differentiators.WholeConfigDifferentiator;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.differentiators.interfaces.Differentiator;
|
||||
import org.apache.nifi.minifi.bootstrap.util.ByteBufferInputStream;
|
||||
import org.apache.nifi.minifi.commons.schema.ConfigSchema;
|
||||
import org.apache.nifi.minifi.commons.schema.SecurityPropertiesSchema;
|
||||
|
@ -78,7 +78,7 @@ public class PullHttpChangeIngestor extends AbstractPullChangeIngestor {
|
|||
private static final String DEFAULT_CONNECT_TIMEOUT_MS = "5000";
|
||||
private static final String DEFAULT_READ_TIMEOUT_MS = "15000";
|
||||
|
||||
private static final String PULL_HTTP_BASE_KEY = NOTIFIER_INGESTORS_KEY + ".pull.http";
|
||||
public static final String PULL_HTTP_BASE_KEY = NOTIFIER_INGESTORS_KEY + ".pull.http";
|
||||
public static final String PULL_HTTP_POLLING_PERIOD_KEY = PULL_HTTP_BASE_KEY + ".period.ms";
|
||||
public static final String PORT_KEY = PULL_HTTP_BASE_KEY + ".port";
|
||||
public static final String HOST_KEY = PULL_HTTP_BASE_KEY + ".hostname";
|
||||
|
|
|
@ -19,11 +19,11 @@ package org.apache.nifi.minifi.bootstrap.configuration.ingestors;
|
|||
|
||||
import org.apache.commons.io.input.TeeInputStream;
|
||||
import org.apache.commons.io.output.ByteArrayOutputStream;
|
||||
import org.apache.nifi.minifi.bootstrap.ConfigurationFileHolder;
|
||||
import org.apache.nifi.c2.client.api.ConfigurationFileHolder;
|
||||
import org.apache.nifi.c2.client.api.Differentiator;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeNotifier;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.ListenerHandleResult;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.differentiators.WholeConfigDifferentiator;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.differentiators.interfaces.Differentiator;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.ingestors.interfaces.ChangeIngestor;
|
||||
import org.eclipse.jetty.server.Request;
|
||||
import org.eclipse.jetty.server.Server;
|
||||
|
|
|
@ -17,7 +17,7 @@
|
|||
|
||||
package org.apache.nifi.minifi.bootstrap.configuration.ingestors.interfaces;
|
||||
|
||||
import org.apache.nifi.minifi.bootstrap.ConfigurationFileHolder;
|
||||
import org.apache.nifi.c2.client.api.ConfigurationFileHolder;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeNotifier;
|
||||
|
||||
import java.io.IOException;
|
||||
|
|
|
@ -0,0 +1,41 @@
|
|||
/*
|
||||
* 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.exception;
|
||||
|
||||
public class StartupFailureException extends RuntimeException {
|
||||
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
public StartupFailureException() {
|
||||
}
|
||||
|
||||
public StartupFailureException(String message) {
|
||||
super(message);
|
||||
}
|
||||
|
||||
public StartupFailureException(String message, Throwable cause) {
|
||||
super(message, cause);
|
||||
}
|
||||
|
||||
public StartupFailureException(Throwable cause) {
|
||||
super(cause);
|
||||
}
|
||||
|
||||
public StartupFailureException(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) {
|
||||
super(message, cause, enableSuppression, writableStackTrace);
|
||||
}
|
||||
}
|
|
@ -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 = Boolean.TRUE.toString();
|
||||
private static final String FALSE = Boolean.FALSE.toString();
|
||||
|
||||
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);
|
||||
}
|
||||
}
|
||||
|
||||
private void processRequest(String cmd, String[] args) throws InvalidCommandException, IOException {
|
||||
switch (cmd) {
|
||||
case "PORT":
|
||||
handlePortCommand(args);
|
||||
break;
|
||||
case "STARTED":
|
||||
handleStartedCommand(args);
|
||||
break;
|
||||
case "SHUTDOWN":
|
||||
handleShutDownCommand();
|
||||
break;
|
||||
case "RELOAD":
|
||||
handleReloadCommand();
|
||||
break;
|
||||
default:
|
||||
throw new InvalidCommandException("Unknown command: " + cmd);
|
||||
}
|
||||
}
|
||||
|
||||
private void handleReloadCommand() throws IOException {
|
||||
logger.debug("Received 'RELOAD' command from MINIFI");
|
||||
writeOk();
|
||||
}
|
||||
|
||||
private void handleShutDownCommand() throws IOException {
|
||||
logger.debug("Received 'SHUTDOWN' command from MINIFI");
|
||||
runner.shutdownChangeNotifier();
|
||||
runner.getPeriodicStatusReporterManager().shutdownPeriodicStatusReporters();
|
||||
writeOk();
|
||||
}
|
||||
|
||||
private void handleStartedCommand(String[] args) throws InvalidCommandException, IOException {
|
||||
logger.debug("Received 'STARTED' command from MINIFI");
|
||||
if (args.length != 1) {
|
||||
throw new InvalidCommandException("STARTED command must contain a status argument");
|
||||
}
|
||||
|
||||
if (!TRUE.equalsIgnoreCase(args[0]) && !FALSE.equalsIgnoreCase(args[0])) {
|
||||
throw new InvalidCommandException("Invalid status for STARTED command; should be true or false, but was '" + args[0] + "'");
|
||||
}
|
||||
|
||||
runner.getPeriodicStatusReporterManager().shutdownPeriodicStatusReporters();
|
||||
runner.getPeriodicStatusReporterManager().startPeriodicNotifiers();
|
||||
runner.getConfigurationChangeCoordinator().start();
|
||||
|
||||
runner.setNiFiStarted(Boolean.parseBoolean(args[0]));
|
||||
writeOk();
|
||||
}
|
||||
|
||||
private void handlePortCommand(String[] args) throws InvalidCommandException, IOException {
|
||||
logger.debug("Received 'PORT' command from MINIFI");
|
||||
if (args.length != 2) {
|
||||
throw new InvalidCommandException("PORT command must contain the port and secretKey arguments");
|
||||
}
|
||||
|
||||
int port;
|
||||
try {
|
||||
port = Integer.parseInt(args[0]);
|
||||
} catch (NumberFormatException nfe) {
|
||||
throw new InvalidCommandException("Invalid Port number; should be integer between 1 and 65535");
|
||||
}
|
||||
|
||||
if (port < 1 || port > 65535) {
|
||||
throw new InvalidCommandException("Invalid Port number; should be integer between 1 and 65535");
|
||||
}
|
||||
|
||||
runner.setMiNiFiParameters(port, args[1]);
|
||||
writeOk();
|
||||
}
|
||||
|
||||
private void writeOk() throws IOException {
|
||||
writer.write("OK");
|
||||
writer.newLine();
|
||||
writer.flush();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,235 @@
|
|||
/*
|
||||
* 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 static org.apache.nifi.minifi.bootstrap.SensitiveProperty.SENSITIVE_PROPERTIES;
|
||||
|
||||
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 !SENSITIVE_PROPERTIES.contains(key);
|
||||
})
|
||||
.collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)));
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,60 @@
|
|||
/*
|
||||
* 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.DEFAULT_LOGGER;
|
||||
import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.UNINITIALIZED;
|
||||
|
||||
import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
|
||||
import org.apache.nifi.minifi.bootstrap.util.UnixProcessUtils;
|
||||
|
||||
public class CurrentPortProvider {
|
||||
private final MiNiFiCommandSender miNiFiCommandSender;
|
||||
private final MiNiFiParameters miNiFiParameters;
|
||||
|
||||
public CurrentPortProvider(MiNiFiCommandSender miNiFiCommandSender, MiNiFiParameters miNiFiParameters) {
|
||||
this.miNiFiCommandSender = miNiFiCommandSender;
|
||||
this.miNiFiParameters = miNiFiParameters;
|
||||
}
|
||||
|
||||
public Integer getCurrentPort() {
|
||||
int miNiFiPort = miNiFiParameters.getMiNiFiPort();
|
||||
if (miNiFiPort == UNINITIALIZED) {
|
||||
DEFAULT_LOGGER.debug("Port is not defined");
|
||||
return null;
|
||||
}
|
||||
|
||||
DEFAULT_LOGGER.debug("Current port: {}", miNiFiPort);
|
||||
|
||||
boolean success = miNiFiCommandSender.isPingSuccessful(miNiFiPort);
|
||||
if (success) {
|
||||
DEFAULT_LOGGER.debug("Successful PING on port {}", miNiFiPort);
|
||||
return miNiFiPort;
|
||||
}
|
||||
|
||||
long minifiPid = miNiFiParameters.getMinifiPid();
|
||||
DEFAULT_LOGGER.debug("Current PID {}", minifiPid);
|
||||
|
||||
boolean procRunning = UnixProcessUtils.isProcessRunning(minifiPid);
|
||||
if (procRunning) {
|
||||
return miNiFiPort;
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,57 @@
|
|||
/*
|
||||
* 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.IOException;
|
||||
import java.util.Properties;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
public class GracefulShutdownParameterProvider {
|
||||
private static final Logger LOGGER = LoggerFactory.getLogger(GracefulShutdownParameterProvider.class);
|
||||
private static final String GRACEFUL_SHUTDOWN_PROP = "graceful.shutdown.seconds";
|
||||
private static final String DEFAULT_GRACEFUL_SHUTDOWN_VALUE = "20";
|
||||
private static final String INVALID_GRACEFUL_SHUTDOWN_SECONDS_MESSAGE =
|
||||
"The {} property in Bootstrap Config File has an invalid value. Must be a non-negative integer, Falling back to the default {} value";
|
||||
|
||||
private final BootstrapFileProvider bootstrapFileProvider;
|
||||
|
||||
public GracefulShutdownParameterProvider(BootstrapFileProvider bootstrapFileProvider) {
|
||||
this.bootstrapFileProvider = bootstrapFileProvider;
|
||||
}
|
||||
|
||||
public int getGracefulShutdownSeconds() throws IOException {
|
||||
Properties bootstrapProperties = bootstrapFileProvider.getBootstrapProperties();
|
||||
|
||||
String gracefulShutdown = bootstrapProperties.getProperty(GRACEFUL_SHUTDOWN_PROP, DEFAULT_GRACEFUL_SHUTDOWN_VALUE);
|
||||
|
||||
int gracefulShutdownSeconds;
|
||||
try {
|
||||
gracefulShutdownSeconds = Integer.parseInt(gracefulShutdown);
|
||||
} catch (NumberFormatException nfe) {
|
||||
gracefulShutdownSeconds = Integer.parseInt(DEFAULT_GRACEFUL_SHUTDOWN_VALUE);
|
||||
LOGGER.warn(INVALID_GRACEFUL_SHUTDOWN_SECONDS_MESSAGE, GRACEFUL_SHUTDOWN_PROP, gracefulShutdownSeconds);
|
||||
}
|
||||
|
||||
if (gracefulShutdownSeconds < 0) {
|
||||
gracefulShutdownSeconds = Integer.parseInt(DEFAULT_GRACEFUL_SHUTDOWN_VALUE);
|
||||
LOGGER.warn(INVALID_GRACEFUL_SHUTDOWN_SECONDS_MESSAGE, GRACEFUL_SHUTDOWN_PROP, gracefulShutdownSeconds);
|
||||
}
|
||||
return gracefulShutdownSeconds;
|
||||
}
|
||||
}
|
|
@ -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, e);
|
||||
}
|
||||
return response;
|
||||
}
|
||||
|
||||
public boolean isPingSuccessful(int port) {
|
||||
try {
|
||||
return sendCommand(PING_CMD, port).filter(PING_CMD::equals).isPresent();
|
||||
} catch (IOException ioe) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,206 @@
|
|||
/*
|
||||
* 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));
|
||||
|
||||
IOUtils.copy(configInputStream, bufferedConfigOs);
|
||||
|
||||
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);
|
||||
transformConfigurationFiles(confDir, newConfigIs, configFile, swapConfigFile);
|
||||
} 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 (Exception e) {
|
||||
throw new ConfigurationChangeException("Unable to perform reload of received configuration change", e);
|
||||
} finally {
|
||||
IOUtils.closeQuietly(configInputStream);
|
||||
handlingLock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getDescriptor() {
|
||||
return "MiNiFiConfigurationChangeListener";
|
||||
}
|
||||
|
||||
private void transformConfigurationFiles(String confDir, FileInputStream newConfigIs, File configFile, File swapConfigFile) throws Exception {
|
||||
try {
|
||||
logger.info("Performing transformation for input and saving outputs to {}", confDir);
|
||||
ByteBuffer tempConfigFile = generateConfigFiles(newConfigIs, confDir, bootstrapFileProvider.getBootstrapProperties());
|
||||
runner.getConfigFileReference().set(tempConfigFile.asReadOnlyBuffer());
|
||||
reloadNewConfiguration(swapConfigFile, confDir);
|
||||
} 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;
|
||||
}
|
||||
}
|
||||
|
||||
private void reloadNewConfiguration(File swapConfigFile, String confDir) throws Exception {
|
||||
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;
|
||||
}
|
||||
}
|
||||
|
||||
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", e);
|
||||
}
|
||||
}
|
||||
|
||||
private static boolean overrideSecurityProperties(Properties properties) {
|
||||
String overrideSecurityProperties = (String) properties.getOrDefault(OVERRIDE_SECURITY, "false");
|
||||
boolean overrideSecurity;
|
||||
if ("true".equalsIgnoreCase(overrideSecurityProperties) || "false".equalsIgnoreCase(overrideSecurityProperties)) {
|
||||
overrideSecurity = Boolean.parseBoolean(overrideSecurityProperties);
|
||||
} else {
|
||||
throw new IllegalArgumentException(
|
||||
"Property, " + OVERRIDE_SECURITY + ", to specify whether to override security properties must either be a value boolean value (\"true\" or \"false\")" +
|
||||
" or left to the default value of \"false\". It is set to \"" + overrideSecurityProperties + "\".");
|
||||
}
|
||||
|
||||
return overrideSecurity;
|
||||
}
|
||||
|
||||
private static boolean overrideCoreProperties(Properties properties) {
|
||||
String overrideCorePropertiesKey = PULL_HTTP_BASE_KEY + ".override.core";
|
||||
String overrideCoreProps = (String) properties.getOrDefault(overrideCorePropertiesKey, "false");
|
||||
boolean overrideCoreProperties;
|
||||
if ("true".equalsIgnoreCase(overrideCoreProps) || "false".equalsIgnoreCase(overrideCoreProps)) {
|
||||
overrideCoreProperties = Boolean.parseBoolean(overrideCoreProps);
|
||||
} else {
|
||||
throw new IllegalArgumentException(
|
||||
"Property, " + overrideCorePropertiesKey + ", to specify whether to override core properties must either be a value boolean value (\"true\" or \"false\")" +
|
||||
" or left to the default value of \"false\". It is set to \"" + overrideCoreProps + "\".");
|
||||
}
|
||||
|
||||
return overrideCoreProperties;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,158 @@
|
|||
/*
|
||||
* 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.CONF_DIR_KEY;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Map.Entry;
|
||||
import java.util.Optional;
|
||||
import java.util.Properties;
|
||||
|
||||
public class MiNiFiExecCommandProvider {
|
||||
|
||||
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 String DEFAULT_CONFIG_FILE = DEFAULT_CONF_DIR + "/bootstrap.conf";
|
||||
private static final String WINDOWS_FILE_EXTENSION = ".exe";
|
||||
|
||||
private final BootstrapFileProvider bootstrapFileProvider;
|
||||
|
||||
public MiNiFiExecCommandProvider(BootstrapFileProvider bootstrapFileProvider) {
|
||||
this.bootstrapFileProvider = bootstrapFileProvider;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the process arguments required for the bootstrap to start the MiNiFi process.
|
||||
*
|
||||
* @param listenPort the port where the Bootstrap process is listening
|
||||
* @param workingDir working dir of the MiNiFi
|
||||
* @return the list of arguments to start the process
|
||||
* @throws IOException throws IOException if any of the configuration file read fails
|
||||
*/
|
||||
public List<String> getMiNiFiExecCommand(int listenPort, File workingDir) throws IOException {
|
||||
Properties props = bootstrapFileProvider.getBootstrapProperties();
|
||||
File confDir = getFile(props.getProperty(CONF_DIR_KEY, DEFAULT_CONF_DIR).trim(), workingDir);
|
||||
File libDir = getFile(props.getProperty("lib.dir", DEFAULT_LIB_DIR).trim(), workingDir);
|
||||
String minifiLogDir = System.getProperty("org.apache.nifi.minifi.bootstrap.config.log.dir", DEFAULT_LOG_DIR).trim();
|
||||
|
||||
List<String> cmd = new ArrayList<>();
|
||||
cmd.add(getJavaCommand(props));
|
||||
cmd.add("-classpath");
|
||||
cmd.add(buildClassPath(props, confDir, libDir));
|
||||
cmd.addAll(getJavaAdditionalArgs(props));
|
||||
cmd.add("-Dnifi.properties.file.path=" + getMiNiFiPropsFileName(props, confDir));
|
||||
cmd.add("-Dnifi.bootstrap.listen.port=" + listenPort);
|
||||
cmd.add("-Dapp=MiNiFi");
|
||||
cmd.add("-Dorg.apache.nifi.minifi.bootstrap.config.log.dir=" + minifiLogDir);
|
||||
cmd.add("org.apache.nifi.minifi.MiNiFi");
|
||||
|
||||
return cmd;
|
||||
}
|
||||
|
||||
private String getJavaCommand(Properties props) {
|
||||
String javaCmd = props.getProperty("java");
|
||||
if (javaCmd == null) {
|
||||
javaCmd = DEFAULT_JAVA_CMD;
|
||||
}
|
||||
if (javaCmd.equals(DEFAULT_JAVA_CMD)) {
|
||||
Optional.ofNullable(System.getenv("JAVA_HOME"))
|
||||
.map(javaHome -> getJavaCommandBasedOnExtension(javaHome, WINDOWS_FILE_EXTENSION)
|
||||
.orElseGet(() -> getJavaCommandBasedOnExtension(javaHome, "").orElse(DEFAULT_JAVA_CMD)));
|
||||
}
|
||||
return javaCmd;
|
||||
}
|
||||
|
||||
private Optional<String> getJavaCommandBasedOnExtension(String javaHome, String extension) {
|
||||
String javaCmd = null;
|
||||
File javaFile = new File(javaHome + File.separatorChar + "bin" + File.separatorChar + "java" + extension);
|
||||
if (javaFile.exists() && javaFile.canExecute()) {
|
||||
javaCmd = javaFile.getAbsolutePath();
|
||||
}
|
||||
return Optional.ofNullable(javaCmd);
|
||||
}
|
||||
|
||||
private String buildClassPath(Properties props, File confDir, File libDir) {
|
||||
|
||||
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);
|
||||
}
|
||||
}
|
||||
|
||||
return classPathBuilder.toString();
|
||||
}
|
||||
|
||||
private List<String> getJavaAdditionalArgs(Properties props) {
|
||||
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);
|
||||
}
|
||||
}
|
||||
return javaAdditionalArgs;
|
||||
}
|
||||
|
||||
private String getMiNiFiPropsFileName(Properties props, File confDir) {
|
||||
String minifiPropsFilename = props.getProperty("props.file");
|
||||
if (minifiPropsFilename == null) {
|
||||
if (confDir.exists()) {
|
||||
minifiPropsFilename = new File(confDir, "nifi.properties").getAbsolutePath();
|
||||
} else {
|
||||
minifiPropsFilename = DEFAULT_CONFIG_FILE;
|
||||
}
|
||||
}
|
||||
|
||||
return minifiPropsFilename.trim();
|
||||
}
|
||||
|
||||
private File getFile(String filename, File workingDir) {
|
||||
File file = new File(filename);
|
||||
if (!file.isAbsolute()) {
|
||||
file = new File(workingDir, filename);
|
||||
}
|
||||
return file;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,147 @@
|
|||
/*
|
||||
* 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.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.net.ServerSocket;
|
||||
import java.net.Socket;
|
||||
import java.util.Optional;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import org.apache.nifi.minifi.bootstrap.RunMiNiFi;
|
||||
import org.apache.nifi.minifi.bootstrap.util.LimitingInputStream;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
public class MiNiFiListener {
|
||||
|
||||
private static final Logger LOGGER = LoggerFactory.getLogger(MiNiFiListener.class);
|
||||
|
||||
private Listener listener;
|
||||
private ServerSocket serverSocket;
|
||||
|
||||
public int start(RunMiNiFi runner) throws IOException {
|
||||
serverSocket = new ServerSocket();
|
||||
serverSocket.bind(new InetSocketAddress("localhost", 0));
|
||||
|
||||
listener = new Listener(serverSocket, runner);
|
||||
Thread listenThread = new Thread(listener);
|
||||
listenThread.setName("MiNiFi listener");
|
||||
listenThread.setDaemon(true);
|
||||
listenThread.start();
|
||||
return serverSocket.getLocalPort();
|
||||
}
|
||||
|
||||
public void stop() {
|
||||
try {
|
||||
if (serverSocket != null) {
|
||||
serverSocket.close();
|
||||
}
|
||||
} catch (IOException e) {
|
||||
LOGGER.error("Failed to close socket");
|
||||
}
|
||||
Optional.ofNullable(listener).ifPresent(Listener::stop);
|
||||
}
|
||||
|
||||
private static class Listener implements Runnable {
|
||||
|
||||
private final ServerSocket serverSocket;
|
||||
private final ExecutorService executor;
|
||||
private final RunMiNiFi runner;
|
||||
private volatile boolean stopped = false;
|
||||
|
||||
public Listener(ServerSocket serverSocket, RunMiNiFi runner) {
|
||||
this.serverSocket = serverSocket;
|
||||
this.executor = Executors.newFixedThreadPool(2, runnable -> {
|
||||
Thread t = Executors.defaultThreadFactory().newThread(runnable);
|
||||
t.setDaemon(true);
|
||||
t.setName("MiNiFi Bootstrap Command Listener");
|
||||
return t;
|
||||
});
|
||||
|
||||
this.runner = runner;
|
||||
}
|
||||
|
||||
public void stop() {
|
||||
stopped = true;
|
||||
|
||||
try {
|
||||
executor.shutdown();
|
||||
try {
|
||||
if (!executor.awaitTermination(3, TimeUnit.SECONDS)) {
|
||||
executor.shutdownNow();
|
||||
}
|
||||
} catch (InterruptedException e) {
|
||||
LOGGER.warn("Failed to stop the MiNiFi listener executor", e);
|
||||
executor.shutdownNow();
|
||||
}
|
||||
|
||||
serverSocket.close();
|
||||
} catch (IOException e) {
|
||||
LOGGER.warn("Failed to close socket", e);
|
||||
} catch (Exception e) {
|
||||
LOGGER.warn("Failed to stop the MiNiFi listener executor", e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
while (!serverSocket.isClosed()) {
|
||||
try {
|
||||
if (stopped) {
|
||||
return;
|
||||
}
|
||||
|
||||
Socket socket;
|
||||
try {
|
||||
socket = serverSocket.accept();
|
||||
} catch (IOException ioe) {
|
||||
if (stopped) {
|
||||
return;
|
||||
}
|
||||
throw ioe;
|
||||
}
|
||||
|
||||
executor.submit(() -> {
|
||||
// we want to ensure that we don't try to read data from an InputStream directly
|
||||
// by a BufferedReader because any user on the system could open a socket and send
|
||||
// a multi-gigabyte file without any new lines in order to crash the Bootstrap,
|
||||
// which in turn may cause the Shutdown Hook to shutdown MiNiFi.
|
||||
// So we will limit the amount of data to read to 4 KB
|
||||
try (InputStream limitingIn = new LimitingInputStream(socket.getInputStream(), 4096)) {
|
||||
BootstrapCodec codec = new BootstrapCodec(runner, limitingIn, socket.getOutputStream());
|
||||
codec.communicate();
|
||||
} catch (Exception t) {
|
||||
LOGGER.error("Failed to communicate with MiNiFi due to exception: ", t);
|
||||
} finally {
|
||||
try {
|
||||
socket.close();
|
||||
} catch (IOException ioe) {
|
||||
LOGGER.warn("Failed to close the socket ", ioe);
|
||||
}
|
||||
}
|
||||
});
|
||||
} catch (Exception t) {
|
||||
LOGGER.error("Failed to receive information from MiNiFi due to exception: ", t);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,48 @@
|
|||
/*
|
||||
* 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.UNINITIALIZED;
|
||||
import static org.apache.nifi.minifi.bootstrap.util.UnixProcessUtils.isProcessRunning;
|
||||
|
||||
import org.apache.nifi.minifi.bootstrap.MiNiFiStatus;
|
||||
|
||||
public class MiNiFiStatusProvider {
|
||||
|
||||
private final MiNiFiCommandSender miNiFiCommandSender;
|
||||
|
||||
public MiNiFiStatusProvider(MiNiFiCommandSender miNiFiCommandSender) {
|
||||
this.miNiFiCommandSender = miNiFiCommandSender;
|
||||
}
|
||||
|
||||
public MiNiFiStatus getStatus(int port, long pid) {
|
||||
if (port == UNINITIALIZED && pid == UNINITIALIZED) {
|
||||
return new MiNiFiStatus();
|
||||
}
|
||||
|
||||
boolean pingSuccess = false;
|
||||
if (port != UNINITIALIZED) {
|
||||
pingSuccess = miNiFiCommandSender.isPingSuccessful(port);
|
||||
}
|
||||
|
||||
if (pingSuccess) {
|
||||
return new MiNiFiStatus(port, pid, true, true);
|
||||
}
|
||||
|
||||
return new MiNiFiStatus(port, pid, false, isProcessRunning(pid));
|
||||
}
|
||||
}
|
|
@ -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.minifi.bootstrap.service;
|
||||
|
||||
import static org.apache.nifi.minifi.bootstrap.service.MiNiFiStdLogHandler.LoggerType.ERROR;
|
||||
import static org.apache.nifi.minifi.bootstrap.service.MiNiFiStdLogHandler.LoggerType.STDOUT;
|
||||
|
||||
import java.io.BufferedReader;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.InputStreamReader;
|
||||
import java.util.HashSet;
|
||||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.Future;
|
||||
import java.util.stream.Stream;
|
||||
import org.jetbrains.annotations.NotNull;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
public class MiNiFiStdLogHandler {
|
||||
|
||||
private static final Logger LOGGER = LoggerFactory.getLogger(MiNiFiStdLogHandler.class);
|
||||
private static final String READ_FAILURE_MESSAGE = "Failed to read from MiNiFi's Standard {} stream";
|
||||
private static final String EXCEPTION_MESSAGE = "Exception: ";
|
||||
|
||||
private final ExecutorService loggingExecutor;
|
||||
private Set<Future<?>> loggingFutures;
|
||||
|
||||
public MiNiFiStdLogHandler() {
|
||||
loggingExecutor = Executors.newFixedThreadPool(2, runnable -> {
|
||||
Thread t = Executors.defaultThreadFactory().newThread(runnable);
|
||||
t.setDaemon(true);
|
||||
t.setName("MiNiFi logging handler");
|
||||
return t;
|
||||
});
|
||||
}
|
||||
|
||||
public void initLogging(Process process) {
|
||||
LOGGER.debug("Initializing MiNiFi's standard output/error loggers...");
|
||||
Optional.ofNullable(loggingFutures)
|
||||
.map(Set::stream)
|
||||
.orElse(Stream.empty())
|
||||
.forEach(future -> future.cancel(false));
|
||||
|
||||
Set<Future<?>> futures = new HashSet<>();
|
||||
futures.add(getFuture(process.getInputStream(), STDOUT));
|
||||
futures.add(getFuture(process.getErrorStream(), ERROR));
|
||||
loggingFutures = futures;
|
||||
}
|
||||
|
||||
@NotNull
|
||||
private Future<?> getFuture(InputStream in, LoggerType loggerType) {
|
||||
return loggingExecutor.submit(() -> {
|
||||
Logger logger = LoggerFactory.getLogger(loggerType.getLoggerName());
|
||||
try (BufferedReader reader = new BufferedReader(new InputStreamReader(in))) {
|
||||
String line;
|
||||
while ((line = reader.readLine()) != null) {
|
||||
if (loggerType == ERROR) {
|
||||
logger.error(line);
|
||||
} else {
|
||||
logger.info(line);
|
||||
}
|
||||
}
|
||||
} catch (IOException e) {
|
||||
LOGGER.warn(READ_FAILURE_MESSAGE, loggerType.getDisplayName());
|
||||
LOGGER.warn(EXCEPTION_MESSAGE, e);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
public void shutdown() {
|
||||
LOGGER.debug("Shutting down MiNiFi's standard output/error loggers...");
|
||||
loggingExecutor.shutdown();
|
||||
}
|
||||
|
||||
enum LoggerType {
|
||||
STDOUT("Output", "org.apache.nifi.minifi.StdOut"),
|
||||
ERROR("Error", "org.apache.nifi.minifi.StdErr");
|
||||
|
||||
final String displayName;
|
||||
final String loggerName;
|
||||
|
||||
LoggerType(String displayName, String loggerName) {
|
||||
this.displayName = displayName;
|
||||
this.loggerName = loggerName;
|
||||
}
|
||||
|
||||
public String getDisplayName() {
|
||||
return displayName;
|
||||
}
|
||||
|
||||
public String getLoggerName() {
|
||||
return loggerName;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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, e);
|
||||
}
|
||||
return flowStatusReport;
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,84 @@
|
|||
/*
|
||||
* 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.DEFAULT_LOGGER;
|
||||
import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.UNINITIALIZED;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.Optional;
|
||||
import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
|
||||
import org.apache.nifi.minifi.bootstrap.RunMiNiFi;
|
||||
import org.apache.nifi.minifi.bootstrap.util.UnixProcessUtils;
|
||||
|
||||
public class ReloadService {
|
||||
private final BootstrapFileProvider bootstrapFileProvider;
|
||||
private final MiNiFiParameters miNiFiParameters;
|
||||
private final MiNiFiCommandSender miNiFiCommandSender;
|
||||
private static final String RELOAD_CMD = "RELOAD";
|
||||
private final CurrentPortProvider currentPortProvider;
|
||||
private final GracefulShutdownParameterProvider gracefulShutdownParameterProvider;
|
||||
private final RunMiNiFi runMiNiFi;
|
||||
|
||||
public ReloadService(BootstrapFileProvider bootstrapFileProvider, MiNiFiParameters miNiFiParameters,
|
||||
MiNiFiCommandSender miNiFiCommandSender, CurrentPortProvider currentPortProvider,
|
||||
GracefulShutdownParameterProvider gracefulShutdownParameterProvider, RunMiNiFi runMiNiFi) {
|
||||
this.bootstrapFileProvider = bootstrapFileProvider;
|
||||
this.miNiFiParameters = miNiFiParameters;
|
||||
this.miNiFiCommandSender = miNiFiCommandSender;
|
||||
this.currentPortProvider = currentPortProvider;
|
||||
this.gracefulShutdownParameterProvider = gracefulShutdownParameterProvider;
|
||||
this.runMiNiFi = runMiNiFi;
|
||||
}
|
||||
|
||||
public void reload() throws IOException {
|
||||
// indicate that a reload command is in progress
|
||||
File reloadLockFile = bootstrapFileProvider.getReloadLockFile();
|
||||
if (!reloadLockFile.exists()) {
|
||||
reloadLockFile.createNewFile();
|
||||
}
|
||||
|
||||
long minifiPid = miNiFiParameters.getMinifiPid();
|
||||
try {
|
||||
Optional<String> commandResponse = miNiFiCommandSender.sendCommand(RELOAD_CMD, currentPortProvider.getCurrentPort());
|
||||
if (commandResponse.filter(RELOAD_CMD::equals).isPresent()) {
|
||||
DEFAULT_LOGGER.info("Apache MiNiFi has accepted the Reload Command and is reloading");
|
||||
if (minifiPid != UNINITIALIZED) {
|
||||
UnixProcessUtils.gracefulShutDownMiNiFiProcess(minifiPid, "MiNiFi has not finished shutting down after {} seconds as part of configuration reload. Killing process.",
|
||||
gracefulShutdownParameterProvider.getGracefulShutdownSeconds());
|
||||
runMiNiFi.setReloading(true);
|
||||
DEFAULT_LOGGER.info("MiNiFi has finished shutting down and will be reloaded.");
|
||||
}
|
||||
} else {
|
||||
DEFAULT_LOGGER.error("When sending RELOAD command to MiNiFi, got unexpected response {}.", commandResponse.orElse(null));
|
||||
}
|
||||
} catch (IOException e) {
|
||||
if (minifiPid == UNINITIALIZED) {
|
||||
DEFAULT_LOGGER.error("No PID found for the MiNiFi process, so unable to kill process; The process should be killed manually.");
|
||||
} else {
|
||||
DEFAULT_LOGGER.error("Will kill the MiNiFi Process with PID {}", minifiPid);
|
||||
UnixProcessUtils.killProcessTree(minifiPid);
|
||||
}
|
||||
} finally {
|
||||
if (reloadLockFile.exists() && !reloadLockFile.delete()) {
|
||||
DEFAULT_LOGGER.error("Failed to delete reload lock file {}; this file should be cleaned up manually", reloadLockFile);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -17,6 +17,29 @@
|
|||
|
||||
package org.apache.nifi.minifi.bootstrap.util;
|
||||
|
||||
import java.io.FileOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
import java.nio.file.Paths;
|
||||
import java.security.SecureRandom;
|
||||
import java.util.Arrays;
|
||||
import java.util.Base64;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.Properties;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.zip.GZIPOutputStream;
|
||||
import javax.xml.transform.dom.DOMSource;
|
||||
import javax.xml.transform.stream.StreamResult;
|
||||
import org.apache.commons.io.input.TeeInputStream;
|
||||
import org.apache.commons.io.output.ByteArrayOutputStream;
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeException;
|
||||
|
@ -55,26 +78,6 @@ import org.w3c.dom.DOMException;
|
|||
import org.w3c.dom.Document;
|
||||
import org.w3c.dom.Element;
|
||||
|
||||
import javax.xml.transform.dom.DOMSource;
|
||||
import javax.xml.transform.stream.StreamResult;
|
||||
import java.io.FileOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
import java.nio.file.Paths;
|
||||
import java.security.SecureRandom;
|
||||
import java.util.Arrays;
|
||||
import java.util.Base64;
|
||||
import java.util.Collection;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.zip.GZIPOutputStream;
|
||||
|
||||
public final class ConfigTransformer {
|
||||
// Underlying version of NIFI will be using
|
||||
public static final String ROOT_GROUP = "Root-Group";
|
||||
|
@ -88,42 +91,68 @@ public final class ConfigTransformer {
|
|||
private ConfigTransformer() {
|
||||
}
|
||||
|
||||
public static ByteBuffer generateConfigFiles(InputStream configIs, String configDestinationPath, Properties bootstrapProperties) throws ConfigurationChangeException, IOException {
|
||||
try (java.io.ByteArrayOutputStream byteArrayOutputStream = new java.io.ByteArrayOutputStream();
|
||||
TeeInputStream teeInputStream = new TeeInputStream(configIs, byteArrayOutputStream)) {
|
||||
|
||||
ConfigTransformer.transformConfigFile(
|
||||
teeInputStream,
|
||||
configDestinationPath,
|
||||
bootstrapProperties
|
||||
);
|
||||
|
||||
return ByteBuffer.wrap(byteArrayOutputStream.toByteArray());
|
||||
} catch (ConfigurationChangeException e){
|
||||
throw e;
|
||||
} catch (Exception e) {
|
||||
throw new IOException("Unable to successfully transform the provided configuration", e);
|
||||
}
|
||||
}
|
||||
|
||||
public static void transformConfigFile(InputStream sourceStream, String destPath, Properties bootstrapProperties) throws Exception {
|
||||
ConvertableSchema<ConfigSchema> convertableSchema = throwIfInvalid(SchemaLoader.loadConvertableSchemaFromYaml(sourceStream));
|
||||
ConfigSchema configSchema = throwIfInvalid(convertableSchema.convert());
|
||||
ConvertableSchema<ConfigSchema> convertableSchemaNew = throwIfInvalid(SchemaLoader.loadConvertableSchemaFromYaml(sourceStream));
|
||||
ConfigSchema configSchemaNew = throwIfInvalid(convertableSchemaNew.convert());
|
||||
|
||||
SecurityPropertiesSchema securityProperties = BootstrapTransformer.buildSecurityPropertiesFromBootstrap(bootstrapProperties).orElse(null);
|
||||
ProvenanceReportingSchema provenanceReportingProperties = BootstrapTransformer.buildProvenanceReportingPropertiesFromBootstrap(bootstrapProperties).orElse(null);
|
||||
|
||||
// See if we are providing defined properties from the filesystem configurations and use those as the definitive values
|
||||
if (securityProperties != null) {
|
||||
configSchema.setSecurityProperties(securityProperties);
|
||||
configSchemaNew.setSecurityProperties(securityProperties);
|
||||
logger.info("Bootstrap flow override: Replaced security properties");
|
||||
}
|
||||
|
||||
if (provenanceReportingProperties != null) {
|
||||
configSchema.setProvenanceReportingProperties(provenanceReportingProperties);
|
||||
configSchemaNew.setProvenanceReportingProperties(provenanceReportingProperties);
|
||||
logger.info("Bootstrap flow override: Replaced provenance reporting properties");
|
||||
}
|
||||
|
||||
// Replace all processor SSL controller services with MiNiFi parent, if bootstrap boolean is set to true
|
||||
if (BootstrapTransformer.processorSSLOverride(bootstrapProperties)) {
|
||||
for (ProcessorSchema processorConfig : configSchema.getProcessGroupSchema().getProcessors()) {
|
||||
for (ProcessorSchema processorConfig : configSchemaNew.getProcessGroupSchema().getProcessors()) {
|
||||
processorConfig.getProperties().replace("SSL Context Service", processorConfig.getProperties().get("SSL Context Service"), "SSL-Context-Service");
|
||||
logger.info("Bootstrap flow override: Replaced {} SSL Context Service with parent MiNiFi SSL", processorConfig.getName());
|
||||
}
|
||||
}
|
||||
|
||||
Optional.ofNullable(bootstrapProperties)
|
||||
.map(Properties::entrySet)
|
||||
.orElse(Collections.emptySet())
|
||||
.stream()
|
||||
.filter(entry -> ((String) entry.getKey()).startsWith("c2"))
|
||||
.forEach(entry -> configSchemaNew.getNifiPropertiesOverrides().putIfAbsent((String) entry.getKey(), (String) entry.getValue()));
|
||||
|
||||
// Create nifi.properties and flow.xml.gz in memory
|
||||
ByteArrayOutputStream nifiPropertiesOutputStream = new ByteArrayOutputStream();
|
||||
writeNiFiProperties(configSchema, nifiPropertiesOutputStream);
|
||||
writeNiFiProperties(configSchemaNew, nifiPropertiesOutputStream);
|
||||
|
||||
writeFlowXmlFile(configSchema, destPath);
|
||||
writeFlowXmlFile(configSchemaNew, destPath);
|
||||
|
||||
// Write nifi.properties and flow.xml.gz
|
||||
writeNiFiPropertiesFile(nifiPropertiesOutputStream, destPath);
|
||||
}
|
||||
|
||||
private static <T extends Schema> T throwIfInvalid(T schema) throws InvalidConfigurationException {
|
||||
public static <T extends Schema> T throwIfInvalid(T schema) throws InvalidConfigurationException {
|
||||
if (!schema.isValid()) {
|
||||
throw new InvalidConfigurationException("Failed to transform config file due to:["
|
||||
+ schema.getValidationIssues().stream().sorted().collect(Collectors.joining("], [")) + "]");
|
||||
|
@ -705,8 +734,8 @@ public final class ConfigTransformer {
|
|||
|
||||
protected static void addPosition(final Element parentElement) {
|
||||
final Element element = parentElement.getOwnerDocument().createElement("position");
|
||||
element.setAttribute("x", String.valueOf("0"));
|
||||
element.setAttribute("y", String.valueOf("0"));
|
||||
element.setAttribute("x", "0");
|
||||
element.setAttribute("y", "0");
|
||||
parentElement.appendChild(element);
|
||||
}
|
||||
|
||||
|
|
|
@ -0,0 +1,147 @@
|
|||
/*
|
||||
* 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;
|
||||
|
||||
/*
|
||||
* Utility class for providing information about the running MiNiFi process.
|
||||
* The methods which are using the PID are working only on unix systems, and should be used only as a fallback in case the PING command fails.
|
||||
* */
|
||||
public class UnixProcessUtils {
|
||||
private static final Logger LOGGER = LoggerFactory.getLogger(UnixProcessUtils.class);
|
||||
|
||||
public static boolean isProcessRunning(Long pid) {
|
||||
if (pid == null) {
|
||||
LOGGER.error("Unable to get process status due to missing process id");
|
||||
return false;
|
||||
}
|
||||
try {
|
||||
// We use the "ps" command to check if the process is still running.
|
||||
ProcessBuilder builder = new ProcessBuilder();
|
||||
String pidString = String.valueOf(pid);
|
||||
|
||||
builder.command("ps", "-p", pidString);
|
||||
Process proc = builder.start();
|
||||
|
||||
// Look for the pid in the output of the 'ps' command.
|
||||
boolean running = false;
|
||||
String line;
|
||||
try (InputStream in = proc.getInputStream();
|
||||
Reader streamReader = new InputStreamReader(in);
|
||||
BufferedReader reader = new BufferedReader(streamReader)) {
|
||||
|
||||
while ((line = reader.readLine()) != null) {
|
||||
if (line.trim().startsWith(pidString)) {
|
||||
running = true;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// If output of the ps command had our PID, the process is running.
|
||||
LOGGER.debug("Process with PID {} is {}running", pid, running ? "" : "not ");
|
||||
|
||||
return running;
|
||||
} catch (IOException ioe) {
|
||||
LOGGER.error("Failed to determine if Process {} is running; assuming that it is not", pid);
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
public static void gracefulShutDownMiNiFiProcess(Long pid, String s, int gracefulShutdownSeconds) {
|
||||
long startWait = System.nanoTime();
|
||||
while (UnixProcessUtils.isProcessRunning(pid)) {
|
||||
LOGGER.info("Waiting for Apache MiNiFi to finish shutting down...");
|
||||
long waitNanos = System.nanoTime() - startWait;
|
||||
long waitSeconds = TimeUnit.NANOSECONDS.toSeconds(waitNanos);
|
||||
if (waitSeconds >= gracefulShutdownSeconds || gracefulShutdownSeconds == 0) {
|
||||
if (UnixProcessUtils.isProcessRunning(pid)) {
|
||||
LOGGER.warn(s, gracefulShutdownSeconds);
|
||||
try {
|
||||
UnixProcessUtils.killProcessTree(pid);
|
||||
} catch (IOException ioe) {
|
||||
LOGGER.error("Failed to kill Process with PID {}", pid);
|
||||
}
|
||||
}
|
||||
break;
|
||||
} else {
|
||||
try {
|
||||
Thread.sleep(2000L);
|
||||
} catch (InterruptedException ie) {
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public static void killProcessTree(Long pid) throws IOException {
|
||||
LOGGER.debug("Killing Process Tree for PID {}", pid);
|
||||
|
||||
List<Long> children = getChildProcesses(pid);
|
||||
LOGGER.debug("Children of PID {}: {}", pid, children);
|
||||
|
||||
for (Long childPid : children) {
|
||||
killProcessTree(childPid);
|
||||
}
|
||||
|
||||
Runtime.getRuntime().exec(new String[]{"kill", "-9", String.valueOf(pid)});
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks the status of the given process.
|
||||
*
|
||||
* @param process the process object what we want to check
|
||||
* @return true if the process is Alive
|
||||
*/
|
||||
public static boolean isAlive(Process process) {
|
||||
try {
|
||||
process.exitValue();
|
||||
return false;
|
||||
} catch (IllegalStateException | IllegalThreadStateException itse) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
private static List<Long> getChildProcesses(Long ppid) throws IOException {
|
||||
Process proc = Runtime.getRuntime().exec(new String[]{"ps", "-o", "pid", "--no-headers", "--ppid", String.valueOf(ppid)});
|
||||
List<Long> childPids = new ArrayList<>();
|
||||
try (InputStream in = proc.getInputStream();
|
||||
BufferedReader reader = new BufferedReader(new InputStreamReader(in))) {
|
||||
|
||||
String line;
|
||||
while ((line = reader.readLine()) != null) {
|
||||
try {
|
||||
Long childPid = Long.valueOf(line.trim());
|
||||
childPids.add(childPid);
|
||||
} catch (NumberFormatException e) {
|
||||
LOGGER.trace("Failed to parse PID", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return childPids;
|
||||
}
|
||||
}
|
|
@ -36,7 +36,6 @@ public class RunMiNiFiTest {
|
|||
|
||||
@Test
|
||||
public void buildSecurityPropertiesNotDefined() throws Exception {
|
||||
final RunMiNiFi testMiNiFi = new RunMiNiFi(null);
|
||||
final Properties bootstrapProperties = getTestBootstrapProperties("bootstrap-ssl-ctx/bootstrap.conf.default");
|
||||
final Optional<SecurityPropertiesSchema> securityPropsOptional = BootstrapTransformer.buildSecurityPropertiesFromBootstrap(bootstrapProperties);
|
||||
assertFalse(securityPropsOptional.isPresent());
|
||||
|
@ -44,7 +43,6 @@ public class RunMiNiFiTest {
|
|||
|
||||
@Test
|
||||
public void buildSecurityPropertiesDefined() throws Exception {
|
||||
final RunMiNiFi testMiNiFi = new RunMiNiFi(null);
|
||||
final Properties bootstrapProperties = getTestBootstrapProperties("bootstrap-ssl-ctx/bootstrap.conf.configured");
|
||||
final Optional<SecurityPropertiesSchema> securityPropsOptional = BootstrapTransformer.buildSecurityPropertiesFromBootstrap(bootstrapProperties);
|
||||
assertTrue(securityPropsOptional.isPresent());
|
||||
|
@ -73,7 +71,6 @@ public class RunMiNiFiTest {
|
|||
|
||||
@Test
|
||||
public void buildSecurityPropertiesDefinedButInvalid() throws Exception {
|
||||
final RunMiNiFi testMiNiFi = new RunMiNiFi(null);
|
||||
final Properties bootstrapProperties = getTestBootstrapProperties("bootstrap-ssl-ctx/bootstrap.conf.configured.invalid");
|
||||
final Optional<SecurityPropertiesSchema> securityPropsOptional = BootstrapTransformer.buildSecurityPropertiesFromBootstrap(bootstrapProperties);
|
||||
assertTrue(securityPropsOptional.isPresent());
|
||||
|
@ -99,7 +96,6 @@ public class RunMiNiFiTest {
|
|||
|
||||
@Test
|
||||
public void buildProvenanceReportingNotDefined() throws Exception {
|
||||
final RunMiNiFi testMiNiFi = new RunMiNiFi(null);
|
||||
final Properties bootstrapProperties = getTestBootstrapProperties("bootstrap-provenance-reporting/bootstrap.conf.default");
|
||||
final Optional<ProvenanceReportingSchema> provenanceReportingPropsOptional = BootstrapTransformer.buildProvenanceReportingPropertiesFromBootstrap(bootstrapProperties);
|
||||
assertFalse(provenanceReportingPropsOptional.isPresent());
|
||||
|
@ -107,7 +103,6 @@ public class RunMiNiFiTest {
|
|||
|
||||
@Test
|
||||
public void buildProvenanceReportingDefined() throws Exception {
|
||||
final RunMiNiFi testMiNiFi = new RunMiNiFi(null);
|
||||
final Properties bootstrapProperties = getTestBootstrapProperties("bootstrap-provenance-reporting/bootstrap.conf.configured");
|
||||
final Optional<ProvenanceReportingSchema> provenanceReportingPropsOptional = BootstrapTransformer.buildProvenanceReportingPropertiesFromBootstrap(bootstrapProperties);
|
||||
assertTrue(provenanceReportingPropsOptional.isPresent());
|
||||
|
|
|
@ -1,84 +0,0 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.nifi.minifi.bootstrap.configuration;
|
||||
|
||||
import org.apache.nifi.minifi.bootstrap.ConfigurationFileHolder;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
import java.io.InputStream;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.Properties;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.mockito.Mockito.verify;
|
||||
|
||||
public class ConfigurationChangeCoordinatorTest {
|
||||
|
||||
private ConfigurationChangeCoordinator coordinatorSpy;
|
||||
private final Properties properties = new Properties();
|
||||
|
||||
@BeforeEach
|
||||
public void setUp() {
|
||||
coordinatorSpy = Mockito.spy(new ConfigurationChangeCoordinator());
|
||||
}
|
||||
|
||||
@AfterEach
|
||||
public void tearDown() throws Exception {
|
||||
coordinatorSpy.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInit() {
|
||||
properties.put("nifi.minifi.notifier.ingestors", "org.apache.nifi.minifi.bootstrap.configuration.ingestors.RestChangeIngestor");
|
||||
final ConfigurationChangeListener testListener = Mockito.mock(ConfigurationChangeListener.class);
|
||||
coordinatorSpy.initialize(properties, Mockito.mock(ConfigurationFileHolder.class), Collections.singleton(testListener));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNotifyListeners() throws Exception {
|
||||
final ConfigurationChangeListener testListener = Mockito.mock(ConfigurationChangeListener.class);
|
||||
coordinatorSpy.initialize(properties, Mockito.mock(ConfigurationFileHolder.class), Collections.singleton(testListener));
|
||||
|
||||
assertEquals(coordinatorSpy.getChangeListeners().size(), 1, "Did not receive the correct number of registered listeners");
|
||||
|
||||
coordinatorSpy.notifyListeners(ByteBuffer.allocate(1));
|
||||
|
||||
verify(testListener, Mockito.atMost(1)).handleChange(Mockito.any(InputStream.class));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRegisterListener() {
|
||||
final ConfigurationChangeListener firstListener = Mockito.mock(ConfigurationChangeListener.class);
|
||||
coordinatorSpy.initialize(properties, Mockito.mock(ConfigurationFileHolder.class), Collections.singleton(firstListener));
|
||||
|
||||
assertEquals(coordinatorSpy.getChangeListeners().size(), 1, "Did not receive the correct number of registered listeners");
|
||||
|
||||
coordinatorSpy.initialize(properties, Mockito.mock(ConfigurationFileHolder.class), Arrays.asList(firstListener, firstListener));
|
||||
assertEquals(coordinatorSpy.getChangeListeners().size(), 1, "Did not receive the correct number of registered listeners");
|
||||
|
||||
final ConfigurationChangeListener secondListener = Mockito.mock(ConfigurationChangeListener.class);
|
||||
coordinatorSpy.initialize(properties, Mockito.mock(ConfigurationFileHolder.class), Arrays.asList(firstListener, secondListener));
|
||||
assertEquals(coordinatorSpy.getChangeListeners().size(), 2, "Did not receive the correct number of registered listeners");
|
||||
|
||||
}
|
||||
}
|
|
@ -17,13 +17,9 @@
|
|||
|
||||
package org.apache.nifi.minifi.bootstrap.configuration.differentiators;
|
||||
|
||||
import okhttp3.Request;
|
||||
import org.apache.commons.io.FileUtils;
|
||||
import org.apache.nifi.minifi.bootstrap.ConfigurationFileHolder;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.differentiators.interfaces.Differentiator;
|
||||
import org.junit.jupiter.api.BeforeAll;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.mockito.Mockito;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
import java.io.FileInputStream;
|
||||
import java.io.IOException;
|
||||
|
@ -33,10 +29,13 @@ import java.nio.file.Path;
|
|||
import java.nio.file.Paths;
|
||||
import java.util.Properties;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
import static org.mockito.Mockito.when;
|
||||
import okhttp3.Request;
|
||||
import org.apache.commons.io.FileUtils;
|
||||
import org.apache.nifi.c2.client.api.ConfigurationFileHolder;
|
||||
import org.apache.nifi.c2.client.api.Differentiator;
|
||||
import org.junit.jupiter.api.BeforeAll;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
public class WholeConfigDifferentiatorTest {
|
||||
|
||||
|
|
|
@ -28,13 +28,11 @@ import java.nio.file.Paths;
|
|||
import java.nio.file.WatchEvent;
|
||||
import java.nio.file.WatchKey;
|
||||
import java.nio.file.WatchService;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Collections;
|
||||
import java.util.Properties;
|
||||
|
||||
import org.apache.nifi.minifi.bootstrap.ConfigurationFileHolder;
|
||||
import org.apache.nifi.c2.client.api.ConfigurationFileHolder;
|
||||
import org.apache.nifi.c2.client.api.Differentiator;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeNotifier;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.differentiators.interfaces.Differentiator;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
@ -140,21 +138,14 @@ public class FileChangeIngestorTest {
|
|||
|
||||
/* Helper methods to establish mock environment */
|
||||
private WatchKey createMockWatchKeyForPath(String configFilePath) {
|
||||
final WatchKey mockWatchKey = Mockito.mock(WatchKey.class);
|
||||
final List<WatchEvent<?>> mockWatchEvents = (List<WatchEvent<?>>) Mockito.mock(List.class);
|
||||
when(mockWatchKey.pollEvents()).thenReturn(mockWatchEvents);
|
||||
when(mockWatchKey.reset()).thenReturn(true);
|
||||
|
||||
final Iterator mockIterator = Mockito.mock(Iterator.class);
|
||||
when(mockWatchEvents.iterator()).thenReturn(mockIterator);
|
||||
|
||||
final WatchEvent mockWatchEvent = Mockito.mock(WatchEvent.class);
|
||||
when(mockIterator.hasNext()).thenReturn(true, false);
|
||||
when(mockIterator.next()).thenReturn(mockWatchEvent);
|
||||
WatchKey mockWatchKey = Mockito.mock(WatchKey.class);
|
||||
WatchEvent mockWatchEvent = Mockito.mock(WatchEvent.class);
|
||||
|
||||
// In this case, we receive a trigger event for the directory monitored, and it was the file monitored
|
||||
when(mockWatchEvent.context()).thenReturn(Paths.get(configFilePath));
|
||||
when(mockWatchEvent.kind()).thenReturn(ENTRY_MODIFY);
|
||||
when(mockWatchKey.pollEvents()).thenReturn(Collections.singletonList(mockWatchEvent));
|
||||
when(mockWatchKey.reset()).thenReturn(true);
|
||||
|
||||
return mockWatchKey;
|
||||
}
|
||||
|
|
|
@ -17,7 +17,7 @@
|
|||
|
||||
package org.apache.nifi.minifi.bootstrap.configuration.ingestors;
|
||||
|
||||
import org.apache.nifi.minifi.bootstrap.ConfigurationFileHolder;
|
||||
import org.apache.nifi.c2.client.api.ConfigurationFileHolder;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.ingestors.common.PullHttpChangeIngestorCommonTest;
|
||||
import org.eclipse.jetty.server.ServerConnector;
|
||||
import org.eclipse.jetty.util.ssl.SslContextFactory;
|
||||
|
|
|
@ -17,7 +17,7 @@
|
|||
|
||||
package org.apache.nifi.minifi.bootstrap.configuration.ingestors;
|
||||
|
||||
import org.apache.nifi.minifi.bootstrap.ConfigurationFileHolder;
|
||||
import org.apache.nifi.c2.client.api.ConfigurationFileHolder;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.ingestors.common.PullHttpChangeIngestorCommonTest;
|
||||
import org.eclipse.jetty.server.ServerConnector;
|
||||
import org.junit.jupiter.api.BeforeAll;
|
||||
|
|
|
@ -18,7 +18,7 @@
|
|||
package org.apache.nifi.minifi.bootstrap.configuration.ingestors;
|
||||
|
||||
import okhttp3.OkHttpClient;
|
||||
import org.apache.nifi.minifi.bootstrap.ConfigurationFileHolder;
|
||||
import org.apache.nifi.c2.client.api.ConfigurationFileHolder;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeListener;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeNotifier;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.ListenerHandleResult;
|
||||
|
|
|
@ -18,7 +18,7 @@
|
|||
package org.apache.nifi.minifi.bootstrap.configuration.ingestors;
|
||||
|
||||
import okhttp3.OkHttpClient;
|
||||
import org.apache.nifi.minifi.bootstrap.ConfigurationFileHolder;
|
||||
import org.apache.nifi.c2.client.api.ConfigurationFileHolder;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeNotifier;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.ingestors.common.RestChangeIngestorCommonTest;
|
||||
import org.junit.jupiter.api.AfterAll;
|
||||
|
|
|
@ -17,11 +17,11 @@
|
|||
|
||||
package org.apache.nifi.minifi.bootstrap.configuration.ingestors.common;
|
||||
|
||||
import org.apache.nifi.c2.client.api.Differentiator;
|
||||
import org.apache.nifi.minifi.bootstrap.RunMiNiFi;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeListener;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeNotifier;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.ListenerHandleResult;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.differentiators.interfaces.Differentiator;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.ingestors.PullHttpChangeIngestor;
|
||||
import org.apache.nifi.minifi.bootstrap.util.ByteBufferInputStream;
|
||||
import org.apache.nifi.minifi.commons.schema.ConfigSchema;
|
||||
|
|
|
@ -23,10 +23,10 @@ import okhttp3.OkHttpClient;
|
|||
import okhttp3.Request;
|
||||
import okhttp3.RequestBody;
|
||||
import okhttp3.Response;
|
||||
import org.apache.nifi.c2.client.api.Differentiator;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeListener;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeNotifier;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.ListenerHandleResult;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.differentiators.interfaces.Differentiator;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.ingestors.RestChangeIngestor;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
|
|
@ -17,9 +17,11 @@
|
|||
|
||||
package org.apache.nifi.minifi.commons.schema.common;
|
||||
|
||||
import org.apache.nifi.minifi.commons.schema.ProvenanceReportingSchema;
|
||||
import org.apache.nifi.minifi.commons.schema.SecurityPropertiesSchema;
|
||||
import org.apache.nifi.minifi.commons.schema.SensitivePropsSchema;
|
||||
import static org.apache.nifi.minifi.commons.schema.RemoteProcessGroupSchema.TIMEOUT_KEY;
|
||||
import static org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys.COMMENT_KEY;
|
||||
import static org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys.SCHEDULING_PERIOD_KEY;
|
||||
import static org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys.SCHEDULING_STRATEGY_KEY;
|
||||
import static org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys.USE_COMPRESSION_KEY;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
|
@ -27,18 +29,12 @@ import java.util.HashMap;
|
|||
import java.util.HashSet;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
import static org.apache.nifi.minifi.commons.schema.RemoteProcessGroupSchema.TIMEOUT_KEY;
|
||||
import static org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys.COMMENT_KEY;
|
||||
import static org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys.SCHEDULING_PERIOD_KEY;
|
||||
import static org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys.SCHEDULING_STRATEGY_KEY;
|
||||
import static org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys.USE_COMPRESSION_KEY;
|
||||
import org.apache.nifi.minifi.commons.schema.ProvenanceReportingSchema;
|
||||
import org.apache.nifi.minifi.commons.schema.SecurityPropertiesSchema;
|
||||
import org.apache.nifi.minifi.commons.schema.SensitivePropsSchema;
|
||||
|
||||
public class BootstrapPropertyKeys {
|
||||
|
||||
public static final String NOTIFIER_PROPERTY_PREFIX = "nifi.minifi.notifier";
|
||||
public static final String NOTIFIER_COMPONENTS_KEY = NOTIFIER_PROPERTY_PREFIX + ".components";
|
||||
|
||||
public static final String STATUS_REPORTER_PROPERTY_PREFIX = "nifi.minifi.status.reporter";
|
||||
public static final String STATUS_REPORTER_COMPONENTS_KEY = STATUS_REPORTER_PROPERTY_PREFIX + ".components";
|
||||
|
||||
|
|
|
@ -17,12 +17,14 @@
|
|||
|
||||
package org.apache.nifi.minifi.commons.schema.serialization;
|
||||
|
||||
import java.io.Writer;
|
||||
import org.apache.nifi.minifi.commons.schema.ConfigSchema;
|
||||
import org.apache.nifi.minifi.commons.schema.common.ConvertableSchema;
|
||||
import org.apache.nifi.minifi.commons.schema.common.StringUtil;
|
||||
import org.apache.nifi.minifi.commons.schema.exception.SchemaLoaderException;
|
||||
import org.apache.nifi.minifi.commons.schema.v1.ConfigSchemaV1;
|
||||
import org.apache.nifi.minifi.commons.schema.v2.ConfigSchemaV2;
|
||||
import org.yaml.snakeyaml.DumperOptions;
|
||||
import org.yaml.snakeyaml.Yaml;
|
||||
import org.yaml.snakeyaml.error.YAMLException;
|
||||
|
||||
|
@ -67,6 +69,15 @@ public class SchemaLoader {
|
|||
}
|
||||
}
|
||||
|
||||
public static void toYaml(ConfigSchema schema, Writer writer) {
|
||||
final DumperOptions options = new DumperOptions();
|
||||
options.setDefaultFlowStyle(DumperOptions.FlowStyle.BLOCK);
|
||||
options.setPrettyFlow(true);
|
||||
|
||||
Yaml yaml = new Yaml(options);
|
||||
yaml.dump(schema.toMap(), writer);
|
||||
}
|
||||
|
||||
public static ConfigSchema loadConfigSchemaFromYaml(InputStream sourceStream) throws IOException, SchemaLoaderException {
|
||||
return loadConfigSchemaFromYaml(loadYamlAsMap(sourceStream));
|
||||
}
|
||||
|
|
|
@ -16,7 +16,9 @@
|
|||
# under the License.
|
||||
#
|
||||
|
||||
FROM openjdk:8-jre-alpine
|
||||
ARG IMAGE_NAME=openjdk
|
||||
ARG IMAGE_TAG=8-jre-alpine
|
||||
FROM ${IMAGE_NAME}:${IMAGE_TAG}
|
||||
MAINTAINER Apache MiNiFi <dev@nifi.apache.org>
|
||||
|
||||
# Values are set by Maven
|
||||
|
|
|
@ -30,6 +30,8 @@ limitations under the License.
|
|||
|
||||
<properties>
|
||||
<minifi.version>${project.version}</minifi.version>
|
||||
<docker.image.name>openjdk</docker.image.name>
|
||||
<docker.image.tag>8-jre-alpine</docker.image.tag>
|
||||
</properties>
|
||||
|
||||
<profiles>
|
||||
|
@ -57,6 +59,8 @@ limitations under the License.
|
|||
<imageTag>${minifi.version}</imageTag>
|
||||
</imageTags>
|
||||
<buildArgs>
|
||||
<IMAGE_NAME>${docker.image.name}</IMAGE_NAME>
|
||||
<IMAGE_TAG>${docker.image.tag}</IMAGE_TAG>
|
||||
<UID>1000</UID>
|
||||
<GID>1000</GID>
|
||||
<MINIFI_VERSION>${minifi.version}</MINIFI_VERSION>
|
||||
|
|
|
@ -87,10 +87,12 @@ For Windows users, navigate to the folder where MiNiFi was installed. Navigate t
|
|||
|
||||
This launches MiNiFi and leaves it running in the foreground. To shut down NiFi, select the window that was launched and hold the Ctrl key while pressing C.
|
||||
|
||||
# Working with dataflows
|
||||
# Working with DataFlows
|
||||
When you are working with a MiNiFi dataflow, you should design it, add any additional configuration your environment or use case requires, and then deploy your dataflow. MiNiFi is not designed to accommodate substantial mid-dataflow configuration.
|
||||
|
||||
## Setting up Your Dataflow
|
||||
## Setting up Your DataFlow
|
||||
|
||||
### Manually from a NiFi Dataflow
|
||||
You can use the MiNiFi Toolkit, located in your MiNiFi installation directory, and any NiFi instance to set up the dataflow you want MiNiFi to run:
|
||||
|
||||
1. Launch NiFi
|
||||
|
@ -106,11 +108,51 @@ config.sh transform input_file output_file
|
|||
|
||||
**Note:** You can use one template at a time, per MiNiFi instance.
|
||||
|
||||
|
||||
**Result:** Once you have your _config.yml_ file in the `minifi/conf` directory, launch that instance of MiNiFi and your dataflow begins automatically.
|
||||
|
||||
### Utilizing a C2 Server via the c2 protocol
|
||||
If you have a [C2 server](../../../../minifi-c2/README.md) running, you can expose the whole _config.yml_ for the agent to download. As the agent is heartbeating via the C2 protocol, changes in flow version will trigger automatic config updates.
|
||||
|
||||
1. Launch C2 server
|
||||
2. Configure MiNiFi for C2 capability
|
||||
```
|
||||
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/heartbeat
|
||||
c2.rest.url.ack=http://localhost:10090/c2/config/acknowledge
|
||||
c2.agent.heartbeat.period=5000
|
||||
#(Optional) c2.rest.callTimeout=10 sec
|
||||
#(Optional) c2.agent.identifier=123-456-789
|
||||
c2.agent.class=agentClassName
|
||||
```
|
||||
3. Configure MiNiFi to recognize _config.yml_ changes
|
||||
```
|
||||
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
|
||||
```
|
||||
4. Start MiNiFi
|
||||
5. When a new flow is available on the C2 server, MiNiFi will download it via C2 and restart itself to pick up the changes
|
||||
|
||||
**Note:** Flow definitions are class based. Each class has one flow defined for it. As a result, all the agents belonging to the same class will get the flow at update.
|
||||
|
||||
## Loading a New Dataflow
|
||||
|
||||
### Manually
|
||||
To load a new dataflow for a MiNiFi instance to run:
|
||||
|
||||
1. Create a new _config.yml_ file with the new dataflow.
|
||||
2. Replace the existing _config.yml_ in `minifi/conf` with the new file.
|
||||
3. Restart MiNiFi.
|
||||
|
||||
### Utilizing C2 protocol
|
||||
1. Change the flow definition on the C2 Server
|
||||
2. When a new flow is available on the C2 server, MiNiFi will download it via C2 and restart itself to pick up the changes
|
||||
|
||||
## Using Processors Not Packaged with MiNiFi
|
||||
MiNiFi is able to use following processors out of the box:
|
||||
MiNiFi is able to use the following processors out of the box:
|
||||
* UpdateAttribute
|
||||
* AttributesToJSON
|
||||
* Base64EncodeContent
|
||||
|
@ -272,13 +314,6 @@ minifi.sh flowStatus processor:TailFile:health,stats,bulletins
|
|||
|
||||
For details on the `flowStatus` option, see the "FlowStatus Query Option" section of the [Administration Guide](https://nifi.apache.org/minifi/system-admin-guide.html).
|
||||
|
||||
## Loading a New Dataflow
|
||||
You can load a new dataflow for a MiNiFi instance to run:
|
||||
|
||||
1. Create a new _config.yml_ file with the new dataflow.
|
||||
2. Replace the existing _config.yml_ in `minifi/conf` with the new file.
|
||||
3. Restart MiNiFi.
|
||||
|
||||
## Stopping MiNiFi
|
||||
|
||||
You can stop MiNiFi at any time.
|
||||
|
|
|
@ -39,7 +39,7 @@ done
|
|||
|
||||
# Compute the canonicalized name by finding the physical path
|
||||
# for the directory we're in and appending the target file.
|
||||
PHYS_DIR=`pwd -P`
|
||||
PHYS_DIR=$(pwd -P)
|
||||
|
||||
SCRIPT_DIR=$PHYS_DIR
|
||||
SCRIPT_NAME=$(basename "$0")
|
||||
|
@ -82,6 +82,11 @@ detectOS() {
|
|||
export LDR_CNTRL=MAXDATA=0xB0000000@DSA
|
||||
echo ${LDR_CNTRL}
|
||||
fi
|
||||
# In addition to those, go around the linux space and query the widely
|
||||
# adopted /etc/os-release to detect linux variants
|
||||
if [ -f /etc/os-release ]; then
|
||||
. /etc/os-release
|
||||
fi
|
||||
}
|
||||
|
||||
unlimitFD() {
|
||||
|
@ -223,11 +228,26 @@ SERVICEDESCRIPTOR
|
|||
# Provide the user execute access on the file
|
||||
chmod u+x ${SVC_FILE}
|
||||
|
||||
rm -f "/etc/rc2.d/S65${SVC_NAME}"
|
||||
ln -s "/etc/init.d/${SVC_NAME}" "/etc/rc2.d/S65${SVC_NAME}" || { echo "Could not create link /etc/rc2.d/S65${SVC_NAME}"; exit 1; }
|
||||
rm -f "/etc/rc2.d/K65${SVC_NAME}"
|
||||
ln -s "/etc/init.d/${SVC_NAME}" "/etc/rc2.d/K65${SVC_NAME}" || { echo "Could not create link /etc/rc2.d/K65${SVC_NAME}"; exit 1; }
|
||||
echo "Service ${SVC_NAME} installed"
|
||||
# If SLES or OpenSuse...
|
||||
if [ "${ID}" = "opensuse" ] || [ "${ID}" = "sles" ]; then
|
||||
rm -f "/etc/rc.d/rc2.d/S65${SVC_NAME}"
|
||||
ln -s "/etc/init.d/${SVC_NAME}" "/etc/rc.d/rc2.d/S65${SVC_NAME}" || { echo "Could not create link /etc/rc.d/rc2.d/S65${SVC_NAME}"; exit 1; }
|
||||
rm -f "/etc/rc.d/rc2.d/K65${SVC_NAME}"
|
||||
ln -s "/etc/init.d/${SVC_NAME}" "/etc/rc.d/rc2.d/K65${SVC_NAME}" || { echo "Could not create link /etc/rc.d/rc2.d/K65${SVC_NAME}"; exit 1; }
|
||||
echo "Service ${SVC_NAME} installed"
|
||||
# Anything other fallback to the old approach
|
||||
else
|
||||
rm -f "/etc/rc2.d/S65${SVC_NAME}"
|
||||
ln -s "/etc/init.d/${SVC_NAME}" "/etc/rc2.d/S65${SVC_NAME}" || { echo "Could not create link /etc/rc2.d/S65${SVC_NAME}"; exit 1; }
|
||||
rm -f "/etc/rc2.d/K65${SVC_NAME}"
|
||||
ln -s "/etc/init.d/${SVC_NAME}" "/etc/rc2.d/K65${SVC_NAME}" || { echo "Could not create link /etc/rc2.d/K65${SVC_NAME}"; exit 1; }
|
||||
echo "Service ${SVC_NAME} installed"
|
||||
fi
|
||||
|
||||
# systemd: generate minifi.service from init.d
|
||||
if [ -d "/run/systemd/system/" ] || [ ! -z "$(pidof systemd 2>/dev/null)" ]; then
|
||||
systemctl daemon-reload
|
||||
fi
|
||||
}
|
||||
|
||||
run() {
|
||||
|
@ -236,15 +256,14 @@ run() {
|
|||
MINIFI_LIBS="${MINIFI_HOME}/lib/*"
|
||||
BOOTSTRAP_LIBS="${MINIFI_HOME}/lib/bootstrap/*"
|
||||
|
||||
run_as=$(grep '^\s*run.as' "${BOOTSTRAP_CONF}" | cut -d'=' -f2)
|
||||
run_as_user=$(grep '^\s*run.as' "${BOOTSTRAP_CONF}" | cut -d'=' -f2)
|
||||
# If the run as user is the same as that starting the process, ignore this configuration
|
||||
if [ "$run_as" = "$(whoami)" ]; then
|
||||
unset run_as
|
||||
if [ "$run_as_user" = "$(whoami)" ]; then
|
||||
unset run_as_user
|
||||
fi
|
||||
|
||||
sudo_cmd_prefix=""
|
||||
if $cygwin; then
|
||||
if [ -n "${run_as}" ]; then
|
||||
if [ -n "${run_as_user}" ]; then
|
||||
echo "The run.as option is not supported in a Cygwin environment. Exiting."
|
||||
exit 1
|
||||
fi;
|
||||
|
@ -262,11 +281,9 @@ run() {
|
|||
BOOTSTRAP_CLASSPATH="${TOOLS_JAR};${BOOTSTRAP_CLASSPATH};${MINIFI_LIBS}"
|
||||
fi
|
||||
else
|
||||
if [ -n "${run_as}" ]; then
|
||||
if id -u "${run_as}" >/dev/null 2>&1; then
|
||||
sudo_cmd_prefix="sudo -u ${run_as}"
|
||||
else
|
||||
echo "The specified run.as user ${run_as} does not exist. Exiting."
|
||||
if [ -n "${run_as_user}" ]; then
|
||||
if ! id -u "${run_as_user}" >/dev/null 2>&1; then
|
||||
echo "The specified run.as user ${run_as_user} does not exist. Exiting."
|
||||
exit 1
|
||||
fi
|
||||
fi;
|
||||
|
@ -290,16 +307,35 @@ run() {
|
|||
BOOTSTRAP_PID_PARAMS="-Dorg.apache.nifi.minifi.bootstrap.config.pid.dir="\""${MINIFI_PID_DIR}"\"""
|
||||
BOOTSTRAP_CONF_PARAMS="-Dorg.apache.nifi.minifi.bootstrap.config.file="\""${BOOTSTRAP_CONF}"\"""
|
||||
|
||||
# uncomment to allow debugging of the bootstrap process
|
||||
#BOOTSTRAP_DEBUG_PARAMS="-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=8000"
|
||||
|
||||
BOOTSTRAP_DIR_PARAMS="${BOOTSTRAP_LOG_PARAMS} ${BOOTSTRAP_PID_PARAMS} ${BOOTSTRAP_CONF_PARAMS}"
|
||||
|
||||
RUN_MINIFI_CMD="cd "\""${MINIFI_HOME}"\"" && ${sudo_cmd_prefix} "\""${JAVA}"\"" -cp "\""${BOOTSTRAP_CLASSPATH}"\"" -Xms12m -Xmx24m ${BOOTSTRAP_DIR_PARAMS} org.apache.nifi.minifi.bootstrap.RunMiNiFi"
|
||||
RUN_BOOTSTRAP_CMD="'${JAVA}' -cp '${BOOTSTRAP_CLASSPATH}' -Xms12m -Xmx24m ${BOOTSTRAP_DIR_PARAMS} ${BOOTSTRAP_DEBUG_PARAMS} org.apache.nifi.minifi.bootstrap.RunMiNiFi"
|
||||
RUN_MINIFI_CMD="${RUN_BOOTSTRAP_CMD} $@"
|
||||
|
||||
if [ -n "${run_as_user}" ]; then
|
||||
preserve_environment=$(grep '^\s*preserve.environment' "${BOOTSTRAP_CONF}" | cut -d'=' -f2 | tr '[:upper:]' '[:lower:]')
|
||||
SUDO="sudo"
|
||||
if [ "$preserve_environment" = "true" ]; then
|
||||
SUDO="sudo -E"
|
||||
fi
|
||||
# Provide SCRIPT_DIR and execute nifi-env for the run.as user command
|
||||
RUN_MINIFI_CMD="${SUDO} -u ${run_as_user} sh -c \"SCRIPT_DIR='${SCRIPT_DIR}' && . '${SCRIPT_DIR}/minifi-env.sh' && ${RUN_MINIFI_CMD}\""
|
||||
fi
|
||||
|
||||
if [ "$1" = "run" ]; then
|
||||
# Use exec to handover PID to RunMiNiFi java process, instead of foking it as a child process
|
||||
RUN_MINIFI_CMD="exec ${RUN_MINIFI_CMD}"
|
||||
fi
|
||||
|
||||
# run 'start' in the background because the process will continue to run, monitoring MiNiFi.
|
||||
# all other commands will terminate quickly so want to just wait for them
|
||||
if [ "$1" = "start" ]; then
|
||||
(eval $RUN_MINIFI_CMD $@ &)
|
||||
(eval "cd ${MINIFI_HOME} && ${RUN_MINIFI_CMD}" &)
|
||||
else
|
||||
(eval $RUN_MINIFI_CMD $@)
|
||||
eval "cd ${MINIFI_HOME} && ${RUN_MINIFI_CMD}"
|
||||
fi
|
||||
EXIT_STATUS=$?
|
||||
|
||||
|
@ -320,16 +356,10 @@ case "$1" in
|
|||
install)
|
||||
install "$@"
|
||||
;;
|
||||
start|stop|run|status|flowStatus|dump|env)
|
||||
start|stop|restart|run|status|flowStatus|dump|env)
|
||||
main "$@"
|
||||
exit $EXIT_STATUS
|
||||
;;
|
||||
restart)
|
||||
init
|
||||
run "stop"
|
||||
run "start"
|
||||
exit $EXIT_STATUS
|
||||
;;
|
||||
*)
|
||||
echo "Usage minifi {start|stop|run|restart|status|flowStatus|dump|install}"
|
||||
;;
|
||||
|
|
|
@ -59,10 +59,9 @@ nifi.minifi.provenance.reporting.batch.size=
|
|||
nifi.minifi.provenance.reporting.communications.timeout=
|
||||
|
||||
# Ignore all processor SSL controller services and use parent minifi SSL instead
|
||||
nifi.minifi.flow.use.parent.ssl=false
|
||||
nifi.minifi.flow.use.parent.ssl=false
|
||||
|
||||
# Notifiers to use for the associated agent, comma separated list of class names
|
||||
#nifi.minifi.notifier.ingestors=org.apache.nifi.minifi.bootstrap.configuration.ingestors.FileChangeIngestor
|
||||
#nifi.minifi.notifier.ingestors=org.apache.nifi.minifi.bootstrap.configuration.ingestors.RestChangeIngestor
|
||||
#nifi.minifi.notifier.ingestors=org.apache.nifi.minifi.bootstrap.configuration.ingestors.PullHttpChangeIngestor
|
||||
|
||||
|
@ -129,3 +128,35 @@ 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
|
||||
# The following ingestor configuration needs to be enabled in order to apply configuration updates coming from C2 server
|
||||
#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
|
|
@ -62,5 +62,14 @@ limitations under the License.
|
|||
<groupId>org.slf4j</groupId>
|
||||
<artifactId>jul-to-slf4j</artifactId>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>com.fasterxml.jackson.core</groupId>
|
||||
<artifactId>jackson-core</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.fasterxml.jackson.core</groupId>
|
||||
<artifactId>jackson-databind</artifactId>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
</project>
|
||||
|
|
|
@ -16,12 +16,13 @@
|
|||
*/
|
||||
package org.apache.nifi.minifi;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonInclude;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import java.io.BufferedReader;
|
||||
import java.io.BufferedWriter;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.InputStreamReader;
|
||||
import java.io.ObjectOutputStream;
|
||||
import java.io.OutputStream;
|
||||
import java.io.OutputStreamWriter;
|
||||
import java.lang.management.LockInfo;
|
||||
|
@ -41,7 +42,6 @@ import java.util.List;
|
|||
import java.util.UUID;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
|
||||
import org.apache.nifi.minifi.commons.status.FlowStatusReport;
|
||||
import org.apache.nifi.minifi.status.StatusRequestException;
|
||||
import org.apache.nifi.util.LimitingInputStream;
|
||||
|
@ -55,6 +55,7 @@ public class BootstrapListener {
|
|||
private final MiNiFi minifi;
|
||||
private final int bootstrapPort;
|
||||
private final String secretKey;
|
||||
private final ObjectMapper objectMapper;
|
||||
|
||||
private volatile Listener listener;
|
||||
|
||||
|
@ -62,6 +63,9 @@ public class BootstrapListener {
|
|||
this.minifi = minifi;
|
||||
this.bootstrapPort = bootstrapPort;
|
||||
secretKey = UUID.randomUUID().toString();
|
||||
|
||||
objectMapper = new ObjectMapper();
|
||||
objectMapper.setSerializationInclusion(JsonInclude.Include.NON_NULL);
|
||||
}
|
||||
|
||||
public void start() throws IOException {
|
||||
|
@ -212,6 +216,10 @@ public class BootstrapListener {
|
|||
String flowStatusRequestString = request.getArgs()[0];
|
||||
writeStatusReport(flowStatusRequestString, socket.getOutputStream());
|
||||
break;
|
||||
case ENV:
|
||||
logger.info("Received ENV request from Bootstrap");
|
||||
writeEnv(socket.getOutputStream());
|
||||
break;
|
||||
}
|
||||
} catch (final Throwable t) {
|
||||
logger.error("Failed to process request from Bootstrap due to " + t.toString(), t);
|
||||
|
@ -231,10 +239,22 @@ public class BootstrapListener {
|
|||
}
|
||||
|
||||
private void writeStatusReport(String flowStatusRequestString, final OutputStream out) throws IOException, StatusRequestException {
|
||||
ObjectOutputStream oos = new ObjectOutputStream(out);
|
||||
FlowStatusReport flowStatusReport = minifi.getMinifiServer().getStatusReport(flowStatusRequestString);
|
||||
oos.writeObject(flowStatusReport);
|
||||
oos.close();
|
||||
objectMapper.writeValue(out, flowStatusReport);
|
||||
}
|
||||
|
||||
private static void writeEnv(OutputStream out) throws IOException {
|
||||
try (BufferedWriter writer = new BufferedWriter(new OutputStreamWriter(out))) {
|
||||
StringBuilder sb = new StringBuilder();
|
||||
|
||||
System.getProperties()
|
||||
.entrySet()
|
||||
.stream()
|
||||
.forEach(entry -> sb.append(entry.getKey()).append("=").append(entry.getValue()).append("\n"));
|
||||
|
||||
writer.write(sb.toString());
|
||||
writer.flush();
|
||||
}
|
||||
}
|
||||
|
||||
private static void writeDump(final OutputStream out) throws IOException {
|
||||
|
@ -394,7 +414,8 @@ public class BootstrapListener {
|
|||
SHUTDOWN,
|
||||
DUMP,
|
||||
PING,
|
||||
FLOW_STATUS_REPORT
|
||||
FLOW_STATUS_REPORT,
|
||||
ENV
|
||||
}
|
||||
|
||||
private final RequestType requestType;
|
||||
|
|
|
@ -45,7 +45,6 @@ import java.util.concurrent.TimeUnit;
|
|||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
|
||||
public class MiNiFi {
|
||||
|
||||
private static final Logger logger = LoggerFactory.getLogger(MiNiFi.class);
|
||||
|
@ -73,7 +72,7 @@ public class MiNiFi {
|
|||
final File kerberosConfigFile = properties.getKerberosConfigurationFile();
|
||||
if (kerberosConfigFile != null) {
|
||||
final String kerberosConfigFilePath = kerberosConfigFile.getAbsolutePath();
|
||||
logger.info("Setting java.security.krb5.conf to {}", new Object[]{kerberosConfigFilePath});
|
||||
logger.info("Setting java.security.krb5.conf to {}", kerberosConfigFilePath);
|
||||
System.setProperty("java.security.krb5.conf", kerberosConfigFilePath);
|
||||
}
|
||||
|
||||
|
@ -151,6 +150,7 @@ public class MiNiFi {
|
|||
}
|
||||
minifiServer = (MiNiFiServer) nifiServer;
|
||||
Thread.currentThread().setContextClassLoader(minifiServer.getClass().getClassLoader());
|
||||
|
||||
// Filter out the framework NAR from being loaded by the NiFiServer
|
||||
minifiServer.initialize(properties,
|
||||
systemBundle,
|
||||
|
@ -261,7 +261,7 @@ public class MiNiFi {
|
|||
public static void main(String[] args) {
|
||||
logger.info("Launching MiNiFi...");
|
||||
try {
|
||||
NiFiProperties niFiProperties = NiFiProperties.createBasicNiFiProperties(null, (Map<String,String>) null);
|
||||
NiFiProperties niFiProperties = NiFiProperties.createBasicNiFiProperties(null, (Map<String, String>) null);
|
||||
new MiNiFi(niFiProperties);
|
||||
} catch (final Throwable t) {
|
||||
logger.error("Failure to launch MiNiFi due to " + t, t);
|
||||
|
|
|
@ -37,5 +37,4 @@ public class StandardMiNiFiServer extends HeadlessNiFiServer implements MiNiFiSe
|
|||
public FlowStatusReport getStatusReport(String requestString) throws StatusRequestException {
|
||||
return StatusConfigReporter.getStatus(this.flowController, requestString, logger);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -43,6 +43,11 @@
|
|||
<artifactId>nifi-headless-server</artifactId>
|
||||
<version>1.17.0-SNAPSHOT</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.commons</groupId>
|
||||
<artifactId>commons-lang3</artifactId>
|
||||
<scope>compile</scope>
|
||||
</dependency>
|
||||
<!-- mark these nifi artifacts as provided since it is included in the lib -->
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
|
|
|
@ -39,6 +39,11 @@ limitations under the License.
|
|||
<artifactId>minifi-framework-api</artifactId>
|
||||
<version>1.17.0-SNAPSHOT</version>
|
||||
</dependency>
|
||||
<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>nifi-headless-server</artifactId>
|
||||
|
|
|
@ -448,11 +448,18 @@ limitations under the License.
|
|||
<version>1.17.0-SNAPSHOT</version>
|
||||
</dependency>
|
||||
|
||||
<!-- C2 modules -->
|
||||
<dependency>
|
||||
<groupId>com.squareup.okhttp3</groupId>
|
||||
<artifactId>okhttp</artifactId>
|
||||
<version>3.12.3</version>
|
||||
<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>org.apache.commons</groupId>
|
||||
<artifactId>commons-compress</artifactId>
|
||||
|
|
|
@ -56,7 +56,7 @@ public class StorageUsage implements Cloneable {
|
|||
}
|
||||
|
||||
public int getDiskUtilization() {
|
||||
return DiagnosticUtils.getUtilization(getUsedSpace(), totalSpace);
|
||||
return Math.round(((float)getUsedSpace() / totalSpace) * 100);
|
||||
}
|
||||
|
||||
@Override
|
|
@ -168,7 +168,7 @@ public class SystemDiagnostics implements Cloneable {
|
|||
if (maxHeap == -1) {
|
||||
return -1;
|
||||
} else {
|
||||
return DiagnosticUtils.getUtilization(usedHeap, maxHeap);
|
||||
return Math.round(((float)usedHeap / maxHeap) * 100);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -176,7 +176,7 @@ public class SystemDiagnostics implements Cloneable {
|
|||
if (maxNonHeap == -1) {
|
||||
return -1;
|
||||
} else {
|
||||
return DiagnosticUtils.getUtilization(usedNonHeap, maxNonHeap);
|
||||
return Math.round(((float)usedNonHeap / maxNonHeap) * 100);
|
||||
}
|
||||
}
|
||||
|
|
@ -185,6 +185,11 @@
|
|||
<groupId>org.xerial.snappy</groupId>
|
||||
<artifactId>snappy-java</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>c2-client-service</artifactId>
|
||||
<version>1.17.0-SNAPSHOT</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.zookeeper</groupId>
|
||||
<artifactId>zookeeper</artifactId>
|
||||
|
|
|
@ -0,0 +1,71 @@
|
|||
/*
|
||||
* 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";
|
||||
|
||||
// Defaults
|
||||
// Heartbeat period of 1 second
|
||||
public static final long C2_AGENT_DEFAULT_HEARTBEAT_PERIOD = TimeUnit.SECONDS.toMillis(1);
|
||||
|
||||
// Connection timeout of 5 seconds
|
||||
public static final String C2_DEFAULT_CONNECTION_TIMEOUT = "5 sec";
|
||||
// Read timeout of 5 seconds
|
||||
public static final String C2_DEFAULT_READ_TIMEOUT = "5 sec";
|
||||
// Call timeout of 10 seconds
|
||||
public static final String C2_DEFAULT_CALL_TIMEOUT = "10 sec";
|
||||
}
|
|
@ -0,0 +1,211 @@
|
|||
/*
|
||||
* 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 = 10000L;
|
||||
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.parseLong(properties.getProperty(C2NiFiProperties.C2_AGENT_HEARTBEAT_PERIOD_KEY,
|
||||
String.valueOf(C2NiFiProperties.C2_AGENT_DEFAULT_HEARTBEAT_PERIOD))))
|
||||
.connectTimeout((long) FormatUtils.getPreciseTimeDuration(properties.getProperty(C2NiFiProperties.C2_CONNECTION_TIMEOUT,
|
||||
C2NiFiProperties.C2_DEFAULT_CONNECTION_TIMEOUT), TimeUnit.MILLISECONDS))
|
||||
.readTimeout((long) FormatUtils.getPreciseTimeDuration(properties.getProperty(C2NiFiProperties.C2_READ_TIMEOUT,
|
||||
C2NiFiProperties.C2_DEFAULT_READ_TIMEOUT), TimeUnit.MILLISECONDS))
|
||||
.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, e);
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
private File getTargetConfigFile() {
|
||||
return Optional.ofNullable(propertiesDir)
|
||||
.map(File::new)
|
||||
.map(File::getParent)
|
||||
.map(parentDir -> new File(parentDir + TARGET_CONFIG_FILE))
|
||||
.orElse( new File(DEFAULT_CONF_DIR + TARGET_CONFIG_FILE));
|
||||
}
|
||||
}
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue