mirror of https://github.com/apache/nifi.git
NIFI-11514 Flow JSON support and deprecating YAML format. Revised parameter generation. Generic refactors.
Signed-off-by: Ferenc Erdei <erdei.ferenc90@gmail.com> This closes #7344
This commit is contained in:
parent
0d119f6f39
commit
3aa02a022e
|
@ -14,6 +14,7 @@
|
|||
* 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;
|
||||
|
@ -47,7 +48,15 @@ public interface C2Client {
|
|||
* @param callbackUrl 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 callbackUrl);
|
||||
Optional<byte[]> retrieveUpdateConfigurationContent(String callbackUrl);
|
||||
|
||||
/**
|
||||
* Retrieve the asset from the C2 Server
|
||||
*
|
||||
* @param callbackUrl url where the asset should be downloaded from
|
||||
* @return the actual downloaded asset. Will be empty if no content can be downloaded
|
||||
*/
|
||||
Optional<byte[]> retrieveUpdateAssetContent(String callbackUrl);
|
||||
|
||||
/**
|
||||
* Uploads a binary bundle to C2 server
|
||||
|
|
|
@ -14,8 +14,17 @@
|
|||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.nifi.c2.client;
|
||||
|
||||
import static java.util.Optional.ofNullable;
|
||||
import static java.util.stream.Collectors.toUnmodifiableMap;
|
||||
import static org.apache.commons.lang3.StringUtils.EMPTY;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Map;
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
|
||||
/**
|
||||
* Configuration for a C2 Client.
|
||||
*/
|
||||
|
@ -42,6 +51,7 @@ public class C2ClientConfig {
|
|||
private final String truststoreType;
|
||||
private final long callTimeout;
|
||||
private final long readTimeout;
|
||||
private final Map<String, String> httpHeaders;
|
||||
private final long connectTimeout;
|
||||
private final int maxIdleConnections;
|
||||
private final long keepAliveDuration;
|
||||
|
@ -71,6 +81,7 @@ public class C2ClientConfig {
|
|||
this.truststoreType = builder.truststoreType;
|
||||
this.readTimeout = builder.readTimeout;
|
||||
this.connectTimeout = builder.connectTimeout;
|
||||
this.httpHeaders = builder.httpHeaders;
|
||||
this.maxIdleConnections = builder.maxIdleConnections;
|
||||
this.keepAliveDuration = builder.keepAliveDuration;
|
||||
this.c2RequestCompression = builder.c2RequestCompression;
|
||||
|
@ -165,6 +176,10 @@ public class C2ClientConfig {
|
|||
return connectTimeout;
|
||||
}
|
||||
|
||||
public Map<String, String> getHttpHeaders() {
|
||||
return httpHeaders;
|
||||
}
|
||||
|
||||
public String getC2RequestCompression() {
|
||||
return c2RequestCompression;
|
||||
}
|
||||
|
@ -186,6 +201,9 @@ public class C2ClientConfig {
|
|||
*/
|
||||
public static class Builder {
|
||||
|
||||
private static final String HTTP_HEADERS_SEPARATOR = "#";
|
||||
private static final String HTTP_HEADER_KEY_VALUE_SEPARATOR = ":";
|
||||
|
||||
private String c2Url;
|
||||
private String c2AckUrl;
|
||||
private String c2RestPathBase;
|
||||
|
@ -208,6 +226,7 @@ public class C2ClientConfig {
|
|||
private String truststoreType;
|
||||
private long readTimeout;
|
||||
private long connectTimeout;
|
||||
private Map<String, String> httpHeaders;
|
||||
private int maxIdleConnections;
|
||||
private long keepAliveDuration;
|
||||
private String c2RequestCompression;
|
||||
|
@ -227,10 +246,12 @@ public class C2ClientConfig {
|
|||
this.c2RestPathBase = c2RestPathBase;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder c2RestPathHeartbeat(String c2RestPathHeartbeat) {
|
||||
this.c2RestPathHeartbeat = c2RestPathHeartbeat;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder c2RestPathAcknowledge(String c2RestPathAcknowledge) {
|
||||
this.c2RestPathAcknowledge = c2RestPathAcknowledge;
|
||||
return this;
|
||||
|
@ -321,6 +342,21 @@ public class C2ClientConfig {
|
|||
return this;
|
||||
}
|
||||
|
||||
public Builder httpHeaders(String httpHeaders) {
|
||||
this.httpHeaders = ofNullable(httpHeaders)
|
||||
.filter(StringUtils::isNotBlank)
|
||||
.map(headers -> headers.split(HTTP_HEADERS_SEPARATOR))
|
||||
.stream()
|
||||
.flatMap(Arrays::stream)
|
||||
.map(String::trim)
|
||||
.map(header -> header.split(HTTP_HEADER_KEY_VALUE_SEPARATOR))
|
||||
.filter(split -> split.length == 2)
|
||||
.collect(toUnmodifiableMap(
|
||||
split -> ofNullable(split[0]).map(String::trim).orElse(EMPTY),
|
||||
split -> ofNullable(split[1]).map(String::trim).orElse(EMPTY)));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder maxIdleConnections(int maxIdleConnections) {
|
||||
this.maxIdleConnections = maxIdleConnections;
|
||||
return this;
|
||||
|
|
|
@ -20,8 +20,10 @@ package org.apache.nifi.c2.client.http;
|
|||
import static okhttp3.MultipartBody.FORM;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import okhttp3.Headers;
|
||||
import okhttp3.MediaType;
|
||||
import okhttp3.MultipartBody;
|
||||
import okhttp3.OkHttpClient;
|
||||
|
@ -84,33 +86,13 @@ public class C2HttpClient implements C2Client {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Optional<byte[]> retrieveUpdateContent(String callbackUrl) {
|
||||
Optional<byte[]> updateContent = Optional.empty();
|
||||
public Optional<byte[]> retrieveUpdateConfigurationContent(String callbackUrl) {
|
||||
return retrieveContent(callbackUrl, clientConfig.getHttpHeaders());
|
||||
}
|
||||
|
||||
Request.Builder requestBuilder = new Request.Builder()
|
||||
.get()
|
||||
.url(callbackUrl);
|
||||
Request request = requestBuilder.build();
|
||||
|
||||
try (Response response = httpClientReference.get().newCall(request).execute()) {
|
||||
Optional<ResponseBody> body = Optional.ofNullable(response.body());
|
||||
|
||||
if (!response.isSuccessful()) {
|
||||
StringBuilder messageBuilder = new StringBuilder(String.format("Configuration retrieval failed: HTTP %d", response.code()));
|
||||
body.map(Object::toString).ifPresent(messageBuilder::append);
|
||||
throw new C2ServerException(messageBuilder.toString());
|
||||
}
|
||||
|
||||
if (body.isPresent()) {
|
||||
updateContent = Optional.of(body.get().bytes());
|
||||
} else {
|
||||
logger.warn("No body returned when pulling a new configuration");
|
||||
}
|
||||
} catch (Exception e) {
|
||||
logger.warn("Configuration retrieval failed", e);
|
||||
}
|
||||
|
||||
return updateContent;
|
||||
@Override
|
||||
public Optional<byte[]> retrieveUpdateAssetContent(String callbackUrl) {
|
||||
return retrieveContent(callbackUrl, Map.of());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -181,4 +163,34 @@ public class C2HttpClient implements C2Client {
|
|||
logger.error("Could not transmit ack to C2 server {}", c2UrlProvider.getAcknowledgeUrl(), e);
|
||||
}
|
||||
}
|
||||
|
||||
private Optional<byte[]> retrieveContent(String callbackUrl, Map<String, String> httpHeaders) {
|
||||
Optional<byte[]> content = Optional.empty();
|
||||
|
||||
Request.Builder requestBuilder = new Request.Builder()
|
||||
.get()
|
||||
.headers(Headers.of(httpHeaders))
|
||||
.url(callbackUrl);
|
||||
Request request = requestBuilder.build();
|
||||
|
||||
try (Response response = httpClientReference.get().newCall(request).execute()) {
|
||||
Optional<ResponseBody> body = Optional.ofNullable(response.body());
|
||||
|
||||
if (!response.isSuccessful()) {
|
||||
StringBuilder messageBuilder = new StringBuilder(String.format("Update content retrieval failed: HTTP %d", response.code()));
|
||||
body.map(Object::toString).ifPresent(messageBuilder::append);
|
||||
throw new C2ServerException(messageBuilder.toString());
|
||||
}
|
||||
|
||||
if (body.isPresent()) {
|
||||
content = Optional.of(body.get().bytes());
|
||||
} else {
|
||||
logger.warn("No body returned when pulling new content");
|
||||
}
|
||||
} catch (Exception e) {
|
||||
logger.warn("Update content retrieval failed", e);
|
||||
}
|
||||
|
||||
return content;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -124,7 +124,7 @@ public class C2HttpClientTest {
|
|||
mockWebServer.enqueue(new MockResponse().setBody("updateContent").setResponseCode(HTTP_STATUS_BAD_REQUEST));
|
||||
|
||||
C2HttpClient c2HttpClient = C2HttpClient.create(c2ClientConfig, serializer);
|
||||
Optional<byte[]> response = c2HttpClient.retrieveUpdateContent(baseUrl + UPDATE_PATH);
|
||||
Optional<byte[]> response = c2HttpClient.retrieveUpdateConfigurationContent(baseUrl + UPDATE_PATH);
|
||||
|
||||
assertFalse(response.isPresent());
|
||||
|
||||
|
@ -138,7 +138,7 @@ public class C2HttpClientTest {
|
|||
mockWebServer.enqueue(new MockResponse().setBody(content).setResponseCode(HTTP_STATUS_OK));
|
||||
|
||||
C2HttpClient c2HttpClient = C2HttpClient.create(c2ClientConfig, serializer);
|
||||
Optional<byte[]> response = c2HttpClient.retrieveUpdateContent(baseUrl + UPDATE_PATH);
|
||||
Optional<byte[]> response = c2HttpClient.retrieveUpdateConfigurationContent(baseUrl + UPDATE_PATH);
|
||||
|
||||
assertTrue(response.isPresent());
|
||||
assertArrayEquals(content.getBytes(StandardCharsets.UTF_8), response.get());
|
||||
|
|
|
@ -33,8 +33,9 @@ import static org.apache.nifi.c2.protocol.api.OperationType.TRANSFER;
|
|||
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.File;
|
||||
import java.io.FileInputStream;
|
||||
import java.io.FileOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.UncheckedIOException;
|
||||
import java.nio.charset.Charset;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
|
@ -45,9 +46,11 @@ import java.util.Map;
|
|||
import java.util.Optional;
|
||||
import java.util.function.Predicate;
|
||||
import java.util.stream.Stream;
|
||||
import java.util.zip.GZIPInputStream;
|
||||
import org.apache.commons.compress.archivers.tar.TarArchiveEntry;
|
||||
import org.apache.commons.compress.archivers.tar.TarArchiveOutputStream;
|
||||
import org.apache.commons.compress.compressors.gzip.GzipCompressorOutputStream;
|
||||
import org.apache.commons.compress.compressors.gzip.GzipUtils;
|
||||
import org.apache.nifi.c2.client.api.C2Client;
|
||||
import org.apache.nifi.c2.protocol.api.C2Operation;
|
||||
import org.apache.nifi.c2.protocol.api.C2OperationAck;
|
||||
|
@ -124,11 +127,11 @@ public class TransferDebugOperationHandler implements C2OperationHandler {
|
|||
return operationAck(operation, operationState(NOT_APPLIED, C2_CALLBACK_URL_NOT_FOUND));
|
||||
}
|
||||
|
||||
List<Path> contentFilteredFilePaths = null;
|
||||
List<Path> preparedFiles = null;
|
||||
C2OperationState operationState;
|
||||
try {
|
||||
contentFilteredFilePaths = filterContent(operation.getIdentifier(), bundleFilePaths);
|
||||
operationState = createDebugBundle(contentFilteredFilePaths)
|
||||
preparedFiles = prepareFiles(operation.getIdentifier(), bundleFilePaths);
|
||||
operationState = createDebugBundle(preparedFiles)
|
||||
.map(bundle -> c2Client.uploadBundle(callbackUrl.get(), bundle)
|
||||
.map(errorMessage -> operationState(NOT_APPLIED, errorMessage))
|
||||
.orElseGet(() -> operationState(FULLY_APPLIED, SUCCESSFUL_UPLOAD)))
|
||||
|
@ -137,7 +140,7 @@ public class TransferDebugOperationHandler implements C2OperationHandler {
|
|||
LOG.error("Unexpected error happened", e);
|
||||
operationState = operationState(NOT_APPLIED, UNABLE_TO_CREATE_BUNDLE);
|
||||
} finally {
|
||||
ofNullable(contentFilteredFilePaths).ifPresent(this::cleanup);
|
||||
ofNullable(preparedFiles).ifPresent(this::cleanup);
|
||||
}
|
||||
|
||||
LOG.debug("Returning operation ack for operation {} with state {} and details {}", operation.getIdentifier(), operationState.getState(), operationState.getDetails());
|
||||
|
@ -158,21 +161,40 @@ public class TransferDebugOperationHandler implements C2OperationHandler {
|
|||
return state;
|
||||
}
|
||||
|
||||
private List<Path> filterContent(String operationId, List<Path> bundleFilePaths) {
|
||||
List<Path> contentFilteredFilePaths = new ArrayList<>();
|
||||
for (Path path : bundleFilePaths) {
|
||||
String fileName = path.getFileName().toString();
|
||||
try (Stream<String> fileStream = lines(path, Charset.defaultCharset())) {
|
||||
Path tempDirectory = createTempDirectory(operationId);
|
||||
Path tempFile = Paths.get(tempDirectory.toAbsolutePath().toString(), fileName);
|
||||
Files.write(tempFile, (Iterable<String>) fileStream.filter(contentFilter)::iterator);
|
||||
contentFilteredFilePaths.add(tempFile);
|
||||
} catch (IOException e) {
|
||||
LOG.error("Error during filtering file content: " + path.toAbsolutePath(), e);
|
||||
throw new UncheckedIOException(e);
|
||||
}
|
||||
private List<Path> prepareFiles(String operationId, List<Path> bundleFilePaths) throws IOException {
|
||||
List<Path> preparedFiles = new ArrayList<>();
|
||||
for (Path bundleFile : bundleFilePaths) {
|
||||
Path tempDirectory = createTempDirectory(operationId);
|
||||
String fileName = bundleFile.getFileName().toString();
|
||||
|
||||
Path preparedFile = GzipUtils.isCompressedFilename(fileName)
|
||||
? handleGzipFile(bundleFile, Paths.get(tempDirectory.toAbsolutePath().toString(), GzipUtils.getUncompressedFilename(fileName)))
|
||||
: handleUncompressedFile(bundleFile, Paths.get(tempDirectory.toAbsolutePath().toString(), fileName));
|
||||
preparedFiles.add(preparedFile);
|
||||
}
|
||||
return preparedFiles;
|
||||
}
|
||||
|
||||
private Path handleGzipFile(Path sourceFile, Path targetFile) throws IOException {
|
||||
try (GZIPInputStream gzipInputStream = new GZIPInputStream(new FileInputStream(sourceFile.toFile()));
|
||||
FileOutputStream fileOutputStream = new FileOutputStream(targetFile.toFile())) {
|
||||
// no content filter is applied here as flow.json.gz has encoded properties
|
||||
gzipInputStream.transferTo(fileOutputStream);
|
||||
return targetFile;
|
||||
} catch (IOException e) {
|
||||
LOG.error("Error during filtering gzip file content: " + sourceFile.toAbsolutePath(), e);
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
|
||||
private Path handleUncompressedFile(Path sourceFile, Path targetFile) throws IOException {
|
||||
try (Stream<String> fileStream = lines(sourceFile, Charset.defaultCharset())) {
|
||||
Files.write(targetFile, (Iterable<String>) fileStream.filter(contentFilter)::iterator);
|
||||
return targetFile;
|
||||
} catch (IOException e) {
|
||||
LOG.error("Error during filtering uncompressed file content: " + sourceFile.toAbsolutePath(), e);
|
||||
throw e;
|
||||
}
|
||||
return contentFilteredFilePaths;
|
||||
}
|
||||
|
||||
private Optional<byte[]> createDebugBundle(List<Path> filePaths) {
|
||||
|
|
|
@ -123,11 +123,11 @@ public class UpdateAssetOperationHandler implements C2OperationHandler {
|
|||
LOG.info("Initiating asset update from url {} with name {}, force update is {}", callbackUrl, assetFileName, forceDownload);
|
||||
|
||||
C2OperationState operationState = assetUpdatePrecondition.test(assetFileName, forceDownload)
|
||||
? c2Client.retrieveUpdateContent(callbackUrl.get())
|
||||
.map(content -> assetPersistFunction.apply(assetFileName, content)
|
||||
? operationState(FULLY_APPLIED, SUCCESSFULLY_UPDATE_ASSET)
|
||||
: operationState(NOT_APPLIED, FAILED_TO_PERSIST_ASSET_TO_DISK))
|
||||
.orElseGet(() -> operationState(NOT_APPLIED, UPDATE_ASSET_RETRIEVAL_RESULTED_IN_EMPTY_CONTENT))
|
||||
? c2Client.retrieveUpdateAssetContent(callbackUrl.get())
|
||||
.map(content -> assetPersistFunction.apply(assetFileName, content)
|
||||
? operationState(FULLY_APPLIED, SUCCESSFULLY_UPDATE_ASSET)
|
||||
: operationState(NOT_APPLIED, FAILED_TO_PERSIST_ASSET_TO_DISK))
|
||||
.orElseGet(() -> operationState(NOT_APPLIED, UPDATE_ASSET_RETRIEVAL_RESULTED_IN_EMPTY_CONTENT))
|
||||
: operationState(NO_OPERATION, UPDATE_ASSET_PRECONDITIONS_WERE_NOT_MET);
|
||||
|
||||
return operationAck(operationId, operationState);
|
||||
|
|
|
@ -29,7 +29,6 @@ import static org.apache.nifi.c2.protocol.api.OperationType.UPDATE;
|
|||
import java.net.URI;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.function.Function;
|
||||
import java.util.regex.Matcher;
|
||||
import java.util.regex.Pattern;
|
||||
import org.apache.nifi.c2.client.api.C2Client;
|
||||
|
@ -43,22 +42,26 @@ 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 Pattern FLOW_ID_PATTERN = Pattern.compile("/[^/]+?/[^/]+?/[^/]+?/([^/]+)?/?.*");
|
||||
static final String FLOW_ID = "flowId";
|
||||
static final String LOCATION = "location";
|
||||
|
||||
public static final String FLOW_URL_KEY = "flowUrl";
|
||||
public static final String FLOW_RELATIVE_URL_KEY = "relativeFlowUrl";
|
||||
|
||||
static final String FLOW_ID = "flowId";
|
||||
static final String LOCATION = "location";
|
||||
|
||||
private static final Logger logger = LoggerFactory.getLogger(UpdateConfigurationOperationHandler.class);
|
||||
|
||||
private static final Pattern FLOW_ID_PATTERN = Pattern.compile("/[^/]+?/[^/]+?/[^/]+?/([^/]+)?/?.*");
|
||||
|
||||
private final C2Client client;
|
||||
private final Function<byte[], Boolean> updateFlow;
|
||||
private final UpdateConfigurationStrategy updateConfigurationStrategy;
|
||||
private final FlowIdHolder flowIdHolder;
|
||||
private final OperandPropertiesProvider operandPropertiesProvider;
|
||||
|
||||
public UpdateConfigurationOperationHandler(C2Client client, FlowIdHolder flowIdHolder, Function<byte[], Boolean> updateFlow,
|
||||
public UpdateConfigurationOperationHandler(C2Client client, FlowIdHolder flowIdHolder, UpdateConfigurationStrategy updateConfigurationStrategy,
|
||||
OperandPropertiesProvider operandPropertiesProvider) {
|
||||
this.client = client;
|
||||
this.updateFlow = updateFlow;
|
||||
this.updateConfigurationStrategy = updateConfigurationStrategy;
|
||||
this.flowIdHolder = flowIdHolder;
|
||||
this.operandPropertiesProvider = operandPropertiesProvider;
|
||||
}
|
||||
|
@ -80,7 +83,7 @@ public class UpdateConfigurationOperationHandler implements C2OperationHandler {
|
|||
|
||||
@Override
|
||||
public boolean requiresRestart() {
|
||||
return true;
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -108,19 +111,22 @@ public class UpdateConfigurationOperationHandler implements C2OperationHandler {
|
|||
|
||||
logger.info("Will perform flow update from {} for operation #{}. Previous flow id was {}, replacing with new id {}",
|
||||
callbackUrl, operationId, ofNullable(flowIdHolder.getFlowId()).orElse("not set"), flowId);
|
||||
flowIdHolder.setFlowId(flowId);
|
||||
return operationAck(operationId, updateFlow(operationId, callbackUrl.get()));
|
||||
C2OperationState state = updateFlow(operationId, callbackUrl.get());
|
||||
if (state.getState() == FULLY_APPLIED) {
|
||||
flowIdHolder.setFlowId(flowId);
|
||||
}
|
||||
return operationAck(operationId, state);
|
||||
}
|
||||
|
||||
private C2OperationState updateFlow(String opIdentifier, String callbackUrl) {
|
||||
Optional<byte[]> updateContent = client.retrieveUpdateContent(callbackUrl);
|
||||
Optional<byte[]> updateContent = client.retrieveUpdateConfigurationContent(callbackUrl);
|
||||
|
||||
if (!updateContent.isPresent()) {
|
||||
logger.error("Update content retrieval resulted in empty content so flow update was omitted for operation #{}.", opIdentifier);
|
||||
return operationState(NOT_APPLIED, "Update content retrieval resulted in empty content");
|
||||
}
|
||||
|
||||
if (!updateFlow.apply(updateContent.get())) {
|
||||
if (!updateConfigurationStrategy.update(updateContent.get())) {
|
||||
logger.error("Update resulted in error for operation #{}.", opIdentifier);
|
||||
return operationState(NOT_APPLIED, "Update resulted in error");
|
||||
}
|
||||
|
|
|
@ -15,19 +15,19 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.nifi.minifi.c2.provider.nifi.rest;
|
||||
package org.apache.nifi.c2.client.service.operation;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
/**
|
||||
* Common interface for classes implementing approach for MiNiFi flow update
|
||||
*/
|
||||
@FunctionalInterface
|
||||
public interface UpdateConfigurationStrategy {
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.mockito.Mockito.mock;
|
||||
|
||||
public class TemplatesIteratorExceptionTest {
|
||||
@Test
|
||||
public void testCauseConstructor() {
|
||||
IOException ioException = mock(IOException.class);
|
||||
assertEquals(ioException, new TemplatesIteratorException(ioException).getCause());
|
||||
}
|
||||
/**
|
||||
* Updates the MiNiFi agent's flow with the flow passed as parameter
|
||||
*
|
||||
* @param flow the MiNiFi flow config JSON represented as a byte array
|
||||
* @return true if the flow update was true, false otherwise
|
||||
*/
|
||||
boolean update(byte[] flow);
|
||||
}
|
|
@ -160,7 +160,7 @@ public class UpdateAssetOperationHandlerTest {
|
|||
// given
|
||||
C2Operation operation = operation(ASSET_URL, ASSET_FILE_NAME, FORCE_DOWNLOAD);
|
||||
when(assetUpdatePrecondition.test(ASSET_FILE_NAME, parseBoolean(FORCE_DOWNLOAD))).thenReturn(TRUE);
|
||||
when(c2Client.retrieveUpdateContent(ASSET_URL)).thenReturn(empty());
|
||||
when(c2Client.retrieveUpdateAssetContent(ASSET_URL)).thenReturn(empty());
|
||||
|
||||
// when
|
||||
C2OperationAck result = testHandler.handle(operation);
|
||||
|
@ -177,7 +177,7 @@ public class UpdateAssetOperationHandlerTest {
|
|||
C2Operation operation = operation(ASSET_URL, ASSET_FILE_NAME, FORCE_DOWNLOAD);
|
||||
when(assetUpdatePrecondition.test(ASSET_FILE_NAME, parseBoolean(FORCE_DOWNLOAD))).thenReturn(TRUE);
|
||||
byte[] mockUpdateContent = new byte[0];
|
||||
when(c2Client.retrieveUpdateContent(ASSET_URL)).thenReturn(Optional.of(mockUpdateContent));
|
||||
when(c2Client.retrieveUpdateAssetContent(ASSET_URL)).thenReturn(Optional.of(mockUpdateContent));
|
||||
when(assetPersistFunction.apply(ASSET_FILE_NAME, mockUpdateContent)).thenReturn(FALSE);
|
||||
|
||||
// when
|
||||
|
@ -195,7 +195,7 @@ public class UpdateAssetOperationHandlerTest {
|
|||
C2Operation operation = operation(ASSET_URL, ASSET_FILE_NAME, FORCE_DOWNLOAD);
|
||||
when(assetUpdatePrecondition.test(ASSET_FILE_NAME, parseBoolean(FORCE_DOWNLOAD))).thenReturn(TRUE);
|
||||
byte[] mockUpdateContent = new byte[0];
|
||||
when(c2Client.retrieveUpdateContent(ASSET_URL)).thenReturn(Optional.of(mockUpdateContent));
|
||||
when(c2Client.retrieveUpdateAssetContent(ASSET_URL)).thenReturn(Optional.of(mockUpdateContent));
|
||||
when(assetPersistFunction.apply(ASSET_FILE_NAME, mockUpdateContent)).thenReturn(TRUE);
|
||||
|
||||
// when
|
||||
|
|
|
@ -30,7 +30,6 @@ import java.util.Collections;
|
|||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
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;
|
||||
|
@ -82,9 +81,9 @@ public class UpdateConfigurationOperationHandlerTest {
|
|||
|
||||
@Test
|
||||
void testHandleFlowIdInArg() {
|
||||
Function<byte[], Boolean> successUpdate = x -> true;
|
||||
UpdateConfigurationStrategy successUpdate = flow -> true;
|
||||
when(flowIdHolder.getFlowId()).thenReturn(FLOW_ID);
|
||||
when(client.retrieveUpdateContent(any())).thenReturn(Optional.of("content".getBytes()));
|
||||
when(client.retrieveUpdateConfigurationContent(any())).thenReturn(Optional.of("content".getBytes()));
|
||||
when(client.getCallbackUrl(any(), any())).thenReturn(Optional.of(INCORRECT_LOCATION));
|
||||
UpdateConfigurationOperationHandler handler = new UpdateConfigurationOperationHandler(client, flowIdHolder, successUpdate, operandPropertiesProvider);
|
||||
C2Operation operation = new C2Operation();
|
||||
|
@ -117,9 +116,9 @@ public class UpdateConfigurationOperationHandlerTest {
|
|||
|
||||
@Test
|
||||
void testHandleReturnsNotAppliedWithContentApplyIssues() {
|
||||
Function<byte[], Boolean> failedToUpdate = x -> false;
|
||||
UpdateConfigurationStrategy failedToUpdate = flow -> false;
|
||||
when(flowIdHolder.getFlowId()).thenReturn("previous_flow_id");
|
||||
when(client.retrieveUpdateContent(any())).thenReturn(Optional.of("content".getBytes()));
|
||||
when(client.retrieveUpdateConfigurationContent(any())).thenReturn(Optional.of("content".getBytes()));
|
||||
when(client.getCallbackUrl(any(), any())).thenReturn(Optional.of(CORRECT_LOCATION));
|
||||
UpdateConfigurationOperationHandler handler = new UpdateConfigurationOperationHandler(client, flowIdHolder, failedToUpdate, operandPropertiesProvider);
|
||||
C2Operation operation = new C2Operation();
|
||||
|
@ -134,10 +133,10 @@ public class UpdateConfigurationOperationHandlerTest {
|
|||
|
||||
@Test
|
||||
void testHandleReturnsFullyApplied() {
|
||||
Function<byte[], Boolean> successUpdate = x -> true;
|
||||
UpdateConfigurationStrategy successUpdate = flow -> true;
|
||||
when(flowIdHolder.getFlowId()).thenReturn("previous_flow_id");
|
||||
when(client.getCallbackUrl(any(), any())).thenReturn(Optional.of(CORRECT_LOCATION));
|
||||
when(client.retrieveUpdateContent(any())).thenReturn(Optional.of("content".getBytes()));
|
||||
when(client.retrieveUpdateConfigurationContent(any())).thenReturn(Optional.of("content".getBytes()));
|
||||
UpdateConfigurationOperationHandler handler = new UpdateConfigurationOperationHandler(client, flowIdHolder, successUpdate, operandPropertiesProvider);
|
||||
C2Operation operation = new C2Operation();
|
||||
operation.setIdentifier(OPERATION_ID);
|
||||
|
|
|
@ -39,7 +39,7 @@ Specific goals for MiNiFi are comprised of:
|
|||
Perspectives of the role of MiNiFi should be from the perspective of the agent acting immediately at, or directly adjacent to, source sensors, systems, or servers.
|
||||
|
||||
## Requirements
|
||||
* JRE 1.8
|
||||
* JRE 17
|
||||
|
||||
## Getting Started
|
||||
|
||||
|
@ -55,7 +55,7 @@ To run MiNiFi:
|
|||
- View the logs located in the logs folder
|
||||
$ tail -F ~/example-minifi-deploy/logs/minifi-app.log
|
||||
|
||||
- For help building your first data flow and sending data to a NiFi instance see the System Admin Guide located in the docs folder or making use of the minifi-toolkit, which aids in adapting NiFi templates to MiNiFi YAML configuration file format.
|
||||
- For help building your first data flow and sending data to a NiFi instance see the System Admin Guide located in the docs folder or making use of the minifi-toolkit, which aids in converting legacy YAML templates to the new JSON configuration file format.
|
||||
|
||||
## Getting Help
|
||||
If you have questions, you can reach out to our mailing list: dev@nifi.apache.org
|
||||
|
|
|
@ -79,13 +79,13 @@ limitations under the License.
|
|||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi.minifi</groupId>
|
||||
<artifactId>minifi-commons-schema</artifactId>
|
||||
<version>${project.version}</version>
|
||||
<artifactId>minifi-commons-api</artifactId>
|
||||
<version>2.0.0-SNAPSHOT</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi.minifi</groupId>
|
||||
<artifactId>minifi-commons-api</artifactId>
|
||||
<version>${project.version}</version>
|
||||
<artifactId>minifi-commons-framework</artifactId>
|
||||
<version>2.0.0-SNAPSHOT</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.eclipse.jetty</groupId>
|
||||
|
|
|
@ -46,6 +46,8 @@ import org.apache.nifi.minifi.bootstrap.service.ReloadService;
|
|||
import org.apache.nifi.minifi.bootstrap.util.ProcessUtils;
|
||||
import org.apache.nifi.minifi.bootstrap.util.UnixProcessUtils;
|
||||
import org.apache.nifi.minifi.commons.api.MiNiFiCommandState;
|
||||
import org.apache.nifi.minifi.commons.service.FlowEnrichService;
|
||||
import org.apache.nifi.properties.ApplicationProperties;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
|
@ -71,7 +73,6 @@ public class RunMiNiFi implements ConfigurationFileHolder {
|
|||
public static final Logger DEFAULT_LOGGER = LoggerFactory.getLogger(RunMiNiFi.class);
|
||||
|
||||
public static final String CONF_DIR_KEY = "conf.dir";
|
||||
public static final String MINIFI_CONFIG_FILE_KEY = "nifi.minifi.config";
|
||||
public static final String STATUS_FILE_PID_KEY = "pid";
|
||||
public static final int UNINITIALIZED = -1;
|
||||
private static final String STATUS_FILE_PORT_KEY = "port";
|
||||
|
@ -115,7 +116,8 @@ public class RunMiNiFi implements ConfigurationFileHolder {
|
|||
MiNiFiStatusProvider miNiFiStatusProvider = new MiNiFiStatusProvider(miNiFiCommandSender, processUtils);
|
||||
periodicStatusReporterManager =
|
||||
new PeriodicStatusReporterManager(bootstrapProperties, miNiFiStatusProvider, miNiFiCommandSender, miNiFiParameters);
|
||||
MiNiFiConfigurationChangeListener configurationChangeListener = new MiNiFiConfigurationChangeListener(this, DEFAULT_LOGGER, bootstrapFileProvider);
|
||||
MiNiFiConfigurationChangeListener configurationChangeListener = new MiNiFiConfigurationChangeListener(this, DEFAULT_LOGGER, bootstrapFileProvider,
|
||||
new FlowEnrichService(new ApplicationProperties(bootstrapProperties)));
|
||||
configurationChangeCoordinator = new ConfigurationChangeCoordinator(bootstrapFileProvider, this, singleton(configurationChangeListener));
|
||||
|
||||
currentPortProvider = new CurrentPortProvider(miNiFiCommandSender, miNiFiParameters, processUtils);
|
||||
|
|
|
@ -18,27 +18,21 @@
|
|||
package org.apache.nifi.minifi.bootstrap.command;
|
||||
|
||||
import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
|
||||
import static org.apache.nifi.minifi.commons.api.MiNiFiCommandState.FULLY_APPLIED;
|
||||
import static org.apache.nifi.minifi.commons.api.MiNiFiCommandState.NOT_APPLIED_WITH_RESTART;
|
||||
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.RunMiNiFi.UNINITIALIZED;
|
||||
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.asByteArrayInputStream;
|
||||
import static org.apache.nifi.minifi.bootstrap.util.ConfigTransformer.generateConfigFiles;
|
||||
import static org.apache.nifi.minifi.commons.api.MiNiFiCommandState.FULLY_APPLIED;
|
||||
import static org.apache.nifi.minifi.commons.api.MiNiFiCommandState.NOT_APPLIED_WITH_RESTART;
|
||||
import static org.apache.nifi.minifi.commons.api.MiNiFiConstants.RAW_EXTENSION;
|
||||
|
||||
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.nio.file.Path;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.Properties;
|
||||
|
@ -46,6 +40,7 @@ import java.util.concurrent.TimeUnit;
|
|||
import java.util.concurrent.locks.Condition;
|
||||
import java.util.concurrent.locks.Lock;
|
||||
import java.util.concurrent.locks.ReentrantLock;
|
||||
import org.apache.commons.io.FilenameUtils;
|
||||
import org.apache.nifi.minifi.bootstrap.MiNiFiParameters;
|
||||
import org.apache.nifi.minifi.bootstrap.RunMiNiFi;
|
||||
import org.apache.nifi.minifi.bootstrap.ShutdownHook;
|
||||
|
@ -56,10 +51,13 @@ 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.MiNiFiPropertiesGenerator;
|
||||
import org.apache.nifi.minifi.bootstrap.service.MiNiFiStdLogHandler;
|
||||
import org.apache.nifi.minifi.bootstrap.service.PeriodicStatusReporterManager;
|
||||
import org.apache.nifi.minifi.commons.api.MiNiFiProperties;
|
||||
|
||||
public class StartRunner implements CommandRunner {
|
||||
|
||||
private static final int STARTUP_WAIT_SECONDS = 60;
|
||||
|
||||
private final CurrentPortProvider currentPortProvider;
|
||||
|
@ -71,15 +69,18 @@ public class StartRunner implements CommandRunner {
|
|||
private final Lock lock = new ReentrantLock();
|
||||
private final Condition startupCondition = lock.newCondition();
|
||||
private final RunMiNiFi runMiNiFi;
|
||||
private volatile ShutdownHook shutdownHook;
|
||||
private final MiNiFiExecCommandProvider miNiFiExecCommandProvider;
|
||||
private final ConfigurationChangeListener configurationChangeListener;
|
||||
private final MiNiFiPropertiesGenerator miNiFiPropertiesGenerator;
|
||||
|
||||
private volatile ShutdownHook shutdownHook;
|
||||
|
||||
private int listenPort;
|
||||
|
||||
public StartRunner(CurrentPortProvider currentPortProvider, BootstrapFileProvider bootstrapFileProvider,
|
||||
PeriodicStatusReporterManager periodicStatusReporterManager, MiNiFiStdLogHandler miNiFiStdLogHandler, MiNiFiParameters miNiFiParameters, File bootstrapConfigFile,
|
||||
RunMiNiFi runMiNiFi, MiNiFiExecCommandProvider miNiFiExecCommandProvider, ConfigurationChangeListener configurationChangeListener) {
|
||||
PeriodicStatusReporterManager periodicStatusReporterManager, MiNiFiStdLogHandler miNiFiStdLogHandler,
|
||||
MiNiFiParameters miNiFiParameters, File bootstrapConfigFile, RunMiNiFi runMiNiFi,
|
||||
MiNiFiExecCommandProvider miNiFiExecCommandProvider, ConfigurationChangeListener configurationChangeListener) {
|
||||
this.currentPortProvider = currentPortProvider;
|
||||
this.bootstrapFileProvider = bootstrapFileProvider;
|
||||
this.periodicStatusReporterManager = periodicStatusReporterManager;
|
||||
|
@ -89,10 +90,12 @@ public class StartRunner implements CommandRunner {
|
|||
this.runMiNiFi = runMiNiFi;
|
||||
this.miNiFiExecCommandProvider = miNiFiExecCommandProvider;
|
||||
this.configurationChangeListener = configurationChangeListener;
|
||||
this.miNiFiPropertiesGenerator = new MiNiFiPropertiesGenerator();
|
||||
}
|
||||
|
||||
/**
|
||||
* Starts (and restarts) MiNiFi process during the whole lifecycle of the bootstrap process.
|
||||
*
|
||||
* @param args the input arguments
|
||||
* @return status code
|
||||
*/
|
||||
|
@ -107,7 +110,7 @@ public class StartRunner implements CommandRunner {
|
|||
return OK.getStatusCode();
|
||||
}
|
||||
|
||||
private void start() throws IOException, InterruptedException {
|
||||
private void start() throws IOException, InterruptedException, ConfigurationChangeException {
|
||||
Integer port = currentPortProvider.getCurrentPort();
|
||||
if (port != null) {
|
||||
CMD_LOGGER.info("Apache MiNiFi is already running, listening to Bootstrap on port {}", port);
|
||||
|
@ -121,10 +124,11 @@ public class StartRunner implements CommandRunner {
|
|||
|
||||
Properties bootstrapProperties = bootstrapFileProvider.getBootstrapProperties();
|
||||
String confDir = bootstrapProperties.getProperty(CONF_DIR_KEY);
|
||||
initConfigFiles(bootstrapProperties, confDir);
|
||||
|
||||
generateMiNiFiProperties(bootstrapProperties, confDir);
|
||||
regenerateFlowConfiguration(bootstrapProperties.getProperty(MiNiFiProperties.NIFI_MINIFI_FLOW_CONFIG.getKey()));
|
||||
|
||||
Process process = startMiNiFi();
|
||||
|
||||
try {
|
||||
while (true) {
|
||||
if (process.isAlive()) {
|
||||
|
@ -144,7 +148,7 @@ public class StartRunner implements CommandRunner {
|
|||
Thread.sleep(5000L);
|
||||
continue;
|
||||
}
|
||||
process = restartMiNifi(bootstrapProperties, confDir);
|
||||
process = restartMiNifi(confDir);
|
||||
// failed to start process
|
||||
if (process == null) {
|
||||
return;
|
||||
|
@ -161,18 +165,13 @@ public class StartRunner implements CommandRunner {
|
|||
}
|
||||
}
|
||||
|
||||
private Process restartMiNifi(Properties bootstrapProperties, String confDir) throws IOException {
|
||||
private Process restartMiNifi(String confDir) throws IOException {
|
||||
Process process;
|
||||
boolean previouslyStarted = runMiNiFi.isNiFiStarted();
|
||||
boolean configChangeSuccessful = true;
|
||||
if (!previouslyStarted) {
|
||||
File swapConfigFile = bootstrapFileProvider.getConfigYmlSwapFile();
|
||||
File bootstrapSwapConfigFile = bootstrapFileProvider.getBootstrapConfSwapFile();
|
||||
if (swapConfigFile.exists()) {
|
||||
if (!revertFlowConfig(bootstrapProperties, confDir, swapConfigFile)) {
|
||||
return null;
|
||||
}
|
||||
} else if(bootstrapSwapConfigFile.exists()) {
|
||||
if (bootstrapSwapConfigFile.exists()) {
|
||||
if (!revertBootstrapConfig(confDir, bootstrapSwapConfigFile)) {
|
||||
return null;
|
||||
}
|
||||
|
@ -202,36 +201,14 @@ public class StartRunner implements CommandRunner {
|
|||
return process;
|
||||
}
|
||||
|
||||
private boolean revertFlowConfig(Properties bootstrapProperties, String confDir, File swapConfigFile) throws IOException {
|
||||
DEFAULT_LOGGER.info("Flow Swap file exists, MiNiFi failed trying to change configuration. Reverting to old configuration.");
|
||||
|
||||
try {
|
||||
ByteBuffer tempConfigFile = generateConfigFiles(Files.newInputStream(swapConfigFile.toPath()), confDir, bootstrapProperties);
|
||||
runMiNiFi.getConfigFileReference().set(tempConfigFile.asReadOnlyBuffer());
|
||||
} catch (ConfigurationChangeException e) {
|
||||
DEFAULT_LOGGER.error("The flow swap file is malformed, unable to restart from prior state. Will not attempt to restart MiNiFi. Swap File should be cleaned up manually.");
|
||||
return false;
|
||||
}
|
||||
|
||||
Files.copy(swapConfigFile.toPath(), Paths.get(bootstrapProperties.getProperty(MINIFI_CONFIG_FILE_KEY)), REPLACE_EXISTING);
|
||||
|
||||
DEFAULT_LOGGER.info("Replacing flow config file with swap file and deleting swap file");
|
||||
if (!swapConfigFile.delete()) {
|
||||
DEFAULT_LOGGER.warn("The flow swap file failed to delete after replacing using it to revert to the old configuration. It should be cleaned up manually.");
|
||||
}
|
||||
runMiNiFi.setReloading(false);
|
||||
return true;
|
||||
}
|
||||
|
||||
private boolean revertBootstrapConfig(String confDir, File bootstrapSwapConfigFile) throws IOException {
|
||||
DEFAULT_LOGGER.info("Bootstrap Swap file exists, MiNiFi failed trying to change configuration. Reverting to old configuration.");
|
||||
|
||||
Files.copy(bootstrapSwapConfigFile.toPath(), bootstrapConfigFile.toPath(), REPLACE_EXISTING);
|
||||
try {
|
||||
ByteBuffer tempConfigFile = generateConfigFiles(asByteArrayInputStream(runMiNiFi.getConfigFileReference().get().duplicate()), confDir, bootstrapFileProvider.getBootstrapProperties());
|
||||
runMiNiFi.getConfigFileReference().set(tempConfigFile.asReadOnlyBuffer());
|
||||
miNiFiPropertiesGenerator.generateMinifiProperties(confDir, bootstrapFileProvider.getBootstrapProperties());
|
||||
} catch (ConfigurationChangeException e) {
|
||||
DEFAULT_LOGGER.error("The bootstrap swap file is malformed, unable to restart from prior state. Will not attempt to restart MiNiFi. Swap File should be cleaned up manually.");
|
||||
DEFAULT_LOGGER.error("Unable to create MiNiFi properties file. Will not attempt to restart MiNiFi. Swap File should be cleaned up manually.");
|
||||
return false;
|
||||
}
|
||||
|
||||
|
@ -262,7 +239,6 @@ public class StartRunner implements CommandRunner {
|
|||
try {
|
||||
Thread.sleep(1000L);
|
||||
if (runMiNiFi.getReloading() && runMiNiFi.isNiFiStarted()) {
|
||||
deleteSwapFile(bootstrapFileProvider.getConfigYmlSwapFile());
|
||||
deleteSwapFile(bootstrapFileProvider.getBootstrapConfSwapFile());
|
||||
runMiNiFi.setReloading(false);
|
||||
}
|
||||
|
@ -283,24 +259,26 @@ public class StartRunner implements CommandRunner {
|
|||
}
|
||||
}
|
||||
|
||||
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);
|
||||
private void generateMiNiFiProperties(Properties bootstrapProperties, String confDir) {
|
||||
DEFAULT_LOGGER.debug("Generating minifi.properties from bootstrap.conf");
|
||||
try {
|
||||
miNiFiPropertiesGenerator.generateMinifiProperties(confDir, bootstrapProperties);
|
||||
} catch (ConfigurationChangeException e) {
|
||||
String malformedConfigFileMessage = "The config file is malformed, unable to start.";
|
||||
DEFAULT_LOGGER.error(malformedConfigFileMessage, e);
|
||||
throw new StartupFailureException(malformedConfigFileMessage);
|
||||
throw new StartupFailureException("Unable to create MiNiFi properties file", e);
|
||||
}
|
||||
}
|
||||
|
||||
private void regenerateFlowConfiguration(String flowConfigFileLocation) throws ConfigurationChangeException, IOException {
|
||||
Path flowConfigFile = Path.of(flowConfigFileLocation).toAbsolutePath();
|
||||
String currentFlowConfigFileBaseName = FilenameUtils.getBaseName(flowConfigFile.toString());
|
||||
Path rawConfigFile = flowConfigFile.getParent().resolve(currentFlowConfigFileBaseName + RAW_EXTENSION);
|
||||
if (Files.exists(rawConfigFile)) {
|
||||
DEFAULT_LOGGER.debug("Regenerating flow configuration {} from raw flow configuration {}", flowConfigFile, rawConfigFile);
|
||||
configurationChangeListener.handleChange(Files.newInputStream(rawConfigFile));
|
||||
}
|
||||
}
|
||||
|
||||
private Process startMiNiFi() throws IOException {
|
||||
|
||||
MiNiFiListener listener = new MiNiFiListener();
|
||||
listenPort = listener.start(runMiNiFi, bootstrapFileProvider, configurationChangeListener);
|
||||
|
||||
|
@ -309,7 +287,7 @@ public class StartRunner implements CommandRunner {
|
|||
return startMiNiFiProcess(getProcessBuilder());
|
||||
}
|
||||
|
||||
private ProcessBuilder getProcessBuilder() throws IOException{
|
||||
private ProcessBuilder getProcessBuilder() throws IOException {
|
||||
ProcessBuilder builder = new ProcessBuilder();
|
||||
File workingDir = getWorkingDir();
|
||||
|
||||
|
|
|
@ -14,18 +14,22 @@
|
|||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.nifi.minifi.bootstrap.configuration;
|
||||
|
||||
import static java.util.Optional.ofNullable;
|
||||
import static java.util.function.Predicate.not;
|
||||
import static java.util.stream.Collectors.toList;
|
||||
|
||||
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 java.util.stream.Stream;
|
||||
import org.apache.nifi.minifi.bootstrap.RunMiNiFi;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.ingestors.interfaces.ChangeIngestor;
|
||||
import org.apache.nifi.minifi.bootstrap.service.BootstrapFileProvider;
|
||||
|
@ -35,64 +39,72 @@ import org.slf4j.LoggerFactory;
|
|||
|
||||
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 static final Logger LOGGER = LoggerFactory.getLogger(ConfigurationChangeCoordinator.class);
|
||||
public static final String NOTIFIER_INGESTORS_KEY = "nifi.minifi.notifier.ingestors";
|
||||
|
||||
private final Set<ConfigurationChangeListener> configurationChangeListeners;
|
||||
private final Set<ChangeIngestor> changeIngestors = new HashSet<>();
|
||||
private static final Logger LOGGER = LoggerFactory.getLogger(ConfigurationChangeCoordinator.class);
|
||||
private static final String COMMA = ",";
|
||||
|
||||
private final BootstrapFileProvider bootstrapFileProvider;
|
||||
private final RunMiNiFi runMiNiFi;
|
||||
private final Set<ConfigurationChangeListener> configurationChangeListeners;
|
||||
private final Set<ChangeIngestor> changeIngestors;
|
||||
|
||||
public ConfigurationChangeCoordinator(BootstrapFileProvider bootstrapFileProvider, RunMiNiFi runMiNiFi,
|
||||
Set<ConfigurationChangeListener> miNiFiConfigurationChangeListeners) {
|
||||
Set<ConfigurationChangeListener> miNiFiConfigurationChangeListeners) {
|
||||
this.bootstrapFileProvider = bootstrapFileProvider;
|
||||
this.runMiNiFi = runMiNiFi;
|
||||
this.configurationChangeListeners = Optional.ofNullable(miNiFiConfigurationChangeListeners).map(Collections::unmodifiableSet).orElse(Collections.emptySet());
|
||||
this.configurationChangeListeners = ofNullable(miNiFiConfigurationChangeListeners).map(Collections::unmodifiableSet).orElse(Collections.emptySet());
|
||||
this.changeIngestors = new HashSet<>();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Collection<ListenerHandleResult> notifyListeners(ByteBuffer newFlowConfig) {
|
||||
LOGGER.info("Notifying Listeners of a change");
|
||||
return configurationChangeListeners.stream()
|
||||
.map(listener -> notifyListener(newFlowConfig, listener))
|
||||
.collect(toList());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
closeIngestors();
|
||||
}
|
||||
|
||||
/**
|
||||
* 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() throws IOException{
|
||||
public void start() throws IOException {
|
||||
initialize();
|
||||
changeIngestors.forEach(ChangeIngestor::start);
|
||||
}
|
||||
|
||||
/**
|
||||
* Provides an immutable collection of listeners for the notifier instance
|
||||
*
|
||||
* @return a collection of those listeners registered for notifications
|
||||
*/
|
||||
public Set<ConfigurationChangeListener> getChangeListeners() {
|
||||
return Collections.unmodifiableSet(configurationChangeListeners);
|
||||
}
|
||||
|
||||
/**
|
||||
* Provide the mechanism by which listeners are notified
|
||||
*/
|
||||
public Collection<ListenerHandleResult> notifyListeners(ByteBuffer newConfig) {
|
||||
LOGGER.info("Notifying Listeners of a change");
|
||||
|
||||
Collection<ListenerHandleResult> listenerHandleResults = new ArrayList<>(configurationChangeListeners.size());
|
||||
for (final ConfigurationChangeListener listener : getChangeListeners()) {
|
||||
ListenerHandleResult result;
|
||||
try {
|
||||
listener.handleChange(new ByteBufferInputStream(newConfig.duplicate()));
|
||||
result = new ListenerHandleResult(listener);
|
||||
} catch (ConfigurationChangeException ex) {
|
||||
result = new ListenerHandleResult(listener, ex);
|
||||
}
|
||||
listenerHandleResults.add(result);
|
||||
LOGGER.info("Listener notification result: {}", result);
|
||||
private ListenerHandleResult notifyListener(ByteBuffer newFlowConfig, ConfigurationChangeListener listener) {
|
||||
try {
|
||||
listener.handleChange(new ByteBufferInputStream(newFlowConfig.duplicate()));
|
||||
ListenerHandleResult listenerHandleResult = new ListenerHandleResult(listener);
|
||||
LOGGER.info("Listener notification result {}", listenerHandleResult);
|
||||
return listenerHandleResult;
|
||||
} catch (ConfigurationChangeException ex) {
|
||||
ListenerHandleResult listenerHandleResult = new ListenerHandleResult(listener, ex);
|
||||
LOGGER.error("Listener notification result {} with failure {}", listenerHandleResult, ex);
|
||||
return listenerHandleResult;
|
||||
}
|
||||
return listenerHandleResults;
|
||||
}
|
||||
|
||||
private void initialize() throws IOException {
|
||||
closeIngestors();
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
Properties bootstrapProperties = bootstrapFileProvider.getBootstrapProperties();
|
||||
ofNullable(bootstrapProperties.getProperty(NOTIFIER_INGESTORS_KEY))
|
||||
.filter(not(String::isBlank))
|
||||
.map(ingestors -> ingestors.split(COMMA))
|
||||
.stream()
|
||||
.flatMap(Stream::of)
|
||||
.map(String::trim)
|
||||
.forEach(ingestorClassname -> instantiateIngestor(bootstrapProperties, ingestorClassname));
|
||||
}
|
||||
|
||||
private void closeIngestors() {
|
||||
try {
|
||||
for (ChangeIngestor changeIngestor : changeIngestors) {
|
||||
changeIngestor.close();
|
||||
|
@ -103,25 +115,15 @@ public class ConfigurationChangeCoordinator implements Closeable, ConfigurationC
|
|||
}
|
||||
}
|
||||
|
||||
private void initialize() throws IOException {
|
||||
close();
|
||||
Properties bootstrapProperties = bootstrapFileProvider.getBootstrapProperties();
|
||||
// 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);
|
||||
}
|
||||
}
|
||||
private void instantiateIngestor(Properties bootstrapProperties, String ingestorClassname) {
|
||||
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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,8 +19,7 @@ package org.apache.nifi.minifi.bootstrap.configuration;
|
|||
import java.io.InputStream;
|
||||
|
||||
/**
|
||||
* Interface for handling events detected and driven by an associated {@link ConfigurationChangeCoordinator} to which the listener
|
||||
* has registered via {@link ConfigurationChangeCoordinator#registerListener(ConfigurationChangeListener)}.
|
||||
* Interface for handling events detected and driven by an associated {@link ConfigurationChangeCoordinator}
|
||||
*/
|
||||
public interface ConfigurationChangeListener {
|
||||
|
||||
|
|
|
@ -19,11 +19,11 @@ package org.apache.nifi.minifi.bootstrap.configuration;
|
|||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Collection;
|
||||
import java.util.Set;
|
||||
|
||||
public interface ConfigurationChangeNotifier {
|
||||
|
||||
Set<ConfigurationChangeListener> getChangeListeners();
|
||||
|
||||
Collection<ListenerHandleResult> notifyListeners(ByteBuffer is);
|
||||
/**
|
||||
* Provides the mechanism by which listeners are notified
|
||||
*/
|
||||
Collection<ListenerHandleResult> notifyListeners(ByteBuffer newFlowConfig);
|
||||
}
|
||||
|
|
|
@ -17,10 +17,8 @@
|
|||
|
||||
package org.apache.nifi.minifi.bootstrap.configuration.differentiators;
|
||||
|
||||
import java.io.DataInputStream;
|
||||
import java.io.EOFException;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import static java.util.Optional.ofNullable;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import org.apache.nifi.minifi.bootstrap.ConfigurationFileHolder;
|
||||
|
@ -29,59 +27,30 @@ import org.slf4j.LoggerFactory;
|
|||
|
||||
public abstract class WholeConfigDifferentiator {
|
||||
|
||||
public static final String WHOLE_CONFIG_KEY = "Whole Config";
|
||||
|
||||
private final static Logger logger = LoggerFactory.getLogger(WholeConfigDifferentiator.class);
|
||||
|
||||
public static final String WHOLE_CONFIG_KEY = "Whole Config";
|
||||
|
||||
volatile ConfigurationFileHolder configurationFileHolder;
|
||||
|
||||
boolean compareInputStreamToConfigFile(InputStream inputStream) throws IOException {
|
||||
logger.debug("Checking if change is different");
|
||||
AtomicReference<ByteBuffer> currentConfigFileReference = configurationFileHolder.getConfigFileReference();
|
||||
ByteBuffer currentConfigFile = currentConfigFileReference.get();
|
||||
ByteBuffer byteBuffer = ByteBuffer.allocate(currentConfigFile.limit());
|
||||
DataInputStream dataInputStream = new DataInputStream(inputStream);
|
||||
try {
|
||||
dataInputStream.readFully(byteBuffer.array());
|
||||
} catch (EOFException e) {
|
||||
logger.debug("New config is shorter than the current. Must be different.");
|
||||
return true;
|
||||
}
|
||||
logger.debug("Read the input");
|
||||
|
||||
if (dataInputStream.available() != 0) {
|
||||
return true;
|
||||
} else {
|
||||
return byteBuffer.compareTo(currentConfigFile) != 0;
|
||||
}
|
||||
}
|
||||
protected volatile ConfigurationFileHolder configurationFileHolder;
|
||||
|
||||
public void initialize(ConfigurationFileHolder configurationFileHolder) {
|
||||
this.configurationFileHolder = configurationFileHolder;
|
||||
}
|
||||
|
||||
|
||||
public static class InputStreamInput extends WholeConfigDifferentiator implements Differentiator<InputStream> {
|
||||
public boolean isNew(InputStream inputStream) throws IOException {
|
||||
return compareInputStreamToConfigFile(inputStream);
|
||||
public static class ByteBufferInputDifferentiator extends WholeConfigDifferentiator implements Differentiator<ByteBuffer> {
|
||||
public boolean isNew(ByteBuffer newFlowConfig) {
|
||||
AtomicReference<ByteBuffer> currentFlowConfigReference = configurationFileHolder.getConfigFileReference();
|
||||
ByteBuffer currentFlowConfig = currentFlowConfigReference.get();
|
||||
boolean compareResult = ofNullable(currentFlowConfig)
|
||||
.map(newFlowConfig::compareTo)
|
||||
.map(result -> result != 0)
|
||||
.orElse(Boolean.TRUE);
|
||||
logger.debug("New flow is different from existing flow: {}", compareResult);
|
||||
return compareResult;
|
||||
}
|
||||
}
|
||||
|
||||
public static class ByteBufferInput extends WholeConfigDifferentiator implements Differentiator<ByteBuffer> {
|
||||
public boolean isNew(ByteBuffer inputBuffer) {
|
||||
AtomicReference<ByteBuffer> currentConfigFileReference = configurationFileHolder.getConfigFileReference();
|
||||
ByteBuffer currentConfigFile = currentConfigFileReference.get();
|
||||
return inputBuffer.compareTo(currentConfigFile) != 0;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
public static Differentiator<InputStream> getInputStreamDifferentiator() {
|
||||
return new InputStreamInput();
|
||||
}
|
||||
|
||||
public static Differentiator<ByteBuffer> getByteBufferDifferentiator() {
|
||||
return new ByteBufferInput();
|
||||
return new ByteBufferInputDifferentiator();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -26,25 +26,22 @@ import java.util.concurrent.atomic.AtomicReference;
|
|||
import org.apache.nifi.minifi.bootstrap.ConfigurationFileHolder;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeNotifier;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.ingestors.interfaces.ChangeIngestor;
|
||||
import org.slf4j.Logger;
|
||||
|
||||
public abstract class AbstractPullChangeIngestor implements Runnable, ChangeIngestor {
|
||||
|
||||
// 5 minute default pulling period
|
||||
protected static final String DEFAULT_POLLING_PERIOD = "300000";
|
||||
protected static Logger logger;
|
||||
protected static final String DEFAULT_POLLING_PERIOD_MILLISECONDS = "300000";
|
||||
|
||||
protected final AtomicInteger pollingPeriodMS = new AtomicInteger();
|
||||
private final ScheduledThreadPoolExecutor scheduledThreadPoolExecutor = new ScheduledThreadPoolExecutor(1);
|
||||
protected volatile ConfigurationChangeNotifier configurationChangeNotifier;
|
||||
protected volatile ConfigurationFileHolder configurationFileHolder;
|
||||
protected final AtomicReference<Properties> properties = new AtomicReference<>();
|
||||
|
||||
private final ScheduledThreadPoolExecutor scheduledThreadPoolExecutor = new ScheduledThreadPoolExecutor(1);
|
||||
|
||||
protected volatile ConfigurationChangeNotifier configurationChangeNotifier;
|
||||
|
||||
@Override
|
||||
public void initialize(Properties properties, ConfigurationFileHolder configurationFileHolder, ConfigurationChangeNotifier configurationChangeNotifier) {
|
||||
this.configurationChangeNotifier = configurationChangeNotifier;
|
||||
this.properties.set(properties);
|
||||
this.configurationFileHolder = configurationFileHolder;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -14,24 +14,29 @@
|
|||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.nifi.minifi.bootstrap.configuration.ingestors;
|
||||
|
||||
import static java.nio.ByteBuffer.wrap;
|
||||
import static java.nio.file.StandardWatchEventKinds.ENTRY_MODIFY;
|
||||
import static java.util.Collections.emptyList;
|
||||
import static java.util.Optional.ofNullable;
|
||||
import static java.util.concurrent.TimeUnit.SECONDS;
|
||||
import static java.util.function.Predicate.not;
|
||||
import static org.apache.commons.io.FilenameUtils.getBaseName;
|
||||
import static org.apache.commons.io.IOUtils.toByteArray;
|
||||
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 static org.apache.nifi.minifi.commons.api.MiNiFiConstants.RAW_EXTENSION;
|
||||
|
||||
import java.io.FileInputStream;
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.file.FileSystems;
|
||||
import java.nio.file.Path;
|
||||
import java.nio.file.Paths;
|
||||
import java.nio.file.WatchEvent;
|
||||
import java.nio.file.WatchKey;
|
||||
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;
|
||||
|
@ -39,13 +44,12 @@ import java.util.concurrent.Executors;
|
|||
import java.util.concurrent.ScheduledExecutorService;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.function.Supplier;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.nifi.minifi.bootstrap.ConfigurationFileHolder;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeNotifier;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.differentiators.Differentiator;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.differentiators.WholeConfigDifferentiator;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.ingestors.interfaces.ChangeIngestor;
|
||||
import org.apache.nifi.minifi.bootstrap.util.ConfigTransformer;
|
||||
import org.apache.nifi.minifi.commons.api.MiNiFiProperties;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
|
@ -56,165 +60,160 @@ import org.slf4j.LoggerFactory;
|
|||
*/
|
||||
public class FileChangeIngestor implements Runnable, ChangeIngestor {
|
||||
|
||||
private static final Map<String, Supplier<Differentiator<ByteBuffer>>> DIFFERENTIATOR_CONSTRUCTOR_MAP;
|
||||
private static final Map<String, Supplier<Differentiator<ByteBuffer>>> DIFFERENTIATOR_CONSTRUCTOR_MAP = Map.of(
|
||||
WHOLE_CONFIG_KEY, WholeConfigDifferentiator::getByteBufferDifferentiator
|
||||
);
|
||||
|
||||
static {
|
||||
HashMap<String, Supplier<Differentiator<ByteBuffer>>> tempMap = new HashMap<>();
|
||||
tempMap.put(WHOLE_CONFIG_KEY, WholeConfigDifferentiator::getByteBufferDifferentiator);
|
||||
|
||||
DIFFERENTIATOR_CONSTRUCTOR_MAP = Collections.unmodifiableMap(tempMap);
|
||||
}
|
||||
|
||||
|
||||
protected static final int DEFAULT_POLLING_PERIOD_INTERVAL = 15;
|
||||
protected static final TimeUnit DEFAULT_POLLING_PERIOD_UNIT = TimeUnit.SECONDS;
|
||||
static final String CONFIG_FILE_BASE_KEY = NOTIFIER_INGESTORS_KEY + ".file";
|
||||
static final String CONFIG_FILE_PATH_KEY = CONFIG_FILE_BASE_KEY + ".config.path";
|
||||
static final String POLLING_PERIOD_INTERVAL_KEY = CONFIG_FILE_BASE_KEY + ".polling.period.seconds";
|
||||
static final int DEFAULT_POLLING_PERIOD_INTERVAL = 15;
|
||||
|
||||
private final static Logger logger = LoggerFactory.getLogger(FileChangeIngestor.class);
|
||||
private static final String CONFIG_FILE_BASE_KEY = NOTIFIER_INGESTORS_KEY + ".file";
|
||||
|
||||
protected static final String CONFIG_FILE_PATH_KEY = CONFIG_FILE_BASE_KEY + ".config.path";
|
||||
protected static final String POLLING_PERIOD_INTERVAL_KEY = CONFIG_FILE_BASE_KEY + ".polling.period.seconds";
|
||||
public static final String DIFFERENTIATOR_KEY = CONFIG_FILE_BASE_KEY + ".differentiator";
|
||||
private static final TimeUnit DEFAULT_POLLING_PERIOD_UNIT = SECONDS;
|
||||
private static final String DIFFERENTIATOR_KEY = CONFIG_FILE_BASE_KEY + ".differentiator";
|
||||
|
||||
private volatile Differentiator<ByteBuffer> differentiator;
|
||||
private volatile ConfigurationChangeNotifier configurationChangeNotifier;
|
||||
|
||||
private ScheduledExecutorService executorService;
|
||||
|
||||
private Path configFilePath;
|
||||
private WatchService watchService;
|
||||
private long pollingSeconds;
|
||||
private volatile Differentiator<ByteBuffer> differentiator;
|
||||
private volatile ConfigurationChangeNotifier configurationChangeNotifier;
|
||||
private volatile ConfigurationFileHolder configurationFileHolder;
|
||||
private volatile Properties properties;
|
||||
private ScheduledExecutorService executorService;
|
||||
|
||||
protected static WatchService initializeWatcher(Path filePath) {
|
||||
@Override
|
||||
public void initialize(Properties properties, ConfigurationFileHolder configurationFileHolder, ConfigurationChangeNotifier configurationChangeNotifier) {
|
||||
Path configFile = ofNullable(properties.getProperty(CONFIG_FILE_PATH_KEY))
|
||||
.filter(not(String::isBlank))
|
||||
.map(Path::of)
|
||||
.map(Path::toAbsolutePath)
|
||||
.orElseThrow(() -> new IllegalArgumentException("Property, " + CONFIG_FILE_PATH_KEY + ", for the path of the config file must be specified"));
|
||||
try {
|
||||
final WatchService fsWatcher = FileSystems.getDefault().newWatchService();
|
||||
final Path watchDirectory = filePath.getParent();
|
||||
watchDirectory.register(fsWatcher, ENTRY_MODIFY);
|
||||
this.configurationChangeNotifier = configurationChangeNotifier;
|
||||
this.configFilePath = configFile;
|
||||
this.pollingSeconds = ofNullable(properties.getProperty(POLLING_PERIOD_INTERVAL_KEY, Long.toString(DEFAULT_POLLING_PERIOD_INTERVAL)))
|
||||
.map(Long::parseLong)
|
||||
.filter(duration -> duration > 0)
|
||||
.map(duration -> SECONDS.convert(duration, DEFAULT_POLLING_PERIOD_UNIT))
|
||||
.orElseThrow(() -> new IllegalArgumentException("Cannot specify a polling period with duration <=0"));
|
||||
this.watchService = initializeWatcher(configFile);
|
||||
this.differentiator = ofNullable(properties.getProperty(DIFFERENTIATOR_KEY))
|
||||
.filter(not(String::isBlank))
|
||||
.map(differentiator -> ofNullable(DIFFERENTIATOR_CONSTRUCTOR_MAP.get(differentiator))
|
||||
.map(Supplier::get)
|
||||
.orElseThrow(unableToFindDifferentiatorExceptionSupplier(differentiator)))
|
||||
.orElseGet(WholeConfigDifferentiator::getByteBufferDifferentiator);
|
||||
this.differentiator.initialize(configurationFileHolder);
|
||||
} catch (Exception e) {
|
||||
throw new IllegalStateException("Could not successfully initialize file change notifier", e);
|
||||
}
|
||||
|
||||
return fsWatcher;
|
||||
} catch (IOException ioe) {
|
||||
throw new IllegalStateException("Unable to initialize a file system watcher for the path " + filePath, ioe);
|
||||
checkConfigFileLocationCorrectness(properties, configFile);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void start() {
|
||||
executorService = Executors.newScheduledThreadPool(1, runnable -> {
|
||||
Thread notifierThread = Executors.defaultThreadFactory().newThread(runnable);
|
||||
notifierThread.setName("File Change Notifier Thread");
|
||||
notifierThread.setDaemon(true);
|
||||
return notifierThread;
|
||||
});
|
||||
executorService.scheduleWithFixedDelay(this, 0, pollingSeconds, DEFAULT_POLLING_PERIOD_UNIT);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
logger.debug("Checking for a change in {}", configFilePath);
|
||||
if (targetFileChanged()) {
|
||||
logger.debug("Target file changed, checking if it's different than current flow");
|
||||
try (FileInputStream flowCandidateInputStream = new FileInputStream(configFilePath.toFile())) {
|
||||
ByteBuffer newFlowConfig = wrap(toByteArray(flowCandidateInputStream));
|
||||
if (differentiator.isNew(newFlowConfig)) {
|
||||
logger.debug("Current flow and new flow is different, notifying listener");
|
||||
configurationChangeNotifier.notifyListeners(newFlowConfig);
|
||||
logger.debug("Listeners have been notified");
|
||||
}
|
||||
} catch (Exception e) {
|
||||
logger.error("Could not successfully notify listeners.", e);
|
||||
}
|
||||
} else {
|
||||
logger.debug("No change detected in {}", configFilePath);
|
||||
}
|
||||
}
|
||||
|
||||
protected boolean targetChanged() {
|
||||
boolean targetChanged;
|
||||
@Override
|
||||
public void close() {
|
||||
if (executorService != null) {
|
||||
executorService.shutdownNow();
|
||||
}
|
||||
}
|
||||
|
||||
Optional<WatchKey> watchKey = Optional.ofNullable(watchService.poll());
|
||||
|
||||
targetChanged = watchKey
|
||||
boolean targetFileChanged() {
|
||||
logger.trace("Attempting to acquire watch key");
|
||||
Optional<WatchKey> watchKey = ofNullable(watchService.poll());
|
||||
logger.trace("Watch key acquire with value {}", watchKey);
|
||||
boolean targetChanged = watchKey
|
||||
.map(WatchKey::pollEvents)
|
||||
.orElse(emptyList())
|
||||
.stream()
|
||||
.anyMatch(watchEvent -> ENTRY_MODIFY == watchEvent.kind()
|
||||
&& ((WatchEvent<Path>) watchEvent).context().equals(configFilePath.getName(configFilePath.getNameCount() - 1)));
|
||||
|
||||
logger.debug("Target file changed: {}", targetChanged);
|
||||
// After completing inspection, reset for detection of subsequent change events
|
||||
watchKey.map(WatchKey::reset)
|
||||
.filter(valid -> !valid)
|
||||
.ifPresent(valid -> {
|
||||
logger.error("Unable to reinitialize file system watcher.");
|
||||
throw new IllegalStateException("Unable to reinitialize file system watcher.");
|
||||
});
|
||||
|
||||
logger.trace("Watch key has been reset successfully");
|
||||
return targetChanged;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
logger.debug("Checking for a change");
|
||||
if (targetChanged()) {
|
||||
logger.debug("Target changed, checking if it's different than current flow.");
|
||||
try (FileInputStream configFile = new FileInputStream(configFilePath.toFile())) {
|
||||
ByteBuffer readOnlyNewConfig =
|
||||
ConfigTransformer.overrideNonFlowSectionsFromOriginalSchema(
|
||||
IOUtils.toByteArray(configFile), configurationFileHolder.getConfigFileReference().get().duplicate(), properties);
|
||||
|
||||
if (differentiator.isNew(readOnlyNewConfig)) {
|
||||
logger.debug("New change, notifying listener");
|
||||
configurationChangeNotifier.notifyListeners(readOnlyNewConfig);
|
||||
logger.debug("Listeners notified");
|
||||
}
|
||||
} catch (Exception e) {
|
||||
logger.error("Could not successfully notify listeners.", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void initialize(Properties properties, ConfigurationFileHolder configurationFileHolder, ConfigurationChangeNotifier configurationChangeNotifier) {
|
||||
this.properties = properties;
|
||||
this.configurationFileHolder = configurationFileHolder;
|
||||
final String rawPath = properties.getProperty(CONFIG_FILE_PATH_KEY);
|
||||
final String rawPollingDuration = properties.getProperty(POLLING_PERIOD_INTERVAL_KEY, Long.toString(DEFAULT_POLLING_PERIOD_INTERVAL));
|
||||
|
||||
if (rawPath == null || rawPath.isEmpty()) {
|
||||
throw new IllegalArgumentException("Property, " + CONFIG_FILE_PATH_KEY + ", for the path of the config file must be specified.");
|
||||
}
|
||||
|
||||
private WatchService initializeWatcher(Path filePath) {
|
||||
try {
|
||||
setConfigFilePath(Paths.get(rawPath));
|
||||
setPollingPeriod(Long.parseLong(rawPollingDuration), DEFAULT_POLLING_PERIOD_UNIT);
|
||||
setWatchService(initializeWatcher(configFilePath));
|
||||
} catch (Exception e) {
|
||||
throw new IllegalStateException("Could not successfully initialize file change notifier.", e);
|
||||
WatchService fileSystemWatcher = FileSystems.getDefault().newWatchService();
|
||||
Path watchDirectory = filePath.getParent();
|
||||
watchDirectory.register(fileSystemWatcher, ENTRY_MODIFY);
|
||||
logger.trace("Watch service registered for {}", watchDirectory);
|
||||
return fileSystemWatcher;
|
||||
} catch (IOException ioe) {
|
||||
throw new IllegalStateException("Unable to initialize a file system watcher for the path " + filePath, ioe);
|
||||
}
|
||||
|
||||
this.configurationChangeNotifier = configurationChangeNotifier;
|
||||
|
||||
final String differentiatorName = properties.getProperty(DIFFERENTIATOR_KEY);
|
||||
|
||||
if (differentiatorName != null && !differentiatorName.isEmpty()) {
|
||||
Supplier<Differentiator<ByteBuffer>> differentiatorSupplier = DIFFERENTIATOR_CONSTRUCTOR_MAP.get(differentiatorName);
|
||||
if (differentiatorSupplier == null) {
|
||||
throw new IllegalArgumentException("Property, " + DIFFERENTIATOR_KEY + ", has value " + differentiatorName + " which does not " +
|
||||
"correspond to any in the PullHttpChangeIngestor Map:" + DIFFERENTIATOR_CONSTRUCTOR_MAP.keySet());
|
||||
}
|
||||
differentiator = differentiatorSupplier.get();
|
||||
} else {
|
||||
differentiator = WholeConfigDifferentiator.getByteBufferDifferentiator();
|
||||
}
|
||||
differentiator.initialize(configurationFileHolder);
|
||||
}
|
||||
|
||||
protected void setConfigFilePath(Path configFilePath) {
|
||||
private Supplier<IllegalArgumentException> unableToFindDifferentiatorExceptionSupplier(String differentiator) {
|
||||
return () -> new IllegalArgumentException("Property, " + DIFFERENTIATOR_KEY + ", has value " + differentiator
|
||||
+ " which does not correspond to any in the FileChangeIngestor Map:" + DIFFERENTIATOR_CONSTRUCTOR_MAP.keySet());
|
||||
}
|
||||
|
||||
private void checkConfigFileLocationCorrectness(Properties properties, Path configFile) {
|
||||
Path flowConfigFile = Path.of(properties.getProperty(MiNiFiProperties.NIFI_MINIFI_FLOW_CONFIG.getKey())).toAbsolutePath();
|
||||
Path rawFlowConfigFile = flowConfigFile.getParent().resolve(getBaseName(flowConfigFile.toString()) + RAW_EXTENSION);
|
||||
if (flowConfigFile.equals(configFile) || rawFlowConfigFile.equals(configFile)) {
|
||||
throw new IllegalStateException("File ingestor config file (" + CONFIG_FILE_PATH_KEY
|
||||
+ ") must point to a different file than MiNiFi flow config file and raw flow config file");
|
||||
}
|
||||
}
|
||||
|
||||
// Methods exposed only for enable testing
|
||||
void setConfigFilePath(Path configFilePath) {
|
||||
this.configFilePath = configFilePath;
|
||||
}
|
||||
|
||||
protected void setWatchService(WatchService watchService) {
|
||||
void setWatchService(WatchService watchService) {
|
||||
this.watchService = watchService;
|
||||
}
|
||||
|
||||
protected void setConfigurationChangeNotifier(ConfigurationChangeNotifier configurationChangeNotifier) {
|
||||
void setConfigurationChangeNotifier(ConfigurationChangeNotifier configurationChangeNotifier) {
|
||||
this.configurationChangeNotifier = configurationChangeNotifier;
|
||||
}
|
||||
|
||||
protected void setDifferentiator(Differentiator<ByteBuffer> differentiator) {
|
||||
void setDifferentiator(Differentiator<ByteBuffer> differentiator) {
|
||||
this.differentiator = differentiator;
|
||||
}
|
||||
|
||||
protected void setPollingPeriod(long duration, TimeUnit unit) {
|
||||
if (duration < 0) {
|
||||
throw new IllegalArgumentException("Cannot specify a polling period with duration <=0");
|
||||
}
|
||||
this.pollingSeconds = TimeUnit.SECONDS.convert(duration, unit);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void start() {
|
||||
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);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
if (this.executorService != null) {
|
||||
this.executorService.shutdownNow();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -17,8 +17,21 @@
|
|||
|
||||
package org.apache.nifi.minifi.bootstrap.configuration.ingestors;
|
||||
|
||||
import static java.lang.Boolean.FALSE;
|
||||
import static java.lang.Boolean.parseBoolean;
|
||||
import static java.lang.Long.parseLong;
|
||||
import static java.nio.ByteBuffer.wrap;
|
||||
import static java.util.Optional.ofNullable;
|
||||
import static java.util.concurrent.TimeUnit.MILLISECONDS;
|
||||
import static java.util.function.Predicate.not;
|
||||
import static java.util.stream.Collectors.joining;
|
||||
import static java.util.stream.Collectors.toMap;
|
||||
import static org.apache.commons.lang3.StringUtils.EMPTY;
|
||||
import static org.apache.commons.lang3.StringUtils.isNotBlank;
|
||||
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 static org.eclipse.jetty.http.HttpScheme.HTTP;
|
||||
import static org.eclipse.jetty.http.HttpScheme.HTTPS;
|
||||
|
||||
import java.io.FileInputStream;
|
||||
import java.io.IOException;
|
||||
|
@ -26,49 +39,35 @@ import java.net.InetSocketAddress;
|
|||
import java.net.Proxy;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.security.KeyStore;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Arrays;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import java.util.function.Supplier;
|
||||
import javax.net.ssl.SSLContext;
|
||||
import java.util.stream.Stream;
|
||||
import javax.net.ssl.SSLSocketFactory;
|
||||
import javax.net.ssl.X509TrustManager;
|
||||
import okhttp3.Authenticator;
|
||||
import okhttp3.Credentials;
|
||||
import okhttp3.HttpUrl;
|
||||
import okhttp3.OkHttpClient;
|
||||
import okhttp3.Request;
|
||||
import okhttp3.Response;
|
||||
import okhttp3.ResponseBody;
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.minifi.bootstrap.ConfigurationFileHolder;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeNotifier;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.differentiators.Differentiator;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.differentiators.WholeConfigDifferentiator;
|
||||
import org.apache.nifi.minifi.bootstrap.util.ConfigTransformer;
|
||||
import org.apache.nifi.minifi.commons.schema.common.StringUtil;
|
||||
import org.apache.nifi.security.ssl.StandardKeyStoreBuilder;
|
||||
import org.apache.nifi.security.ssl.StandardSslContextBuilder;
|
||||
import org.apache.nifi.security.ssl.StandardTrustManagerBuilder;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
|
||||
public class PullHttpChangeIngestor extends AbstractPullChangeIngestor {
|
||||
|
||||
private static final int NOT_MODIFIED_STATUS_CODE = 304;
|
||||
private static final Map<String, Supplier<Differentiator<ByteBuffer>>> DIFFERENTIATOR_CONSTRUCTOR_MAP;
|
||||
|
||||
static {
|
||||
HashMap<String, Supplier<Differentiator<ByteBuffer>>> tempMap = new HashMap<>();
|
||||
tempMap.put(WHOLE_CONFIG_KEY, WholeConfigDifferentiator::getByteBufferDifferentiator);
|
||||
|
||||
DIFFERENTIATOR_CONSTRUCTOR_MAP = Collections.unmodifiableMap(tempMap);
|
||||
}
|
||||
|
||||
private static final String DEFAULT_CONNECT_TIMEOUT_MS = "5000";
|
||||
private static final String DEFAULT_READ_TIMEOUT_MS = "15000";
|
||||
|
||||
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";
|
||||
|
@ -90,261 +89,233 @@ public class PullHttpChangeIngestor extends AbstractPullChangeIngestor {
|
|||
public static final String DIFFERENTIATOR_KEY = PULL_HTTP_BASE_KEY + ".differentiator";
|
||||
public static final String USE_ETAG_KEY = PULL_HTTP_BASE_KEY + ".use.etag";
|
||||
public static final String OVERRIDE_SECURITY = PULL_HTTP_BASE_KEY + ".override.security";
|
||||
public static final String HTTP_HEADERS = PULL_HTTP_BASE_KEY + ".headers";
|
||||
|
||||
private static final Logger logger = LoggerFactory.getLogger(PullHttpChangeIngestor.class);
|
||||
|
||||
private static final Map<String, Supplier<Differentiator<ByteBuffer>>> DIFFERENTIATOR_CONSTRUCTOR_MAP = Map.of(
|
||||
WHOLE_CONFIG_KEY, WholeConfigDifferentiator::getByteBufferDifferentiator
|
||||
);
|
||||
private static final int NOT_MODIFIED_STATUS_CODE = 304;
|
||||
private static final String DEFAULT_CONNECT_TIMEOUT_MS = "5000";
|
||||
private static final String DEFAULT_READ_TIMEOUT_MS = "15000";
|
||||
private static final String DOUBLE_QUOTES = "\"";
|
||||
private static final String ETAG_HEADER = "ETag";
|
||||
private static final String PROXY_AUTHORIZATION_HEADER = "Proxy-Authorization";
|
||||
private static final String DEFAULT_PATH = "/";
|
||||
private static final int BAD_REQUEST_STATUS_CODE = 400;
|
||||
private static final String IF_NONE_MATCH_HEADER_KEY = "If-None-Match";
|
||||
private static final String HTTP_HEADERS_SEPARATOR = ",";
|
||||
private static final String HTTP_HEADER_KEY_VALUE_SEPARATOR = ":";
|
||||
|
||||
private final AtomicReference<OkHttpClient> httpClientReference = new AtomicReference<>();
|
||||
private final AtomicReference<Integer> portReference = new AtomicReference<>();
|
||||
private final AtomicReference<String> hostReference = new AtomicReference<>();
|
||||
private final AtomicReference<String> pathReference = new AtomicReference<>();
|
||||
private final AtomicReference<String> queryReference = new AtomicReference<>();
|
||||
private final AtomicReference<Map<String, String>> httpHeadersReference = new AtomicReference<>();
|
||||
|
||||
private volatile Differentiator<ByteBuffer> differentiator;
|
||||
private volatile String connectionScheme;
|
||||
private volatile String lastEtag = "";
|
||||
private volatile boolean useEtag = false;
|
||||
|
||||
public PullHttpChangeIngestor() {
|
||||
logger = LoggerFactory.getLogger(PullHttpChangeIngestor.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void initialize(Properties properties, ConfigurationFileHolder configurationFileHolder, ConfigurationChangeNotifier configurationChangeNotifier) {
|
||||
super.initialize(properties, configurationFileHolder, configurationChangeNotifier);
|
||||
|
||||
pollingPeriodMS.set(Integer.parseInt(properties.getProperty(PULL_HTTP_POLLING_PERIOD_KEY, DEFAULT_POLLING_PERIOD)));
|
||||
pollingPeriodMS.set(Integer.parseInt(properties.getProperty(PULL_HTTP_POLLING_PERIOD_KEY, DEFAULT_POLLING_PERIOD_MILLISECONDS)));
|
||||
if (pollingPeriodMS.get() < 1) {
|
||||
throw new IllegalArgumentException("Property, " + PULL_HTTP_POLLING_PERIOD_KEY + ", for the polling period ms must be set with a positive integer.");
|
||||
throw new IllegalArgumentException("Property, " + PULL_HTTP_POLLING_PERIOD_KEY + ", for the polling period ms must be set with a positive integer");
|
||||
}
|
||||
|
||||
final String host = properties.getProperty(HOST_KEY);
|
||||
if (host == null || host.isEmpty()) {
|
||||
throw new IllegalArgumentException("Property, " + HOST_KEY + ", for the hostname to pull configurations from must be specified.");
|
||||
}
|
||||
String host = ofNullable(properties.getProperty(HOST_KEY))
|
||||
.filter(StringUtils::isNotBlank)
|
||||
.orElseThrow(() -> new IllegalArgumentException("Property, " + HOST_KEY + ", for the hostname to pull configurations from must be specified"));
|
||||
String path = properties.getProperty(PATH_KEY, DEFAULT_PATH);
|
||||
String query = properties.getProperty(QUERY_KEY, EMPTY);
|
||||
Map<String, String> httpHeaders = ofNullable(properties.getProperty(HTTP_HEADERS))
|
||||
.filter(StringUtils::isNotBlank)
|
||||
.map(headers -> headers.split(HTTP_HEADERS_SEPARATOR))
|
||||
.stream()
|
||||
.flatMap(Arrays::stream)
|
||||
.map(String::trim)
|
||||
.map(header -> header.split(HTTP_HEADER_KEY_VALUE_SEPARATOR))
|
||||
.filter(split -> split.length == 2)
|
||||
.collect(toMap(split -> ofNullable(split[0]).map(String::trim).orElse(EMPTY), split -> ofNullable(split[1]).map(String::trim).orElse(EMPTY)));
|
||||
logger.debug("Configured HTTP headers: {}", httpHeaders);
|
||||
|
||||
final String path = properties.getProperty(PATH_KEY, "/");
|
||||
final String query = properties.getProperty(QUERY_KEY, "");
|
||||
|
||||
final String portString = (String) properties.get(PORT_KEY);
|
||||
final Integer port;
|
||||
if (portString == null) {
|
||||
throw new IllegalArgumentException("Property, " + PORT_KEY + ", for the hostname to pull configurations from must be specified.");
|
||||
} else {
|
||||
port = Integer.parseInt(portString);
|
||||
}
|
||||
|
||||
portReference.set(port);
|
||||
ofNullable(properties.get(PORT_KEY))
|
||||
.map(String.class::cast)
|
||||
.map(Integer::parseInt)
|
||||
.ifPresentOrElse(
|
||||
portReference::set,
|
||||
() -> {
|
||||
throw new IllegalArgumentException("Property, " + PORT_KEY + ", for the hostname to pull configurations from must be specified");
|
||||
});
|
||||
hostReference.set(host);
|
||||
pathReference.set(path);
|
||||
queryReference.set(query);
|
||||
|
||||
final String useEtagString = (String) properties.getOrDefault(USE_ETAG_KEY, "false");
|
||||
if ("true".equalsIgnoreCase(useEtagString) || "false".equalsIgnoreCase(useEtagString)) {
|
||||
useEtag = Boolean.parseBoolean(useEtagString);
|
||||
} else {
|
||||
throw new IllegalArgumentException("Property, " + USE_ETAG_KEY + ", to specify whether to use the ETag header, must either be a value boolean value (\"true\" or \"false\") or left to " +
|
||||
"the default value of \"false\". It is set to \"" + useEtagString + "\".");
|
||||
}
|
||||
httpHeadersReference.set(httpHeaders);
|
||||
useEtag = parseBoolean((String) properties.getOrDefault(USE_ETAG_KEY, FALSE.toString()));
|
||||
|
||||
httpClientReference.set(null);
|
||||
|
||||
final OkHttpClient.Builder okHttpClientBuilder = new OkHttpClient.Builder();
|
||||
OkHttpClient.Builder okHttpClientBuilder = new OkHttpClient.Builder()
|
||||
.connectTimeout(parseLong(properties.getProperty(CONNECT_TIMEOUT_KEY, DEFAULT_CONNECT_TIMEOUT_MS)), MILLISECONDS)
|
||||
.readTimeout(parseLong(properties.getProperty(READ_TIMEOUT_KEY, DEFAULT_READ_TIMEOUT_MS)), MILLISECONDS)
|
||||
.followRedirects(true);
|
||||
|
||||
// Set timeouts
|
||||
okHttpClientBuilder.connectTimeout(Long.parseLong(properties.getProperty(CONNECT_TIMEOUT_KEY, DEFAULT_CONNECT_TIMEOUT_MS)), TimeUnit.MILLISECONDS);
|
||||
okHttpClientBuilder.readTimeout(Long.parseLong(properties.getProperty(READ_TIMEOUT_KEY, DEFAULT_READ_TIMEOUT_MS)), TimeUnit.MILLISECONDS);
|
||||
String proxyHost = properties.getProperty(PROXY_HOST_KEY);
|
||||
if (isNotBlank(proxyHost)) {
|
||||
ofNullable(properties.getProperty(PROXY_PORT_KEY))
|
||||
.filter(StringUtils::isNotBlank)
|
||||
.map(Integer::parseInt)
|
||||
.map(port -> new InetSocketAddress(proxyHost, port))
|
||||
.map(inetSocketAddress -> new Proxy(Proxy.Type.HTTP, inetSocketAddress))
|
||||
.ifPresentOrElse(
|
||||
okHttpClientBuilder::proxy,
|
||||
() -> {
|
||||
throw new IllegalArgumentException("Proxy port required if proxy specified");
|
||||
});
|
||||
|
||||
// Set whether to follow redirects
|
||||
okHttpClientBuilder.followRedirects(true);
|
||||
|
||||
String proxyHost = properties.getProperty(PROXY_HOST_KEY, "");
|
||||
if (!proxyHost.isEmpty()) {
|
||||
String proxyPort = properties.getProperty(PROXY_PORT_KEY);
|
||||
if (proxyPort == null || proxyPort.isEmpty()) {
|
||||
throw new IllegalArgumentException("Proxy port required if proxy specified.");
|
||||
}
|
||||
okHttpClientBuilder.proxy(new Proxy(Proxy.Type.HTTP, new InetSocketAddress(proxyHost, Integer.parseInt(proxyPort))));
|
||||
String proxyUsername = properties.getProperty(PROXY_USERNAME);
|
||||
if (proxyUsername != null) {
|
||||
String proxyPassword = properties.getProperty(PROXY_PASSWORD);
|
||||
if (proxyPassword == null) {
|
||||
throw new IllegalArgumentException("Must specify proxy password with proxy username.");
|
||||
}
|
||||
okHttpClientBuilder.proxyAuthenticator((route, response) -> response.request().newBuilder().addHeader("Proxy-Authorization", Credentials.basic(proxyUsername, proxyPassword)).build());
|
||||
}
|
||||
ofNullable(properties.getProperty(PROXY_USERNAME))
|
||||
.filter(StringUtils::isNotBlank)
|
||||
.ifPresent(proxyUserName ->
|
||||
ofNullable(properties.getProperty(PROXY_PASSWORD))
|
||||
.map(proxyPassword -> Credentials.basic(proxyUserName, proxyPassword))
|
||||
.map(credentials -> (Authenticator) (route, response) -> response.request().newBuilder().addHeader(PROXY_AUTHORIZATION_HEADER, credentials).build())
|
||||
.ifPresentOrElse(
|
||||
okHttpClientBuilder::proxyAuthenticator,
|
||||
() -> {
|
||||
throw new IllegalArgumentException("Must specify proxy password with proxy username");
|
||||
}));
|
||||
}
|
||||
|
||||
// check if the ssl path is set and add the factory if so
|
||||
if (properties.containsKey(KEYSTORE_LOCATION_KEY)) {
|
||||
try {
|
||||
setSslSocketFactory(okHttpClientBuilder, properties);
|
||||
connectionScheme = "https";
|
||||
} catch (Exception e) {
|
||||
throw new IllegalStateException(e);
|
||||
}
|
||||
connectionScheme = HTTPS.toString();
|
||||
setSslSocketFactory(okHttpClientBuilder, properties);
|
||||
} else {
|
||||
connectionScheme = "http";
|
||||
connectionScheme = HTTP.toString();
|
||||
}
|
||||
|
||||
httpClientReference.set(okHttpClientBuilder.build());
|
||||
final String differentiatorName = properties.getProperty(DIFFERENTIATOR_KEY);
|
||||
|
||||
if (differentiatorName != null && !differentiatorName.isEmpty()) {
|
||||
Supplier<Differentiator<ByteBuffer>> differentiatorSupplier = DIFFERENTIATOR_CONSTRUCTOR_MAP.get(differentiatorName);
|
||||
if (differentiatorSupplier == null) {
|
||||
throw new IllegalArgumentException("Property, " + DIFFERENTIATOR_KEY + ", has value " + differentiatorName + " which does not " +
|
||||
"correspond to any in the PullHttpChangeIngestor Map:" + DIFFERENTIATOR_CONSTRUCTOR_MAP.keySet());
|
||||
}
|
||||
differentiator = differentiatorSupplier.get();
|
||||
} else {
|
||||
differentiator = WholeConfigDifferentiator.getByteBufferDifferentiator();
|
||||
}
|
||||
differentiator = ofNullable(properties.getProperty(DIFFERENTIATOR_KEY))
|
||||
.filter(not(String::isBlank))
|
||||
.map(differentiator -> ofNullable(DIFFERENTIATOR_CONSTRUCTOR_MAP.get(differentiator))
|
||||
.map(Supplier::get)
|
||||
.orElseThrow(unableToFindDifferentiatorExceptionSupplier(differentiator)))
|
||||
.orElseGet(WholeConfigDifferentiator::getByteBufferDifferentiator);
|
||||
differentiator.initialize(configurationFileHolder);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
logger.debug("Attempting to pull new config");
|
||||
HttpUrl.Builder builder = new HttpUrl.Builder()
|
||||
.host(hostReference.get())
|
||||
.port(portReference.get())
|
||||
.encodedPath(pathReference.get());
|
||||
final String query = queryReference.get();
|
||||
if (!StringUtil.isNullOrEmpty(query)) {
|
||||
builder = builder.encodedQuery(query);
|
||||
}
|
||||
final HttpUrl url = builder
|
||||
.scheme(connectionScheme)
|
||||
.build();
|
||||
|
||||
final Request.Builder requestBuilder = new Request.Builder()
|
||||
.get()
|
||||
.url(url);
|
||||
.host(hostReference.get())
|
||||
.port(portReference.get())
|
||||
.encodedPath(pathReference.get());
|
||||
ofNullable(queryReference.get())
|
||||
.filter(StringUtils::isNotBlank)
|
||||
.ifPresent(builder::encodedQuery);
|
||||
HttpUrl url = builder.scheme(connectionScheme).build();
|
||||
|
||||
Request.Builder requestBuilder = new Request.Builder().get().url(url);
|
||||
if (useEtag) {
|
||||
requestBuilder.addHeader("If-None-Match", lastEtag);
|
||||
requestBuilder.addHeader(IF_NONE_MATCH_HEADER_KEY, lastEtag);
|
||||
}
|
||||
httpHeadersReference.get().forEach(requestBuilder::addHeader);
|
||||
|
||||
final Request request = requestBuilder.build();
|
||||
Request request = requestBuilder.build();
|
||||
logger.debug("Sending request: {}", request);
|
||||
|
||||
ResponseBody body = null;
|
||||
try (Response response = httpClientReference.get().newCall(request).execute()) {
|
||||
logger.debug("Response received: {}", response.toString());
|
||||
|
||||
logger.debug("Response received: {}", response);
|
||||
int code = response.code();
|
||||
|
||||
if (code == NOT_MODIFIED_STATUS_CODE) {
|
||||
return;
|
||||
}
|
||||
|
||||
if (code >= 400) {
|
||||
if (code >= BAD_REQUEST_STATUS_CODE) {
|
||||
throw new IOException("Got response code " + code + " while trying to pull configuration: " + response.body().string());
|
||||
}
|
||||
|
||||
body = response.body();
|
||||
|
||||
ResponseBody body = response.body();
|
||||
if (body == null) {
|
||||
logger.warn("No body returned when pulling a new configuration");
|
||||
return;
|
||||
}
|
||||
|
||||
// checking if some parts of the configuration must be preserved
|
||||
ByteBuffer readOnlyNewConfig =
|
||||
ConfigTransformer.overrideNonFlowSectionsFromOriginalSchema(body.bytes(), configurationFileHolder.getConfigFileReference().get().duplicate(), properties.get());
|
||||
|
||||
if (differentiator.isNew(readOnlyNewConfig)) {
|
||||
ByteBuffer newFlowConfig = wrap(body.bytes()).duplicate();
|
||||
if (differentiator.isNew(newFlowConfig)) {
|
||||
logger.debug("New change received, notifying listener");
|
||||
configurationChangeNotifier.notifyListeners(readOnlyNewConfig);
|
||||
configurationChangeNotifier.notifyListeners(newFlowConfig);
|
||||
logger.debug("Listeners notified");
|
||||
} else {
|
||||
logger.debug("Pulled config same as currently running.");
|
||||
logger.debug("Pulled config same as currently running");
|
||||
}
|
||||
|
||||
if (useEtag) {
|
||||
lastEtag = (new StringBuilder("\""))
|
||||
.append(response.header("ETag").trim())
|
||||
.append("\"").toString();
|
||||
lastEtag = Stream.of(DOUBLE_QUOTES, response.header(ETAG_HEADER).trim(), DOUBLE_QUOTES).collect(joining());
|
||||
}
|
||||
} catch (Exception e) {
|
||||
logger.warn("Hit an exception while trying to pull", e);
|
||||
}
|
||||
}
|
||||
|
||||
private void setSslSocketFactory(OkHttpClient.Builder okHttpClientBuilder, Properties properties) throws Exception {
|
||||
final String keystoreLocation = properties.getProperty(KEYSTORE_LOCATION_KEY);
|
||||
final String keystorePass = properties.getProperty(KEYSTORE_PASSWORD_KEY);
|
||||
final String keystoreType = properties.getProperty(KEYSTORE_TYPE_KEY);
|
||||
assertKeystorePropertiesSet(keystoreLocation, keystorePass, keystoreType);
|
||||
private void setSslSocketFactory(OkHttpClient.Builder okHttpClientBuilder, Properties properties) {
|
||||
String keystorePass = properties.getProperty(KEYSTORE_PASSWORD_KEY);
|
||||
KeyStore keyStore = buildKeyStore(properties, KEYSTORE_LOCATION_KEY, KEYSTORE_PASSWORD_KEY, KEYSTORE_TYPE_KEY);
|
||||
KeyStore truststore = buildKeyStore(properties, TRUSTSTORE_LOCATION_KEY, TRUSTSTORE_PASSWORD_KEY, TRUSTSTORE_TYPE_KEY);
|
||||
|
||||
final KeyStore keyStore;
|
||||
try (final FileInputStream keyStoreStream = new FileInputStream(keystoreLocation)) {
|
||||
keyStore = new StandardKeyStoreBuilder()
|
||||
.type(keystoreType)
|
||||
.inputStream(keyStoreStream)
|
||||
.password(keystorePass.toCharArray())
|
||||
.build();
|
||||
}
|
||||
|
||||
final String truststoreLocation = properties.getProperty(TRUSTSTORE_LOCATION_KEY);
|
||||
final String truststorePass = properties.getProperty(TRUSTSTORE_PASSWORD_KEY);
|
||||
final String truststoreType = properties.getProperty(TRUSTSTORE_TYPE_KEY);
|
||||
assertTruststorePropertiesSet(truststoreLocation, truststorePass, truststoreType);
|
||||
|
||||
final KeyStore truststore;
|
||||
try (final FileInputStream trustStoreStream = new FileInputStream(truststoreLocation)) {
|
||||
truststore = new StandardKeyStoreBuilder()
|
||||
.type(truststoreType)
|
||||
.inputStream(trustStoreStream)
|
||||
.password(truststorePass.toCharArray())
|
||||
.build();
|
||||
}
|
||||
|
||||
final X509TrustManager trustManager = new StandardTrustManagerBuilder().trustStore(truststore).build();
|
||||
final SSLContext sslContext = new StandardSslContextBuilder()
|
||||
.keyStore(keyStore)
|
||||
.keyPassword(keystorePass.toCharArray())
|
||||
.trustStore(truststore)
|
||||
.build();
|
||||
final SSLSocketFactory sslSocketFactory = sslContext.getSocketFactory();
|
||||
SSLSocketFactory sslSocketFactory = new StandardSslContextBuilder()
|
||||
.keyStore(keyStore)
|
||||
.keyPassword(keystorePass.toCharArray())
|
||||
.trustStore(truststore)
|
||||
.build()
|
||||
.getSocketFactory();
|
||||
X509TrustManager trustManager = new StandardTrustManagerBuilder().trustStore(truststore).build();
|
||||
|
||||
okHttpClientBuilder.sslSocketFactory(sslSocketFactory, trustManager);
|
||||
}
|
||||
|
||||
private void assertKeystorePropertiesSet(String location, String password, String type) {
|
||||
if (location == null || location.isEmpty()) {
|
||||
throw new IllegalArgumentException(KEYSTORE_LOCATION_KEY + " is null or is empty");
|
||||
}
|
||||
private KeyStore buildKeyStore(Properties properties, String locationKey, String passKey, String typeKey) {
|
||||
String keystoreLocation = ofNullable(properties.getProperty(locationKey))
|
||||
.filter(StringUtils::isNotBlank)
|
||||
.orElseThrow(() -> new IllegalArgumentException(locationKey + " is null or empty"));
|
||||
String keystorePass = ofNullable(properties.getProperty(passKey))
|
||||
.filter(StringUtils::isNotBlank)
|
||||
.orElseThrow(() -> new IllegalArgumentException(passKey + " is null or empty"));
|
||||
String keystoreType = ofNullable(properties.getProperty(typeKey))
|
||||
.filter(StringUtils::isNotBlank)
|
||||
.orElseThrow(() -> new IllegalArgumentException(typeKey + " is null or empty"));
|
||||
|
||||
if (password == null || password.isEmpty()) {
|
||||
throw new IllegalArgumentException(KEYSTORE_LOCATION_KEY + " is set but " + KEYSTORE_PASSWORD_KEY + " is not (or is empty). If the location is set, the password must also be.");
|
||||
}
|
||||
|
||||
if (type == null || type.isEmpty()) {
|
||||
throw new IllegalArgumentException(KEYSTORE_LOCATION_KEY + " is set but " + KEYSTORE_TYPE_KEY + " is not (or is empty). If the location is set, the type must also be.");
|
||||
try (FileInputStream keyStoreStream = new FileInputStream(keystoreLocation)) {
|
||||
return new StandardKeyStoreBuilder()
|
||||
.type(keystoreType)
|
||||
.inputStream(keyStoreStream)
|
||||
.password(keystorePass.toCharArray())
|
||||
.build();
|
||||
} catch (IOException e) {
|
||||
throw new IllegalStateException("Unable to create keyStore", e);
|
||||
}
|
||||
}
|
||||
|
||||
private void assertTruststorePropertiesSet(String location, String password, String type) {
|
||||
if (location == null || location.isEmpty()) {
|
||||
throw new IllegalArgumentException(TRUSTSTORE_LOCATION_KEY + " is not set or is empty");
|
||||
}
|
||||
|
||||
if (password == null || password.isEmpty()) {
|
||||
throw new IllegalArgumentException(TRUSTSTORE_LOCATION_KEY + " is set but " + TRUSTSTORE_PASSWORD_KEY + " is not (or is empty). If the location is set, the password must also be.");
|
||||
}
|
||||
|
||||
if (type == null || type.isEmpty()) {
|
||||
throw new IllegalArgumentException(TRUSTSTORE_LOCATION_KEY + " is set but " + TRUSTSTORE_TYPE_KEY + " is not (or is empty). If the location is set, the type must also be.");
|
||||
}
|
||||
private Supplier<IllegalArgumentException> unableToFindDifferentiatorExceptionSupplier(String differentiator) {
|
||||
return () -> new IllegalArgumentException("Property, " + DIFFERENTIATOR_KEY + ", has value " + differentiator
|
||||
+ " which does not correspond to any in the FileChangeIngestor Map:" + DIFFERENTIATOR_CONSTRUCTOR_MAP.keySet());
|
||||
}
|
||||
|
||||
protected void setDifferentiator(Differentiator<ByteBuffer> differentiator) {
|
||||
// Methods exposed only for enable testing
|
||||
void setDifferentiator(Differentiator<ByteBuffer> differentiator) {
|
||||
this.differentiator = differentiator;
|
||||
}
|
||||
|
||||
public void setLastEtag(String lastEtag) {
|
||||
void setLastEtag(String lastEtag) {
|
||||
this.lastEtag = lastEtag;
|
||||
}
|
||||
|
||||
public void setUseEtag(boolean useEtag) {
|
||||
void setUseEtag(boolean useEtag) {
|
||||
this.useEtag = useEtag;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -17,6 +17,10 @@
|
|||
|
||||
package org.apache.nifi.minifi.bootstrap.configuration.ingestors;
|
||||
|
||||
import static java.nio.ByteBuffer.wrap;
|
||||
import static java.util.Optional.ofNullable;
|
||||
import static java.util.function.Predicate.not;
|
||||
import static org.apache.commons.io.IOUtils.toByteArray;
|
||||
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;
|
||||
|
||||
|
@ -29,15 +33,12 @@ import java.nio.ByteBuffer;
|
|||
import java.security.KeyStore;
|
||||
import java.security.Security;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
import java.util.function.Supplier;
|
||||
import javax.net.ssl.SSLContext;
|
||||
import javax.servlet.http.HttpServletRequest;
|
||||
import javax.servlet.http.HttpServletResponse;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.nifi.jetty.configuration.connector.StandardServerConnectorFactory;
|
||||
import org.apache.nifi.minifi.bootstrap.ConfigurationFileHolder;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeNotifier;
|
||||
|
@ -45,7 +46,6 @@ import org.apache.nifi.minifi.bootstrap.configuration.ListenerHandleResult;
|
|||
import org.apache.nifi.minifi.bootstrap.configuration.differentiators.Differentiator;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.differentiators.WholeConfigDifferentiator;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.ingestors.interfaces.ChangeIngestor;
|
||||
import org.apache.nifi.minifi.bootstrap.util.ConfigTransformer;
|
||||
import org.apache.nifi.security.ssl.StandardKeyStoreBuilder;
|
||||
import org.apache.nifi.security.ssl.StandardSslContextBuilder;
|
||||
import org.apache.nifi.security.util.TlsPlatform;
|
||||
|
@ -62,25 +62,17 @@ import org.slf4j.LoggerFactory;
|
|||
|
||||
public class RestChangeIngestor implements ChangeIngestor {
|
||||
|
||||
private static final Map<String, Supplier<Differentiator<ByteBuffer>>> DIFFERENTIATOR_CONSTRUCTOR_MAP;
|
||||
|
||||
static {
|
||||
HashMap<String, Supplier<Differentiator<ByteBuffer>>> tempMap = new HashMap<>();
|
||||
tempMap.put(WHOLE_CONFIG_KEY, WholeConfigDifferentiator::getByteBufferDifferentiator);
|
||||
|
||||
DIFFERENTIATOR_CONSTRUCTOR_MAP = Collections.unmodifiableMap(tempMap);
|
||||
Security.addProvider(new BouncyCastleProvider());
|
||||
}
|
||||
|
||||
|
||||
public static final String GET_TEXT = "This is a config change listener for an Apache NiFi - MiNiFi instance.\n" +
|
||||
"Use this rest server to upload a conf.yml to configure the MiNiFi instance.\n" +
|
||||
"Send a POST http request to '/' to upload the file.";
|
||||
public static final String OTHER_TEXT = "This is not a support HTTP operation. Please use GET to get more information or POST to upload a new config.yml file.\n";
|
||||
"Use this rest server to upload a flow.json to configure the MiNiFi instance.\n" +
|
||||
"Send a POST http request to '/' to upload the file.";
|
||||
public static final String OTHER_TEXT = "This is not a supported HTTP operation. Please use GET to get more information or POST to upload a new flow.json file.\n";
|
||||
public static final String POST = "POST";
|
||||
public static final String GET = "GET";
|
||||
private final static Logger logger = LoggerFactory.getLogger(RestChangeIngestor.class);
|
||||
private static final String RECEIVE_HTTP_BASE_KEY = NOTIFIER_INGESTORS_KEY + ".receive.http";
|
||||
public static final String RECEIVE_HTTP_BASE_KEY = NOTIFIER_INGESTORS_KEY + ".receive.http";
|
||||
public static final String PORT_KEY = RECEIVE_HTTP_BASE_KEY + ".port";
|
||||
public static final String HOST_KEY = RECEIVE_HTTP_BASE_KEY + ".host";
|
||||
public static final String TRUSTSTORE_LOCATION_KEY = RECEIVE_HTTP_BASE_KEY + ".truststore.location";
|
||||
|
@ -91,12 +83,17 @@ public class RestChangeIngestor implements ChangeIngestor {
|
|||
public static final String KEYSTORE_TYPE_KEY = RECEIVE_HTTP_BASE_KEY + ".keystore.type";
|
||||
public static final String NEED_CLIENT_AUTH_KEY = RECEIVE_HTTP_BASE_KEY + ".need.client.auth";
|
||||
public static final String DIFFERENTIATOR_KEY = RECEIVE_HTTP_BASE_KEY + ".differentiator";
|
||||
|
||||
private final static Logger logger = LoggerFactory.getLogger(RestChangeIngestor.class);
|
||||
|
||||
private static final Map<String, Supplier<Differentiator<ByteBuffer>>> DIFFERENTIATOR_CONSTRUCTOR_MAP = Map.of(
|
||||
WHOLE_CONFIG_KEY, WholeConfigDifferentiator::getByteBufferDifferentiator
|
||||
);
|
||||
|
||||
private final Server jetty;
|
||||
|
||||
private volatile Differentiator<ByteBuffer> differentiator;
|
||||
private volatile ConfigurationChangeNotifier configurationChangeNotifier;
|
||||
private volatile ConfigurationFileHolder configurationFileHolder;
|
||||
private volatile Properties properties;
|
||||
|
||||
public RestChangeIngestor() {
|
||||
QueuedThreadPool queuedThreadPool = new QueuedThreadPool();
|
||||
|
@ -106,36 +103,23 @@ public class RestChangeIngestor implements ChangeIngestor {
|
|||
|
||||
@Override
|
||||
public void initialize(Properties properties, ConfigurationFileHolder configurationFileHolder, ConfigurationChangeNotifier configurationChangeNotifier) {
|
||||
this.configurationFileHolder = configurationFileHolder;
|
||||
this.properties = properties;
|
||||
logger.info("Initializing");
|
||||
String differentiatorName = properties.getProperty(DIFFERENTIATOR_KEY);
|
||||
logger.info("Initializing RestChangeIngestor");
|
||||
this.differentiator = ofNullable(properties.getProperty(DIFFERENTIATOR_KEY))
|
||||
.filter(not(String::isBlank))
|
||||
.map(differentiator -> ofNullable(DIFFERENTIATOR_CONSTRUCTOR_MAP.get(differentiator))
|
||||
.map(Supplier::get)
|
||||
.orElseThrow(unableToFindDifferentiatorExceptionSupplier(differentiator)))
|
||||
.orElseGet(WholeConfigDifferentiator::getByteBufferDifferentiator);
|
||||
this.differentiator.initialize(configurationFileHolder);
|
||||
|
||||
if (differentiatorName != null && !differentiatorName.isEmpty()) {
|
||||
Supplier<Differentiator<ByteBuffer>> differentiatorSupplier = DIFFERENTIATOR_CONSTRUCTOR_MAP.get(differentiatorName);
|
||||
if (differentiatorSupplier == null) {
|
||||
throw new IllegalArgumentException("Property, " + DIFFERENTIATOR_KEY + ", has value " + differentiatorName + " which does not " +
|
||||
"correspond to any in the PullHttpChangeIngestor Map:" + DIFFERENTIATOR_CONSTRUCTOR_MAP.keySet());
|
||||
}
|
||||
differentiator = differentiatorSupplier.get();
|
||||
} else {
|
||||
differentiator = WholeConfigDifferentiator.getByteBufferDifferentiator();
|
||||
}
|
||||
differentiator.initialize(configurationFileHolder);
|
||||
|
||||
// create the secure connector if keystore location is specified
|
||||
if (properties.getProperty(KEYSTORE_LOCATION_KEY) != null) {
|
||||
createSecureConnector(properties);
|
||||
} else {
|
||||
// create the unsecure connector otherwise
|
||||
createConnector(properties);
|
||||
}
|
||||
ofNullable(properties.getProperty(KEYSTORE_LOCATION_KEY))
|
||||
.ifPresentOrElse(keyStoreLocation -> createSecureConnector(properties), () -> createConnector(properties));
|
||||
|
||||
this.configurationChangeNotifier = configurationChangeNotifier;
|
||||
|
||||
HandlerCollection handlerCollection = new HandlerCollection(true);
|
||||
handlerCollection.addHandler(new JettyHandler());
|
||||
jetty.setHandler(handlerCollection);
|
||||
this.jetty.setHandler(handlerCollection);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -231,7 +215,13 @@ public class RestChangeIngestor implements ChangeIngestor {
|
|||
logger.info("HTTPS Connector added for Host [{}] and Port [{}]", https.getHost(), https.getPort());
|
||||
}
|
||||
|
||||
protected void setDifferentiator(Differentiator<ByteBuffer> differentiator) {
|
||||
private Supplier<IllegalArgumentException> unableToFindDifferentiatorExceptionSupplier(String differentiator) {
|
||||
return () -> new IllegalArgumentException("Property, " + DIFFERENTIATOR_KEY + ", has value " + differentiator
|
||||
+ " which does not correspond to any in the FileChangeIngestor Map:" + DIFFERENTIATOR_CONSTRUCTOR_MAP.keySet());
|
||||
}
|
||||
|
||||
// Method exposed only for enable testing
|
||||
void setDifferentiator(Differentiator<ByteBuffer> differentiator) {
|
||||
this.differentiator = differentiator;
|
||||
}
|
||||
|
||||
|
@ -239,7 +229,7 @@ public class RestChangeIngestor implements ChangeIngestor {
|
|||
|
||||
@Override
|
||||
public void handle(String target, Request baseRequest, HttpServletRequest request, HttpServletResponse response)
|
||||
throws IOException {
|
||||
throws IOException {
|
||||
|
||||
logRequest(request);
|
||||
|
||||
|
@ -249,13 +239,9 @@ public class RestChangeIngestor implements ChangeIngestor {
|
|||
int statusCode;
|
||||
String responseText;
|
||||
try {
|
||||
ByteBuffer readOnlyNewConfig =
|
||||
ConfigTransformer.overrideNonFlowSectionsFromOriginalSchema(
|
||||
IOUtils.toByteArray(request.getInputStream()), configurationFileHolder.getConfigFileReference().get().duplicate(), properties);
|
||||
|
||||
if (differentiator.isNew(readOnlyNewConfig)) {
|
||||
|
||||
Collection<ListenerHandleResult> listenerHandleResults = configurationChangeNotifier.notifyListeners(readOnlyNewConfig);
|
||||
ByteBuffer newFlowConfig = wrap(toByteArray(request.getInputStream())).duplicate();
|
||||
if (differentiator.isNew(newFlowConfig)) {
|
||||
Collection<ListenerHandleResult> listenerHandleResults = configurationChangeNotifier.notifyListeners(newFlowConfig);
|
||||
|
||||
statusCode = 200;
|
||||
for (ListenerHandleResult result : listenerHandleResults) {
|
||||
|
@ -312,6 +298,5 @@ public class RestChangeIngestor implements ChangeIngestor {
|
|||
logger.info("request content type = " + request.getContentType());
|
||||
logger.info("<<<<<<<<<<<<<<<<<<<<<<<<<<<<<<<<<<<<<<<<<<<<");
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
|
@ -17,6 +17,8 @@
|
|||
|
||||
package org.apache.nifi.minifi.bootstrap.service;
|
||||
|
||||
import static org.apache.commons.lang3.StringUtils.SPACE;
|
||||
|
||||
import java.io.BufferedReader;
|
||||
import java.io.BufferedWriter;
|
||||
import java.io.IOException;
|
||||
|
@ -40,12 +42,10 @@ public class BootstrapCodec {
|
|||
private final RunMiNiFi runner;
|
||||
private final Logger logger = LoggerFactory.getLogger(BootstrapCodec.class);
|
||||
private final UpdatePropertiesService updatePropertiesService;
|
||||
private final UpdateConfigurationService updateConfigurationService;
|
||||
|
||||
public BootstrapCodec(RunMiNiFi runner, BootstrapFileProvider bootstrapFileProvider, ConfigurationChangeListener configurationChangeListener) {
|
||||
this.runner = runner;
|
||||
this.updatePropertiesService = new UpdatePropertiesService(runner, logger, bootstrapFileProvider);
|
||||
this.updateConfigurationService = new UpdateConfigurationService(runner, configurationChangeListener, bootstrapFileProvider);
|
||||
}
|
||||
|
||||
public void communicate(InputStream in, OutputStream out) throws IOException {
|
||||
|
@ -53,18 +53,13 @@ public class BootstrapCodec {
|
|||
BufferedWriter writer = new BufferedWriter(new OutputStreamWriter(out));
|
||||
|
||||
String line = reader.readLine();
|
||||
String[] splits = Optional.ofNullable(line).map(l -> l.split(" ")).orElse(new String[0]);
|
||||
if (splits.length == 0) {
|
||||
throw new IOException("Received invalid command from MiNiFi: " + line);
|
||||
}
|
||||
String[] splits = Optional.ofNullable(line)
|
||||
.map(l -> l.split(SPACE))
|
||||
.filter(items -> items.length > 0)
|
||||
.orElseThrow(() -> new IOException("Received invalid command from MiNiFi: " + line));
|
||||
|
||||
BootstrapCommand cmd = BootstrapCommand.safeValueOf(splits[0]);
|
||||
String[] args;
|
||||
if (splits.length == 1) {
|
||||
args = new String[0];
|
||||
} else {
|
||||
args = Arrays.copyOfRange(splits, 1, splits.length);
|
||||
}
|
||||
String[] args = splits.length == 1 ? new String[0] : Arrays.copyOfRange(splits, 1, splits.length);
|
||||
|
||||
try {
|
||||
processRequest(cmd, args, writer);
|
||||
|
@ -90,21 +85,11 @@ public class BootstrapCodec {
|
|||
case UPDATE_PROPERTIES:
|
||||
handlePropertiesUpdateCommand(writer);
|
||||
break;
|
||||
case UPDATE_CONFIGURATION:
|
||||
handleUpdateConfigurationCommand(writer);
|
||||
break;
|
||||
default:
|
||||
throw new InvalidCommandException("Unknown command: " + cmd);
|
||||
}
|
||||
}
|
||||
|
||||
private void handleUpdateConfigurationCommand(BufferedWriter writer) throws IOException {
|
||||
logger.debug("Received 'UPDATE_CONFIGURATION' command from MINIFI");
|
||||
writeOk(writer);
|
||||
runner.setCommandInProgress(true);
|
||||
updateConfigurationService.handleUpdate().ifPresent(runner::sendAcknowledgeToMiNiFi);
|
||||
}
|
||||
|
||||
private void handlePropertiesUpdateCommand(BufferedWriter writer) throws IOException {
|
||||
logger.debug("Received 'UPDATE_PROPERTIES' command from MINIFI");
|
||||
writeOk(writer);
|
||||
|
@ -170,7 +155,7 @@ public class BootstrapCodec {
|
|||
}
|
||||
|
||||
private enum BootstrapCommand {
|
||||
PORT, STARTED, SHUTDOWN, RELOAD, UPDATE_PROPERTIES, UPDATE_CONFIGURATION, UNKNOWN;
|
||||
PORT, STARTED, SHUTDOWN, RELOAD, UPDATE_PROPERTIES, UNKNOWN;
|
||||
|
||||
public static BootstrapCommand safeValueOf(String value) {
|
||||
try {
|
||||
|
|
|
@ -94,14 +94,6 @@ public class BootstrapFileProvider {
|
|||
return reloadFile;
|
||||
}
|
||||
|
||||
public File getConfigYmlSwapFile() {
|
||||
File confDir = bootstrapConfigFile.getParentFile();
|
||||
File swapFile = new File(confDir, "swap.yml");
|
||||
|
||||
LOGGER.debug("Swap File: {}", swapFile);
|
||||
return swapFile;
|
||||
}
|
||||
|
||||
public File getBootstrapConfSwapFile() {
|
||||
File confDir = bootstrapConfigFile.getParentFile();
|
||||
File swapFile = new File(confDir, "bootstrap-swap.conf");
|
||||
|
|
|
@ -14,82 +14,48 @@
|
|||
* 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.util.ConfigTransformer.generateConfigFiles;
|
||||
import static java.nio.ByteBuffer.wrap;
|
||||
import static java.nio.charset.StandardCharsets.UTF_8;
|
||||
import static org.apache.commons.io.IOUtils.closeQuietly;
|
||||
import static org.apache.commons.io.IOUtils.toByteArray;
|
||||
import static org.apache.nifi.minifi.commons.api.MiNiFiConstants.BACKUP_EXTENSION;
|
||||
import static org.apache.nifi.minifi.commons.api.MiNiFiConstants.RAW_EXTENSION;
|
||||
import static org.apache.nifi.minifi.commons.util.FlowUpdateUtils.backup;
|
||||
import static org.apache.nifi.minifi.commons.util.FlowUpdateUtils.persist;
|
||||
import static org.apache.nifi.minifi.commons.util.FlowUpdateUtils.removeIfExists;
|
||||
import static org.apache.nifi.minifi.commons.util.FlowUpdateUtils.revert;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileInputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
import java.util.Properties;
|
||||
import java.util.concurrent.locks.ReentrantLock;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.commons.io.FilenameUtils;
|
||||
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.commons.api.MiNiFiProperties;
|
||||
import org.apache.nifi.minifi.commons.service.FlowEnrichService;
|
||||
import org.slf4j.Logger;
|
||||
|
||||
public class MiNiFiConfigurationChangeListener implements ConfigurationChangeListener {
|
||||
|
||||
private static final ReentrantLock handlingLock = new ReentrantLock();
|
||||
|
||||
private final RunMiNiFi runner;
|
||||
private final Logger logger;
|
||||
private final BootstrapFileProvider bootstrapFileProvider;
|
||||
private final FlowEnrichService flowEnrichService;
|
||||
|
||||
private static final ReentrantLock handlingLock = new ReentrantLock();
|
||||
|
||||
public MiNiFiConfigurationChangeListener(RunMiNiFi runner, Logger logger, BootstrapFileProvider bootstrapFileProvider) {
|
||||
public MiNiFiConfigurationChangeListener(RunMiNiFi runner, Logger logger, BootstrapFileProvider bootstrapFileProvider, FlowEnrichService flowEnrichService) {
|
||||
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 {
|
||||
Properties bootstrapProperties = bootstrapFileProvider.getBootstrapProperties();
|
||||
File configFile = new File(bootstrapProperties.getProperty(MINIFI_CONFIG_FILE_KEY));
|
||||
|
||||
File swapConfigFile = bootstrapFileProvider.getConfigYmlSwapFile();
|
||||
logger.info("Persisting old configuration to {}", swapConfigFile.getAbsolutePath());
|
||||
|
||||
try (FileInputStream configFileInputStream = new FileInputStream(configFile)) {
|
||||
Files.copy(configFileInputStream, swapConfigFile.toPath(), REPLACE_EXISTING);
|
||||
}
|
||||
|
||||
// write out new config to file
|
||||
Files.copy(configInputStream, configFile.toPath(), REPLACE_EXISTING);
|
||||
|
||||
// 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();
|
||||
}
|
||||
this.flowEnrichService = flowEnrichService;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -97,33 +63,53 @@ public class MiNiFiConfigurationChangeListener implements ConfigurationChangeLis
|
|||
return "MiNiFiConfigurationChangeListener";
|
||||
}
|
||||
|
||||
private void transformConfigurationFiles(String confDir, FileInputStream newConfigIs, File configFile, File swapConfigFile) throws Exception {
|
||||
@Override
|
||||
public void handleChange(InputStream flowConfigInputStream) throws ConfigurationChangeException {
|
||||
logger.info("Received notification of a change");
|
||||
|
||||
if (!handlingLock.tryLock()) {
|
||||
throw new ConfigurationChangeException("Instance is already handling another change");
|
||||
}
|
||||
|
||||
Path currentFlowConfigFile = null;
|
||||
Path backupFlowConfigFile = null;
|
||||
Path currentRawFlowConfigFile = null;
|
||||
Path backupRawFlowConfigFile = null;
|
||||
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);
|
||||
Properties bootstrapProperties = bootstrapFileProvider.getBootstrapProperties();
|
||||
|
||||
currentFlowConfigFile = Path.of(bootstrapProperties.getProperty(MiNiFiProperties.NIFI_MINIFI_FLOW_CONFIG.getKey())).toAbsolutePath();
|
||||
backupFlowConfigFile = Path.of(currentFlowConfigFile + BACKUP_EXTENSION);
|
||||
String currentFlowConfigFileBaseName = FilenameUtils.getBaseName(currentFlowConfigFile.toString());
|
||||
currentRawFlowConfigFile = currentFlowConfigFile.getParent().resolve(currentFlowConfigFileBaseName + RAW_EXTENSION);
|
||||
backupRawFlowConfigFile = currentFlowConfigFile.getParent().resolve(currentFlowConfigFileBaseName + RAW_EXTENSION + BACKUP_EXTENSION);
|
||||
|
||||
backup(currentFlowConfigFile, backupFlowConfigFile);
|
||||
backup(currentRawFlowConfigFile, backupRawFlowConfigFile);
|
||||
|
||||
byte[] rawFlow = toByteArray(flowConfigInputStream);
|
||||
byte[] enrichedFlow = flowEnrichService.enrichFlow(rawFlow);
|
||||
persist(enrichedFlow, currentFlowConfigFile, true);
|
||||
restartInstance();
|
||||
persist(rawFlow, currentRawFlowConfigFile, false);
|
||||
setActiveFlowReference(wrap(rawFlow));
|
||||
logger.info("MiNiFi has finished reloading successfully and applied the new flow configuration");
|
||||
} 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;
|
||||
logger.error("Configuration update failed. Reverting to previous flow", e);
|
||||
revert(backupFlowConfigFile, currentFlowConfigFile);
|
||||
revert(backupRawFlowConfigFile, currentRawFlowConfigFile);
|
||||
throw new ConfigurationChangeException("Unable to perform reload of received configuration change", e);
|
||||
} finally {
|
||||
removeIfExists(backupFlowConfigFile);
|
||||
removeIfExists(backupRawFlowConfigFile);
|
||||
closeQuietly(flowConfigInputStream);
|
||||
handlingLock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
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 setActiveFlowReference(ByteBuffer flowConfig) {
|
||||
logger.debug("Setting active flow reference {} with content:\n{}", flowConfig, new String(flowConfig.array(), UTF_8));
|
||||
runner.getConfigFileReference().set(flowConfig);
|
||||
}
|
||||
|
||||
private void restartInstance() throws IOException {
|
||||
|
|
|
@ -17,15 +17,19 @@
|
|||
|
||||
package org.apache.nifi.minifi.bootstrap.service;
|
||||
|
||||
import static java.lang.String.join;
|
||||
import static java.util.Optional.ofNullable;
|
||||
import static java.util.stream.Collectors.joining;
|
||||
import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.CONF_DIR_KEY;
|
||||
import static org.apache.nifi.util.NiFiProperties.PROPERTIES_FILE_PATH;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.nio.file.Path;
|
||||
import java.util.List;
|
||||
import java.util.Map.Entry;
|
||||
import java.util.Optional;
|
||||
import java.util.Properties;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
public class MiNiFiExecCommandProvider {
|
||||
|
||||
|
@ -40,12 +44,28 @@ public class MiNiFiExecCommandProvider {
|
|||
public static final String DEFAULT_BOOTSTRAP_LOG_FILE_NAME = "minifi-bootstrap";
|
||||
public static final String DEFAULT_LOG_FILE_EXTENSION = "log";
|
||||
|
||||
private static final String PROPERTIES_FILE_KEY = "props.file";
|
||||
private static final String LIB_DIR_KEY = "lib.dir";
|
||||
private static final String JAVA_COMMAND_KEY = "java";
|
||||
private static final String JAVA_ARG_KEY_PREFIX = "java.arg";
|
||||
|
||||
private static final String DEFAULT_JAVA_CMD = "java";
|
||||
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 DEFAULT_MINIFI_PROPERTIES_FILE = "minifi.properties";
|
||||
|
||||
private static final String WINDOWS_FILE_EXTENSION = ".exe";
|
||||
private static final String LINUX_FILE_EXTENSION = "";
|
||||
private static final String JAR_FILE_EXTENSION = ".jar";
|
||||
|
||||
private static final String JAVA_HOME_ENVIRONMENT_VARIABLE = "JAVA_HOME";
|
||||
private static final String MINIFI_CLASS_NAME = "MiNiFi";
|
||||
private static final String MINIFI_FULLY_QUALIFIED_CLASS_NAME = "org.apache.nifi.minifi." + MINIFI_CLASS_NAME;
|
||||
private static final String SYSTEM_PROPERTY_TEMPLATE = "-D%s=%s";
|
||||
private static final String NIFI_BOOTSTRAP_LISTEN_PORT = "nifi.bootstrap.listen.port";
|
||||
private static final String APP = "app";
|
||||
private static final String CLASSPATH = "-classpath";
|
||||
private static final String BIN_DIRECTORY = "bin";
|
||||
|
||||
private final BootstrapFileProvider bootstrapFileProvider;
|
||||
|
||||
|
@ -53,6 +73,16 @@ public class MiNiFiExecCommandProvider {
|
|||
this.bootstrapFileProvider = bootstrapFileProvider;
|
||||
}
|
||||
|
||||
public static String getMiNiFiPropertiesPath(Properties props, File confDir) {
|
||||
return ofNullable(props.getProperty(PROPERTIES_FILE_KEY))
|
||||
.orElseGet(() -> ofNullable(confDir)
|
||||
.filter(File::exists)
|
||||
.map(File::getAbsolutePath)
|
||||
.map(parent -> Path.of(parent, DEFAULT_MINIFI_PROPERTIES_FILE).toAbsolutePath().toString())
|
||||
.orElseGet(() -> Path.of(DEFAULT_CONF_DIR, DEFAULT_MINIFI_PROPERTIES_FILE).toAbsolutePath().toString()))
|
||||
.trim();
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the process arguments required for the bootstrap to start the MiNiFi process.
|
||||
*
|
||||
|
@ -62,116 +92,82 @@ public class MiNiFiExecCommandProvider {
|
|||
* @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);
|
||||
Properties bootstrapProperties = bootstrapFileProvider.getBootstrapProperties();
|
||||
|
||||
File confDir = getFile(bootstrapProperties.getProperty(CONF_DIR_KEY, DEFAULT_CONF_DIR).trim(), workingDir);
|
||||
File libDir = getFile(bootstrapProperties.getProperty(LIB_DIR_KEY, DEFAULT_LIB_DIR).trim(), workingDir);
|
||||
|
||||
String minifiLogDir = System.getProperty(LOG_DIR, DEFAULT_LOG_DIR).trim();
|
||||
String minifiAppLogFileName = System.getProperty(APP_LOG_FILE_NAME, DEFAULT_APP_LOG_FILE_NAME).trim();
|
||||
String minifiAppLogFileExtension = System.getProperty(APP_LOG_FILE_EXTENSION, DEFAULT_LOG_FILE_EXTENSION).trim();
|
||||
String minifiBootstrapLogFileName = System.getProperty(BOOTSTRAP_LOG_FILE_NAME, DEFAULT_BOOTSTRAP_LOG_FILE_NAME).trim();
|
||||
String minifiBootstrapLogFileExtension = System.getProperty(BOOTSTRAP_LOG_FILE_EXTENSION, DEFAULT_LOG_FILE_EXTENSION).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("-D" + LOG_DIR + "=" + minifiLogDir);
|
||||
cmd.add("-D" + APP_LOG_FILE_NAME + "=" + minifiAppLogFileName);
|
||||
cmd.add("-D" + APP_LOG_FILE_EXTENSION + "=" + minifiAppLogFileExtension);
|
||||
cmd.add("-D" + BOOTSTRAP_LOG_FILE_NAME + "=" + minifiBootstrapLogFileName);
|
||||
cmd.add("-D" + BOOTSTRAP_LOG_FILE_EXTENSION + "=" + minifiBootstrapLogFileExtension);
|
||||
cmd.add("org.apache.nifi.minifi.MiNiFi");
|
||||
List<String> javaCommand = List.of(getJavaCommand(bootstrapProperties), CLASSPATH, buildClassPath(confDir, libDir));
|
||||
List<String> javaAdditionalArgs = getJavaAdditionalArgs(bootstrapProperties);
|
||||
List<String> systemProperties = List.of(
|
||||
systemProperty(PROPERTIES_FILE_PATH, getMiNiFiPropertiesPath(bootstrapProperties, confDir)),
|
||||
systemProperty(NIFI_BOOTSTRAP_LISTEN_PORT, listenPort),
|
||||
systemProperty(APP, MINIFI_CLASS_NAME),
|
||||
systemProperty(LOG_DIR, minifiLogDir),
|
||||
systemProperty(APP_LOG_FILE_NAME, minifiAppLogFileName),
|
||||
systemProperty(APP_LOG_FILE_EXTENSION, minifiAppLogFileExtension),
|
||||
systemProperty(BOOTSTRAP_LOG_FILE_NAME, minifiBootstrapLogFileName),
|
||||
systemProperty(BOOTSTRAP_LOG_FILE_EXTENSION, minifiBootstrapLogFileExtension)
|
||||
);
|
||||
|
||||
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();
|
||||
return List.of(javaCommand, javaAdditionalArgs, systemProperties, List.of(MINIFI_FULLY_QUALIFIED_CLASS_NAME))
|
||||
.stream()
|
||||
.flatMap(List::stream)
|
||||
.toList();
|
||||
}
|
||||
|
||||
private File getFile(String filename, File workingDir) {
|
||||
File file = new File(filename);
|
||||
if (!file.isAbsolute()) {
|
||||
file = new File(workingDir, filename);
|
||||
}
|
||||
return file;
|
||||
return file.isAbsolute() ? file : new File(workingDir, filename).getAbsoluteFile();
|
||||
}
|
||||
}
|
||||
|
||||
private String getJavaCommand(Properties bootstrapProperties) {
|
||||
String javaCommand = bootstrapProperties.getProperty(JAVA_COMMAND_KEY, DEFAULT_JAVA_CMD);
|
||||
return javaCommand.equals(DEFAULT_JAVA_CMD)
|
||||
? ofNullable(System.getenv(JAVA_HOME_ENVIRONMENT_VARIABLE))
|
||||
.flatMap(javaHome ->
|
||||
getJavaCommandBasedOnExtension(javaHome, javaCommand, WINDOWS_FILE_EXTENSION)
|
||||
.or(() -> getJavaCommandBasedOnExtension(javaHome, javaCommand, LINUX_FILE_EXTENSION)))
|
||||
.orElse(DEFAULT_JAVA_CMD)
|
||||
: javaCommand;
|
||||
}
|
||||
|
||||
private Optional<String> getJavaCommandBasedOnExtension(String javaHome, String javaCommand, String extension) {
|
||||
return Optional.of(new File(join(File.separator, javaHome, BIN_DIRECTORY, javaCommand + extension)))
|
||||
.filter(File::exists)
|
||||
.filter(File::canExecute)
|
||||
.map(File::getAbsolutePath);
|
||||
}
|
||||
|
||||
private String buildClassPath(File confDir, File libDir) {
|
||||
File[] libFiles = ofNullable(libDir.listFiles((dir, filename) -> filename.toLowerCase().endsWith(JAR_FILE_EXTENSION)))
|
||||
.filter(files -> files.length > 0)
|
||||
.orElseThrow(() -> new RuntimeException("Could not find lib directory at " + libDir.getAbsolutePath()));
|
||||
|
||||
ofNullable(confDir.listFiles())
|
||||
.filter(files -> files.length > 0)
|
||||
.orElseThrow(() -> new RuntimeException("Could not find conf directory at " + confDir.getAbsolutePath()));
|
||||
|
||||
return Stream.concat(Stream.of(confDir), Stream.of(libFiles))
|
||||
.map(File::getAbsolutePath)
|
||||
.collect(joining(File.pathSeparator));
|
||||
}
|
||||
|
||||
private List<String> getJavaAdditionalArgs(Properties props) {
|
||||
return props.entrySet()
|
||||
.stream()
|
||||
.filter(entry -> ((String) entry.getKey()).startsWith(JAVA_ARG_KEY_PREFIX))
|
||||
.map(entry -> (String) entry.getValue())
|
||||
.toList();
|
||||
}
|
||||
|
||||
private String systemProperty(String key, Object value) {
|
||||
return String.format(SYSTEM_PROPERTY_TEMPLATE, key, value);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,276 @@
|
|||
/*
|
||||
* Licensed to the Apache 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.lang.String.join;
|
||||
import static java.lang.System.getProperty;
|
||||
import static java.util.Optional.ofNullable;
|
||||
import static org.apache.commons.lang3.StringUtils.EMPTY;
|
||||
import static org.apache.commons.lang3.StringUtils.isNotBlank;
|
||||
import static org.apache.nifi.minifi.bootstrap.service.BootstrapFileProvider.getBootstrapConfFile;
|
||||
import static org.apache.nifi.minifi.bootstrap.service.MiNiFiExecCommandProvider.APP_LOG_FILE_EXTENSION;
|
||||
import static org.apache.nifi.minifi.bootstrap.service.MiNiFiExecCommandProvider.APP_LOG_FILE_NAME;
|
||||
import static org.apache.nifi.minifi.bootstrap.service.MiNiFiExecCommandProvider.BOOTSTRAP_LOG_FILE_EXTENSION;
|
||||
import static org.apache.nifi.minifi.bootstrap.service.MiNiFiExecCommandProvider.BOOTSTRAP_LOG_FILE_NAME;
|
||||
import static org.apache.nifi.minifi.bootstrap.service.MiNiFiExecCommandProvider.DEFAULT_APP_LOG_FILE_NAME;
|
||||
import static org.apache.nifi.minifi.bootstrap.service.MiNiFiExecCommandProvider.DEFAULT_BOOTSTRAP_LOG_FILE_NAME;
|
||||
import static org.apache.nifi.minifi.bootstrap.service.MiNiFiExecCommandProvider.DEFAULT_LOG_DIR;
|
||||
import static org.apache.nifi.minifi.bootstrap.service.MiNiFiExecCommandProvider.DEFAULT_LOG_FILE_EXTENSION;
|
||||
import static org.apache.nifi.minifi.bootstrap.service.MiNiFiExecCommandProvider.LOG_DIR;
|
||||
import static org.apache.nifi.minifi.bootstrap.service.MiNiFiExecCommandProvider.getMiNiFiPropertiesPath;
|
||||
import static org.apache.nifi.minifi.commons.api.MiNiFiProperties.MINIFI_APP_LOG_FILE;
|
||||
import static org.apache.nifi.minifi.commons.api.MiNiFiProperties.MINIFI_BOOTSTRAP_FILE_PATH;
|
||||
import static org.apache.nifi.minifi.commons.api.MiNiFiProperties.MINIFI_BOOTSTRAP_LOG_FILE;
|
||||
import static org.apache.nifi.minifi.commons.api.MiNiFiProperties.MINIFI_LOG_DIRECTORY;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
import java.security.SecureRandom;
|
||||
import java.util.Base64;
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
import java.util.Set;
|
||||
import java.util.function.Predicate;
|
||||
import org.apache.commons.io.output.ByteArrayOutputStream;
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.commons.lang3.tuple.Pair;
|
||||
import org.apache.commons.lang3.tuple.Triple;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeException;
|
||||
import org.apache.nifi.minifi.bootstrap.util.OrderedProperties;
|
||||
import org.apache.nifi.minifi.commons.api.MiNiFiProperties;
|
||||
import org.apache.nifi.util.NiFiProperties;
|
||||
|
||||
public class MiNiFiPropertiesGenerator {
|
||||
|
||||
public static final String PROPERTIES_FILE_APACHE_2_0_LICENSE =
|
||||
" Licensed to the Apache Software Foundation (ASF) under one or more\n" +
|
||||
"# contributor license agreements. See the NOTICE file distributed with\n" +
|
||||
"# this work for additional information regarding copyright ownership.\n" +
|
||||
"# The ASF licenses this file to You under the Apache License, Version 2.0\n" +
|
||||
"# (the \"License\"); you may not use this file except in compliance with\n" +
|
||||
"# the License. You may obtain a copy of the License at\n" +
|
||||
"#\n" +
|
||||
"# http://www.apache.org/licenses/LICENSE-2.0\n" +
|
||||
"#\n" +
|
||||
"# Unless required by applicable law or agreed to in writing, software\n" +
|
||||
"# distributed under the License is distributed on an \"AS IS\" BASIS,\n" +
|
||||
"# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n" +
|
||||
"# See the License for the specific language governing permissions and\n" +
|
||||
"# limitations under the License.\n" +
|
||||
"\n";
|
||||
|
||||
static final List<Triple<String, String, String>> NIFI_PROPERTIES_WITH_DEFAULT_VALUES_AND_COMMENTS = List.of(
|
||||
Triple.of(NiFiProperties.FLOW_CONFIGURATION_ARCHIVE_ENABLED, "false", EMPTY),
|
||||
Triple.of(NiFiProperties.FLOW_CONFIGURATION_ARCHIVE_DIR, "./conf/archive/", EMPTY),
|
||||
Triple.of(NiFiProperties.AUTO_RESUME_STATE, "true", EMPTY),
|
||||
Triple.of(NiFiProperties.FLOW_CONTROLLER_GRACEFUL_SHUTDOWN_PERIOD, "10 sec", EMPTY),
|
||||
Triple.of(NiFiProperties.WRITE_DELAY_INTERVAL, "500 ms", EMPTY),
|
||||
Triple.of(NiFiProperties.ADMINISTRATIVE_YIELD_DURATION, "30 sec", EMPTY),
|
||||
Triple.of(NiFiProperties.VARIABLE_REGISTRY_PROPERTIES, EMPTY, EMPTY),
|
||||
Triple.of(NiFiProperties.BORED_YIELD_DURATION, "10 millis",
|
||||
"# If a component has no work to do (is \"bored\"), how long should we wait before checking again for work"),
|
||||
Triple.of(NiFiProperties.LOGIN_IDENTITY_PROVIDER_CONFIGURATION_FILE, "./conf/login-identity-providers.xml", EMPTY),
|
||||
Triple.of(NiFiProperties.TEMPLATE_DIRECTORY, "./conf/templates", EMPTY),
|
||||
Triple.of(NiFiProperties.UI_BANNER_TEXT, EMPTY, EMPTY),
|
||||
Triple.of(NiFiProperties.UI_AUTO_REFRESH_INTERVAL, "30 sec", EMPTY),
|
||||
Triple.of(NiFiProperties.NAR_LIBRARY_DIRECTORY, "./lib", EMPTY),
|
||||
Triple.of(NiFiProperties.NAR_WORKING_DIRECTORY, "./work/nar/", EMPTY),
|
||||
Triple.of(NiFiProperties.NAR_LIBRARY_AUTOLOAD_DIRECTORY, "./extensions", EMPTY),
|
||||
Triple.of(NiFiProperties.COMPONENT_DOCS_DIRECTORY, "./work/docs/components", EMPTY),
|
||||
Triple.of(NiFiProperties.STATE_MANAGEMENT_CONFIG_FILE, "./conf/state-management.xml", "# State Management"),
|
||||
Triple.of(NiFiProperties.STATE_MANAGEMENT_LOCAL_PROVIDER_ID, "local-provider", "# The ID of the local state provider"),
|
||||
Triple.of(NiFiProperties.REPOSITORY_DATABASE_DIRECTORY, "./database_repository", "# H2 Settings"),
|
||||
Triple.of(NiFiProperties.H2_URL_APPEND, ";LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE", EMPTY),
|
||||
Triple.of(NiFiProperties.FLOWFILE_REPOSITORY_IMPLEMENTATION, "org.apache.nifi.controller.repository.WriteAheadFlowFileRepository", "# FlowFile Repository"),
|
||||
Triple.of(NiFiProperties.FLOWFILE_REPOSITORY_DIRECTORY, "./flowfile_repository", EMPTY),
|
||||
Triple.of(NiFiProperties.FLOWFILE_REPOSITORY_CHECKPOINT_INTERVAL, "20 secs", EMPTY),
|
||||
Triple.of(NiFiProperties.FLOWFILE_REPOSITORY_ALWAYS_SYNC, "false", EMPTY),
|
||||
Triple.of(NiFiProperties.FLOWFILE_SWAP_MANAGER_IMPLEMENTATION, "org.apache.nifi.controller.FileSystemSwapManager", EMPTY),
|
||||
Triple.of(NiFiProperties.QUEUE_SWAP_THRESHOLD, "20000", EMPTY),
|
||||
Triple.of(NiFiProperties.CONTENT_REPOSITORY_IMPLEMENTATION, "org.apache.nifi.controller.repository.FileSystemRepository", "# Content Repository"),
|
||||
Triple.of(NiFiProperties.MAX_APPENDABLE_CLAIM_SIZE, "50 KB", EMPTY),
|
||||
Triple.of(NiFiProperties.CONTENT_ARCHIVE_MAX_RETENTION_PERIOD, "7 days", EMPTY),
|
||||
Triple.of(NiFiProperties.CONTENT_ARCHIVE_MAX_USAGE_PERCENTAGE, "50%", EMPTY),
|
||||
Triple.of(NiFiProperties.CONTENT_ARCHIVE_ENABLED, "false", EMPTY),
|
||||
Triple.of(NiFiProperties.REPOSITORY_CONTENT_PREFIX + ".default", "./content_repository", EMPTY),
|
||||
Triple.of(NiFiProperties.PROVENANCE_REPO_IMPLEMENTATION_CLASS, "org.apache.nifi.provenance.NoOpProvenanceRepository",
|
||||
"# Provenance Repository Properties"),
|
||||
Triple.of(NiFiProperties.PROVENANCE_ROLLOVER_TIME, EMPTY, EMPTY),
|
||||
Triple.of(NiFiProperties.PROVENANCE_INDEX_SHARD_SIZE, EMPTY, EMPTY),
|
||||
Triple.of(NiFiProperties.PROVENANCE_MAX_STORAGE_SIZE, EMPTY, EMPTY),
|
||||
Triple.of(NiFiProperties.PROVENANCE_MAX_STORAGE_TIME, EMPTY, EMPTY),
|
||||
Triple.of(NiFiProperties.COMPONENT_STATUS_REPOSITORY_IMPLEMENTATION, "org.apache.nifi.controller.status.history.VolatileComponentStatusRepository",
|
||||
"# Component Status Repository"),
|
||||
Triple.of(NiFiProperties.COMPONENT_STATUS_SNAPSHOT_FREQUENCY, "1 min", EMPTY),
|
||||
Triple.of(NiFiProperties.WEB_HTTP_HOST, EMPTY, EMPTY),
|
||||
Triple.of(NiFiProperties.WEB_HTTP_PORT, EMPTY, EMPTY),
|
||||
Triple.of(NiFiProperties.WEB_HTTPS_HOST, EMPTY, EMPTY),
|
||||
Triple.of(NiFiProperties.WEB_HTTPS_PORT, EMPTY, EMPTY),
|
||||
Triple.of(NiFiProperties.WEB_WORKING_DIR, "./work/jetty", EMPTY),
|
||||
Triple.of(NiFiProperties.WEB_THREADS, "200", EMPTY),
|
||||
Triple.of(NiFiProperties.SECURITY_KEYSTORE, EMPTY, EMPTY),
|
||||
Triple.of(NiFiProperties.SECURITY_KEYSTORE_TYPE, EMPTY, EMPTY),
|
||||
Triple.of(NiFiProperties.SECURITY_KEYSTORE_PASSWD, EMPTY, EMPTY),
|
||||
Triple.of(NiFiProperties.SECURITY_KEY_PASSWD, EMPTY, EMPTY),
|
||||
Triple.of(NiFiProperties.SECURITY_TRUSTSTORE, EMPTY, EMPTY),
|
||||
Triple.of(NiFiProperties.SECURITY_TRUSTSTORE_TYPE, EMPTY, EMPTY),
|
||||
Triple.of(NiFiProperties.SECURITY_TRUSTSTORE_PASSWD, EMPTY, EMPTY),
|
||||
Triple.of(NiFiProperties.SECURITY_OCSP_RESPONDER_URL, EMPTY, EMPTY),
|
||||
Triple.of(NiFiProperties.SECURITY_OCSP_RESPONDER_CERTIFICATE, EMPTY, EMPTY),
|
||||
Triple.of(NiFiProperties.CLUSTER_IS_NODE, "false", EMPTY),
|
||||
Triple.of(NiFiProperties.FLOW_CONFIGURATION_FILE, "./conf/flow.xml.gz", EMPTY)
|
||||
);
|
||||
|
||||
static final Map<String, String> MINIFI_TO_NIFI_PROPERTY_MAPPING = Map.of(
|
||||
MiNiFiProperties.NIFI_MINIFI_FLOW_CONFIG.getKey(), NiFiProperties.FLOW_CONFIGURATION_JSON_FILE,
|
||||
MiNiFiProperties.NIFI_MINIFI_SECURITY_KEYSTORE.getKey(), NiFiProperties.SECURITY_KEYSTORE,
|
||||
MiNiFiProperties.NIFI_MINIFI_SECURITY_KEYSTORE_TYPE.getKey(), NiFiProperties.SECURITY_KEYSTORE_TYPE,
|
||||
MiNiFiProperties.NIFI_MINIFI_SECURITY_KEYSTORE_PASSWD.getKey(), NiFiProperties.SECURITY_KEYSTORE_PASSWD,
|
||||
MiNiFiProperties.NIFI_MINIFI_SECURITY_KEY_PASSWD.getKey(), NiFiProperties.SECURITY_KEY_PASSWD,
|
||||
MiNiFiProperties.NIFI_MINIFI_SECURITY_TRUSTSTORE.getKey(), NiFiProperties.SECURITY_TRUSTSTORE,
|
||||
MiNiFiProperties.NIFI_MINIFI_SECURITY_TRUSTSTORE_TYPE.getKey(), NiFiProperties.SECURITY_TRUSTSTORE_TYPE,
|
||||
MiNiFiProperties.NIFI_MINIFI_SECURITY_TRUSTSTORE_PASSWD.getKey(), NiFiProperties.SECURITY_TRUSTSTORE_PASSWD
|
||||
);
|
||||
|
||||
static final String DEFAULT_SENSITIVE_PROPERTIES_ENCODING_ALGORITHM = "NIFI_PBKDF2_AES_GCM_256";
|
||||
|
||||
private static final Base64.Encoder KEY_ENCODER = Base64.getEncoder().withoutPadding();
|
||||
private static final int SENSITIVE_PROPERTIES_KEY_LENGTH = 24;
|
||||
|
||||
private static final String C2_PROPERTY_PREFIX = "c2.";
|
||||
private static final String NIFI_PREFIX = "nifi.";
|
||||
|
||||
public static final String FILE_EXTENSION_DELIMITER = ".";
|
||||
|
||||
public void generateMinifiProperties(String configDirectory, Properties bootstrapProperties) throws ConfigurationChangeException {
|
||||
String minifiPropertiesFileName = Path.of(getMiNiFiPropertiesPath(bootstrapProperties, new File(configDirectory))).getFileName().toString();
|
||||
Path minifiPropertiesFile = Path.of(configDirectory, minifiPropertiesFileName);
|
||||
|
||||
Map<String, String> existingSensitivePropertiesConfiguration = extractSensitivePropertiesConfiguration(minifiPropertiesFile);
|
||||
OrderedProperties minifiProperties = prepareMinifiProperties(bootstrapProperties, existingSensitivePropertiesConfiguration);
|
||||
|
||||
persistMinifiProperties(minifiPropertiesFile, minifiProperties);
|
||||
}
|
||||
|
||||
private Map<String, String> extractSensitivePropertiesConfiguration(Path minifiPropertiesFile) throws ConfigurationChangeException {
|
||||
if (!Files.exists(minifiPropertiesFile)) {
|
||||
return Map.of();
|
||||
}
|
||||
|
||||
Properties minifiProperties = new Properties();
|
||||
try (InputStream inputStream = Files.newInputStream(minifiPropertiesFile)) {
|
||||
minifiProperties.load(inputStream);
|
||||
} catch (IOException e) {
|
||||
throw new ConfigurationChangeException("Unable to load MiNiFi properties from " + minifiPropertiesFile, e);
|
||||
}
|
||||
|
||||
return Map.of(
|
||||
NiFiProperties.SENSITIVE_PROPS_KEY, minifiProperties.getProperty(NiFiProperties.SENSITIVE_PROPS_KEY),
|
||||
NiFiProperties.SENSITIVE_PROPS_ALGORITHM, minifiProperties.getProperty(NiFiProperties.SENSITIVE_PROPS_ALGORITHM)
|
||||
);
|
||||
}
|
||||
|
||||
private OrderedProperties prepareMinifiProperties(Properties bootstrapProperties, Map<String, String> existingSensitivePropertiesConfiguration) {
|
||||
OrderedProperties minifiProperties = new OrderedProperties();
|
||||
|
||||
NIFI_PROPERTIES_WITH_DEFAULT_VALUES_AND_COMMENTS
|
||||
.forEach(triple -> minifiProperties.setProperty(triple.getLeft(), triple.getMiddle(), triple.getRight()));
|
||||
|
||||
getNonBlankPropertiesWithPredicate(bootstrapProperties, entry -> MINIFI_TO_NIFI_PROPERTY_MAPPING.containsKey(entry.getKey()))
|
||||
.forEach(entry -> minifiProperties.setProperty(MINIFI_TO_NIFI_PROPERTY_MAPPING.get(entry.getKey()), entry.getValue()));
|
||||
|
||||
getSensitiveProperties(bootstrapProperties, existingSensitivePropertiesConfiguration)
|
||||
.forEach(entry -> minifiProperties.setProperty(entry.getKey(), entry.getValue()));
|
||||
|
||||
getNonBlankPropertiesWithPredicate(bootstrapProperties, entry -> ((String) entry.getKey()).startsWith(C2_PROPERTY_PREFIX))
|
||||
.forEach(entry -> minifiProperties.setProperty(entry.getKey(), entry.getValue()));
|
||||
|
||||
getNonBlankPropertiesWithPredicate(bootstrapProperties, entry -> ((String) entry.getKey()).startsWith(NIFI_PREFIX))
|
||||
.forEach(entry -> minifiProperties.setProperty(entry.getKey(), entry.getValue()));
|
||||
|
||||
bootstrapFileAndLogProperties()
|
||||
.forEach(entry -> minifiProperties.setProperty(entry.getKey(), entry.getValue()));
|
||||
|
||||
return minifiProperties;
|
||||
}
|
||||
|
||||
private List<Pair<String, String>> getNonBlankPropertiesWithPredicate(Properties bootstrapProperties, Predicate<Map.Entry> predicate) {
|
||||
return ofNullable(bootstrapProperties)
|
||||
.map(Properties::entrySet)
|
||||
.orElseGet(Set::of)
|
||||
.stream()
|
||||
.filter(predicate)
|
||||
.filter(entry -> isNotBlank((String) entry.getValue()))
|
||||
.map(entry -> Pair.of((String) entry.getKey(), (String) entry.getValue()))
|
||||
.sorted((o1, o2) -> Comparator.<String>naturalOrder().compare(o1.getKey(), o2.getKey()))
|
||||
.toList();
|
||||
}
|
||||
|
||||
private List<Pair<String, String>> getSensitiveProperties(Properties bootstrapProperties, Map<String, String> existingSensitivePropertiesConfiguration) {
|
||||
return existingSensitivePropertiesConfiguration.isEmpty()
|
||||
? List.of(
|
||||
Pair.of(NiFiProperties.SENSITIVE_PROPS_KEY,
|
||||
ofNullable(bootstrapProperties.getProperty(MiNiFiProperties.NIFI_MINIFI_SENSITIVE_PROPS_KEY.getKey()))
|
||||
.filter(StringUtils::isNotBlank)
|
||||
.orElseGet(this::generateSensitivePropertiesKey)),
|
||||
Pair.of(NiFiProperties.SENSITIVE_PROPS_ALGORITHM,
|
||||
ofNullable(bootstrapProperties.getProperty(MiNiFiProperties.NIFI_MINIFI_SENSITIVE_PROPS_ALGORITHM.getKey()))
|
||||
.filter(StringUtils::isNotBlank)
|
||||
.orElse(DEFAULT_SENSITIVE_PROPERTIES_ENCODING_ALGORITHM)))
|
||||
: existingSensitivePropertiesConfiguration.entrySet()
|
||||
.stream()
|
||||
.map(entry -> Pair.of(entry.getKey(), entry.getValue()))
|
||||
.toList();
|
||||
}
|
||||
|
||||
private String generateSensitivePropertiesKey() {
|
||||
SecureRandom secureRandom = new SecureRandom();
|
||||
byte[] sensitivePropertiesKeyBinary = new byte[SENSITIVE_PROPERTIES_KEY_LENGTH];
|
||||
secureRandom.nextBytes(sensitivePropertiesKeyBinary);
|
||||
return KEY_ENCODER.encodeToString(sensitivePropertiesKeyBinary);
|
||||
}
|
||||
|
||||
private List<Pair<String, String>> bootstrapFileAndLogProperties() {
|
||||
return List.of(
|
||||
Pair.of(MINIFI_BOOTSTRAP_FILE_PATH, getBootstrapConfFile().getAbsolutePath()),
|
||||
Pair.of(MINIFI_LOG_DIRECTORY, getProperty(LOG_DIR, DEFAULT_LOG_DIR).trim()),
|
||||
Pair.of(MINIFI_APP_LOG_FILE, join(FILE_EXTENSION_DELIMITER,
|
||||
getProperty(APP_LOG_FILE_NAME, DEFAULT_APP_LOG_FILE_NAME).trim(),
|
||||
getProperty(APP_LOG_FILE_EXTENSION, DEFAULT_LOG_FILE_EXTENSION).trim())),
|
||||
Pair.of(MINIFI_BOOTSTRAP_LOG_FILE, join(FILE_EXTENSION_DELIMITER,
|
||||
getProperty(BOOTSTRAP_LOG_FILE_NAME, DEFAULT_BOOTSTRAP_LOG_FILE_NAME).trim(),
|
||||
getProperty(BOOTSTRAP_LOG_FILE_EXTENSION, DEFAULT_LOG_FILE_EXTENSION).trim()))
|
||||
);
|
||||
}
|
||||
|
||||
private void persistMinifiProperties(Path minifiPropertiesFile, OrderedProperties minifiProperties) throws ConfigurationChangeException {
|
||||
try (ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
|
||||
FileOutputStream fileOutputStream = new FileOutputStream(minifiPropertiesFile.toString())) {
|
||||
minifiProperties.store(byteArrayOutputStream, PROPERTIES_FILE_APACHE_2_0_LICENSE);
|
||||
byteArrayOutputStream.writeTo(fileOutputStream);
|
||||
} catch (Exception e) {
|
||||
throw new ConfigurationChangeException("Failed to write MiNiFi properties to " + minifiPropertiesFile, e);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -14,9 +14,10 @@
|
|||
* 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 static org.apache.nifi.minifi.commons.api.MiNiFiProperties.NIFI_MINIFI_STATUS_REPORTER_COMPONENTS;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
|
@ -44,7 +45,7 @@ public class PeriodicStatusReporterManager implements QueryableStatusAggregator
|
|||
private Set<PeriodicStatusReporter> periodicStatusReporters = Collections.emptySet();
|
||||
|
||||
public PeriodicStatusReporterManager(Properties bootstrapProperties, MiNiFiStatusProvider miNiFiStatusProvider, MiNiFiCommandSender miNiFiCommandSender,
|
||||
MiNiFiParameters miNiFiParameters) {
|
||||
MiNiFiParameters miNiFiParameters) {
|
||||
this.bootstrapProperties = bootstrapProperties;
|
||||
this.miNiFiStatusProvider = miNiFiStatusProvider;
|
||||
this.miNiFiCommandSender = miNiFiCommandSender;
|
||||
|
@ -54,7 +55,7 @@ public class PeriodicStatusReporterManager implements QueryableStatusAggregator
|
|||
public void startPeriodicNotifiers() {
|
||||
periodicStatusReporters = initializePeriodicNotifiers();
|
||||
|
||||
for (PeriodicStatusReporter periodicStatusReporter: periodicStatusReporters) {
|
||||
for (PeriodicStatusReporter periodicStatusReporter : periodicStatusReporters) {
|
||||
periodicStatusReporter.start();
|
||||
LOGGER.debug("Started {} notifier", periodicStatusReporter.getClass().getCanonicalName());
|
||||
}
|
||||
|
@ -96,7 +97,7 @@ public class PeriodicStatusReporterManager implements QueryableStatusAggregator
|
|||
LOGGER.debug("Initiating bootstrap periodic status reporters...");
|
||||
Set<PeriodicStatusReporter> statusReporters = new HashSet<>();
|
||||
|
||||
String reportersCsv = bootstrapProperties.getProperty(STATUS_REPORTER_COMPONENTS_KEY);
|
||||
String reportersCsv = bootstrapProperties.getProperty(NIFI_MINIFI_STATUS_REPORTER_COMPONENTS.getKey());
|
||||
|
||||
if (reportersCsv != null && !reportersCsv.isEmpty()) {
|
||||
for (String reporterClassname : reportersCsv.split(",")) {
|
||||
|
|
|
@ -1,95 +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.service;
|
||||
|
||||
import static java.util.Optional.ofNullable;
|
||||
import static org.apache.nifi.minifi.commons.api.MiNiFiConstants.CONFIG_UPDATED_FILE_NAME;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileInputStream;
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.file.Path;
|
||||
import java.util.Optional;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.nifi.minifi.bootstrap.RunMiNiFi;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeListener;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.differentiators.Differentiator;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.differentiators.WholeConfigDifferentiator;
|
||||
import org.apache.nifi.minifi.bootstrap.util.ByteBufferInputStream;
|
||||
import org.apache.nifi.minifi.bootstrap.util.ConfigTransformer;
|
||||
import org.apache.nifi.minifi.commons.api.MiNiFiCommandState;
|
||||
import org.apache.nifi.util.NiFiProperties;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
public class UpdateConfigurationService {
|
||||
|
||||
private static final Logger logger = LoggerFactory.getLogger(UpdateConfigurationService.class);
|
||||
private static final String FALLBACK_CONFIG_FILE_DIR = "./conf/";
|
||||
|
||||
private final Differentiator<ByteBuffer> differentiator;
|
||||
private final RunMiNiFi runMiNiFi;
|
||||
private final ConfigurationChangeListener miNiFiConfigurationChangeListener;
|
||||
private final BootstrapFileProvider bootstrapFileProvider;
|
||||
|
||||
public UpdateConfigurationService(RunMiNiFi runMiNiFi, ConfigurationChangeListener miNiFiConfigurationChangeListener, BootstrapFileProvider bootstrapFileProvider) {
|
||||
this.differentiator = WholeConfigDifferentiator.getByteBufferDifferentiator();
|
||||
this.differentiator.initialize(runMiNiFi);
|
||||
this.runMiNiFi = runMiNiFi;
|
||||
this.miNiFiConfigurationChangeListener = miNiFiConfigurationChangeListener;
|
||||
this.bootstrapFileProvider = bootstrapFileProvider;
|
||||
}
|
||||
|
||||
public Optional<MiNiFiCommandState> handleUpdate() {
|
||||
logger.info("Handling configuration update");
|
||||
MiNiFiCommandState commandState = null;
|
||||
try (FileInputStream configFile = new FileInputStream(getConfigFilePath().toFile())) {
|
||||
ByteBuffer readOnlyNewConfig = ConfigTransformer.overrideNonFlowSectionsFromOriginalSchema(
|
||||
IOUtils.toByteArray(configFile), runMiNiFi.getConfigFileReference().get().duplicate(), bootstrapFileProvider.getBootstrapProperties());
|
||||
if (differentiator.isNew(readOnlyNewConfig)) {
|
||||
miNiFiConfigurationChangeListener.handleChange(new ByteBufferInputStream(readOnlyNewConfig.duplicate()));
|
||||
} else {
|
||||
logger.info("The given configuration does not contain any update. No operation required");
|
||||
commandState = MiNiFiCommandState.NO_OPERATION;
|
||||
}
|
||||
} catch (Exception e) {
|
||||
commandState = MiNiFiCommandState.NOT_APPLIED_WITHOUT_RESTART;
|
||||
logger.error("Could not handle configuration update", e);
|
||||
}
|
||||
return Optional.ofNullable(commandState);
|
||||
}
|
||||
|
||||
private Path getConfigFilePath() {
|
||||
return ofNullable(safeGetPropertiesFilePath())
|
||||
.map(File::new)
|
||||
.map(File::getParent)
|
||||
.map(parentDir -> new File(parentDir + CONFIG_UPDATED_FILE_NAME))
|
||||
.orElse(new File(FALLBACK_CONFIG_FILE_DIR + CONFIG_UPDATED_FILE_NAME)).toPath();
|
||||
}
|
||||
|
||||
private String safeGetPropertiesFilePath() {
|
||||
String propertyFile = null;
|
||||
try {
|
||||
propertyFile = bootstrapFileProvider.getBootstrapProperties().getProperty(NiFiProperties.PROPERTIES_FILE_PATH, null);
|
||||
} catch (IOException e) {
|
||||
logger.error("Failed to get properties file path");
|
||||
}
|
||||
return propertyFile;
|
||||
}
|
||||
}
|
|
@ -19,13 +19,10 @@ 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.util.ConfigTransformer.asByteArrayInputStream;
|
||||
import static org.apache.nifi.minifi.bootstrap.util.ConfigTransformer.generateConfigFiles;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileInputStream;
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.file.Files;
|
||||
import java.util.Optional;
|
||||
import java.util.Properties;
|
||||
|
@ -38,11 +35,13 @@ public class UpdatePropertiesService {
|
|||
private final RunMiNiFi runner;
|
||||
private final Logger logger;
|
||||
private final BootstrapFileProvider bootstrapFileProvider;
|
||||
private final MiNiFiPropertiesGenerator miNiFiPropertiesGenerator;
|
||||
|
||||
public UpdatePropertiesService(RunMiNiFi runner, Logger logger, BootstrapFileProvider bootstrapFileProvider) {
|
||||
this.runner = runner;
|
||||
this.logger = logger;
|
||||
this.bootstrapFileProvider = bootstrapFileProvider;
|
||||
this.miNiFiPropertiesGenerator = new MiNiFiPropertiesGenerator();
|
||||
}
|
||||
|
||||
public Optional<MiNiFiCommandState> handleUpdate() {
|
||||
|
@ -72,9 +71,7 @@ public class UpdatePropertiesService {
|
|||
throws IOException, ConfigurationChangeException {
|
||||
Optional<MiNiFiCommandState> commandState = Optional.empty();
|
||||
try {
|
||||
ByteBuffer byteBuffer = generateConfigFiles(asByteArrayInputStream(runner.getConfigFileReference().get().duplicate()),
|
||||
bootstrapProperties.getProperty(CONF_DIR_KEY), bootstrapProperties);
|
||||
runner.getConfigFileReference().set(byteBuffer.asReadOnlyBuffer());
|
||||
miNiFiPropertiesGenerator.generateMinifiProperties(bootstrapProperties.getProperty(CONF_DIR_KEY), bootstrapProperties);
|
||||
restartInstance();
|
||||
} catch (Exception e) {
|
||||
commandState = Optional.of(MiNiFiCommandState.NOT_APPLIED_WITHOUT_RESTART);
|
||||
|
@ -84,10 +81,7 @@ public class UpdatePropertiesService {
|
|||
}
|
||||
// read reverted properties
|
||||
bootstrapProperties = bootstrapFileProvider.getBootstrapProperties();
|
||||
|
||||
ByteBuffer byteBuffer = generateConfigFiles(
|
||||
asByteArrayInputStream(runner.getConfigFileReference().get().duplicate()), bootstrapProperties.getProperty(CONF_DIR_KEY), bootstrapProperties);
|
||||
runner.getConfigFileReference().set(byteBuffer.asReadOnlyBuffer());
|
||||
miNiFiPropertiesGenerator.generateMinifiProperties(bootstrapProperties.getProperty(CONF_DIR_KEY), bootstrapProperties);
|
||||
|
||||
logger.debug("Transformation of new config file failed after swap file was created, deleting it.");
|
||||
if (!bootstrapSwapConfigFile.delete()) {
|
||||
|
|
|
@ -17,6 +17,12 @@
|
|||
|
||||
package org.apache.nifi.minifi.bootstrap.status.reporters;
|
||||
|
||||
import static org.apache.nifi.minifi.commons.api.MiNiFiProperties.NIFI_MINIFI_STATUS_REPORTER_LOG_LEVEL;
|
||||
import static org.apache.nifi.minifi.commons.api.MiNiFiProperties.NIFI_MINIFI_STATUS_REPORTER_LOG_PERIOD;
|
||||
import static org.apache.nifi.minifi.commons.api.MiNiFiProperties.NIFI_MINIFI_STATUS_REPORTER_LOG_QUERY;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Properties;
|
||||
import org.apache.nifi.logging.LogLevel;
|
||||
import org.apache.nifi.minifi.bootstrap.QueryableStatusAggregator;
|
||||
import org.apache.nifi.minifi.bootstrap.status.PeriodicStatusReporter;
|
||||
|
@ -24,61 +30,49 @@ import org.apache.nifi.minifi.commons.status.FlowStatusReport;
|
|||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Properties;
|
||||
|
||||
import static org.apache.nifi.minifi.commons.schema.common.BootstrapPropertyKeys.STATUS_REPORTER_PROPERTY_PREFIX;
|
||||
|
||||
public class StatusLogger extends PeriodicStatusReporter {
|
||||
|
||||
|
||||
private volatile QueryableStatusAggregator queryableStatusAggregator;
|
||||
private volatile LogLevel logLevel;
|
||||
private volatile String statusQuery;
|
||||
|
||||
protected static Logger logger = LoggerFactory.getLogger(StatusLogger.class);
|
||||
|
||||
|
||||
public static final String LOGGER_STATUS_REPORTER_PROPERTY_PREFIX = STATUS_REPORTER_PROPERTY_PREFIX + ".log";
|
||||
public static final String REPORT_PERIOD_KEY = LOGGER_STATUS_REPORTER_PROPERTY_PREFIX + ".period";
|
||||
public static final String LOGGING_LEVEL_KEY = LOGGER_STATUS_REPORTER_PROPERTY_PREFIX + ".level";
|
||||
public static final String QUERY_KEY = LOGGER_STATUS_REPORTER_PROPERTY_PREFIX + ".query";
|
||||
|
||||
static final String ENCOUNTERED_IO_EXCEPTION = "Encountered an IO Exception while attempting to query the flow status.";
|
||||
|
||||
@Override
|
||||
public void initialize(Properties properties, QueryableStatusAggregator queryableStatusAggregator) {
|
||||
public void initialize(Properties properties, QueryableStatusAggregator queryableStatusAggregator) {
|
||||
this.queryableStatusAggregator = queryableStatusAggregator;
|
||||
|
||||
String periodString = properties.getProperty(REPORT_PERIOD_KEY);
|
||||
String periodString = properties.getProperty(NIFI_MINIFI_STATUS_REPORTER_LOG_PERIOD.getKey());
|
||||
if (periodString == null) {
|
||||
throw new IllegalStateException(REPORT_PERIOD_KEY + " is null but it is required. Please configure it.");
|
||||
throw new IllegalStateException(NIFI_MINIFI_STATUS_REPORTER_LOG_PERIOD.getKey() + " is null but it is required. Please configure it.");
|
||||
}
|
||||
try {
|
||||
setPeriod(Integer.parseInt(periodString));
|
||||
} catch (NumberFormatException e) {
|
||||
throw new IllegalStateException(REPORT_PERIOD_KEY + " is not a valid number.", e);
|
||||
throw new IllegalStateException(NIFI_MINIFI_STATUS_REPORTER_LOG_PERIOD.getKey() + " is not a valid number.", e);
|
||||
}
|
||||
|
||||
|
||||
String loglevelString = properties.getProperty(LOGGING_LEVEL_KEY);
|
||||
String loglevelString = properties.getProperty(NIFI_MINIFI_STATUS_REPORTER_LOG_LEVEL.getKey());
|
||||
if (loglevelString == null) {
|
||||
throw new IllegalStateException(LOGGING_LEVEL_KEY + " is null but it is required. Please configure it.");
|
||||
throw new IllegalStateException(NIFI_MINIFI_STATUS_REPORTER_LOG_LEVEL.getKey() + " is null but it is required. Please configure it.");
|
||||
}
|
||||
|
||||
try {
|
||||
logLevel = LogLevel.valueOf(loglevelString.toUpperCase());
|
||||
} catch (IllegalArgumentException e) {
|
||||
throw new IllegalStateException("Value set for " + LOGGING_LEVEL_KEY + " is not a valid log level.");
|
||||
throw new IllegalStateException("Value set for " + NIFI_MINIFI_STATUS_REPORTER_LOG_LEVEL.getKey() + " is not a valid log level.");
|
||||
}
|
||||
|
||||
if (LogLevel.FATAL.equals(logLevel)){
|
||||
throw new IllegalStateException("Cannot log status at the FATAL level. Please configure " + LOGGING_LEVEL_KEY + " to another value.");
|
||||
if (LogLevel.FATAL.equals(logLevel)) {
|
||||
throw new IllegalStateException("Cannot log status at the FATAL level. Please configure " + NIFI_MINIFI_STATUS_REPORTER_LOG_LEVEL.getKey() + " to another value.");
|
||||
}
|
||||
|
||||
statusQuery = properties.getProperty(QUERY_KEY);
|
||||
statusQuery = properties.getProperty(NIFI_MINIFI_STATUS_REPORTER_LOG_QUERY.getKey());
|
||||
if (statusQuery == null) {
|
||||
throw new IllegalStateException(QUERY_KEY + " is null but it is required. Please configure it.");
|
||||
throw new IllegalStateException(NIFI_MINIFI_STATUS_REPORTER_LOG_QUERY.getKey() + " is null but it is required. Please configure it.");
|
||||
}
|
||||
|
||||
reportRunner = new ReportStatusRunner();
|
||||
|
|
|
@ -1,102 +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.util;
|
||||
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.minifi.commons.schema.ProvenanceReportingSchema;
|
||||
import org.apache.nifi.minifi.commons.schema.SecurityPropertiesSchema;
|
||||
import org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.Properties;
|
||||
|
||||
import static org.apache.nifi.minifi.commons.schema.common.BootstrapPropertyKeys.BOOTSTRAP_KEYS_TO_YML_KEYS;
|
||||
import static org.apache.nifi.minifi.commons.schema.common.BootstrapPropertyKeys.BOOTSTRAP_PROVENANCE_REPORTING_KEYS;
|
||||
import static org.apache.nifi.minifi.commons.schema.common.BootstrapPropertyKeys.BOOTSTRAP_SECURITY_PROPERTY_KEYS;
|
||||
import static org.apache.nifi.minifi.commons.schema.common.BootstrapPropertyKeys.BOOTSTRAP_SENSITIVE_PROPERTY_KEYS;
|
||||
import static org.apache.nifi.minifi.commons.schema.common.BootstrapPropertyKeys.USE_PARENT_SSL;
|
||||
|
||||
public class BootstrapTransformer {
|
||||
|
||||
public static Optional<SecurityPropertiesSchema> buildSecurityPropertiesFromBootstrap(final Properties bootstrapProperties) {
|
||||
|
||||
Optional<SecurityPropertiesSchema> securityPropsOptional = Optional.empty();
|
||||
|
||||
final Map<String, Object> securityProperties = new HashMap<>();
|
||||
|
||||
if (bootstrapProperties != null) {
|
||||
BOOTSTRAP_SECURITY_PROPERTY_KEYS.stream()
|
||||
.filter(key -> StringUtils.isNotBlank(bootstrapProperties.getProperty(key)))
|
||||
.forEach(key ->
|
||||
securityProperties.put(BOOTSTRAP_KEYS_TO_YML_KEYS.get(key), bootstrapProperties.getProperty(key))
|
||||
);
|
||||
|
||||
if (!securityProperties.isEmpty()) {
|
||||
// Determine if sensitive properties were provided
|
||||
final Map<String, String> sensitiveProperties = new HashMap<>();
|
||||
BOOTSTRAP_SENSITIVE_PROPERTY_KEYS.stream()
|
||||
.filter(key -> StringUtils.isNotBlank(bootstrapProperties.getProperty(key)))
|
||||
.forEach(key ->
|
||||
sensitiveProperties.put(BOOTSTRAP_KEYS_TO_YML_KEYS.get(key), bootstrapProperties.getProperty(key))
|
||||
);
|
||||
if (!sensitiveProperties.isEmpty()) {
|
||||
securityProperties.put(CommonPropertyKeys.SENSITIVE_PROPS_KEY, sensitiveProperties);
|
||||
}
|
||||
|
||||
final SecurityPropertiesSchema securityPropertiesSchema = new SecurityPropertiesSchema(securityProperties);
|
||||
securityPropsOptional = Optional.of(securityPropertiesSchema);
|
||||
}
|
||||
}
|
||||
|
||||
return securityPropsOptional;
|
||||
}
|
||||
|
||||
public static Optional<ProvenanceReportingSchema> buildProvenanceReportingPropertiesFromBootstrap(final Properties bootstrapProperties) {
|
||||
|
||||
Optional<ProvenanceReportingSchema> provenanceReportingPropsOptional = Optional.empty();
|
||||
|
||||
final Map<String, Object> provenanceReportingProperties = new HashMap<>();
|
||||
if (bootstrapProperties != null) {
|
||||
BOOTSTRAP_PROVENANCE_REPORTING_KEYS.stream()
|
||||
.filter(key -> StringUtils.isNotBlank(bootstrapProperties.getProperty(key)))
|
||||
.forEach(key ->
|
||||
provenanceReportingProperties.put(BOOTSTRAP_KEYS_TO_YML_KEYS.get(key), bootstrapProperties.getProperty(key))
|
||||
);
|
||||
|
||||
if (!provenanceReportingProperties.isEmpty()) {
|
||||
final ProvenanceReportingSchema provenanceReportingSchema = new ProvenanceReportingSchema(provenanceReportingProperties);
|
||||
provenanceReportingPropsOptional = Optional.of(provenanceReportingSchema);
|
||||
}
|
||||
}
|
||||
|
||||
return provenanceReportingPropsOptional;
|
||||
}
|
||||
|
||||
public static boolean processorSSLOverride(final Properties bootstrapProperties) {
|
||||
boolean shouldOverride = false;
|
||||
|
||||
if (bootstrapProperties != null) {
|
||||
shouldOverride = Boolean.parseBoolean(bootstrapProperties.getProperty(USE_PARENT_SSL));
|
||||
}
|
||||
|
||||
return shouldOverride;
|
||||
}
|
||||
|
||||
}
|
|
@ -1,856 +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.util;
|
||||
|
||||
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.service.MiNiFiExecCommandProvider.APP_LOG_FILE_EXTENSION;
|
||||
import static org.apache.nifi.minifi.bootstrap.service.MiNiFiExecCommandProvider.APP_LOG_FILE_NAME;
|
||||
import static org.apache.nifi.minifi.bootstrap.service.MiNiFiExecCommandProvider.BOOTSTRAP_LOG_FILE_EXTENSION;
|
||||
import static org.apache.nifi.minifi.bootstrap.service.MiNiFiExecCommandProvider.BOOTSTRAP_LOG_FILE_NAME;
|
||||
import static org.apache.nifi.minifi.bootstrap.service.MiNiFiExecCommandProvider.DEFAULT_APP_LOG_FILE_NAME;
|
||||
import static org.apache.nifi.minifi.bootstrap.service.MiNiFiExecCommandProvider.DEFAULT_BOOTSTRAP_LOG_FILE_NAME;
|
||||
import static org.apache.nifi.minifi.bootstrap.service.MiNiFiExecCommandProvider.DEFAULT_LOG_DIR;
|
||||
import static org.apache.nifi.minifi.bootstrap.service.MiNiFiExecCommandProvider.DEFAULT_LOG_FILE_EXTENSION;
|
||||
import static org.apache.nifi.minifi.bootstrap.service.MiNiFiExecCommandProvider.LOG_DIR;
|
||||
|
||||
import java.io.ByteArrayInputStream;
|
||||
import java.io.FileOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
import java.io.StringWriter;
|
||||
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.RunMiNiFi;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeException;
|
||||
import org.apache.nifi.minifi.bootstrap.exception.InvalidConfigurationException;
|
||||
import org.apache.nifi.minifi.bootstrap.service.BootstrapFileProvider;
|
||||
import org.apache.nifi.minifi.commons.schema.ComponentStatusRepositorySchema;
|
||||
import org.apache.nifi.minifi.commons.schema.ConfigSchema;
|
||||
import org.apache.nifi.minifi.commons.schema.ConnectionSchema;
|
||||
import org.apache.nifi.minifi.commons.schema.ContentRepositorySchema;
|
||||
import org.apache.nifi.minifi.commons.schema.ControllerServiceSchema;
|
||||
import org.apache.nifi.minifi.commons.schema.CorePropertiesSchema;
|
||||
import org.apache.nifi.minifi.commons.schema.FlowControllerSchema;
|
||||
import org.apache.nifi.minifi.commons.schema.FlowFileRepositorySchema;
|
||||
import org.apache.nifi.minifi.commons.schema.FunnelSchema;
|
||||
import org.apache.nifi.minifi.commons.schema.PortSchema;
|
||||
import org.apache.nifi.minifi.commons.schema.ProcessGroupSchema;
|
||||
import org.apache.nifi.minifi.commons.schema.ProcessorSchema;
|
||||
import org.apache.nifi.minifi.commons.schema.ProvenanceReportingSchema;
|
||||
import org.apache.nifi.minifi.commons.schema.ProvenanceRepositorySchema;
|
||||
import org.apache.nifi.minifi.commons.schema.RemotePortSchema;
|
||||
import org.apache.nifi.minifi.commons.schema.RemoteProcessGroupSchema;
|
||||
import org.apache.nifi.minifi.commons.schema.ReportingSchema;
|
||||
import org.apache.nifi.minifi.commons.schema.SecurityPropertiesSchema;
|
||||
import org.apache.nifi.minifi.commons.schema.SensitivePropsSchema;
|
||||
import org.apache.nifi.minifi.commons.schema.SwapSchema;
|
||||
import org.apache.nifi.minifi.commons.schema.common.ConvertableSchema;
|
||||
import org.apache.nifi.minifi.commons.schema.common.Schema;
|
||||
import org.apache.nifi.minifi.commons.schema.common.StringUtil;
|
||||
import org.apache.nifi.minifi.commons.schema.serialization.SchemaLoader;
|
||||
import org.apache.nifi.util.NiFiProperties;
|
||||
import org.apache.nifi.xml.processing.ProcessingException;
|
||||
import org.apache.nifi.xml.processing.parsers.StandardDocumentProvider;
|
||||
import org.apache.nifi.xml.processing.transform.StandardTransformProvider;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
import org.w3c.dom.DOMException;
|
||||
import org.w3c.dom.Document;
|
||||
import org.w3c.dom.Element;
|
||||
|
||||
public final class ConfigTransformer {
|
||||
// Underlying version of NIFI will be using
|
||||
public static final String ROOT_GROUP = "Root-Group";
|
||||
|
||||
static final String MINIFI_CONFIG_FILE_PATH = "nifi.minifi.config.file";
|
||||
static final String MINIFI_BOOTSTRAP_FILE_PATH = "nifi.minifi.bootstrap.file";
|
||||
static final String MINIFI_LOG_DIRECTORY = "nifi.minifi.log.directory";
|
||||
static final String MINIFI_APP_LOG_FILE = "nifi.minifi.app.log.file";
|
||||
static final String MINIFI_BOOTSTRAP_LOG_FILE = "nifi.minifi.bootstrap.log.file";
|
||||
|
||||
private static final Logger LOGGER = LoggerFactory.getLogger(ConfigTransformer.class);
|
||||
|
||||
private static final String OVERRIDE_CORE_PROPERTIES_KEY = PULL_HTTP_BASE_KEY + ".override.core";
|
||||
private static final Base64.Encoder KEY_ENCODER = Base64.getEncoder().withoutPadding();
|
||||
private static final int SENSITIVE_PROPERTIES_KEY_LENGTH = 24;
|
||||
|
||||
// Final util classes should have private constructor
|
||||
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> 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) {
|
||||
configSchemaNew.setSecurityProperties(securityProperties);
|
||||
LOGGER.info("Bootstrap flow override: Replaced security properties");
|
||||
}
|
||||
|
||||
if (provenanceReportingProperties != null) {
|
||||
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 : 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()));
|
||||
|
||||
// Config files and log files
|
||||
if (bootstrapProperties != null) {
|
||||
configSchemaNew.getNifiPropertiesOverrides().putIfAbsent(MINIFI_CONFIG_FILE_PATH, bootstrapProperties.getProperty(RunMiNiFi.MINIFI_CONFIG_FILE_KEY));
|
||||
}
|
||||
configSchemaNew.getNifiPropertiesOverrides().putIfAbsent(MINIFI_BOOTSTRAP_FILE_PATH, BootstrapFileProvider.getBootstrapConfFile().getAbsolutePath());
|
||||
configSchemaNew.getNifiPropertiesOverrides().putIfAbsent(MINIFI_LOG_DIRECTORY, System.getProperty(LOG_DIR, DEFAULT_LOG_DIR).trim());
|
||||
configSchemaNew.getNifiPropertiesOverrides().putIfAbsent(MINIFI_APP_LOG_FILE,
|
||||
System.getProperty(APP_LOG_FILE_NAME, DEFAULT_APP_LOG_FILE_NAME).trim() + "." + System.getProperty(APP_LOG_FILE_EXTENSION, DEFAULT_LOG_FILE_EXTENSION).trim());
|
||||
configSchemaNew.getNifiPropertiesOverrides().putIfAbsent(MINIFI_BOOTSTRAP_LOG_FILE,
|
||||
System.getProperty(BOOTSTRAP_LOG_FILE_NAME, DEFAULT_BOOTSTRAP_LOG_FILE_NAME).trim() + "." + System.getProperty(BOOTSTRAP_LOG_FILE_EXTENSION, DEFAULT_LOG_FILE_EXTENSION).trim());
|
||||
|
||||
// Create nifi.properties and flow.xml.gz in memory
|
||||
ByteArrayOutputStream nifiPropertiesOutputStream = new ByteArrayOutputStream();
|
||||
writeNiFiProperties(configSchemaNew, nifiPropertiesOutputStream);
|
||||
|
||||
writeFlowXmlFile(configSchemaNew, destPath);
|
||||
|
||||
// Write nifi.properties and flow.xml.gz
|
||||
writeNiFiPropertiesFile(nifiPropertiesOutputStream, destPath);
|
||||
}
|
||||
|
||||
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("], [")) + "]");
|
||||
}
|
||||
return schema;
|
||||
}
|
||||
|
||||
public static ByteArrayInputStream asByteArrayInputStream(ByteBuffer byteBuffer) {
|
||||
byte[] config = new byte[byteBuffer.remaining()];
|
||||
byteBuffer.get(config);
|
||||
return new ByteArrayInputStream(config);
|
||||
}
|
||||
|
||||
protected static void writeNiFiPropertiesFile(ByteArrayOutputStream nifiPropertiesOutputStream, String destPath) throws IOException {
|
||||
final Path nifiPropertiesPath = Paths.get(destPath, "nifi.properties");
|
||||
try (FileOutputStream nifiProperties = new FileOutputStream(nifiPropertiesPath.toString())) {
|
||||
nifiPropertiesOutputStream.writeTo(nifiProperties);
|
||||
} finally {
|
||||
if (nifiPropertiesOutputStream != null) {
|
||||
nifiPropertiesOutputStream.flush();
|
||||
nifiPropertiesOutputStream.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
protected static void writeFlowXmlFile(ConfigSchema configSchema, OutputStream outputStream) throws ConfigTransformerException {
|
||||
final StreamResult streamResult = new StreamResult(outputStream);
|
||||
|
||||
// configure the transformer and convert the DOM
|
||||
final StandardTransformProvider transformProvider = new StandardTransformProvider();
|
||||
transformProvider.setIndent(true);
|
||||
|
||||
// transform the document to byte stream
|
||||
transformProvider.transform(createFlowXml(configSchema), streamResult);
|
||||
}
|
||||
|
||||
protected static void writeFlowXmlFile(ConfigSchema configSchema, String path) throws IOException, ConfigTransformerException {
|
||||
try (OutputStream fileOut = Files.newOutputStream(Paths.get(path, "flow.xml.gz"))) {
|
||||
try (OutputStream outStream = new GZIPOutputStream(fileOut)) {
|
||||
writeFlowXmlFile(configSchema, outStream);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
protected static void writeNiFiProperties(ConfigSchema configSchema, OutputStream outputStream) throws IOException, ConfigurationChangeException {
|
||||
try {
|
||||
CorePropertiesSchema coreProperties = configSchema.getCoreProperties();
|
||||
FlowFileRepositorySchema flowfileRepoSchema = configSchema.getFlowfileRepositoryProperties();
|
||||
SwapSchema swapProperties = flowfileRepoSchema.getSwapProperties();
|
||||
ContentRepositorySchema contentRepoProperties = configSchema.getContentRepositoryProperties();
|
||||
ComponentStatusRepositorySchema componentStatusRepoProperties = configSchema.getComponentStatusRepositoryProperties();
|
||||
SecurityPropertiesSchema securityProperties = configSchema.getSecurityProperties();
|
||||
SensitivePropsSchema sensitiveProperties = securityProperties.getSensitiveProps();
|
||||
ProvenanceRepositorySchema provenanceRepositorySchema = configSchema.getProvenanceRepositorySchema();
|
||||
|
||||
OrderedProperties orderedProperties = new OrderedProperties();
|
||||
orderedProperties.setProperty("nifi.flow.configuration.file", "./conf/flow.xml.gz", "# Core Properties #" + System.lineSeparator());
|
||||
orderedProperties.setProperty("nifi.flow.configuration.archive.enabled", "false");
|
||||
orderedProperties.setProperty("nifi.flow.configuration.archive.dir", "./conf/archive/");
|
||||
orderedProperties.setProperty("nifi.flowcontroller.autoResumeState", "true");
|
||||
orderedProperties.setProperty("nifi.flowcontroller.graceful.shutdown.period", coreProperties.getFlowControllerGracefulShutdownPeriod());
|
||||
orderedProperties.setProperty("nifi.flowservice.writedelay.interval", coreProperties.getFlowServiceWriteDelayInterval());
|
||||
orderedProperties.setProperty("nifi.administrative.yield.duration", coreProperties.getAdministrativeYieldDuration());
|
||||
orderedProperties.setProperty("nifi.variable.registry.properties", coreProperties.getVariableRegistryProperties());
|
||||
|
||||
orderedProperties.setProperty("nifi.bored.yield.duration", coreProperties.getBoredYieldDuration(),
|
||||
"# If a component has no work to do (is \"bored\"), how long should we wait before checking again for work?");
|
||||
|
||||
orderedProperties.setProperty("nifi.authority.provider.configuration.file", "./conf/authority-providers.xml", "");
|
||||
orderedProperties.setProperty("nifi.login.identity.provider.configuration.file", "./conf/login-identity-providers.xml");
|
||||
orderedProperties.setProperty("nifi.templates.directory", "./conf/templates");
|
||||
orderedProperties.setProperty("nifi.ui.banner.text", "");
|
||||
orderedProperties.setProperty("nifi.ui.autorefresh.interval", "30 sec");
|
||||
orderedProperties.setProperty("nifi.nar.library.directory", "./lib");
|
||||
orderedProperties.setProperty("nifi.nar.working.directory", "./work/nar/");
|
||||
orderedProperties.setProperty("nifi.nar.library.autoload.directory", "./extensions");
|
||||
orderedProperties.setProperty("nifi.documentation.working.directory", "./work/docs/components");
|
||||
|
||||
orderedProperties.setProperty("nifi.state.management.configuration.file", "./conf/state-management.xml", System.lineSeparator() +
|
||||
"####################" +
|
||||
"# State Management #" +
|
||||
"####################");
|
||||
|
||||
orderedProperties.setProperty("nifi.state.management.provider.local", "local-provider", "# The ID of the local state provider");
|
||||
|
||||
orderedProperties.setProperty("nifi.database.directory", "./database_repository", System.lineSeparator() + "# H2 Settings");
|
||||
orderedProperties.setProperty("nifi.h2.url.append", ";LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE");
|
||||
orderedProperties.setProperty("nifi.flowfile.repository.implementation", String.valueOf(flowfileRepoSchema.getFlowFileRepository()),
|
||||
System.lineSeparator() + "# FlowFile Repository");
|
||||
orderedProperties.setProperty("nifi.flowfile.repository.directory", "./flowfile_repository");
|
||||
orderedProperties.setProperty("nifi.flowfile.repository.partitions", String.valueOf(flowfileRepoSchema.getPartitions()));
|
||||
orderedProperties.setProperty("nifi.flowfile.repository.checkpoint.interval", flowfileRepoSchema.getCheckpointInterval());
|
||||
orderedProperties.setProperty("nifi.flowfile.repository.always.sync", Boolean.toString(flowfileRepoSchema.getAlwaysSync()));
|
||||
|
||||
orderedProperties.setProperty("nifi.swap.manager.implementation", "org.apache.nifi.controller.FileSystemSwapManager", "");
|
||||
orderedProperties.setProperty("nifi.queue.swap.threshold", String.valueOf(swapProperties.getThreshold()));
|
||||
orderedProperties.setProperty("nifi.swap.in.period", swapProperties.getInPeriod());
|
||||
orderedProperties.setProperty("nifi.swap.in.threads", String.valueOf(swapProperties.getInThreads()));
|
||||
orderedProperties.setProperty("nifi.swap.out.period", swapProperties.getOutPeriod());
|
||||
orderedProperties.setProperty("nifi.swap.out.threads", String.valueOf(swapProperties.getOutThreads()));
|
||||
|
||||
orderedProperties.setProperty("nifi.content.repository.implementation", contentRepoProperties.getContentRepository(), System.lineSeparator() + "# Content Repository");
|
||||
orderedProperties.setProperty("nifi.content.claim.max.appendable.size", contentRepoProperties.getContentClaimMaxAppendableSize());
|
||||
orderedProperties.setProperty("nifi.content.claim.max.flow.files", String.valueOf(contentRepoProperties.getContentClaimMaxFlowFiles()));
|
||||
orderedProperties.setProperty("nifi.content.repository.archive.max.retention.period", contentRepoProperties.getContentRepoArchiveMaxRetentionPeriod());
|
||||
orderedProperties.setProperty("nifi.content.repository.archive.max.usage.percentage", contentRepoProperties.getContentRepoArchiveMaxUsagePercentage());
|
||||
orderedProperties.setProperty("nifi.content.repository.archive.enabled", Boolean.toString(contentRepoProperties.getContentRepoArchiveEnabled()));
|
||||
orderedProperties.setProperty("nifi.content.repository.directory.default", "./content_repository");
|
||||
orderedProperties.setProperty("nifi.content.repository.always.sync", Boolean.toString(contentRepoProperties.getAlwaysSync()));
|
||||
|
||||
orderedProperties.setProperty("nifi.provenance.repository.implementation", provenanceRepositorySchema.getProvenanceRepository(),
|
||||
System.lineSeparator() + "# Provenance Repository Properties");
|
||||
|
||||
orderedProperties.setProperty("nifi.provenance.repository.rollover.time", provenanceRepositorySchema.getProvenanceRepoRolloverTimeKey());
|
||||
|
||||
orderedProperties.setProperty("nifi.provenance.repository.index.shard.size", provenanceRepositorySchema.getProvenanceRepoIndexShardSize());
|
||||
orderedProperties.setProperty("nifi.provenance.repository.max.storage.size", provenanceRepositorySchema.getProvenanceRepoMaxStorageSize());
|
||||
orderedProperties.setProperty("nifi.provenance.repository.max.storage.time", provenanceRepositorySchema.getProvenanceRepoMaxStorageTime());
|
||||
|
||||
orderedProperties.setProperty("nifi.provenance.repository.buffer.size", String.valueOf(provenanceRepositorySchema.getProvenanceRepoBufferSize()),
|
||||
System.lineSeparator() + "# Volatile Provenance Respository Properties");
|
||||
|
||||
orderedProperties.setProperty("nifi.components.status.repository.implementation", "org.apache.nifi.controller.status.history.VolatileComponentStatusRepository",
|
||||
System.lineSeparator() + "# Component Status Repository");
|
||||
orderedProperties.setProperty("nifi.components.status.repository.buffer.size", String.valueOf(componentStatusRepoProperties.getBufferSize()));
|
||||
orderedProperties.setProperty("nifi.components.status.snapshot.frequency", componentStatusRepoProperties.getSnapshotFrequency());
|
||||
|
||||
orderedProperties.setProperty("nifi.web.war.directory", "./lib", System.lineSeparator() + "# web properties #");
|
||||
orderedProperties.setProperty("nifi.web.http.host", "");
|
||||
orderedProperties.setProperty("nifi.web.http.port", "8081");
|
||||
orderedProperties.setProperty("nifi.web.https.host", "");
|
||||
orderedProperties.setProperty("nifi.web.https.port", "");
|
||||
orderedProperties.setProperty("nifi.web.jetty.working.directory", "./work/jetty");
|
||||
orderedProperties.setProperty("nifi.web.jetty.threads", "200");
|
||||
|
||||
final String sensitivePropertiesKey = sensitiveProperties.getKey();
|
||||
final String notnullSensitivePropertiesKey;
|
||||
// Auto-generate the sensitive properties key if not provided, NiFi security libraries require it
|
||||
if (StringUtil.isNullOrEmpty(sensitivePropertiesKey)) {
|
||||
LOGGER.warn("Generating Random Sensitive Properties Key [{}]", NiFiProperties.SENSITIVE_PROPS_KEY);
|
||||
final SecureRandom secureRandom = new SecureRandom();
|
||||
final byte[] sensitivePropertiesKeyBinary = new byte[SENSITIVE_PROPERTIES_KEY_LENGTH];
|
||||
secureRandom.nextBytes(sensitivePropertiesKeyBinary);
|
||||
notnullSensitivePropertiesKey = KEY_ENCODER.encodeToString(sensitivePropertiesKeyBinary);
|
||||
} else {
|
||||
notnullSensitivePropertiesKey = sensitivePropertiesKey;
|
||||
}
|
||||
orderedProperties.setProperty("nifi.sensitive.props.key", notnullSensitivePropertiesKey, System.lineSeparator() + "# security properties #");
|
||||
orderedProperties.setProperty("nifi.sensitive.props.algorithm", sensitiveProperties.getAlgorithm());
|
||||
|
||||
orderedProperties.setProperty("nifi.security.keystore", securityProperties.getKeystore(), "");
|
||||
orderedProperties.setProperty("nifi.security.keystoreType", securityProperties.getKeystoreType());
|
||||
orderedProperties.setProperty("nifi.security.keystorePasswd", securityProperties.getKeystorePassword());
|
||||
orderedProperties.setProperty("nifi.security.keyPasswd", securityProperties.getKeyPassword());
|
||||
orderedProperties.setProperty("nifi.security.truststore", securityProperties.getTruststore());
|
||||
orderedProperties.setProperty("nifi.security.truststoreType", securityProperties.getTruststoreType());
|
||||
orderedProperties.setProperty("nifi.security.truststorePasswd", securityProperties.getTruststorePassword());
|
||||
orderedProperties.setProperty("nifi.security.needClientAuth", "");
|
||||
orderedProperties.setProperty("nifi.security.user.credential.cache.duration", "24 hours");
|
||||
orderedProperties.setProperty("nifi.security.user.authority.provider", "file-provider");
|
||||
orderedProperties.setProperty("nifi.security.user.login.identity.provider", "");
|
||||
orderedProperties.setProperty("nifi.security.support.new.account.requests", "");
|
||||
|
||||
orderedProperties.setProperty("nifi.security.anonymous.authorities", "", "# Valid Authorities include: ROLE_MONITOR,ROLE_DFM,ROLE_ADMIN,ROLE_PROVENANCE,ROLE_NIFI");
|
||||
orderedProperties.setProperty("nifi.security.ocsp.responder.url", "");
|
||||
orderedProperties.setProperty("nifi.security.ocsp.responder.certificate", "");
|
||||
|
||||
orderedProperties.setProperty("nifi.cluster.is.node", "false", System.lineSeparator() + System.lineSeparator() + "# cluster node properties (only configure for cluster nodes) #");
|
||||
orderedProperties.setProperty("nifi.cluster.is.manager", "false", System.lineSeparator() + "# cluster manager properties (only configure for cluster manager) #");
|
||||
|
||||
for (Map.Entry<String, String> entry : configSchema.getNifiPropertiesOverrides().entrySet()) {
|
||||
orderedProperties.setProperty(entry.getKey(), entry.getValue());
|
||||
}
|
||||
|
||||
orderedProperties.store(outputStream, PROPERTIES_FILE_APACHE_2_0_LICENSE);
|
||||
} catch (NullPointerException e) {
|
||||
throw new ConfigurationChangeException("Failed to parse the config YAML while creating the nifi.properties", e);
|
||||
} finally {
|
||||
outputStream.close();
|
||||
}
|
||||
}
|
||||
|
||||
protected static DOMSource createFlowXml(ConfigSchema configSchema) throws ConfigTransformerException {
|
||||
try {
|
||||
// create a new, empty document
|
||||
final StandardDocumentProvider documentProvider = new StandardDocumentProvider();
|
||||
documentProvider.setNamespaceAware(true);
|
||||
final Document doc = documentProvider.newDocument();
|
||||
|
||||
// populate document with controller state
|
||||
final Element rootNode = doc.createElement("flowController");
|
||||
doc.appendChild(rootNode);
|
||||
CorePropertiesSchema coreProperties = configSchema.getCoreProperties();
|
||||
addTextElement(rootNode, "maxTimerDrivenThreadCount", String.valueOf(coreProperties.getMaxConcurrentThreads()));
|
||||
addTextElement(rootNode, "maxEventDrivenThreadCount", String.valueOf(coreProperties.getMaxConcurrentThreads()));
|
||||
|
||||
FlowControllerSchema flowControllerProperties = configSchema.getFlowControllerProperties();
|
||||
|
||||
final Element element = doc.createElement("rootGroup");
|
||||
rootNode.appendChild(element);
|
||||
|
||||
ProcessGroupSchema processGroupSchema = configSchema.getProcessGroupSchema();
|
||||
processGroupSchema.setId(ROOT_GROUP);
|
||||
processGroupSchema.setName(flowControllerProperties.getName());
|
||||
processGroupSchema.setComment(flowControllerProperties.getComment());
|
||||
|
||||
addProcessGroup(doc, element, processGroupSchema, new ParentGroupIdResolver(processGroupSchema));
|
||||
|
||||
SecurityPropertiesSchema securityProperties = configSchema.getSecurityProperties();
|
||||
boolean useSSL = securityProperties.useSSL();
|
||||
if (useSSL) {
|
||||
Element controllerServicesNode = doc.getElementById("controllerServices");
|
||||
if (controllerServicesNode == null) {
|
||||
controllerServicesNode = doc.createElement("controllerServices");
|
||||
}
|
||||
|
||||
rootNode.appendChild(controllerServicesNode);
|
||||
addSSLControllerService(controllerServicesNode, securityProperties);
|
||||
}
|
||||
|
||||
List<ReportingSchema> reportingTasks = configSchema.getReportingTasksSchema();
|
||||
ProvenanceReportingSchema provenanceProperties = configSchema.getProvenanceReportingProperties();
|
||||
if (provenanceProperties != null) {
|
||||
provenanceProperties.setSSL(useSSL);
|
||||
ReportingSchema provenance = provenanceProperties.convert();
|
||||
provenance.setId("Provenance-Reporting");
|
||||
provenance.setName("Site-To-Site-Provenance-Reporting");
|
||||
reportingTasks.add(provenance);
|
||||
}
|
||||
if (reportingTasks != null) {
|
||||
final Element reportingTasksNode = doc.createElement("reportingTasks");
|
||||
rootNode.appendChild(reportingTasksNode);
|
||||
for (ReportingSchema task : reportingTasks) {
|
||||
addReportingTask(reportingTasksNode, task);
|
||||
}
|
||||
}
|
||||
|
||||
return new DOMSource(doc);
|
||||
} catch (final ProcessingException | DOMException | IllegalArgumentException e) {
|
||||
throw new ConfigTransformerException(e);
|
||||
} catch (Exception e) {
|
||||
throw new ConfigTransformerException("Failed to parse the config YAML while writing the top level of the flow xml", e);
|
||||
}
|
||||
}
|
||||
|
||||
protected static void addSSLControllerService(final Element element, SecurityPropertiesSchema securityProperties) throws ConfigurationChangeException {
|
||||
try {
|
||||
final Element serviceElement = element.getOwnerDocument().createElement("controllerService");
|
||||
addTextElement(serviceElement, "id", "SSL-Context-Service");
|
||||
addTextElement(serviceElement, "name", "SSL-Context-Service");
|
||||
addTextElement(serviceElement, "comment", "");
|
||||
addTextElement(serviceElement, "class", "org.apache.nifi.ssl.StandardSSLContextService");
|
||||
|
||||
addTextElement(serviceElement, "enabled", "true");
|
||||
|
||||
Map<String, Object> attributes = new HashMap<>();
|
||||
attributes.put("Keystore Filename", securityProperties.getKeystore());
|
||||
attributes.put("Keystore Type", securityProperties.getKeystoreType());
|
||||
attributes.put("Keystore Password", securityProperties.getKeystorePassword());
|
||||
attributes.put("Truststore Filename", securityProperties.getTruststore());
|
||||
attributes.put("Truststore Type", securityProperties.getTruststoreType());
|
||||
attributes.put("Truststore Password", securityProperties.getTruststorePassword());
|
||||
attributes.put("SSL Protocol", securityProperties.getSslProtocol());
|
||||
|
||||
addConfiguration(serviceElement, attributes);
|
||||
|
||||
element.appendChild(serviceElement);
|
||||
} catch (Exception e) {
|
||||
throw new ConfigurationChangeException("Failed to parse the config YAML while trying to create an SSL Controller Service", e);
|
||||
}
|
||||
}
|
||||
|
||||
protected static void addControllerService(final Element element, ControllerServiceSchema controllerServiceSchema) throws ConfigurationChangeException {
|
||||
try {
|
||||
final Element serviceElement = element.getOwnerDocument().createElement("controllerService");
|
||||
addTextElement(serviceElement, "id", controllerServiceSchema.getId());
|
||||
addTextElement(serviceElement, "name", controllerServiceSchema.getName());
|
||||
addTextElement(serviceElement, "comment", "");
|
||||
addTextElement(serviceElement, "class", controllerServiceSchema.getServiceClass());
|
||||
|
||||
addTextElement(serviceElement, "enabled", "true");
|
||||
|
||||
Map<String, Object> attributes = controllerServiceSchema.getProperties();
|
||||
|
||||
addConfiguration(serviceElement, attributes);
|
||||
|
||||
String annotationData = controllerServiceSchema.getAnnotationData();
|
||||
if (annotationData != null && !annotationData.isEmpty()) {
|
||||
addTextElement(element, "annotationData", annotationData);
|
||||
}
|
||||
|
||||
element.appendChild(serviceElement);
|
||||
} catch (Exception e) {
|
||||
throw new ConfigurationChangeException("Failed to parse the config YAML while trying to create an SSL Controller Service", e);
|
||||
}
|
||||
}
|
||||
|
||||
protected static void addProcessGroup(Document doc, Element element, ProcessGroupSchema processGroupSchema, ParentGroupIdResolver parentGroupIdResolver) throws ConfigurationChangeException {
|
||||
try {
|
||||
String processGroupId = processGroupSchema.getId();
|
||||
addTextElement(element, "id", processGroupId);
|
||||
addTextElement(element, "name", processGroupSchema.getName());
|
||||
addPosition(element);
|
||||
addTextElement(element, "comment", processGroupSchema.getComment());
|
||||
|
||||
for (ProcessorSchema processorConfig : processGroupSchema.getProcessors()) {
|
||||
addProcessor(element, processorConfig);
|
||||
}
|
||||
|
||||
for (PortSchema portSchema : processGroupSchema.getInputPortSchemas()) {
|
||||
addPort(doc, element, portSchema, "inputPort");
|
||||
}
|
||||
|
||||
for (PortSchema portSchema : processGroupSchema.getOutputPortSchemas()) {
|
||||
addPort(doc, element, portSchema, "outputPort");
|
||||
}
|
||||
|
||||
for (FunnelSchema funnelSchema : processGroupSchema.getFunnels()) {
|
||||
addFunnel(element, funnelSchema);
|
||||
}
|
||||
|
||||
for (ProcessGroupSchema child : processGroupSchema.getProcessGroupSchemas()) {
|
||||
Element processGroups = doc.createElement("processGroup");
|
||||
element.appendChild(processGroups);
|
||||
addProcessGroup(doc, processGroups, child, parentGroupIdResolver);
|
||||
}
|
||||
|
||||
for (RemoteProcessGroupSchema remoteProcessGroupSchema : processGroupSchema.getRemoteProcessGroups()) {
|
||||
addRemoteProcessGroup(element, remoteProcessGroupSchema);
|
||||
}
|
||||
|
||||
for (ConnectionSchema connectionConfig : processGroupSchema.getConnections()) {
|
||||
addConnection(element, connectionConfig, parentGroupIdResolver);
|
||||
}
|
||||
|
||||
for (ControllerServiceSchema controllerServiceSchema : processGroupSchema.getControllerServices()) {
|
||||
addControllerService(element, controllerServiceSchema);
|
||||
}
|
||||
} catch (ConfigurationChangeException e) {
|
||||
throw e;
|
||||
} catch (Exception e) {
|
||||
throw new ConfigurationChangeException("Failed to parse the config YAML while trying to creating the root Process Group", e);
|
||||
}
|
||||
}
|
||||
|
||||
protected static void addPort(Document doc, Element parentElement, PortSchema portSchema, String tag) {
|
||||
Element element = doc.createElement(tag);
|
||||
parentElement.appendChild(element);
|
||||
|
||||
addTextElement(element, "id", portSchema.getId());
|
||||
addTextElement(element, "name", portSchema.getName());
|
||||
|
||||
addPosition(element);
|
||||
addTextElement(element, "comments", null);
|
||||
|
||||
addTextElement(element, "scheduledState", "RUNNING");
|
||||
}
|
||||
|
||||
protected static void addProcessor(final Element parentElement, ProcessorSchema processorConfig) throws ConfigurationChangeException {
|
||||
try {
|
||||
final Document doc = parentElement.getOwnerDocument();
|
||||
final Element element = doc.createElement("processor");
|
||||
parentElement.appendChild(element);
|
||||
|
||||
addTextElement(element, "id", processorConfig.getId());
|
||||
addTextElement(element, "name", processorConfig.getName());
|
||||
|
||||
addPosition(element);
|
||||
addStyle(element);
|
||||
|
||||
addTextElement(element, "comment", "");
|
||||
addTextElement(element, "class", processorConfig.getProcessorClass());
|
||||
addTextElement(element, "maxConcurrentTasks", String.valueOf(processorConfig.getMaxConcurrentTasks()));
|
||||
addTextElement(element, "schedulingPeriod", processorConfig.getSchedulingPeriod());
|
||||
addTextElement(element, "penalizationPeriod", processorConfig.getPenalizationPeriod());
|
||||
addTextElement(element, "yieldPeriod", processorConfig.getYieldPeriod());
|
||||
addTextElement(element, "bulletinLevel", "WARN");
|
||||
addTextElement(element, "lossTolerant", "false");
|
||||
addTextElement(element, "scheduledState", "RUNNING");
|
||||
addTextElement(element, "schedulingStrategy", processorConfig.getSchedulingStrategy());
|
||||
addTextElement(element, "runDurationNanos", String.valueOf(processorConfig.getRunDurationNanos()));
|
||||
|
||||
String annotationData = processorConfig.getAnnotationData();
|
||||
if (annotationData != null && !annotationData.isEmpty()) {
|
||||
addTextElement(element, "annotationData", annotationData);
|
||||
}
|
||||
|
||||
addConfiguration(element, processorConfig.getProperties());
|
||||
|
||||
Collection<String> autoTerminatedRelationships = processorConfig.getAutoTerminatedRelationshipsList();
|
||||
if (autoTerminatedRelationships != null) {
|
||||
for (String rel : autoTerminatedRelationships) {
|
||||
addTextElement(element, "autoTerminatedRelationship", rel);
|
||||
}
|
||||
}
|
||||
} catch (Exception e) {
|
||||
throw new ConfigurationChangeException("Failed to parse the config YAML while trying to add a Processor", e);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
protected static void addFunnel(final Element parentElement, FunnelSchema funnelSchema) {
|
||||
Document document = parentElement.getOwnerDocument();
|
||||
Element element = document.createElement("funnel");
|
||||
parentElement.appendChild(element);
|
||||
|
||||
addTextElement(element, "id", funnelSchema.getId());
|
||||
|
||||
addPosition(element);
|
||||
}
|
||||
|
||||
protected static void addReportingTask(final Element parentElement, ReportingSchema reportingSchema) throws ConfigurationChangeException {
|
||||
try {
|
||||
final Document doc = parentElement.getOwnerDocument();
|
||||
final Element element = doc.createElement("reportingTask");
|
||||
parentElement.appendChild(element);
|
||||
|
||||
addTextElement(element, "id", reportingSchema.getId());
|
||||
addTextElement(element, "name", reportingSchema.getName());
|
||||
addTextElement(element, "comment", reportingSchema.getComment());
|
||||
addTextElement(element, "class", reportingSchema.getReportingClass());
|
||||
addTextElement(element, "schedulingPeriod", reportingSchema.getSchedulingPeriod());
|
||||
addTextElement(element, "scheduledState", "RUNNING");
|
||||
addTextElement(element, "schedulingStrategy", reportingSchema.getSchedulingStrategy());
|
||||
|
||||
addConfiguration(element, reportingSchema.getProperties());
|
||||
|
||||
parentElement.appendChild(element);
|
||||
} catch (Exception e) {
|
||||
throw new ConfigurationChangeException("Failed to parse the config YAML while trying to add the Provenance Reporting Task", e);
|
||||
}
|
||||
}
|
||||
|
||||
protected static void addConfiguration(final Element element, Map<String, Object> elementConfig) {
|
||||
final Document doc = element.getOwnerDocument();
|
||||
if (elementConfig == null) {
|
||||
return;
|
||||
}
|
||||
for (final Map.Entry<String, Object> entry : elementConfig.entrySet()) {
|
||||
|
||||
final Element propElement = doc.createElement("property");
|
||||
addTextElement(propElement, "name", entry.getKey());
|
||||
if (entry.getValue() != null) {
|
||||
addTextElement(propElement, "value", entry.getValue().toString());
|
||||
}
|
||||
|
||||
element.appendChild(propElement);
|
||||
}
|
||||
}
|
||||
|
||||
protected static void addStyle(final Element parentElement) {
|
||||
final Element element = parentElement.getOwnerDocument().createElement("styles");
|
||||
parentElement.appendChild(element);
|
||||
}
|
||||
|
||||
protected static void addRemoteProcessGroup(final Element parentElement, RemoteProcessGroupSchema remoteProcessGroupProperties) throws ConfigurationChangeException {
|
||||
try {
|
||||
final Document doc = parentElement.getOwnerDocument();
|
||||
final Element element = doc.createElement("remoteProcessGroup");
|
||||
parentElement.appendChild(element);
|
||||
addTextElement(element, "id", remoteProcessGroupProperties.getId());
|
||||
addTextElement(element, "name", remoteProcessGroupProperties.getName());
|
||||
addPosition(element);
|
||||
addTextElement(element, "comment", remoteProcessGroupProperties.getComment());
|
||||
// In the case we have multiple urls, select the first
|
||||
addTextElement(element, "url", Arrays.asList(remoteProcessGroupProperties.getUrls().split(",")).get(0));
|
||||
addTextElement(element, "urls", remoteProcessGroupProperties.getUrls());
|
||||
addTextElement(element, "timeout", remoteProcessGroupProperties.getTimeout());
|
||||
addTextElement(element, "yieldPeriod", remoteProcessGroupProperties.getYieldPeriod());
|
||||
addTextElement(element, "transmitting", "true");
|
||||
addTextElement(element, "transportProtocol", remoteProcessGroupProperties.getTransportProtocol());
|
||||
addTextElement(element, "proxyHost", remoteProcessGroupProperties.getProxyHost());
|
||||
if (remoteProcessGroupProperties.getProxyPort() != null) {
|
||||
addTextElement(element, "proxyPort", Integer.toString(remoteProcessGroupProperties.getProxyPort()));
|
||||
}
|
||||
addTextElement(element, "proxyUser", remoteProcessGroupProperties.getProxyUser());
|
||||
if (!StringUtils.isEmpty(remoteProcessGroupProperties.getProxyPassword())) {
|
||||
addTextElement(element, "proxyPassword", remoteProcessGroupProperties.getProxyPassword());
|
||||
}
|
||||
|
||||
List<RemotePortSchema> remoteInputPorts = remoteProcessGroupProperties.getInputPorts();
|
||||
for (RemotePortSchema remoteInputPortSchema : remoteInputPorts) {
|
||||
addRemoteGroupPort(element, remoteInputPortSchema, "inputPort");
|
||||
}
|
||||
|
||||
List<RemotePortSchema> remoteOutputPorts = remoteProcessGroupProperties.getOutputPorts();
|
||||
for (RemotePortSchema remoteOutputPortSchema : remoteOutputPorts) {
|
||||
addRemoteGroupPort(element, remoteOutputPortSchema, "outputPort");
|
||||
}
|
||||
addTextElement(element, "networkInterface", remoteProcessGroupProperties.getLocalNetworkInterface());
|
||||
|
||||
parentElement.appendChild(element);
|
||||
} catch (Exception e) {
|
||||
throw new ConfigurationChangeException("Failed to parse the config YAML while trying to add the Remote Process Group", e);
|
||||
}
|
||||
}
|
||||
|
||||
protected static void addRemoteGroupPort(final Element parentElement, RemotePortSchema inputPort, String tagName) throws ConfigurationChangeException {
|
||||
try {
|
||||
final Document doc = parentElement.getOwnerDocument();
|
||||
final Element element = doc.createElement(tagName);
|
||||
parentElement.appendChild(element);
|
||||
addTextElement(element, "id", inputPort.getId());
|
||||
addTextElement(element, "name", inputPort.getName());
|
||||
addPosition(element);
|
||||
addTextElement(element, "comments", inputPort.getComment());
|
||||
addTextElement(element, "scheduledState", "RUNNING");
|
||||
addTextElement(element, "maxConcurrentTasks", String.valueOf(inputPort.getMax_concurrent_tasks()));
|
||||
addTextElement(element, "useCompression", String.valueOf(inputPort.getUseCompression()));
|
||||
|
||||
parentElement.appendChild(element);
|
||||
} catch (Exception e) {
|
||||
throw new ConfigurationChangeException("Failed to parse the config YAML while trying to add the input port of the Remote Process Group", e);
|
||||
}
|
||||
}
|
||||
|
||||
protected static void addConnection(final Element parentElement, ConnectionSchema connectionProperties, ParentGroupIdResolver parentGroupIdResolver) throws ConfigurationChangeException {
|
||||
try {
|
||||
final Document doc = parentElement.getOwnerDocument();
|
||||
final Element element = doc.createElement("connection");
|
||||
parentElement.appendChild(element);
|
||||
|
||||
addTextElement(element, "id", connectionProperties.getId());
|
||||
addTextElement(element, "name", connectionProperties.getName());
|
||||
|
||||
final Element bendPointsElement = doc.createElement("bendPoints");
|
||||
element.appendChild(bendPointsElement);
|
||||
|
||||
addTextElement(element, "labelIndex", "1");
|
||||
addTextElement(element, "zIndex", "0");
|
||||
|
||||
addConnectionSourceOrDestination(element, "source", connectionProperties.getSourceId(), parentGroupIdResolver);
|
||||
addConnectionSourceOrDestination(element, "destination", connectionProperties.getDestinationId(), parentGroupIdResolver);
|
||||
|
||||
List<String> sourceRelationshipNames = connectionProperties.getSourceRelationshipNames();
|
||||
if (sourceRelationshipNames.isEmpty()) {
|
||||
addTextElement(element, "relationship", null);
|
||||
} else {
|
||||
for (String relationshipName : sourceRelationshipNames) {
|
||||
addTextElement(element, "relationship", relationshipName);
|
||||
}
|
||||
}
|
||||
|
||||
addTextElement(element, "maxWorkQueueSize", String.valueOf(connectionProperties.getMaxWorkQueueSize()));
|
||||
addTextElement(element, "maxWorkQueueDataSize", connectionProperties.getMaxWorkQueueDataSize());
|
||||
|
||||
addTextElement(element, "flowFileExpiration", connectionProperties.getFlowfileExpiration());
|
||||
addTextElementIfNotNullOrEmpty(element, "queuePrioritizerClass", connectionProperties.getQueuePrioritizerClass());
|
||||
|
||||
parentElement.appendChild(element);
|
||||
} catch (Exception e) {
|
||||
throw new ConfigurationChangeException("Failed to parse the config YAML while trying to add the connection from the Processor to the input port of the Remote Process Group", e);
|
||||
}
|
||||
}
|
||||
|
||||
protected static void addConnectionSourceOrDestination(Element element, String sourceOrDestination, String id, ParentGroupIdResolver parentGroupIdResolver) {
|
||||
String idTag = sourceOrDestination + "Id";
|
||||
String groupIdTag = sourceOrDestination + "GroupId";
|
||||
String typeTag = sourceOrDestination + "Type";
|
||||
|
||||
String parentId;
|
||||
String type;
|
||||
|
||||
if ((parentId = parentGroupIdResolver.getRemoteInputPortParentId(id)) != null) {
|
||||
type = "REMOTE_INPUT_PORT";
|
||||
} else if ((parentId = parentGroupIdResolver.getRemoteOutputPortParentId(id)) != null) {
|
||||
type = "REMOTE_OUTPUT_PORT";
|
||||
} else if ((parentId = parentGroupIdResolver.getInputPortParentId(id)) != null) {
|
||||
type = "INPUT_PORT";
|
||||
} else if ((parentId = parentGroupIdResolver.getOutputPortParentId(id)) != null) {
|
||||
type = "OUTPUT_PORT";
|
||||
} else if ((parentId = parentGroupIdResolver.getFunnelParentId(id)) != null) {
|
||||
type = "FUNNEL";
|
||||
} else {
|
||||
parentId = parentGroupIdResolver.getProcessorParentId(id);
|
||||
type = "PROCESSOR";
|
||||
}
|
||||
|
||||
addTextElement(element, idTag, id);
|
||||
if (parentId != null) {
|
||||
addTextElement(element, groupIdTag, parentId);
|
||||
}
|
||||
addTextElement(element, typeTag, type);
|
||||
}
|
||||
|
||||
protected static void addPosition(final Element parentElement) {
|
||||
final Element element = parentElement.getOwnerDocument().createElement("position");
|
||||
element.setAttribute("x", "0");
|
||||
element.setAttribute("y", "0");
|
||||
parentElement.appendChild(element);
|
||||
}
|
||||
|
||||
protected static void addTextElementIfNotNullOrEmpty(final Element element, final String name, final String value) {
|
||||
StringUtil.doIfNotNullOrEmpty(value, s -> addTextElement(element, name, value));
|
||||
}
|
||||
|
||||
protected static void addTextElement(final Element element, final String name, final String value) {
|
||||
final Document doc = element.getOwnerDocument();
|
||||
final Element toAdd = doc.createElement(name);
|
||||
toAdd.setTextContent(value);
|
||||
element.appendChild(toAdd);
|
||||
}
|
||||
|
||||
public static ByteBuffer overrideNonFlowSectionsFromOriginalSchema(byte[] newSchema, ByteBuffer currentConfigScheme, Properties bootstrapProperties)
|
||||
throws InvalidConfigurationException {
|
||||
try {
|
||||
boolean overrideCoreProperties = ConfigTransformer.overrideCoreProperties(bootstrapProperties);
|
||||
boolean overrideSecurityProperties = ConfigTransformer.overrideSecurityProperties(bootstrapProperties);
|
||||
if (overrideCoreProperties && overrideSecurityProperties) {
|
||||
return ByteBuffer.wrap(newSchema);
|
||||
} else {
|
||||
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(currentConfigScheme)));
|
||||
ConfigSchema configSchemaOld = ConfigTransformer.throwIfInvalid(schemaOld.convert());
|
||||
|
||||
configSchemaNew.setNifiPropertiesOverrides(configSchemaOld.getNifiPropertiesOverrides());
|
||||
|
||||
if (!overrideCoreProperties) {
|
||||
LOGGER.debug("Preserving previous core properties...");
|
||||
configSchemaNew.setCoreProperties(configSchemaOld.getCoreProperties());
|
||||
}
|
||||
|
||||
if (!overrideSecurityProperties) {
|
||||
LOGGER.debug("Preserving previous security properties...");
|
||||
configSchemaNew.setSecurityProperties(configSchemaOld.getSecurityProperties());
|
||||
}
|
||||
|
||||
StringWriter writer = new StringWriter();
|
||||
SchemaLoader.toYaml(configSchemaNew, writer);
|
||||
return ByteBuffer.wrap(writer.toString().getBytes()).asReadOnlyBuffer();
|
||||
}
|
||||
} catch (Exception e) {
|
||||
throw new InvalidConfigurationException("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");
|
||||
return Boolean.parseBoolean(overrideSecurityProperties);
|
||||
}
|
||||
|
||||
private static boolean overrideCoreProperties(Properties properties) {
|
||||
String overrideCoreProps = (String) properties.getOrDefault(OVERRIDE_CORE_PROPERTIES_KEY, "false");
|
||||
return Boolean.parseBoolean(overrideCoreProps);
|
||||
}
|
||||
|
||||
public static final String PROPERTIES_FILE_APACHE_2_0_LICENSE =
|
||||
" Licensed to the Apache Software Foundation (ASF) under one or more\n" +
|
||||
"# contributor license agreements. See the NOTICE file distributed with\n" +
|
||||
"# this work for additional information regarding copyright ownership.\n" +
|
||||
"# The ASF licenses this file to You under the Apache License, Version 2.0\n" +
|
||||
"# (the \"License\"); you may not use this file except in compliance with\n" +
|
||||
"# the License. You may obtain a copy of the License at\n" +
|
||||
"#\n" +
|
||||
"# http://www.apache.org/licenses/LICENSE-2.0\n" +
|
||||
"#\n" +
|
||||
"# Unless required by applicable law or agreed to in writing, software\n" +
|
||||
"# distributed under the License is distributed on an \"AS IS\" BASIS,\n" +
|
||||
"# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n" +
|
||||
"# See the License for the specific language governing permissions and\n" +
|
||||
"# limitations under the License.\n" +
|
||||
"\n";
|
||||
|
||||
}
|
|
@ -1,32 +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.util;
|
||||
|
||||
public class ConfigTransformerException extends Exception {
|
||||
public ConfigTransformerException(String message) {
|
||||
super(message);
|
||||
}
|
||||
|
||||
public ConfigTransformerException(Throwable cause) {
|
||||
super(cause);
|
||||
}
|
||||
|
||||
public ConfigTransformerException(String message, Throwable cause) {
|
||||
super(message, cause);
|
||||
}
|
||||
}
|
|
@ -34,6 +34,7 @@ import java.util.Map;
|
|||
import java.util.Properties;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
|
||||
public class OrderedProperties extends Properties {
|
||||
private final Map<String, String> textBeforeMap = new HashMap<>();
|
||||
|
@ -74,7 +75,7 @@ public class OrderedProperties extends Properties {
|
|||
int equalsIndex = line.indexOf('=');
|
||||
if (equalsIndex != -1) {
|
||||
String textBefore = textBeforeMap.get(line.substring(0, equalsIndex));
|
||||
if (textBefore != null) {
|
||||
if (StringUtils.isNotBlank(textBefore)) {
|
||||
bufferedWriter.write(textBefore);
|
||||
bufferedWriter.newLine();
|
||||
}
|
||||
|
|
|
@ -1,101 +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.util;
|
||||
|
||||
import org.apache.nifi.minifi.commons.schema.ProcessGroupSchema;
|
||||
import org.apache.nifi.minifi.commons.schema.RemotePortSchema;
|
||||
import org.apache.nifi.minifi.commons.schema.RemoteProcessGroupSchema;
|
||||
import org.apache.nifi.minifi.commons.schema.common.BaseSchemaWithId;
|
||||
|
||||
import java.util.Collection;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.function.Function;
|
||||
|
||||
public class ParentGroupIdResolver {
|
||||
private final Map<String, String> processorIdToParentIdMap;
|
||||
private final Map<String, String> inputPortIdToParentIdMap;
|
||||
private final Map<String, String> outputPortIdToParentIdMap;
|
||||
private final Map<String, String> funnelIdToParentIdMap;
|
||||
private final Map<String, String> remoteInputPortIdToParentIdMap;
|
||||
private final Map<String, String> remoteOutputPortIdToParentIdMap;
|
||||
|
||||
public ParentGroupIdResolver(ProcessGroupSchema processGroupSchema) {
|
||||
this.processorIdToParentIdMap = getParentIdMap(processGroupSchema, ProcessGroupSchema::getProcessors);
|
||||
this.inputPortIdToParentIdMap = getParentIdMap(processGroupSchema, ProcessGroupSchema::getInputPortSchemas);
|
||||
this.outputPortIdToParentIdMap = getParentIdMap(processGroupSchema, ProcessGroupSchema::getOutputPortSchemas);
|
||||
this.funnelIdToParentIdMap = getParentIdMap(processGroupSchema, ProcessGroupSchema::getFunnels);
|
||||
this.remoteInputPortIdToParentIdMap = getRemotePortParentIdMap(processGroupSchema, RemoteProcessGroupSchema::getInputPorts);
|
||||
this.remoteOutputPortIdToParentIdMap = getRemotePortParentIdMap(processGroupSchema, RemoteProcessGroupSchema::getOutputPorts);
|
||||
}
|
||||
|
||||
protected static Map<String, String> getParentIdMap(ProcessGroupSchema processGroupSchema, Function<ProcessGroupSchema, Collection<? extends BaseSchemaWithId>> schemaAccessor) {
|
||||
Map<String, String> map = new HashMap<>();
|
||||
getParentIdMap(processGroupSchema, map, schemaAccessor);
|
||||
return map;
|
||||
}
|
||||
|
||||
protected static void getParentIdMap(ProcessGroupSchema processGroupSchema, Map<String, String> output, Function<ProcessGroupSchema,
|
||||
Collection<? extends BaseSchemaWithId>> schemaAccessor) {
|
||||
schemaAccessor.apply(processGroupSchema).forEach(p -> output.put(p.getId(), processGroupSchema.getId()));
|
||||
processGroupSchema.getProcessGroupSchemas().forEach(p -> getParentIdMap(p, output, schemaAccessor));
|
||||
}
|
||||
|
||||
protected static Map<String, String> getRemotePortParentIdMap(ProcessGroupSchema processGroupSchema, Function<RemoteProcessGroupSchema, List<RemotePortSchema>> getPortsFunction) {
|
||||
Map<String, String> result = new HashMap<>();
|
||||
getRemotePortParentIdMap(processGroupSchema, result, getPortsFunction);
|
||||
return result;
|
||||
}
|
||||
|
||||
protected static void getRemotePortParentIdMap(ProcessGroupSchema processGroupSchema, Map<String, String> output, Function<RemoteProcessGroupSchema,
|
||||
List<RemotePortSchema>> getPortsFunction) {
|
||||
for (RemoteProcessGroupSchema remoteProcessGroupSchema : processGroupSchema.getRemoteProcessGroups()) {
|
||||
for (RemotePortSchema remotePortSchema : getPortsFunction.apply(remoteProcessGroupSchema)) {
|
||||
output.put(remotePortSchema.getId(), remoteProcessGroupSchema.getId());
|
||||
}
|
||||
}
|
||||
processGroupSchema.getProcessGroupSchemas().forEach(p -> getRemotePortParentIdMap(p, output, getPortsFunction));
|
||||
}
|
||||
|
||||
public String getRemoteInputPortParentId(String id) {
|
||||
return remoteInputPortIdToParentIdMap.get(id);
|
||||
}
|
||||
|
||||
public String getRemoteOutputPortParentId(String id) {
|
||||
return remoteOutputPortIdToParentIdMap.get(id);
|
||||
}
|
||||
|
||||
public String getInputPortParentId(String id) {
|
||||
return inputPortIdToParentIdMap.get(id);
|
||||
}
|
||||
|
||||
public String getOutputPortParentId(String id) {
|
||||
return outputPortIdToParentIdMap.get(id);
|
||||
}
|
||||
|
||||
public String getProcessorParentId(String id) {
|
||||
return processorIdToParentIdMap.get(id);
|
||||
}
|
||||
|
||||
public String getFunnelParentId(String id) {
|
||||
return funnelIdToParentIdMap.get(id);
|
||||
}
|
||||
}
|
|
@ -1,129 +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.BootstrapTransformer;
|
||||
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 org.junit.jupiter.api.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.util.Optional;
|
||||
import java.util.Properties;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertNotNull;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
public class RunMiNiFiTest {
|
||||
|
||||
@Test
|
||||
public void buildSecurityPropertiesNotDefined() throws Exception {
|
||||
final Properties bootstrapProperties = getTestBootstrapProperties("bootstrap-ssl-ctx/bootstrap.conf.default");
|
||||
final Optional<SecurityPropertiesSchema> securityPropsOptional = BootstrapTransformer.buildSecurityPropertiesFromBootstrap(bootstrapProperties);
|
||||
assertFalse(securityPropsOptional.isPresent());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void buildSecurityPropertiesDefined() throws Exception {
|
||||
final Properties bootstrapProperties = getTestBootstrapProperties("bootstrap-ssl-ctx/bootstrap.conf.configured");
|
||||
final Optional<SecurityPropertiesSchema> securityPropsOptional = BootstrapTransformer.buildSecurityPropertiesFromBootstrap(bootstrapProperties);
|
||||
assertTrue(securityPropsOptional.isPresent());
|
||||
|
||||
final SecurityPropertiesSchema securityPropertiesSchema = securityPropsOptional.get();
|
||||
assertEquals("/my/test/keystore.jks", securityPropertiesSchema.getKeystore());
|
||||
assertEquals("JKS", securityPropertiesSchema.getKeystoreType());
|
||||
assertEquals("mykeystorepassword", securityPropertiesSchema.getKeystorePassword());
|
||||
assertEquals("mykeypassword", securityPropertiesSchema.getKeyPassword());
|
||||
|
||||
assertEquals("/my/test/truststore.jks", securityPropertiesSchema.getTruststore());
|
||||
assertEquals("JKS", securityPropertiesSchema.getTruststoreType());
|
||||
assertEquals("mytruststorepassword", securityPropertiesSchema.getTruststorePassword());
|
||||
|
||||
assertEquals("TLS", securityPropertiesSchema.getSslProtocol());
|
||||
|
||||
final SensitivePropsSchema sensitiveProps = securityPropertiesSchema.getSensitiveProps();
|
||||
assertNotNull(sensitiveProps);
|
||||
|
||||
assertEquals("sensitivepropskey", sensitiveProps.getKey());
|
||||
assertEquals("algo", sensitiveProps.getAlgorithm());
|
||||
|
||||
|
||||
assertTrue(securityPropertiesSchema.isValid());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void buildSecurityPropertiesDefinedButInvalid() throws Exception {
|
||||
final Properties bootstrapProperties = getTestBootstrapProperties("bootstrap-ssl-ctx/bootstrap.conf.configured.invalid");
|
||||
final Optional<SecurityPropertiesSchema> securityPropsOptional = BootstrapTransformer.buildSecurityPropertiesFromBootstrap(bootstrapProperties);
|
||||
assertTrue(securityPropsOptional.isPresent());
|
||||
|
||||
final SecurityPropertiesSchema securityPropertiesSchema = securityPropsOptional.get();
|
||||
assertEquals("/my/test/keystore.jks", securityPropertiesSchema.getKeystore());
|
||||
assertEquals("NOTAKEYSTORETYPE", securityPropertiesSchema.getKeystoreType());
|
||||
assertEquals("mykeystorepassword", securityPropertiesSchema.getKeystorePassword());
|
||||
assertEquals("mykeypassword", securityPropertiesSchema.getKeyPassword());
|
||||
|
||||
assertEquals("/my/test/truststore.jks", securityPropertiesSchema.getTruststore());
|
||||
assertEquals("JKS", securityPropertiesSchema.getTruststoreType());
|
||||
assertEquals("mytruststorepassword", securityPropertiesSchema.getTruststorePassword());
|
||||
|
||||
final SensitivePropsSchema sensitiveProps = securityPropertiesSchema.getSensitiveProps();
|
||||
assertNotNull(sensitiveProps);
|
||||
|
||||
assertEquals("sensitivepropskey", sensitiveProps.getKey());
|
||||
assertEquals("algo", sensitiveProps.getAlgorithm());
|
||||
|
||||
assertFalse(securityPropertiesSchema.isValid());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void buildProvenanceReportingNotDefined() throws Exception {
|
||||
final Properties bootstrapProperties = getTestBootstrapProperties("bootstrap-provenance-reporting/bootstrap.conf.default");
|
||||
final Optional<ProvenanceReportingSchema> provenanceReportingPropsOptional = BootstrapTransformer.buildProvenanceReportingPropertiesFromBootstrap(bootstrapProperties);
|
||||
assertFalse(provenanceReportingPropsOptional.isPresent());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void buildProvenanceReportingDefined() throws Exception {
|
||||
final Properties bootstrapProperties = getTestBootstrapProperties("bootstrap-provenance-reporting/bootstrap.conf.configured");
|
||||
final Optional<ProvenanceReportingSchema> provenanceReportingPropsOptional = BootstrapTransformer.buildProvenanceReportingPropertiesFromBootstrap(bootstrapProperties);
|
||||
assertTrue(provenanceReportingPropsOptional.isPresent());
|
||||
|
||||
final ProvenanceReportingSchema provenanceReportingSchema = provenanceReportingPropsOptional.get();
|
||||
assertEquals("This is a comment!", provenanceReportingSchema.getComment());
|
||||
assertEquals("TIMER_DRIVEN", provenanceReportingSchema.getSchedulingStrategy());
|
||||
assertEquals("15 secs", provenanceReportingSchema.getSchedulingPeriod());
|
||||
assertEquals("http://localhost:8080/", provenanceReportingSchema.getDestinationUrl());
|
||||
assertEquals("provenance", provenanceReportingSchema.getPortName());
|
||||
assertEquals("http://${hostname(true)}:8081/nifi", provenanceReportingSchema.getOriginatingUrl());
|
||||
assertEquals("10 secs", provenanceReportingSchema.getTimeout());
|
||||
}
|
||||
|
||||
|
||||
public static Properties getTestBootstrapProperties(final String fileName) throws IOException {
|
||||
final Properties bootstrapProperties = new Properties();
|
||||
try (final InputStream fis = RunMiNiFiTest.class.getClassLoader().getResourceAsStream(fileName)) {
|
||||
bootstrapProperties.load(fis);
|
||||
}
|
||||
return bootstrapProperties;
|
||||
}
|
||||
|
||||
}
|
|
@ -21,9 +21,7 @@ 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;
|
||||
import java.io.InputStream;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.file.Path;
|
||||
import java.nio.file.Paths;
|
||||
|
@ -49,9 +47,9 @@ public class WholeConfigDifferentiatorTest {
|
|||
@BeforeAll
|
||||
public static void setConfiguration() throws IOException {
|
||||
dummyRequest = new Request.Builder()
|
||||
.get()
|
||||
.url("https://nifi.apache.org/index.html")
|
||||
.build();
|
||||
.get()
|
||||
.url("https://nifi.apache.org/index.html")
|
||||
.build();
|
||||
|
||||
defaultConfigBuffer = ByteBuffer.wrap(FileUtils.readFileToByteArray(defaultConfigPath.toFile()));
|
||||
newConfigBuffer = ByteBuffer.wrap(FileUtils.readFileToByteArray(newConfigPath.toFile()));
|
||||
|
@ -61,28 +59,6 @@ public class WholeConfigDifferentiatorTest {
|
|||
when(configurationFileHolder.getConfigFileReference()).thenReturn(new AtomicReference<>(defaultConfigBuffer));
|
||||
}
|
||||
|
||||
// InputStream differentiator methods
|
||||
|
||||
@Test
|
||||
public void TestSameInputStream() throws IOException {
|
||||
Differentiator<InputStream> differentiator = WholeConfigDifferentiator.getInputStreamDifferentiator();
|
||||
differentiator.initialize(configurationFileHolder);
|
||||
|
||||
FileInputStream fileInputStream = new FileInputStream(defaultConfigPath.toFile());
|
||||
assertFalse(differentiator.isNew(fileInputStream));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void TestNewInputStream() throws IOException {
|
||||
Differentiator<InputStream> differentiator = WholeConfigDifferentiator.getInputStreamDifferentiator();
|
||||
differentiator.initialize(configurationFileHolder);
|
||||
|
||||
FileInputStream fileInputStream = new FileInputStream(newConfigPath.toFile());
|
||||
assertTrue(differentiator.isNew(fileInputStream));
|
||||
}
|
||||
|
||||
// Bytebuffer differentiator methods
|
||||
|
||||
@Test
|
||||
public void TestSameByteBuffer() throws IOException {
|
||||
Differentiator<ByteBuffer> differentiator = WholeConfigDifferentiator.getByteBufferDifferentiator();
|
||||
|
|
|
@ -5,20 +5,22 @@
|
|||
* 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
|
||||
*
|
||||
* <p>
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* <p>
|
||||
* 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.ingestors;
|
||||
|
||||
import static java.nio.file.StandardWatchEventKinds.ENTRY_MODIFY;
|
||||
import static org.apache.nifi.minifi.bootstrap.configuration.ingestors.PullHttpChangeIngestor.PULL_HTTP_BASE_KEY;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.verify;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
|
@ -34,6 +36,7 @@ import java.util.concurrent.atomic.AtomicReference;
|
|||
import org.apache.nifi.minifi.bootstrap.ConfigurationFileHolder;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeNotifier;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.differentiators.Differentiator;
|
||||
import org.apache.nifi.minifi.commons.api.MiNiFiProperties;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
@ -50,12 +53,13 @@ public class FileChangeIngestorTest {
|
|||
private Differentiator<ByteBuffer> mockDifferentiator;
|
||||
private ConfigurationChangeNotifier testNotifier;
|
||||
|
||||
|
||||
@BeforeEach
|
||||
public void setUp() {
|
||||
mockWatchService = Mockito.mock(WatchService.class);
|
||||
mockWatchService = mock(WatchService.class);
|
||||
notifierSpy = Mockito.spy(new FileChangeIngestor());
|
||||
mockDifferentiator = Mockito.mock(Differentiator.class);
|
||||
testNotifier = Mockito.mock(ConfigurationChangeNotifier.class);
|
||||
mockDifferentiator = mock(Differentiator.class);
|
||||
testNotifier = mock(ConfigurationChangeNotifier.class);
|
||||
|
||||
setMocks();
|
||||
|
||||
|
@ -64,6 +68,7 @@ public class FileChangeIngestorTest {
|
|||
testProperties.put(PullHttpChangeIngestor.OVERRIDE_SECURITY, "true");
|
||||
testProperties.put(PULL_HTTP_BASE_KEY + ".override.core", "true");
|
||||
testProperties.put(FileChangeIngestor.POLLING_PERIOD_INTERVAL_KEY, FileChangeIngestor.DEFAULT_POLLING_PERIOD_INTERVAL);
|
||||
testProperties.put(MiNiFiProperties.NIFI_MINIFI_FLOW_CONFIG.getKey(), MiNiFiProperties.NIFI_MINIFI_FLOW_CONFIG.getDefaultValue());
|
||||
}
|
||||
|
||||
@AfterEach
|
||||
|
@ -74,31 +79,31 @@ public class FileChangeIngestorTest {
|
|||
@Test
|
||||
public void testInitializeInvalidFile() {
|
||||
testProperties.put(FileChangeIngestor.CONFIG_FILE_PATH_KEY, "/land/of/make/believe");
|
||||
assertThrows(IllegalStateException.class, () -> notifierSpy.initialize(testProperties, Mockito.mock(ConfigurationFileHolder.class), Mockito.mock(ConfigurationChangeNotifier.class)));
|
||||
assertThrows(IllegalStateException.class, () -> notifierSpy.initialize(testProperties, mock(ConfigurationFileHolder.class), mock(ConfigurationChangeNotifier.class)));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInitializeValidFile() {
|
||||
notifierSpy.initialize(testProperties, Mockito.mock(ConfigurationFileHolder.class), Mockito.mock(ConfigurationChangeNotifier.class));
|
||||
public void testInitializeValidFile() {
|
||||
notifierSpy.initialize(testProperties, mock(ConfigurationFileHolder.class), mock(ConfigurationChangeNotifier.class));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInitializeInvalidPollingPeriod() {
|
||||
testProperties.put(FileChangeIngestor.POLLING_PERIOD_INTERVAL_KEY, "abc");
|
||||
assertThrows(IllegalStateException.class, () -> notifierSpy.initialize(testProperties, Mockito.mock(ConfigurationFileHolder.class), Mockito.mock(ConfigurationChangeNotifier.class)));
|
||||
assertThrows(IllegalStateException.class, () -> notifierSpy.initialize(testProperties, mock(ConfigurationFileHolder.class), mock(ConfigurationChangeNotifier.class)));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInitializeUseDefaultPolling() {
|
||||
testProperties.remove(FileChangeIngestor.POLLING_PERIOD_INTERVAL_KEY);
|
||||
notifierSpy.initialize(testProperties, Mockito.mock(ConfigurationFileHolder.class), Mockito.mock(ConfigurationChangeNotifier.class));
|
||||
notifierSpy.initialize(testProperties, mock(ConfigurationFileHolder.class), mock(ConfigurationChangeNotifier.class));
|
||||
}
|
||||
|
||||
/* Verify handleChange events */
|
||||
@Test
|
||||
public void testTargetChangedNoModification() throws Exception {
|
||||
when(mockDifferentiator.isNew(Mockito.any(ByteBuffer.class))).thenReturn(false);
|
||||
final ConfigurationChangeNotifier testNotifier = Mockito.mock(ConfigurationChangeNotifier.class);
|
||||
final ConfigurationChangeNotifier testNotifier = mock(ConfigurationChangeNotifier.class);
|
||||
|
||||
// In this case the WatchKey is null because there were no events found
|
||||
establishMockEnvironmentForChangeTests(null);
|
||||
|
@ -109,14 +114,14 @@ public class FileChangeIngestorTest {
|
|||
@Test
|
||||
public void testTargetChangedWithModificationEventNonConfigFile() throws Exception {
|
||||
when(mockDifferentiator.isNew(Mockito.any(ByteBuffer.class))).thenReturn(false);
|
||||
final ConfigurationChangeNotifier testNotifier = Mockito.mock(ConfigurationChangeNotifier.class);
|
||||
final ConfigurationChangeNotifier testNotifier = mock(ConfigurationChangeNotifier.class);
|
||||
|
||||
// In this case, we receive a trigger event for the directory monitored, but it was another file not being monitored
|
||||
final WatchKey mockWatchKey = createMockWatchKeyForPath("footage_not_found.yml");
|
||||
|
||||
establishMockEnvironmentForChangeTests(mockWatchKey);
|
||||
|
||||
notifierSpy.targetChanged();
|
||||
notifierSpy.targetFileChanged();
|
||||
|
||||
verify(testNotifier, Mockito.never()).notifyListeners(Mockito.any(ByteBuffer.class));
|
||||
}
|
||||
|
@ -129,8 +134,9 @@ public class FileChangeIngestorTest {
|
|||
// Provided as a spy to allow injection of mock objects for some tests when dealing with the finalized FileSystems class
|
||||
establishMockEnvironmentForChangeTests(mockWatchKey);
|
||||
|
||||
ConfigurationFileHolder configurationFileHolder = Mockito.mock(ConfigurationFileHolder.class);
|
||||
ConfigurationFileHolder configurationFileHolder = mock(ConfigurationFileHolder.class);
|
||||
when(configurationFileHolder.getConfigFileReference()).thenReturn(new AtomicReference<>(ByteBuffer.wrap(new byte[0])));
|
||||
|
||||
notifierSpy.initialize(testProperties, configurationFileHolder, testNotifier);
|
||||
setMocks();
|
||||
|
||||
|
@ -143,8 +149,8 @@ public class FileChangeIngestorTest {
|
|||
|
||||
/* Helper methods to establish mock environment */
|
||||
private WatchKey createMockWatchKeyForPath(String configFilePath) {
|
||||
WatchKey mockWatchKey = Mockito.mock(WatchKey.class);
|
||||
WatchEvent mockWatchEvent = Mockito.mock(WatchEvent.class);
|
||||
WatchKey mockWatchKey = mock(WatchKey.class);
|
||||
WatchEvent mockWatchEvent = 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));
|
||||
|
@ -157,8 +163,8 @@ public class FileChangeIngestorTest {
|
|||
|
||||
private void establishMockEnvironmentForChangeTests(final WatchKey watchKey) {
|
||||
// Establish the file mock and its parent directory
|
||||
final Path mockConfigFilePath = Mockito.mock(Path.class);
|
||||
final Path mockConfigFileParentPath = Mockito.mock(Path.class);
|
||||
final Path mockConfigFilePath = mock(Path.class);
|
||||
final Path mockConfigFileParentPath = mock(Path.class);
|
||||
|
||||
// When getting the parent of the file, get the directory
|
||||
when(mockConfigFilePath.getParent()).thenReturn(mockConfigFileParentPath);
|
||||
|
|
|
@ -15,15 +15,12 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.nifi.minifi.bootstrap.configuration.ingestors.common;
|
||||
package org.apache.nifi.minifi.bootstrap.configuration.ingestors;
|
||||
|
||||
import static org.apache.nifi.minifi.bootstrap.configuration.ingestors.PullHttpChangeIngestor.PATH_KEY;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertNotNull;
|
||||
import static org.mockito.Mockito.verify;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.PrintWriter;
|
||||
import java.nio.ByteBuffer;
|
||||
|
@ -32,18 +29,10 @@ import java.util.Collections;
|
|||
import java.util.Properties;
|
||||
import javax.servlet.http.HttpServletRequest;
|
||||
import javax.servlet.http.HttpServletResponse;
|
||||
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.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;
|
||||
import org.apache.nifi.minifi.commons.schema.common.ConvertableSchema;
|
||||
import org.apache.nifi.minifi.commons.schema.exception.SchemaLoaderException;
|
||||
import org.apache.nifi.minifi.commons.schema.serialization.SchemaLoader;
|
||||
import org.apache.nifi.util.file.FileUtils;
|
||||
import org.eclipse.jetty.server.Request;
|
||||
import org.eclipse.jetty.server.Server;
|
||||
import org.eclipse.jetty.server.handler.AbstractHandler;
|
||||
|
@ -52,11 +41,11 @@ import org.eclipse.jetty.util.thread.QueuedThreadPool;
|
|||
import org.junit.jupiter.api.AfterAll;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.mockito.ArgumentCaptor;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
public abstract class PullHttpChangeIngestorCommonTest {
|
||||
|
||||
public static final String FLOW_JSON = "/flow.json";
|
||||
public static volatile Server jetty;
|
||||
public static volatile int port;
|
||||
public static volatile PullHttpChangeIngestor pullHttpChangeIngestor;
|
||||
|
@ -64,8 +53,8 @@ public abstract class PullHttpChangeIngestorCommonTest {
|
|||
public static Differentiator<ByteBuffer> mockDifferentiator = Mockito.mock(Differentiator.class);
|
||||
public static final String RESPONSE_STRING = "test";
|
||||
public static final String PATH_RESPONSE_STRING = "path";
|
||||
public static ByteBuffer configBuffer= ByteBuffer.wrap(RESPONSE_STRING.getBytes());
|
||||
public static ByteBuffer pathConfigBuffer= ByteBuffer.wrap(PATH_RESPONSE_STRING.getBytes());
|
||||
public static ByteBuffer configBuffer = ByteBuffer.wrap(RESPONSE_STRING.getBytes());
|
||||
public static ByteBuffer pathConfigBuffer = ByteBuffer.wrap(PATH_RESPONSE_STRING.getBytes());
|
||||
public static final String ETAG = "testEtag";
|
||||
public static final String QUOTED_ETAG = "\"testEtag\"";
|
||||
|
||||
|
@ -79,7 +68,7 @@ public abstract class PullHttpChangeIngestorCommonTest {
|
|||
jetty.setHandler(handlerCollection);
|
||||
}
|
||||
|
||||
public abstract void pullHttpChangeIngestorInit(Properties properties) throws IOException, SchemaLoaderException;
|
||||
public abstract void pullHttpChangeIngestorInit(Properties properties) throws IOException;
|
||||
|
||||
@BeforeEach
|
||||
public void setListeners() {
|
||||
|
@ -95,7 +84,7 @@ public abstract class PullHttpChangeIngestorCommonTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testNewUpdate() throws IOException, SchemaLoaderException {
|
||||
public void testNewUpdate() throws IOException {
|
||||
Properties properties = new Properties();
|
||||
properties.put(PullHttpChangeIngestor.OVERRIDE_SECURITY, "true");
|
||||
pullHttpChangeIngestorInit(properties);
|
||||
|
@ -108,28 +97,7 @@ public abstract class PullHttpChangeIngestorCommonTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testSecurityOverride() throws IOException, SchemaLoaderException {
|
||||
Properties properties = new Properties();
|
||||
properties.put(PullHttpChangeIngestor.OVERRIDE_SECURITY, "false");
|
||||
properties.put(RunMiNiFi.MINIFI_CONFIG_FILE_KEY, "src/test/resources/config.yml");
|
||||
properties.put(PATH_KEY, "/config-minimal.yml");
|
||||
pullHttpChangeIngestorInit(properties);
|
||||
when(mockDifferentiator.isNew(Mockito.any(ByteBuffer.class))).thenReturn(true);
|
||||
|
||||
pullHttpChangeIngestor.run();
|
||||
|
||||
ArgumentCaptor<ByteBuffer> argument = ArgumentCaptor.forClass(ByteBuffer.class);
|
||||
verify(testNotifier, Mockito.times(1)).notifyListeners(argument.capture());
|
||||
|
||||
ConvertableSchema<ConfigSchema> configSchema = SchemaLoader.loadConvertableSchemaFromYaml(new ByteBufferInputStream(argument.getValue()));
|
||||
ConfigSchema newSchema = configSchema.convert();
|
||||
|
||||
assertNotNull(newSchema.getSecurityProperties().getKeystore());
|
||||
assertEquals(newSchema.getProcessGroupSchema().getProcessors().size(), 2);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNoUpdate() throws IOException, SchemaLoaderException {
|
||||
public void testNoUpdate() throws IOException {
|
||||
Properties properties = new Properties();
|
||||
properties.put(PullHttpChangeIngestor.OVERRIDE_SECURITY, "true");
|
||||
pullHttpChangeIngestorInit(properties);
|
||||
|
@ -142,7 +110,7 @@ public abstract class PullHttpChangeIngestorCommonTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testUseEtag() throws IOException, SchemaLoaderException {
|
||||
public void testUseEtag() throws IOException {
|
||||
Properties properties = new Properties();
|
||||
properties.put(PullHttpChangeIngestor.OVERRIDE_SECURITY, "true");
|
||||
pullHttpChangeIngestorInit(properties);
|
||||
|
@ -159,13 +127,12 @@ public abstract class PullHttpChangeIngestorCommonTest {
|
|||
pullHttpChangeIngestor.run();
|
||||
|
||||
verify(testNotifier, Mockito.times(1)).notifyListeners(Mockito.any());
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNewUpdateWithPath() throws IOException, SchemaLoaderException {
|
||||
public void testNewUpdateWithPath() throws IOException {
|
||||
Properties properties = new Properties();
|
||||
properties.put(PATH_KEY, "/config.yml");
|
||||
properties.put(PATH_KEY, FLOW_JSON);
|
||||
properties.put(PullHttpChangeIngestor.OVERRIDE_SECURITY, "true");
|
||||
pullHttpChangeIngestorInit(properties);
|
||||
pullHttpChangeIngestor.setUseEtag(false);
|
||||
|
@ -177,10 +144,10 @@ public abstract class PullHttpChangeIngestorCommonTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testNoUpdateWithPath() throws IOException, SchemaLoaderException {
|
||||
public void testNoUpdateWithPath() throws IOException {
|
||||
Properties properties = new Properties();
|
||||
properties.put(PullHttpChangeIngestor.OVERRIDE_SECURITY, "true");
|
||||
properties.put(PATH_KEY, "/config.yml");
|
||||
properties.put(PATH_KEY, FLOW_JSON);
|
||||
pullHttpChangeIngestorInit(properties);
|
||||
pullHttpChangeIngestor.setUseEtag(false);
|
||||
when(mockDifferentiator.isNew(Mockito.any(ByteBuffer.class))).thenReturn(false);
|
||||
|
@ -191,10 +158,10 @@ public abstract class PullHttpChangeIngestorCommonTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testUseEtagWithPath() throws IOException, SchemaLoaderException {
|
||||
public void testUseEtagWithPath() throws IOException {
|
||||
Properties properties = new Properties();
|
||||
properties.put(PullHttpChangeIngestor.OVERRIDE_SECURITY, "true");
|
||||
properties.put(PATH_KEY, "/config.yml");
|
||||
properties.put(PATH_KEY, FLOW_JSON);
|
||||
pullHttpChangeIngestorInit(properties);
|
||||
pullHttpChangeIngestor.setLastEtag("");
|
||||
|
||||
|
@ -209,33 +176,30 @@ public abstract class PullHttpChangeIngestorCommonTest {
|
|||
pullHttpChangeIngestor.run();
|
||||
|
||||
verify(testNotifier, Mockito.times(1)).notifyListeners(Mockito.any());
|
||||
|
||||
}
|
||||
|
||||
static class JettyHandler extends AbstractHandler {
|
||||
volatile String configResponse;
|
||||
volatile String pathResponse;
|
||||
|
||||
public JettyHandler(String configResponse, String pathResponse){
|
||||
public JettyHandler(String configResponse, String pathResponse) {
|
||||
this.configResponse = configResponse;
|
||||
this.pathResponse = pathResponse;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void handle(String target, Request baseRequest, HttpServletRequest request, HttpServletResponse response)
|
||||
throws IOException {
|
||||
throws IOException {
|
||||
|
||||
baseRequest.setHandled(true);
|
||||
|
||||
if ("GET".equals(request.getMethod())) {
|
||||
|
||||
if (QUOTED_ETAG.equals(baseRequest.getHeader("If-None-Match"))){
|
||||
if (QUOTED_ETAG.equals(baseRequest.getHeader("If-None-Match"))) {
|
||||
writeOutput(response, null, 304);
|
||||
} else {
|
||||
if ("/config.yml".equals(baseRequest.getPathInfo())) {
|
||||
if (FLOW_JSON.equals(baseRequest.getPathInfo())) {
|
||||
writeOutput(response, pathResponse, 200);
|
||||
} else if ("/config-minimal.yml".equals(baseRequest.getPathInfo())) {
|
||||
writeFileOutput(response, new File("src/test/resources/config-minimal.yml"), 200);
|
||||
} else {
|
||||
writeOutput(response, configResponse, 200);
|
||||
}
|
||||
|
@ -260,17 +224,5 @@ public abstract class PullHttpChangeIngestorCommonTest {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void writeFileOutput(HttpServletResponse response, File file, int responseCode) throws IOException {
|
||||
response.setStatus(responseCode);
|
||||
response.setHeader("ETag", ETAG);
|
||||
if (file != null) {
|
||||
response.setContentType("text/plain");
|
||||
response.setContentLength((int) file.length());
|
||||
response.setCharacterEncoding(StandardCharsets.UTF_8.displayName());
|
||||
FileUtils.copyFile(file, response.getOutputStream(), true, true);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
}
|
|
@ -18,18 +18,10 @@
|
|||
package org.apache.nifi.minifi.bootstrap.configuration.ingestors;
|
||||
|
||||
import static org.apache.nifi.minifi.bootstrap.configuration.ingestors.PullHttpChangeIngestor.PULL_HTTP_BASE_KEY;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.StringWriter;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Properties;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import org.apache.nifi.minifi.bootstrap.ConfigurationFileHolder;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.ingestors.common.PullHttpChangeIngestorCommonTest;
|
||||
import org.apache.nifi.minifi.commons.schema.ConfigSchema;
|
||||
import org.apache.nifi.minifi.commons.schema.exception.SchemaLoaderException;
|
||||
import org.apache.nifi.minifi.commons.schema.serialization.SchemaLoader;
|
||||
import org.eclipse.jetty.server.ServerConnector;
|
||||
import org.eclipse.jetty.util.ssl.SslContextFactory;
|
||||
import org.junit.jupiter.api.BeforeAll;
|
||||
|
@ -74,7 +66,7 @@ public class PullHttpChangeIngestorSSLTest extends PullHttpChangeIngestorCommonT
|
|||
}
|
||||
|
||||
@Override
|
||||
public void pullHttpChangeIngestorInit(Properties properties) throws IOException, SchemaLoaderException {
|
||||
public void pullHttpChangeIngestorInit(Properties properties) throws IOException {
|
||||
properties.setProperty(PullHttpChangeIngestor.TRUSTSTORE_LOCATION_KEY, "./src/test/resources/localhost-ts.jks");
|
||||
properties.setProperty(PullHttpChangeIngestor.TRUSTSTORE_PASSWORD_KEY, "localtest");
|
||||
properties.setProperty(PullHttpChangeIngestor.TRUSTSTORE_TYPE_KEY, "JKS");
|
||||
|
@ -88,12 +80,6 @@ public class PullHttpChangeIngestorSSLTest extends PullHttpChangeIngestorCommonT
|
|||
properties.put(PULL_HTTP_BASE_KEY + ".override.core", "true");
|
||||
ConfigurationFileHolder configurationFileHolder = Mockito.mock(ConfigurationFileHolder.class);
|
||||
|
||||
ConfigSchema configSchema =
|
||||
SchemaLoader.loadConfigSchemaFromYaml(PullHttpChangeIngestorSSLTest.class.getClassLoader().getResourceAsStream("config.yml"));
|
||||
StringWriter writer = new StringWriter();
|
||||
SchemaLoader.toYaml(configSchema, writer);
|
||||
when(configurationFileHolder.getConfigFileReference()).thenReturn(new AtomicReference<>(ByteBuffer.wrap(writer.toString().getBytes())));
|
||||
|
||||
pullHttpChangeIngestor = new PullHttpChangeIngestor();
|
||||
|
||||
pullHttpChangeIngestor.initialize(properties, configurationFileHolder, testNotifier);
|
||||
|
|
|
@ -18,18 +18,10 @@
|
|||
package org.apache.nifi.minifi.bootstrap.configuration.ingestors;
|
||||
|
||||
import static org.apache.nifi.minifi.bootstrap.configuration.ingestors.PullHttpChangeIngestor.PULL_HTTP_BASE_KEY;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.StringWriter;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Properties;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import org.apache.nifi.minifi.bootstrap.ConfigurationFileHolder;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.ingestors.common.PullHttpChangeIngestorCommonTest;
|
||||
import org.apache.nifi.minifi.commons.schema.ConfigSchema;
|
||||
import org.apache.nifi.minifi.commons.schema.exception.SchemaLoaderException;
|
||||
import org.apache.nifi.minifi.commons.schema.serialization.SchemaLoader;
|
||||
import org.eclipse.jetty.server.ServerConnector;
|
||||
import org.junit.jupiter.api.BeforeAll;
|
||||
import org.mockito.Mockito;
|
||||
|
@ -59,7 +51,7 @@ public class PullHttpChangeIngestorTest extends PullHttpChangeIngestorCommonTest
|
|||
|
||||
|
||||
@Override
|
||||
public void pullHttpChangeIngestorInit(Properties properties) throws IOException, SchemaLoaderException {
|
||||
public void pullHttpChangeIngestorInit(Properties properties) throws IOException {
|
||||
port = ((ServerConnector) jetty.getConnectors()[0]).getLocalPort();
|
||||
properties.put(PullHttpChangeIngestor.PORT_KEY, String.valueOf(port));
|
||||
properties.put(PullHttpChangeIngestor.HOST_KEY, "localhost");
|
||||
|
@ -67,12 +59,6 @@ public class PullHttpChangeIngestorTest extends PullHttpChangeIngestorCommonTest
|
|||
properties.put(PULL_HTTP_BASE_KEY + ".override.core", "true");
|
||||
ConfigurationFileHolder configurationFileHolder = Mockito.mock(ConfigurationFileHolder.class);
|
||||
|
||||
ConfigSchema configSchema =
|
||||
SchemaLoader.loadConfigSchemaFromYaml(PullHttpChangeIngestorTest.class.getClassLoader().getResourceAsStream("config.yml"));
|
||||
StringWriter writer = new StringWriter();
|
||||
SchemaLoader.toYaml(configSchema, writer);
|
||||
when(configurationFileHolder.getConfigFileReference()).thenReturn(new AtomicReference<>(ByteBuffer.wrap(writer.toString().getBytes())));
|
||||
|
||||
pullHttpChangeIngestor = new PullHttpChangeIngestor();
|
||||
pullHttpChangeIngestor.initialize(properties, configurationFileHolder, testNotifier);
|
||||
pullHttpChangeIngestor.setDifferentiator(mockDifferentiator);
|
||||
|
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.nifi.minifi.bootstrap.configuration.ingestors.common;
|
||||
package org.apache.nifi.minifi.bootstrap.configuration.ingestors;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.mockito.Mockito.verify;
|
||||
|
@ -30,11 +30,10 @@ import okhttp3.OkHttpClient;
|
|||
import okhttp3.Request;
|
||||
import okhttp3.RequestBody;
|
||||
import okhttp3.Response;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.differentiators.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.ingestors.RestChangeIngestor;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.differentiators.Differentiator;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.mockito.Mockito;
|
||||
|
@ -45,7 +44,7 @@ public abstract class RestChangeIngestorCommonTest {
|
|||
|
||||
public static OkHttpClient client;
|
||||
public static RestChangeIngestor restChangeIngestor;
|
||||
public static final MediaType MEDIA_TYPE_MARKDOWN = MediaType.parse("text/x-markdown; charset=utf-8");
|
||||
public static final MediaType MEDIA_TYPE_MARKDOWN = MediaType.parse("text/x-markdown; charset=utf-8");
|
||||
public static String url;
|
||||
public static ConfigurationChangeNotifier testNotifier = Mockito.mock(ConfigurationChangeNotifier.class);
|
||||
public static Differentiator<ByteBuffer> mockDifferentiator = Mockito.mock(Differentiator.class);
|
||||
|
@ -61,11 +60,13 @@ public abstract class RestChangeIngestorCommonTest {
|
|||
@Test
|
||||
public void testGet() throws Exception {
|
||||
Request request = new Request.Builder()
|
||||
.url(url)
|
||||
.build();
|
||||
.url(url)
|
||||
.build();
|
||||
|
||||
try (Response response = client.newCall(request).execute()) {
|
||||
if (!response.isSuccessful()) throw new IOException("Unexpected code " + response);
|
||||
if (!response.isSuccessful()) {
|
||||
throw new IOException("Unexpected code " + response);
|
||||
}
|
||||
|
||||
Headers responseHeaders = response.headers();
|
||||
for (int i = 0; i < responseHeaders.size(); i++) {
|
||||
|
@ -82,13 +83,15 @@ public abstract class RestChangeIngestorCommonTest {
|
|||
when(mockDifferentiator.isNew(Mockito.any(ByteBuffer.class))).thenReturn(true);
|
||||
|
||||
Request request = new Request.Builder()
|
||||
.url(url)
|
||||
.post(RequestBody.create(MEDIA_TYPE_MARKDOWN, testString))
|
||||
.addHeader("charset","UTF-8")
|
||||
.build();
|
||||
.url(url)
|
||||
.post(RequestBody.create(MEDIA_TYPE_MARKDOWN, testString))
|
||||
.addHeader("charset", "UTF-8")
|
||||
.build();
|
||||
|
||||
try (Response response = client.newCall(request).execute()) {
|
||||
if (!response.isSuccessful()) throw new IOException("Unexpected code " + response);
|
||||
if (!response.isSuccessful()) {
|
||||
throw new IOException("Unexpected code " + response);
|
||||
}
|
||||
|
||||
Headers responseHeaders = response.headers();
|
||||
for (int i = 0; i < responseHeaders.size(); i++) {
|
||||
|
@ -106,13 +109,15 @@ public abstract class RestChangeIngestorCommonTest {
|
|||
when(mockDifferentiator.isNew(Mockito.any(ByteBuffer.class))).thenReturn(false);
|
||||
|
||||
Request request = new Request.Builder()
|
||||
.url(url)
|
||||
.post(RequestBody.create(MEDIA_TYPE_MARKDOWN, testString))
|
||||
.addHeader("charset","UTF-8")
|
||||
.build();
|
||||
.url(url)
|
||||
.post(RequestBody.create(MEDIA_TYPE_MARKDOWN, testString))
|
||||
.addHeader("charset", "UTF-8")
|
||||
.build();
|
||||
|
||||
try (Response response = client.newCall(request).execute()) {
|
||||
if (response.isSuccessful()) throw new IOException("Unexpected code " + response);
|
||||
if (response.isSuccessful()) {
|
||||
throw new IOException("Unexpected code " + response);
|
||||
}
|
||||
|
||||
Headers responseHeaders = response.headers();
|
||||
for (int i = 0; i < responseHeaders.size(); i++) {
|
|
@ -17,14 +17,24 @@
|
|||
|
||||
package org.apache.nifi.minifi.bootstrap.configuration.ingestors;
|
||||
|
||||
import static org.apache.nifi.minifi.bootstrap.configuration.ingestors.PullHttpChangeIngestor.PULL_HTTP_BASE_KEY;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
import java.io.FileInputStream;
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.security.KeyStore;
|
||||
import java.util.Collections;
|
||||
import java.util.Properties;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import javax.net.ssl.SSLContext;
|
||||
import javax.net.ssl.SSLSocketFactory;
|
||||
import javax.net.ssl.X509TrustManager;
|
||||
import okhttp3.OkHttpClient;
|
||||
import org.apache.nifi.minifi.bootstrap.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;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.ingestors.common.RestChangeIngestorCommonTest;
|
||||
import org.apache.nifi.security.ssl.StandardKeyStoreBuilder;
|
||||
import org.apache.nifi.security.ssl.StandardSslContextBuilder;
|
||||
import org.apache.nifi.security.ssl.StandardTrustManagerBuilder;
|
||||
|
@ -34,18 +44,6 @@ import org.junit.jupiter.api.AfterAll;
|
|||
import org.junit.jupiter.api.BeforeAll;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
import javax.net.ssl.SSLContext;
|
||||
import javax.net.ssl.SSLSocketFactory;
|
||||
import javax.net.ssl.X509TrustManager;
|
||||
import java.io.FileInputStream;
|
||||
import java.io.IOException;
|
||||
import java.security.KeyStore;
|
||||
import java.util.Collections;
|
||||
import java.util.Properties;
|
||||
|
||||
import static org.apache.nifi.minifi.bootstrap.configuration.ingestors.PullHttpChangeIngestor.PULL_HTTP_BASE_KEY;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
public class RestChangeIngestorSSLTest extends RestChangeIngestorCommonTest {
|
||||
|
||||
@BeforeAll
|
||||
|
@ -82,27 +80,27 @@ public class RestChangeIngestorSSLTest extends RestChangeIngestorCommonTest {
|
|||
final KeyStore keyStore;
|
||||
try (final FileInputStream keyStoreStream = new FileInputStream(tlsConfiguration.getKeystorePath())) {
|
||||
keyStore = new StandardKeyStoreBuilder()
|
||||
.type(tlsConfiguration.getKeystoreType().getType())
|
||||
.inputStream(keyStoreStream)
|
||||
.password(tlsConfiguration.getKeystorePassword().toCharArray())
|
||||
.build();
|
||||
.type(tlsConfiguration.getKeystoreType().getType())
|
||||
.inputStream(keyStoreStream)
|
||||
.password(tlsConfiguration.getKeystorePassword().toCharArray())
|
||||
.build();
|
||||
}
|
||||
|
||||
final KeyStore truststore;
|
||||
try (final FileInputStream trustStoreStream = new FileInputStream(tlsConfiguration.getTruststorePath())) {
|
||||
truststore = new StandardKeyStoreBuilder()
|
||||
.type(tlsConfiguration.getTruststoreType().getType())
|
||||
.inputStream(trustStoreStream)
|
||||
.password(tlsConfiguration.getTruststorePassword().toCharArray())
|
||||
.build();
|
||||
.type(tlsConfiguration.getTruststoreType().getType())
|
||||
.inputStream(trustStoreStream)
|
||||
.password(tlsConfiguration.getTruststorePassword().toCharArray())
|
||||
.build();
|
||||
}
|
||||
|
||||
final X509TrustManager trustManager = new StandardTrustManagerBuilder().trustStore(truststore).build();
|
||||
final SSLContext sslContext = new StandardSslContextBuilder()
|
||||
.keyStore(keyStore)
|
||||
.keyPassword(tlsConfiguration.getKeyPassword().toCharArray())
|
||||
.trustStore(truststore)
|
||||
.build();
|
||||
.keyStore(keyStore)
|
||||
.keyPassword(tlsConfiguration.getKeyPassword().toCharArray())
|
||||
.trustStore(truststore)
|
||||
.build();
|
||||
final SSLSocketFactory sslSocketFactory = sslContext.getSocketFactory();
|
||||
|
||||
clientBuilder.sslSocketFactory(sslSocketFactory, trustManager);
|
||||
|
|
|
@ -20,19 +20,17 @@ package org.apache.nifi.minifi.bootstrap.configuration.ingestors;
|
|||
import static org.apache.nifi.minifi.bootstrap.configuration.ingestors.PullHttpChangeIngestor.PULL_HTTP_BASE_KEY;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
import java.net.MalformedURLException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Properties;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import okhttp3.OkHttpClient;
|
||||
import org.apache.nifi.minifi.bootstrap.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;
|
||||
import org.junit.jupiter.api.BeforeAll;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
import java.net.MalformedURLException;
|
||||
import java.util.Properties;
|
||||
|
||||
public class RestChangeIngestorTest extends RestChangeIngestorCommonTest {
|
||||
|
||||
@BeforeAll
|
||||
|
|
|
@ -61,8 +61,6 @@ class BootstrapCodecTest {
|
|||
@Mock
|
||||
private ConfigurationChangeListener configurationChangeListener;
|
||||
@Mock
|
||||
private UpdateConfigurationService updateConfigurationService;
|
||||
@Mock
|
||||
private UpdatePropertiesService updatePropertiesService;
|
||||
|
||||
@InjectMocks
|
||||
|
@ -70,7 +68,6 @@ class BootstrapCodecTest {
|
|||
|
||||
@BeforeEach
|
||||
void setup() throws IllegalAccessException, NoSuchFieldException {
|
||||
mockFinal("updateConfigurationService", updateConfigurationService);
|
||||
mockFinal("updatePropertiesService", updatePropertiesService);
|
||||
}
|
||||
|
||||
|
@ -193,19 +190,6 @@ class BootstrapCodecTest {
|
|||
assertEquals(OK, outputStream.toString().trim());
|
||||
}
|
||||
|
||||
@Test
|
||||
void testUpdateConfigurationCommandShouldHandleUpdateConfiguration() throws IOException {
|
||||
InputStream inputStream = new ByteArrayInputStream("UPDATE_CONFIGURATION".getBytes(StandardCharsets.UTF_8));
|
||||
C2Operation c2Operation = new C2Operation();
|
||||
c2Operation.setIdentifier("id");
|
||||
ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
|
||||
|
||||
bootstrapCodec.communicate(inputStream, outputStream);
|
||||
|
||||
assertEquals(OK, outputStream.toString().trim());
|
||||
verify(updateConfigurationService).handleUpdate();
|
||||
}
|
||||
|
||||
@Test
|
||||
void testUpdatePropertiesCommandShouldHandleUpdateProperties() throws IOException {
|
||||
InputStream inputStream = new ByteArrayInputStream("UPDATE_PROPERTIES".getBytes(StandardCharsets.UTF_8));
|
||||
|
|
|
@ -0,0 +1,267 @@
|
|||
/*
|
||||
* Licensed to the Apache 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.lang.Boolean.TRUE;
|
||||
import static java.nio.file.Files.newInputStream;
|
||||
import static java.nio.file.Files.newOutputStream;
|
||||
import static java.util.UUID.randomUUID;
|
||||
import static java.util.stream.Collectors.toList;
|
||||
import static java.util.stream.Collectors.toMap;
|
||||
import static java.util.stream.Stream.concat;
|
||||
import static org.apache.commons.lang3.StringUtils.EMPTY;
|
||||
import static org.apache.commons.lang3.StringUtils.SPACE;
|
||||
import static org.apache.commons.lang3.StringUtils.isNotBlank;
|
||||
import static org.apache.nifi.minifi.bootstrap.service.MiNiFiPropertiesGenerator.DEFAULT_SENSITIVE_PROPERTIES_ENCODING_ALGORITHM;
|
||||
import static org.apache.nifi.minifi.bootstrap.service.MiNiFiPropertiesGenerator.MINIFI_TO_NIFI_PROPERTY_MAPPING;
|
||||
import static org.apache.nifi.minifi.bootstrap.service.MiNiFiPropertiesGenerator.NIFI_PROPERTIES_WITH_DEFAULT_VALUES_AND_COMMENTS;
|
||||
import static org.apache.nifi.minifi.commons.api.MiNiFiProperties.MINIFI_APP_LOG_FILE;
|
||||
import static org.apache.nifi.minifi.commons.api.MiNiFiProperties.MINIFI_BOOTSTRAP_FILE_PATH;
|
||||
import static org.apache.nifi.minifi.commons.api.MiNiFiProperties.MINIFI_BOOTSTRAP_LOG_FILE;
|
||||
import static org.apache.nifi.minifi.commons.api.MiNiFiProperties.MINIFI_LOG_DIRECTORY;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertNull;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
import java.io.UncheckedIOException;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.Properties;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Stream;
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeException;
|
||||
import org.apache.nifi.minifi.commons.api.MiNiFiProperties;
|
||||
import org.apache.nifi.util.NiFiProperties;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.io.TempDir;
|
||||
|
||||
public class MiNiFiPropertiesGeneratorTest {
|
||||
|
||||
@TempDir
|
||||
private Path tempDir;
|
||||
|
||||
private Path configDirectory;
|
||||
private Path bootstrapPropertiesFile;
|
||||
private Path minifiPropertiesFile;
|
||||
|
||||
private MiNiFiPropertiesGenerator testPropertiesGenerator;
|
||||
|
||||
@BeforeEach
|
||||
public void setup() throws IOException {
|
||||
configDirectory = tempDir.toAbsolutePath().resolve("conf");
|
||||
Files.createDirectories(configDirectory);
|
||||
bootstrapPropertiesFile = configDirectory.resolve("bootstrap.conf");
|
||||
minifiPropertiesFile = configDirectory.resolve("minifi.properties");
|
||||
|
||||
testPropertiesGenerator = new MiNiFiPropertiesGenerator();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGenerateDefaultNiFiProperties() throws ConfigurationChangeException {
|
||||
// given
|
||||
Properties bootstrapProperties = createBootstrapProperties(Map.of());
|
||||
|
||||
// when
|
||||
testPropertiesGenerator.generateMinifiProperties(configDirectory.toString(), bootstrapProperties);
|
||||
|
||||
// then
|
||||
List<String> expectedMiNiFiProperties = NIFI_PROPERTIES_WITH_DEFAULT_VALUES_AND_COMMENTS.stream()
|
||||
.map(triplet -> triplet.getLeft() + "=" + triplet.getMiddle())
|
||||
.collect(toList());
|
||||
List<String> resultMiNiFiProperties = loadMiNiFiProperties().entrySet()
|
||||
.stream()
|
||||
.map(entry -> entry.getKey() + "=" + entry.getValue())
|
||||
.collect(toList());
|
||||
assertTrue(resultMiNiFiProperties.containsAll(expectedMiNiFiProperties));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMiNiFiPropertiesMappedToAppropriateNiFiProperties() throws ConfigurationChangeException {
|
||||
// given
|
||||
Properties bootstrapProperties = createBootstrapProperties(Stream.of(
|
||||
MiNiFiProperties.NIFI_MINIFI_FLOW_CONFIG.getKey(),
|
||||
MiNiFiProperties.NIFI_MINIFI_SECURITY_KEYSTORE.getKey(),
|
||||
MiNiFiProperties.NIFI_MINIFI_SECURITY_KEYSTORE_TYPE.getKey(),
|
||||
MiNiFiProperties.NIFI_MINIFI_SECURITY_KEYSTORE_PASSWD.getKey(),
|
||||
MiNiFiProperties.NIFI_MINIFI_SECURITY_KEY_PASSWD.getKey(),
|
||||
MiNiFiProperties.NIFI_MINIFI_SECURITY_TRUSTSTORE.getKey(),
|
||||
MiNiFiProperties.NIFI_MINIFI_SECURITY_TRUSTSTORE_TYPE.getKey(),
|
||||
MiNiFiProperties.NIFI_MINIFI_SECURITY_TRUSTSTORE_PASSWD.getKey())
|
||||
.collect(toMap(Function.identity(), __ -> randomUUID().toString()))
|
||||
);
|
||||
|
||||
// when
|
||||
testPropertiesGenerator.generateMinifiProperties(configDirectory.toString(), bootstrapProperties);
|
||||
|
||||
// then
|
||||
Properties miNiFiProperties = loadMiNiFiProperties();
|
||||
assertTrue(
|
||||
MINIFI_TO_NIFI_PROPERTY_MAPPING.entrySet().stream()
|
||||
.allMatch(entry -> Objects.equals(bootstrapProperties.getProperty(entry.getKey()), miNiFiProperties.getProperty(entry.getValue()))));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSensitivePropertiesAreGeneratedWhenNotProvidedInBootstrap() throws ConfigurationChangeException {
|
||||
// given
|
||||
Properties bootstrapProperties = createBootstrapProperties(Map.of());
|
||||
|
||||
// when
|
||||
testPropertiesGenerator.generateMinifiProperties(configDirectory.toString(), bootstrapProperties);
|
||||
|
||||
// then
|
||||
Properties miNiFiProperties = loadMiNiFiProperties();
|
||||
assertTrue(isNotBlank(miNiFiProperties.getProperty(NiFiProperties.SENSITIVE_PROPS_KEY)));
|
||||
assertEquals(DEFAULT_SENSITIVE_PROPERTIES_ENCODING_ALGORITHM, miNiFiProperties.getProperty(NiFiProperties.SENSITIVE_PROPS_ALGORITHM));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSensitivePropertiesAreUsedWhenProvidedInBootstrap() throws ConfigurationChangeException {
|
||||
// given
|
||||
String sensitivePropertiesKey = "sensitive_properties_key";
|
||||
String sensitivePropertiesAlgorithm = "sensitive_properties_algorithm";
|
||||
Properties bootstrapProperties = createBootstrapProperties(Map.of(
|
||||
MiNiFiProperties.NIFI_MINIFI_SENSITIVE_PROPS_KEY.getKey(), sensitivePropertiesKey,
|
||||
MiNiFiProperties.NIFI_MINIFI_SENSITIVE_PROPS_ALGORITHM.getKey(), sensitivePropertiesAlgorithm
|
||||
));
|
||||
|
||||
// when
|
||||
testPropertiesGenerator.generateMinifiProperties(configDirectory.toString(), bootstrapProperties);
|
||||
|
||||
// then
|
||||
Properties miNiFiProperties = loadMiNiFiProperties();
|
||||
assertEquals(sensitivePropertiesKey, miNiFiProperties.getProperty(NiFiProperties.SENSITIVE_PROPS_KEY));
|
||||
assertEquals(sensitivePropertiesAlgorithm, miNiFiProperties.getProperty(NiFiProperties.SENSITIVE_PROPS_ALGORITHM));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNonBlankC2PropertiesAreCopiedToMiNiFiProperties() throws ConfigurationChangeException {
|
||||
// given
|
||||
Properties bootstrapProperties = createBootstrapProperties(Map.of(
|
||||
MiNiFiProperties.C2_ENABLE.getKey(), TRUE.toString(),
|
||||
MiNiFiProperties.C2_AGENT_CLASS.getKey(), EMPTY,
|
||||
MiNiFiProperties.C2_AGENT_IDENTIFIER.getKey(), SPACE
|
||||
));
|
||||
|
||||
// when
|
||||
testPropertiesGenerator.generateMinifiProperties(configDirectory.toString(), bootstrapProperties);
|
||||
|
||||
// then
|
||||
Properties miNiFiProperties = loadMiNiFiProperties();
|
||||
assertEquals(TRUE.toString(), miNiFiProperties.getProperty(MiNiFiProperties.C2_ENABLE.getKey()));
|
||||
assertNull(miNiFiProperties.getProperty(MiNiFiProperties.C2_AGENT_CLASS.getKey()));
|
||||
assertNull(miNiFiProperties.getProperty(MiNiFiProperties.C2_AGENT_IDENTIFIER.getKey()));
|
||||
assertNull(miNiFiProperties.getProperty(MiNiFiProperties.C2_REST_URL.getKey()));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDefaultNiFiPropertiesAreOverridden() throws ConfigurationChangeException {
|
||||
// given
|
||||
String archiveDir = "/path/to";
|
||||
Properties bootstrapProperties = createBootstrapProperties(Map.of(
|
||||
NiFiProperties.FLOW_CONFIGURATION_ARCHIVE_ENABLED, TRUE.toString(),
|
||||
NiFiProperties.FLOW_CONFIGURATION_ARCHIVE_DIR, archiveDir
|
||||
));
|
||||
|
||||
// when
|
||||
testPropertiesGenerator.generateMinifiProperties(configDirectory.toString(), bootstrapProperties);
|
||||
|
||||
// then
|
||||
List<String> expectedMiNiFiProperties = concat(
|
||||
NIFI_PROPERTIES_WITH_DEFAULT_VALUES_AND_COMMENTS.stream()
|
||||
.filter(triplet ->
|
||||
!triplet.getLeft().equals(NiFiProperties.FLOW_CONFIGURATION_ARCHIVE_ENABLED)
|
||||
&& !triplet.getLeft().equals(NiFiProperties.FLOW_CONFIGURATION_ARCHIVE_DIR))
|
||||
.map(triplet -> triplet.getLeft() + "=" + triplet.getMiddle()),
|
||||
Stream.of(
|
||||
NiFiProperties.FLOW_CONFIGURATION_ARCHIVE_ENABLED + "=" + TRUE,
|
||||
NiFiProperties.FLOW_CONFIGURATION_ARCHIVE_DIR + "=" + archiveDir
|
||||
))
|
||||
.collect(toList());
|
||||
List<String> resultMiNiFiProperties = loadMiNiFiProperties().entrySet()
|
||||
.stream()
|
||||
.map(entry -> entry.getKey() + "=" + entry.getValue())
|
||||
.collect(toList());
|
||||
assertTrue(resultMiNiFiProperties.containsAll(expectedMiNiFiProperties));
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testArbitraryNiFiPropertyCanBePassedViaBootstrapConf() throws ConfigurationChangeException {
|
||||
// given
|
||||
Properties bootstrapProperties = createBootstrapProperties(Map.of(
|
||||
"nifi.new.property", "new_property_value",
|
||||
"nifi.other.new.property", "other_new_property_value"
|
||||
));
|
||||
|
||||
// when
|
||||
testPropertiesGenerator.generateMinifiProperties(configDirectory.toString(), bootstrapProperties);
|
||||
|
||||
// then
|
||||
Properties miNiFiProperties = loadMiNiFiProperties();
|
||||
assertEquals("new_property_value", miNiFiProperties.getProperty("nifi.new.property"));
|
||||
assertEquals("other_new_property_value", miNiFiProperties.getProperty("nifi.other.new.property"));
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void bootstrapFileAndLogPropertiesAreGeneratedIntoMiNiFiProperties() throws ConfigurationChangeException {
|
||||
// given
|
||||
Properties bootstrapProperties = createBootstrapProperties(Map.of());
|
||||
|
||||
// when
|
||||
testPropertiesGenerator.generateMinifiProperties(configDirectory.toString(), bootstrapProperties);
|
||||
|
||||
// then
|
||||
Properties miNiFiProperties = loadMiNiFiProperties();
|
||||
assertTrue(List.of(MINIFI_BOOTSTRAP_FILE_PATH, MINIFI_LOG_DIRECTORY, MINIFI_APP_LOG_FILE, MINIFI_BOOTSTRAP_LOG_FILE)
|
||||
.stream()
|
||||
.map(miNiFiProperties::getProperty)
|
||||
.allMatch(StringUtils::isNotBlank)
|
||||
);
|
||||
}
|
||||
|
||||
private Properties createBootstrapProperties(Map<String, String> keyValues) {
|
||||
try (OutputStream outputStream = newOutputStream(bootstrapPropertiesFile)) {
|
||||
Properties properties = new Properties();
|
||||
properties.putAll(keyValues);
|
||||
properties.store(outputStream, EMPTY);
|
||||
return properties;
|
||||
} catch (IOException e) {
|
||||
throw new UncheckedIOException(e);
|
||||
}
|
||||
}
|
||||
|
||||
private Properties loadMiNiFiProperties() {
|
||||
try (InputStream inputStream = newInputStream(minifiPropertiesFile)) {
|
||||
Properties properties = new Properties();
|
||||
properties.load(inputStream);
|
||||
return properties;
|
||||
} catch (IOException e) {
|
||||
throw new UncheckedIOException(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -17,6 +17,18 @@
|
|||
|
||||
package org.apache.nifi.minifi.bootstrap.status.reporters;
|
||||
|
||||
import static org.apache.nifi.minifi.bootstrap.status.reporters.StatusLogger.ENCOUNTERED_IO_EXCEPTION;
|
||||
import static org.apache.nifi.minifi.commons.api.MiNiFiProperties.NIFI_MINIFI_STATUS_REPORTER_LOG_LEVEL;
|
||||
import static org.apache.nifi.minifi.commons.api.MiNiFiProperties.NIFI_MINIFI_STATUS_REPORTER_LOG_PERIOD;
|
||||
import static org.apache.nifi.minifi.commons.api.MiNiFiProperties.NIFI_MINIFI_STATUS_REPORTER_LOG_QUERY;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
import static org.mockito.Mockito.verify;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Properties;
|
||||
import java.util.concurrent.ScheduledFuture;
|
||||
import java.util.concurrent.ScheduledThreadPoolExecutor;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import org.apache.nifi.logging.LogLevel;
|
||||
import org.apache.nifi.minifi.bootstrap.QueryableStatusAggregator;
|
||||
import org.apache.nifi.minifi.commons.status.FlowStatusReport;
|
||||
|
@ -25,22 +37,10 @@ import org.junit.jupiter.api.Test;
|
|||
import org.mockito.Mockito;
|
||||
import org.slf4j.Logger;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Properties;
|
||||
import java.util.concurrent.ScheduledFuture;
|
||||
import java.util.concurrent.ScheduledThreadPoolExecutor;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import static org.apache.nifi.minifi.bootstrap.status.reporters.StatusLogger.ENCOUNTERED_IO_EXCEPTION;
|
||||
import static org.apache.nifi.minifi.bootstrap.status.reporters.StatusLogger.LOGGING_LEVEL_KEY;
|
||||
import static org.apache.nifi.minifi.bootstrap.status.reporters.StatusLogger.QUERY_KEY;
|
||||
import static org.apache.nifi.minifi.bootstrap.status.reporters.StatusLogger.REPORT_PERIOD_KEY;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
import static org.mockito.Mockito.verify;
|
||||
|
||||
public class StatusLoggerTest {
|
||||
|
||||
private static final String MOCK_STATUS = "FlowStatusReport{controllerServiceStatusList=null, processorStatusList=[{name='TailFile', processorHealth={runStatus='Running', hasBulletins=false, " +
|
||||
private static final String MOCK_STATUS =
|
||||
"FlowStatusReport{controllerServiceStatusList=null, processorStatusList=[{name='TailFile', processorHealth={runStatus='Running', hasBulletins=false, " +
|
||||
"validationErrorList=[]}, processorStats=null, bulletinList=null}], connectionStatusList=null, remoteProcessingGroupStatusList=null, instanceStatus=null, systemDiagnosticsStatus=null," +
|
||||
" reportingTaskStatusList=null, errorsGeneratingReport=[]}";
|
||||
|
||||
|
@ -69,9 +69,9 @@ public class StatusLoggerTest {
|
|||
@Test
|
||||
public void testFailedInitDueToFatalLogLevel() {
|
||||
Properties properties = new Properties();
|
||||
properties.setProperty(REPORT_PERIOD_KEY, "1");
|
||||
properties.setProperty(LOGGING_LEVEL_KEY, LogLevel.FATAL.name());
|
||||
properties.setProperty(QUERY_KEY, MOCK_QUERY);
|
||||
properties.setProperty(NIFI_MINIFI_STATUS_REPORTER_LOG_PERIOD.getKey(), "1");
|
||||
properties.setProperty(NIFI_MINIFI_STATUS_REPORTER_LOG_LEVEL.getKey(), LogLevel.FATAL.name());
|
||||
properties.setProperty(NIFI_MINIFI_STATUS_REPORTER_LOG_QUERY.getKey(), MOCK_QUERY);
|
||||
|
||||
assertThrows(IllegalStateException.class, () -> statusLogger.initialize(properties, queryableStatusAggregator));
|
||||
}
|
||||
|
@ -79,8 +79,8 @@ public class StatusLoggerTest {
|
|||
@Test
|
||||
public void testFailedInitDueToNoPeriod() {
|
||||
Properties properties = new Properties();
|
||||
properties.setProperty(LOGGING_LEVEL_KEY, LogLevel.INFO.name());
|
||||
properties.setProperty(QUERY_KEY, MOCK_QUERY);
|
||||
properties.setProperty(NIFI_MINIFI_STATUS_REPORTER_LOG_LEVEL.getKey(), LogLevel.INFO.name());
|
||||
properties.setProperty(NIFI_MINIFI_STATUS_REPORTER_LOG_QUERY.getKey(), MOCK_QUERY);
|
||||
|
||||
assertThrows(IllegalStateException.class, () -> statusLogger.initialize(properties, queryableStatusAggregator));
|
||||
}
|
||||
|
@ -88,8 +88,8 @@ public class StatusLoggerTest {
|
|||
@Test
|
||||
public void testFailedInitDueToNoQuery() {
|
||||
Properties properties = new Properties();
|
||||
properties.setProperty(REPORT_PERIOD_KEY, "1");
|
||||
properties.setProperty(LOGGING_LEVEL_KEY, LogLevel.INFO.name());
|
||||
properties.setProperty(NIFI_MINIFI_STATUS_REPORTER_LOG_PERIOD.getKey(), "1");
|
||||
properties.setProperty(NIFI_MINIFI_STATUS_REPORTER_LOG_LEVEL.getKey(), LogLevel.INFO.name());
|
||||
|
||||
assertThrows(IllegalStateException.class, () -> statusLogger.initialize(properties, queryableStatusAggregator));
|
||||
}
|
||||
|
@ -143,9 +143,9 @@ public class StatusLoggerTest {
|
|||
@Test
|
||||
public void testTraceException() throws IOException {
|
||||
Properties properties = new Properties();
|
||||
properties.setProperty(REPORT_PERIOD_KEY, "1");
|
||||
properties.setProperty(LOGGING_LEVEL_KEY, LogLevel.TRACE.name());
|
||||
properties.setProperty(QUERY_KEY, MOCK_QUERY);
|
||||
properties.setProperty(NIFI_MINIFI_STATUS_REPORTER_LOG_PERIOD.getKey(), "1");
|
||||
properties.setProperty(NIFI_MINIFI_STATUS_REPORTER_LOG_LEVEL.getKey(), LogLevel.TRACE.name());
|
||||
properties.setProperty(NIFI_MINIFI_STATUS_REPORTER_LOG_QUERY.getKey(), MOCK_QUERY);
|
||||
|
||||
IOException ioException = new IOException("This is an expected test exception");
|
||||
Mockito.when(queryableStatusAggregator.statusReport(MOCK_QUERY)).thenThrow(ioException);
|
||||
|
@ -207,9 +207,9 @@ public class StatusLoggerTest {
|
|||
|
||||
private static Properties getProperties(LogLevel logLevel) {
|
||||
Properties properties = new Properties();
|
||||
properties.setProperty(REPORT_PERIOD_KEY, "1");
|
||||
properties.setProperty(LOGGING_LEVEL_KEY, logLevel.name());
|
||||
properties.setProperty(QUERY_KEY, MOCK_QUERY);
|
||||
properties.setProperty(NIFI_MINIFI_STATUS_REPORTER_LOG_PERIOD.getKey(), "1");
|
||||
properties.setProperty(NIFI_MINIFI_STATUS_REPORTER_LOG_LEVEL.getKey(), logLevel.name());
|
||||
properties.setProperty(NIFI_MINIFI_STATUS_REPORTER_LOG_QUERY.getKey(), MOCK_QUERY);
|
||||
return properties;
|
||||
}
|
||||
|
||||
|
|
|
@ -1,809 +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.util;
|
||||
|
||||
import org.apache.commons.io.FileUtils;
|
||||
import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeException;
|
||||
import org.apache.nifi.minifi.bootstrap.exception.InvalidConfigurationException;
|
||||
import org.apache.nifi.minifi.commons.schema.ConfigSchema;
|
||||
import org.apache.nifi.minifi.commons.schema.ConnectionSchema;
|
||||
import org.apache.nifi.minifi.commons.schema.ControllerServiceSchema;
|
||||
import org.apache.nifi.minifi.commons.schema.FunnelSchema;
|
||||
import org.apache.nifi.minifi.commons.schema.PortSchema;
|
||||
import org.apache.nifi.minifi.commons.schema.ProcessGroupSchema;
|
||||
import org.apache.nifi.minifi.commons.schema.ProcessorSchema;
|
||||
import org.apache.nifi.minifi.commons.schema.RemotePortSchema;
|
||||
import org.apache.nifi.minifi.commons.schema.RemoteProcessGroupSchema;
|
||||
import org.apache.nifi.minifi.commons.schema.ReportingSchema;
|
||||
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.serialization.SchemaLoader;
|
||||
import org.apache.nifi.util.StringUtils;
|
||||
import org.apache.nifi.xml.processing.parsers.DocumentProvider;
|
||||
import org.apache.nifi.xml.processing.parsers.StandardDocumentProvider;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.w3c.dom.Document;
|
||||
import org.w3c.dom.Element;
|
||||
import org.w3c.dom.NodeList;
|
||||
|
||||
import javax.xml.xpath.XPath;
|
||||
import javax.xml.xpath.XPathConstants;
|
||||
import javax.xml.xpath.XPathExpressionException;
|
||||
import javax.xml.xpath.XPathFactory;
|
||||
import java.io.ByteArrayInputStream;
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.File;
|
||||
import java.io.FileInputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.nio.charset.Charset;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.zip.GZIPInputStream;
|
||||
|
||||
import static org.apache.nifi.minifi.bootstrap.util.ConfigTransformer.MINIFI_APP_LOG_FILE;
|
||||
import static org.apache.nifi.minifi.bootstrap.util.ConfigTransformer.MINIFI_BOOTSTRAP_FILE_PATH;
|
||||
import static org.apache.nifi.minifi.bootstrap.util.ConfigTransformer.MINIFI_BOOTSTRAP_LOG_FILE;
|
||||
import static org.apache.nifi.minifi.bootstrap.util.ConfigTransformer.MINIFI_CONFIG_FILE_PATH;
|
||||
import static org.apache.nifi.minifi.bootstrap.util.ConfigTransformer.MINIFI_LOG_DIRECTORY;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertNotEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertNull;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
import static org.junit.jupiter.api.Assertions.fail;
|
||||
|
||||
public class ConfigTransformerTest {
|
||||
public static final Map<String, Integer> PG_ELEMENT_ORDER_MAP = generateOrderMap(
|
||||
Arrays.asList("processor", "inputPort", "outputPort", "funnel", "processGroup", "remoteProcessGroup", "connection"));
|
||||
private XPathFactory xPathFactory;
|
||||
private Element config;
|
||||
|
||||
@BeforeEach
|
||||
public void setup() {
|
||||
final DocumentProvider documentProvider = new StandardDocumentProvider();
|
||||
final Document document = documentProvider.newDocument();
|
||||
config = document.createElement("config");
|
||||
xPathFactory = XPathFactory.newInstance();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNullQueuePrioritizerNotWritten() throws ConfigurationChangeException, XPathExpressionException {
|
||||
ConfigTransformer.addConnection(config, new ConnectionSchema(Collections.emptyMap()), new ParentGroupIdResolver(new ProcessGroupSchema(Collections.emptyMap(), ConfigSchema.TOP_LEVEL_NAME)));
|
||||
XPath xpath = xPathFactory.newXPath();
|
||||
String expression = "connection/queuePrioritizerClass";
|
||||
assertNull(xpath.evaluate(expression, config, XPathConstants.NODE));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testEmptyQueuePrioritizerNotWritten() throws ConfigurationChangeException, XPathExpressionException {
|
||||
Map<String, Object> map = new HashMap<>();
|
||||
map.put(ConnectionSchema.QUEUE_PRIORITIZER_CLASS_KEY, "");
|
||||
|
||||
ConfigTransformer.addConnection(config, new ConnectionSchema(map), new ParentGroupIdResolver(new ProcessGroupSchema(Collections.emptyMap(), ConfigSchema.TOP_LEVEL_NAME)));
|
||||
XPath xpath = xPathFactory.newXPath();
|
||||
String expression = "connection/queuePrioritizerClass";
|
||||
assertNull(xpath.evaluate(expression, config, XPathConstants.NODE));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testQueuePrioritizerWritten() throws ConfigurationChangeException, XPathExpressionException {
|
||||
Map<String, Object> map = new HashMap<>();
|
||||
map.put(ConnectionSchema.QUEUE_PRIORITIZER_CLASS_KEY, "org.apache.nifi.prioritizer.FirstInFirstOutPrioritizer");
|
||||
|
||||
ConfigTransformer.addConnection(config, new ConnectionSchema(map), new ParentGroupIdResolver(new ProcessGroupSchema(Collections.emptyMap(), ConfigSchema.TOP_LEVEL_NAME)));
|
||||
XPath xpath = xPathFactory.newXPath();
|
||||
String expression = "connection/queuePrioritizerClass/text()";
|
||||
assertEquals("org.apache.nifi.prioritizer.FirstInFirstOutPrioritizer", xpath.evaluate(expression, config, XPathConstants.STRING));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testReportingTasksTransform() throws Exception {
|
||||
testConfigFileTransform("config-reporting-task.yml");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testProcessGroupsTransform() throws Exception {
|
||||
testConfigFileTransform("config-process-groups.yml");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFunnelsTransform() throws Exception {
|
||||
testConfigFileTransform("stress-test-framework-funnel.yml");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFunnelAndRpgTransform() throws Exception {
|
||||
testConfigFileTransform("config-funnel-and-rpg.yml");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRpgTransform() throws Exception {
|
||||
testConfigFileTransform("config-multiple-RPGs.yml");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRpgProxyNoPassTransform() throws Exception {
|
||||
testConfigFileTransform("InvokeHttpMiNiFiProxyNoPasswordTemplateTest.yml");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRpgProxyPassTransform() throws Exception {
|
||||
testConfigFileTransform("InvokeHttpMiNiFiProxyPasswordTemplateTest.yml");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRpgOutputPort() throws Exception {
|
||||
testConfigFileTransform("SimpleRPGToLogAttributes.yml");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNifiPropertiesNoOverrides() throws IOException, ConfigurationChangeException, SchemaLoaderException {
|
||||
Properties pre216Properties = new Properties();
|
||||
try (InputStream pre216PropertiesStream = ConfigTransformerTest.class.getClassLoader().getResourceAsStream("MINIFI-216/nifi.properties.before")) {
|
||||
pre216Properties.load(pre216PropertiesStream);
|
||||
}
|
||||
ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
|
||||
try (InputStream configStream = ConfigTransformerTest.class.getClassLoader().getResourceAsStream("MINIFI-216/config.yml")) {
|
||||
ConfigTransformer.writeNiFiProperties(SchemaLoader.loadConfigSchemaFromYaml(configStream), outputStream);
|
||||
}
|
||||
Properties properties = new Properties();
|
||||
properties.load(new ByteArrayInputStream(outputStream.toByteArray()));
|
||||
|
||||
for (String name : pre216Properties.stringPropertyNames()) {
|
||||
assertEquals(pre216Properties.getProperty(name), properties.getProperty(name), "Property key " + name + " doesn't match.");
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNifiPropertiesOverrides() throws IOException, ConfigurationChangeException, SchemaLoaderException {
|
||||
Properties pre216Properties = new Properties();
|
||||
try (InputStream pre216PropertiesStream = ConfigTransformerTest.class.getClassLoader().getResourceAsStream("MINIFI-216/nifi.properties.before")) {
|
||||
pre216Properties.load(pre216PropertiesStream);
|
||||
}
|
||||
ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
|
||||
try (InputStream configStream = ConfigTransformerTest.class.getClassLoader().getResourceAsStream("MINIFI-216/configOverrides.yml")) {
|
||||
ConfigSchema configSchema = SchemaLoader.loadConfigSchemaFromYaml(configStream);
|
||||
assertTrue(configSchema.getNifiPropertiesOverrides().size() > 0);
|
||||
for (Map.Entry<String, String> entry : configSchema.getNifiPropertiesOverrides().entrySet()) {
|
||||
pre216Properties.setProperty(entry.getKey(), entry.getValue());
|
||||
}
|
||||
ConfigTransformer.writeNiFiProperties(configSchema, outputStream);
|
||||
}
|
||||
Properties properties = new Properties();
|
||||
properties.load(new ByteArrayInputStream(outputStream.toByteArray()));
|
||||
|
||||
for (String name : pre216Properties.stringPropertyNames()) {
|
||||
assertEquals(pre216Properties.getProperty(name), properties.getProperty(name), "Property key " + name + " doesn't match.");
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNifiPropertiesVariableRegistry() throws IOException, ConfigurationChangeException, SchemaLoaderException {
|
||||
Properties initialProperties = new Properties();
|
||||
try (InputStream pre216PropertiesStream = ConfigTransformerTest.class.getClassLoader().getResourceAsStream("MINIFI-277/nifi.properties")) {
|
||||
initialProperties.load(pre216PropertiesStream);
|
||||
}
|
||||
ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
|
||||
try (InputStream configStream = ConfigTransformerTest.class.getClassLoader().getResourceAsStream("MINIFI-277/config.yml")) {
|
||||
ConfigSchema configSchema = SchemaLoader.loadConfigSchemaFromYaml(configStream);
|
||||
ConfigTransformer.writeNiFiProperties(configSchema, outputStream);
|
||||
}
|
||||
Properties properties = new Properties();
|
||||
properties.load(new ByteArrayInputStream(outputStream.toByteArray()));
|
||||
|
||||
for (String name : initialProperties.stringPropertyNames()) {
|
||||
assertEquals(initialProperties.getProperty(name), properties.getProperty(name), "Property key " + name + " doesn't match.");
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void doesTransformFile() throws Exception {
|
||||
File inputFile = new File("./src/test/resources/config.yml");
|
||||
ConfigTransformer.transformConfigFile(new FileInputStream(inputFile), "./target/", null);
|
||||
File nifiPropertiesFile = new File("./target/nifi.properties");
|
||||
|
||||
assertTrue(nifiPropertiesFile.exists());
|
||||
assertTrue(nifiPropertiesFile.canRead());
|
||||
|
||||
nifiPropertiesFile.deleteOnExit();
|
||||
|
||||
File flowXml = new File("./target/flow.xml.gz");
|
||||
assertTrue(flowXml.exists());
|
||||
assertTrue(flowXml.canRead());
|
||||
|
||||
flowXml.deleteOnExit();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void doesTransformV1File() throws Exception {
|
||||
File inputFile = new File("./src/test/resources/config-v1.yml");
|
||||
ConfigTransformer.transformConfigFile(new FileInputStream(inputFile), "./target/", null);
|
||||
File nifiPropertiesFile = new File("./target/nifi.properties");
|
||||
|
||||
assertTrue(nifiPropertiesFile.exists());
|
||||
assertTrue(nifiPropertiesFile.canRead());
|
||||
|
||||
nifiPropertiesFile.deleteOnExit();
|
||||
|
||||
File flowXml = new File("./target/flow.xml.gz");
|
||||
assertTrue(flowXml.exists());
|
||||
assertTrue(flowXml.canRead());
|
||||
|
||||
flowXml.deleteOnExit();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void doesTransformInputStream() throws Exception {
|
||||
File inputFile = new File("./src/test/resources/config.yml");
|
||||
ConfigTransformer.transformConfigFile(new FileInputStream(inputFile), "./target/", null);
|
||||
|
||||
File nifiPropertiesFile = new File("./target/nifi.properties");
|
||||
assertTrue(nifiPropertiesFile.exists());
|
||||
assertTrue(nifiPropertiesFile.canRead());
|
||||
|
||||
nifiPropertiesFile.deleteOnExit();
|
||||
|
||||
File flowXml = new File("./target/flow.xml.gz");
|
||||
assertTrue(flowXml.exists());
|
||||
assertTrue(flowXml.canRead());
|
||||
|
||||
flowXml.deleteOnExit();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void doesTransformOnDefaultFile() throws Exception {
|
||||
File inputFile = new File("./src/test/resources/default.yml");
|
||||
ConfigTransformer.transformConfigFile(new FileInputStream(inputFile), "./target/", null);
|
||||
File nifiPropertiesFile = new File("./target/nifi.properties");
|
||||
|
||||
assertTrue(nifiPropertiesFile.exists());
|
||||
assertTrue(nifiPropertiesFile.canRead());
|
||||
|
||||
nifiPropertiesFile.deleteOnExit();
|
||||
|
||||
File flowXml = new File("./target/flow.xml.gz");
|
||||
assertTrue(flowXml.exists());
|
||||
assertTrue(flowXml.canRead());
|
||||
|
||||
flowXml.deleteOnExit();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void doesTransformOnMultipleProcessors() throws Exception {
|
||||
File inputFile = new File("./src/test/resources/config-multiple-processors.yml");
|
||||
ConfigTransformer.transformConfigFile(new FileInputStream(inputFile), "./target/", null);
|
||||
File nifiPropertiesFile = new File("./target/nifi.properties");
|
||||
|
||||
assertTrue(nifiPropertiesFile.exists());
|
||||
assertTrue(nifiPropertiesFile.canRead());
|
||||
|
||||
nifiPropertiesFile.deleteOnExit();
|
||||
|
||||
File flowXml = new File("./target/flow.xml.gz");
|
||||
assertTrue(flowXml.exists());
|
||||
assertTrue(flowXml.canRead());
|
||||
|
||||
flowXml.deleteOnExit();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void doesTransformOnMultipleRemoteProcessGroups() throws Exception {
|
||||
File inputFile = new File("./src/test/resources/config-multiple-RPGs.yml");
|
||||
ConfigTransformer.transformConfigFile(new FileInputStream(inputFile), "./target/", null);
|
||||
File nifiPropertiesFile = new File("./target/nifi.properties");
|
||||
|
||||
assertTrue(nifiPropertiesFile.exists());
|
||||
assertTrue(nifiPropertiesFile.canRead());
|
||||
|
||||
nifiPropertiesFile.deleteOnExit();
|
||||
|
||||
File flowXml = new File("./target/flow.xml.gz");
|
||||
assertTrue(flowXml.exists());
|
||||
assertTrue(flowXml.canRead());
|
||||
|
||||
flowXml.deleteOnExit();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void doesTransformOnMultipleInputPorts() throws Exception {
|
||||
File inputFile = new File("./src/test/resources/config-multiple-input-ports.yml");
|
||||
ConfigTransformer.transformConfigFile(new FileInputStream(inputFile), "./target/", null);
|
||||
File nifiPropertiesFile = new File("./target/nifi.properties");
|
||||
|
||||
assertTrue(nifiPropertiesFile.exists());
|
||||
assertTrue(nifiPropertiesFile.canRead());
|
||||
|
||||
nifiPropertiesFile.deleteOnExit();
|
||||
|
||||
File flowXml = new File("./target/flow.xml.gz");
|
||||
assertTrue(flowXml.exists());
|
||||
assertTrue(flowXml.canRead());
|
||||
|
||||
flowXml.deleteOnExit();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void doesTransformOnMinimal() throws Exception {
|
||||
File inputFile = new File("./src/test/resources/config-minimal.yml");
|
||||
ConfigTransformer.transformConfigFile(new FileInputStream(inputFile), "./target/", null);
|
||||
File nifiPropertiesFile = new File("./target/nifi.properties");
|
||||
|
||||
assertTrue(nifiPropertiesFile.exists());
|
||||
assertTrue(nifiPropertiesFile.canRead());
|
||||
|
||||
nifiPropertiesFile.deleteOnExit();
|
||||
|
||||
File flowXml = new File("./target/flow.xml.gz");
|
||||
assertTrue(flowXml.exists());
|
||||
assertTrue(flowXml.canRead());
|
||||
|
||||
flowXml.deleteOnExit();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void doesTransformOnProvenanceRepository() throws Exception {
|
||||
File inputFile = new File("./src/test/resources/config-provenance-repository.yml");
|
||||
ConfigTransformer.transformConfigFile(new FileInputStream(inputFile), "./target/", null);
|
||||
File nifiPropertiesFile = new File("./target/nifi.properties");
|
||||
|
||||
assertTrue(nifiPropertiesFile.exists());
|
||||
assertTrue(nifiPropertiesFile.canRead());
|
||||
|
||||
String nifi = FileUtils.readFileToString(nifiPropertiesFile, Charset.defaultCharset());
|
||||
assertTrue(nifi.contains("nifi.provenance.repository.implementation=org.apache.nifi.provenance.WriteAheadProvenanceRepository"));
|
||||
|
||||
nifiPropertiesFile.deleteOnExit();
|
||||
|
||||
File flowXml = new File("./target/flow.xml.gz");
|
||||
assertTrue(flowXml.exists());
|
||||
assertTrue(flowXml.canRead());
|
||||
|
||||
flowXml.deleteOnExit();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void doesTransformOnCustomProvenanceRepository() throws Exception {
|
||||
File inputFile = new File("./src/test/resources/config-provenance-custom-repository.yml");
|
||||
ConfigTransformer.transformConfigFile(new FileInputStream(inputFile), "./target/", null);
|
||||
File nifiPropertiesFile = new File("./target/nifi.properties");
|
||||
|
||||
assertTrue(nifiPropertiesFile.exists());
|
||||
assertTrue(nifiPropertiesFile.canRead());
|
||||
|
||||
String nifi = FileUtils.readFileToString(nifiPropertiesFile, Charset.defaultCharset());
|
||||
assertTrue(nifi.contains("nifi.provenance.repository.implementation=org.apache.nifi.provenance.CustomProvenanceRepository"));
|
||||
|
||||
nifiPropertiesFile.deleteOnExit();
|
||||
|
||||
File flowXml = new File("./target/flow.xml.gz");
|
||||
assertTrue(flowXml.exists());
|
||||
assertTrue(flowXml.canRead());
|
||||
|
||||
flowXml.deleteOnExit();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void handleTransformInvalidFile() throws Exception {
|
||||
try {
|
||||
File inputFile = new File("./src/test/resources/config-invalid.yml");
|
||||
ConfigTransformer.transformConfigFile(new FileInputStream(inputFile), "./target/", null);
|
||||
fail("Invalid configuration file was not detected.");
|
||||
} catch (SchemaLoaderException e) {
|
||||
assertEquals("Provided YAML configuration is not a Map", e.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void handleTransformMalformedField() throws Exception {
|
||||
try {
|
||||
File inputFile = new File("./src/test/resources/config-malformed-field.yml");
|
||||
ConfigTransformer.transformConfigFile(new FileInputStream(inputFile), "./target/", null);
|
||||
fail("Invalid configuration file was not detected.");
|
||||
} catch (InvalidConfigurationException e) {
|
||||
assertEquals("Failed to transform config file due to:['threshold' in section 'Swap' because it is found but could not be parsed as a Number]", e.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void handleTransformEmptyFile() throws Exception {
|
||||
try {
|
||||
File inputFile = new File("./src/test/resources/config-empty.yml");
|
||||
ConfigTransformer.transformConfigFile(new FileInputStream(inputFile), "./target/", null);
|
||||
fail("Invalid configuration file was not detected.");
|
||||
} catch (SchemaLoaderException e) {
|
||||
assertEquals("Provided YAML configuration is not a Map", e.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void handleTransformFileMissingRequiredField() throws Exception {
|
||||
try {
|
||||
File inputFile = new File("./src/test/resources/config-missing-required-field.yml");
|
||||
ConfigTransformer.transformConfigFile(new FileInputStream(inputFile), "./target/", null);
|
||||
fail("Invalid configuration file was not detected.");
|
||||
} catch (InvalidConfigurationException e) {
|
||||
assertEquals("Failed to transform config file due to:['class' in section 'Processors' because it was not found and it is required]", e.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void handleTransformFileMultipleProblems() throws Exception {
|
||||
try {
|
||||
File inputFile = new File("./src/test/resources/config-multiple-problems.yml");
|
||||
ConfigTransformer.transformConfigFile(new FileInputStream(inputFile), "./target/", null);
|
||||
fail("Invalid configuration file was not detected.");
|
||||
} catch (InvalidConfigurationException e) {
|
||||
assertEquals("Failed to transform config file due to:['class' in section 'Processors' because it was not found and it is required], " +
|
||||
"['scheduling strategy' in section 'Provenance Reporting' because it is not a valid scheduling strategy], " +
|
||||
"['source name' in section 'Connections' because it was not found and it is required]", e.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void checkSSLOverrides() throws Exception {
|
||||
File inputFile = new File("./src/test/resources/MINIFI-516/config.yml");
|
||||
final Properties bootstrapProperties = getTestBootstrapProperties("MINIFI-516/bootstrap.conf");
|
||||
ConfigTransformer.transformConfigFile(new FileInputStream(inputFile), "./target/", bootstrapProperties);
|
||||
|
||||
// nifi.properties testing
|
||||
File nifiPropertiesFile = new File("./target/nifi.properties");
|
||||
assertTrue(nifiPropertiesFile.exists());
|
||||
assertTrue(nifiPropertiesFile.canRead());
|
||||
|
||||
nifiPropertiesFile.deleteOnExit();
|
||||
|
||||
// flow.xml.gz testing
|
||||
File flowXml = new File("./target/flow.xml.gz");
|
||||
assertTrue(flowXml.exists());
|
||||
assertTrue(flowXml.canRead());
|
||||
|
||||
final DocumentProvider documentProvider = new StandardDocumentProvider();
|
||||
final Document xml;
|
||||
try (final InputStream inputStream = new GZIPInputStream(new FileInputStream(flowXml))) {
|
||||
xml = documentProvider.parse(inputStream);
|
||||
}
|
||||
|
||||
XPath xPath = XPathFactory.newInstance().newXPath();
|
||||
String result = xPath.evaluate("/flowController/rootGroup/processor/property[name = \"SSL Context Service\"]/value/text()", xml);
|
||||
|
||||
assertEquals(result, "SSL-Context-Service");
|
||||
|
||||
flowXml.deleteOnExit();
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void checkLogAndConfigFilePathOverrides() throws Exception {
|
||||
File inputFile = new File("./src/test/resources/config-minimal.yml");
|
||||
Properties bootstrapProperties = getTestBootstrapProperties("bootstrap.conf");
|
||||
ConfigTransformer.transformConfigFile(new FileInputStream(inputFile), "./target/", bootstrapProperties);
|
||||
|
||||
File nifiPropertiesFile = new File("./target/nifi.properties");
|
||||
|
||||
assertTrue(nifiPropertiesFile.exists());
|
||||
assertTrue(nifiPropertiesFile.canRead());
|
||||
|
||||
Properties nifiProperties = new Properties();
|
||||
nifiProperties.load(new FileInputStream(nifiPropertiesFile));
|
||||
|
||||
// an absolute path is returned for bootstrap which is different per every environment. only testing that path ends with expected relative path
|
||||
assertTrue(nifiProperties.getProperty(MINIFI_BOOTSTRAP_FILE_PATH).endsWith("bootstrap.conf"));
|
||||
assertEquals("./conf/config.yml", nifiProperties.getProperty(MINIFI_CONFIG_FILE_PATH));
|
||||
assertEquals("./logs", nifiProperties.getProperty(MINIFI_LOG_DIRECTORY));
|
||||
assertEquals("minifi-app.log", nifiProperties.getProperty(MINIFI_APP_LOG_FILE));
|
||||
assertEquals("minifi-bootstrap.log", nifiProperties.getProperty(MINIFI_BOOTSTRAP_LOG_FILE));
|
||||
}
|
||||
|
||||
public void testConfigFileTransform(String configFile) throws Exception {
|
||||
ConfigSchema configSchema = SchemaLoader.loadConfigSchemaFromYaml(ConfigTransformerTest.class.getClassLoader().getResourceAsStream(configFile));
|
||||
|
||||
ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
|
||||
ConfigTransformer.writeFlowXmlFile(configSchema, outputStream);
|
||||
final DocumentProvider documentProvider = new StandardDocumentProvider();
|
||||
Document document = documentProvider.parse(new ByteArrayInputStream(outputStream.toByteArray()));
|
||||
|
||||
testProcessGroup((Element) xPathFactory.newXPath().evaluate("flowController/rootGroup", document, XPathConstants.NODE), configSchema.getProcessGroupSchema());
|
||||
testReportingTasks((Element) xPathFactory.newXPath().evaluate("flowController/reportingTasks", document, XPathConstants.NODE), configSchema.getReportingTasksSchema());
|
||||
}
|
||||
|
||||
private void testProcessGroup(Element element, ProcessGroupSchema processGroupSchema) throws XPathExpressionException {
|
||||
assertEquals(processGroupSchema.getId(), getText(element, "id"));
|
||||
assertEquals(processGroupSchema.getName(), getText(element, "name"));
|
||||
assertEquals(nullToEmpty(processGroupSchema.getComment()), nullToEmpty(getText(element, "comment")));
|
||||
|
||||
checkOrderOfChildren(element, PG_ELEMENT_ORDER_MAP);
|
||||
|
||||
NodeList processorElements = (NodeList) xPathFactory.newXPath().evaluate("processor", element, XPathConstants.NODESET);
|
||||
assertEquals(processGroupSchema.getProcessors().size(), processorElements.getLength());
|
||||
for (int i = 0; i < processorElements.getLength(); i++) {
|
||||
testProcessor((Element) processorElements.item(i), processGroupSchema.getProcessors().get(i));
|
||||
}
|
||||
NodeList controllerServiceElements = (NodeList) xPathFactory.newXPath().evaluate("controllerService", element, XPathConstants.NODESET);
|
||||
assertEquals(processGroupSchema.getControllerServices().size(), controllerServiceElements.getLength());
|
||||
for (int i = 0; i < controllerServiceElements.getLength(); i++) {
|
||||
testControllerService((Element) controllerServiceElements.item(i), processGroupSchema.getControllerServices().get(i));
|
||||
}
|
||||
|
||||
NodeList remoteProcessGroupElements = (NodeList) xPathFactory.newXPath().evaluate("remoteProcessGroup", element, XPathConstants.NODESET);
|
||||
assertEquals(processGroupSchema.getRemoteProcessGroups().size(), remoteProcessGroupElements.getLength());
|
||||
for (int i = 0; i < remoteProcessGroupElements.getLength(); i++) {
|
||||
testRemoteProcessGroups((Element) remoteProcessGroupElements.item(i), processGroupSchema.getRemoteProcessGroups().get(i));
|
||||
}
|
||||
|
||||
NodeList funnelElements = (NodeList) xPathFactory.newXPath().evaluate("funnel", element, XPathConstants.NODESET);
|
||||
assertEquals(processGroupSchema.getFunnels().size(), funnelElements.getLength());
|
||||
for (int i = 0; i < funnelElements.getLength(); i++) {
|
||||
testFunnel((Element) funnelElements.item(i), processGroupSchema.getFunnels().get(i));
|
||||
}
|
||||
|
||||
NodeList inputPortElements = (NodeList) xPathFactory.newXPath().evaluate("inputPort", element, XPathConstants.NODESET);
|
||||
assertEquals(processGroupSchema.getInputPortSchemas().size(), inputPortElements.getLength());
|
||||
for (int i = 0; i < inputPortElements.getLength(); i++) {
|
||||
testPort((Element) inputPortElements.item(i), processGroupSchema.getInputPortSchemas().get(i));
|
||||
}
|
||||
|
||||
NodeList outputPortElements = (NodeList) xPathFactory.newXPath().evaluate("outputPort", element, XPathConstants.NODESET);
|
||||
assertEquals(processGroupSchema.getOutputPortSchemas().size(), outputPortElements.getLength());
|
||||
for (int i = 0; i < outputPortElements.getLength(); i++) {
|
||||
testPort((Element) outputPortElements.item(i), processGroupSchema.getOutputPortSchemas().get(i));
|
||||
}
|
||||
|
||||
NodeList processGroupElements = (NodeList) xPathFactory.newXPath().evaluate("processGroup", element, XPathConstants.NODESET);
|
||||
assertEquals(processGroupSchema.getProcessGroupSchemas().size(), processGroupElements.getLength());
|
||||
for (int i = 0; i < processGroupElements.getLength(); i++) {
|
||||
testProcessGroup((Element) processGroupElements.item(i), processGroupSchema.getProcessGroupSchemas().get(i));
|
||||
}
|
||||
|
||||
NodeList connectionElements = (NodeList) xPathFactory.newXPath().evaluate("connection", element, XPathConstants.NODESET);
|
||||
assertEquals(processGroupSchema.getConnections().size(), connectionElements.getLength());
|
||||
for (int i = 0; i < connectionElements.getLength(); i++) {
|
||||
testConnection((Element) connectionElements.item(i), processGroupSchema.getConnections().get(i));
|
||||
}
|
||||
}
|
||||
|
||||
private void testProcessor(Element element, ProcessorSchema processorSchema) throws XPathExpressionException {
|
||||
assertEquals(processorSchema.getId(), getText(element, "id"));
|
||||
assertEquals(processorSchema.getName(), getText(element, "name"));
|
||||
assertEquals(processorSchema.getProcessorClass(), getText(element, "class"));
|
||||
assertEquals(processorSchema.getMaxConcurrentTasks().toString(), getText(element, "maxConcurrentTasks"));
|
||||
assertEquals(processorSchema.getSchedulingPeriod(), getText(element, "schedulingPeriod"));
|
||||
assertEquals(processorSchema.getPenalizationPeriod(), getText(element, "penalizationPeriod"));
|
||||
assertEquals(processorSchema.getYieldPeriod(), getText(element, "yieldPeriod"));
|
||||
assertEquals(processorSchema.getSchedulingStrategy(), getText(element, "schedulingStrategy"));
|
||||
assertEquals(processorSchema.getRunDurationNanos().toString(), getText(element, "runDurationNanos"));
|
||||
assertEquals(processorSchema.getAnnotationData(), getText(element, "annotationData"));
|
||||
|
||||
testProperties(element, processorSchema.getProperties());
|
||||
}
|
||||
|
||||
private void testReportingTasks(Element element, List<ReportingSchema> reportingSchemas) throws XPathExpressionException {
|
||||
NodeList taskElements = (NodeList) xPathFactory.newXPath().evaluate("reportingTask", element, XPathConstants.NODESET);
|
||||
assertEquals(reportingSchemas.size(), taskElements.getLength());
|
||||
for (int i = 0; i < taskElements.getLength(); i++) {
|
||||
testReportingTask((Element) taskElements.item(i), reportingSchemas.get(i));
|
||||
}
|
||||
}
|
||||
|
||||
private void testReportingTask(Element element, ReportingSchema reportingSchema) throws XPathExpressionException {
|
||||
assertEquals(reportingSchema.getId(), getText(element, "id"));
|
||||
assertEquals(reportingSchema.getName(), getText(element, "name"));
|
||||
assertEquals(reportingSchema.getComment(), getText(element, "comment"));
|
||||
assertEquals(reportingSchema.getReportingClass(), getText(element, "class"));
|
||||
assertEquals(reportingSchema.getSchedulingPeriod(), getText(element, "schedulingPeriod"));
|
||||
assertEquals(reportingSchema.getSchedulingStrategy(), getText(element, "schedulingStrategy"));
|
||||
|
||||
testProperties(element, reportingSchema.getProperties());
|
||||
}
|
||||
|
||||
private void testControllerService(Element element, ControllerServiceSchema controllerServiceSchema) throws XPathExpressionException {
|
||||
assertEquals(controllerServiceSchema.getId(), getText(element, "id"));
|
||||
assertEquals(controllerServiceSchema.getName(), getText(element, "name"));
|
||||
assertEquals(controllerServiceSchema.getServiceClass(), getText(element, "class"));
|
||||
assertEquals(controllerServiceSchema.getAnnotationData(), getText(element, "annotationData"));
|
||||
|
||||
testProperties(element, controllerServiceSchema.getProperties());
|
||||
}
|
||||
|
||||
private void testRemoteProcessGroups(Element element, RemoteProcessGroupSchema remoteProcessingGroupSchema) throws XPathExpressionException {
|
||||
assertEquals(remoteProcessingGroupSchema.getId(), getText(element, "id"));
|
||||
assertEquals(remoteProcessingGroupSchema.getName(), getText(element, "name"));
|
||||
assertEquals(remoteProcessingGroupSchema.getComment(), getText(element, "comment"));
|
||||
assertEquals(remoteProcessingGroupSchema.getUrls(), getText(element, "url"));
|
||||
assertEquals(remoteProcessingGroupSchema.getTimeout(), getText(element, "timeout"));
|
||||
assertEquals(remoteProcessingGroupSchema.getYieldPeriod(), getText(element, "yieldPeriod"));
|
||||
assertEquals(remoteProcessingGroupSchema.getTransportProtocol(), getText(element, "transportProtocol"));
|
||||
assertEquals(remoteProcessingGroupSchema.getProxyHost(), getText(element, "proxyHost"));
|
||||
String proxyPortText = getText(element, "proxyPort");
|
||||
assertEquals(remoteProcessingGroupSchema.getProxyPort(), StringUtil.isNullOrEmpty(proxyPortText) ? null : Integer.parseInt(proxyPortText));
|
||||
assertEquals(remoteProcessingGroupSchema.getProxyUser(), getText(element, "proxyUser"));
|
||||
assertEquals(remoteProcessingGroupSchema.getProxyPassword(), getText(element, "proxyPassword"));
|
||||
|
||||
NodeList inputPortElements = (NodeList) xPathFactory.newXPath().evaluate("inputPort", element, XPathConstants.NODESET);
|
||||
assertEquals(remoteProcessingGroupSchema.getInputPorts().size(), inputPortElements.getLength());
|
||||
for (int i = 0; i < inputPortElements.getLength(); i++) {
|
||||
testRemotePort((Element) inputPortElements.item(i), remoteProcessingGroupSchema.getInputPorts().get(i));
|
||||
}
|
||||
|
||||
NodeList outputPortElements = (NodeList) xPathFactory.newXPath().evaluate("outputPort", element, XPathConstants.NODESET);
|
||||
assertEquals(remoteProcessingGroupSchema.getOutputPorts().size(), outputPortElements.getLength());
|
||||
for (int i = 0; i < outputPortElements.getLength(); i++) {
|
||||
testRemotePort((Element) outputPortElements.item(i), remoteProcessingGroupSchema.getOutputPorts().get(i));
|
||||
}
|
||||
}
|
||||
|
||||
private void testRemotePort(Element element, RemotePortSchema remoteInputPortSchema) throws XPathExpressionException {
|
||||
assertEquals(remoteInputPortSchema.getId(), getText(element, "id"));
|
||||
assertEquals(remoteInputPortSchema.getName(), getText(element, "name"));
|
||||
assertEquals(remoteInputPortSchema.getComment(), getText(element, "comment"));
|
||||
assertEquals(remoteInputPortSchema.getMax_concurrent_tasks().toString(), getText(element, "maxConcurrentTasks"));
|
||||
assertEquals(remoteInputPortSchema.getUseCompression(), Boolean.parseBoolean(getText(element, "useCompression")));
|
||||
}
|
||||
|
||||
private void testPort(Element element, PortSchema portSchema) throws XPathExpressionException {
|
||||
assertEquals(portSchema.getId(), getText(element, "id"));
|
||||
assertEquals(portSchema.getName(), getText(element, "name"));
|
||||
assertEquals("RUNNING", getText(element, "scheduledState"));
|
||||
}
|
||||
|
||||
private void testFunnel(Element element, FunnelSchema funnelSchema) throws XPathExpressionException {
|
||||
assertEquals(funnelSchema.getId(), getText(element, "id"));
|
||||
}
|
||||
|
||||
private void testConnection(Element element, ConnectionSchema connectionSchema) throws XPathExpressionException {
|
||||
assertEquals(connectionSchema.getId(), getText(element, "id"));
|
||||
assertEquals(connectionSchema.getName(), getText(element, "name"));
|
||||
|
||||
assertEquals(connectionSchema.getSourceId(), getText(element, "sourceId"));
|
||||
assertEquals(connectionSchema.getDestinationId(), getText(element, "destinationId"));
|
||||
|
||||
NodeList relationshipNodes = (NodeList) xPathFactory.newXPath().evaluate("relationship", element, XPathConstants.NODESET);
|
||||
Set<String> sourceRelationships = new HashSet<>();
|
||||
for (int i = 0; i < relationshipNodes.getLength(); i++) {
|
||||
String textContent = relationshipNodes.item(i).getTextContent();
|
||||
if (!StringUtil.isNullOrEmpty(textContent)) {
|
||||
sourceRelationships.add(textContent);
|
||||
}
|
||||
}
|
||||
|
||||
assertEquals(new HashSet<>(connectionSchema.getSourceRelationshipNames()), sourceRelationships);
|
||||
|
||||
assertEquals(connectionSchema.getMaxWorkQueueSize().toString(), getText(element, "maxWorkQueueSize"));
|
||||
assertEquals(connectionSchema.getMaxWorkQueueDataSize(), getText(element, "maxWorkQueueDataSize"));
|
||||
assertEquals(connectionSchema.getFlowfileExpiration(), getText(element, "flowFileExpiration"));
|
||||
assertEquals(connectionSchema.getQueuePrioritizerClass(), getText(element, "queuePrioritizerClass"));
|
||||
}
|
||||
|
||||
private void testProperties(Element element, Map<String, Object> expected) throws XPathExpressionException {
|
||||
NodeList propertyElements = (NodeList) xPathFactory.newXPath().evaluate("property", element, XPathConstants.NODESET);
|
||||
Map<String, String> properties = new HashMap<>();
|
||||
for (int i = 0; i < propertyElements.getLength(); i++) {
|
||||
Element item = (Element) propertyElements.item(i);
|
||||
properties.put(getText(item, "name"), getText(item, "value"));
|
||||
}
|
||||
assertEquals(expected.entrySet().stream().collect(Collectors.toMap(Map.Entry<String, Object>::getKey, e -> nullToEmpty(e.getValue()))), properties);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testContentRepoOverride() throws IOException, ConfigurationChangeException, SchemaLoaderException {
|
||||
Properties pre216Properties = new Properties();
|
||||
try (InputStream pre216PropertiesStream = ConfigTransformerTest.class.getClassLoader().getResourceAsStream("MINIFI-245/nifi.properties.before")) {
|
||||
pre216Properties.load(pre216PropertiesStream);
|
||||
}
|
||||
ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
|
||||
try (InputStream configStream = ConfigTransformerTest.class.getClassLoader().getResourceAsStream("MINIFI-245/config.yml")) {
|
||||
ConfigTransformer.writeNiFiProperties(SchemaLoader.loadConfigSchemaFromYaml(configStream), outputStream);
|
||||
}
|
||||
Properties properties = new Properties();
|
||||
properties.load(new ByteArrayInputStream(outputStream.toByteArray()));
|
||||
|
||||
for (String name : pre216Properties.stringPropertyNames()) {
|
||||
// Verify the Content Repo property was overridden
|
||||
if ("nifi.content.repository.implementation".equals(name)) {
|
||||
assertNotEquals(pre216Properties.getProperty(name), properties.getProperty(name), "Property key " + name + " was not overridden.");
|
||||
} else {
|
||||
assertEquals(pre216Properties.getProperty(name), properties.getProperty(name), "Property key " + name + " doesn't match.");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFlowFileRepoOverride() throws IOException, ConfigurationChangeException, SchemaLoaderException {
|
||||
Properties pre216Properties = new Properties();
|
||||
try (InputStream pre216PropertiesStream = ConfigTransformerTest.class.getClassLoader().getResourceAsStream("NIFI-8753/nifi.properties.before")) {
|
||||
pre216Properties.load(pre216PropertiesStream);
|
||||
}
|
||||
ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
|
||||
try (InputStream configStream = ConfigTransformerTest.class.getClassLoader().getResourceAsStream("NIFI-8753/config.yml")) {
|
||||
ConfigTransformer.writeNiFiProperties(SchemaLoader.loadConfigSchemaFromYaml(configStream), outputStream);
|
||||
}
|
||||
Properties properties = new Properties();
|
||||
properties.load(new ByteArrayInputStream(outputStream.toByteArray()));
|
||||
|
||||
for (String name : pre216Properties.stringPropertyNames()) {
|
||||
// Verify the Content Repo property was overridden
|
||||
if ("nifi.flowfile.repository.implementation".equals(name)) {
|
||||
assertNotEquals(pre216Properties.getProperty(name), properties.getProperty(name), "Property key " + name + " was not overridden.");
|
||||
} else {
|
||||
assertEquals(pre216Properties.getProperty(name), properties.getProperty(name), "Property key " + name + " doesn't match.");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNullSensitiveKey() throws IOException, ConfigurationChangeException, SchemaLoaderException {
|
||||
final ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
|
||||
try (final InputStream configStream = ConfigTransformerTest.class.getClassLoader().getResourceAsStream("MINIFI-537/config.yml")) {
|
||||
ConfigTransformer.writeNiFiProperties(SchemaLoader.loadConfigSchemaFromYaml(configStream), outputStream);
|
||||
}
|
||||
final Properties properties = new Properties();
|
||||
properties.load(new ByteArrayInputStream(outputStream.toByteArray()));
|
||||
// The property should not be empty/null as it is auto-generated when missing
|
||||
assertTrue(StringUtils.isNotEmpty(properties.getProperty("nifi.sensitive.props.key")));
|
||||
}
|
||||
|
||||
private String getText(Element element, String path) throws XPathExpressionException {
|
||||
return (String) xPathFactory.newXPath().evaluate(path + "/text()", element, XPathConstants.STRING);
|
||||
}
|
||||
|
||||
private String nullToEmpty(Object val) {
|
||||
return val == null ? "" : val.toString();
|
||||
}
|
||||
|
||||
private static Map<String, Integer> generateOrderMap(List<String> elements) {
|
||||
Map<String, Integer> map = new HashMap<>();
|
||||
for (int i = 0; i < elements.size(); i++) {
|
||||
map.put(elements.get(i), i);
|
||||
}
|
||||
return Collections.unmodifiableMap(map);
|
||||
}
|
||||
|
||||
private static void checkOrderOfChildren(Element element, Map<String, Integer> orderMap) {
|
||||
int elementOrderList = 0;
|
||||
NodeList childNodes = element.getChildNodes();
|
||||
String lastOrderedElementName = null;
|
||||
for (int i = 0; i < childNodes.getLength(); i++) {
|
||||
String nodeName = childNodes.item(i).getNodeName();
|
||||
Integer index = orderMap.get(nodeName);
|
||||
if (index != null) {
|
||||
if (elementOrderList > index) {
|
||||
fail("Found " + nodeName + " after " + lastOrderedElementName + "; expected all " + nodeName + " elements to come before the following elements: " + orderMap.entrySet().stream()
|
||||
.filter(e -> e.getValue() > index).sorted(Comparator.comparingInt(Map.Entry::getValue)).map(Map.Entry::getKey).collect(Collectors.joining(", ")));
|
||||
}
|
||||
lastOrderedElementName = nodeName;
|
||||
elementOrderList = index;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public static Properties getTestBootstrapProperties(final String fileName) throws IOException {
|
||||
final Properties bootstrapProperties = new Properties();
|
||||
try (final InputStream fis = ConfigTransformerTest.class.getClassLoader().getResourceAsStream(fileName)) {
|
||||
bootstrapProperties.load(fis);
|
||||
}
|
||||
return bootstrapProperties;
|
||||
}
|
||||
}
|
|
@ -17,6 +17,7 @@
|
|||
|
||||
package org.apache.nifi.minifi.bootstrap.util;
|
||||
|
||||
import org.apache.nifi.minifi.bootstrap.service.MiNiFiPropertiesGenerator;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.io.BufferedReader;
|
||||
|
@ -25,7 +26,6 @@ import java.io.ByteArrayOutputStream;
|
|||
import java.io.IOException;
|
||||
import java.io.InputStreamReader;
|
||||
|
||||
import static org.apache.nifi.minifi.bootstrap.util.ConfigTransformer.PROPERTIES_FILE_APACHE_2_0_LICENSE;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertNull;
|
||||
|
||||
|
@ -41,7 +41,7 @@ public class OrderedPropertiesTest {
|
|||
orderedProperties.setProperty("prop3", "newVal3");
|
||||
orderedProperties.setProperty("prop1", "newVal1");
|
||||
ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
|
||||
orderedProperties.store(byteArrayOutputStream, PROPERTIES_FILE_APACHE_2_0_LICENSE);
|
||||
orderedProperties.store(byteArrayOutputStream, MiNiFiPropertiesGenerator.PROPERTIES_FILE_APACHE_2_0_LICENSE);
|
||||
|
||||
try (BufferedReader actualReader = new BufferedReader(new InputStreamReader(new ByteArrayInputStream(byteArrayOutputStream.toByteArray())));
|
||||
BufferedReader expectedReader = new BufferedReader(new InputStreamReader(OrderedPropertiesTest.class.getClassLoader().getResourceAsStream("orderedPropertiesExpected.properties")))) {
|
||||
|
|
|
@ -1,200 +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.util;
|
||||
|
||||
import org.apache.nifi.minifi.commons.schema.exception.SchemaLoaderException;
|
||||
import org.apache.nifi.minifi.commons.schema.serialization.SchemaLoader;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.io.ByteArrayInputStream;
|
||||
import java.io.IOException;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertNull;
|
||||
|
||||
public class ParentGroupIdResolverTest {
|
||||
|
||||
@Test
|
||||
public void testRemoteInputPortParentId() throws IOException, SchemaLoaderException {
|
||||
List<String> configLines = new ArrayList<>();
|
||||
configLines.add("MiNiFi Config Version: 2");
|
||||
configLines.add("Remote Process Groups:");
|
||||
configLines.add("- name: rpgOne");
|
||||
configLines.add(" id: rpgOneId");
|
||||
configLines.add(" Input Ports:");
|
||||
configLines.add(" - id: one");
|
||||
configLines.add("Process Groups:");
|
||||
configLines.add("- Remote Process Groups:");
|
||||
configLines.add(" - name: rpgTwo");
|
||||
configLines.add(" id: rpgTwoId");
|
||||
configLines.add(" Input Ports:");
|
||||
configLines.add(" - id: two");
|
||||
ParentGroupIdResolver parentGroupIdResolver = createParentGroupIdResolver(configLines);
|
||||
assertEquals("rpgOneId", parentGroupIdResolver.getRemoteInputPortParentId("one"));
|
||||
assertEquals("rpgTwoId", parentGroupIdResolver.getRemoteInputPortParentId("two"));
|
||||
assertNull(parentGroupIdResolver.getRemoteOutputPortParentId("one"));
|
||||
assertNull(parentGroupIdResolver.getRemoteOutputPortParentId("two"));
|
||||
assertNull(parentGroupIdResolver.getInputPortParentId("one"));
|
||||
assertNull(parentGroupIdResolver.getInputPortParentId("two"));
|
||||
assertNull(parentGroupIdResolver.getOutputPortParentId("one"));
|
||||
assertNull(parentGroupIdResolver.getOutputPortParentId("two"));
|
||||
assertNull(parentGroupIdResolver.getProcessorParentId("one"));
|
||||
assertNull(parentGroupIdResolver.getProcessorParentId("two"));
|
||||
assertNull(parentGroupIdResolver.getFunnelParentId("one"));
|
||||
assertNull(parentGroupIdResolver.getFunnelParentId("two"));
|
||||
}
|
||||
@Test
|
||||
public void testRemoteOutputPortParentId() throws IOException, SchemaLoaderException {
|
||||
List<String> configLines = new ArrayList<>();
|
||||
configLines.add("MiNiFi Config Version: 3");
|
||||
configLines.add("Remote Process Groups:");
|
||||
configLines.add("- name: rpgOne");
|
||||
configLines.add(" id: rpgOneId");
|
||||
configLines.add(" Output Ports:");
|
||||
configLines.add(" - id: one");
|
||||
configLines.add("Process Groups:");
|
||||
configLines.add("- Remote Process Groups:");
|
||||
configLines.add(" - name: rpgTwo");
|
||||
configLines.add(" id: rpgTwoId");
|
||||
configLines.add(" Output Ports:");
|
||||
configLines.add(" - id: two");
|
||||
ParentGroupIdResolver parentGroupIdResolver = createParentGroupIdResolver(configLines);
|
||||
assertNull(parentGroupIdResolver.getRemoteInputPortParentId("one"));
|
||||
assertNull(parentGroupIdResolver.getRemoteInputPortParentId("two"));
|
||||
assertEquals("rpgOneId", parentGroupIdResolver.getRemoteOutputPortParentId("one"));
|
||||
assertEquals("rpgTwoId", parentGroupIdResolver.getRemoteOutputPortParentId("two"));
|
||||
assertNull(parentGroupIdResolver.getInputPortParentId("one"));
|
||||
assertNull(parentGroupIdResolver.getInputPortParentId("two"));
|
||||
assertNull(parentGroupIdResolver.getOutputPortParentId("one"));
|
||||
assertNull(parentGroupIdResolver.getOutputPortParentId("two"));
|
||||
assertNull(parentGroupIdResolver.getProcessorParentId("one"));
|
||||
assertNull(parentGroupIdResolver.getProcessorParentId("two"));
|
||||
assertNull(parentGroupIdResolver.getFunnelParentId("one"));
|
||||
assertNull(parentGroupIdResolver.getFunnelParentId("two"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInputPortParentId() throws IOException, SchemaLoaderException {
|
||||
List<String> configLines = new ArrayList<>();
|
||||
configLines.add("MiNiFi Config Version: 2");
|
||||
configLines.add("Input Ports:");
|
||||
configLines.add("- id: one");
|
||||
configLines.add("Process Groups:");
|
||||
configLines.add("- id: pgTwo");
|
||||
configLines.add(" Input Ports:");
|
||||
configLines.add(" - id: two");
|
||||
ParentGroupIdResolver parentGroupIdResolver = createParentGroupIdResolver(configLines);
|
||||
assertNull(parentGroupIdResolver.getRemoteInputPortParentId("one"));
|
||||
assertNull(parentGroupIdResolver.getRemoteInputPortParentId("two"));
|
||||
assertNull(parentGroupIdResolver.getRemoteOutputPortParentId("one"));
|
||||
assertNull(parentGroupIdResolver.getRemoteOutputPortParentId("two"));
|
||||
assertEquals(ConfigTransformer.ROOT_GROUP, parentGroupIdResolver.getInputPortParentId("one"));
|
||||
assertEquals("pgTwo", parentGroupIdResolver.getInputPortParentId("two"));
|
||||
assertNull(parentGroupIdResolver.getOutputPortParentId("one"));
|
||||
assertNull(parentGroupIdResolver.getOutputPortParentId("two"));
|
||||
assertNull(parentGroupIdResolver.getProcessorParentId("one"));
|
||||
assertNull(parentGroupIdResolver.getProcessorParentId("two"));
|
||||
assertNull(parentGroupIdResolver.getFunnelParentId("one"));
|
||||
assertNull(parentGroupIdResolver.getFunnelParentId("two"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testOutputPortParentId() throws IOException, SchemaLoaderException {
|
||||
List<String> configLines = new ArrayList<>();
|
||||
configLines.add("MiNiFi Config Version: 2");
|
||||
configLines.add("Output Ports:");
|
||||
configLines.add("- id: one");
|
||||
configLines.add("Process Groups:");
|
||||
configLines.add("- id: pgTwo");
|
||||
configLines.add(" Output Ports:");
|
||||
configLines.add(" - id: two");
|
||||
ParentGroupIdResolver parentGroupIdResolver = createParentGroupIdResolver(configLines);
|
||||
assertNull(parentGroupIdResolver.getRemoteInputPortParentId("one"));
|
||||
assertNull(parentGroupIdResolver.getRemoteInputPortParentId("two"));
|
||||
assertNull(parentGroupIdResolver.getRemoteOutputPortParentId("one"));
|
||||
assertNull(parentGroupIdResolver.getRemoteOutputPortParentId("two"));
|
||||
assertNull(parentGroupIdResolver.getInputPortParentId("one"));
|
||||
assertNull(parentGroupIdResolver.getInputPortParentId("two"));
|
||||
assertEquals(ConfigTransformer.ROOT_GROUP, parentGroupIdResolver.getOutputPortParentId("one"));
|
||||
assertEquals("pgTwo", parentGroupIdResolver.getOutputPortParentId("two"));
|
||||
assertNull(parentGroupIdResolver.getProcessorParentId("one"));
|
||||
assertNull(parentGroupIdResolver.getProcessorParentId("two"));
|
||||
assertNull(parentGroupIdResolver.getFunnelParentId("one"));
|
||||
assertNull(parentGroupIdResolver.getFunnelParentId("two"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testProcessorParentId() throws IOException, SchemaLoaderException {
|
||||
List<String> configLines = new ArrayList<>();
|
||||
configLines.add("MiNiFi Config Version: 2");
|
||||
configLines.add("Processors:");
|
||||
configLines.add("- id: one");
|
||||
configLines.add("Process Groups:");
|
||||
configLines.add("- id: pgTwo");
|
||||
configLines.add(" Processors:");
|
||||
configLines.add(" - id: two");
|
||||
ParentGroupIdResolver parentGroupIdResolver = createParentGroupIdResolver(configLines);
|
||||
assertNull(parentGroupIdResolver.getRemoteInputPortParentId("one"));
|
||||
assertNull(parentGroupIdResolver.getRemoteInputPortParentId("two"));
|
||||
assertNull(parentGroupIdResolver.getRemoteOutputPortParentId("one"));
|
||||
assertNull(parentGroupIdResolver.getRemoteOutputPortParentId("two"));
|
||||
assertNull(parentGroupIdResolver.getInputPortParentId("one"));
|
||||
assertNull(parentGroupIdResolver.getInputPortParentId("two"));
|
||||
assertNull(parentGroupIdResolver.getOutputPortParentId("one"));
|
||||
assertNull(parentGroupIdResolver.getOutputPortParentId("two"));
|
||||
assertEquals(ConfigTransformer.ROOT_GROUP, parentGroupIdResolver.getProcessorParentId("one"));
|
||||
assertEquals("pgTwo", parentGroupIdResolver.getProcessorParentId("two"));
|
||||
assertNull(parentGroupIdResolver.getFunnelParentId("one"));
|
||||
assertNull(parentGroupIdResolver.getFunnelParentId("two"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFunnelParentId() throws IOException, SchemaLoaderException {
|
||||
List<String> configLines = new ArrayList<>();
|
||||
configLines.add("MiNiFi Config Version: 2");
|
||||
configLines.add("Funnels:");
|
||||
configLines.add("- id: one");
|
||||
configLines.add("Process Groups:");
|
||||
configLines.add("- id: pgTwo");
|
||||
configLines.add(" Funnels:");
|
||||
configLines.add(" - id: two");
|
||||
ParentGroupIdResolver parentGroupIdResolver = createParentGroupIdResolver(configLines);
|
||||
assertNull(parentGroupIdResolver.getRemoteInputPortParentId("one"));
|
||||
assertNull(parentGroupIdResolver.getRemoteInputPortParentId("two"));
|
||||
assertNull(parentGroupIdResolver.getRemoteOutputPortParentId("one"));
|
||||
assertNull(parentGroupIdResolver.getRemoteOutputPortParentId("two"));
|
||||
assertNull(parentGroupIdResolver.getInputPortParentId("one"));
|
||||
assertNull(parentGroupIdResolver.getInputPortParentId("two"));
|
||||
assertNull(parentGroupIdResolver.getOutputPortParentId("one"));
|
||||
assertNull(parentGroupIdResolver.getOutputPortParentId("two"));
|
||||
assertNull(parentGroupIdResolver.getProcessorParentId("one"));
|
||||
assertNull(parentGroupIdResolver.getProcessorParentId("two"));
|
||||
assertEquals(ConfigTransformer.ROOT_GROUP, parentGroupIdResolver.getFunnelParentId("one"));
|
||||
assertEquals("pgTwo", parentGroupIdResolver.getFunnelParentId("two"));
|
||||
}
|
||||
|
||||
private ParentGroupIdResolver createParentGroupIdResolver(List<String> configLines) throws IOException, SchemaLoaderException {
|
||||
return new ParentGroupIdResolver(SchemaLoader.loadConfigSchemaFromYaml(new ByteArrayInputStream(String.join("\n", configLines)
|
||||
.getBytes(StandardCharsets.UTF_8))).getProcessGroupSchema());
|
||||
}
|
||||
}
|
|
@ -38,17 +38,16 @@ When using the `CacheConfigurationProvider`, by default, the server will look fo
|
|||
|
||||
The pattern can be configured in [./conf/minifi-c2-context.xml](./minifi-c2-assembly/src/main/resources/conf/minifi-c2-context.xml) and the default value (${class}/config) will replace ${class} with the class query parameter and then look for CLASS/config.CONTENT_TYPE.vVERSION in the directory structure.
|
||||
|
||||
Ex: http://localhost:10090/c2/config?class=raspi&version=1 with an Accept header that matches text/yml would result in [./files/raspi3/config.text.yml.v1](./minifi-c2-assembly/src/main/resources/files/raspi3/config.text.yml.v1) and omitting the version parameter would look for the highest version number in the directory.
|
||||
Ex: http://localhost:10090/c2/config?class=raspi&version=1 with an Accept header that matches application/json would result in [./files/raspi3/config.application.json.v1](./minifi-c2-assembly/src/main/resources/files/raspi3/config.application.json.v1) and omitting the version parameter would look for the highest version number in the directory.
|
||||
|
||||
The version resolution is cached in memory to accommodate many devices polling periodically. The cache duration can be configured with additional arguments in [./conf/minifi-c2-context.xml](../minifi-integration-tests/src/test/resources/c2/hierarchical/c2-edge2/conf/minifi-c2-context.xml#L55) that call the [overloaded constructor.](./minifi-c2-service/src/main/java/org/apache/nifi/minifi/c2/service/ConfigService.java#L81)
|
||||
|
||||
MiNiFi Java agents can be configured to poll the C2 server by [configuring the PullHttpChangeIngestor in their bootstrap.conf.](../minifi-integration-tests/src/test/resources/c2/hierarchical/minifi-edge1/bootstrap.conf#L37)
|
||||
|
||||
### Configuration Providers:
|
||||
There are three `ConfigurationProvider` implementations provided out of the box.
|
||||
There are two `ConfigurationProvider` implementations provided out of the box.
|
||||
1. The [CacheConfigurationProvider](./minifi-c2-assembly/src/main/resources/conf/minifi-c2-context.xml) looks at a directory on the filesystem or in an Amazon S3 bucket. Which backend storage is used can be selected in `minifi-c2-context.xml` via the constructors to `CacheConfigurationProvider`.
|
||||
2. The [DelegatingConfigurationProvider](./minifi-c2-integration-tests/src/test/resources/c2-unsecure-delegating/conf/minifi-c2-context.xml) delegates to another C2 server to allow for hierarchical C2 structures to help with scaling and/or bridging networks.
|
||||
3. The [NiFiRestConfigurationProvider](./minifi-c2-integration-tests/src/test/resources/c2-unsecure-rest/conf/minifi-c2-context.xml) pulls templates from a NiFi instance over its REST API. (Note: sensitive values are NOT included in templates so this is unsuitable for flows with sensitive configuration currently)
|
||||
|
||||
### Example network diagram:
|
||||
Below is a network diagram showing the different configurations tested by [our hierarchical integration test docker-compose file.](../minifi-integration-tests/src/test/resources/docker-compose-c2-hierarchical.yml) It consists of a "cluster" network where real processing might occur as well as 3 "edge" networks that can get configuration from the cluster network a few different ways. The edge1 instance can directly access the authoritative C2 server via HTTPS. The edge2 instance is representative of a segmented network where the MiNiFi agents can talk to a local delegating C2 server over HTTP which asks the authoritative C2 server over HTTPS. The edge 3 instance can talk to the authoritative C2 server through a Squid proxy over HTTPS.
|
||||
|
|
|
@ -26,7 +26,7 @@ MiNiFi is a child project effort of Apache NiFi. The MiNiFi C2 Server aids in t
|
|||
- [Export Control](#export-control)
|
||||
|
||||
## Requirements
|
||||
* JRE 1.8
|
||||
* JRE 17
|
||||
* Apache Maven 3
|
||||
|
||||
## Getting Started
|
||||
|
|
|
@ -100,13 +100,23 @@ limitations under the License.
|
|||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi.minifi</groupId>
|
||||
<artifactId>minifi-c2-provider-nifi-rest</artifactId>
|
||||
<artifactId>minifi-c2-jetty</artifactId>
|
||||
<version>${project.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi.minifi</groupId>
|
||||
<artifactId>minifi-c2-jetty</artifactId>
|
||||
<artifactId>minifi-toolkit-configuration</artifactId>
|
||||
<version>${project.version}</version>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>org.slf4j</groupId>
|
||||
<artifactId>slf4j-log4j12</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>org.glassfish.jersey.media</groupId>
|
||||
<artifactId>jersey-media-json-jackson</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.eclipse.jetty</groupId>
|
||||
|
|
|
@ -26,7 +26,7 @@
|
|||
<bean class="org.apache.nifi.minifi.c2.provider.cache.CacheConfigurationProvider">
|
||||
<constructor-arg>
|
||||
<list>
|
||||
<value>text/yml</value>
|
||||
<value>application/json</value>
|
||||
</list>
|
||||
</constructor-arg>
|
||||
<constructor-arg>
|
||||
|
@ -62,24 +62,6 @@
|
|||
</bean>
|
||||
</constructor-arg>-->
|
||||
</bean>
|
||||
<!--<bean class="org.apache.nifi.minifi.c2.provider.nifi.rest.NiFiRestConfigurationProvider">
|
||||
<constructor-arg>
|
||||
<bean class="org.apache.nifi.minifi.c2.cache.filesystem.FileSystemConfigurationCache">
|
||||
<constructor-arg>
|
||||
<value>./cache</value>
|
||||
</constructor-arg>
|
||||
<constructor-arg>
|
||||
<value>\${class}/\${class}</value>
|
||||
</constructor-arg>
|
||||
</bean>
|
||||
</constructor-arg>
|
||||
<constructor-arg>
|
||||
<value>${minifi.c2.server.provider.nifi.rest.api.url}</value>
|
||||
</constructor-arg>
|
||||
<constructor-arg>
|
||||
<value>\${class}.v\${version}</value>
|
||||
</constructor-arg>
|
||||
</bean>-->
|
||||
</list>
|
||||
</constructor-arg>
|
||||
<constructor-arg>
|
||||
|
|
|
@ -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.
|
||||
|
||||
{
|
||||
"encodingVersion": {
|
||||
"majorVersion": 2,
|
||||
"minorVersion": 0
|
||||
},
|
||||
"maxTimerDrivenThreadCount": 10,
|
||||
"maxEventDrivenThreadCount": 1,
|
||||
"registries": [],
|
||||
"parameterContexts": [],
|
||||
"parameterProviders": [],
|
||||
"controllerServices": [],
|
||||
"reportingTasks": [],
|
||||
"templates": [],
|
||||
"rootGroup": {
|
||||
"identifier": "c1b4e586-2011-3f81-a11e-8d669f084d1c",
|
||||
"instanceIdentifier": "29db3dbc-0188-1000-7025-4cab8b52d278",
|
||||
"name": "MiNiFi Flow",
|
||||
"comments": "",
|
||||
"position": {
|
||||
"x": 0.0,
|
||||
"y": 0.0
|
||||
},
|
||||
"processGroups": [],
|
||||
"remoteProcessGroups": [],
|
||||
"processors": [],
|
||||
"inputPorts": [],
|
||||
"outputPorts": [],
|
||||
"connections": [],
|
||||
"labels": [],
|
||||
"funnels": [],
|
||||
"controllerServices": [],
|
||||
"variables": {},
|
||||
"defaultFlowFileExpiration": "0 sec",
|
||||
"defaultBackPressureObjectThreshold": 10000,
|
||||
"defaultBackPressureDataSizeThreshold": "1 GB",
|
||||
"componentType": "PROCESS_GROUP",
|
||||
"flowFileConcurrency": "UNBOUNDED",
|
||||
"flowFileOutboundPolicy": "STREAM_WHEN_AVAILABLE"
|
||||
}
|
||||
}
|
|
@ -1,63 +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.
|
||||
|
||||
MiNiFi Config Version: 3
|
||||
Flow Controller:
|
||||
name: MiNiFi Flow
|
||||
comment: ''
|
||||
Core Properties:
|
||||
flow controller graceful shutdown period: 10 sec
|
||||
flow service write delay interval: 500 ms
|
||||
administrative yield duration: 30 sec
|
||||
bored yield duration: 10 millis
|
||||
max concurrent threads: 1
|
||||
FlowFile Repository:
|
||||
partitions: 256
|
||||
checkpoint interval: 2 mins
|
||||
always sync: false
|
||||
Swap:
|
||||
threshold: 20000
|
||||
in period: 5 sec
|
||||
in threads: 1
|
||||
out period: 5 sec
|
||||
out threads: 4
|
||||
Content Repository:
|
||||
content claim max appendable size: 10 MB
|
||||
content claim max flow files: 100
|
||||
always sync: false
|
||||
Provenance Repository:
|
||||
provenance rollover time: 1 min
|
||||
Component Status Repository:
|
||||
buffer size: 1440
|
||||
snapshot frequency: 1 min
|
||||
Security Properties:
|
||||
keystore: ''
|
||||
keystore type: ''
|
||||
keystore password: ''
|
||||
key password: ''
|
||||
truststore: ''
|
||||
truststore type: ''
|
||||
truststore password: ''
|
||||
ssl protocol: ''
|
||||
Sensitive Props:
|
||||
key: ''
|
||||
algorithm: NIFI_PBKDF2_AES_GCM_256
|
||||
provider: BC
|
||||
Processors: []
|
||||
Process Groups: []
|
||||
Funnels: []
|
||||
Connections: []
|
||||
Remote Process Groups: []
|
||||
NiFi Properties Overrides: {}
|
|
@ -36,7 +36,10 @@ import org.apache.nifi.minifi.c2.cache.filesystem.FileSystemConfigurationCache;
|
|||
import org.junit.jupiter.api.Test;
|
||||
|
||||
public class FileSystemConfigurationCacheTest {
|
||||
|
||||
private static final String PATH_ROOT = "src/test/resources/files";
|
||||
private static final String FLOW_CONTENT_TYPE = "test/json";
|
||||
public static final String NOT_REGISTERED_CONTENT_TYPE = "test/contenttype";
|
||||
|
||||
@Test
|
||||
public void getConfigurationTest() throws IOException, ConfigurationProviderException {
|
||||
|
@ -46,11 +49,11 @@ public class FileSystemConfigurationCacheTest {
|
|||
|
||||
Map<String, List<String>> parameters = new HashMap<>();
|
||||
|
||||
ConfigurationCacheFileInfo info = cache.getCacheFileInfo("text/yaml", parameters);
|
||||
ConfigurationCacheFileInfo info = cache.getCacheFileInfo(FLOW_CONTENT_TYPE, parameters);
|
||||
|
||||
WriteableConfiguration configuration = info.getConfiguration(1);
|
||||
|
||||
assertEquals("config.text.yaml.v1", configuration.getName());
|
||||
assertEquals("config.test.json.v1", configuration.getName());
|
||||
assertEquals("1", configuration.getVersion());
|
||||
assertTrue(configuration.exists());
|
||||
}
|
||||
|
@ -63,7 +66,7 @@ public class FileSystemConfigurationCacheTest {
|
|||
|
||||
Map<String, List<String>> parameters = new HashMap<>();
|
||||
|
||||
ConfigurationCacheFileInfo info = cache.getCacheFileInfo("test/contenttype", parameters);
|
||||
ConfigurationCacheFileInfo info = cache.getCacheFileInfo(NOT_REGISTERED_CONTENT_TYPE, parameters);
|
||||
|
||||
WriteableConfiguration configuration = info.getConfiguration(1);
|
||||
|
||||
|
@ -80,7 +83,7 @@ public class FileSystemConfigurationCacheTest {
|
|||
|
||||
Map<String, List<String>> parameters = new HashMap<>();
|
||||
|
||||
ConfigurationCacheFileInfo info = cache.getCacheFileInfo("text/yaml", parameters);
|
||||
ConfigurationCacheFileInfo info = cache.getCacheFileInfo(FLOW_CONTENT_TYPE, parameters);
|
||||
|
||||
Stream<WriteableConfiguration> configs = info.getCachedConfigurations();
|
||||
|
||||
|
@ -96,6 +99,6 @@ public class FileSystemConfigurationCacheTest {
|
|||
|
||||
Map<String, List<String>> parameters = new HashMap<>();
|
||||
|
||||
assertThrows(InvalidParameterException.class, () -> cache.getCacheFileInfo("test/contenttype", parameters));
|
||||
assertThrows(InvalidParameterException.class, () -> cache.getCacheFileInfo(NOT_REGISTERED_CONTENT_TYPE, parameters));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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.
|
||||
|
||||
{
|
||||
"encodingVersion": {
|
||||
"majorVersion": 2,
|
||||
"minorVersion": 0
|
||||
},
|
||||
"maxTimerDrivenThreadCount": 10,
|
||||
"maxEventDrivenThreadCount": 1,
|
||||
"registries": [],
|
||||
"parameterContexts": [],
|
||||
"parameterProviders": [],
|
||||
"controllerServices": [],
|
||||
"reportingTasks": [],
|
||||
"templates": [],
|
||||
"rootGroup": {
|
||||
"identifier": "c1b4e586-2011-3f81-a11e-8d669f084d1c",
|
||||
"instanceIdentifier": "29db3dbc-0188-1000-7025-4cab8b52d278",
|
||||
"name": "NiFi Flow",
|
||||
"comments": "",
|
||||
"position": {
|
||||
"x": 0.0,
|
||||
"y": 0.0
|
||||
},
|
||||
"processGroups": [],
|
||||
"remoteProcessGroups": [],
|
||||
"processors": [],
|
||||
"inputPorts": [],
|
||||
"outputPorts": [],
|
||||
"connections": [],
|
||||
"labels": [],
|
||||
"funnels": [],
|
||||
"controllerServices": [],
|
||||
"variables": {},
|
||||
"defaultFlowFileExpiration": "0 sec",
|
||||
"defaultBackPressureObjectThreshold": 10000,
|
||||
"defaultBackPressureDataSizeThreshold": "1 GB",
|
||||
"componentType": "PROCESS_GROUP",
|
||||
"flowFileConcurrency": "UNBOUNDED",
|
||||
"flowFileOutboundPolicy": "STREAM_WHEN_AVAILABLE"
|
||||
}
|
||||
}
|
|
@ -1,63 +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.
|
||||
|
||||
MiNiFi Config Version: 3
|
||||
Flow Controller:
|
||||
name: MiNiFi Flow
|
||||
comment: ''
|
||||
Core Properties:
|
||||
flow controller graceful shutdown period: 10 sec
|
||||
flow service write delay interval: 500 ms
|
||||
administrative yield duration: 30 sec
|
||||
bored yield duration: 10 millis
|
||||
max concurrent threads: 1
|
||||
FlowFile Repository:
|
||||
partitions: 256
|
||||
checkpoint interval: 2 mins
|
||||
always sync: false
|
||||
Swap:
|
||||
threshold: 20000
|
||||
in period: 5 sec
|
||||
in threads: 1
|
||||
out period: 5 sec
|
||||
out threads: 4
|
||||
Content Repository:
|
||||
content claim max appendable size: 10 MB
|
||||
content claim max flow files: 100
|
||||
always sync: false
|
||||
Provenance Repository:
|
||||
provenance rollover time: 1 min
|
||||
Component Status Repository:
|
||||
buffer size: 1440
|
||||
snapshot frequency: 1 min
|
||||
Security Properties:
|
||||
keystore: ''
|
||||
keystore type: ''
|
||||
keystore password: ''
|
||||
key password: ''
|
||||
truststore: ''
|
||||
truststore type: ''
|
||||
truststore password: ''
|
||||
ssl protocol: ''
|
||||
Sensitive Props:
|
||||
key: ''
|
||||
algorithm: NIFI_PBKDF2_AES_GCM_256
|
||||
provider: BC
|
||||
Processors: []
|
||||
Process Groups: []
|
||||
Funnels: []
|
||||
Connections: []
|
||||
Remote Process Groups: []
|
||||
NiFi Properties Overrides: {}
|
|
@ -16,7 +16,7 @@
|
|||
# under the License.
|
||||
#
|
||||
|
||||
FROM eclipse-temurin:11-jre
|
||||
FROM eclipse-temurin:17-jre
|
||||
LABEL maintainer="Apache NiFi <dev@nifi.apache.org>"
|
||||
LABEL site="https://nifi.apache.org"
|
||||
|
||||
|
|
|
@ -17,7 +17,7 @@
|
|||
#
|
||||
|
||||
ARG IMAGE_NAME=eclipse-temurin
|
||||
ARG IMAGE_TAG=11-jre
|
||||
ARG IMAGE_TAG=17-jre
|
||||
FROM ${IMAGE_NAME}:${IMAGE_TAG} as builder
|
||||
LABEL stage=builder
|
||||
|
||||
|
|
|
@ -30,12 +30,6 @@ limitations under the License.
|
|||
</properties>
|
||||
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi.minifi</groupId>
|
||||
<artifactId>minifi-commons-schema</artifactId>
|
||||
<version>${project.version}</version>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi.minifi</groupId>
|
||||
<artifactId>minifi-c2-api</artifactId>
|
||||
|
@ -59,6 +53,15 @@ limitations under the License.
|
|||
<version>2.0.0</version>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-framework-core-api</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.fasterxml.jackson.module</groupId>
|
||||
<artifactId>jackson-module-jaxb-annotations</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
|
||||
<build>
|
||||
|
@ -91,7 +94,7 @@ limitations under the License.
|
|||
<artifactId>apache-rat-plugin</artifactId>
|
||||
<configuration>
|
||||
<excludes combine.children="append">
|
||||
<exclude>src/test/resources/mocknifi/www/nifi-api/flow/templates</exclude>
|
||||
<exclude>**/config.application.json.*</exclude>
|
||||
</excludes>
|
||||
</configuration>
|
||||
</plugin>
|
||||
|
|
|
@ -17,23 +17,11 @@
|
|||
|
||||
package org.apache.nifi.minifi.c2.integration.test;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
|
||||
import com.palantir.docker.compose.DockerComposeExtension;
|
||||
import com.palantir.docker.compose.connection.DockerPort;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.nifi.minifi.commons.schema.ConfigSchema;
|
||||
import org.apache.nifi.minifi.commons.schema.serialization.SchemaLoader;
|
||||
import org.apache.nifi.security.util.KeyStoreUtils;
|
||||
import org.apache.nifi.security.util.KeystoreType;
|
||||
import org.apache.nifi.security.util.SslContextFactory;
|
||||
import org.apache.nifi.security.util.StandardTlsConfiguration;
|
||||
import org.apache.nifi.security.util.TlsConfiguration;
|
||||
import org.apache.nifi.toolkit.tls.standalone.TlsToolkitStandalone;
|
||||
import org.apache.nifi.toolkit.tls.standalone.TlsToolkitStandaloneCommandLine;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import javax.net.ssl.HttpsURLConnection;
|
||||
import javax.net.ssl.SSLContext;
|
||||
import javax.net.ssl.TrustManagerFactory;
|
||||
import java.io.FileInputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
|
@ -49,9 +37,19 @@ import java.security.KeyStore;
|
|||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
import javax.net.ssl.HttpsURLConnection;
|
||||
import javax.net.ssl.SSLContext;
|
||||
import javax.net.ssl.TrustManagerFactory;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.nifi.controller.flow.VersionedDataflow;
|
||||
import org.apache.nifi.security.util.KeyStoreUtils;
|
||||
import org.apache.nifi.security.util.KeystoreType;
|
||||
import org.apache.nifi.security.util.SslContextFactory;
|
||||
import org.apache.nifi.security.util.StandardTlsConfiguration;
|
||||
import org.apache.nifi.security.util.TlsConfiguration;
|
||||
import org.apache.nifi.toolkit.tls.standalone.TlsToolkitStandalone;
|
||||
import org.apache.nifi.toolkit.tls.standalone.TlsToolkitStandaloneCommandLine;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
public abstract class AbstractTestSecure extends AbstractTestUnsecure {
|
||||
public static final String C2_URL = "https://c2:10443/c2/config";
|
||||
|
@ -73,7 +71,7 @@ public abstract class AbstractTestSecure extends AbstractTestUnsecure {
|
|||
|
||||
public static SSLContext initCertificates(Path certificatesDirectory, List<String> serverHostnames) throws Exception {
|
||||
List<String> toolkitCommandLine = new ArrayList<>(Arrays.asList("-O", "-o", certificatesDirectory.toFile().getAbsolutePath(),
|
||||
"-C", "CN=user1", "-C", "CN=user2", "-C", "CN=user3", "-C", "CN=user4", "-S", "badKeystorePass", "-K", "badKeyPass", "-P", "badTrustPass"));
|
||||
"-C", "CN=user1", "-C", "CN=user2", "-C", "CN=user3", "-C", "CN=user4", "-S", "badKeystorePass", "-K", "badKeyPass", "-P", "badTrustPass"));
|
||||
for (String serverHostname : serverHostnames) {
|
||||
toolkitCommandLine.add("-n");
|
||||
toolkitCommandLine.add(serverHostname);
|
||||
|
@ -95,8 +93,8 @@ public abstract class AbstractTestSecure extends AbstractTestUnsecure {
|
|||
trustManagerFactory.init(trustStore);
|
||||
|
||||
TlsConfiguration tlsConfiguration = new StandardTlsConfiguration(
|
||||
null, null, null,
|
||||
certificatesDirectory.resolve("c2").resolve("truststore.jks").toFile().getAbsolutePath(), "badTrustPass", KeystoreType.JKS);
|
||||
null, null, null,
|
||||
certificatesDirectory.resolve("c2").resolve("truststore.jks").toFile().getAbsolutePath(), "badTrustPass", KeystoreType.JKS);
|
||||
|
||||
|
||||
return SslContextFactory.createSslContext(tlsConfiguration);
|
||||
|
@ -115,8 +113,8 @@ public abstract class AbstractTestSecure extends AbstractTestUnsecure {
|
|||
|
||||
assertReturnCode("", sslContext, 403);
|
||||
|
||||
ConfigSchema configSchema = assertReturnCode("?class=raspi2", sslContext, 200);
|
||||
assertEquals("raspi2.v1", configSchema.getFlowControllerProperties().getName());
|
||||
VersionedDataflow versionedDataflow = assertReturnCode("?class=raspi2", sslContext, 200);
|
||||
assertEquals("raspi2.v1", versionedDataflow.getRootGroup().getName());
|
||||
|
||||
assertReturnCode("?class=raspi3", sslContext, 403);
|
||||
}
|
||||
|
@ -128,8 +126,8 @@ public abstract class AbstractTestSecure extends AbstractTestUnsecure {
|
|||
assertReturnCode("", sslContext, 403);
|
||||
assertReturnCode("?class=raspi2", sslContext, 403);
|
||||
|
||||
ConfigSchema configSchema = assertReturnCode("?class=raspi3", sslContext, 200);
|
||||
assertEquals("raspi3.v2", configSchema.getFlowControllerProperties().getName());
|
||||
VersionedDataflow versionedDataflow = assertReturnCode("?class=raspi3", sslContext, 200);
|
||||
assertEquals("raspi3.v2", versionedDataflow.getRootGroup().getName());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -138,11 +136,11 @@ public abstract class AbstractTestSecure extends AbstractTestUnsecure {
|
|||
|
||||
assertReturnCode("", sslContext, 400);
|
||||
|
||||
ConfigSchema configSchema = assertReturnCode("?class=raspi2", sslContext, 200);
|
||||
assertEquals("raspi2.v1", configSchema.getFlowControllerProperties().getName());
|
||||
VersionedDataflow dataflowRaspi2 = assertReturnCode("?class=raspi2", sslContext, 200);
|
||||
assertEquals("raspi2.v1", dataflowRaspi2.getRootGroup().getName());
|
||||
|
||||
configSchema = assertReturnCode("?class=raspi3", sslContext, 200);
|
||||
assertEquals("raspi3.v2", configSchema.getFlowControllerProperties().getName());
|
||||
VersionedDataflow dataflowRaspi3 = assertReturnCode("?class=raspi3", sslContext, 200);
|
||||
assertEquals("raspi3.v2", dataflowRaspi3.getRootGroup().getName());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -169,22 +167,25 @@ public abstract class AbstractTestSecure extends AbstractTestUnsecure {
|
|||
keystorePasswd = IOUtils.toString(inputStream, StandardCharsets.UTF_8);
|
||||
}
|
||||
TlsConfiguration tlsConfiguration = new StandardTlsConfiguration(
|
||||
directory.resolve("CN=" + username + ".p12").toFile().getAbsolutePath(),
|
||||
keystorePasswd,
|
||||
KeystoreType.PKCS12,
|
||||
certificatesDirectory.resolve("c2").resolve("truststore.jks").toFile().getAbsolutePath(),
|
||||
"badTrustPass",
|
||||
KeystoreType.JKS);
|
||||
directory.resolve("CN=" + username + ".p12").toFile().getAbsolutePath(),
|
||||
keystorePasswd,
|
||||
KeystoreType.PKCS12,
|
||||
certificatesDirectory.resolve("c2").resolve("truststore.jks").toFile().getAbsolutePath(),
|
||||
"badTrustPass",
|
||||
KeystoreType.JKS);
|
||||
|
||||
return SslContextFactory.createSslContext(tlsConfiguration);
|
||||
}
|
||||
|
||||
protected ConfigSchema assertReturnCode(String query, SSLContext sslContext, int expectedReturnCode) throws Exception {
|
||||
protected VersionedDataflow assertReturnCode(String query, SSLContext sslContext, int expectedReturnCode) throws Exception {
|
||||
HttpsURLConnection httpsURLConnection = openUrlConnection(C2_URL + query, sslContext);
|
||||
httpsURLConnection.setRequestProperty("Accept", "application/json");
|
||||
try {
|
||||
assertEquals(expectedReturnCode, httpsURLConnection.getResponseCode());
|
||||
if (expectedReturnCode == 200) {
|
||||
return SchemaLoader.loadConfigSchemaFromYaml(httpsURLConnection.getInputStream());
|
||||
try (InputStream inputStream = httpsURLConnection.getInputStream()) {
|
||||
return toVersionedDataFlow(inputStream);
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
httpsURLConnection.disconnect();
|
||||
|
@ -195,7 +196,7 @@ public abstract class AbstractTestSecure extends AbstractTestUnsecure {
|
|||
protected HttpsURLConnection openUrlConnection(String url, SSLContext sslContext) throws IOException {
|
||||
DockerPort dockerPort = docker.containers().container("squid").port(3128);
|
||||
HttpsURLConnection httpURLConnection = (HttpsURLConnection) new URL(url).openConnection(
|
||||
new Proxy(Proxy.Type.HTTP, new InetSocketAddress(dockerPort.getIp(), dockerPort.getExternalPort())));
|
||||
new Proxy(Proxy.Type.HTTP, new InetSocketAddress(dockerPort.getIp(), dockerPort.getExternalPort())));
|
||||
httpURLConnection.setSSLSocketFactory(sslContext.getSocketFactory());
|
||||
return httpURLConnection;
|
||||
}
|
||||
|
|
|
@ -17,20 +17,21 @@
|
|||
|
||||
package org.apache.nifi.minifi.c2.integration.test;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
|
||||
import com.fasterxml.jackson.databind.DeserializationFeature;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
|
||||
import com.palantir.docker.compose.DockerComposeExtension;
|
||||
import com.palantir.docker.compose.connection.Container;
|
||||
import com.palantir.docker.compose.connection.DockerPort;
|
||||
import org.apache.nifi.minifi.commons.schema.ConfigSchema;
|
||||
import org.apache.nifi.minifi.commons.schema.exception.SchemaLoaderException;
|
||||
import org.apache.nifi.minifi.commons.schema.serialization.SchemaLoader;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.net.HttpURLConnection;
|
||||
import java.net.URL;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.nifi.controller.flow.VersionedDataflow;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
public abstract class AbstractTestUnsecure {
|
||||
protected String c2Url;
|
||||
|
@ -49,31 +50,28 @@ public abstract class AbstractTestUnsecure {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testCurrentVersion() throws IOException, SchemaLoaderException {
|
||||
ConfigSchema configSchema = getConfigSchema(c2Url + "?class=raspi3");
|
||||
assertEquals(3, configSchema.getVersion());
|
||||
assertEquals("raspi3.v2", configSchema.getFlowControllerProperties().getName());
|
||||
public void testCurrentVersion() throws IOException {
|
||||
VersionedDataflow versionedDataflow = getFlowDefinition(c2Url + "?class=raspi3");
|
||||
assertEquals("raspi3.v2", versionedDataflow.getRootGroup().getName());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testVersion1() throws IOException, SchemaLoaderException {
|
||||
ConfigSchema configSchema = getConfigSchema(c2Url + "?class=raspi3&version=1");
|
||||
assertEquals(3, configSchema.getVersion());
|
||||
assertEquals("raspi3.v1", configSchema.getFlowControllerProperties().getName());
|
||||
public void testVersion1() throws IOException {
|
||||
VersionedDataflow versionedDataflow = getFlowDefinition(c2Url + "?class=raspi3&version=1");
|
||||
assertEquals("raspi3.v1", versionedDataflow.getRootGroup().getName());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testVersion2() throws IOException, SchemaLoaderException {
|
||||
ConfigSchema configSchema = getConfigSchema(c2Url + "?class=raspi3&version=2");
|
||||
assertEquals(3, configSchema.getVersion());
|
||||
assertEquals("raspi3.v2", configSchema.getFlowControllerProperties().getName());
|
||||
public void testVersion2() throws IOException {
|
||||
VersionedDataflow versionedDataflow = getFlowDefinition(c2Url + "?class=raspi3&version=2");
|
||||
assertEquals("raspi3.v2", versionedDataflow.getRootGroup().getName());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUnacceptable() throws IOException {
|
||||
HttpURLConnection urlConnection = openSuperUserUrlConnection(c2Url + "?class=raspi3");
|
||||
try {
|
||||
urlConnection.setRequestProperty("Accept", "text/xml");
|
||||
urlConnection.setRequestProperty("Accept", "text/invalid");
|
||||
assertEquals(406, urlConnection.getResponseCode());
|
||||
} finally {
|
||||
urlConnection.disconnect();
|
||||
|
@ -90,18 +88,24 @@ public abstract class AbstractTestUnsecure {
|
|||
}
|
||||
}
|
||||
|
||||
public ConfigSchema getConfigSchema(String urlString) throws IOException, SchemaLoaderException {
|
||||
public VersionedDataflow getFlowDefinition(String urlString) throws IOException {
|
||||
HttpURLConnection urlConnection = openSuperUserUrlConnection(urlString);
|
||||
ConfigSchema configSchema;
|
||||
urlConnection.setRequestProperty("Accept", "application/json");
|
||||
try (InputStream inputStream = urlConnection.getInputStream()) {
|
||||
configSchema = SchemaLoader.loadConfigSchemaFromYaml(inputStream);
|
||||
return toVersionedDataFlow(inputStream);
|
||||
} finally {
|
||||
urlConnection.disconnect();
|
||||
}
|
||||
return configSchema;
|
||||
}
|
||||
|
||||
protected HttpURLConnection openSuperUserUrlConnection(String url) throws IOException {
|
||||
return (HttpURLConnection) new URL(url).openConnection();
|
||||
}
|
||||
|
||||
protected VersionedDataflow toVersionedDataFlow(InputStream inputStream) throws IOException {
|
||||
ObjectMapper objectMapper = new ObjectMapper();
|
||||
objectMapper.setAnnotationIntrospector(new JaxbAnnotationIntrospector(objectMapper.getTypeFactory()));
|
||||
objectMapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
|
||||
return objectMapper.readValue(IOUtils.toByteArray(inputStream), VersionedDataflow.class);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,118 +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.c2.integration.test;
|
||||
|
||||
import com.palantir.docker.compose.DockerComposeExtension;
|
||||
import org.apache.nifi.minifi.c2.integration.test.health.HttpsStatusCodeHealthCheck;
|
||||
import org.apache.nifi.security.util.KeystoreType;
|
||||
import org.apache.nifi.security.util.SslContextFactory;
|
||||
import org.apache.nifi.security.util.StandardTlsConfiguration;
|
||||
import org.apache.nifi.security.util.TlsConfiguration;
|
||||
import org.apache.nifi.toolkit.tls.standalone.TlsToolkitStandalone;
|
||||
import org.bouncycastle.openssl.jcajce.JcaMiscPEMGenerator;
|
||||
import org.bouncycastle.util.io.pem.PemWriter;
|
||||
import org.junit.jupiter.api.AfterAll;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.BeforeAll;
|
||||
|
||||
import javax.net.ssl.SSLContext;
|
||||
import javax.net.ssl.SSLSocketFactory;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStreamWriter;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
import java.nio.file.Paths;
|
||||
import java.security.KeyStore;
|
||||
import java.security.cert.Certificate;
|
||||
import java.util.Arrays;
|
||||
|
||||
public class NiFiRestConfigurationProviderSecureTest extends AbstractTestSecure {
|
||||
private static SSLSocketFactory healthCheckSocketFactory;
|
||||
private static Path certificatesDirectory;
|
||||
private static SSLContext trustSslContext;
|
||||
|
||||
// Not annotated as rule because we need to generate certificatesDirectory first
|
||||
public static DockerComposeExtension docker = DockerComposeExtension.builder()
|
||||
.file("target/test-classes/docker-compose-NiFiRestConfigurationProviderSecureTest.yml")
|
||||
.waitingForServices(Arrays.asList("squid", "mocknifi"),
|
||||
new HttpsStatusCodeHealthCheck(container -> "https://mocknifi:8443/", containers -> containers.get(0), containers -> containers.get(1), () -> {
|
||||
Path c2 = certificatesDirectory.resolve("c2");
|
||||
try {
|
||||
TlsConfiguration tlsConfiguration = new StandardTlsConfiguration(
|
||||
c2.resolve("keystore.jks").toFile().getAbsolutePath(),
|
||||
"badKeystorePass",
|
||||
"badKeyPass", KeystoreType.JKS,
|
||||
c2.resolve("truststore.jks").toFile().getAbsolutePath(),
|
||||
"badTrustPass",
|
||||
KeystoreType.JKS);
|
||||
|
||||
return SslContextFactory.createSslContext(tlsConfiguration).getSocketFactory();
|
||||
} catch (Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}, 200))
|
||||
.waitingForServices(Arrays.asList("squid", "c2"),
|
||||
new HttpsStatusCodeHealthCheck(container -> C2_URL, containers -> containers.get(0), containers -> containers.get(1), () -> healthCheckSocketFactory, 403))
|
||||
.build();
|
||||
|
||||
public NiFiRestConfigurationProviderSecureTest() {
|
||||
super(docker, certificatesDirectory, trustSslContext);
|
||||
}
|
||||
|
||||
/**
|
||||
* Generates certificates with the tls-toolkit and then starts up the docker compose file
|
||||
*/
|
||||
@BeforeAll
|
||||
public static void initCertificates() throws Exception {
|
||||
certificatesDirectory = Paths.get(NiFiRestConfigurationProviderSecureTest.class.getClassLoader()
|
||||
.getResource("docker-compose-NiFiRestConfigurationProviderSecureTest.yml").getFile()).getParent().toAbsolutePath().resolve("certificates-NiFiRestConfigurationProviderSecureTest");
|
||||
trustSslContext = initCertificates(certificatesDirectory, Arrays.asList("c2", "mocknifi"));
|
||||
healthCheckSocketFactory = trustSslContext.getSocketFactory();
|
||||
|
||||
KeyStore mockNiFiKeyStore = KeyStore.getInstance("JKS");
|
||||
try (InputStream inputStream = Files.newInputStream(certificatesDirectory.resolve("mocknifi").resolve("keystore.jks"))) {
|
||||
mockNiFiKeyStore.load(inputStream, "badKeystorePass".toCharArray());
|
||||
}
|
||||
try (PemWriter pemWriter = new PemWriter(new OutputStreamWriter(Files.newOutputStream(certificatesDirectory.resolve("mocknifi").resolve("cert.pem"))))) {
|
||||
pemWriter.writeObject(new JcaMiscPEMGenerator(mockNiFiKeyStore.getKey(TlsToolkitStandalone.NIFI_KEY, "badKeyPass".toCharArray())));
|
||||
for (Certificate certificate : mockNiFiKeyStore.getCertificateChain(TlsToolkitStandalone.NIFI_KEY)) {
|
||||
pemWriter.writeObject(new JcaMiscPEMGenerator(certificate));
|
||||
}
|
||||
}
|
||||
|
||||
KeyStore mockNiFiTrustStore = KeyStore.getInstance("JKS");
|
||||
try (InputStream inputStream = Files.newInputStream(certificatesDirectory.resolve("mocknifi").resolve("truststore.jks"))) {
|
||||
mockNiFiTrustStore.load(inputStream, "badTrustPass".toCharArray());
|
||||
}
|
||||
try (PemWriter pemWriter = new PemWriter(new OutputStreamWriter(Files.newOutputStream(certificatesDirectory.resolve("mocknifi").resolve("ca.pem"))))) {
|
||||
pemWriter.writeObject(new JcaMiscPEMGenerator(mockNiFiTrustStore.getCertificate(TlsToolkitStandalone.NIFI_CERT)));
|
||||
}
|
||||
|
||||
docker.before();
|
||||
}
|
||||
|
||||
@AfterAll
|
||||
public static void cleanup() {
|
||||
docker.after();
|
||||
}
|
||||
|
||||
@BeforeEach
|
||||
public void setup() {
|
||||
super.setup(docker);
|
||||
}
|
||||
}
|
|
@ -1,39 +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.c2.integration.test;
|
||||
|
||||
import com.palantir.docker.compose.DockerComposeExtension;
|
||||
import com.palantir.docker.compose.connection.waiting.HealthChecks;
|
||||
import org.apache.nifi.minifi.c2.integration.test.health.HttpStatusCodeHealthCheck;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.extension.RegisterExtension;
|
||||
|
||||
public class NiFiRestConfigurationProviderUnsecureTest extends AbstractTestUnsecure {
|
||||
@RegisterExtension
|
||||
public static DockerComposeExtension docker = DockerComposeExtension.builder()
|
||||
.file("target/test-classes/docker-compose-NiFiRestConfigurationProviderUnsecureTest.yml")
|
||||
.waitingForService("mocknifi", HealthChecks.toRespond2xxOverHttp(8080,
|
||||
dockerPort -> "http://" + dockerPort.getIp() + ":" + dockerPort.getExternalPort() + "/"))
|
||||
.waitingForService("c2", new HttpStatusCodeHealthCheck(FileSystemCacheProviderUnsecureTest::getUnsecureConfigUrl, 400))
|
||||
.build();
|
||||
|
||||
@BeforeEach
|
||||
public void setup() {
|
||||
super.setup(docker);
|
||||
}
|
||||
}
|
|
@ -1,60 +0,0 @@
|
|||
<?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.
|
||||
-->
|
||||
|
||||
<beans default-lazy-init="true"
|
||||
xmlns="http://www.springframework.org/schema/beans"
|
||||
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
|
||||
xmlns:util="http://www.springframework.org/schema/util"
|
||||
xmlns:context="http://www.springframework.org/schema/context"
|
||||
xmlns:aop="http://www.springframework.org/schema/aop"
|
||||
xsi:schemaLocation="http://www.springframework.org/schema/beans https://www.springframework.org/schema/beans/spring-beans-3.1.xsd
|
||||
http://www.springframework.org/schema/util https://www.springframework.org/schema/util/spring-util-3.1.xsd
|
||||
http://www.springframework.org/schema/context https://www.springframework.org/schema/context/spring-context-3.1.xsd
|
||||
http://www.springframework.org/schema/aop https://www.springframework.org/schema/aop/spring-aop-3.1.xsd">
|
||||
|
||||
<bean id="configService" class="org.apache.nifi.minifi.c2.service.ConfigService" scope="singleton">
|
||||
<constructor-arg>
|
||||
<list>
|
||||
<bean class="org.apache.nifi.minifi.c2.provider.nifi.rest.NiFiRestConfigurationProvider">
|
||||
<constructor-arg>
|
||||
<bean class="org.apache.nifi.minifi.c2.cache.filesystem.FileSystemConfigurationCache">
|
||||
<constructor-arg>
|
||||
<value>./cache</value>
|
||||
</constructor-arg>
|
||||
<constructor-arg>
|
||||
<value>${class}/${class}</value>
|
||||
</constructor-arg>
|
||||
</bean>
|
||||
</constructor-arg>
|
||||
<constructor-arg>
|
||||
<value>https://mocknifi:8443/nifi-api</value>
|
||||
</constructor-arg>
|
||||
<constructor-arg>
|
||||
<value>${class}.v${version}</value>
|
||||
</constructor-arg>
|
||||
</bean>
|
||||
</list>
|
||||
</constructor-arg>
|
||||
<constructor-arg>
|
||||
<bean class="org.apache.nifi.minifi.c2.security.authorization.GrantedAuthorityAuthorizer">
|
||||
<constructor-arg value="classpath:authorizations.yaml"/>
|
||||
</bean>
|
||||
</constructor-arg>
|
||||
</bean>
|
||||
</beans>
|
|
@ -1,59 +0,0 @@
|
|||
<?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.
|
||||
-->
|
||||
<beans default-lazy-init="true"
|
||||
xmlns="http://www.springframework.org/schema/beans"
|
||||
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
|
||||
xmlns:util="http://www.springframework.org/schema/util"
|
||||
xmlns:context="http://www.springframework.org/schema/context"
|
||||
xmlns:aop="http://www.springframework.org/schema/aop"
|
||||
xsi:schemaLocation="http://www.springframework.org/schema/beans https://www.springframework.org/schema/beans/spring-beans-3.1.xsd
|
||||
http://www.springframework.org/schema/util https://www.springframework.org/schema/util/spring-util-3.1.xsd
|
||||
http://www.springframework.org/schema/context https://www.springframework.org/schema/context/spring-context-3.1.xsd
|
||||
http://www.springframework.org/schema/aop https://www.springframework.org/schema/aop/spring-aop-3.1.xsd">
|
||||
|
||||
<bean id="configService" class="org.apache.nifi.minifi.c2.service.ConfigService" scope="singleton">
|
||||
<constructor-arg>
|
||||
<list>
|
||||
<bean class="org.apache.nifi.minifi.c2.provider.nifi.rest.NiFiRestConfigurationProvider">
|
||||
<constructor-arg>
|
||||
<bean class="org.apache.nifi.minifi.c2.cache.filesystem.FileSystemConfigurationCache">
|
||||
<constructor-arg>
|
||||
<value>./cache</value>
|
||||
</constructor-arg>
|
||||
<constructor-arg>
|
||||
<value>${class}/${class}</value>
|
||||
</constructor-arg>
|
||||
</bean>
|
||||
</constructor-arg>
|
||||
<constructor-arg>
|
||||
<value>http://mocknifi:8080/nifi-api</value>
|
||||
</constructor-arg>
|
||||
<constructor-arg>
|
||||
<value>${class}.v${version}</value>
|
||||
</constructor-arg>
|
||||
</bean>
|
||||
</list>
|
||||
</constructor-arg>
|
||||
<constructor-arg>
|
||||
<bean class="org.apache.nifi.minifi.c2.security.authorization.GrantedAuthorityAuthorizer">
|
||||
<constructor-arg value="classpath:authorizations.yaml"/>
|
||||
</bean>
|
||||
</constructor-arg>
|
||||
</bean>
|
||||
</beans>
|
|
@ -0,0 +1,40 @@
|
|||
{
|
||||
"encodingVersion": {
|
||||
"majorVersion": 2,
|
||||
"minorVersion": 0
|
||||
},
|
||||
"maxTimerDrivenThreadCount": 10,
|
||||
"maxEventDrivenThreadCount": 1,
|
||||
"registries": [],
|
||||
"parameterContexts": [],
|
||||
"parameterProviders": [],
|
||||
"controllerServices": [],
|
||||
"reportingTasks": [],
|
||||
"templates": [],
|
||||
"rootGroup": {
|
||||
"identifier": "c1b4e586-2011-3f81-a11e-8d669f084d1c",
|
||||
"instanceIdentifier": "29db3dbc-0188-1000-7025-4cab8b52d278",
|
||||
"name": "raspi2.v1",
|
||||
"comments": "",
|
||||
"position": {
|
||||
"x": 0.0,
|
||||
"y": 0.0
|
||||
},
|
||||
"processGroups": [],
|
||||
"remoteProcessGroups": [],
|
||||
"processors": [],
|
||||
"inputPorts": [],
|
||||
"outputPorts": [],
|
||||
"connections": [],
|
||||
"labels": [],
|
||||
"funnels": [],
|
||||
"controllerServices": [],
|
||||
"variables": {},
|
||||
"defaultFlowFileExpiration": "0 sec",
|
||||
"defaultBackPressureObjectThreshold": 10000,
|
||||
"defaultBackPressureDataSizeThreshold": "1 GB",
|
||||
"componentType": "PROCESS_GROUP",
|
||||
"flowFileConcurrency": "UNBOUNDED",
|
||||
"flowFileOutboundPolicy": "STREAM_WHEN_AVAILABLE"
|
||||
}
|
||||
}
|
|
@ -1,63 +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.
|
||||
|
||||
MiNiFi Config Version: 3
|
||||
Flow Controller:
|
||||
name: raspi2.v1
|
||||
comment: ''
|
||||
Core Properties:
|
||||
flow controller graceful shutdown period: 10 sec
|
||||
flow service write delay interval: 500 ms
|
||||
administrative yield duration: 30 sec
|
||||
bored yield duration: 10 millis
|
||||
max concurrent threads: 1
|
||||
FlowFile Repository:
|
||||
partitions: 256
|
||||
checkpoint interval: 2 mins
|
||||
always sync: false
|
||||
Swap:
|
||||
threshold: 20000
|
||||
in period: 5 sec
|
||||
in threads: 1
|
||||
out period: 5 sec
|
||||
out threads: 4
|
||||
Content Repository:
|
||||
content claim max appendable size: 10 MB
|
||||
content claim max flow files: 100
|
||||
always sync: false
|
||||
Provenance Repository:
|
||||
provenance rollover time: 1 min
|
||||
Component Status Repository:
|
||||
buffer size: 1440
|
||||
snapshot frequency: 1 min
|
||||
Security Properties:
|
||||
keystore: ''
|
||||
keystore type: ''
|
||||
keystore password: ''
|
||||
key password: ''
|
||||
truststore: ''
|
||||
truststore type: ''
|
||||
truststore password: ''
|
||||
ssl protocol: ''
|
||||
Sensitive Props:
|
||||
key: ''
|
||||
algorithm: NIFI_PBKDF2_AES_GCM_256
|
||||
provider: BC
|
||||
Processors: []
|
||||
Process Groups: []
|
||||
Funnels: []
|
||||
Connections: []
|
||||
Remote Process Groups: []
|
||||
NiFi Properties Overrides: {}
|
|
@ -0,0 +1,40 @@
|
|||
{
|
||||
"encodingVersion": {
|
||||
"majorVersion": 2,
|
||||
"minorVersion": 0
|
||||
},
|
||||
"maxTimerDrivenThreadCount": 10,
|
||||
"maxEventDrivenThreadCount": 1,
|
||||
"registries": [],
|
||||
"parameterContexts": [],
|
||||
"parameterProviders": [],
|
||||
"controllerServices": [],
|
||||
"reportingTasks": [],
|
||||
"templates": [],
|
||||
"rootGroup": {
|
||||
"identifier": "c1b4e586-2011-3f81-a11e-8d669f084d1c",
|
||||
"instanceIdentifier": "29db3dbc-0188-1000-7025-4cab8b52d278",
|
||||
"name": "raspi3.v1",
|
||||
"comments": "",
|
||||
"position": {
|
||||
"x": 0.0,
|
||||
"y": 0.0
|
||||
},
|
||||
"processGroups": [],
|
||||
"remoteProcessGroups": [],
|
||||
"processors": [],
|
||||
"inputPorts": [],
|
||||
"outputPorts": [],
|
||||
"connections": [],
|
||||
"labels": [],
|
||||
"funnels": [],
|
||||
"controllerServices": [],
|
||||
"variables": {},
|
||||
"defaultFlowFileExpiration": "0 sec",
|
||||
"defaultBackPressureObjectThreshold": 10000,
|
||||
"defaultBackPressureDataSizeThreshold": "1 GB",
|
||||
"componentType": "PROCESS_GROUP",
|
||||
"flowFileConcurrency": "UNBOUNDED",
|
||||
"flowFileOutboundPolicy": "STREAM_WHEN_AVAILABLE"
|
||||
}
|
||||
}
|
|
@ -0,0 +1,40 @@
|
|||
{
|
||||
"encodingVersion": {
|
||||
"majorVersion": 2,
|
||||
"minorVersion": 0
|
||||
},
|
||||
"maxTimerDrivenThreadCount": 10,
|
||||
"maxEventDrivenThreadCount": 1,
|
||||
"registries": [],
|
||||
"parameterContexts": [],
|
||||
"parameterProviders": [],
|
||||
"controllerServices": [],
|
||||
"reportingTasks": [],
|
||||
"templates": [],
|
||||
"rootGroup": {
|
||||
"identifier": "c1b4e586-2011-3f81-a11e-8d669f084d1c",
|
||||
"instanceIdentifier": "29db3dbc-0188-1000-7025-4cab8b52d278",
|
||||
"name": "raspi3.v2",
|
||||
"comments": "",
|
||||
"position": {
|
||||
"x": 0.0,
|
||||
"y": 0.0
|
||||
},
|
||||
"processGroups": [],
|
||||
"remoteProcessGroups": [],
|
||||
"processors": [],
|
||||
"inputPorts": [],
|
||||
"outputPorts": [],
|
||||
"connections": [],
|
||||
"labels": [],
|
||||
"funnels": [],
|
||||
"controllerServices": [],
|
||||
"variables": {},
|
||||
"defaultFlowFileExpiration": "0 sec",
|
||||
"defaultBackPressureObjectThreshold": 10000,
|
||||
"defaultBackPressureDataSizeThreshold": "1 GB",
|
||||
"componentType": "PROCESS_GROUP",
|
||||
"flowFileConcurrency": "UNBOUNDED",
|
||||
"flowFileOutboundPolicy": "STREAM_WHEN_AVAILABLE"
|
||||
}
|
||||
}
|
|
@ -1,63 +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.
|
||||
|
||||
MiNiFi Config Version: 3
|
||||
Flow Controller:
|
||||
name: raspi3.v1
|
||||
comment: ''
|
||||
Core Properties:
|
||||
flow controller graceful shutdown period: 10 sec
|
||||
flow service write delay interval: 500 ms
|
||||
administrative yield duration: 30 sec
|
||||
bored yield duration: 10 millis
|
||||
max concurrent threads: 1
|
||||
FlowFile Repository:
|
||||
partitions: 256
|
||||
checkpoint interval: 2 mins
|
||||
always sync: false
|
||||
Swap:
|
||||
threshold: 20000
|
||||
in period: 5 sec
|
||||
in threads: 1
|
||||
out period: 5 sec
|
||||
out threads: 4
|
||||
Content Repository:
|
||||
content claim max appendable size: 10 MB
|
||||
content claim max flow files: 100
|
||||
always sync: false
|
||||
Provenance Repository:
|
||||
provenance rollover time: 1 min
|
||||
Component Status Repository:
|
||||
buffer size: 1440
|
||||
snapshot frequency: 1 min
|
||||
Security Properties:
|
||||
keystore: ''
|
||||
keystore type: ''
|
||||
keystore password: ''
|
||||
key password: ''
|
||||
truststore: ''
|
||||
truststore type: ''
|
||||
truststore password: ''
|
||||
ssl protocol: ''
|
||||
Sensitive Props:
|
||||
key: ''
|
||||
algorithm: NIFI_PBKDF2_AES_GCM_256
|
||||
provider: BC
|
||||
Processors: []
|
||||
Process Groups: []
|
||||
Funnels: []
|
||||
Connections: []
|
||||
Remote Process Groups: []
|
||||
NiFi Properties Overrides: {}
|
|
@ -1,63 +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.
|
||||
|
||||
MiNiFi Config Version: 3
|
||||
Flow Controller:
|
||||
name: raspi3.v2
|
||||
comment: ''
|
||||
Core Properties:
|
||||
flow controller graceful shutdown period: 10 sec
|
||||
flow service write delay interval: 500 ms
|
||||
administrative yield duration: 30 sec
|
||||
bored yield duration: 10 millis
|
||||
max concurrent threads: 1
|
||||
FlowFile Repository:
|
||||
partitions: 256
|
||||
checkpoint interval: 2 mins
|
||||
always sync: false
|
||||
Swap:
|
||||
threshold: 20000
|
||||
in period: 5 sec
|
||||
in threads: 1
|
||||
out period: 5 sec
|
||||
out threads: 4
|
||||
Content Repository:
|
||||
content claim max appendable size: 10 MB
|
||||
content claim max flow files: 100
|
||||
always sync: false
|
||||
Provenance Repository:
|
||||
provenance rollover time: 1 min
|
||||
Component Status Repository:
|
||||
buffer size: 1440
|
||||
snapshot frequency: 1 min
|
||||
Security Properties:
|
||||
keystore: ''
|
||||
keystore type: ''
|
||||
keystore password: ''
|
||||
key password: ''
|
||||
truststore: ''
|
||||
truststore type: ''
|
||||
truststore password: ''
|
||||
ssl protocol: ''
|
||||
Sensitive Props:
|
||||
key: ''
|
||||
algorithm: NIFI_PBKDF2_AES_GCM_256
|
||||
provider: BC
|
||||
Processors: []
|
||||
Process Groups: []
|
||||
Funnels: []
|
||||
Connections: []
|
||||
Remote Process Groups: []
|
||||
NiFi Properties Overrides: {}
|
|
@ -1,55 +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.
|
||||
|
||||
version: "2"
|
||||
|
||||
services:
|
||||
c2:
|
||||
image: apache/nifi-minifi-c2:${minifi.c2.version}-maven
|
||||
ports:
|
||||
- "10443"
|
||||
hostname: c2
|
||||
volumes:
|
||||
- ./c2-secure/conf/c2.properties:/opt/minifi-c2/minifi-c2-${minifi.c2.version}/conf/c2.properties
|
||||
- ./c2-secure/conf/authorities.yaml:/opt/minifi-c2/minifi-c2-${minifi.c2.version}/conf/authorities.yaml
|
||||
- ./c2-secure/conf/authorizations.yaml:/opt/minifi-c2/minifi-c2-${minifi.c2.version}/conf/authorizations.yaml
|
||||
- ./c2-secure-rest/conf/minifi-c2-context.xml:/opt/minifi-c2/minifi-c2-${minifi.c2.version}/conf/minifi-c2-context.xml
|
||||
|
||||
- ./certificates-NiFiRestConfigurationProviderSecureTest/c2/keystore.jks:/opt/minifi-c2/minifi-c2-${minifi.c2.version}/conf/keystore.jks
|
||||
- ./certificates-NiFiRestConfigurationProviderSecureTest/c2/truststore.jks:/opt/minifi-c2/minifi-c2-${minifi.c2.version}/conf/truststore.jks
|
||||
|
||||
squid:
|
||||
image: chrisdaish/squid
|
||||
ports:
|
||||
- "3128"
|
||||
hostname: squid
|
||||
volumes:
|
||||
- ./squid/squid.conf:/etc/squid/squid.conf
|
||||
|
||||
mocknifi:
|
||||
image: python:2
|
||||
ports:
|
||||
- "8443"
|
||||
hostname: mocknifi
|
||||
volumes:
|
||||
- ./mocknifi-secure/server.py:/root/server.py
|
||||
- ./mocknifi/www:/var/www
|
||||
|
||||
- ./certificates-NiFiRestConfigurationProviderSecureTest/mocknifi/cert.pem:/root/cert.pem
|
||||
- ./certificates-NiFiRestConfigurationProviderSecureTest/mocknifi/ca.pem:/root/ca.pem
|
||||
working_dir: /var/www
|
||||
entrypoint:
|
||||
- python
|
||||
- /root/server.py
|
|
@ -1,37 +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.
|
||||
|
||||
version: "2"
|
||||
|
||||
services:
|
||||
c2:
|
||||
image: apache/nifi-minifi-c2:${minifi.c2.version}-maven
|
||||
ports:
|
||||
- "10090"
|
||||
hostname: c2
|
||||
volumes:
|
||||
- ./c2-unsecure-rest/conf/minifi-c2-context.xml:/opt/minifi-c2/minifi-c2-${minifi.c2.version}/conf/minifi-c2-context.xml
|
||||
mocknifi:
|
||||
image: python:2
|
||||
ports:
|
||||
- "8080"
|
||||
hostname: mocknifi
|
||||
volumes:
|
||||
- ./mocknifi-unsecure/server.py:/root/server.py
|
||||
- ./mocknifi/www:/var/www
|
||||
working_dir: /var/www
|
||||
entrypoint:
|
||||
- python
|
||||
- /root/server.py
|
|
@ -1,48 +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.
|
||||
|
||||
#!/usr/bin/env python
|
||||
|
||||
import logging
|
||||
|
||||
from argparse import ArgumentParser
|
||||
from BaseHTTPServer import HTTPServer
|
||||
from os import chdir
|
||||
from SimpleHTTPServer import SimpleHTTPRequestHandler
|
||||
from SocketServer import ThreadingMixIn
|
||||
from ssl import CERT_REQUIRED, wrap_socket
|
||||
|
||||
# Needs to be threaded or health check hangs the server
|
||||
class ThreadedHTTPServer(ThreadingMixIn, HTTPServer):
|
||||
pass
|
||||
|
||||
if __name__ == '__main__':
|
||||
logging.basicConfig(level=logging.DEBUG)
|
||||
parser = ArgumentParser(description='Serve up directory with ssl')
|
||||
parser.add_argument('--dir', default='/var/www')
|
||||
parser.add_argument('--port', type=int, default=8443)
|
||||
parser.add_argument('--cert', default='/root/cert.pem')
|
||||
parser.add_argument('--ca', default='/root/ca.pem')
|
||||
|
||||
logging.debug('About to parse arguments')
|
||||
args = parser.parse_args()
|
||||
|
||||
logging.debug('Serving directory ' + args.dir + ' via HTTPS at port ' + str(args.port))
|
||||
|
||||
chdir(args.dir)
|
||||
|
||||
server = ThreadedHTTPServer(('', args.port), SimpleHTTPRequestHandler)
|
||||
server.socket = wrap_socket(server.socket, certfile=args.cert, ca_certs=args.ca, server_side=True, cert_reqs=CERT_REQUIRED)
|
||||
server.serve_forever()
|
|
@ -1,44 +0,0 @@
|
|||
# Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
# contributor license agreements. See the NOTICE file distributed with
|
||||
# this work for additional information regarding copyright ownership.
|
||||
# The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
# (the \"License\"); you may not use this file except in compliance with
|
||||
# the License. You may obtain a copy of the License at
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing, software
|
||||
# distributed under the License is distributed on an \"AS IS\" BASIS,
|
||||
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
# See the License for the specific language governing permissions and
|
||||
# limitations under the License.
|
||||
|
||||
#!/usr/bin/env python
|
||||
|
||||
import logging
|
||||
|
||||
from argparse import ArgumentParser
|
||||
from BaseHTTPServer import HTTPServer
|
||||
from os import chdir
|
||||
from SimpleHTTPServer import SimpleHTTPRequestHandler
|
||||
from SocketServer import ThreadingMixIn
|
||||
|
||||
# Needs to be threaded or health check hangs the server
|
||||
class ThreadedHTTPServer(ThreadingMixIn, HTTPServer):
|
||||
pass
|
||||
|
||||
if __name__ == '__main__':
|
||||
logging.basicConfig(level=logging.DEBUG)
|
||||
parser = ArgumentParser(description='Serve up directory over http')
|
||||
parser.add_argument('--dir', default='/var/www')
|
||||
parser.add_argument('--port', type=int, default=8080)
|
||||
|
||||
logging.debug('About to parse arguments')
|
||||
args = parser.parse_args()
|
||||
|
||||
logging.debug('Serving directory ' + args.dir + ' via HTTP at port ' + str(args.port))
|
||||
|
||||
chdir(args.dir)
|
||||
|
||||
server = ThreadedHTTPServer(('', args.port), SimpleHTTPRequestHandler)
|
||||
server.serve_forever()
|
|
@ -1,53 +0,0 @@
|
|||
{
|
||||
"generated": "15:41:45 EDT",
|
||||
"templates": [
|
||||
{
|
||||
"id": "f080ec50-ca32-4b36-8453-5a7145bec4c5",
|
||||
"permissions": {
|
||||
"canRead": true,
|
||||
"canWrite": true
|
||||
},
|
||||
"template": {
|
||||
"description": "",
|
||||
"encoding-version": "1.0",
|
||||
"groupId": "8f8eda5e-015a-1000-a9c1-b7e4fe10ae83",
|
||||
"id": "f080ec50-ca32-4b36-8453-5a7145bec4c5",
|
||||
"name": "raspi3.v1",
|
||||
"timestamp": "03/07/2017 11:13:03 EST",
|
||||
"uri": "http://localhost:8080/nifi-api/templates/f080ec50-ca32-4b36-8453-5a7145bec4c5"
|
||||
}
|
||||
},
|
||||
{
|
||||
"id": "87048385-a6ca-42fe-b2d8-6a563cedd036",
|
||||
"permissions": {
|
||||
"canRead": true,
|
||||
"canWrite": true
|
||||
},
|
||||
"template": {
|
||||
"description": "",
|
||||
"encoding-version": "1.0",
|
||||
"groupId": "8f8eda5e-015a-1000-a9c1-b7e4fe10ae83",
|
||||
"id": "87048385-a6ca-42fe-b2d8-6a563cedd036",
|
||||
"name": "raspi2.v1",
|
||||
"timestamp": "03/17/2017 15:41:33 EDT",
|
||||
"uri": "http://localhost:8080/nifi-api/templates/87048385-a6ca-42fe-b2d8-6a563cedd036"
|
||||
}
|
||||
},
|
||||
{
|
||||
"id": "dd737a3e-333e-40df-a0bc-d7e28c8e6843",
|
||||
"permissions": {
|
||||
"canRead": true,
|
||||
"canWrite": true
|
||||
},
|
||||
"template": {
|
||||
"description": "",
|
||||
"encoding-version": "1.0",
|
||||
"groupId": "8f8eda5e-015a-1000-a9c1-b7e4fe10ae83",
|
||||
"id": "dd737a3e-333e-40df-a0bc-d7e28c8e6843",
|
||||
"name": "raspi3.v2",
|
||||
"timestamp": "03/17/2017 13:22:58 EDT",
|
||||
"uri": "http://localhost:8080/nifi-api/templates/dd737a3e-333e-40df-a0bc-d7e28c8e6843"
|
||||
}
|
||||
}
|
||||
]
|
||||
}
|
|
@ -1,203 +0,0 @@
|
|||
<?xml version="1.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.
|
||||
-->
|
||||
<template encoding-version="1.0">
|
||||
<description></description>
|
||||
<groupId>8f8eda5e-015a-1000-a9c1-b7e4fe10ae83</groupId>
|
||||
<name>raspi2.v1</name>
|
||||
<snippet>
|
||||
<connections>
|
||||
<id>8f96f2a9-015a-1000-0000-000000000000</id>
|
||||
<parentGroupId>8f8eda5e-015a-1000-0000-000000000000</parentGroupId>
|
||||
<backPressureDataSizeThreshold>1 GB</backPressureDataSizeThreshold>
|
||||
<backPressureObjectThreshold>10000</backPressureObjectThreshold>
|
||||
<destination>
|
||||
<groupId>8f8eda5e-015a-1000-0000-000000000000</groupId>
|
||||
<id>8f96bf68-015a-1000-0000-000000000000</id>
|
||||
<type>PROCESSOR</type>
|
||||
</destination>
|
||||
<flowFileExpiration>0 sec</flowFileExpiration>
|
||||
<labelIndex>1</labelIndex>
|
||||
<name></name>
|
||||
<selectedRelationships>success</selectedRelationships>
|
||||
<source>
|
||||
<groupId>8f8eda5e-015a-1000-0000-000000000000</groupId>
|
||||
<id>8f96e313-015a-1000-0000-000000000000</id>
|
||||
<type>PROCESSOR</type>
|
||||
</source>
|
||||
<zIndex>0</zIndex>
|
||||
</connections>
|
||||
<processors>
|
||||
<id>8f96bf68-015a-1000-0000-000000000000</id>
|
||||
<parentGroupId>8f8eda5e-015a-1000-0000-000000000000</parentGroupId>
|
||||
<position>
|
||||
<x>14.0</x>
|
||||
<y>253.0</y>
|
||||
</position>
|
||||
<config>
|
||||
<bulletinLevel>WARN</bulletinLevel>
|
||||
<comments></comments>
|
||||
<concurrentlySchedulableTaskCount>1</concurrentlySchedulableTaskCount>
|
||||
<descriptors>
|
||||
<entry>
|
||||
<key>Log Level</key>
|
||||
<value>
|
||||
<name>Log Level</name>
|
||||
</value>
|
||||
</entry>
|
||||
<entry>
|
||||
<key>Log Payload</key>
|
||||
<value>
|
||||
<name>Log Payload</name>
|
||||
</value>
|
||||
</entry>
|
||||
<entry>
|
||||
<key>Attributes to Log</key>
|
||||
<value>
|
||||
<name>Attributes to Log</name>
|
||||
</value>
|
||||
</entry>
|
||||
<entry>
|
||||
<key>Attributes to Ignore</key>
|
||||
<value>
|
||||
<name>Attributes to Ignore</name>
|
||||
</value>
|
||||
</entry>
|
||||
<entry>
|
||||
<key>Log prefix</key>
|
||||
<value>
|
||||
<name>Log prefix</name>
|
||||
</value>
|
||||
</entry>
|
||||
</descriptors>
|
||||
<executionNode>ALL</executionNode>
|
||||
<lossTolerant>false</lossTolerant>
|
||||
<penaltyDuration>30 sec</penaltyDuration>
|
||||
<properties>
|
||||
<entry>
|
||||
<key>Log Level</key>
|
||||
<value>info</value>
|
||||
</entry>
|
||||
<entry>
|
||||
<key>Log Payload</key>
|
||||
<value>true</value>
|
||||
</entry>
|
||||
<entry>
|
||||
<key>Attributes to Log</key>
|
||||
</entry>
|
||||
<entry>
|
||||
<key>Attributes to Ignore</key>
|
||||
</entry>
|
||||
<entry>
|
||||
<key>Log prefix</key>
|
||||
</entry>
|
||||
</properties>
|
||||
<runDurationMillis>0</runDurationMillis>
|
||||
<schedulingPeriod>0 sec</schedulingPeriod>
|
||||
<schedulingStrategy>TIMER_DRIVEN</schedulingStrategy>
|
||||
<yieldDuration>1 sec</yieldDuration>
|
||||
</config>
|
||||
<name>LogAttribute</name>
|
||||
<relationships>
|
||||
<autoTerminate>true</autoTerminate>
|
||||
<name>success</name>
|
||||
</relationships>
|
||||
<style></style>
|
||||
<type>org.apache.nifi.processors.standard.LogAttribute</type>
|
||||
</processors>
|
||||
<processors>
|
||||
<id>8f96e313-015a-1000-0000-000000000000</id>
|
||||
<parentGroupId>8f8eda5e-015a-1000-0000-000000000000</parentGroupId>
|
||||
<position>
|
||||
<x>0.0</x>
|
||||
<y>0.0</y>
|
||||
</position>
|
||||
<config>
|
||||
<bulletinLevel>WARN</bulletinLevel>
|
||||
<comments></comments>
|
||||
<concurrentlySchedulableTaskCount>1</concurrentlySchedulableTaskCount>
|
||||
<descriptors>
|
||||
<entry>
|
||||
<key>File Size</key>
|
||||
<value>
|
||||
<name>File Size</name>
|
||||
</value>
|
||||
</entry>
|
||||
<entry>
|
||||
<key>Batch Size</key>
|
||||
<value>
|
||||
<name>Batch Size</name>
|
||||
</value>
|
||||
</entry>
|
||||
<entry>
|
||||
<key>Data Format</key>
|
||||
<value>
|
||||
<name>Data Format</name>
|
||||
</value>
|
||||
</entry>
|
||||
<entry>
|
||||
<key>Unique FlowFiles</key>
|
||||
<value>
|
||||
<name>Unique FlowFiles</name>
|
||||
</value>
|
||||
</entry>
|
||||
<entry>
|
||||
<key>generate-ff-custom-text</key>
|
||||
<value>
|
||||
<name>generate-ff-custom-text</name>
|
||||
</value>
|
||||
</entry>
|
||||
</descriptors>
|
||||
<executionNode>ALL</executionNode>
|
||||
<lossTolerant>false</lossTolerant>
|
||||
<penaltyDuration>30 sec</penaltyDuration>
|
||||
<properties>
|
||||
<entry>
|
||||
<key>File Size</key>
|
||||
<value>0B</value>
|
||||
</entry>
|
||||
<entry>
|
||||
<key>Batch Size</key>
|
||||
<value>1</value>
|
||||
</entry>
|
||||
<entry>
|
||||
<key>Data Format</key>
|
||||
<value>Text</value>
|
||||
</entry>
|
||||
<entry>
|
||||
<key>Unique FlowFiles</key>
|
||||
<value>false</value>
|
||||
</entry>
|
||||
<entry>
|
||||
<key>generate-ff-custom-text</key>
|
||||
<value>abcdefg</value>
|
||||
</entry>
|
||||
</properties>
|
||||
<runDurationMillis>0</runDurationMillis>
|
||||
<schedulingPeriod>0 sec</schedulingPeriod>
|
||||
<schedulingStrategy>TIMER_DRIVEN</schedulingStrategy>
|
||||
<yieldDuration>1 sec</yieldDuration>
|
||||
</config>
|
||||
<name>GenerateFlowFile</name>
|
||||
<relationships>
|
||||
<autoTerminate>false</autoTerminate>
|
||||
<name>success</name>
|
||||
</relationships>
|
||||
<style></style>
|
||||
<type>org.apache.nifi.processors.standard.GenerateFlowFile</type>
|
||||
</processors>
|
||||
</snippet>
|
||||
<timestamp>03/17/2017 15:41:33 EDT</timestamp>
|
||||
</template>
|
|
@ -1,203 +0,0 @@
|
|||
<?xml version="1.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.
|
||||
-->
|
||||
<template encoding-version="1.0">
|
||||
<description></description>
|
||||
<groupId>8f8eda5e-015a-1000-a9c1-b7e4fe10ae83</groupId>
|
||||
<name>raspi3.v2</name>
|
||||
<snippet>
|
||||
<connections>
|
||||
<id>8f96f2a9-015a-1000-0000-000000000000</id>
|
||||
<parentGroupId>8f8eda5e-015a-1000-0000-000000000000</parentGroupId>
|
||||
<backPressureDataSizeThreshold>1 GB</backPressureDataSizeThreshold>
|
||||
<backPressureObjectThreshold>10000</backPressureObjectThreshold>
|
||||
<destination>
|
||||
<groupId>8f8eda5e-015a-1000-0000-000000000000</groupId>
|
||||
<id>8f96bf68-015a-1000-0000-000000000000</id>
|
||||
<type>PROCESSOR</type>
|
||||
</destination>
|
||||
<flowFileExpiration>0 sec</flowFileExpiration>
|
||||
<labelIndex>1</labelIndex>
|
||||
<name></name>
|
||||
<selectedRelationships>success</selectedRelationships>
|
||||
<source>
|
||||
<groupId>8f8eda5e-015a-1000-0000-000000000000</groupId>
|
||||
<id>8f96e313-015a-1000-0000-000000000000</id>
|
||||
<type>PROCESSOR</type>
|
||||
</source>
|
||||
<zIndex>0</zIndex>
|
||||
</connections>
|
||||
<processors>
|
||||
<id>8f96bf68-015a-1000-0000-000000000000</id>
|
||||
<parentGroupId>8f8eda5e-015a-1000-0000-000000000000</parentGroupId>
|
||||
<position>
|
||||
<x>14.0</x>
|
||||
<y>253.0</y>
|
||||
</position>
|
||||
<config>
|
||||
<bulletinLevel>WARN</bulletinLevel>
|
||||
<comments></comments>
|
||||
<concurrentlySchedulableTaskCount>1</concurrentlySchedulableTaskCount>
|
||||
<descriptors>
|
||||
<entry>
|
||||
<key>Log Level</key>
|
||||
<value>
|
||||
<name>Log Level</name>
|
||||
</value>
|
||||
</entry>
|
||||
<entry>
|
||||
<key>Log Payload</key>
|
||||
<value>
|
||||
<name>Log Payload</name>
|
||||
</value>
|
||||
</entry>
|
||||
<entry>
|
||||
<key>Attributes to Log</key>
|
||||
<value>
|
||||
<name>Attributes to Log</name>
|
||||
</value>
|
||||
</entry>
|
||||
<entry>
|
||||
<key>Attributes to Ignore</key>
|
||||
<value>
|
||||
<name>Attributes to Ignore</name>
|
||||
</value>
|
||||
</entry>
|
||||
<entry>
|
||||
<key>Log prefix</key>
|
||||
<value>
|
||||
<name>Log prefix</name>
|
||||
</value>
|
||||
</entry>
|
||||
</descriptors>
|
||||
<executionNode>ALL</executionNode>
|
||||
<lossTolerant>false</lossTolerant>
|
||||
<penaltyDuration>30 sec</penaltyDuration>
|
||||
<properties>
|
||||
<entry>
|
||||
<key>Log Level</key>
|
||||
<value>info</value>
|
||||
</entry>
|
||||
<entry>
|
||||
<key>Log Payload</key>
|
||||
<value>true</value>
|
||||
</entry>
|
||||
<entry>
|
||||
<key>Attributes to Log</key>
|
||||
</entry>
|
||||
<entry>
|
||||
<key>Attributes to Ignore</key>
|
||||
</entry>
|
||||
<entry>
|
||||
<key>Log prefix</key>
|
||||
</entry>
|
||||
</properties>
|
||||
<runDurationMillis>0</runDurationMillis>
|
||||
<schedulingPeriod>0 sec</schedulingPeriod>
|
||||
<schedulingStrategy>TIMER_DRIVEN</schedulingStrategy>
|
||||
<yieldDuration>1 sec</yieldDuration>
|
||||
</config>
|
||||
<name>LogAttribute</name>
|
||||
<relationships>
|
||||
<autoTerminate>true</autoTerminate>
|
||||
<name>success</name>
|
||||
</relationships>
|
||||
<style></style>
|
||||
<type>org.apache.nifi.processors.standard.LogAttribute</type>
|
||||
</processors>
|
||||
<processors>
|
||||
<id>8f96e313-015a-1000-0000-000000000000</id>
|
||||
<parentGroupId>8f8eda5e-015a-1000-0000-000000000000</parentGroupId>
|
||||
<position>
|
||||
<x>0.0</x>
|
||||
<y>0.0</y>
|
||||
</position>
|
||||
<config>
|
||||
<bulletinLevel>WARN</bulletinLevel>
|
||||
<comments></comments>
|
||||
<concurrentlySchedulableTaskCount>1</concurrentlySchedulableTaskCount>
|
||||
<descriptors>
|
||||
<entry>
|
||||
<key>File Size</key>
|
||||
<value>
|
||||
<name>File Size</name>
|
||||
</value>
|
||||
</entry>
|
||||
<entry>
|
||||
<key>Batch Size</key>
|
||||
<value>
|
||||
<name>Batch Size</name>
|
||||
</value>
|
||||
</entry>
|
||||
<entry>
|
||||
<key>Data Format</key>
|
||||
<value>
|
||||
<name>Data Format</name>
|
||||
</value>
|
||||
</entry>
|
||||
<entry>
|
||||
<key>Unique FlowFiles</key>
|
||||
<value>
|
||||
<name>Unique FlowFiles</name>
|
||||
</value>
|
||||
</entry>
|
||||
<entry>
|
||||
<key>generate-ff-custom-text</key>
|
||||
<value>
|
||||
<name>generate-ff-custom-text</name>
|
||||
</value>
|
||||
</entry>
|
||||
</descriptors>
|
||||
<executionNode>ALL</executionNode>
|
||||
<lossTolerant>false</lossTolerant>
|
||||
<penaltyDuration>30 sec</penaltyDuration>
|
||||
<properties>
|
||||
<entry>
|
||||
<key>File Size</key>
|
||||
<value>0B</value>
|
||||
</entry>
|
||||
<entry>
|
||||
<key>Batch Size</key>
|
||||
<value>1</value>
|
||||
</entry>
|
||||
<entry>
|
||||
<key>Data Format</key>
|
||||
<value>Text</value>
|
||||
</entry>
|
||||
<entry>
|
||||
<key>Unique FlowFiles</key>
|
||||
<value>false</value>
|
||||
</entry>
|
||||
<entry>
|
||||
<key>generate-ff-custom-text</key>
|
||||
<value>abcdefg</value>
|
||||
</entry>
|
||||
</properties>
|
||||
<runDurationMillis>0</runDurationMillis>
|
||||
<schedulingPeriod>0 sec</schedulingPeriod>
|
||||
<schedulingStrategy>TIMER_DRIVEN</schedulingStrategy>
|
||||
<yieldDuration>1 sec</yieldDuration>
|
||||
</config>
|
||||
<name>GenerateFlowFile</name>
|
||||
<relationships>
|
||||
<autoTerminate>false</autoTerminate>
|
||||
<name>success</name>
|
||||
</relationships>
|
||||
<style></style>
|
||||
<type>org.apache.nifi.processors.standard.GenerateFlowFile</type>
|
||||
</processors>
|
||||
</snippet>
|
||||
<timestamp>03/17/2017 13:22:58 EDT</timestamp>
|
||||
</template>
|
|
@ -1,202 +0,0 @@
|
|||
<?xml version="1.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.
|
||||
-->
|
||||
<template encoding-version="1.0">
|
||||
<description></description>
|
||||
<groupId>8f8eda5e-015a-1000-a9c1-b7e4fe10ae83</groupId>
|
||||
<name>raspi3.v1</name>
|
||||
<snippet>
|
||||
<connections>
|
||||
<id>8f96f2a9-015a-1000-0000-000000000000</id>
|
||||
<parentGroupId>8f8eda5e-015a-1000-0000-000000000000</parentGroupId>
|
||||
<backPressureDataSizeThreshold>1 GB</backPressureDataSizeThreshold>
|
||||
<backPressureObjectThreshold>10000</backPressureObjectThreshold>
|
||||
<destination>
|
||||
<groupId>8f8eda5e-015a-1000-0000-000000000000</groupId>
|
||||
<id>8f96bf68-015a-1000-0000-000000000000</id>
|
||||
<type>PROCESSOR</type>
|
||||
</destination>
|
||||
<flowFileExpiration>0 sec</flowFileExpiration>
|
||||
<labelIndex>1</labelIndex>
|
||||
<name></name>
|
||||
<selectedRelationships>success</selectedRelationships>
|
||||
<source>
|
||||
<groupId>8f8eda5e-015a-1000-0000-000000000000</groupId>
|
||||
<id>8f96e313-015a-1000-0000-000000000000</id>
|
||||
<type>PROCESSOR</type>
|
||||
</source>
|
||||
<zIndex>0</zIndex>
|
||||
</connections>
|
||||
<processors>
|
||||
<id>8f96bf68-015a-1000-0000-000000000000</id>
|
||||
<parentGroupId>8f8eda5e-015a-1000-0000-000000000000</parentGroupId>
|
||||
<position>
|
||||
<x>14.0</x>
|
||||
<y>253.0</y>
|
||||
</position>
|
||||
<config>
|
||||
<bulletinLevel>WARN</bulletinLevel>
|
||||
<comments></comments>
|
||||
<concurrentlySchedulableTaskCount>1</concurrentlySchedulableTaskCount>
|
||||
<descriptors>
|
||||
<entry>
|
||||
<key>Log prefix</key>
|
||||
<value>
|
||||
<name>Log prefix</name>
|
||||
</value>
|
||||
</entry>
|
||||
<entry>
|
||||
<key>Log Level</key>
|
||||
<value>
|
||||
<name>Log Level</name>
|
||||
</value>
|
||||
</entry>
|
||||
<entry>
|
||||
<key>Attributes to Ignore</key>
|
||||
<value>
|
||||
<name>Attributes to Ignore</name>
|
||||
</value>
|
||||
</entry>
|
||||
<entry>
|
||||
<key>Attributes to Log</key>
|
||||
<value>
|
||||
<name>Attributes to Log</name>
|
||||
</value>
|
||||
</entry>
|
||||
<entry>
|
||||
<key>Log Payload</key>
|
||||
<value>
|
||||
<name>Log Payload</name>
|
||||
</value>
|
||||
</entry>
|
||||
</descriptors>
|
||||
<executionNode>ALL</executionNode>
|
||||
<lossTolerant>false</lossTolerant>
|
||||
<penaltyDuration>30 sec</penaltyDuration>
|
||||
<properties>
|
||||
<entry>
|
||||
<key>Log prefix</key>
|
||||
</entry>
|
||||
<entry>
|
||||
<key>Log Level</key>
|
||||
<value>info</value>
|
||||
</entry>
|
||||
<entry>
|
||||
<key>Attributes to Ignore</key>
|
||||
</entry>
|
||||
<entry>
|
||||
<key>Attributes to Log</key>
|
||||
</entry>
|
||||
<entry>
|
||||
<key>Log Payload</key>
|
||||
<value>true</value>
|
||||
</entry>
|
||||
</properties>
|
||||
<runDurationMillis>0</runDurationMillis>
|
||||
<schedulingPeriod>0 sec</schedulingPeriod>
|
||||
<schedulingStrategy>TIMER_DRIVEN</schedulingStrategy>
|
||||
<yieldDuration>1 sec</yieldDuration>
|
||||
</config>
|
||||
<name>LogAttribute</name>
|
||||
<relationships>
|
||||
<autoTerminate>true</autoTerminate>
|
||||
<name>success</name>
|
||||
</relationships>
|
||||
<style></style>
|
||||
<type>org.apache.nifi.processors.standard.LogAttribute</type>
|
||||
</processors>
|
||||
<processors>
|
||||
<id>8f96e313-015a-1000-0000-000000000000</id>
|
||||
<parentGroupId>8f8eda5e-015a-1000-0000-000000000000</parentGroupId>
|
||||
<position>
|
||||
<x>0.0</x>
|
||||
<y>0.0</y>
|
||||
</position>
|
||||
<config>
|
||||
<bulletinLevel>WARN</bulletinLevel>
|
||||
<comments></comments>
|
||||
<concurrentlySchedulableTaskCount>1</concurrentlySchedulableTaskCount>
|
||||
<descriptors>
|
||||
<entry>
|
||||
<key>File Size</key>
|
||||
<value>
|
||||
<name>File Size</name>
|
||||
</value>
|
||||
</entry>
|
||||
<entry>
|
||||
<key>generate-ff-custom-text</key>
|
||||
<value>
|
||||
<name>generate-ff-custom-text</name>
|
||||
</value>
|
||||
</entry>
|
||||
<entry>
|
||||
<key>Batch Size</key>
|
||||
<value>
|
||||
<name>Batch Size</name>
|
||||
</value>
|
||||
</entry>
|
||||
<entry>
|
||||
<key>Unique FlowFiles</key>
|
||||
<value>
|
||||
<name>Unique FlowFiles</name>
|
||||
</value>
|
||||
</entry>
|
||||
<entry>
|
||||
<key>Data Format</key>
|
||||
<value>
|
||||
<name>Data Format</name>
|
||||
</value>
|
||||
</entry>
|
||||
</descriptors>
|
||||
<executionNode>ALL</executionNode>
|
||||
<lossTolerant>false</lossTolerant>
|
||||
<penaltyDuration>30 sec</penaltyDuration>
|
||||
<properties>
|
||||
<entry>
|
||||
<key>File Size</key>
|
||||
<value>0B</value>
|
||||
</entry>
|
||||
<entry>
|
||||
<key>generate-ff-custom-text</key>
|
||||
</entry>
|
||||
<entry>
|
||||
<key>Batch Size</key>
|
||||
<value>1</value>
|
||||
</entry>
|
||||
<entry>
|
||||
<key>Unique FlowFiles</key>
|
||||
<value>false</value>
|
||||
</entry>
|
||||
<entry>
|
||||
<key>Data Format</key>
|
||||
<value>Text</value>
|
||||
</entry>
|
||||
</properties>
|
||||
<runDurationMillis>0</runDurationMillis>
|
||||
<schedulingPeriod>0 sec</schedulingPeriod>
|
||||
<schedulingStrategy>TIMER_DRIVEN</schedulingStrategy>
|
||||
<yieldDuration>1 sec</yieldDuration>
|
||||
</config>
|
||||
<name>GenerateFlowFile</name>
|
||||
<relationships>
|
||||
<autoTerminate>false</autoTerminate>
|
||||
<name>success</name>
|
||||
</relationships>
|
||||
<style></style>
|
||||
<type>org.apache.nifi.processors.standard.GenerateFlowFile</type>
|
||||
</processors>
|
||||
</snippet>
|
||||
<timestamp>03/07/2017 11:13:03 EST</timestamp>
|
||||
</template>
|
|
@ -38,7 +38,6 @@ import java.io.UnsupportedEncodingException;
|
|||
import java.net.HttpURLConnection;
|
||||
import java.net.URLEncoder;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.security.GeneralSecurityException;
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
@ -52,10 +51,6 @@ public class DelegatingConfigurationProvider implements ConfigurationProvider {
|
|||
private final HttpConnector httpConnector;
|
||||
private final ObjectMapper objectMapper;
|
||||
|
||||
public DelegatingConfigurationProvider(ConfigurationCache configurationCache, String delegateUrl) throws InvalidParameterException, GeneralSecurityException, IOException {
|
||||
this(configurationCache, new HttpConnector(delegateUrl));
|
||||
}
|
||||
|
||||
public DelegatingConfigurationProvider(ConfigurationCache configurationCache, HttpConnector httpConnector) {
|
||||
this.configurationCache = configurationCache;
|
||||
this.httpConnector = httpConnector;
|
||||
|
|
|
@ -57,7 +57,7 @@ public class DelegatingConfigurationProviderTest {
|
|||
|
||||
@BeforeEach
|
||||
public void setup() throws ConfigurationProviderException {
|
||||
contentType = "text/yml";
|
||||
contentType = "test/json";
|
||||
version = 2;
|
||||
parameters = new HashMap<>();
|
||||
parameters.put("net", Collections.singletonList("edge"));
|
||||
|
|
|
@ -1,188 +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.c2.provider.nifi.rest;
|
||||
|
||||
import com.fasterxml.jackson.core.JsonFactory;
|
||||
import org.apache.nifi.minifi.c2.api.Configuration;
|
||||
import org.apache.nifi.minifi.c2.api.ConfigurationProvider;
|
||||
import org.apache.nifi.minifi.c2.api.ConfigurationProviderException;
|
||||
import org.apache.nifi.minifi.c2.api.InvalidParameterException;
|
||||
import org.apache.nifi.minifi.c2.api.cache.ConfigurationCache;
|
||||
import org.apache.nifi.minifi.c2.api.cache.WriteableConfiguration;
|
||||
import org.apache.nifi.minifi.c2.api.util.Pair;
|
||||
import org.apache.nifi.minifi.c2.provider.util.HttpConnector;
|
||||
import org.apache.nifi.minifi.commons.schema.ConfigSchema;
|
||||
import org.apache.nifi.minifi.commons.schema.serialization.SchemaSaver;
|
||||
import org.apache.nifi.minifi.toolkit.configuration.ConfigMain;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import javax.xml.bind.JAXBException;
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.net.HttpURLConnection;
|
||||
import java.security.GeneralSecurityException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.Spliterator;
|
||||
import java.util.Spliterators;
|
||||
import java.util.regex.Matcher;
|
||||
import java.util.regex.Pattern;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
import java.util.stream.StreamSupport;
|
||||
|
||||
public class NiFiRestConfigurationProvider implements ConfigurationProvider {
|
||||
public static final String CONTENT_TYPE = "text/yml";
|
||||
private static final Logger logger = LoggerFactory.getLogger(NiFiRestConfigurationProvider.class);
|
||||
private final JsonFactory jsonFactory = new JsonFactory();
|
||||
private final ConfigurationCache configurationCache;
|
||||
private final HttpConnector httpConnector;
|
||||
private final String templateNamePattern;
|
||||
|
||||
public NiFiRestConfigurationProvider(ConfigurationCache configurationCache, String nifiUrl, String templateNamePattern) throws InvalidParameterException, GeneralSecurityException, IOException {
|
||||
this(configurationCache, new HttpConnector(nifiUrl), templateNamePattern);
|
||||
}
|
||||
|
||||
public NiFiRestConfigurationProvider(ConfigurationCache configurationCache, HttpConnector httpConnector, String templateNamePattern) {
|
||||
this.configurationCache = configurationCache;
|
||||
this.httpConnector = httpConnector;
|
||||
this.templateNamePattern = templateNamePattern;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<String> getContentTypes() {
|
||||
return Collections.singletonList(CONTENT_TYPE);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Configuration getConfiguration(String contentType, Integer version, Map<String, List<String>> parameters) throws ConfigurationProviderException {
|
||||
if (!CONTENT_TYPE.equals(contentType)) {
|
||||
throw new ConfigurationProviderException("Unsupported content type: " + contentType + " supported value is " + CONTENT_TYPE);
|
||||
}
|
||||
String filename = templateNamePattern;
|
||||
for (Map.Entry<String, List<String>> entry : parameters.entrySet()) {
|
||||
if (entry.getValue().size() != 1) {
|
||||
throw new InvalidParameterException("Multiple values for same parameter not supported in this provider.");
|
||||
}
|
||||
filename = filename.replaceAll(Pattern.quote("${" + entry.getKey() + "}"), entry.getValue().get(0));
|
||||
}
|
||||
int index = filename.indexOf("${");
|
||||
while (index != -1) {
|
||||
int endIndex = filename.indexOf("}", index);
|
||||
if (endIndex == -1) {
|
||||
break;
|
||||
}
|
||||
String variable = filename.substring(index + 2, endIndex);
|
||||
if (!"version".equals(variable)) {
|
||||
throw new InvalidParameterException("Found unsubstituted parameter " + variable);
|
||||
}
|
||||
index = endIndex + 1;
|
||||
}
|
||||
|
||||
String id = null;
|
||||
if (version == null) {
|
||||
String filenamePattern = Arrays.stream(filename.split(Pattern.quote("${version}"), -1)).map(Pattern::quote).collect(Collectors.joining("([0-9]+)"));
|
||||
Pair<String, Integer> maxIdAndVersion = getMaxIdAndVersion(filenamePattern);
|
||||
id = maxIdAndVersion.getFirst();
|
||||
version = maxIdAndVersion.getSecond();
|
||||
}
|
||||
filename = filename.replaceAll(Pattern.quote("${version}"), Integer.toString(version));
|
||||
WriteableConfiguration configuration = configurationCache.getCacheFileInfo(contentType, parameters).getConfiguration(version);
|
||||
if (configuration.exists()) {
|
||||
if (logger.isDebugEnabled()) {
|
||||
logger.debug("Configuration " + configuration + " exists and can be served from configurationCache.");
|
||||
}
|
||||
} else {
|
||||
if (logger.isDebugEnabled()) {
|
||||
logger.debug("Configuration " + configuration + " doesn't exist, will need to download and convert template.");
|
||||
}
|
||||
if (id == null) {
|
||||
try {
|
||||
String tmpFilename = templateNamePattern;
|
||||
for (Map.Entry<String, List<String>> entry : parameters.entrySet()) {
|
||||
if (entry.getValue().size() != 1) {
|
||||
throw new InvalidParameterException("Multiple values for same parameter not supported in this provider.");
|
||||
}
|
||||
tmpFilename = tmpFilename.replaceAll(Pattern.quote("${" + entry.getKey() + "}"), entry.getValue().get(0));
|
||||
}
|
||||
Pair<Stream<Pair<String, String>>, Closeable> streamCloseablePair = getIdAndFilenameStream();
|
||||
try {
|
||||
String finalFilename = filename;
|
||||
id = streamCloseablePair.getFirst().filter(p -> finalFilename.equals(p.getSecond())).map(Pair::getFirst).findFirst()
|
||||
.orElseThrow(() -> new InvalidParameterException("Unable to find template named " + finalFilename));
|
||||
} finally {
|
||||
streamCloseablePair.getSecond().close();
|
||||
}
|
||||
} catch (IOException|TemplatesIteratorException e) {
|
||||
throw new ConfigurationProviderException("Unable to retrieve template list", e);
|
||||
}
|
||||
}
|
||||
|
||||
HttpURLConnection urlConnection = httpConnector.get("/templates/" + id + "/download");
|
||||
|
||||
try (InputStream inputStream = urlConnection.getInputStream()){
|
||||
ConfigSchema configSchema = ConfigMain.transformTemplateToSchema(inputStream);
|
||||
SchemaSaver.saveConfigSchema(configSchema, configuration.getOutputStream());
|
||||
} catch (IOException e) {
|
||||
throw new ConfigurationProviderException("Unable to download template from url " + urlConnection.getURL(), e);
|
||||
} catch (JAXBException e) {
|
||||
throw new ConfigurationProviderException("Unable to convert template to yaml", e);
|
||||
} finally {
|
||||
urlConnection.disconnect();
|
||||
}
|
||||
}
|
||||
return configuration;
|
||||
}
|
||||
|
||||
private Pair<Stream<Pair<String, String>>, Closeable> getIdAndFilenameStream() throws ConfigurationProviderException, IOException {
|
||||
TemplatesIterator templatesIterator = new TemplatesIterator(httpConnector, jsonFactory);
|
||||
return new Pair<>(StreamSupport.stream(Spliterators.spliteratorUnknownSize(templatesIterator, Spliterator.ORDERED), false), templatesIterator);
|
||||
}
|
||||
|
||||
private Pair<Stream<Pair<String, Integer>>, Closeable> getIdAndVersionStream(String filenamePattern) throws ConfigurationProviderException, IOException {
|
||||
Pattern filename = Pattern.compile(filenamePattern);
|
||||
Pair<Stream<Pair<String, String>>, Closeable> streamCloseablePair = getIdAndFilenameStream();
|
||||
return new Pair<>(streamCloseablePair.getFirst().map(p -> {
|
||||
Matcher matcher = filename.matcher(p.getSecond());
|
||||
if (!matcher.matches()) {
|
||||
return null;
|
||||
}
|
||||
return new Pair<>(p.getFirst(), Integer.parseInt(matcher.group(1)));
|
||||
}).filter(Objects::nonNull), streamCloseablePair.getSecond());
|
||||
}
|
||||
|
||||
private Pair<String, Integer> getMaxIdAndVersion(String filenamePattern) throws ConfigurationProviderException {
|
||||
try {
|
||||
Pair<Stream<Pair<String, Integer>>, Closeable> streamCloseablePair = getIdAndVersionStream(filenamePattern);
|
||||
try {
|
||||
return streamCloseablePair.getFirst().sorted(Comparator.comparing(p -> ((Pair<String, Integer>) p).getSecond()).reversed()).findFirst()
|
||||
.orElseThrow(() -> new ConfigurationProviderException("Didn't find any templates that matched " + filenamePattern));
|
||||
} finally {
|
||||
streamCloseablePair.getSecond().close();
|
||||
}
|
||||
} catch (IOException|TemplatesIteratorException e) {
|
||||
throw new ConfigurationProviderException("Unable to retrieve template list", e);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,116 +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.c2.provider.nifi.rest;
|
||||
|
||||
import com.fasterxml.jackson.core.JsonFactory;
|
||||
import com.fasterxml.jackson.core.JsonParser;
|
||||
import com.fasterxml.jackson.core.JsonToken;
|
||||
import org.apache.nifi.minifi.c2.api.ConfigurationProviderException;
|
||||
import org.apache.nifi.minifi.c2.api.util.Pair;
|
||||
import org.apache.nifi.minifi.c2.provider.util.HttpConnector;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.net.HttpURLConnection;
|
||||
import java.util.Iterator;
|
||||
import java.util.NoSuchElementException;
|
||||
|
||||
public class TemplatesIterator implements Iterator<Pair<String, String>>, Closeable {
|
||||
public static final String FLOW_TEMPLATES = "/flow/templates";
|
||||
|
||||
private final HttpURLConnection urlConnection;
|
||||
private final InputStream inputStream;
|
||||
private final JsonParser parser;
|
||||
private Pair<String, String> next;
|
||||
|
||||
public TemplatesIterator(HttpConnector httpConnector, JsonFactory jsonFactory) throws ConfigurationProviderException, IOException {
|
||||
urlConnection = httpConnector.get(FLOW_TEMPLATES);
|
||||
inputStream = urlConnection.getInputStream();
|
||||
parser = jsonFactory.createParser(inputStream);
|
||||
while (parser.nextToken() != JsonToken.END_OBJECT) {
|
||||
if ("templates".equals(parser.getCurrentName())) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
next = getNext();
|
||||
}
|
||||
|
||||
private Pair<String, String> getNext() throws IOException {
|
||||
while (parser.nextToken() != JsonToken.END_ARRAY) {
|
||||
if ("template".equals(parser.getCurrentName())) {
|
||||
String id = null;
|
||||
String name = null;
|
||||
while (parser.nextToken() != JsonToken.END_OBJECT) {
|
||||
String currentName = parser.getCurrentName();
|
||||
if ("id".equals(currentName)) {
|
||||
parser.nextToken();
|
||||
id = parser.getText();
|
||||
} else if ("name".equals(currentName)) {
|
||||
parser.nextToken();
|
||||
name = parser.getText();
|
||||
}
|
||||
}
|
||||
return new Pair<>(id, name);
|
||||
}
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasNext() {
|
||||
return next != null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Pair<String, String> next() {
|
||||
if (next == null) {
|
||||
throw new NoSuchElementException();
|
||||
}
|
||||
try {
|
||||
return next;
|
||||
} finally {
|
||||
try {
|
||||
next = getNext();
|
||||
} catch (IOException e) {
|
||||
throw new TemplatesIteratorException(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
if (parser != null) {
|
||||
try {
|
||||
parser.close();
|
||||
} catch (IOException e) {
|
||||
//Ignore
|
||||
}
|
||||
}
|
||||
if (inputStream != null) {
|
||||
try {
|
||||
inputStream.close();
|
||||
} catch (IOException e) {
|
||||
//Ignore
|
||||
}
|
||||
}
|
||||
if (urlConnection != null) {
|
||||
urlConnection.disconnect();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,66 +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.c2.provider.nifi.rest;
|
||||
|
||||
import org.apache.nifi.minifi.c2.api.cache.ConfigurationCache;
|
||||
import org.apache.nifi.minifi.c2.provider.util.HttpConnector;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.mockito.Mockito.mock;
|
||||
|
||||
public class NiFiRestConfigurationProviderTest {
|
||||
private ConfigurationCache configConfigurationCache;
|
||||
private HttpConnector httpConnector;
|
||||
private NiFiRestConfigurationProvider niFiRestConfigurationProvider;
|
||||
private Path cachePath;
|
||||
|
||||
@BeforeEach
|
||||
public void setup() throws IOException {
|
||||
configConfigurationCache = mock(ConfigurationCache.class);
|
||||
httpConnector = mock(HttpConnector.class);
|
||||
niFiRestConfigurationProvider = new NiFiRestConfigurationProvider(configConfigurationCache, httpConnector, "${class}.v${version}");
|
||||
cachePath = Files.createTempDirectory(NiFiRestConfigurationProviderTest.class.getCanonicalName());
|
||||
}
|
||||
|
||||
@AfterEach
|
||||
public void teardown() throws IOException {
|
||||
Files.walk(cachePath)
|
||||
.sorted(Comparator.reverseOrder())
|
||||
.forEach(p -> {
|
||||
try {
|
||||
Files.deleteIfExists(p);
|
||||
} catch (IOException e) {
|
||||
p.toFile().deleteOnExit();
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testContentType() {
|
||||
assertEquals(Collections.singletonList(NiFiRestConfigurationProvider.CONTENT_TYPE), niFiRestConfigurationProvider.getContentTypes());
|
||||
}
|
||||
}
|
|
@ -1,110 +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.c2.provider.nifi.rest;
|
||||
|
||||
import com.fasterxml.jackson.core.JsonFactory;
|
||||
import org.apache.nifi.minifi.c2.api.ConfigurationProviderException;
|
||||
import org.apache.nifi.minifi.c2.api.util.Pair;
|
||||
import org.apache.nifi.minifi.c2.provider.util.HttpConnector;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.HttpURLConnection;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.NoSuchElementException;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.verify;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
public class TemplatesIteratorTest {
|
||||
private JsonFactory jsonFactory;
|
||||
private HttpURLConnection httpURLConnection;
|
||||
private HttpConnector httpConnector;
|
||||
|
||||
@BeforeEach
|
||||
public void setup() throws ConfigurationProviderException {
|
||||
jsonFactory = new JsonFactory();
|
||||
httpURLConnection = mock(HttpURLConnection.class);
|
||||
httpConnector = mock(HttpConnector.class);
|
||||
when(httpConnector.get(TemplatesIterator.FLOW_TEMPLATES)).thenReturn(httpURLConnection);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testIteratorNoSuchElementException() throws ConfigurationProviderException, IOException {
|
||||
when(httpURLConnection.getInputStream()).thenReturn(TemplatesIteratorTest.class.getClassLoader().getResourceAsStream("noTemplates.json"));
|
||||
|
||||
try (TemplatesIterator templatesIterator = new TemplatesIterator(httpConnector, jsonFactory)) {
|
||||
assertFalse(templatesIterator.hasNext());
|
||||
assertThrows(NoSuchElementException.class, templatesIterator::next);
|
||||
} finally {
|
||||
verify(httpURLConnection).disconnect();
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testIteratorNoTemplates() throws ConfigurationProviderException, IOException {
|
||||
when(httpURLConnection.getInputStream()).thenReturn(TemplatesIteratorTest.class.getClassLoader().getResourceAsStream("noTemplates.json"));
|
||||
List<Pair<String, String>> idToNameList = new ArrayList<>();
|
||||
try (TemplatesIterator templatesIterator = new TemplatesIterator(httpConnector, jsonFactory)) {
|
||||
templatesIterator.forEachRemaining(idToNameList::add);
|
||||
}
|
||||
assertEquals(0, idToNameList.size());
|
||||
|
||||
verify(httpURLConnection).disconnect();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testIteratorSingleTemplate() throws ConfigurationProviderException, IOException {
|
||||
when(httpURLConnection.getInputStream()).thenReturn(TemplatesIteratorTest.class.getClassLoader().getResourceAsStream("oneTemplate.json"));
|
||||
List<Pair<String, String>> idToNameList = new ArrayList<>();
|
||||
try (TemplatesIterator templatesIterator = new TemplatesIterator(httpConnector, jsonFactory)) {
|
||||
templatesIterator.forEachRemaining(idToNameList::add);
|
||||
}
|
||||
assertEquals(1, idToNameList.size());
|
||||
Pair<String, String> idNamePair = idToNameList.get(0);
|
||||
assertEquals("d05845ae-ceda-4c50-b7c2-037e42ddf1d3", idNamePair.getFirst());
|
||||
assertEquals("raspi3.v1", idNamePair.getSecond());
|
||||
|
||||
verify(httpURLConnection).disconnect();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testIteratorTwoTemplates() throws ConfigurationProviderException, IOException {
|
||||
when(httpURLConnection.getInputStream()).thenReturn(TemplatesIteratorTest.class.getClassLoader().getResourceAsStream("twoTemplates.json"));
|
||||
List<Pair<String, String>> idToNameList = new ArrayList<>();
|
||||
try (TemplatesIterator templatesIterator = new TemplatesIterator(httpConnector, jsonFactory)) {
|
||||
templatesIterator.forEachRemaining(idToNameList::add);
|
||||
}
|
||||
assertEquals(2, idToNameList.size());
|
||||
Pair<String, String> idNamePair = idToNameList.get(0);
|
||||
assertEquals("d05845ae-ceda-4c50-b7c2-037e42ddf1d3", idNamePair.getFirst());
|
||||
assertEquals("raspi3.v1", idNamePair.getSecond());
|
||||
|
||||
idNamePair = idToNameList.get(1);
|
||||
assertEquals("9384b48d-85b4-478a-bf3e-64d113f8fbc5", idNamePair.getFirst());
|
||||
assertEquals("raspi3.v2", idNamePair.getSecond());
|
||||
|
||||
verify(httpURLConnection).disconnect();
|
||||
}
|
||||
}
|
|
@ -1 +0,0 @@
|
|||
{"templates":[],"generated":"14:55:13 EST"}
|
|
@ -1,21 +0,0 @@
|
|||
{
|
||||
"generated": "10:26:04 EST",
|
||||
"templates": [
|
||||
{
|
||||
"id": "d05845ae-ceda-4c50-b7c2-037e42ddf1d3",
|
||||
"permissions": {
|
||||
"canRead": true,
|
||||
"canWrite": true
|
||||
},
|
||||
"template": {
|
||||
"description": "",
|
||||
"encoding-version": "1.0",
|
||||
"groupId": "8f8eda5e-015a-1000-a9c1-b7e4fe10ae83",
|
||||
"id": "d05845ae-ceda-4c50-b7c2-037e42ddf1d3",
|
||||
"name": "raspi3.v1",
|
||||
"timestamp": "03/02/2017 10:26:01 EST",
|
||||
"uri": "http://localhost:8081/nifi-api/templates/d05845ae-ceda-4c50-b7c2-037e42ddf1d3"
|
||||
}
|
||||
}
|
||||
]
|
||||
}
|
|
@ -1,37 +0,0 @@
|
|||
{
|
||||
"generated": "14:53:15 EST",
|
||||
"templates": [
|
||||
{
|
||||
"id": "d05845ae-ceda-4c50-b7c2-037e42ddf1d3",
|
||||
"permissions": {
|
||||
"canRead": true,
|
||||
"canWrite": true
|
||||
},
|
||||
"template": {
|
||||
"description": "",
|
||||
"encoding-version": "1.0",
|
||||
"groupId": "8f8eda5e-015a-1000-a9c1-b7e4fe10ae83",
|
||||
"id": "d05845ae-ceda-4c50-b7c2-037e42ddf1d3",
|
||||
"name": "raspi3.v1",
|
||||
"timestamp": "03/02/2017 10:26:01 EST",
|
||||
"uri": "http://localhost:8081/nifi-api/templates/d05845ae-ceda-4c50-b7c2-037e42ddf1d3"
|
||||
}
|
||||
},
|
||||
{
|
||||
"id": "9384b48d-85b4-478a-bf3e-64d113f8fbc5",
|
||||
"permissions": {
|
||||
"canRead": true,
|
||||
"canWrite": true
|
||||
},
|
||||
"template": {
|
||||
"description": "",
|
||||
"encoding-version": "1.0",
|
||||
"groupId": "8f8eda5e-015a-1000-a9c1-b7e4fe10ae83",
|
||||
"id": "9384b48d-85b4-478a-bf3e-64d113f8fbc5",
|
||||
"name": "raspi3.v2",
|
||||
"timestamp": "03/02/2017 13:08:14 EST",
|
||||
"uri": "http://localhost:8081/nifi-api/templates/9384b48d-85b4-478a-bf3e-64d113f8fbc5"
|
||||
}
|
||||
}
|
||||
]
|
||||
}
|
|
@ -29,6 +29,5 @@ limitations under the License.
|
|||
<module>minifi-c2-provider-util</module>
|
||||
<module>minifi-c2-provider-cache</module>
|
||||
<module>minifi-c2-provider-delegating</module>
|
||||
<module>minifi-c2-provider-nifi-rest</module>
|
||||
</modules>
|
||||
</project>
|
||||
|
|
|
@ -16,7 +16,8 @@
|
|||
~ 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">
|
||||
<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>
|
||||
<groupId>org.apache.nifi.minifi</groupId>
|
||||
|
|
|
@ -19,5 +19,7 @@ package org.apache.nifi.minifi.commons.api;
|
|||
|
||||
public interface MiNiFiConstants {
|
||||
String BOOTSTRAP_UPDATED_FILE_NAME = "bootstrap-updated.conf";
|
||||
String CONFIG_UPDATED_FILE_NAME = "config-updated.yml";
|
||||
|
||||
String BACKUP_EXTENSION = ".backup";
|
||||
String RAW_EXTENSION = ".raw";
|
||||
}
|
||||
|
|
|
@ -15,17 +15,19 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.nifi.minifi;
|
||||
package org.apache.nifi.minifi.commons.api;
|
||||
|
||||
import static org.apache.nifi.minifi.MiNiFiProperties.ValidatorNames.BOOLEAN_VALIDATOR;
|
||||
import static org.apache.nifi.minifi.MiNiFiProperties.ValidatorNames.LONG_VALIDATOR;
|
||||
import static org.apache.nifi.minifi.MiNiFiProperties.ValidatorNames.NON_NEGATIVE_INTEGER_VALIDATOR;
|
||||
import static org.apache.nifi.minifi.MiNiFiProperties.ValidatorNames.PORT_VALIDATOR;
|
||||
import static org.apache.nifi.minifi.MiNiFiProperties.ValidatorNames.TIME_PERIOD_VALIDATOR;
|
||||
import static org.apache.nifi.minifi.MiNiFiProperties.ValidatorNames.VALID;
|
||||
import static java.util.stream.Collectors.toSet;
|
||||
import static org.apache.nifi.minifi.commons.api.MiNiFiProperties.ValidatorNames.BOOLEAN_VALIDATOR;
|
||||
import static org.apache.nifi.minifi.commons.api.MiNiFiProperties.ValidatorNames.LONG_VALIDATOR;
|
||||
import static org.apache.nifi.minifi.commons.api.MiNiFiProperties.ValidatorNames.NON_NEGATIVE_INTEGER_VALIDATOR;
|
||||
import static org.apache.nifi.minifi.commons.api.MiNiFiProperties.ValidatorNames.PORT_VALIDATOR;
|
||||
import static org.apache.nifi.minifi.commons.api.MiNiFiProperties.ValidatorNames.TIME_PERIOD_VALIDATOR;
|
||||
import static org.apache.nifi.minifi.commons.api.MiNiFiProperties.ValidatorNames.VALID;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.Set;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
|
@ -35,39 +37,6 @@ public enum MiNiFiProperties {
|
|||
LIB_DIR("lib.dir", "./lib", false, false, VALID),
|
||||
CONF_DIR("conf.dir", "./conf", false, false, VALID),
|
||||
GRACEFUL_SHUTDOWN_SECOND("graceful.shutdown.seconds", "20", false, true, NON_NEGATIVE_INTEGER_VALIDATOR),
|
||||
NIFI_MINIFI_CONFIG("nifi.minifi.config", "./conf/config.yml", false, true, VALID),
|
||||
NIFI_MINIFI_SECURITY_KEYSTORE("nifi.minifi.security.keystore", null, false, false, VALID),
|
||||
NIFI_MINIFI_SECURITY_KEYSTORE_TYPE("nifi.minifi.security.keystoreType", null, false, false, VALID),
|
||||
NIFI_MINIFI_SECURITY_KEYSTORE_PASSWD("nifi.minifi.security.keystorePasswd", null, true, false, VALID),
|
||||
NIFI_MINIFI_SECURITY_KEY_PASSWD("nifi.minifi.security.keyPasswd", null, true, false, VALID),
|
||||
NIFI_MINIFI_SECURITY_TRUSTSTORE("nifi.minifi.security.truststore", null, false, false, VALID),
|
||||
NIFI_MINIFI_SECURITY_TRUSTSTORE_TYPE("nifi.minifi.security.truststoreType", null, false, false, VALID),
|
||||
NIFI_MINIFI_SECURITY_TRUSTSTORE_PASSWD("nifi.minifi.security.truststorePasswd", null, true, false, VALID),
|
||||
NIFI_MINIFI_SECURITY_SSL_PROTOCOL("nifi.minifi.security.ssl.protocol", null, false, false, VALID),
|
||||
NIFI_MINIFI_SENSITIVE_PROPS_KEY("nifi.minifi.sensitive.props.key", null, true, false, VALID),
|
||||
NIFI_MINIFI_SENSITIVE_PROPS_ALGORITHM("nifi.minifi.sensitive.props.algorithm", null, true, false, VALID),
|
||||
NIFI_MINIFI_PROVENANCE_REPORTING_COMMENT("nifi.minifi.provenance.reporting.comment", null, false, true, VALID),
|
||||
NIFI_MINIFI_PROVENANCE_REPORTING_SCHEDULING_STRATEGY("nifi.minifi.provenance.reporting.scheduling.strategy", null, false, true, VALID),
|
||||
NIFI_MINIFI_PROVENANCE_REPORTING_SCHEDULING_PERIOD("nifi.minifi.provenance.reporting.scheduling.period", null, false, true, TIME_PERIOD_VALIDATOR),
|
||||
NIFI_MINIFI_PROVENANCE_REPORTING_DESTINATION_URL("nifi.minifi.provenance.reporting.destination.url", null, false, true, VALID),
|
||||
NIFI_MINIFI_PROVENANCE_REPORTING_INPUT_PORT_NAME("nifi.minifi.provenance.reporting.input.port.name", null, false, true, VALID),
|
||||
NIFI_MINIFI_PROVENANCE_REPORTING_INSTANCE_URL("nifi.minifi.provenance.reporting.instance.url", null, false, true, VALID),
|
||||
NIFI_MINIFI_PROVENANCE_REPORTING_BATCH_SIZE("nifi.minifi.provenance.reporting.batch.size", null, false, true, NON_NEGATIVE_INTEGER_VALIDATOR),
|
||||
NIFI_MINIFI_PROVENANCE_REPORTING_COMMUNICATIONS_TIMEOUT("nifi.minifi.provenance.reporting.communications.timeout", null, false, true, TIME_PERIOD_VALIDATOR),
|
||||
NIFI_MINIFI_FLOW_USE_PARENT_SSL("nifi.minifi.flow.use.parent.ssl", null, false, true, VALID),
|
||||
NIFI_MINIFI_NOTIFIER_INGESTORS("nifi.minifi.notifier.ingestors", null, false, true, VALID),
|
||||
NIFI_MINIFI_NOTIFIER_INGESTORS_FILE_CONFIG_PATH("nifi.minifi.notifier.ingestors.file.config.path", null, false, true, VALID),
|
||||
NIFI_MINIFI_NOTIFIER_INGESTORS_FILE_POLLING_PERIOD_SECONDS("nifi.minifi.notifier.ingestors.file.polling.period.seconds", null, false, true, NON_NEGATIVE_INTEGER_VALIDATOR),
|
||||
NIFI_MINIFI_NOTIFIER_INGESTORS_RECEIVE_HTTP_PORT("nifi.minifi.notifier.ingestors.receive.http.port", null, false, true, PORT_VALIDATOR),
|
||||
NIFI_MINIFI_NOTIFIER_INGESTORS_PULL_HTTP_HOSTNAME("nifi.minifi.notifier.ingestors.pull.http.hostname", null, false, true, VALID),
|
||||
NIFI_MINIFI_NOTIFIER_INGESTORS_PULL_HTTP_PORT("nifi.minifi.notifier.ingestors.pull.http.port", null, false, true, PORT_VALIDATOR),
|
||||
NIFI_MINIFI_NOTIFIER_INGESTORS_PULL_HTTP_PATH("nifi.minifi.notifier.ingestors.pull.http.path", null, false, true, VALID),
|
||||
NIFI_MINIFI_NOTIFIER_INGESTORS_PULL_HTTP_QUERY("nifi.minifi.notifier.ingestors.pull.http.query", null, false, true, VALID),
|
||||
NIFI_MINIFI_NOTIFIER_INGESTORS_PULL_HTTP_PERIOD_MS("nifi.minifi.notifier.ingestors.pull.http.period.ms", null, false, true, NON_NEGATIVE_INTEGER_VALIDATOR),
|
||||
NIFI_MINIFI_STATUS_REPORTER_COMPONENTS("nifi.minifi.status.reporter.components", null, false, true, VALID),
|
||||
NIFI_MINIFI_STATUS_REPORTER_LOG_QUERY("nifi.minifi.status.reporter.log.query", null, false, true, VALID),
|
||||
NIFI_MINIFI_STATUS_REPORTER_LOG_LEVEL("nifi.minifi.status.reporter.log.level", null, false, true, VALID),
|
||||
NIFI_MINIFI_STATUS_REPORTER_LOG_PERIOD("nifi.minifi.status.reporter.log.period", null, false, true, VALID),
|
||||
JAVA_ARG_1("java.arg.1", null, false, true, VALID),
|
||||
JAVA_ARG_2("java.arg.2", null, false, true, VALID),
|
||||
JAVA_ARG_3("java.arg.3", null, false, true, VALID),
|
||||
|
@ -82,7 +51,24 @@ public enum MiNiFiProperties {
|
|||
JAVA_ARG_12("java.arg.12", null, false, true, VALID),
|
||||
JAVA_ARG_13("java.arg.13", null, false, true, VALID),
|
||||
JAVA_ARG_14("java.arg.14", null, false, true, VALID),
|
||||
NIFI_MINIFI_FLOW_CONFIG("nifi.minifi.flow.config", "./conf/flow.json.gz", false, false, VALID),
|
||||
NIFI_MINIFI_FLOW_MAX_CONCURRENT_THREADS("nifi.minifi.flow.max.concurrent.threads", null, false, true, VALID),
|
||||
NIFI_MINIFI_SECURITY_KEYSTORE("nifi.minifi.security.keystore", null, false, false, VALID),
|
||||
NIFI_MINIFI_SECURITY_KEYSTORE_TYPE("nifi.minifi.security.keystoreType", null, false, false, VALID),
|
||||
NIFI_MINIFI_SECURITY_KEYSTORE_PASSWD("nifi.minifi.security.keystorePasswd", null, true, false, VALID),
|
||||
NIFI_MINIFI_SECURITY_KEY_PASSWD("nifi.minifi.security.keyPasswd", null, true, false, VALID),
|
||||
NIFI_MINIFI_SECURITY_TRUSTSTORE("nifi.minifi.security.truststore", null, false, false, VALID),
|
||||
NIFI_MINIFI_SECURITY_TRUSTSTORE_TYPE("nifi.minifi.security.truststoreType", null, false, false, VALID),
|
||||
NIFI_MINIFI_SECURITY_TRUSTSTORE_PASSWD("nifi.minifi.security.truststorePasswd", null, true, false, VALID),
|
||||
NIFI_MINIFI_SECURITY_SSL_PROTOCOL("nifi.minifi.security.ssl.protocol", null, false, false, VALID),
|
||||
NIFI_MINIFI_FLOW_USE_PARENT_SSL("nifi.minifi.flow.use.parent.ssl", null, false, true, VALID),
|
||||
NIFI_MINIFI_SENSITIVE_PROPS_KEY("nifi.minifi.sensitive.props.key", null, true, false, VALID),
|
||||
NIFI_MINIFI_SENSITIVE_PROPS_ALGORITHM("nifi.minifi.sensitive.props.algorithm", null, true, false, VALID),
|
||||
C2_ENABLE("c2.enable", "false", false, true, BOOLEAN_VALIDATOR),
|
||||
C2_AGENT_HEARTBEAT_PERIOD("c2.agent.heartbeat.period", "1000", false, true, LONG_VALIDATOR),
|
||||
C2_AGENT_CLASS("c2.agent.class", "", false, true, VALID),
|
||||
C2_AGENT_IDENTIFIER("c2.agent.identifier", null, false, true, VALID),
|
||||
C2_FULL_HEARTBEAT("c2.full.heartbeat", "true", false, true, BOOLEAN_VALIDATOR),
|
||||
C2_REST_URL("c2.rest.url", "", false, true, VALID),
|
||||
C2_REST_URL_ACK("c2.rest.url.ack", "", false, true, VALID),
|
||||
C2_REST_PATH_BASE("c2.rest.path.base", "", false, true, VALID),
|
||||
|
@ -93,13 +79,11 @@ public enum MiNiFiProperties {
|
|||
C2_REST_CALL_TIMEOUT("c2.rest.callTimeout", "10 sec", false, true, TIME_PERIOD_VALIDATOR),
|
||||
C2_MAX_IDLE_CONNECTIONS("c2.rest.maxIdleConnections", "5", false, true, NON_NEGATIVE_INTEGER_VALIDATOR),
|
||||
C2_KEEP_ALIVE_DURATION("c2.rest.keepAliveDuration", "5 min", false, true, TIME_PERIOD_VALIDATOR),
|
||||
C2_AGENT_HEARTBEAT_PERIOD("c2.agent.heartbeat.period", "1000", false, true, LONG_VALIDATOR),
|
||||
C2_AGENT_CLASS("c2.agent.class", "", false, true, VALID),
|
||||
C2_REST_HTTP_HEADERS("c2.rest.http.headers", "Accept:application/json", false, true, VALID),
|
||||
C2_CONFIG_DIRECTORY("c2.config.directory", "./conf", false, true, VALID),
|
||||
C2_RUNTIME_MANIFEST_IDENTIFIER("c2.runtime.manifest.identifier", "", false, true, VALID),
|
||||
C2_RUNTIME_TYPE("c2.runtime.type", "", false, true, VALID),
|
||||
C2_AGENT_IDENTIFIER("c2.agent.identifier", null, false, true, VALID),
|
||||
C2_FULL_HEARTBEAT("c2.full.heartbeat", "true", false, true, BOOLEAN_VALIDATOR),
|
||||
C2_ASSET_DIRECTORY("c2.asset.directory", "./asset", false, true, VALID),
|
||||
C2_SECURITY_TRUSTSTORE_LOCATION("c2.security.truststore.location", "", false, false, VALID),
|
||||
C2_SECURITY_TRUSTSTORE_PASSWORD("c2.security.truststore.password", "", true, false, VALID),
|
||||
C2_SECURITY_TRUSTSTORE_TYPE("c2.security.truststore.type", "JKS", false, false, VALID),
|
||||
|
@ -107,11 +91,35 @@ public enum MiNiFiProperties {
|
|||
C2_SECURITY_KEYSTORE_PASSWORD("c2.security.keystore.password", "", true, false, VALID),
|
||||
C2_SECURITY_KEYSTORE_TYPE("c2.security.keystore.type", "JKS", false, false, VALID),
|
||||
C2_REQUEST_COMPRESSION("c2.request.compression", "none", false, true, VALID),
|
||||
C2_ASSET_DIRECTORY("c2.asset.directory", "./asset", false, true, VALID);
|
||||
NIFI_MINIFI_NOTIFIER_INGESTORS("nifi.minifi.notifier.ingestors", null, false, true, VALID),
|
||||
NIFI_MINIFI_NOTIFIER_INGESTORS_FILE_CONFIG_PATH("nifi.minifi.notifier.ingestors.file.config.path", null, false, true, VALID),
|
||||
NIFI_MINIFI_NOTIFIER_INGESTORS_FILE_POLLING_PERIOD_SECONDS("nifi.minifi.notifier.ingestors.file.polling.period.seconds", null, false, true, NON_NEGATIVE_INTEGER_VALIDATOR),
|
||||
NIFI_MINIFI_NOTIFIER_INGESTORS_RECEIVE_HTTP_PORT("nifi.minifi.notifier.ingestors.receive.http.port", null, false, true, PORT_VALIDATOR),
|
||||
NIFI_MINIFI_NOTIFIER_INGESTORS_PULL_HTTP_HOSTNAME("nifi.minifi.notifier.ingestors.pull.http.hostname", null, false, true, VALID),
|
||||
NIFI_MINIFI_NOTIFIER_INGESTORS_PULL_HTTP_PORT("nifi.minifi.notifier.ingestors.pull.http.port", null, false, true, PORT_VALIDATOR),
|
||||
NIFI_MINIFI_NOTIFIER_INGESTORS_PULL_HTTP_PATH("nifi.minifi.notifier.ingestors.pull.http.path", null, false, true, VALID),
|
||||
NIFI_MINIFI_NOTIFIER_INGESTORS_PULL_HTTP_QUERY("nifi.minifi.notifier.ingestors.pull.http.query", null, false, true, VALID),
|
||||
NIFI_MINIFI_NOTIFIER_INGESTORS_PULL_HTTP_PERIOD_MS("nifi.minifi.notifier.ingestors.pull.http.period.ms", null, false, true, NON_NEGATIVE_INTEGER_VALIDATOR),
|
||||
NIFI_MINIFI_NOTIFIER_INGESTORS_PULL_HTTP_HEADERS("nifi.minifi.notifier.ingestors.pull.http.headers", null, false, true, NON_NEGATIVE_INTEGER_VALIDATOR),
|
||||
NIFI_MINIFI_STATUS_REPORTER_COMPONENTS("nifi.minifi.status.reporter.components", null, false, true, VALID),
|
||||
NIFI_MINIFI_STATUS_REPORTER_LOG_QUERY("nifi.minifi.status.reporter.log.query", null, false, true, VALID),
|
||||
NIFI_MINIFI_STATUS_REPORTER_LOG_LEVEL("nifi.minifi.status.reporter.log.level", null, false, true, VALID),
|
||||
NIFI_MINIFI_STATUS_REPORTER_LOG_PERIOD("nifi.minifi.status.reporter.log.period", null, false, true, VALID),
|
||||
NIFI_MINIFI_PROVENANCE_REPORTING_COMMENT("nifi.minifi.provenance.reporting.comment", null, false, true, VALID),
|
||||
NIFI_MINIFI_PROVENANCE_REPORTING_SCHEDULING_STRATEGY("nifi.minifi.provenance.reporting.scheduling.strategy", null, false, true, VALID),
|
||||
NIFI_MINIFI_PROVENANCE_REPORTING_SCHEDULING_PERIOD("nifi.minifi.provenance.reporting.scheduling.period", null, false, true, TIME_PERIOD_VALIDATOR),
|
||||
NIFI_MINIFI_PROVENANCE_REPORTING_DESTINATION_URL("nifi.minifi.provenance.reporting.destination.url", null, false, true, VALID),
|
||||
NIFI_MINIFI_PROVENANCE_REPORTING_INPUT_PORT_NAME("nifi.minifi.provenance.reporting.input.port.name", null, false, true, VALID),
|
||||
NIFI_MINIFI_PROVENANCE_REPORTING_INSTANCE_URL("nifi.minifi.provenance.reporting.instance.url", null, false, true, VALID),
|
||||
NIFI_MINIFI_PROVENANCE_REPORTING_COMPRESS_EVENTS("nifi.minifi.provenance.reporting.compress.events", null, false, true, VALID),
|
||||
NIFI_MINIFI_PROVENANCE_REPORTING_BATCH_SIZE("nifi.minifi.provenance.reporting.batch.size", null, false, true, NON_NEGATIVE_INTEGER_VALIDATOR),
|
||||
NIFI_MINIFI_PROVENANCE_REPORTING_COMMUNICATIONS_TIMEOUT("nifi.minifi.provenance.reporting.communications.timeout", null, false, true, TIME_PERIOD_VALIDATOR);
|
||||
|
||||
public static final LinkedHashMap<String, MiNiFiProperties> PROPERTIES_BY_KEY = Arrays.stream(MiNiFiProperties.values())
|
||||
.sorted()
|
||||
.collect(Collectors.toMap(MiNiFiProperties::getKey, Function.identity(), (x, y) -> y, LinkedHashMap::new));
|
||||
// These are not "real" bootstrap properties. They are generated runtime in bootstrap and populated into also generated minifi.properties
|
||||
public static final String MINIFI_BOOTSTRAP_FILE_PATH = "nifi.minifi.bootstrap.file";
|
||||
public static final String MINIFI_LOG_DIRECTORY = "nifi.minifi.log.directory";
|
||||
public static final String MINIFI_APP_LOG_FILE = "nifi.minifi.app.log.file";
|
||||
public static final String MINIFI_BOOTSTRAP_LOG_FILE = "nifi.minifi.bootstrap.log.file";
|
||||
|
||||
private final String key;
|
||||
private final String defaultValue;
|
||||
|
@ -127,6 +135,43 @@ public enum MiNiFiProperties {
|
|||
this.validator = validator;
|
||||
}
|
||||
|
||||
public static LinkedHashMap<String, MiNiFiProperties> sortedPropertiesByKey() {
|
||||
return Arrays.stream(MiNiFiProperties.values())
|
||||
.sorted()
|
||||
.collect(Collectors.toMap(MiNiFiProperties::getKey, Function.identity(), (x, y) -> y, LinkedHashMap::new));
|
||||
}
|
||||
|
||||
public static Set<String> securityPropertyKeys() {
|
||||
return Set.of(
|
||||
NIFI_MINIFI_SECURITY_KEYSTORE,
|
||||
NIFI_MINIFI_SECURITY_KEYSTORE_TYPE,
|
||||
NIFI_MINIFI_SECURITY_KEYSTORE_PASSWD,
|
||||
NIFI_MINIFI_SECURITY_KEY_PASSWD,
|
||||
NIFI_MINIFI_SECURITY_TRUSTSTORE,
|
||||
NIFI_MINIFI_SECURITY_TRUSTSTORE_TYPE,
|
||||
NIFI_MINIFI_SECURITY_TRUSTSTORE_PASSWD,
|
||||
NIFI_MINIFI_SECURITY_SSL_PROTOCOL)
|
||||
.stream()
|
||||
.map(MiNiFiProperties::getKey)
|
||||
.collect(toSet());
|
||||
}
|
||||
|
||||
public static Set<String> provenanceReportingPropertyKeys() {
|
||||
return Set.of(
|
||||
NIFI_MINIFI_PROVENANCE_REPORTING_COMMENT,
|
||||
NIFI_MINIFI_PROVENANCE_REPORTING_SCHEDULING_STRATEGY,
|
||||
NIFI_MINIFI_PROVENANCE_REPORTING_SCHEDULING_PERIOD,
|
||||
NIFI_MINIFI_PROVENANCE_REPORTING_DESTINATION_URL,
|
||||
NIFI_MINIFI_PROVENANCE_REPORTING_INPUT_PORT_NAME,
|
||||
NIFI_MINIFI_PROVENANCE_REPORTING_INSTANCE_URL,
|
||||
NIFI_MINIFI_PROVENANCE_REPORTING_COMPRESS_EVENTS,
|
||||
NIFI_MINIFI_PROVENANCE_REPORTING_BATCH_SIZE,
|
||||
NIFI_MINIFI_PROVENANCE_REPORTING_COMMUNICATIONS_TIMEOUT)
|
||||
.stream()
|
||||
.map(MiNiFiProperties::getKey)
|
||||
.collect(toSet());
|
||||
}
|
||||
|
||||
public String getKey() {
|
||||
return key;
|
||||
}
|
|
@ -0,0 +1,38 @@
|
|||
target/
|
||||
!.mvn/wrapper/maven-wrapper.jar
|
||||
!**/src/main/**/target/
|
||||
!**/src/test/**/target/
|
||||
|
||||
### IntelliJ IDEA ###
|
||||
.idea/modules.xml
|
||||
.idea/jarRepositories.xml
|
||||
.idea/compiler.xml
|
||||
.idea/libraries/
|
||||
*.iws
|
||||
*.iml
|
||||
*.ipr
|
||||
|
||||
### Eclipse ###
|
||||
.apt_generated
|
||||
.classpath
|
||||
.factorypath
|
||||
.project
|
||||
.settings
|
||||
.springBeans
|
||||
.sts4-cache
|
||||
|
||||
### NetBeans ###
|
||||
/nbproject/private/
|
||||
/nbbuild/
|
||||
/dist/
|
||||
/nbdist/
|
||||
/.nb-gradle/
|
||||
build/
|
||||
!**/src/main/**/build/
|
||||
!**/src/test/**/build/
|
||||
|
||||
### VS Code ###
|
||||
.vscode/
|
||||
|
||||
### Mac OS ###
|
||||
.DS_Store
|
|
@ -15,45 +15,39 @@ 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 https://maven.apache.org/xsd/maven-4.0.0.xsd">
|
||||
<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>minifi-c2-provider</artifactId>
|
||||
<groupId>org.apache.nifi.minifi</groupId>
|
||||
<artifactId>minifi-commons</artifactId>
|
||||
<version>2.0.0-SNAPSHOT</version>
|
||||
</parent>
|
||||
<artifactId>minifi-c2-provider-nifi-rest</artifactId>
|
||||
<packaging>jar</packaging>
|
||||
|
||||
<artifactId>minifi-commons-framework</artifactId>
|
||||
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi.minifi</groupId>
|
||||
<artifactId>minifi-c2-api</artifactId>
|
||||
<artifactId>minifi-commons-api</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-framework-core-api</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-api</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-property-utils</artifactId>
|
||||
<version>${project.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi.minifi</groupId>
|
||||
<artifactId>minifi-c2-provider-util</artifactId>
|
||||
<version>${project.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.fasterxml.jackson.core</groupId>
|
||||
<artifactId>jackson-core</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi.minifi</groupId>
|
||||
<artifactId>minifi-toolkit-configuration</artifactId>
|
||||
<version>${project.version}</version>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>org.slf4j</groupId>
|
||||
<artifactId>slf4j-log4j12</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>org.glassfish.jersey.media</groupId>
|
||||
<artifactId>jersey-media-json-jackson</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
<groupId>com.fasterxml.jackson.module</groupId>
|
||||
<artifactId>jackson-module-jaxb-annotations</artifactId>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
|
||||
|
@ -64,12 +58,11 @@ limitations under the License.
|
|||
<artifactId>apache-rat-plugin</artifactId>
|
||||
<configuration>
|
||||
<excludes combine.children="append">
|
||||
<exclude>src/test/resources/noTemplates.json</exclude>
|
||||
<exclude>src/test/resources/oneTemplate.json</exclude>
|
||||
<exclude>src/test/resources/twoTemplates.json</exclude>
|
||||
<exclude>**/org.mockito.plugins.MockMaker</exclude>
|
||||
<exclude>**/default_flow.json</exclude>
|
||||
</excludes>
|
||||
</configuration>
|
||||
</plugin>
|
||||
</plugins>
|
||||
</build>
|
||||
</project>
|
||||
</project>
|
|
@ -0,0 +1,290 @@
|
|||
/*
|
||||
* Licensed to the Apache 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.commons.service;
|
||||
|
||||
import static java.lang.Boolean.FALSE;
|
||||
import static java.lang.Boolean.parseBoolean;
|
||||
import static java.nio.charset.StandardCharsets.UTF_8;
|
||||
import static java.util.Map.entry;
|
||||
import static java.util.Optional.empty;
|
||||
import static java.util.Optional.ofNullable;
|
||||
import static java.util.UUID.randomUUID;
|
||||
import static java.util.stream.Collectors.toMap;
|
||||
import static org.apache.commons.lang3.StringUtils.EMPTY;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonInclude;
|
||||
import com.fasterxml.jackson.core.JsonFactory;
|
||||
import com.fasterxml.jackson.core.JsonGenerator;
|
||||
import com.fasterxml.jackson.databind.DeserializationFeature;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.controller.flow.VersionedDataflow;
|
||||
import org.apache.nifi.controller.serialization.FlowSerializationException;
|
||||
import org.apache.nifi.flow.Bundle;
|
||||
import org.apache.nifi.flow.ComponentType;
|
||||
import org.apache.nifi.flow.ControllerServiceAPI;
|
||||
import org.apache.nifi.flow.ScheduledState;
|
||||
import org.apache.nifi.flow.VersionedComponent;
|
||||
import org.apache.nifi.flow.VersionedControllerService;
|
||||
import org.apache.nifi.flow.VersionedProcessGroup;
|
||||
import org.apache.nifi.flow.VersionedReportingTask;
|
||||
import org.apache.nifi.logging.LogLevel;
|
||||
import org.apache.nifi.minifi.commons.api.MiNiFiProperties;
|
||||
import org.apache.nifi.properties.ReadableProperties;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
public class FlowEnrichService {
|
||||
|
||||
static final String DEFAULT_SSL_CONTEXT_SERVICE_NAME = "SSL Context Service";
|
||||
|
||||
static final String COMMON_SSL_CONTEXT_SERVICE_NAME = "SSL-Context-Service";
|
||||
static final String COMMON_SSL_CONTEXT_SERVICE_ID = "generated-common-ssl-context";
|
||||
static final String SITE_TO_SITE_PROVENANCE_REPORTING_TASK_NAME = "Site-To-Site-Provenance-Reporting";
|
||||
static final String SITE_TO_SITE_PROVENANCE_REPORTING_TASK_ID = "generated-s2s-provenance-reporting-task";
|
||||
|
||||
private static final Logger LOG = LoggerFactory.getLogger(FlowEnrichService.class);
|
||||
|
||||
private static final String NIFI_BUNDLE_GROUP = "org.apache.nifi";
|
||||
private static final String STANDARD_RESTRICTED_SSL_CONTEXT_SERVICE = "org.apache.nifi.ssl.StandardRestrictedSSLContextService";
|
||||
private static final String RESTRICTED_SSL_CONTEXT_SERVICE_API = "org.apache.nifi.ssl.RestrictedSSLContextService";
|
||||
private static final String SSL_CONTEXT_SERVICE_API = "org.apache.nifi.ssl.SSLContextService";
|
||||
private static final String SSL_CONTEXT_SERVICE_NAR = "nifi-ssl-context-service-nar";
|
||||
private static final String STANDARD_SERVICES_API_NAR_ARTIFACT = "nifi-standard-services-api-nar";
|
||||
private static final String SITE_TO_SITE_PROVENANCE_REPORTING_TASK = "org.apache.nifi.reporting.SiteToSiteProvenanceReportingTask";
|
||||
private static final String SITE_TO_SITE_REPORTING_NAR_ARTIFACT = "nifi-site-to-site-reporting-nar";
|
||||
private static final String PROVENANCE_REPORTING_TASK_PROTOCOL = "HTTP";
|
||||
private static final String PROVENANCE_REPORTING_TASK_BEGINNING_OF_STREAM = "beginning-of-stream";
|
||||
|
||||
private final ReadableProperties minifiProperties;
|
||||
|
||||
public FlowEnrichService(ReadableProperties minifiProperties) {
|
||||
this.minifiProperties = minifiProperties;
|
||||
}
|
||||
|
||||
public byte[] enrichFlow(byte[] flowCandidate) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Enriching flow with content: \n{}", new String(flowCandidate, UTF_8));
|
||||
}
|
||||
|
||||
VersionedDataflow versionedDataflow = parseVersionedDataflow(flowCandidate);
|
||||
|
||||
Optional<Integer> maxConcurrentThreads = ofNullable(minifiProperties.getProperty(MiNiFiProperties.NIFI_MINIFI_FLOW_MAX_CONCURRENT_THREADS.getKey()))
|
||||
.map(Integer::parseInt);
|
||||
maxConcurrentThreads.ifPresent(versionedDataflow::setMaxTimerDrivenThreadCount);
|
||||
|
||||
VersionedProcessGroup rootGroup = versionedDataflow.getRootGroup();
|
||||
if (rootGroup.getIdentifier() == null) {
|
||||
rootGroup.setIdentifier(randomUUID().toString());
|
||||
}
|
||||
if (rootGroup.getInstanceIdentifier() == null) {
|
||||
rootGroup.setInstanceIdentifier(randomUUID().toString());
|
||||
}
|
||||
|
||||
Optional<VersionedControllerService> commonSslControllerService = createCommonSslControllerService();
|
||||
commonSslControllerService
|
||||
.ifPresent(sslControllerService -> {
|
||||
List<VersionedControllerService> currentControllerServices = ofNullable(versionedDataflow.getControllerServices()).orElseGet(ArrayList::new);
|
||||
currentControllerServices.add(sslControllerService);
|
||||
versionedDataflow.setControllerServices(currentControllerServices);
|
||||
});
|
||||
|
||||
commonSslControllerService
|
||||
.filter(__ -> parseBoolean(minifiProperties.getProperty(MiNiFiProperties.NIFI_MINIFI_FLOW_USE_PARENT_SSL.getKey())))
|
||||
.map(VersionedComponent::getInstanceIdentifier)
|
||||
.ifPresent(commonSslControllerServiceInstanceId -> overrideProcessorsSslControllerService(rootGroup, commonSslControllerServiceInstanceId));
|
||||
|
||||
createProvenanceReportingTask(commonSslControllerService.map(VersionedComponent::getInstanceIdentifier).orElse(EMPTY))
|
||||
.ifPresent(provenanceReportingTask -> {
|
||||
List<VersionedReportingTask> currentReportingTasks = ofNullable(versionedDataflow.getReportingTasks()).orElseGet(ArrayList::new);
|
||||
currentReportingTasks.add(provenanceReportingTask);
|
||||
versionedDataflow.setReportingTasks(currentReportingTasks);
|
||||
});
|
||||
|
||||
byte[] enrichedFlow = toByteArray(versionedDataflow);
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Enriched flow with content: \n{}", new String(enrichedFlow, UTF_8));
|
||||
}
|
||||
return enrichedFlow;
|
||||
}
|
||||
|
||||
private VersionedDataflow parseVersionedDataflow(byte[] flow) {
|
||||
try {
|
||||
ObjectMapper objectMapper = deserializationObjectMapper();
|
||||
return objectMapper.readValue(flow, VersionedDataflow.class);
|
||||
} catch (final Exception e) {
|
||||
throw new FlowSerializationException("Could not parse flow as a VersionedDataflow", e);
|
||||
}
|
||||
}
|
||||
|
||||
private ObjectMapper deserializationObjectMapper() {
|
||||
ObjectMapper objectMapper = new ObjectMapper();
|
||||
objectMapper.setAnnotationIntrospector(new JaxbAnnotationIntrospector(objectMapper.getTypeFactory()));
|
||||
objectMapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
|
||||
return objectMapper;
|
||||
}
|
||||
|
||||
private Optional<VersionedControllerService> createCommonSslControllerService() {
|
||||
if (!parentSslEnabled()) {
|
||||
LOG.debug("Parent SSL is disabled, skip creating parent SSL Controller Service");
|
||||
return empty();
|
||||
}
|
||||
|
||||
LOG.debug("Parent SSL is enabled, creating parent SSL Controller Service");
|
||||
VersionedControllerService sslControllerService = new VersionedControllerService();
|
||||
sslControllerService.setIdentifier(randomUUID().toString());
|
||||
sslControllerService.setInstanceIdentifier(COMMON_SSL_CONTEXT_SERVICE_ID);
|
||||
sslControllerService.setName(COMMON_SSL_CONTEXT_SERVICE_NAME);
|
||||
sslControllerService.setComments(EMPTY);
|
||||
sslControllerService.setType(STANDARD_RESTRICTED_SSL_CONTEXT_SERVICE);
|
||||
sslControllerService.setScheduledState(ScheduledState.ENABLED);
|
||||
sslControllerService.setBulletinLevel(LogLevel.WARN.name());
|
||||
sslControllerService.setComponentType(ComponentType.CONTROLLER_SERVICE);
|
||||
sslControllerService.setBundle(createBundle(SSL_CONTEXT_SERVICE_NAR));
|
||||
sslControllerService.setProperties(sslControllerServiceProperties());
|
||||
sslControllerService.setControllerServiceApis(List.of(
|
||||
controllerServiceAPI(SSL_CONTEXT_SERVICE_API, createBundle(STANDARD_SERVICES_API_NAR_ARTIFACT)),
|
||||
controllerServiceAPI(RESTRICTED_SSL_CONTEXT_SERVICE_API, createBundle(STANDARD_SERVICES_API_NAR_ARTIFACT))
|
||||
));
|
||||
sslControllerService.setPropertyDescriptors(Map.of());
|
||||
return Optional.of(sslControllerService);
|
||||
}
|
||||
|
||||
private boolean parentSslEnabled() {
|
||||
return MiNiFiProperties.securityPropertyKeys().stream()
|
||||
.map(minifiProperties::getProperty)
|
||||
.allMatch(StringUtils::isNotBlank);
|
||||
}
|
||||
|
||||
private Map<String, String> sslControllerServiceProperties() {
|
||||
return Map.of(
|
||||
"Keystore Filename", minifiProperties.getProperty(MiNiFiProperties.NIFI_MINIFI_SECURITY_KEYSTORE.getKey()),
|
||||
"Keystore Password", minifiProperties.getProperty(MiNiFiProperties.NIFI_MINIFI_SECURITY_KEYSTORE_PASSWD.getKey()),
|
||||
"key-password", minifiProperties.getProperty(MiNiFiProperties.NIFI_MINIFI_SECURITY_KEY_PASSWD.getKey()),
|
||||
"Keystore Type", minifiProperties.getProperty(MiNiFiProperties.NIFI_MINIFI_SECURITY_KEYSTORE_TYPE.getKey()),
|
||||
"Truststore Filename", minifiProperties.getProperty(MiNiFiProperties.NIFI_MINIFI_SECURITY_TRUSTSTORE.getKey()),
|
||||
"Truststore Password", minifiProperties.getProperty(MiNiFiProperties.NIFI_MINIFI_SECURITY_TRUSTSTORE_PASSWD.getKey()),
|
||||
"Truststore Type", minifiProperties.getProperty(MiNiFiProperties.NIFI_MINIFI_SECURITY_TRUSTSTORE_TYPE.getKey()),
|
||||
"SSL Protocol", minifiProperties.getProperty(MiNiFiProperties.NIFI_MINIFI_SECURITY_SSL_PROTOCOL.getKey())
|
||||
);
|
||||
}
|
||||
|
||||
private ControllerServiceAPI controllerServiceAPI(String type, Bundle bundle) {
|
||||
ControllerServiceAPI controllerServiceAPI = new ControllerServiceAPI();
|
||||
controllerServiceAPI.setType(type);
|
||||
controllerServiceAPI.setBundle(bundle);
|
||||
return controllerServiceAPI;
|
||||
}
|
||||
|
||||
private void overrideProcessorsSslControllerService(VersionedProcessGroup processGroup, String commonSslControllerServiceInstanceId) {
|
||||
LOG.debug("Use parent SSL is enabled, overriding processors' SSL Controller service to {}", commonSslControllerServiceInstanceId);
|
||||
processGroup.getProcessors()
|
||||
.forEach(processor -> processor.getProperties()
|
||||
.replace(
|
||||
DEFAULT_SSL_CONTEXT_SERVICE_NAME,
|
||||
processor.getProperties().get(DEFAULT_SSL_CONTEXT_SERVICE_NAME),
|
||||
commonSslControllerServiceInstanceId));
|
||||
processGroup.getProcessGroups()
|
||||
.forEach(childProcessGroup -> overrideProcessorsSslControllerService(childProcessGroup, commonSslControllerServiceInstanceId));
|
||||
}
|
||||
|
||||
private Optional<VersionedReportingTask> createProvenanceReportingTask(String sslControllerServiceIdentifier) {
|
||||
if (!provenanceReportingEnabled()) {
|
||||
LOG.debug("Provenance reporting task is disabled, skip creating provenance reporting task");
|
||||
return empty();
|
||||
}
|
||||
LOG.debug("Provenance reporting task is enabled, creating provenance reporting task");
|
||||
|
||||
VersionedReportingTask reportingTask = new VersionedReportingTask();
|
||||
reportingTask.setIdentifier(randomUUID().toString());
|
||||
reportingTask.setInstanceIdentifier(SITE_TO_SITE_PROVENANCE_REPORTING_TASK_ID);
|
||||
reportingTask.setName(SITE_TO_SITE_PROVENANCE_REPORTING_TASK_NAME);
|
||||
reportingTask.setComments(minifiProperties.getProperty(MiNiFiProperties.NIFI_MINIFI_PROVENANCE_REPORTING_COMMENT.getKey()));
|
||||
reportingTask.setType(SITE_TO_SITE_PROVENANCE_REPORTING_TASK);
|
||||
reportingTask.setBundle(createBundle(SITE_TO_SITE_REPORTING_NAR_ARTIFACT));
|
||||
reportingTask.setScheduledState(ScheduledState.RUNNING);
|
||||
reportingTask.setSchedulingStrategy(minifiProperties.getProperty(MiNiFiProperties.NIFI_MINIFI_PROVENANCE_REPORTING_SCHEDULING_STRATEGY.getKey()));
|
||||
reportingTask.setSchedulingPeriod(minifiProperties.getProperty(MiNiFiProperties.NIFI_MINIFI_PROVENANCE_REPORTING_SCHEDULING_PERIOD.getKey()));
|
||||
reportingTask.setComponentType(ComponentType.REPORTING_TASK);
|
||||
reportingTask.setProperties(provenanceReportingTaskProperties(sslControllerServiceIdentifier));
|
||||
reportingTask.setPropertyDescriptors(Map.of());
|
||||
return Optional.of(reportingTask);
|
||||
}
|
||||
|
||||
private boolean provenanceReportingEnabled() {
|
||||
return MiNiFiProperties.provenanceReportingPropertyKeys().stream()
|
||||
.map(minifiProperties::getProperty)
|
||||
.allMatch(StringUtils::isNotBlank);
|
||||
}
|
||||
|
||||
private Bundle createBundle(String artifact) {
|
||||
Bundle bundle = new Bundle();
|
||||
bundle.setGroup(NIFI_BUNDLE_GROUP);
|
||||
bundle.setArtifact(artifact);
|
||||
bundle.setVersion(EMPTY);
|
||||
return bundle;
|
||||
}
|
||||
|
||||
private Map<String, String> provenanceReportingTaskProperties(String sslControllerServiceIdentifier) {
|
||||
return List.of(
|
||||
entry("Input Port Name", minifiProperties.getProperty(MiNiFiProperties.NIFI_MINIFI_PROVENANCE_REPORTING_INPUT_PORT_NAME.getKey())),
|
||||
entry("s2s-transport-protocol", PROVENANCE_REPORTING_TASK_PROTOCOL),
|
||||
entry("Platform", "nifi"),
|
||||
entry("Destination URL", minifiProperties.getProperty(MiNiFiProperties.NIFI_MINIFI_PROVENANCE_REPORTING_DESTINATION_URL.getKey())),
|
||||
entry("include-null-values", FALSE.toString()),
|
||||
entry("Compress Events", minifiProperties.getProperty(MiNiFiProperties.NIFI_MINIFI_PROVENANCE_REPORTING_COMPRESS_EVENTS.getKey())),
|
||||
entry("Batch Size", minifiProperties.getProperty(MiNiFiProperties.NIFI_MINIFI_PROVENANCE_REPORTING_BATCH_SIZE.getKey())),
|
||||
entry("Communications Timeout", minifiProperties.getProperty(MiNiFiProperties.NIFI_MINIFI_PROVENANCE_REPORTING_COMMUNICATIONS_TIMEOUT.getKey())),
|
||||
entry("start-position", PROVENANCE_REPORTING_TASK_BEGINNING_OF_STREAM),
|
||||
entry("Instance URL", minifiProperties.getProperty(MiNiFiProperties.NIFI_MINIFI_PROVENANCE_REPORTING_INSTANCE_URL.getKey())),
|
||||
entry("SSL Context Service", sslControllerServiceIdentifier))
|
||||
.stream()
|
||||
.filter(entry -> StringUtils.isNotBlank(entry.getValue()))
|
||||
.collect(toMap(Map.Entry::getKey, Map.Entry::getValue));
|
||||
}
|
||||
|
||||
private byte[] toByteArray(VersionedDataflow versionedDataflow) {
|
||||
try (ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream()) {
|
||||
JsonFactory factory = new JsonFactory();
|
||||
JsonGenerator generator = factory.createGenerator(byteArrayOutputStream);
|
||||
generator.setCodec(serializationObjectMapper());
|
||||
generator.writeObject(versionedDataflow);
|
||||
generator.flush();
|
||||
byteArrayOutputStream.flush();
|
||||
return byteArrayOutputStream.toByteArray();
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException("Unable to convert flow to byte array", e);
|
||||
}
|
||||
}
|
||||
|
||||
private ObjectMapper serializationObjectMapper() {
|
||||
ObjectMapper objectMapper = new ObjectMapper();
|
||||
objectMapper.setSerializationInclusion(JsonInclude.Include.NON_NULL);
|
||||
objectMapper.setDefaultPropertyInclusion(JsonInclude.Value.construct(JsonInclude.Include.NON_NULL, JsonInclude.Include.NON_NULL));
|
||||
objectMapper.setAnnotationIntrospector(new JaxbAnnotationIntrospector(objectMapper.getTypeFactory()));
|
||||
objectMapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
|
||||
return objectMapper;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,88 @@
|
|||
/*
|
||||
* Licensed to the Apache 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.commons.util;
|
||||
|
||||
import static java.nio.charset.StandardCharsets.UTF_8;
|
||||
import static java.nio.file.Files.copy;
|
||||
import static java.nio.file.Files.deleteIfExists;
|
||||
import static java.nio.file.Files.exists;
|
||||
import static java.nio.file.Files.newOutputStream;
|
||||
import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
|
||||
import static java.nio.file.StandardOpenOption.CREATE;
|
||||
import static java.nio.file.StandardOpenOption.TRUNCATE_EXISTING;
|
||||
import static java.nio.file.StandardOpenOption.WRITE;
|
||||
|
||||
import java.io.ByteArrayInputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.io.UncheckedIOException;
|
||||
import java.nio.file.Path;
|
||||
import java.util.zip.GZIPOutputStream;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
public abstract class FlowUpdateUtils {
|
||||
|
||||
private static final Logger LOGGER = LoggerFactory.getLogger(FlowUpdateUtils.class);
|
||||
|
||||
private FlowUpdateUtils() {
|
||||
}
|
||||
|
||||
public static void backup(Path current, Path backup) throws IOException {
|
||||
if (current != null && exists(current)) {
|
||||
copy(current, backup, REPLACE_EXISTING);
|
||||
} else {
|
||||
LOGGER.warn("Flow configuration files does not exist. No backup copy will be created");
|
||||
}
|
||||
}
|
||||
|
||||
public static void persist(byte[] flowConfig, Path flowPath, boolean compress) throws IOException {
|
||||
LOGGER.debug("Persisting flow to path {} with content\n{}\n and compress {}", flowPath, new String(flowConfig, UTF_8), compress);
|
||||
try (ByteArrayInputStream flowInputStream = new ByteArrayInputStream(flowConfig);
|
||||
OutputStream fileOut = compress
|
||||
? new GZIPOutputStream(newOutputStream(flowPath, WRITE, CREATE, TRUNCATE_EXISTING))
|
||||
: newOutputStream(flowPath, WRITE, CREATE, TRUNCATE_EXISTING)) {
|
||||
flowInputStream.transferTo(fileOut);
|
||||
}
|
||||
LOGGER.info("Updated configuration was written to: {}", flowPath);
|
||||
}
|
||||
|
||||
public static void revert(Path backupFlowConfigFile, Path currentFlowConfigFile) {
|
||||
if (backupFlowConfigFile != null && exists(backupFlowConfigFile)) {
|
||||
try {
|
||||
copy(backupFlowConfigFile, currentFlowConfigFile, REPLACE_EXISTING);
|
||||
} catch (IOException e) {
|
||||
LOGGER.error("Revert to previous flow failed. Please stop MiNiFi and revert the flow manually");
|
||||
throw new UncheckedIOException("Failed to revert flow", e);
|
||||
}
|
||||
} else {
|
||||
LOGGER.error("Backup flow configuration file does not exist");
|
||||
throw new RuntimeException("Backup flow configuration file does not exist");
|
||||
}
|
||||
}
|
||||
|
||||
public static void removeIfExists(Path flowConfigFile) {
|
||||
if (flowConfigFile != null) {
|
||||
try {
|
||||
deleteIfExists(flowConfigFile);
|
||||
} catch (IOException e) {
|
||||
LOGGER.warn("Unable to remove flow configuration backup file", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue