NIFI-1857: HTTPS Site-to-Site

- Enable HTTP(S) for Site-to-Site communication
- Support HTTP Proxy in the middle of local and remote NiFi
- Support BASIC and DIGEST auth with Proxy Server
- Provide 2-phase style commit same as existing socket version
- [WIP] Test with the latest cluster env (without NCM) hasn't tested yet

- Fixed Buffer handling issues at asyc http client POST
- Fixed JS error when applying Remote Process Group Port setting from UI
- Use compression setting from UI
- Removed already finished TODO comments

- Added additional buffer draining code after receiving EOF
- Added inspection and assert code to make sure Site-to-Site client has
  written data fully to output
stream
- Changed default nifi.remote.input.secure from true to false

This closes #497.
This commit is contained in:
Koji Kawamura 2016-06-06 22:19:26 +09:00 committed by Mark Payne
parent a5fecda5a2
commit c120c4982d
105 changed files with 9984 additions and 1970 deletions

View File

@ -60,9 +60,11 @@ public class NiFiProperties extends Properties {
public static final String SENSITIVE_PROPS_ALGORITHM = "nifi.sensitive.props.algorithm";
public static final String SENSITIVE_PROPS_PROVIDER = "nifi.sensitive.props.provider";
public static final String H2_URL_APPEND = "nifi.h2.url.append";
public static final String REMOTE_INPUT_HOST = "nifi.remote.input.socket.host";
public static final String REMOTE_INPUT_HOST = "nifi.remote.input.host";
public static final String REMOTE_INPUT_PORT = "nifi.remote.input.socket.port";
public static final String SITE_TO_SITE_SECURE = "nifi.remote.input.secure";
public static final String SITE_TO_SITE_HTTP_ENABLED = "nifi.remote.input.http.enabled";
public static final String SITE_TO_SITE_HTTP_TRANSACTION_TTL = "nifi.remote.input.http.transaction.ttl";
public static final String TEMPLATE_DIRECTORY = "nifi.templates.directory";
public static final String ADMINISTRATIVE_YIELD_DURATION = "nifi.administrative.yield.duration";
public static final String PERSISTENT_STATE_DIRECTORY = "nifi.persistent.state.directory";
@ -210,6 +212,7 @@ public class NiFiProperties extends Properties {
public static final String DEFAULT_ZOOKEEPER_CONNECT_TIMEOUT = "3 secs";
public static final String DEFAULT_ZOOKEEPER_SESSION_TIMEOUT = "3 secs";
public static final String DEFAULT_ZOOKEEPER_ROOT_NODE = "/nifi";
public static final String DEFAULT_SITE_TO_SITE_HTTP_TRANSACTION_TTL = "30 secs";
// cluster common defaults
public static final String DEFAULT_CLUSTER_PROTOCOL_HEARTBEAT_INTERVAL = "5 sec";
@ -387,7 +390,7 @@ public class NiFiProperties extends Properties {
/**
* The socket port to listen on for a Remote Input Port.
*
* @return the remote input port
* @return the remote input port for RAW socket communication
*/
public Integer getRemoteInputPort() {
return getPropertyAsPort(REMOTE_INPUT_PORT, DEFAULT_REMOTE_INPUT_PORT);
@ -407,6 +410,38 @@ public class NiFiProperties extends Properties {
}
/**
* @return True if property value is 'true'; False otherwise.
*/
public Boolean isSiteToSiteHttpEnabled() {
final String remoteInputHttpEnabled = getProperty(SITE_TO_SITE_HTTP_ENABLED, "false");
if ("true".equalsIgnoreCase(remoteInputHttpEnabled)) {
return true;
} else {
return false;
}
}
/**
* The HTTP or HTTPS Web API port for a Remote Input Port.
* @return the remote input port for HTTP(S) communication, or null if HTTP(S) Site-to-Site is not enabled
*/
public Integer getRemoteInputHttpPort() {
if (!isSiteToSiteHttpEnabled()) {
return null;
}
String propertyKey = isSiteToSiteSecure() ? NiFiProperties.WEB_HTTPS_PORT : NiFiProperties.WEB_HTTP_PORT;
Integer port = getIntegerProperty(propertyKey, 0);
if (port == 0) {
throw new RuntimeException("Remote input HTTP" + (isSiteToSiteSecure() ? "S" : "")
+ " is enabled but " + propertyKey + " is not specified.");
}
return port;
}
/**
* Returns the directory to which Templates are to be persisted
*

View File

@ -44,6 +44,15 @@
<artifactId>nifi-client-dto</artifactId>
<version>1.0.0-SNAPSHOT</version>
</dependency>
<dependency>
<groupId>org.apache.httpcomponents</groupId>
<artifactId>httpclient</artifactId>
</dependency>
<dependency>
<groupId>org.apache.httpcomponents</groupId>
<artifactId>httpasyncclient</artifactId>
<version>4.1.1</version>
</dependency>
<dependency>
<groupId>junit</groupId>
@ -56,5 +65,15 @@
<version>2.24.0</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.eclipse.jetty</groupId>
<artifactId>jetty-server</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.eclipse.jetty</groupId>
<artifactId>jetty-servlet</artifactId>
<scope>test</scope>
</dependency>
</dependencies>
</project>

View File

@ -0,0 +1,389 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT 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.remote;
import org.apache.nifi.events.EventReporter;
import org.apache.nifi.remote.codec.FlowFileCodec;
import org.apache.nifi.remote.exception.ProtocolException;
import org.apache.nifi.remote.io.CompressionInputStream;
import org.apache.nifi.remote.io.CompressionOutputStream;
import org.apache.nifi.remote.protocol.CommunicationsSession;
import org.apache.nifi.remote.protocol.DataPacket;
import org.apache.nifi.remote.protocol.Response;
import org.apache.nifi.remote.protocol.ResponseCode;
import org.apache.nifi.remote.util.StandardDataPacket;
import org.apache.nifi.reporting.Severity;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.ByteArrayInputStream;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.util.Map;
import java.util.zip.CRC32;
import java.util.zip.CheckedInputStream;
import java.util.zip.CheckedOutputStream;
public abstract class AbstractTransaction implements Transaction {
protected final Logger logger = LoggerFactory.getLogger(getClass());
protected final Peer peer;
protected final TransferDirection direction;
private final CRC32 crc = new CRC32();
private final boolean compress;
protected final FlowFileCodec codec;
protected final EventReporter eventReporter;
protected final int protocolVersion;
private final int penaltyMillis;
protected final String destinationId;
protected TransactionState state;
protected boolean dataAvailable = false;
private final long creationNanoTime = System.nanoTime();
private int transfers = 0;
private long contentBytes = 0;
public AbstractTransaction(final Peer peer, final TransferDirection direction, final boolean useCompression,
final FlowFileCodec codec, final EventReporter eventReporter, final int protocolVersion,
final int penaltyMillis, final String destinationId) {
this.peer = peer;
this.state = TransactionState.TRANSACTION_STARTED;
this.direction = direction;
this.compress = useCompression;
this.codec = codec;
this.eventReporter = eventReporter;
this.protocolVersion = protocolVersion;
this.penaltyMillis = penaltyMillis;
this.destinationId = destinationId;
}
protected void close() throws IOException {
}
@Override
public void send(final byte[] content, final Map<String, String> attributes) throws IOException {
send(new StandardDataPacket(attributes, new ByteArrayInputStream(content), content.length));
}
@Override
public void error() {
this.state = TransactionState.ERROR;
try {
close();
} catch (IOException e) {
logger.warn("Failed to close transaction due to {}", e.getMessage());
if (logger.isDebugEnabled()) {
logger.warn("", e);
}
}
}
@Override
public TransactionState getState() {
return state;
}
@Override
public Communicant getCommunicant() {
return peer;
}
@Override
public final DataPacket receive() throws IOException {
try {
try {
if (state != TransactionState.DATA_EXCHANGED && state != TransactionState.TRANSACTION_STARTED) {
throw new IllegalStateException("Cannot receive data from " + peer + " because Transaction State is " + state);
}
if (direction == TransferDirection.SEND) {
throw new IllegalStateException("Attempting to receive data from " + peer + " but started a SEND Transaction");
}
// if we already know there's no data, just return null
if (!dataAvailable) {
return null;
}
// if we have already received a packet, check if another is available.
if (transfers > 0) {
// Determine if Peer will send us data or has no data to send us
final Response dataAvailableCode = readTransactionResponse();
switch (dataAvailableCode.getCode()) {
case CONTINUE_TRANSACTION:
logger.debug("{} {} Indicates Transaction should continue", this, peer);
this.dataAvailable = true;
break;
case FINISH_TRANSACTION:
logger.debug("{} {} Indicates Transaction should finish", this, peer);
this.dataAvailable = false;
break;
default:
throw new ProtocolException("Got unexpected response from " + peer + " when asking for data: " + dataAvailableCode);
}
}
// if no data available, return null
if (!dataAvailable) {
return null;
}
logger.debug("{} Receiving data from {}", this, peer);
final InputStream is = peer.getCommunicationsSession().getInput().getInputStream();
final InputStream dataIn = compress ? new CompressionInputStream(is) : is;
final DataPacket packet = codec.decode(new CheckedInputStream(dataIn, crc));
if (packet == null) {
this.dataAvailable = false;
} else {
transfers++;
contentBytes += packet.getSize();
}
this.state = TransactionState.DATA_EXCHANGED;
return packet;
} catch (final IOException ioe) {
throw new IOException("Failed to receive data from " + peer + " due to " + ioe, ioe);
}
} catch (final Exception e) {
error();
throw e;
}
}
abstract protected Response readTransactionResponse() throws IOException;
protected final void writeTransactionResponse(ResponseCode response) throws IOException {
writeTransactionResponse(response, null);
}
abstract protected void writeTransactionResponse(ResponseCode response, String explanation) throws IOException;
@Override
public final void confirm() throws IOException {
try {
try {
if (state == TransactionState.TRANSACTION_STARTED && !dataAvailable && direction == TransferDirection.RECEIVE) {
// client requested to receive data but no data available. no need to confirm.
state = TransactionState.TRANSACTION_CONFIRMED;
return;
}
if (state != TransactionState.DATA_EXCHANGED) {
throw new IllegalStateException("Cannot confirm Transaction because state is " + state
+ "; Transaction can only be confirmed when state is " + TransactionState.DATA_EXCHANGED);
}
final CommunicationsSession commsSession = peer.getCommunicationsSession();
if (direction == TransferDirection.RECEIVE) {
if (dataAvailable) {
throw new IllegalStateException("Cannot complete transaction because the sender has already sent more data than client has consumed.");
}
// we received a FINISH_TRANSACTION indicator. Send back a CONFIRM_TRANSACTION message
// to peer so that we can verify that the connection is still open. This is a two-phase commit,
// which helps to prevent the chances of data duplication. Without doing this, we may commit the
// session and then when we send the response back to the peer, the peer may have timed out and may not
// be listening. As a result, it will re-send the data. By doing this two-phase commit, we narrow the
// Critical Section involved in this transaction so that rather than the Critical Section being the
// time window involved in the entire transaction, it is reduced to a simple round-trip conversation.
logger.trace("{} Sending CONFIRM_TRANSACTION Response Code to {}", this, peer);
final String calculatedCRC = String.valueOf(crc.getValue());
writeTransactionResponse(ResponseCode.CONFIRM_TRANSACTION, calculatedCRC);
final Response confirmTransactionResponse;
try {
confirmTransactionResponse = readTransactionResponse();
} catch (final IOException ioe) {
logger.error("Failed to receive response code from {} when expecting confirmation of transaction", peer);
if (eventReporter != null) {
eventReporter.reportEvent(Severity.ERROR, "Site-to-Site", "Failed to receive response code from " + peer + " when expecting confirmation of transaction");
}
throw ioe;
}
logger.trace("{} Received {} from {}", this, confirmTransactionResponse, peer);
switch (confirmTransactionResponse.getCode()) {
case CONFIRM_TRANSACTION:
break;
case BAD_CHECKSUM:
throw new IOException(this + " Received a BadChecksum response from peer " + peer);
default:
throw new ProtocolException(this + " Received unexpected Response from peer " + peer + " : "
+ confirmTransactionResponse + "; expected 'Confirm Transaction' Response Code");
}
state = TransactionState.TRANSACTION_CONFIRMED;
} else {
logger.debug("{} Sent FINISH_TRANSACTION indicator to {}", this, peer);
writeTransactionResponse(ResponseCode.FINISH_TRANSACTION);
final String calculatedCRC = String.valueOf(crc.getValue());
// we've sent a FINISH_TRANSACTION. Now we'll wait for the peer to send a 'Confirm Transaction' response
final Response transactionConfirmationResponse = readTransactionResponse();
if (transactionConfirmationResponse.getCode() == ResponseCode.CONFIRM_TRANSACTION) {
// Confirm checksum and echo back the confirmation.
logger.trace("{} Received {} from {}", this, transactionConfirmationResponse, peer);
final String receivedCRC = transactionConfirmationResponse.getMessage();
// CRC was not used before version 4
if (protocolVersion > 3) {
if (!receivedCRC.equals(calculatedCRC)) {
writeTransactionResponse(ResponseCode.BAD_CHECKSUM);
throw new IOException(this + " Sent data to peer " + peer + " but calculated CRC32 Checksum as "
+ calculatedCRC + " while peer calculated CRC32 Checksum as "
+ receivedCRC + "; canceling transaction and rolling back session");
}
}
writeTransactionResponse(ResponseCode.CONFIRM_TRANSACTION, "");
} else {
throw new ProtocolException("Expected to receive 'Confirm Transaction' response from peer "
+ peer + " but received " + transactionConfirmationResponse);
}
state = TransactionState.TRANSACTION_CONFIRMED;
}
} catch (final IOException ioe) {
throw new IOException("Failed to confirm transaction with " + peer + " due to " + ioe, ioe);
}
} catch (final Exception e) {
error();
throw e;
}
}
@Override
public final TransactionCompletion complete() throws IOException {
try {
try {
if (state != TransactionState.TRANSACTION_CONFIRMED) {
throw new IllegalStateException("Cannot complete transaction with " + peer + " because state is " + state
+ "; Transaction can only be completed when state is " + TransactionState.TRANSACTION_CONFIRMED);
}
boolean backoff = false;
if (direction == TransferDirection.RECEIVE) {
if (transfers == 0) {
state = TransactionState.TRANSACTION_COMPLETED;
return new ClientTransactionCompletion(false, 0, 0L, System.nanoTime() - creationNanoTime);
}
// Confirm that we received the data and the peer can now discard it
logger.debug("{} Sending TRANSACTION_FINISHED to {}", this, peer);
writeTransactionResponse(ResponseCode.TRANSACTION_FINISHED);
state = TransactionState.TRANSACTION_COMPLETED;
} else {
final Response transactionResponse;
try {
transactionResponse = readTransactionResponse();
} catch (final IOException e) {
throw new IOException(this + " Failed to receive a response from " + peer + " when expecting a TransactionFinished Indicator. "
+ "It is unknown whether or not the peer successfully received/processed the data.", e);
}
logger.debug("{} Received {} from {}", this, transactionResponse, peer);
if (transactionResponse.getCode() == ResponseCode.TRANSACTION_FINISHED_BUT_DESTINATION_FULL) {
peer.penalize(destinationId, penaltyMillis);
backoff = true;
} else if (transactionResponse.getCode() != ResponseCode.TRANSACTION_FINISHED) {
throw new ProtocolException("After sending data to " + peer + ", expected TRANSACTION_FINISHED response but got " + transactionResponse);
}
state = TransactionState.TRANSACTION_COMPLETED;
}
return new ClientTransactionCompletion(backoff, transfers, contentBytes, System.nanoTime() - creationNanoTime);
} catch (final IOException ioe) {
throw new IOException("Failed to complete transaction with " + peer + " due to " + ioe, ioe);
}
} catch (final Exception e) {
error();
throw e;
} finally {
close();
}
}
@Override
public final void cancel(final String explanation) throws IOException {
if (state == TransactionState.TRANSACTION_CANCELED || state == TransactionState.TRANSACTION_COMPLETED || state == TransactionState.ERROR) {
throw new IllegalStateException("Cannot cancel transaction because state is already " + state);
}
try {
writeTransactionResponse(ResponseCode.CANCEL_TRANSACTION, explanation == null ? "<No explanation given>" : explanation);
state = TransactionState.TRANSACTION_CANCELED;
} catch (final IOException ioe) {
error();
throw new IOException("Failed to send 'cancel transaction' message to " + peer + " due to " + ioe, ioe);
} finally {
close();
}
}
@Override
public final String toString() {
return getClass().getSimpleName() + "[Url=" + peer.getUrl() + ", TransferDirection=" + direction + ", State=" + state + "]";
}
@Override
public final void send(final DataPacket dataPacket) throws IOException {
try {
try {
if (state != TransactionState.DATA_EXCHANGED && state != TransactionState.TRANSACTION_STARTED) {
throw new IllegalStateException("Cannot send data to " + peer + " because Transaction State is " + state);
}
if (direction == TransferDirection.RECEIVE) {
throw new IllegalStateException("Attempting to send data to " + peer + " but started a RECEIVE Transaction");
}
if (transfers > 0) {
writeTransactionResponse(ResponseCode.CONTINUE_TRANSACTION);
}
logger.debug("{} Sending data to {}", this, peer);
final OutputStream os = peer.getCommunicationsSession().getOutput().getOutputStream();
final OutputStream dataOut = compress ? new CompressionOutputStream(os) : os;
final OutputStream out = new CheckedOutputStream(dataOut, crc);
codec.encode(dataPacket, out);
// need to close the CompressionOutputStream in order to force it write out any remaining bytes.
// Otherwise, do NOT close it because we don't want to close the underlying stream
// (CompressionOutputStream will not close the underlying stream when it's closed)
if (compress) {
out.close();
}
transfers++;
contentBytes += dataPacket.getSize();
this.state = TransactionState.DATA_EXCHANGED;
} catch (final IOException ioe) {
throw new IOException("Failed to send data to " + peer + " due to " + ioe, ioe);
}
} catch (final Exception e) {
error();
throw e;
}
}
}

View File

@ -14,20 +14,20 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.remote.protocol.socket;
package org.apache.nifi.remote;
import java.util.concurrent.TimeUnit;
import org.apache.nifi.remote.TransactionCompletion;
public class SocketClientTransactionCompletion implements TransactionCompletion {
public class ClientTransactionCompletion implements TransactionCompletion {
private final boolean backoff;
private final int dataPacketsTransferred;
private final long bytesTransferred;
private final long durationNanos;
public SocketClientTransactionCompletion(final boolean backoff, final int dataPacketsTransferred, final long bytesTransferred, final long durationNanos) {
public ClientTransactionCompletion(final boolean backoff, final int dataPacketsTransferred, final long bytesTransferred, final long durationNanos) {
this.backoff = backoff;
this.dataPacketsTransferred = dataPacketsTransferred;
this.bytesTransferred = bytesTransferred;

View File

@ -0,0 +1,54 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT 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.remote.client;
import java.net.URI;
import java.net.URISyntaxException;
import java.util.Objects;
import java.util.concurrent.TimeUnit;
public abstract class AbstractSiteToSiteClient implements SiteToSiteClient {
protected final SiteToSiteClientConfig config;
protected final SiteInfoProvider siteInfoProvider;
protected final URI clusterUrl;
public AbstractSiteToSiteClient(final SiteToSiteClientConfig config) {
this.config = config;
try {
Objects.requireNonNull(config.getUrl(), "URL cannot be null");
clusterUrl = new URI(config.getUrl());
} catch (final URISyntaxException e) {
throw new IllegalArgumentException("Invalid Cluster URL: " + config.getUrl());
}
final int commsTimeout = (int) config.getTimeout(TimeUnit.MILLISECONDS);
siteInfoProvider = new SiteInfoProvider();
siteInfoProvider.setClusterUrl(clusterUrl);
siteInfoProvider.setSslContext(config.getSslContext());
siteInfoProvider.setConnectTimeoutMillis(commsTimeout);
siteInfoProvider.setReadTimeoutMillis(commsTimeout);
siteInfoProvider.setProxy(config.getHttpProxy());
}
@Override
public SiteToSiteClientConfig getConfig() {
return config;
}
}

View File

@ -0,0 +1,341 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT 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.remote.client;
import org.apache.nifi.events.EventReporter;
import org.apache.nifi.remote.Peer;
import org.apache.nifi.remote.PeerDescription;
import org.apache.nifi.remote.PeerStatus;
import org.apache.nifi.remote.TransferDirection;
import org.apache.nifi.remote.util.PeerStatusCache;
import org.apache.nifi.stream.io.BufferedOutputStream;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.BufferedReader;
import java.io.File;
import java.io.FileInputStream;
import java.io.FileOutputStream;
import java.io.IOException;
import java.io.InputStream;
import java.io.InputStreamReader;
import java.io.OutputStream;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.locks.ReentrantLock;
import java.util.regex.Pattern;
import static org.apache.nifi.remote.util.EventReportUtil.error;
import static org.apache.nifi.remote.util.EventReportUtil.warn;
public class PeerSelector {
private static final Logger logger = LoggerFactory.getLogger(PeerSelector.class);
private static final long PEER_CACHE_MILLIS = TimeUnit.MILLISECONDS.convert(1, TimeUnit.MINUTES);
private static final long PEER_REFRESH_PERIOD = 60000L;
private final ReentrantLock peerRefreshLock = new ReentrantLock();
private volatile List<PeerStatus> peerStatuses;
private volatile long peerRefreshTime = 0L;
private final AtomicLong peerIndex = new AtomicLong(0L);
private volatile PeerStatusCache peerStatusCache;
private final File persistenceFile;
private EventReporter eventReporter;
private final PeerStatusProvider peerStatusProvider;
private final ConcurrentMap<PeerDescription, Long> peerTimeoutExpirations = new ConcurrentHashMap<>();
public PeerSelector(final PeerStatusProvider peerStatusProvider, final File persistenceFile) {
this.peerStatusProvider = peerStatusProvider;
this.persistenceFile = persistenceFile;
Set<PeerStatus> recoveredStatuses;
if (persistenceFile != null && persistenceFile.exists()) {
try {
recoveredStatuses = recoverPersistedPeerStatuses(persistenceFile);
this.peerStatusCache = new PeerStatusCache(recoveredStatuses, persistenceFile.lastModified());
} catch (final IOException ioe) {
logger.warn("Failed to recover peer statuses from {} due to {}; will continue without loading information from file", persistenceFile, ioe);
}
} else {
peerStatusCache = null;
}
}
private void persistPeerStatuses(final Set<PeerStatus> statuses) {
if (persistenceFile == null) {
return;
}
try (final OutputStream fos = new FileOutputStream(persistenceFile);
final OutputStream out = new BufferedOutputStream(fos)) {
for (final PeerStatus status : statuses) {
final PeerDescription description = status.getPeerDescription();
final String line = description.getHostname() + ":" + description.getPort() + ":" + description.isSecure() + "\n";
out.write(line.getBytes(StandardCharsets.UTF_8));
}
} catch (final IOException e) {
error(logger, eventReporter, "Failed to persist list of Peers due to {}; if restarted and peer's NCM is down," +
" may be unable to transfer data until communications with NCM are restored", e.toString());
logger.error("", e);
}
}
private static Set<PeerStatus> recoverPersistedPeerStatuses(final File file) throws IOException {
if (!file.exists()) {
return null;
}
final Set<PeerStatus> statuses = new HashSet<>();
try (final InputStream fis = new FileInputStream(file);
final BufferedReader reader = new BufferedReader(new InputStreamReader(fis))) {
String line;
while ((line = reader.readLine()) != null) {
final String[] splits = line.split(Pattern.quote(":"));
if (splits.length != 3) {
continue;
}
final String hostname = splits[0];
final int port = Integer.parseInt(splits[1]);
final boolean secure = Boolean.parseBoolean(splits[2]);
statuses.add(new PeerStatus(new PeerDescription(hostname, port, secure), 1));
}
}
return statuses;
}
List<PeerStatus> formulateDestinationList(final Set<PeerStatus> statuses, final TransferDirection direction) {
final int numDestinations = Math.max(128, statuses.size());
final Map<PeerStatus, Integer> entryCountMap = new HashMap<>();
long totalFlowFileCount = 0L;
for (final PeerStatus nodeInfo : statuses) {
totalFlowFileCount += nodeInfo.getFlowFileCount();
}
int totalEntries = 0;
for (final PeerStatus nodeInfo : statuses) {
final int flowFileCount = nodeInfo.getFlowFileCount();
// don't allow any node to get more than 80% of the data
final double percentageOfFlowFiles = Math.min(0.8D, ((double) flowFileCount / (double) totalFlowFileCount));
final double relativeWeighting = (direction == TransferDirection.SEND) ? (1 - percentageOfFlowFiles) : percentageOfFlowFiles;
final int entries = Math.max(1, (int) (numDestinations * relativeWeighting));
entryCountMap.put(nodeInfo, Math.max(1, entries));
totalEntries += entries;
}
final List<PeerStatus> destinations = new ArrayList<>(totalEntries);
for (int i = 0; i < totalEntries; i++) {
destinations.add(null);
}
for (final Map.Entry<PeerStatus, Integer> entry : entryCountMap.entrySet()) {
final PeerStatus nodeInfo = entry.getKey();
final int numEntries = entry.getValue();
int skipIndex = numEntries;
for (int i = 0; i < numEntries; i++) {
int n = (skipIndex * i);
while (true) {
final int index = n % destinations.size();
PeerStatus status = destinations.get(index);
if (status == null) {
status = new PeerStatus(nodeInfo.getPeerDescription(), nodeInfo.getFlowFileCount());
destinations.set(index, status);
break;
} else {
n++;
}
}
}
}
final StringBuilder distributionDescription = new StringBuilder();
distributionDescription.append("New Weighted Distribution of Nodes:");
for (final Map.Entry<PeerStatus, Integer> entry : entryCountMap.entrySet()) {
final double percentage = entry.getValue() * 100D / destinations.size();
distributionDescription.append("\n").append(entry.getKey()).append(" will receive ").append(percentage).append("% of data");
}
logger.info(distributionDescription.toString());
// Jumble the list of destinations.
return destinations;
}
/**
* Updates internal state map to penalize a PeerStatus that points to the
* specified peer
*
* @param peer the peer
* @param penalizationMillis period of time to penalize a given peer
*/
public void penalize(final Peer peer, final long penalizationMillis) {
penalize(peer.getDescription(), penalizationMillis);
}
public void penalize(final PeerDescription peerDescription, final long penalizationMillis) {
Long expiration = peerTimeoutExpirations.get(peerDescription);
if (expiration == null) {
expiration = Long.valueOf(0L);
}
final long newExpiration = Math.max(expiration, System.currentTimeMillis() + penalizationMillis);
peerTimeoutExpirations.put(peerDescription, Long.valueOf(newExpiration));
}
public boolean isPenalized(final PeerStatus peerStatus) {
final Long expirationEnd = peerTimeoutExpirations.get(peerStatus.getPeerDescription());
return (expirationEnd != null && expirationEnd > System.currentTimeMillis());
}
public void clear() {
peerTimeoutExpirations.clear();
}
private boolean isPeerRefreshNeeded(final List<PeerStatus> peerList) {
return (peerList == null || peerList.isEmpty() || System.currentTimeMillis() > peerRefreshTime + PEER_REFRESH_PERIOD);
}
/**
* Return status of a peer that will be used for the next communication.
* The peer with less workload will be selected with higher probability.
* @param direction the amount of workload is calculated based on transaction direction,
* for SEND, a peer with less flow files is preferred,
* for RECEIVE, a peer with more flow files is preferred
* @return a selected peer, if there is no available peer or all peers are penalized, then return null
*/
public PeerStatus getNextPeerStatus(final TransferDirection direction) {
List<PeerStatus> peerList = peerStatuses;
if (isPeerRefreshNeeded(peerList)) {
peerRefreshLock.lock();
try {
// now that we have the lock, check again that we need to refresh (because another thread
// could have been refreshing while we were waiting for the lock).
peerList = peerStatuses;
if (isPeerRefreshNeeded(peerList)) {
try {
peerList = createPeerStatusList(direction);
} catch (final Exception e) {
final String message = String.format("%s Failed to update list of peers due to %s", this, e.toString());
warn(logger, eventReporter, message);
if (logger.isDebugEnabled()) {
logger.warn("", e);
}
}
this.peerStatuses = peerList;
peerRefreshTime = System.currentTimeMillis();
}
} finally {
peerRefreshLock.unlock();
}
}
if (peerList == null || peerList.isEmpty()) {
return null;
}
PeerStatus peerStatus;
for (int i = 0; i < peerList.size(); i++) {
final long idx = peerIndex.getAndIncrement();
final int listIndex = (int) (idx % peerList.size());
peerStatus = peerList.get(listIndex);
if (isPenalized(peerStatus)) {
logger.debug("{} {} is penalized; will not communicate with this peer", this, peerStatus);
} else {
return peerStatus;
}
}
logger.debug("{} All peers appear to be penalized; returning null", this);
return null;
}
private List<PeerStatus> createPeerStatusList(final TransferDirection direction) throws IOException {
Set<PeerStatus> statuses = getPeerStatuses();
if (statuses == null) {
refreshPeers();
statuses = getPeerStatuses();
if (statuses == null) {
logger.debug("{} found no peers to connect to", this);
return Collections.emptyList();
}
}
return formulateDestinationList(statuses, direction);
}
private Set<PeerStatus> getPeerStatuses() {
final PeerStatusCache cache = this.peerStatusCache;
if (cache == null || cache.getStatuses() == null || cache.getStatuses().isEmpty()) {
return null;
}
if (cache.getTimestamp() + PEER_CACHE_MILLIS < System.currentTimeMillis()) {
final Set<PeerStatus> equalizedSet = new HashSet<>(cache.getStatuses().size());
for (final PeerStatus status : cache.getStatuses()) {
final PeerStatus equalizedStatus = new PeerStatus(status.getPeerDescription(), 1);
equalizedSet.add(equalizedStatus);
}
return equalizedSet;
}
return cache.getStatuses();
}
public void refreshPeers() {
final PeerStatusCache existingCache = peerStatusCache;
if (existingCache != null && (existingCache.getTimestamp() + PEER_CACHE_MILLIS > System.currentTimeMillis())) {
return;
}
try {
final Set<PeerStatus> statuses = peerStatusProvider.fetchRemotePeerStatuses();
persistPeerStatuses(statuses);
peerStatusCache = new PeerStatusCache(statuses);
logger.info("{} Successfully refreshed Peer Status; remote instance consists of {} peers", this, statuses.size());
} catch (Exception e) {
warn(logger, eventReporter, "{} Unable to refresh Remote Group's peers due to {}", this, e.getMessage());
if (logger.isDebugEnabled()) {
logger.debug("", e);
}
}
}
public void setEventReporter(EventReporter eventReporter) {
this.eventReporter = eventReporter;
}
}

View File

@ -0,0 +1,27 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.remote.client;
import org.apache.nifi.remote.PeerStatus;
import java.io.IOException;
import java.util.Set;
public interface PeerStatusProvider {
Set<PeerStatus> fetchRemotePeerStatuses() throws IOException;
}

View File

@ -0,0 +1,231 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT 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.remote.client;
import org.apache.nifi.remote.TransferDirection;
import org.apache.nifi.remote.protocol.http.HttpProxy;
import org.apache.nifi.remote.util.SiteToSiteRestApiClient;
import org.apache.nifi.web.api.dto.ControllerDTO;
import org.apache.nifi.web.api.dto.PortDTO;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import javax.net.ssl.SSLContext;
import java.io.IOException;
import java.net.URI;
import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
public class SiteInfoProvider {
private static final Logger logger = LoggerFactory.getLogger(SiteInfoProvider.class);
private static final long REMOTE_REFRESH_MILLIS = TimeUnit.MILLISECONDS.convert(10, TimeUnit.MINUTES);
private final ReadWriteLock listeningPortRWLock = new ReentrantReadWriteLock();
private final Lock remoteInfoReadLock = listeningPortRWLock.readLock();
private final Lock remoteInfoWriteLock = listeningPortRWLock.writeLock();
private Integer siteToSitePort;
private Integer siteToSiteHttpPort;
private Boolean siteToSiteSecure;
private long remoteRefreshTime;
private HttpProxy proxy;
private final Map<String, String> inputPortMap = new HashMap<>(); // map input port name to identifier
private final Map<String, String> outputPortMap = new HashMap<>(); // map output port name to identifier
private URI clusterUrl;
private SSLContext sslContext;
private int connectTimeoutMillis;
private int readTimeoutMillis;
private ControllerDTO refreshRemoteInfo() throws IOException {
final ControllerDTO controller;
try (final SiteToSiteRestApiClient apiClient = new SiteToSiteRestApiClient(sslContext, proxy)) {
apiClient.resolveBaseUrl(clusterUrl);
apiClient.setConnectTimeoutMillis(connectTimeoutMillis);
apiClient.setReadTimeoutMillis(readTimeoutMillis);
controller = apiClient.getController();
}
remoteInfoWriteLock.lock();
try {
this.siteToSitePort = controller.getRemoteSiteListeningPort();
this.siteToSiteHttpPort = controller.getRemoteSiteHttpListeningPort();
this.siteToSiteSecure = controller.isSiteToSiteSecure();
inputPortMap.clear();
for (final PortDTO inputPort : controller.getInputPorts()) {
inputPortMap.put(inputPort.getName(), inputPort.getId());
}
outputPortMap.clear();
for (final PortDTO outputPort : controller.getOutputPorts()) {
outputPortMap.put(outputPort.getName(), outputPort.getId());
}
this.remoteRefreshTime = System.currentTimeMillis();
} finally {
remoteInfoWriteLock.unlock();
}
return controller;
}
public boolean isWebInterfaceSecure() {
return clusterUrl.toString().startsWith("https");
}
/**
* @return the port that the remote instance is listening on for
* RAW Socket site-to-site communication, or <code>null</code> if the remote instance
* is not configured to allow site-to-site communications.
*
* @throws IOException if unable to communicate with the remote instance
*/
public Integer getSiteToSitePort() throws IOException {
Integer listeningPort;
remoteInfoReadLock.lock();
try {
listeningPort = this.siteToSitePort;
if (listeningPort != null && this.remoteRefreshTime > System.currentTimeMillis() - REMOTE_REFRESH_MILLIS) {
return listeningPort;
}
} finally {
remoteInfoReadLock.unlock();
}
final ControllerDTO controller = refreshRemoteInfo();
listeningPort = controller.getRemoteSiteListeningPort();
return listeningPort;
}
/**
* @return the port that the remote instance is listening on for
* HTTP(S) site-to-site communication, or <code>null</code> if the remote instance
* is not configured to allow site-to-site communications.
*
* @throws IOException if unable to communicate with the remote instance
*/
public Integer getSiteToSiteHttpPort() throws IOException {
Integer listeningHttpPort;
remoteInfoReadLock.lock();
try {
listeningHttpPort = this.siteToSiteHttpPort;
if (listeningHttpPort != null && this.remoteRefreshTime > System.currentTimeMillis() - REMOTE_REFRESH_MILLIS) {
return listeningHttpPort;
}
} finally {
remoteInfoReadLock.unlock();
}
final ControllerDTO controller = refreshRemoteInfo();
listeningHttpPort = controller.getRemoteSiteHttpListeningPort();
return listeningHttpPort;
}
/**
* @return {@code true} if the remote instance is configured for secure
* site-to-site communications, {@code false} otherwise
* @throws IOException if unable to check if secure
*/
public boolean isSecure() throws IOException {
remoteInfoReadLock.lock();
try {
final Boolean secure = this.siteToSiteSecure;
if (secure != null && this.remoteRefreshTime > System.currentTimeMillis() - REMOTE_REFRESH_MILLIS) {
return secure;
}
} finally {
remoteInfoReadLock.unlock();
}
final ControllerDTO controller = refreshRemoteInfo();
final Boolean isSecure = controller.isSiteToSiteSecure();
if (isSecure == null) {
throw new IOException("Remote NiFi instance " + clusterUrl + " is not currently configured to accept site-to-site connections");
}
return isSecure;
}
public String getPortIdentifier(final String portName, final TransferDirection transferDirection) throws IOException {
if (transferDirection == TransferDirection.RECEIVE) {
return getOutputPortIdentifier(portName);
} else {
return getInputPortIdentifier(portName);
}
}
public String getInputPortIdentifier(final String portName) throws IOException {
return getPortIdentifier(portName, inputPortMap);
}
public String getOutputPortIdentifier(final String portName) throws IOException {
return getPortIdentifier(portName, outputPortMap);
}
private String getPortIdentifier(final String portName, final Map<String, String> portMap) throws IOException {
String identifier;
remoteInfoReadLock.lock();
try {
identifier = portMap.get(portName);
} finally {
remoteInfoReadLock.unlock();
}
if (identifier != null) {
return identifier;
}
refreshRemoteInfo();
remoteInfoReadLock.lock();
try {
return portMap.get(portName);
} finally {
remoteInfoReadLock.unlock();
}
}
public void setClusterUrl(URI clusterUrl) {
this.clusterUrl = clusterUrl;
}
public void setSslContext(SSLContext sslContext) {
this.sslContext = sslContext;
}
public void setConnectTimeoutMillis(int connectTimeoutMillis) {
this.connectTimeoutMillis = connectTimeoutMillis;
}
public void setReadTimeoutMillis(int readTimeoutMillis) {
this.readTimeoutMillis = readTimeoutMillis;
}
public void setProxy(HttpProxy proxy) {
this.proxy = proxy;
}
}

View File

@ -16,6 +16,22 @@
*/
package org.apache.nifi.remote.client;
import org.apache.nifi.events.EventReporter;
import org.apache.nifi.remote.Transaction;
import org.apache.nifi.remote.TransferDirection;
import org.apache.nifi.remote.client.http.HttpClient;
import org.apache.nifi.remote.client.socket.SocketClient;
import org.apache.nifi.remote.exception.HandshakeException;
import org.apache.nifi.remote.exception.PortNotRunningException;
import org.apache.nifi.remote.exception.ProtocolException;
import org.apache.nifi.remote.exception.UnknownPortException;
import org.apache.nifi.remote.protocol.DataPacket;
import org.apache.nifi.remote.protocol.SiteToSiteTransportProtocol;
import org.apache.nifi.remote.protocol.http.HttpProxy;
import javax.net.ssl.KeyManagerFactory;
import javax.net.ssl.SSLContext;
import javax.net.ssl.TrustManagerFactory;
import java.io.Closeable;
import java.io.File;
import java.io.FileInputStream;
@ -26,20 +42,6 @@ import java.security.KeyStore;
import java.security.SecureRandom;
import java.util.concurrent.TimeUnit;
import javax.net.ssl.KeyManagerFactory;
import javax.net.ssl.SSLContext;
import javax.net.ssl.TrustManagerFactory;
import org.apache.nifi.events.EventReporter;
import org.apache.nifi.remote.Transaction;
import org.apache.nifi.remote.TransferDirection;
import org.apache.nifi.remote.client.socket.SocketClient;
import org.apache.nifi.remote.exception.HandshakeException;
import org.apache.nifi.remote.exception.PortNotRunningException;
import org.apache.nifi.remote.exception.ProtocolException;
import org.apache.nifi.remote.exception.UnknownPortException;
import org.apache.nifi.remote.protocol.DataPacket;
/**
* <p>
* The SiteToSiteClient provides a mechanism for sending data to a remote
@ -163,6 +165,8 @@ public interface SiteToSiteClient extends Closeable {
private int batchCount;
private long batchSize;
private long batchNanos;
private SiteToSiteTransportProtocol transportProtocol = SiteToSiteTransportProtocol.RAW;
private HttpProxy httpProxy;
/**
* Populates the builder with values from the provided config
@ -185,11 +189,13 @@ public interface SiteToSiteClient extends Closeable {
this.eventReporter = config.getEventReporter();
this.peerPersistenceFile = config.getPeerPersistenceFile();
this.useCompression = config.isUseCompression();
this.transportProtocol = config.getTransportProtocol();
this.portName = config.getPortName();
this.portIdentifier = config.getPortIdentifier();
this.batchCount = config.getPreferredBatchCount();
this.batchSize = config.getPreferredBatchSize();
this.batchNanos = config.getPreferredBatchDuration(TimeUnit.NANOSECONDS);
this.httpProxy = config.getHttpProxy();
return this;
}
@ -440,6 +446,16 @@ public interface SiteToSiteClient extends Closeable {
return this;
}
/**
* Specifies the protocol to use for site to site data transport.
* @param transportProtocol transport protocol
* @return the builder
*/
public Builder transportProtocol(final SiteToSiteTransportProtocol transportProtocol) {
this.transportProtocol = transportProtocol;
return this;
}
/**
* Specifies the name of the port to communicate with. Either the port
* name or the port identifier must be specified.
@ -521,7 +537,8 @@ public interface SiteToSiteClient extends Closeable {
* data with remote instances of NiFi
*
* @throws IllegalStateException if either the url is not set or neither
* the port name nor port identifier is set.
* the port name nor port identifier is set,
* or if the transport protocol is not supported.
*/
public SiteToSiteClient build() {
if (url == null) {
@ -532,7 +549,14 @@ public interface SiteToSiteClient extends Closeable {
throw new IllegalStateException("Must specify either Port Name or Port Identifier to build Site-to-Site client");
}
switch (transportProtocol){
case RAW:
return new SocketClient(buildConfig());
case HTTP:
return new HttpClient(buildConfig());
default:
throw new IllegalStateException("Transport protocol '" + transportProtocol + "' is not supported.");
}
}
/**
@ -599,6 +623,13 @@ public interface SiteToSiteClient extends Closeable {
return useCompression;
}
/**
* @return the transport protocol to use, defaults to RAW
*/
public SiteToSiteTransportProtocol getTransportProtocol(){
return transportProtocol;
}
/**
* @return the name of the port that the client is to communicate with
*/
@ -613,6 +644,22 @@ public interface SiteToSiteClient extends Closeable {
public String getPortIdentifier() {
return portIdentifier;
}
/**
* Specify a HTTP proxy information to use with HTTP protocol of Site-to-Site communication.
* @param httpProxy HTTP proxy information
* @return the builder
*/
public Builder httpProxy(final HttpProxy httpProxy) {
this.httpProxy = httpProxy;
return this;
}
public HttpProxy getHttpProxy() {
return httpProxy;
}
}
@ -634,11 +681,13 @@ public interface SiteToSiteClient extends Closeable {
private final EventReporter eventReporter;
private final File peerPersistenceFile;
private final boolean useCompression;
private final SiteToSiteTransportProtocol transportProtocol;
private final String portName;
private final String portIdentifier;
private final int batchCount;
private final long batchSize;
private final long batchNanos;
private final HttpProxy httpProxy;
// some serialization frameworks require a default constructor
private StandardSiteToSiteClientConfig() {
@ -661,6 +710,8 @@ public interface SiteToSiteClient extends Closeable {
this.batchCount = 0;
this.batchSize = 0;
this.batchNanos = 0;
this.transportProtocol = null;
this.httpProxy = null;
}
private StandardSiteToSiteClientConfig(final SiteToSiteClient.Builder builder) {
@ -683,6 +734,8 @@ public interface SiteToSiteClient extends Closeable {
this.batchCount = builder.batchCount;
this.batchSize = builder.batchSize;
this.batchNanos = builder.batchNanos;
this.transportProtocol = builder.getTransportProtocol();
this.httpProxy = builder.getHttpProxy();
}
@Override
@ -830,5 +883,15 @@ public interface SiteToSiteClient extends Closeable {
public KeystoreType getTruststoreType() {
return truststoreType;
}
@Override
public SiteToSiteTransportProtocol getTransportProtocol() {
return transportProtocol;
}
@Override
public HttpProxy getHttpProxy() {
return httpProxy;
}
}
}

View File

@ -24,6 +24,8 @@ import javax.net.ssl.SSLContext;
import org.apache.nifi.events.EventReporter;
import org.apache.nifi.remote.protocol.DataPacket;
import org.apache.nifi.remote.protocol.SiteToSiteTransportProtocol;
import org.apache.nifi.remote.protocol.http.HttpProxy;
public interface SiteToSiteClientConfig extends Serializable {
@ -100,6 +102,11 @@ public interface SiteToSiteClientConfig extends Serializable {
*/
boolean isUseCompression();
/**
* @return a transport protocol to use
*/
SiteToSiteTransportProtocol getTransportProtocol();
/**
* @return the name of the port that the client is to communicate with
*/
@ -146,4 +153,10 @@ public interface SiteToSiteClientConfig extends Serializable {
*/
EventReporter getEventReporter();
/**
* Return Proxy for HTTP Transport Protocol.
* @return proxy or null if not specified
*/
HttpProxy getHttpProxy();
}

View File

@ -0,0 +1,200 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT 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.remote.client.http;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.remote.Peer;
import org.apache.nifi.remote.PeerDescription;
import org.apache.nifi.remote.PeerStatus;
import org.apache.nifi.remote.Transaction;
import org.apache.nifi.remote.TransferDirection;
import org.apache.nifi.remote.client.AbstractSiteToSiteClient;
import org.apache.nifi.remote.client.PeerSelector;
import org.apache.nifi.remote.client.PeerStatusProvider;
import org.apache.nifi.remote.client.SiteToSiteClientConfig;
import org.apache.nifi.remote.exception.HandshakeException;
import org.apache.nifi.remote.exception.PortNotRunningException;
import org.apache.nifi.remote.exception.ProtocolException;
import org.apache.nifi.remote.exception.UnknownPortException;
import org.apache.nifi.remote.io.http.HttpCommunicationsSession;
import org.apache.nifi.remote.protocol.CommunicationsSession;
import org.apache.nifi.remote.protocol.http.HttpClientTransaction;
import org.apache.nifi.remote.util.SiteToSiteRestApiClient;
import org.apache.nifi.web.api.dto.remote.PeerDTO;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.net.URI;
import java.net.URISyntaxException;
import java.util.Collection;
import java.util.Set;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.ThreadFactory;
import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;
public class HttpClient extends AbstractSiteToSiteClient implements PeerStatusProvider {
private static final Logger logger = LoggerFactory.getLogger(HttpClient.class);
private final ScheduledExecutorService taskExecutor;
private final PeerSelector peerSelector;
public HttpClient(final SiteToSiteClientConfig config) {
super(config);
peerSelector = new PeerSelector(this, config.getPeerPersistenceFile());
peerSelector.setEventReporter(config.getEventReporter());
taskExecutor = Executors.newScheduledThreadPool(1, new ThreadFactory() {
private final ThreadFactory defaultFactory = Executors.defaultThreadFactory();
@Override
public Thread newThread(final Runnable r) {
final Thread thread = defaultFactory.newThread(r);
thread.setName("Http Site-to-Site PeerSelector");
thread.setDaemon(true);
return thread;
}
});
taskExecutor.scheduleWithFixedDelay(new Runnable() {
@Override
public void run() {
peerSelector.refreshPeers();
}
}, 0, 5, TimeUnit.SECONDS);
}
@Override
public Set<PeerStatus> fetchRemotePeerStatuses() throws IOException {
if (siteInfoProvider.getSiteToSiteHttpPort() == null) {
throw new IOException("Remote instance of NiFi is not configured to allow HTTP site-to-site communications");
}
final String scheme = siteInfoProvider.isSecure() ? "https" : "http";
final URI clusterUrl;
try {
clusterUrl = new URI(config.getUrl());
} catch (URISyntaxException e) {
throw new IllegalArgumentException("Specified clusterUrl was: " + config.getUrl(), e);
}
try (
SiteToSiteRestApiClient apiClient = new SiteToSiteRestApiClient(config.getSslContext(), config.getHttpProxy())
) {
String clusterApiUrl = apiClient.resolveBaseUrl(scheme, clusterUrl.getHost(), siteInfoProvider.getSiteToSiteHttpPort());
int timeoutMillis = (int) config.getTimeout(TimeUnit.MILLISECONDS);
apiClient.setConnectTimeoutMillis(timeoutMillis);
apiClient.setReadTimeoutMillis(timeoutMillis);
Collection<PeerDTO> peers = apiClient.getPeers();
if(peers == null || peers.size() == 0){
throw new IOException("Couldn't get any peer to communicate with. " + clusterApiUrl + " returned zero peers.");
}
return peers.stream()
.map(p -> new PeerStatus(new PeerDescription(p.getHostname(), p.getPort(), p.isSecure()), p.getFlowFileCount()))
.collect(Collectors.toSet());
}
}
@Override
public Transaction createTransaction(TransferDirection direction) throws HandshakeException, PortNotRunningException, ProtocolException, UnknownPortException, IOException {
int timeoutMillis = (int) config.getTimeout(TimeUnit.MILLISECONDS);
PeerStatus peerStatus;
while ((peerStatus = peerSelector.getNextPeerStatus(direction)) != null) {
logger.debug("peerStatus={}", peerStatus);
CommunicationsSession commSession = new HttpCommunicationsSession();
String nodeApiUrl = resolveNodeApiUrl(peerStatus.getPeerDescription());
commSession.setUri(nodeApiUrl);
String clusterUrl = config.getUrl();
Peer peer = new Peer(peerStatus.getPeerDescription(), commSession, nodeApiUrl, clusterUrl);
int penaltyMillis = (int) config.getPenalizationPeriod(TimeUnit.MILLISECONDS);
String portId = config.getPortIdentifier();
if (StringUtils.isEmpty(portId)) {
portId = siteInfoProvider.getPortIdentifier(config.getPortName(), direction);
if (StringUtils.isEmpty(portId)) {
peer.close();
throw new IOException("Failed to determine the identifier of port " + config.getPortName());
}
}
SiteToSiteRestApiClient apiClient = new SiteToSiteRestApiClient(config.getSslContext(), config.getHttpProxy());
apiClient.setBaseUrl(peer.getUrl());
apiClient.setConnectTimeoutMillis(timeoutMillis);
apiClient.setReadTimeoutMillis(timeoutMillis);
apiClient.setCompress(config.isUseCompression());
apiClient.setRequestExpirationMillis(config.getIdleConnectionExpiration(TimeUnit.MILLISECONDS));
apiClient.setBatchCount(config.getPreferredBatchCount());
apiClient.setBatchSize(config.getPreferredBatchSize());
apiClient.setBatchDurationMillis(config.getPreferredBatchDuration(TimeUnit.MILLISECONDS));
final String transactionUrl;
try {
transactionUrl = apiClient.initiateTransaction(direction, portId);
commSession.setUserDn(apiClient.getTrustedPeerDn());
} catch (Exception e) {
logger.debug("Penalizing a peer due to {}", e.getMessage());
peerSelector.penalize(peer, penaltyMillis);
if (e instanceof UnknownPortException || e instanceof PortNotRunningException) {
throw e;
}
logger.debug("Continue trying other peers...");
continue;
}
// We found a valid peer to communicate with.
Integer transactionProtocolVersion = apiClient.getTransactionProtocolVersion();
HttpClientTransaction transaction = new HttpClientTransaction(transactionProtocolVersion, peer, direction,
config.isUseCompression(), portId, penaltyMillis, config.getEventReporter());
transaction.initialize(apiClient, transactionUrl);
return transaction;
}
logger.info("Couldn't find a valid peer to communicate with.");
return null;
}
private String resolveNodeApiUrl(PeerDescription description) {
return (description.isSecure() ? "https" : "http") + "://" + description.getHostname() + ":" + description.getPort() + "/nifi-api";
}
@Override
public boolean isSecure() throws IOException {
return siteInfoProvider.isWebInterfaceSecure();
}
@Override
public void close() throws IOException {
taskExecutor.shutdown();
peerSelector.clear();
}
}

View File

@ -0,0 +1,42 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT 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.remote.client.http;
import org.apache.nifi.remote.StandardVersionNegotiator;
public class TransportProtocolVersionNegotiator extends StandardVersionNegotiator {
public TransportProtocolVersionNegotiator(final int... supportedVersions) {
super(supportedVersions);
}
/**
* Returns a transaction protocol version for this transport protocol version.
* This method lets transport protocol to move forward independently from transaction protocol.
* @return a transaction protocol version
*/
public int getTransactionProtocolVersion() {
switch (getVersion()) {
case 1:
return 5;
default:
throw new RuntimeException("Transport protocol version " + getVersion()
+ " was not configured with any transaction protocol version.");
}
}
}

View File

@ -16,46 +16,6 @@
*/
package org.apache.nifi.remote.client.socket;
import java.io.BufferedReader;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.File;
import java.io.FileInputStream;
import java.io.FileOutputStream;
import java.io.IOException;
import java.io.InputStream;
import java.io.InputStreamReader;
import java.io.OutputStream;
import java.net.InetSocketAddress;
import java.net.URI;
import java.net.URISyntaxException;
import java.nio.channels.SocketChannel;
import java.nio.charset.StandardCharsets;
import java.security.cert.CertificateException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.Executors;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.ThreadFactory;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.regex.Pattern;
import javax.net.ssl.SSLContext;
import org.apache.nifi.events.EventReporter;
import org.apache.nifi.remote.Peer;
import org.apache.nifi.remote.PeerDescription;
@ -63,9 +23,10 @@ import org.apache.nifi.remote.PeerStatus;
import org.apache.nifi.remote.RemoteDestination;
import org.apache.nifi.remote.RemoteResourceInitiator;
import org.apache.nifi.remote.TransferDirection;
import org.apache.nifi.remote.client.PeerSelector;
import org.apache.nifi.remote.client.PeerStatusProvider;
import org.apache.nifi.remote.client.SiteInfoProvider;
import org.apache.nifi.remote.client.SiteToSiteClientConfig;
import org.apache.nifi.remote.cluster.ClusterNodeInformation;
import org.apache.nifi.remote.cluster.NodeInformation;
import org.apache.nifi.remote.codec.FlowFileCodec;
import org.apache.nifi.remote.exception.HandshakeException;
import org.apache.nifi.remote.exception.PortNotRunningException;
@ -76,98 +37,73 @@ import org.apache.nifi.remote.io.socket.ssl.SSLSocketChannel;
import org.apache.nifi.remote.io.socket.ssl.SSLSocketChannelCommunicationsSession;
import org.apache.nifi.remote.protocol.CommunicationsSession;
import org.apache.nifi.remote.protocol.socket.SocketClientProtocol;
import org.apache.nifi.remote.util.NiFiRestApiUtil;
import org.apache.nifi.remote.util.PeerStatusCache;
import org.apache.nifi.reporting.Severity;
import org.apache.nifi.stream.io.BufferedOutputStream;
import org.apache.nifi.web.api.dto.ControllerDTO;
import org.apache.nifi.web.api.dto.PortDTO;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.slf4j.helpers.MessageFormatter;
public class EndpointConnectionPool {
import javax.net.ssl.SSLContext;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.File;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.net.URI;
import java.nio.channels.SocketChannel;
import java.security.cert.CertificateException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Objects;
import java.util.Set;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.Executors;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.ThreadFactory;
import java.util.concurrent.TimeUnit;
public static final long PEER_REFRESH_PERIOD = 60000L;
public static final String CATEGORY = "Site-to-Site";
public static final long REMOTE_REFRESH_MILLIS = TimeUnit.MILLISECONDS.convert(10, TimeUnit.MINUTES);
import static org.apache.nifi.remote.util.EventReportUtil.error;
import static org.apache.nifi.remote.util.EventReportUtil.warn;
private static final long PEER_CACHE_MILLIS = TimeUnit.MILLISECONDS.convert(1, TimeUnit.MINUTES);
public class EndpointConnectionPool implements PeerStatusProvider {
private static final Logger logger = LoggerFactory.getLogger(EndpointConnectionPool.class);
private final ConcurrentMap<PeerDescription, BlockingQueue<EndpointConnection>> connectionQueueMap = new ConcurrentHashMap<>();
private final ConcurrentMap<PeerDescription, Long> peerTimeoutExpirations = new ConcurrentHashMap<>();
private final URI clusterUrl;
private final String apiUri;
private final AtomicLong peerIndex = new AtomicLong(0L);
private final Set<EndpointConnection> activeConnections = Collections.synchronizedSet(new HashSet<>());
private final ReentrantLock peerRefreshLock = new ReentrantLock();
private volatile List<PeerStatus> peerStatuses;
private volatile long peerRefreshTime = 0L;
private volatile PeerStatusCache peerStatusCache;
private final Set<EndpointConnection> activeConnections = Collections.synchronizedSet(new HashSet<EndpointConnection>());
private final File peersFile;
private final EventReporter eventReporter;
private final SSLContext sslContext;
private final ScheduledExecutorService taskExecutor;
private final int idleExpirationMillis;
private final RemoteDestination remoteDestination;
private final ReadWriteLock listeningPortRWLock = new ReentrantReadWriteLock();
private final Lock remoteInfoReadLock = listeningPortRWLock.readLock();
private final Lock remoteInfoWriteLock = listeningPortRWLock.writeLock();
private Integer siteToSitePort;
private Boolean siteToSiteSecure;
private long remoteRefreshTime;
private final Map<String, String> inputPortMap = new HashMap<>(); // map input port name to identifier
private final Map<String, String> outputPortMap = new HashMap<>(); // map output port name to identifier
private volatile int commsTimeout;
private volatile boolean shutdown = false;
public EndpointConnectionPool(final String clusterUrl, final RemoteDestination remoteDestination, final int commsTimeoutMillis,
final int idleExpirationMillis, final EventReporter eventReporter, final File persistenceFile) {
this(clusterUrl, remoteDestination, commsTimeoutMillis, idleExpirationMillis, null, eventReporter, persistenceFile);
}
private final SiteInfoProvider siteInfoProvider;
private final PeerSelector peerSelector;
public EndpointConnectionPool(final String clusterUrl, final RemoteDestination remoteDestination, final int commsTimeoutMillis, final int idleExpirationMillis,
final SSLContext sslContext, final EventReporter eventReporter, final File persistenceFile) {
public EndpointConnectionPool(final URI clusterUrl, final RemoteDestination remoteDestination, final int commsTimeoutMillis, final int idleExpirationMillis,
final SSLContext sslContext, final EventReporter eventReporter, final File persistenceFile, final SiteInfoProvider siteInfoProvider) {
Objects.requireNonNull(clusterUrl, "URL cannot be null");
Objects.requireNonNull(remoteDestination, "Remote Destination/Port Identifier cannot be null");
try {
this.clusterUrl = new URI(clusterUrl);
} catch (final URISyntaxException e) {
throw new IllegalArgumentException("Invalid Cluster URL: " + clusterUrl);
}
// Trim the trailing /
String uriPath = this.clusterUrl.getPath();
if (uriPath.endsWith("/")) {
uriPath = uriPath.substring(0, uriPath.length() - 1);
}
apiUri = this.clusterUrl.getScheme() + "://" + this.clusterUrl.getHost() + ":" + this.clusterUrl.getPort() + uriPath + "-api";
this.clusterUrl = clusterUrl;
this.remoteDestination = remoteDestination;
this.sslContext = sslContext;
this.peersFile = persistenceFile;
this.eventReporter = eventReporter;
this.commsTimeout = commsTimeoutMillis;
this.idleExpirationMillis = idleExpirationMillis;
Set<PeerStatus> recoveredStatuses;
if (persistenceFile != null && persistenceFile.exists()) {
try {
recoveredStatuses = recoverPersistedPeerStatuses(peersFile);
this.peerStatusCache = new PeerStatusCache(recoveredStatuses, peersFile.lastModified());
} catch (final IOException ioe) {
logger.warn("Failed to recover peer statuses from {} due to {}; will continue without loading information from file", persistenceFile, ioe);
}
} else {
peerStatusCache = null;
}
this.siteInfoProvider = siteInfoProvider;
peerSelector = new PeerSelector(this, persistenceFile);
peerSelector.setEventReporter(eventReporter);
// Initialize a scheduled executor and run some maintenance tasks in the background to kill off old, unused
// connections and keep our list of peers up-to-date.
@ -186,7 +122,7 @@ public class EndpointConnectionPool {
taskExecutor.scheduleWithFixedDelay(new Runnable() {
@Override
public void run() {
refreshPeers();
peerSelector.refreshPeers();
}
}, 0, 5, TimeUnit.SECONDS);
@ -198,38 +134,11 @@ public class EndpointConnectionPool {
}, 5, 5, TimeUnit.SECONDS);
}
void warn(final String msg, final Object... args) {
logger.warn(msg, args);
if (eventReporter != null) {
eventReporter.reportEvent(Severity.WARNING, "Site-to-Site", MessageFormatter.arrayFormat(msg, args).getMessage());
}
}
void warn(final String msg, final Throwable t) {
logger.warn(msg, t);
if (eventReporter != null) {
eventReporter.reportEvent(Severity.WARNING, "Site-to-Site", msg + ": " + t.toString());
}
}
void error(final String msg, final Object... args) {
logger.error(msg, args);
if (eventReporter != null) {
eventReporter.reportEvent(Severity.ERROR, "Site-to-Site", MessageFormatter.arrayFormat(msg, args).getMessage());
}
}
private String getPortIdentifier(final TransferDirection transferDirection) throws IOException {
if (remoteDestination.getIdentifier() != null) {
return remoteDestination.getIdentifier();
}
if (transferDirection == TransferDirection.RECEIVE) {
return getOutputPortIdentifier(remoteDestination.getName());
} else {
return getInputPortIdentifier(remoteDestination.getName());
}
return siteInfoProvider.getPortIdentifier(remoteDestination.getName(), transferDirection);
}
public EndpointConnection getEndpointConnection(final TransferDirection direction) throws IOException {
@ -250,7 +159,7 @@ public class EndpointConnectionPool {
do {
final List<EndpointConnection> addBack = new ArrayList<>();
logger.debug("{} getting next peer status", this);
final PeerStatus peerStatus = getNextPeerStatus(direction);
final PeerStatus peerStatus = peerSelector.getNextPeerStatus(direction);
logger.debug("{} next peer status = {}", this, peerStatus);
if (peerStatus == null) {
return null;
@ -296,7 +205,7 @@ public class EndpointConnectionPool {
logger.debug("{} Establishing site-to-site connection with {}", this, peerStatus);
commsSession = establishSiteToSiteConnection(peerStatus);
} catch (final IOException ioe) {
penalize(peerStatus.getPeerDescription(), penalizationMillis);
peerSelector.penalize(peerStatus.getPeerDescription(), penalizationMillis);
throw ioe;
}
@ -334,7 +243,7 @@ public class EndpointConnectionPool {
logger.warn("{} {} indicates that port {}'s destination is full; penalizing peer",
this, peer, config.getPortName() == null ? config.getPortIdentifier() : config.getPortName());
penalize(peer, penalizationMillis);
peerSelector.penalize(peer, penalizationMillis);
try {
peer.close();
} catch (final IOException ioe) {
@ -342,11 +251,11 @@ public class EndpointConnectionPool {
continue;
} else if (protocol.isPortInvalid()) {
penalize(peer, penalizationMillis);
peerSelector.penalize(peer, penalizationMillis);
cleanup(protocol, peer);
throw new PortNotRunningException(peer.toString() + " indicates that port " + portId + " is not running");
} else if (protocol.isPortUnknown()) {
penalize(peer, penalizationMillis);
peerSelector.penalize(peer, penalizationMillis);
cleanup(protocol, peer);
throw new UnknownPortException(peer.toString() + " indicates that port " + portId + " is not known");
}
@ -358,11 +267,11 @@ public class EndpointConnectionPool {
} catch (final PortNotRunningException | UnknownPortException e) {
throw e;
} catch (final Exception e) {
penalize(peer, penalizationMillis);
peerSelector.penalize(peer, penalizationMillis);
cleanup(protocol, peer);
final String message = String.format("%s failed to communicate with %s due to %s", this, peer == null ? clusterUrl : peer, e.toString());
error(message);
error(logger, eventReporter, message);
if (logger.isDebugEnabled()) {
logger.error("", e);
}
@ -427,27 +336,6 @@ public class EndpointConnectionPool {
}
}
private void penalize(final PeerDescription peerDescription, final long penalizationMillis) {
Long expiration = peerTimeoutExpirations.get(peerDescription);
if (expiration == null) {
expiration = Long.valueOf(0L);
}
final long newExpiration = Math.max(expiration, System.currentTimeMillis() + penalizationMillis);
peerTimeoutExpirations.put(peerDescription, Long.valueOf(newExpiration));
}
/**
* Updates internal state map to penalize a PeerStatus that points to the
* specified peer
*
* @param peer the peer
* @param penalizationMillis period of time to penalize a given peer
*/
public void penalize(final Peer peer, final long penalizationMillis) {
penalize(peer.getDescription(), penalizationMillis);
}
private void cleanup(final SocketClientProtocol protocol, final Peer peer) {
if (protocol != null && peer != null) {
try {
@ -470,113 +358,11 @@ public class EndpointConnectionPool {
}
}
private boolean isPeerRefreshNeeded(final List<PeerStatus> peerList) {
return (peerList == null || peerList.isEmpty() || System.currentTimeMillis() > peerRefreshTime + PEER_REFRESH_PERIOD);
}
private PeerStatus getNextPeerStatus(final TransferDirection direction) {
List<PeerStatus> peerList = peerStatuses;
if (isPeerRefreshNeeded(peerList)) {
peerRefreshLock.lock();
try {
// now that we have the lock, check again that we need to refresh (because another thread
// could have been refreshing while we were waiting for the lock).
peerList = peerStatuses;
if (isPeerRefreshNeeded(peerList)) {
try {
peerList = createPeerStatusList(direction);
} catch (final Exception e) {
final String message = String.format("%s Failed to update list of peers due to %s", this, e.toString());
warn(message);
if (logger.isDebugEnabled()) {
logger.warn("", e);
}
if (eventReporter != null) {
eventReporter.reportEvent(Severity.WARNING, CATEGORY, message);
}
}
this.peerStatuses = peerList;
peerRefreshTime = System.currentTimeMillis();
}
} finally {
peerRefreshLock.unlock();
}
}
if (peerList == null || peerList.isEmpty()) {
return null;
}
PeerStatus peerStatus;
for (int i = 0; i < peerList.size(); i++) {
final long idx = peerIndex.getAndIncrement();
final int listIndex = (int) (idx % peerList.size());
peerStatus = peerList.get(listIndex);
if (isPenalized(peerStatus)) {
logger.debug("{} {} is penalized; will not communicate with this peer", this, peerStatus);
} else {
return peerStatus;
}
}
logger.debug("{} All peers appear to be penalized; returning null", this);
return null;
}
private boolean isPenalized(final PeerStatus peerStatus) {
final Long expirationEnd = peerTimeoutExpirations.get(peerStatus.getPeerDescription());
return (expirationEnd != null && expirationEnd > System.currentTimeMillis());
}
private List<PeerStatus> createPeerStatusList(final TransferDirection direction) throws IOException {
Set<PeerStatus> statuses = getPeerStatuses();
if (statuses == null) {
refreshPeers();
statuses = getPeerStatuses();
if (statuses == null) {
logger.debug("{} found no peers to connect to", this);
return Collections.emptyList();
}
}
final ClusterNodeInformation clusterNodeInfo = new ClusterNodeInformation();
final List<NodeInformation> nodeInfos = new ArrayList<>();
for (final PeerStatus peerStatus : statuses) {
final PeerDescription description = peerStatus.getPeerDescription();
final NodeInformation nodeInfo = new NodeInformation(description.getHostname(), description.getPort(), 0, description.isSecure(), peerStatus.getFlowFileCount());
nodeInfos.add(nodeInfo);
}
clusterNodeInfo.setNodeInformation(nodeInfos);
return formulateDestinationList(clusterNodeInfo, direction);
}
private Set<PeerStatus> getPeerStatuses() {
final PeerStatusCache cache = this.peerStatusCache;
if (cache == null || cache.getStatuses() == null || cache.getStatuses().isEmpty()) {
return null;
}
if (cache.getTimestamp() + PEER_CACHE_MILLIS < System.currentTimeMillis()) {
final Set<PeerStatus> equalizedSet = new HashSet<>(cache.getStatuses().size());
for (final PeerStatus status : cache.getStatuses()) {
final PeerStatus equalizedStatus = new PeerStatus(status.getPeerDescription(), 1);
equalizedSet.add(equalizedStatus);
}
return equalizedSet;
}
return cache.getStatuses();
}
private Set<PeerStatus> fetchRemotePeerStatuses() throws IOException {
public Set<PeerStatus> fetchRemotePeerStatuses() throws IOException {
final String hostname = clusterUrl.getHost();
final Integer port = getSiteToSitePort();
final Integer port = siteInfoProvider.getSiteToSitePort();
if (port == null) {
throw new IOException("Remote instance of NiFi is not configured to allow site-to-site communications");
throw new IOException("Remote instance of NiFi is not configured to allow RAW Socket site-to-site communications");
}
final PeerDescription clusterPeerDescription = new PeerDescription(hostname, port, clusterUrl.toString().startsWith("https://"));
@ -604,13 +390,12 @@ public class EndpointConnectionPool {
}
final Set<PeerStatus> peerStatuses = clientProtocol.getPeerStatuses(peer);
persistPeerStatuses(peerStatuses);
try {
clientProtocol.shutdown(peer);
} catch (final IOException e) {
final String message = String.format("%s Failed to shutdown protocol when updating list of peers due to %s", this, e.toString());
warn(message);
warn(logger, eventReporter, message);
if (logger.isDebugEnabled()) {
logger.warn("", e);
}
@ -620,7 +405,7 @@ public class EndpointConnectionPool {
peer.close();
} catch (final IOException e) {
final String message = String.format("%s Failed to close resources when updating list of peers due to %s", this, e.toString());
warn(message);
warn(logger, eventReporter, message);
if (logger.isDebugEnabled()) {
logger.warn("", e);
}
@ -629,60 +414,13 @@ public class EndpointConnectionPool {
return peerStatuses;
}
private void persistPeerStatuses(final Set<PeerStatus> statuses) {
if (peersFile == null) {
return;
}
try (final OutputStream fos = new FileOutputStream(peersFile);
final OutputStream out = new BufferedOutputStream(fos)) {
for (final PeerStatus status : statuses) {
final PeerDescription description = status.getPeerDescription();
final String line = description.getHostname() + ":" + description.getPort() + ":" + description.isSecure() + "\n";
out.write(line.getBytes(StandardCharsets.UTF_8));
}
} catch (final IOException e) {
error("Failed to persist list of Peers due to {}; if restarted and peer's NCM is down, may be unable to transfer data until communications with NCM are restored", e.toString());
logger.error("", e);
}
}
private Set<PeerStatus> recoverPersistedPeerStatuses(final File file) throws IOException {
if (!file.exists()) {
return null;
}
final Set<PeerStatus> statuses = new HashSet<>();
try (final InputStream fis = new FileInputStream(file);
final BufferedReader reader = new BufferedReader(new InputStreamReader(fis))) {
String line;
while ((line = reader.readLine()) != null) {
final String[] splits = line.split(Pattern.quote(":"));
if (splits.length != 3) {
continue;
}
final String hostname = splits[0];
final int port = Integer.parseInt(splits[1]);
final boolean secure = Boolean.parseBoolean(splits[2]);
statuses.add(new PeerStatus(new PeerDescription(hostname, port, secure), 1));
}
}
return statuses;
}
private CommunicationsSession establishSiteToSiteConnection(final PeerStatus peerStatus) throws IOException {
final PeerDescription description = peerStatus.getPeerDescription();
return establishSiteToSiteConnection(description.getHostname(), description.getPort());
}
private CommunicationsSession establishSiteToSiteConnection(final String hostname, final int port) throws IOException {
final boolean siteToSiteSecure = isSecure();
final boolean siteToSiteSecure = siteInfoProvider.isSecure();
final String destinationUri = "nifi://" + hostname + ":" + port;
CommunicationsSession commsSession = null;
@ -724,66 +462,6 @@ public class EndpointConnectionPool {
return commsSession;
}
static List<PeerStatus> formulateDestinationList(final ClusterNodeInformation clusterNodeInfo, final TransferDirection direction) {
final Collection<NodeInformation> nodeInfoSet = clusterNodeInfo.getNodeInformation();
final int numDestinations = Math.max(128, nodeInfoSet.size());
final Map<NodeInformation, Integer> entryCountMap = new HashMap<>();
long totalFlowFileCount = 0L;
for (final NodeInformation nodeInfo : nodeInfoSet) {
totalFlowFileCount += nodeInfo.getTotalFlowFiles();
}
int totalEntries = 0;
for (final NodeInformation nodeInfo : nodeInfoSet) {
final int flowFileCount = nodeInfo.getTotalFlowFiles();
// don't allow any node to get more than 80% of the data
final double percentageOfFlowFiles = Math.min(0.8D, ((double) flowFileCount / (double) totalFlowFileCount));
final double relativeWeighting = (direction == TransferDirection.SEND) ? (1 - percentageOfFlowFiles) : percentageOfFlowFiles;
final int entries = Math.max(1, (int) (numDestinations * relativeWeighting));
entryCountMap.put(nodeInfo, Math.max(1, entries));
totalEntries += entries;
}
final List<PeerStatus> destinations = new ArrayList<>(totalEntries);
for (int i = 0; i < totalEntries; i++) {
destinations.add(null);
}
for (final Map.Entry<NodeInformation, Integer> entry : entryCountMap.entrySet()) {
final NodeInformation nodeInfo = entry.getKey();
final int numEntries = entry.getValue();
int skipIndex = numEntries;
for (int i = 0; i < numEntries; i++) {
int n = (skipIndex * i);
while (true) {
final int index = n % destinations.size();
PeerStatus status = destinations.get(index);
if (status == null) {
final PeerDescription description = new PeerDescription(nodeInfo.getSiteToSiteHostname(), nodeInfo.getSiteToSitePort(), nodeInfo.isSiteToSiteSecure());
status = new PeerStatus(description, nodeInfo.getTotalFlowFiles());
destinations.set(index, status);
break;
} else {
n++;
}
}
}
}
final StringBuilder distributionDescription = new StringBuilder();
distributionDescription.append("New Weighted Distribution of Nodes:");
for (final Map.Entry<NodeInformation, Integer> entry : entryCountMap.entrySet()) {
final double percentage = entry.getValue() * 100D / destinations.size();
distributionDescription.append("\n").append(entry.getKey()).append(" will receive ").append(percentage).append("% of data");
}
logger.info(distributionDescription.toString());
// Jumble the list of destinations.
return destinations;
}
private void cleanupExpiredSockets() {
for (final BlockingQueue<EndpointConnection> connectionQueue : connectionQueueMap.values()) {
final List<EndpointConnection> connections = new ArrayList<>();
@ -813,7 +491,7 @@ public class EndpointConnectionPool {
public void shutdown() {
shutdown = true;
taskExecutor.shutdown();
peerTimeoutExpirations.clear();
peerSelector.clear();
for (final EndpointConnection conn : activeConnections) {
conn.getPeer().getCommunicationsSession().interrupt();
@ -832,138 +510,11 @@ public class EndpointConnectionPool {
cleanup(connection.getSocketClientProtocol(), connection.getPeer());
}
private void refreshPeers() {
final PeerStatusCache existingCache = peerStatusCache;
if (existingCache != null && (existingCache.getTimestamp() + PEER_CACHE_MILLIS > System.currentTimeMillis())) {
return;
}
try {
final Set<PeerStatus> statuses = fetchRemotePeerStatuses();
peerStatusCache = new PeerStatusCache(statuses);
logger.info("{} Successfully refreshed Peer Status; remote instance consists of {} peers", this, statuses.size());
} catch (Exception e) {
warn("{} Unable to refresh Remote Group's peers due to {}", this, e);
if (logger.isDebugEnabled()) {
logger.warn("", e);
}
}
}
public String getInputPortIdentifier(final String portName) throws IOException {
return getPortIdentifier(portName, inputPortMap);
}
public String getOutputPortIdentifier(final String portName) throws IOException {
return getPortIdentifier(portName, outputPortMap);
}
private String getPortIdentifier(final String portName, final Map<String, String> portMap) throws IOException {
String identifier;
remoteInfoReadLock.lock();
try {
identifier = portMap.get(portName);
} finally {
remoteInfoReadLock.unlock();
}
if (identifier != null) {
return identifier;
}
refreshRemoteInfo();
remoteInfoReadLock.lock();
try {
return portMap.get(portName);
} finally {
remoteInfoReadLock.unlock();
}
}
private ControllerDTO refreshRemoteInfo() throws IOException {
final boolean webInterfaceSecure = clusterUrl.toString().startsWith("https");
final NiFiRestApiUtil utils = new NiFiRestApiUtil(webInterfaceSecure ? sslContext : null);
final ControllerDTO controller = utils.getController(apiUri + "/controller", commsTimeout);
remoteInfoWriteLock.lock();
try {
this.siteToSitePort = controller.getRemoteSiteListeningPort();
this.siteToSiteSecure = controller.isSiteToSiteSecure();
inputPortMap.clear();
for (final PortDTO inputPort : controller.getInputPorts()) {
inputPortMap.put(inputPort.getName(), inputPort.getId());
}
outputPortMap.clear();
for (final PortDTO outputPort : controller.getOutputPorts()) {
outputPortMap.put(outputPort.getName(), outputPort.getId());
}
this.remoteRefreshTime = System.currentTimeMillis();
} finally {
remoteInfoWriteLock.unlock();
}
return controller;
}
/**
* @return the port that the remote instance is listening on for
* site-to-site communication, or <code>null</code> if the remote instance
* is not configured to allow site-to-site communications.
*
* @throws IOException if unable to communicate with the remote instance
*/
private Integer getSiteToSitePort() throws IOException {
Integer listeningPort;
remoteInfoReadLock.lock();
try {
listeningPort = this.siteToSitePort;
if (listeningPort != null && this.remoteRefreshTime > System.currentTimeMillis() - REMOTE_REFRESH_MILLIS) {
return listeningPort;
}
} finally {
remoteInfoReadLock.unlock();
}
final ControllerDTO controller = refreshRemoteInfo();
listeningPort = controller.getRemoteSiteListeningPort();
return listeningPort;
}
@Override
public String toString() {
return "EndpointConnectionPool[Cluster URL=" + clusterUrl + "]";
}
/**
* @return {@code true} if the remote instance is configured for secure
* site-to-site communications, {@code false} otherwise
* @throws IOException if unable to check if secure
*/
public boolean isSecure() throws IOException {
remoteInfoReadLock.lock();
try {
final Boolean secure = this.siteToSiteSecure;
if (secure != null && this.remoteRefreshTime > System.currentTimeMillis() - REMOTE_REFRESH_MILLIS) {
return secure;
}
} finally {
remoteInfoReadLock.unlock();
}
final ControllerDTO controller = refreshRemoteInfo();
final Boolean isSecure = controller.isSiteToSiteSecure();
if (isSecure == null) {
throw new IOException("Remote NiFi instance " + clusterUrl + " is not currently configured to accept site-to-site connections");
}
return isSecure;
}
private class IdEnrichedRemoteDestination implements RemoteDestination {
private final RemoteDestination original;
@ -994,4 +545,6 @@ public class EndpointConnectionPool {
return original.isUseCompression();
}
}
}

View File

@ -16,27 +16,26 @@
*/
package org.apache.nifi.remote.client.socket;
import java.io.IOException;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import org.apache.nifi.remote.Communicant;
import org.apache.nifi.remote.RemoteDestination;
import org.apache.nifi.remote.Transaction;
import org.apache.nifi.remote.TransactionCompletion;
import org.apache.nifi.remote.TransferDirection;
import org.apache.nifi.remote.client.SiteToSiteClient;
import org.apache.nifi.remote.client.AbstractSiteToSiteClient;
import org.apache.nifi.remote.client.SiteToSiteClientConfig;
import org.apache.nifi.remote.protocol.DataPacket;
import org.apache.nifi.util.ObjectHolder;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class SocketClient implements SiteToSiteClient {
import java.io.IOException;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
public class SocketClient extends AbstractSiteToSiteClient {
private static final Logger logger = LoggerFactory.getLogger(SocketClient.class);
private final SiteToSiteClientConfig config;
private final EndpointConnectionPool pool;
private final boolean compress;
private final String portName;
@ -45,27 +44,26 @@ public class SocketClient implements SiteToSiteClient {
private volatile boolean closed = false;
public SocketClient(final SiteToSiteClientConfig config) {
pool = new EndpointConnectionPool(config.getUrl(),
createRemoteDestination(config.getPortIdentifier(), config.getPortName()),
(int) config.getTimeout(TimeUnit.MILLISECONDS),
(int) config.getIdleConnectionExpiration(TimeUnit.MILLISECONDS),
config.getSslContext(), config.getEventReporter(), config.getPeerPersistenceFile());
super(config);
final int commsTimeout = (int) config.getTimeout(TimeUnit.MILLISECONDS);
pool = new EndpointConnectionPool(clusterUrl,
createRemoteDestination(config.getPortIdentifier(), config.getPortName()),
commsTimeout,
(int) config.getIdleConnectionExpiration(TimeUnit.MILLISECONDS),
config.getSslContext(), config.getEventReporter(), config.getPeerPersistenceFile(),
siteInfoProvider
);
this.config = config;
this.compress = config.isUseCompression();
this.portIdentifier = config.getPortIdentifier();
this.portName = config.getPortName();
this.penalizationNanos = config.getPenalizationPeriod(TimeUnit.NANOSECONDS);
}
@Override
public SiteToSiteClientConfig getConfig() {
return config;
}
@Override
public boolean isSecure() throws IOException {
return pool.isSecure();
return siteInfoProvider.isSecure();
}
private String getPortIdentifier(final TransferDirection direction) throws IOException {
@ -76,9 +74,9 @@ public class SocketClient implements SiteToSiteClient {
final String portId;
if (direction == TransferDirection.SEND) {
portId = pool.getInputPortIdentifier(this.portName);
portId = siteInfoProvider.getInputPortIdentifier(this.portName);
} else {
portId = pool.getOutputPortIdentifier(this.portName);
portId = siteInfoProvider.getOutputPortIdentifier(this.portName);
}
if (portId == null) {
@ -142,7 +140,7 @@ public class SocketClient implements SiteToSiteClient {
// Wrap the transaction in a new one that will return the EndpointConnectionState back to the pool whenever
// the transaction is either completed or canceled.
final ObjectHolder<EndpointConnection> connectionStateRef = new ObjectHolder<>(connectionState);
final AtomicReference<EndpointConnection> connectionStateRef = new AtomicReference<>(connectionState);
return new Transaction() {
@Override
public void confirm() throws IOException {

View File

@ -20,6 +20,8 @@ public class AdaptedNodeInformation {
private String hostname;
private Integer siteToSitePort;
private Integer siteToSiteHttpApiPort;
private int apiPort;
private boolean isSiteToSiteSecure;
private int totalFlowFiles;
@ -63,4 +65,13 @@ public class AdaptedNodeInformation {
public void setTotalFlowFiles(int totalFlowFiles) {
this.totalFlowFiles = totalFlowFiles;
}
public Integer getSiteToSiteHttpApiPort() {
return siteToSiteHttpApiPort;
}
public void setSiteToSiteHttpApiPort(Integer siteToSiteHttpApiPort) {
this.siteToSiteHttpApiPort = siteToSiteHttpApiPort;
}
}

View File

@ -20,14 +20,16 @@ public class NodeInformation {
private final String siteToSiteHostname;
private final Integer siteToSitePort;
private final Integer siteToSiteHttpApiPort;
private final int apiPort;
private final boolean isSiteToSiteSecure;
private final int totalFlowFiles;
public NodeInformation(final String siteToSiteHostname, final Integer siteToSitePort, final int apiPort,
final boolean isSiteToSiteSecure, final int totalFlowFiles) {
public NodeInformation(final String siteToSiteHostname, final Integer siteToSitePort, final Integer siteToSiteHttpApiPort,
final int apiPort, final boolean isSiteToSiteSecure, final int totalFlowFiles) {
this.siteToSiteHostname = siteToSiteHostname;
this.siteToSitePort = siteToSitePort;
this.siteToSiteHttpApiPort = siteToSiteHttpApiPort;
this.apiPort = apiPort;
this.isSiteToSiteSecure = isSiteToSiteSecure;
this.totalFlowFiles = totalFlowFiles;
@ -45,6 +47,10 @@ public class NodeInformation {
return siteToSitePort;
}
public Integer getSiteToSiteHttpApiPort() {
return siteToSiteHttpApiPort;
}
public boolean isSiteToSiteSecure() {
return isSiteToSiteSecure;
}
@ -77,6 +83,16 @@ public class NodeInformation {
} else if (siteToSitePort != null && siteToSitePort.intValue() != other.siteToSitePort.intValue()) {
return false;
}
if (siteToSiteHttpApiPort == null && other.siteToSiteHttpApiPort != null) {
return false;
}
if (siteToSiteHttpApiPort != null && other.siteToSiteHttpApiPort == null) {
return false;
} else if (siteToSiteHttpApiPort != null && siteToSiteHttpApiPort.intValue() != other.siteToSiteHttpApiPort.intValue()) {
return false;
}
if (apiPort != other.apiPort) {
return false;
}

View File

@ -24,7 +24,8 @@ public class NodeInformationAdapter extends XmlAdapter<AdaptedNodeInformation, N
@Override
public NodeInformation unmarshal(final AdaptedNodeInformation adapted) throws Exception {
return new NodeInformation(adapted.getHostname(), adapted.getSiteToSitePort(), adapted.getApiPort(), adapted.isSiteToSiteSecure(), adapted.getTotalFlowFiles());
return new NodeInformation(adapted.getHostname(), adapted.getSiteToSitePort(), adapted.getSiteToSiteHttpApiPort(),
adapted.getApiPort(), adapted.isSiteToSiteSecure(), adapted.getTotalFlowFiles());
}
@Override
@ -32,6 +33,7 @@ public class NodeInformationAdapter extends XmlAdapter<AdaptedNodeInformation, N
final AdaptedNodeInformation adapted = new AdaptedNodeInformation();
adapted.setHostname(nodeInformation.getSiteToSiteHostname());
adapted.setSiteToSitePort(nodeInformation.getSiteToSitePort());
adapted.setSiteToSiteHttpApiPort(nodeInformation.getSiteToSiteHttpApiPort());
adapted.setApiPort(nodeInformation.getAPIPort());
adapted.setSiteToSiteSecure(nodeInformation.isSiteToSiteSecure());
adapted.setTotalFlowFiles(nodeInformation.getTotalFlowFiles());

View File

@ -16,6 +16,8 @@
*/
package org.apache.nifi.remote.exception;
import org.apache.nifi.remote.protocol.ResponseCode;
import java.io.IOException;
/**
@ -28,11 +30,24 @@ public class HandshakeException extends IOException {
private static final long serialVersionUID = 178192341908726L;
private final ResponseCode responseCode;
public HandshakeException(final String message) {
super(message);
this.responseCode = null;
}
public HandshakeException(final Throwable cause) {
super(cause);
this.responseCode = null;
}
public HandshakeException(final ResponseCode responseCode, final String message) {
super(message);
this.responseCode = responseCode;
}
public ResponseCode getResponseCode() {
return responseCode;
}
}

View File

@ -0,0 +1,97 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT 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.remote.io.http;
import org.apache.nifi.remote.AbstractCommunicationsSession;
import org.apache.nifi.remote.protocol.CommunicationsInput;
import org.apache.nifi.remote.protocol.CommunicationsOutput;
import java.io.IOException;
public class HttpCommunicationsSession extends AbstractCommunicationsSession {
protected int timeout = 30000;
protected final HttpInput input;
protected final HttpOutput output;
protected String checksum;
public HttpCommunicationsSession(){
super(null);
this.input = new HttpInput();
this.output = new HttpOutput();
}
@Override
public void setTimeout(final int millis) throws IOException {
this.timeout = millis;
}
@Override
public int getTimeout() throws IOException {
return timeout;
}
@Override
public CommunicationsInput getInput() {
return input;
}
@Override
public CommunicationsOutput getOutput() {
return output;
}
@Override
public boolean isDataAvailable() {
return false;
}
@Override
public long getBytesWritten() {
return output.getBytesWritten();
}
@Override
public long getBytesRead() {
return input.getBytesRead();
}
@Override
public void interrupt() {
}
@Override
public boolean isClosed() {
return false;
}
@Override
public void close() throws IOException {
}
public String getChecksum() {
return checksum;
}
public void setChecksum(String checksum) {
this.checksum = checksum;
}
}

View File

@ -0,0 +1,58 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT 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.remote.io.http;
import org.apache.nifi.remote.protocol.CommunicationsInput;
import org.apache.nifi.stream.io.ByteCountingInputStream;
import java.io.IOException;
import java.io.InputStream;
public class HttpInput implements CommunicationsInput {
private ByteCountingInputStream countingIn;
@Override
public InputStream getInputStream() throws IOException {
return countingIn;
}
@Override
public long getBytesRead() {
if (countingIn != null) {
return countingIn.getBytesRead();
}
return 0L;
}
@Override
public void consume() throws IOException {
if (countingIn == null) {
return;
}
final byte[] b = new byte[4096];
int bytesRead;
do {
bytesRead = countingIn.read(b);
} while (bytesRead > 0);
}
public void setInputStream(InputStream inputStream) {
this.countingIn = new ByteCountingInputStream(inputStream);
}
}

View File

@ -0,0 +1,45 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT 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.remote.io.http;
import org.apache.nifi.remote.protocol.CommunicationsOutput;
import org.apache.nifi.stream.io.ByteCountingOutputStream;
import java.io.IOException;
import java.io.OutputStream;
public class HttpOutput implements CommunicationsOutput {
private ByteCountingOutputStream countingOut;
@Override
public OutputStream getOutputStream() throws IOException {
return countingOut;
}
@Override
public long getBytesWritten() {
if (countingOut != null) {
return countingOut.getBytesWritten();
}
return 0L;
}
public void setOutputStream(OutputStream outputStream) {
this.countingOut = new ByteCountingOutputStream(outputStream);
}
}

View File

@ -0,0 +1,72 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.remote.io.http;
import org.apache.nifi.remote.Transaction;
import org.apache.nifi.remote.protocol.HandshakeProperty;
import org.apache.nifi.remote.protocol.ResponseCode;
import java.io.InputStream;
import java.io.OutputStream;
import java.util.HashMap;
import java.util.Map;
public class HttpServerCommunicationsSession extends HttpCommunicationsSession {
private final Map<String, String> handshakeParams = new HashMap<>();
private final String transactionId;
private Transaction.TransactionState status = Transaction.TransactionState.TRANSACTION_STARTED;
private ResponseCode responseCode;
public HttpServerCommunicationsSession(InputStream inputStream, OutputStream outputStream, String transactionId){
super();
input.setInputStream(inputStream);
output.setOutputStream(outputStream);
this.transactionId = transactionId;
}
// This status is only needed by HttpFlowFileServerProtocol, HttpClientTransaction has its own status.
// Because multiple HttpFlowFileServerProtocol instances have to carry on a single transaction
// throughout multiple HTTP requests, status has to be embedded here.
public Transaction.TransactionState getStatus() {
return status;
}
public void setStatus(Transaction.TransactionState status) {
this.status = status;
}
public String getTransactionId() {
return transactionId;
}
public ResponseCode getResponseCode() {
return responseCode;
}
public void setResponseCode(ResponseCode responseCode) {
this.responseCode = responseCode;
}
public void putHandshakeParam(HandshakeProperty key, String value) {
handshakeParams.put(key.name(), value);
}
public Map<String, String> getHandshakeParams() {
return handshakeParams;
}
}

View File

@ -16,11 +16,6 @@
*/
package org.apache.nifi.remote.protocol;
import java.io.IOException;
import java.util.Set;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.remote.Peer;
import org.apache.nifi.remote.PeerStatus;
import org.apache.nifi.remote.Transaction;
@ -32,6 +27,9 @@ import org.apache.nifi.remote.exception.PortNotRunningException;
import org.apache.nifi.remote.exception.ProtocolException;
import org.apache.nifi.remote.exception.UnknownPortException;
import java.io.IOException;
import java.util.Set;
public interface ClientProtocol extends VersionedRemoteResource {
void handshake(Peer peer) throws IOException, HandshakeException, UnknownPortException, PortNotRunningException;
@ -40,14 +38,8 @@ public interface ClientProtocol extends VersionedRemoteResource {
FlowFileCodec negotiateCodec(Peer peer) throws IOException, ProtocolException;
int receiveFlowFiles(Peer peer, ProcessContext context, ProcessSession session, FlowFileCodec codec) throws IOException, ProtocolException;
int transferFlowFiles(Peer peer, ProcessContext context, ProcessSession session, FlowFileCodec codec) throws IOException, ProtocolException;
void shutdown(Peer peer) throws IOException, ProtocolException;
boolean isReadyForFileTransfer();
Transaction startTransaction(Peer peer, FlowFileCodec codec, TransferDirection direction) throws IOException;
/**

View File

@ -14,7 +14,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.remote.protocol.socket;
package org.apache.nifi.remote.protocol;
/**
* Enumeration of Properties that can be used for the Site-to-Site Socket

View File

@ -14,7 +14,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.remote.protocol.socket;
package org.apache.nifi.remote.protocol;
import java.io.DataInputStream;
import java.io.IOException;

View File

@ -14,15 +14,15 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.remote.protocol.socket;
package org.apache.nifi.remote.protocol;
import org.apache.nifi.remote.exception.ProtocolException;
import java.io.DataOutputStream;
import java.io.EOFException;
import java.io.IOException;
import java.io.InputStream;
import org.apache.nifi.remote.exception.ProtocolException;
public enum ResponseCode {
RESERVED(0, "Reserved for Future Use", false), // This will likely be used if we ever need to expand the length of
@ -142,6 +142,10 @@ public enum ResponseCode {
public static ResponseCode fromSequence(final byte[] value) {
final int code = value[3] & 0xFF;
return fromCode(code);
}
public static ResponseCode fromCode(final int code) {
final ResponseCode responseCode = codeArray[code];
return (responseCode == null) ? UNRECOGNIZED_RESPONSE_CODE : responseCode;
}

View File

@ -0,0 +1,22 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT 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.remote.protocol;
public enum SiteToSiteTransportProtocol {
RAW,
HTTP
}

View File

@ -0,0 +1,187 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT 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.remote.protocol.http;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.events.EventReporter;
import org.apache.nifi.remote.AbstractTransaction;
import org.apache.nifi.remote.Peer;
import org.apache.nifi.remote.TransferDirection;
import org.apache.nifi.remote.codec.StandardFlowFileCodec;
import org.apache.nifi.remote.io.http.HttpCommunicationsSession;
import org.apache.nifi.remote.protocol.Response;
import org.apache.nifi.remote.protocol.ResponseCode;
import org.apache.nifi.remote.util.SiteToSiteRestApiClient;
import org.apache.nifi.stream.io.ByteArrayInputStream;
import org.apache.nifi.stream.io.ByteArrayOutputStream;
import org.apache.nifi.web.api.entity.TransactionResultEntity;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.IOException;
public class HttpClientTransaction extends AbstractTransaction {
private SiteToSiteRestApiClient apiClient;
private String transactionUrl;
public HttpClientTransaction(final int protocolVersion, final Peer peer, TransferDirection direction,
final boolean useCompression, final String portId, int penaltyMillis, EventReporter eventReporter) throws IOException {
super(peer, direction, useCompression, new StandardFlowFileCodec(), eventReporter, protocolVersion, penaltyMillis, portId);
}
public void initialize(SiteToSiteRestApiClient apiUtil, String transactionUrl) throws IOException {
this.transactionUrl = transactionUrl;
this.apiClient = apiUtil;
if(TransferDirection.RECEIVE.equals(direction)){
dataAvailable = apiUtil.openConnectionForReceive(transactionUrl, peer.getCommunicationsSession());
} else {
apiUtil.openConnectionForSend(transactionUrl, peer.getCommunicationsSession());
}
}
@Override
protected Response readTransactionResponse() throws IOException {
HttpCommunicationsSession commSession = (HttpCommunicationsSession) peer.getCommunicationsSession();
ByteArrayOutputStream bos = new ByteArrayOutputStream();
DataOutputStream dos = new DataOutputStream(bos);
if(TransferDirection.RECEIVE.equals(direction)){
switch (state){
case TRANSACTION_STARTED:
case DATA_EXCHANGED:
logger.debug("{} {} readTransactionResponse. checksum={}", this, peer, commSession.getChecksum());
if(StringUtils.isEmpty(commSession.getChecksum())){
// We don't know if there's more data to receive, so just continue it.
ResponseCode.CONTINUE_TRANSACTION.writeResponse(dos);
} else {
// We got a checksum to send to server.
if (TransactionState.TRANSACTION_STARTED.equals(state)) {
logger.debug("{} {} There's no transaction to confirm.", this, peer);
ResponseCode.CONFIRM_TRANSACTION.writeResponse(dos, "");
} else {
TransactionResultEntity transactionResult
= apiClient.commitReceivingFlowFiles(transactionUrl, ResponseCode.CONFIRM_TRANSACTION, commSession.getChecksum());
ResponseCode responseCode = ResponseCode.fromCode(transactionResult.getResponseCode());
if(responseCode.containsMessage()){
String message = transactionResult.getMessage();
responseCode.writeResponse(dos, message == null ? "" : message);
} else {
responseCode.writeResponse(dos);
}
}
}
break;
}
} else {
switch (state){
case DATA_EXCHANGED:
// Some flow files have been sent via stream, finish transferring.
apiClient.finishTransferFlowFiles(commSession);
ResponseCode.CONFIRM_TRANSACTION.writeResponse(dos, commSession.getChecksum());
break;
case TRANSACTION_CONFIRMED:
TransactionResultEntity resultEntity = apiClient.commitTransferFlowFiles(transactionUrl, ResponseCode.CONFIRM_TRANSACTION);
ResponseCode responseCode = ResponseCode.fromCode(resultEntity.getResponseCode());
if(responseCode.containsMessage()){
responseCode.writeResponse(dos, resultEntity.getMessage());
} else {
responseCode.writeResponse(dos);
}
break;
}
}
ByteArrayInputStream bis = new ByteArrayInputStream(bos.toByteArray());
return Response.read(new DataInputStream(bis));
}
@Override
protected void writeTransactionResponse(ResponseCode response, String explanation) throws IOException {
HttpCommunicationsSession commSession = (HttpCommunicationsSession) peer.getCommunicationsSession();
if(TransferDirection.RECEIVE.equals(direction)){
switch (response) {
case CONFIRM_TRANSACTION:
logger.debug("{} Confirming transaction. checksum={}", this, explanation);
commSession.setChecksum(explanation);
break;
case TRANSACTION_FINISHED:
logger.debug("{} Finishing transaction.", this);
break;
case CANCEL_TRANSACTION:
logger.debug("{} Canceling transaction. explanation={}", this, explanation);
TransactionResultEntity resultEntity = apiClient.commitReceivingFlowFiles(transactionUrl, ResponseCode.CANCEL_TRANSACTION, null);
ResponseCode cancelResponse = ResponseCode.fromCode(resultEntity.getResponseCode());
switch (cancelResponse) {
case CANCEL_TRANSACTION:
logger.debug("{} CANCEL_TRANSACTION, The transaction is canceled on server properly.", this);
break;
default:
logger.warn("{} CANCEL_TRANSACTION, Expected the transaction is canceled on server, but received {}.", this, cancelResponse);
break;
}
break;
}
} else {
switch (response) {
case FINISH_TRANSACTION:
// The actual HTTP request will be sent in readTransactionResponse.
logger.debug("{} Finished sending flow files.", this);
break;
case BAD_CHECKSUM: {
TransactionResultEntity resultEntity = apiClient.commitTransferFlowFiles(transactionUrl, ResponseCode.BAD_CHECKSUM);
ResponseCode badChecksumCancelResponse = ResponseCode.fromCode(resultEntity.getResponseCode());
switch (badChecksumCancelResponse) {
case CANCEL_TRANSACTION:
logger.debug("{} BAD_CHECKSUM, The transaction is canceled on server properly.", this);
break;
default:
logger.warn("{} BAD_CHECKSUM, Expected the transaction is canceled on server, but received {}.", this, badChecksumCancelResponse);
break;
}
}
break;
case CONFIRM_TRANSACTION:
// The actual HTTP request will be sent in readTransactionResponse.
logger.debug("{} Transaction is confirmed.", this);
break;
case CANCEL_TRANSACTION: {
logger.debug("{} Canceling transaction.", this);
TransactionResultEntity resultEntity = apiClient.commitTransferFlowFiles(transactionUrl, ResponseCode.CANCEL_TRANSACTION);
ResponseCode cancelResponse = ResponseCode.fromCode(resultEntity.getResponseCode());
switch (cancelResponse) {
case CANCEL_TRANSACTION:
logger.debug("{} CANCEL_TRANSACTION, The transaction is canceled on server properly.", this);
break;
default:
logger.warn("{} CANCEL_TRANSACTION, Expected the transaction is canceled on server, but received {}.", this, cancelResponse);
break;
}
}
break;
}
}
}
@Override
protected void close() throws IOException {
if (apiClient != null) {
apiClient.close();
}
}
}

View File

@ -0,0 +1,35 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT 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.remote.protocol.http;
public class HttpHeaders {
public static final String LOCATION_HEADER_NAME = "Location";
public static final String LOCATION_URI_INTENT_NAME = "x-location-uri-intent";
public static final String LOCATION_URI_INTENT_VALUE = "transaction-url";
public static final String ACCEPT_ENCODING = "Accept-Encoding";
public static final String CONTENT_ENCODING = "Content-Encoding";
public static final String PROTOCOL_VERSION = "x-nifi-site-to-site-protocol-version";
public static final String SERVER_SIDE_TRANSACTION_TTL = "x-nifi-site-to-site-server-transaction-ttl";
public static final String HANDSHAKE_PROPERTY_USE_COMPRESSION = "x-nifi-site-to-site-use-compression";
public static final String HANDSHAKE_PROPERTY_REQUEST_EXPIRATION = "x-nifi-site-to-site-request-expiration";
public static final String HANDSHAKE_PROPERTY_BATCH_COUNT = "x-nifi-site-to-site-batch-count";
public static final String HANDSHAKE_PROPERTY_BATCH_SIZE = "x-nifi-site-to-site-batch-size";
public static final String HANDSHAKE_PROPERTY_BATCH_DURATION = "x-nifi-site-to-site-batch-duration";
}

View File

@ -0,0 +1,59 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT 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.remote.protocol.http;
import org.apache.commons.lang3.StringUtils;
import org.apache.http.HttpHost;
public class HttpProxy {
private final String host;
private final Integer port;
private final String username;
private final String password;
public HttpProxy(final String host, final Integer port, final String username, final String password) {
this.host = host;
this.port = port;
this.username = username;
this.password = password;
}
public String getHost() {
return host;
}
public Integer getPort() {
return port;
}
public String getUsername() {
return username;
}
public String getPassword() {
return password;
}
public HttpHost getHttpHost() {
if (StringUtils.isEmpty(host)) {
return null;
}
return new HttpHost(host, port == null ? 80 : port);
}
}

View File

@ -16,24 +16,7 @@
*/
package org.apache.nifi.remote.protocol.socket;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.IOException;
import java.io.InputStream;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import java.util.UUID;
import java.util.concurrent.TimeUnit;
import org.apache.nifi.events.EventReporter;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.flowfile.attributes.CoreAttributes;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.io.InputStreamCallback;
import org.apache.nifi.remote.Peer;
import org.apache.nifi.remote.PeerDescription;
import org.apache.nifi.remote.PeerStatus;
@ -49,14 +32,23 @@ import org.apache.nifi.remote.exception.HandshakeException;
import org.apache.nifi.remote.exception.ProtocolException;
import org.apache.nifi.remote.protocol.ClientProtocol;
import org.apache.nifi.remote.protocol.CommunicationsSession;
import org.apache.nifi.remote.protocol.DataPacket;
import org.apache.nifi.remote.protocol.HandshakeProperty;
import org.apache.nifi.remote.protocol.RequestType;
import org.apache.nifi.remote.util.StandardDataPacket;
import org.apache.nifi.util.FormatUtils;
import org.apache.nifi.util.StopWatch;
import org.apache.nifi.remote.protocol.Response;
import org.apache.nifi.remote.protocol.ResponseCode;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.IOException;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import java.util.UUID;
import java.util.concurrent.TimeUnit;
public class SocketClientProtocol implements ClientProtocol {
private final VersionNegotiator versionNegotiator = new StandardVersionNegotiator(5, 4, 3, 2, 1);
@ -79,8 +71,6 @@ public class SocketClientProtocol implements ClientProtocol {
private long batchMillis;
private EventReporter eventReporter;
private static final long BATCH_SEND_NANOS = TimeUnit.SECONDS.toNanos(5L); // send batches of up to 5 seconds
public SocketClientProtocol() {
}
@ -192,11 +182,6 @@ public class SocketClientProtocol implements ClientProtocol {
handshakeComplete = true;
}
@Override
public boolean isReadyForFileTransfer() {
return readyForFileTransfer;
}
@Override
public boolean isPortInvalid() {
if (!handshakeComplete) {
@ -285,138 +270,6 @@ public class SocketClientProtocol implements ClientProtocol {
direction, useCompression, (int) destination.getYieldPeriod(TimeUnit.MILLISECONDS), eventReporter);
}
@Override
public int receiveFlowFiles(final Peer peer, final ProcessContext context, final ProcessSession session, final FlowFileCodec codec) throws IOException, ProtocolException {
final String userDn = peer.getCommunicationsSession().getUserDn();
final Transaction transaction = startTransaction(peer, codec, TransferDirection.RECEIVE);
final StopWatch stopWatch = new StopWatch(true);
final Set<FlowFile> flowFilesReceived = new HashSet<>();
long bytesReceived = 0L;
while (true) {
final long start = System.nanoTime();
final DataPacket dataPacket = transaction.receive();
if (dataPacket == null) {
if (flowFilesReceived.isEmpty()) {
peer.penalize(destination.getIdentifier(), destination.getYieldPeriod(TimeUnit.MILLISECONDS));
}
break;
}
FlowFile flowFile = session.create();
flowFile = session.putAllAttributes(flowFile, dataPacket.getAttributes());
flowFile = session.importFrom(dataPacket.getData(), flowFile);
final long receiveNanos = System.nanoTime() - start;
String sourceFlowFileIdentifier = dataPacket.getAttributes().get(CoreAttributes.UUID.key());
if (sourceFlowFileIdentifier == null) {
sourceFlowFileIdentifier = "<Unknown Identifier>";
}
final String transitUri = (transitUriPrefix == null) ? peer.getUrl() : transitUriPrefix + sourceFlowFileIdentifier;
session.getProvenanceReporter().receive(flowFile, transitUri, "urn:nifi:" + sourceFlowFileIdentifier, "Remote Host="
+ peer.getHost() + ", Remote DN=" + userDn, TimeUnit.NANOSECONDS.toMillis(receiveNanos));
session.transfer(flowFile, Relationship.ANONYMOUS);
bytesReceived += dataPacket.getSize();
}
// Confirm that what we received was the correct data.
transaction.confirm();
// Commit the session so that we have persisted the data
session.commit();
transaction.complete();
logger.debug("{} Sending TRANSACTION_FINISHED_BUT_DESTINATION_FULL to {}", this, peer);
if (!flowFilesReceived.isEmpty()) {
stopWatch.stop();
final String flowFileDescription = flowFilesReceived.size() < 20 ? flowFilesReceived.toString() : flowFilesReceived.size() + " FlowFiles";
final String uploadDataRate = stopWatch.calculateDataRate(bytesReceived);
final long uploadMillis = stopWatch.getDuration(TimeUnit.MILLISECONDS);
final String dataSize = FormatUtils.formatDataSize(bytesReceived);
logger.info("{} Successfully receveied {} ({}) from {} in {} milliseconds at a rate of {}", new Object[]{
this, flowFileDescription, dataSize, peer, uploadMillis, uploadDataRate});
}
return flowFilesReceived.size();
}
@Override
public int transferFlowFiles(final Peer peer, final ProcessContext context, final ProcessSession session, final FlowFileCodec codec) throws IOException, ProtocolException {
FlowFile flowFile = session.get();
if (flowFile == null) {
return 0;
}
try {
final String userDn = peer.getCommunicationsSession().getUserDn();
final long startSendingNanos = System.nanoTime();
final StopWatch stopWatch = new StopWatch(true);
long bytesSent = 0L;
final Transaction transaction = startTransaction(peer, codec, TransferDirection.SEND);
final Set<FlowFile> flowFilesSent = new HashSet<>();
boolean continueTransaction = true;
while (continueTransaction) {
final long startNanos = System.nanoTime();
// call codec.encode within a session callback so that we have the InputStream to read the FlowFile
final FlowFile toWrap = flowFile;
session.read(flowFile, new InputStreamCallback() {
@Override
public void process(final InputStream in) throws IOException {
final DataPacket dataPacket = new StandardDataPacket(toWrap.getAttributes(), in, toWrap.getSize());
transaction.send(dataPacket);
}
});
final long transferNanos = System.nanoTime() - startNanos;
final long transferMillis = TimeUnit.MILLISECONDS.convert(transferNanos, TimeUnit.NANOSECONDS);
flowFilesSent.add(flowFile);
bytesSent += flowFile.getSize();
logger.debug("{} Sent {} to {}", this, flowFile, peer);
final String transitUri = (transitUriPrefix == null) ? peer.getUrl() : transitUriPrefix + flowFile.getAttribute(CoreAttributes.UUID.key());
session.getProvenanceReporter().send(flowFile, transitUri, "Remote Host=" + peer.getHost() + ", Remote DN=" + userDn, transferMillis, false);
session.remove(flowFile);
final long sendingNanos = System.nanoTime() - startSendingNanos;
if (sendingNanos < BATCH_SEND_NANOS) {
flowFile = session.get();
} else {
flowFile = null;
}
continueTransaction = (flowFile != null);
}
transaction.confirm();
// consume input stream entirely, ignoring its contents. If we
// don't do this, the Connection will not be returned to the pool
stopWatch.stop();
final String uploadDataRate = stopWatch.calculateDataRate(bytesSent);
final long uploadMillis = stopWatch.getDuration(TimeUnit.MILLISECONDS);
final String dataSize = FormatUtils.formatDataSize(bytesSent);
session.commit();
transaction.complete();
final String flowFileDescription = (flowFilesSent.size() < 20) ? flowFilesSent.toString() : flowFilesSent.size() + " FlowFiles";
logger.info("{} Successfully sent {} ({}) to {} in {} milliseconds at a rate of {}", new Object[]{
this, flowFileDescription, dataSize, peer, uploadMillis, uploadDataRate});
return flowFilesSent.size();
} catch (final Exception e) {
session.rollback();
throw e;
}
}
@Override
public VersionNegotiator getVersionNegotiator() {
return versionNegotiator;

View File

@ -16,69 +16,31 @@
*/
package org.apache.nifi.remote.protocol.socket;
import java.io.ByteArrayInputStream;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.util.Map;
import java.util.zip.CRC32;
import java.util.zip.CheckedInputStream;
import java.util.zip.CheckedOutputStream;
import org.apache.nifi.events.EventReporter;
import org.apache.nifi.remote.Communicant;
import org.apache.nifi.remote.AbstractTransaction;
import org.apache.nifi.remote.Peer;
import org.apache.nifi.remote.Transaction;
import org.apache.nifi.remote.TransactionCompletion;
import org.apache.nifi.remote.TransferDirection;
import org.apache.nifi.remote.codec.FlowFileCodec;
import org.apache.nifi.remote.exception.ProtocolException;
import org.apache.nifi.remote.io.CompressionInputStream;
import org.apache.nifi.remote.io.CompressionOutputStream;
import org.apache.nifi.remote.protocol.DataPacket;
import org.apache.nifi.remote.protocol.RequestType;
import org.apache.nifi.remote.util.StandardDataPacket;
import org.apache.nifi.reporting.Severity;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.nifi.remote.protocol.Response;
import org.apache.nifi.remote.protocol.ResponseCode;
public class SocketClientTransaction implements Transaction {
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.IOException;
public class SocketClientTransaction extends AbstractTransaction {
private static final Logger logger = LoggerFactory.getLogger(SocketClientTransaction.class);
private final long creationNanoTime = System.nanoTime();
private final CRC32 crc = new CRC32();
private final int protocolVersion;
private final FlowFileCodec codec;
private final DataInputStream dis;
private final DataOutputStream dos;
private final TransferDirection direction;
private final boolean compress;
private final Peer peer;
private final int penaltyMillis;
private final String destinationId;
private final EventReporter eventReporter;
private boolean dataAvailable = false;
private int transfers = 0;
private long contentBytes = 0;
private TransactionState state;
SocketClientTransaction(final int protocolVersion, final String destinationId, final Peer peer, final FlowFileCodec codec,
final TransferDirection direction, final boolean useCompression, final int penaltyMillis, final EventReporter eventReporter) throws IOException {
this.protocolVersion = protocolVersion;
this.destinationId = destinationId;
this.peer = peer;
this.codec = codec;
this.direction = direction;
super(peer, direction, useCompression, codec, eventReporter, protocolVersion, penaltyMillis, destinationId);
this.dis = new DataInputStream(peer.getCommunicationsSession().getInput().getInputStream());
this.dos = new DataOutputStream(peer.getCommunicationsSession().getOutput().getOutputStream());
this.compress = useCompression;
this.state = TransactionState.TRANSACTION_STARTED;
this.penaltyMillis = penaltyMillis;
this.eventReporter = eventReporter;
initialize();
}
@ -116,291 +78,16 @@ public class SocketClientTransaction implements Transaction {
}
@Override
public DataPacket receive() throws IOException {
try {
try {
if (state != TransactionState.DATA_EXCHANGED && state != TransactionState.TRANSACTION_STARTED) {
throw new IllegalStateException("Cannot receive data from " + peer + " because Transaction State is " + state);
protected Response readTransactionResponse() throws IOException {
return Response.read(dis);
}
if (direction == TransferDirection.SEND) {
throw new IllegalStateException("Attempting to receive data from " + peer + " but started a SEND Transaction");
}
// if we already know there's no data, just return null
if (!dataAvailable) {
return null;
}
// if we have already received a packet, check if another is available.
if (transfers > 0) {
// Determine if Peer will send us data or has no data to send us
final Response dataAvailableCode = Response.read(dis);
switch (dataAvailableCode.getCode()) {
case CONTINUE_TRANSACTION:
logger.debug("{} {} Indicates Transaction should continue", this, peer);
this.dataAvailable = true;
break;
case FINISH_TRANSACTION:
logger.debug("{} {} Indicates Transaction should finish", this, peer);
this.dataAvailable = false;
break;
default:
throw new ProtocolException("Got unexpected response from " + peer + " when asking for data: " + dataAvailableCode);
}
}
// if no data available, return null
if (!dataAvailable) {
return null;
}
logger.debug("{} Receiving data from {}", this, peer);
final InputStream dataIn = compress ? new CompressionInputStream(dis) : dis;
final DataPacket packet = codec.decode(new CheckedInputStream(dataIn, crc));
if (packet == null) {
this.dataAvailable = false;
@Override
protected void writeTransactionResponse(ResponseCode response, String explanation) throws IOException {
if(explanation == null){
response.writeResponse(dos);
} else {
transfers++;
contentBytes += packet.getSize();
response.writeResponse(dos, explanation);
}
this.state = TransactionState.DATA_EXCHANGED;
return packet;
} catch (final IOException ioe) {
throw new IOException("Failed to receive data from " + peer + " due to " + ioe, ioe);
}
} catch (final Exception e) {
error();
throw e;
}
}
@Override
public void send(final byte[] content, final Map<String, String> attributes) throws IOException {
send(new StandardDataPacket(attributes, new ByteArrayInputStream(content), content.length));
}
@Override
public void send(final DataPacket dataPacket) throws IOException {
try {
try {
if (state != TransactionState.DATA_EXCHANGED && state != TransactionState.TRANSACTION_STARTED) {
throw new IllegalStateException("Cannot send data to " + peer + " because Transaction State is " + state);
}
if (direction == TransferDirection.RECEIVE) {
throw new IllegalStateException("Attempting to send data to " + peer + " but started a RECEIVE Transaction");
}
if (transfers > 0) {
ResponseCode.CONTINUE_TRANSACTION.writeResponse(dos);
}
logger.debug("{} Sending data to {}", this, peer);
final OutputStream dataOut = compress ? new CompressionOutputStream(dos) : dos;
final OutputStream out = new CheckedOutputStream(dataOut, crc);
codec.encode(dataPacket, out);
// need to close the CompressionOutputStream in order to force it write out any remaining bytes.
// Otherwise, do NOT close it because we don't want to close the underlying stream
// (CompressionOutputStream will not close the underlying stream when it's closed)
if (compress) {
out.close();
}
transfers++;
contentBytes += dataPacket.getSize();
this.state = TransactionState.DATA_EXCHANGED;
} catch (final IOException ioe) {
throw new IOException("Failed to send data to " + peer + " due to " + ioe, ioe);
}
} catch (final Exception e) {
error();
throw e;
}
}
@Override
public void cancel(final String explanation) throws IOException {
if (state == TransactionState.TRANSACTION_CANCELED || state == TransactionState.TRANSACTION_COMPLETED || state == TransactionState.ERROR) {
throw new IllegalStateException("Cannot cancel transaction because state is already " + state);
}
try {
ResponseCode.CANCEL_TRANSACTION.writeResponse(dos, explanation == null ? "<No explanation given>" : explanation);
state = TransactionState.TRANSACTION_CANCELED;
} catch (final IOException ioe) {
error();
throw new IOException("Failed to send 'cancel transaction' message to " + peer + " due to " + ioe, ioe);
}
}
@Override
public TransactionCompletion complete() throws IOException {
try {
try {
if (state != TransactionState.TRANSACTION_CONFIRMED) {
throw new IllegalStateException("Cannot complete transaction with " + peer + " because state is " + state
+ "; Transaction can only be completed when state is " + TransactionState.TRANSACTION_CONFIRMED);
}
boolean backoff = false;
if (direction == TransferDirection.RECEIVE) {
if (transfers == 0) {
state = TransactionState.TRANSACTION_COMPLETED;
return new SocketClientTransactionCompletion(false, 0, 0L, System.nanoTime() - creationNanoTime);
}
// Confirm that we received the data and the peer can now discard it
logger.debug("{} Sending TRANSACTION_FINISHED to {}", this, peer);
ResponseCode.TRANSACTION_FINISHED.writeResponse(dos);
state = TransactionState.TRANSACTION_COMPLETED;
} else {
final Response transactionResponse;
try {
transactionResponse = Response.read(dis);
} catch (final IOException e) {
throw new IOException(this + " Failed to receive a response from " + peer + " when expecting a TransactionFinished Indicator. "
+ "It is unknown whether or not the peer successfully received/processed the data.", e);
}
logger.debug("{} Received {} from {}", this, transactionResponse, peer);
if (transactionResponse.getCode() == ResponseCode.TRANSACTION_FINISHED_BUT_DESTINATION_FULL) {
peer.penalize(destinationId, penaltyMillis);
backoff = true;
} else if (transactionResponse.getCode() != ResponseCode.TRANSACTION_FINISHED) {
throw new ProtocolException("After sending data to " + peer + ", expected TRANSACTION_FINISHED response but got " + transactionResponse);
}
state = TransactionState.TRANSACTION_COMPLETED;
}
return new SocketClientTransactionCompletion(backoff, transfers, contentBytes, System.nanoTime() - creationNanoTime);
} catch (final IOException ioe) {
throw new IOException("Failed to complete transaction with " + peer + " due to " + ioe, ioe);
}
} catch (final Exception e) {
error();
throw e;
}
}
@Override
public void confirm() throws IOException {
try {
try {
if (state == TransactionState.TRANSACTION_STARTED && !dataAvailable && direction == TransferDirection.RECEIVE) {
// client requested to receive data but no data available. no need to confirm.
state = TransactionState.TRANSACTION_CONFIRMED;
return;
}
if (state != TransactionState.DATA_EXCHANGED) {
throw new IllegalStateException("Cannot confirm Transaction because state is " + state
+ "; Transaction can only be confirmed when state is " + TransactionState.DATA_EXCHANGED);
}
if (direction == TransferDirection.RECEIVE) {
if (dataAvailable) {
throw new IllegalStateException("Cannot complete transaction because the sender has already sent more data than client has consumed.");
}
// we received a FINISH_TRANSACTION indicator. Send back a CONFIRM_TRANSACTION message
// to peer so that we can verify that the connection is still open. This is a two-phase commit,
// which helps to prevent the chances of data duplication. Without doing this, we may commit the
// session and then when we send the response back to the peer, the peer may have timed out and may not
// be listening. As a result, it will re-send the data. By doing this two-phase commit, we narrow the
// Critical Section involved in this transaction so that rather than the Critical Section being the
// time window involved in the entire transaction, it is reduced to a simple round-trip conversation.
logger.trace("{} Sending CONFIRM_TRANSACTION Response Code to {}", this, peer);
final String calculatedCRC = String.valueOf(crc.getValue());
ResponseCode.CONFIRM_TRANSACTION.writeResponse(dos, calculatedCRC);
final Response confirmTransactionResponse;
try {
confirmTransactionResponse = Response.read(dis);
} catch (final IOException ioe) {
logger.error("Failed to receive response code from {} when expecting confirmation of transaction", peer);
if (eventReporter != null) {
eventReporter.reportEvent(Severity.ERROR, "Site-to-Site", "Failed to receive response code from " + peer + " when expecting confirmation of transaction");
}
throw ioe;
}
logger.trace("{} Received {} from {}", this, confirmTransactionResponse, peer);
switch (confirmTransactionResponse.getCode()) {
case CONFIRM_TRANSACTION:
break;
case BAD_CHECKSUM:
throw new IOException(this + " Received a BadChecksum response from peer " + peer);
default:
throw new ProtocolException(this + " Received unexpected Response from peer " + peer + " : "
+ confirmTransactionResponse + "; expected 'Confirm Transaction' Response Code");
}
state = TransactionState.TRANSACTION_CONFIRMED;
} else {
logger.debug("{} Sent FINISH_TRANSACTION indicator to {}", this, peer);
ResponseCode.FINISH_TRANSACTION.writeResponse(dos);
final String calculatedCRC = String.valueOf(crc.getValue());
// we've sent a FINISH_TRANSACTION. Now we'll wait for the peer to send a 'Confirm Transaction' response
final Response transactionConfirmationResponse = Response.read(dis);
if (transactionConfirmationResponse.getCode() == ResponseCode.CONFIRM_TRANSACTION) {
// Confirm checksum and echo back the confirmation.
logger.trace("{} Received {} from {}", this, transactionConfirmationResponse, peer);
final String receivedCRC = transactionConfirmationResponse.getMessage();
// CRC was not used before version 4
if (protocolVersion > 3) {
if (!receivedCRC.equals(calculatedCRC)) {
ResponseCode.BAD_CHECKSUM.writeResponse(dos);
throw new IOException(this + " Sent data to peer " + peer + " but calculated CRC32 Checksum as "
+ calculatedCRC + " while peer calculated CRC32 Checksum as "
+ receivedCRC + "; canceling transaction and rolling back session");
}
}
ResponseCode.CONFIRM_TRANSACTION.writeResponse(dos, "");
} else {
throw new ProtocolException("Expected to receive 'Confirm Transaction' response from peer "
+ peer + " but received " + transactionConfirmationResponse);
}
state = TransactionState.TRANSACTION_CONFIRMED;
}
} catch (final IOException ioe) {
throw new IOException("Failed to confirm transaction with " + peer + " due to " + ioe, ioe);
}
} catch (final Exception e) {
error();
throw e;
}
}
@Override
public void error() {
this.state = TransactionState.ERROR;
}
@Override
public TransactionState getState() {
return state;
}
@Override
public Communicant getCommunicant() {
return peer;
}
@Override
public String toString() {
return "SocketClientTransaction[Url=" + peer.getUrl() + ", TransferDirection=" + direction + ", State=" + state + "]";
}
}

View File

@ -0,0 +1,50 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.remote.util;
import org.apache.nifi.events.EventReporter;
import org.apache.nifi.reporting.Severity;
import org.slf4j.Logger;
import org.slf4j.helpers.MessageFormatter;
public class EventReportUtil {
private static final String CATEGORY = "Site-to-Site";
public static void warn(final Logger logger, final EventReporter eventReporter, final String msg, final Object... args) {
logger.warn(msg, args);
if (eventReporter != null) {
eventReporter.reportEvent(Severity.WARNING, CATEGORY, MessageFormatter.arrayFormat(msg, args).getMessage());
}
}
public static void warn(final Logger logger, final EventReporter eventReporter, final String msg, final Throwable t) {
logger.warn(msg, t);
if (eventReporter != null) {
eventReporter.reportEvent(Severity.WARNING, CATEGORY, msg + ": " + t.toString());
}
}
public static void error(final Logger logger, final EventReporter eventReporter, final String msg, final Object... args) {
logger.error(msg, args);
if (eventReporter != null) {
eventReporter.reportEvent(Severity.ERROR, CATEGORY, MessageFormatter.arrayFormat(msg, args).getMessage());
}
}
}

View File

@ -1,100 +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.remote.util;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.net.HttpURLConnection;
import java.net.URL;
import javax.net.ssl.HostnameVerifier;
import javax.net.ssl.HttpsURLConnection;
import javax.net.ssl.SSLContext;
import javax.net.ssl.SSLSession;
import org.apache.nifi.stream.io.StreamUtils;
import org.apache.nifi.web.api.dto.ControllerDTO;
import org.codehaus.jackson.JsonNode;
import org.codehaus.jackson.map.ObjectMapper;
public class NiFiRestApiUtil {
public static final int RESPONSE_CODE_OK = 200;
private final SSLContext sslContext;
public NiFiRestApiUtil(final SSLContext sslContext) {
this.sslContext = sslContext;
}
private HttpURLConnection getConnection(final String connUrl, final int timeoutMillis) throws IOException {
final URL url = new URL(connUrl);
final HttpURLConnection connection = (HttpURLConnection) url.openConnection();
connection.setConnectTimeout(timeoutMillis);
connection.setReadTimeout(timeoutMillis);
// special handling for https
if (sslContext != null && connection instanceof HttpsURLConnection) {
HttpsURLConnection secureConnection = (HttpsURLConnection) connection;
secureConnection.setSSLSocketFactory(sslContext.getSocketFactory());
// check the trusted hostname property and override the HostnameVerifier
secureConnection.setHostnameVerifier(new OverrideHostnameVerifier(url.getHost(),
secureConnection.getHostnameVerifier()));
}
return connection;
}
public ControllerDTO getController(final String url, final int timeoutMillis) throws IOException {
final HttpURLConnection connection = getConnection(url, timeoutMillis);
connection.setRequestMethod("GET");
final int responseCode = connection.getResponseCode();
final ByteArrayOutputStream baos = new ByteArrayOutputStream();
StreamUtils.copy(connection.getInputStream(), baos);
final String responseMessage = baos.toString();
if (responseCode == RESPONSE_CODE_OK) {
final ObjectMapper mapper = new ObjectMapper();
final JsonNode jsonNode = mapper.readTree(responseMessage);
final JsonNode controllerNode = jsonNode.get("controller");
return mapper.readValue(controllerNode, ControllerDTO.class);
} else {
throw new IOException("Got HTTP response Code " + responseCode + ": " + connection.getResponseMessage() + " with explanation: " + responseMessage);
}
}
private static class OverrideHostnameVerifier implements HostnameVerifier {
private final String trustedHostname;
private final HostnameVerifier delegate;
private OverrideHostnameVerifier(String trustedHostname, HostnameVerifier delegate) {
this.trustedHostname = trustedHostname;
this.delegate = delegate;
}
@Override
public boolean verify(String hostname, SSLSession session) {
if (trustedHostname.equalsIgnoreCase(hostname)) {
return true;
}
return delegate.verify(hostname, session);
}
}
}

View File

@ -0,0 +1,992 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT 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.remote.util;
import org.apache.commons.lang3.StringUtils;
import org.apache.http.Header;
import org.apache.http.HttpEntity;
import org.apache.http.HttpException;
import org.apache.http.HttpHost;
import org.apache.http.HttpInetConnection;
import org.apache.http.HttpRequest;
import org.apache.http.HttpResponse;
import org.apache.http.HttpResponseInterceptor;
import org.apache.http.StatusLine;
import org.apache.http.auth.AuthScope;
import org.apache.http.auth.UsernamePasswordCredentials;
import org.apache.http.client.CredentialsProvider;
import org.apache.http.client.config.RequestConfig;
import org.apache.http.client.methods.CloseableHttpResponse;
import org.apache.http.client.methods.HttpDelete;
import org.apache.http.client.methods.HttpGet;
import org.apache.http.client.methods.HttpPost;
import org.apache.http.client.methods.HttpPut;
import org.apache.http.client.methods.HttpRequestBase;
import org.apache.http.client.utils.URIUtils;
import org.apache.http.conn.ManagedHttpClientConnection;
import org.apache.http.entity.BasicHttpEntity;
import org.apache.http.impl.client.BasicCredentialsProvider;
import org.apache.http.impl.client.CloseableHttpClient;
import org.apache.http.impl.client.HttpClientBuilder;
import org.apache.http.impl.client.HttpClients;
import org.apache.http.impl.nio.client.CloseableHttpAsyncClient;
import org.apache.http.impl.nio.client.HttpAsyncClientBuilder;
import org.apache.http.impl.nio.client.HttpAsyncClients;
import org.apache.http.nio.ContentEncoder;
import org.apache.http.nio.IOControl;
import org.apache.http.nio.conn.ManagedNHttpClientConnection;
import org.apache.http.nio.protocol.BasicAsyncResponseConsumer;
import org.apache.http.nio.protocol.HttpAsyncRequestProducer;
import org.apache.http.protocol.HttpContext;
import org.apache.http.protocol.HttpCoreContext;
import org.apache.http.util.EntityUtils;
import org.apache.nifi.remote.TransferDirection;
import org.apache.nifi.remote.client.http.TransportProtocolVersionNegotiator;
import org.apache.nifi.remote.exception.PortNotRunningException;
import org.apache.nifi.remote.exception.ProtocolException;
import org.apache.nifi.remote.exception.UnknownPortException;
import org.apache.nifi.remote.io.http.HttpCommunicationsSession;
import org.apache.nifi.remote.io.http.HttpInput;
import org.apache.nifi.remote.io.http.HttpOutput;
import org.apache.nifi.remote.protocol.CommunicationsSession;
import org.apache.nifi.remote.protocol.ResponseCode;
import org.apache.nifi.remote.protocol.http.HttpHeaders;
import org.apache.nifi.remote.protocol.http.HttpProxy;
import org.apache.nifi.security.util.CertificateUtils;
import org.apache.nifi.stream.io.ByteArrayInputStream;
import org.apache.nifi.stream.io.ByteArrayOutputStream;
import org.apache.nifi.stream.io.StreamUtils;
import org.apache.nifi.web.api.dto.ControllerDTO;
import org.apache.nifi.web.api.dto.remote.PeerDTO;
import org.apache.nifi.web.api.entity.ControllerEntity;
import org.apache.nifi.web.api.entity.PeersEntity;
import org.apache.nifi.web.api.entity.TransactionResultEntity;
import org.codehaus.jackson.JsonParseException;
import org.codehaus.jackson.map.DeserializationConfig;
import org.codehaus.jackson.map.JsonMappingException;
import org.codehaus.jackson.map.ObjectMapper;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import javax.net.ssl.SSLContext;
import javax.net.ssl.SSLPeerUnverifiedException;
import javax.net.ssl.SSLSession;
import java.io.Closeable;
import java.io.IOException;
import java.io.InputStream;
import java.io.PipedInputStream;
import java.io.PipedOutputStream;
import java.net.URI;
import java.net.URISyntaxException;
import java.nio.ByteBuffer;
import java.nio.channels.Channels;
import java.nio.channels.ReadableByteChannel;
import java.security.cert.Certificate;
import java.security.cert.CertificateException;
import java.security.cert.X509Certificate;
import java.util.Collection;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.ThreadFactory;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.regex.Pattern;
import static org.apache.commons.lang3.StringUtils.isEmpty;
import static org.apache.nifi.remote.protocol.http.HttpHeaders.HANDSHAKE_PROPERTY_BATCH_COUNT;
import static org.apache.nifi.remote.protocol.http.HttpHeaders.HANDSHAKE_PROPERTY_BATCH_DURATION;
import static org.apache.nifi.remote.protocol.http.HttpHeaders.HANDSHAKE_PROPERTY_BATCH_SIZE;
import static org.apache.nifi.remote.protocol.http.HttpHeaders.HANDSHAKE_PROPERTY_REQUEST_EXPIRATION;
import static org.apache.nifi.remote.protocol.http.HttpHeaders.HANDSHAKE_PROPERTY_USE_COMPRESSION;
import static org.apache.nifi.remote.protocol.http.HttpHeaders.LOCATION_HEADER_NAME;
import static org.apache.nifi.remote.protocol.http.HttpHeaders.LOCATION_URI_INTENT_NAME;
import static org.apache.nifi.remote.protocol.http.HttpHeaders.LOCATION_URI_INTENT_VALUE;
public class SiteToSiteRestApiClient implements Closeable {
private static final int RESPONSE_CODE_OK = 200;
private static final int RESPONSE_CODE_CREATED = 201;
private static final int RESPONSE_CODE_ACCEPTED = 202;
private static final int RESPONSE_CODE_SEE_OTHER = 303;
private static final int RESPONSE_CODE_BAD_REQUEST = 400;
private static final int RESPONSE_CODE_UNAUTHORIZED = 401;
private static final int RESPONSE_CODE_NOT_FOUND = 404;
private static final int RESPONSE_CODE_SERVICE_UNAVAILABLE = 503;
private static final Logger logger = LoggerFactory.getLogger(SiteToSiteRestApiClient.class);
private String baseUrl;
protected final SSLContext sslContext;
protected final HttpProxy proxy;
private RequestConfig requestConfig;
private CredentialsProvider credentialsProvider;
private CloseableHttpClient httpClient;
private CloseableHttpAsyncClient httpAsyncClient;
private boolean compress = false;
private long requestExpirationMillis = 0;
private int serverTransactionTtl = 0;
private int batchCount = 0;
private long batchSize = 0;
private long batchDurationMillis = 0;
private TransportProtocolVersionNegotiator transportProtocolVersionNegotiator = new TransportProtocolVersionNegotiator(1);
private String trustedPeerDn;
private final ScheduledExecutorService ttlExtendTaskExecutor;
private ScheduledFuture<?> ttlExtendingThread;
private SiteToSiteRestApiClient extendingApiClient;
private int connectTimeoutMillis;
private int readTimeoutMillis;
private static final Pattern HTTP_ABS_URL = Pattern.compile("^https?://.+$");
public SiteToSiteRestApiClient(final SSLContext sslContext, final HttpProxy proxy) {
this.sslContext = sslContext;
this.proxy = proxy;
ttlExtendTaskExecutor = Executors.newScheduledThreadPool(1, new ThreadFactory() {
private final ThreadFactory defaultFactory = Executors.defaultThreadFactory();
@Override
public Thread newThread(final Runnable r) {
final Thread thread = defaultFactory.newThread(r);
thread.setName(Thread.currentThread().getName() + " TTLExtend");
return thread;
}
});
}
@Override
public void close() throws IOException {
stopExtendingTtl();
closeSilently(httpClient);
closeSilently(httpAsyncClient);
}
private CloseableHttpClient getHttpClient() {
if (httpClient == null) {
setupClient();
}
return httpClient;
}
private CloseableHttpAsyncClient getHttpAsyncClient() {
if (httpAsyncClient == null) {
setupAsyncClient();
}
return httpAsyncClient;
}
private RequestConfig getRequestConfig() {
if (requestConfig == null) {
setupRequestConfig();
}
return requestConfig;
}
private CredentialsProvider getCredentialsProvider() {
if (credentialsProvider == null) {
setupCredentialsProvider();
}
return credentialsProvider;
}
private void setupRequestConfig() {
final RequestConfig.Builder requestConfigBuilder = RequestConfig.custom()
.setConnectionRequestTimeout(connectTimeoutMillis)
.setConnectTimeout(connectTimeoutMillis)
.setSocketTimeout(readTimeoutMillis);
if (proxy != null) {
requestConfigBuilder.setProxy(proxy.getHttpHost());
}
requestConfig = requestConfigBuilder.build();
}
private void setupCredentialsProvider() {
credentialsProvider = new BasicCredentialsProvider();
if (proxy != null) {
if (!isEmpty(proxy.getUsername()) && !isEmpty(proxy.getPassword())) {
credentialsProvider.setCredentials(
new AuthScope(proxy.getHttpHost()),
new UsernamePasswordCredentials(proxy.getUsername(), proxy.getPassword()));
}
}
}
private void setupClient() {
final HttpClientBuilder clientBuilder = HttpClients.custom();
if (sslContext != null) {
clientBuilder.setSslcontext(sslContext);
clientBuilder.addInterceptorFirst(new HttpsResponseInterceptor());
}
httpClient = clientBuilder
.setDefaultCredentialsProvider(getCredentialsProvider()).build();
}
private void setupAsyncClient() {
final HttpAsyncClientBuilder clientBuilder = HttpAsyncClients.custom();
if (sslContext != null) {
clientBuilder.setSSLContext(sslContext);
clientBuilder.addInterceptorFirst(new HttpsResponseInterceptor());
}
httpAsyncClient = clientBuilder.setDefaultCredentialsProvider(getCredentialsProvider()).build();
httpAsyncClient.start();
}
private class HttpsResponseInterceptor implements HttpResponseInterceptor {
@Override
public void process(final HttpResponse response, final HttpContext httpContext) throws HttpException, IOException {
final HttpCoreContext coreContext = HttpCoreContext.adapt(httpContext);
final HttpInetConnection conn = coreContext.getConnection(HttpInetConnection.class);
if (!conn.isOpen()) {
return;
}
final SSLSession sslSession;
if (conn instanceof ManagedHttpClientConnection) {
sslSession = ((ManagedHttpClientConnection)conn).getSSLSession();
} else if (conn instanceof ManagedNHttpClientConnection) {
sslSession = ((ManagedNHttpClientConnection)conn).getSSLSession();
} else {
throw new RuntimeException("Unexpected connection type was used, " + conn);
}
if (sslSession != null) {
final Certificate[] certChain = sslSession.getPeerCertificates();
if (certChain == null || certChain.length == 0) {
throw new SSLPeerUnverifiedException("No certificates found");
}
try {
final X509Certificate cert = CertificateUtils.convertAbstractX509Certificate(certChain[0]);
trustedPeerDn = cert.getSubjectDN().getName().trim();
} catch (CertificateException e) {
final String msg = "Could not extract subject DN from SSL session peer certificate";
logger.warn(msg);
throw new SSLPeerUnverifiedException(msg);
}
}
}
}
public ControllerDTO getController() throws IOException {
try {
HttpGet get = createGet("/site-to-site");
get.setHeader(HttpHeaders.PROTOCOL_VERSION, String.valueOf(transportProtocolVersionNegotiator.getVersion()));
return execute(get, ControllerEntity.class).getController();
} catch (HttpGetFailedException e) {
if (RESPONSE_CODE_NOT_FOUND == e.getResponseCode()) {
logger.debug("getController received NOT_FOUND, trying to access the old NiFi version resource url...");
HttpGet get = createGet("/controller");
return execute(get, ControllerEntity.class).getController();
}
throw e;
}
}
public Collection<PeerDTO> getPeers() throws IOException {
HttpGet get = createGet("/site-to-site/peers");
get.setHeader(HttpHeaders.PROTOCOL_VERSION, String.valueOf(transportProtocolVersionNegotiator.getVersion()));
return execute(get, PeersEntity.class).getPeers();
}
public String initiateTransaction(TransferDirection direction, String portId) throws IOException {
if (TransferDirection.RECEIVE.equals(direction)) {
return initiateTransaction("output-ports", portId);
} else {
return initiateTransaction("input-ports", portId);
}
}
private String initiateTransaction(String portType, String portId) throws IOException {
logger.debug("initiateTransaction handshaking portType={}, portId={}", portType, portId);
HttpPost post = createPost("/site-to-site/" + portType + "/" + portId + "/transactions");
post.setHeader("Accept", "application/json");
post.setHeader(HttpHeaders.PROTOCOL_VERSION, String.valueOf(transportProtocolVersionNegotiator.getVersion()));
setHandshakeProperties(post);
try (CloseableHttpResponse response = getHttpClient().execute(post)) {
int responseCode = response.getStatusLine().getStatusCode();
logger.debug("initiateTransaction responseCode={}", responseCode);
String transactionUrl;
switch (responseCode) {
case RESPONSE_CODE_CREATED :
EntityUtils.consume(response.getEntity());
transactionUrl = readTransactionUrl(response);
if (isEmpty(transactionUrl)) {
throw new ProtocolException("Server returned RESPONSE_CODE_CREATED without Location header");
}
Header transportProtocolVersionHeader = response.getFirstHeader(HttpHeaders.PROTOCOL_VERSION);
if (transportProtocolVersionHeader == null) {
throw new ProtocolException("Server didn't return confirmed protocol version");
}
Integer protocolVersionConfirmedByServer = Integer.valueOf(transportProtocolVersionHeader.getValue());
logger.debug("Finished version negotiation, protocolVersionConfirmedByServer={}", protocolVersionConfirmedByServer);
transportProtocolVersionNegotiator.setVersion(protocolVersionConfirmedByServer);
Header serverTransactionTtlHeader = response.getFirstHeader(HttpHeaders.SERVER_SIDE_TRANSACTION_TTL);
if (serverTransactionTtlHeader == null) {
throw new ProtocolException("Server didn't return " + HttpHeaders.SERVER_SIDE_TRANSACTION_TTL);
}
serverTransactionTtl = Integer.parseInt(serverTransactionTtlHeader.getValue());
break;
default:
try (InputStream content = response.getEntity().getContent()) {
throw handleErrResponse(responseCode, content);
}
}
logger.debug("initiateTransaction handshaking finished, transactionUrl={}", transactionUrl);
return transactionUrl;
}
}
public boolean openConnectionForReceive(String transactionUrl, CommunicationsSession commSession) throws IOException {
HttpGet get = createGet(transactionUrl + "/flow-files");
get.setHeader(HttpHeaders.PROTOCOL_VERSION, String.valueOf(transportProtocolVersionNegotiator.getVersion()));
setHandshakeProperties(get);
CloseableHttpResponse response = getHttpClient().execute(get);
int responseCode = response.getStatusLine().getStatusCode();
logger.debug("responseCode={}", responseCode);
boolean keepItOpen = false;
try {
switch (responseCode) {
case RESPONSE_CODE_OK :
logger.debug("Server returned RESPONSE_CODE_OK, indicating there was no data.");
EntityUtils.consume(response.getEntity());
return false;
case RESPONSE_CODE_ACCEPTED :
InputStream httpIn = response.getEntity().getContent();
InputStream streamCapture = new InputStream() {
boolean closed = false;
@Override
public int read() throws IOException {
if(closed) return -1;
int r = httpIn.read();
if (r < 0) {
closed = true;
logger.debug("Reached to end of input stream. Closing resources...");
stopExtendingTtl();
closeSilently(httpIn);
closeSilently(response);
}
return r;
}
};
((HttpInput)commSession.getInput()).setInputStream(streamCapture);
startExtendingTtl(transactionUrl, httpIn, response);
keepItOpen = true;
return true;
default:
try (InputStream content = response.getEntity().getContent()) {
throw handleErrResponse(responseCode, content);
}
}
} finally {
if (!keepItOpen) {
response.close();
}
}
}
private final int DATA_PACKET_CHANNEL_READ_BUFFER_SIZE = 16384;
private Future<HttpResponse> postResult;
private CountDownLatch transferDataLatch = new CountDownLatch(1);
public void openConnectionForSend(String transactionUrl, CommunicationsSession commSession) throws IOException {
final String flowFilesPath = transactionUrl + "/flow-files";
HttpPost post = createPost(flowFilesPath);
post.setHeader("Content-Type", "application/octet-stream");
post.setHeader("Accept", "text/plain");
post.setHeader(HttpHeaders.PROTOCOL_VERSION, String.valueOf(transportProtocolVersionNegotiator.getVersion()));
setHandshakeProperties(post);
CountDownLatch initConnectionLatch = new CountDownLatch(1);
final URI requestUri = post.getURI();
final PipedOutputStream outputStream = new PipedOutputStream();
final PipedInputStream inputStream = new PipedInputStream(outputStream, DATA_PACKET_CHANNEL_READ_BUFFER_SIZE);
final ReadableByteChannel dataPacketChannel = Channels.newChannel(inputStream);
final HttpAsyncRequestProducer asyncRequestProducer = new HttpAsyncRequestProducer() {
private final ByteBuffer buffer = ByteBuffer.allocate(DATA_PACKET_CHANNEL_READ_BUFFER_SIZE);
@Override
public HttpHost getTarget() {
return URIUtils.extractHost(requestUri);
}
@Override
public HttpRequest generateRequest() throws IOException, HttpException {
// Pass the output stream so that Site-to-Site client thread can send
// data packet through this connection.
logger.debug("sending data to {} has started...", flowFilesPath);
((HttpOutput)commSession.getOutput()).setOutputStream(outputStream);
initConnectionLatch.countDown();
final BasicHttpEntity entity = new BasicHttpEntity();
entity.setChunked(true);
entity.setContentType("application/octet-stream");
post.setEntity(entity);
return post;
}
@Override
public void produceContent(ContentEncoder encoder, IOControl ioControl) throws IOException {
int totalRead = 0;
int totalProduced = 0;
int read;
// This read() blocks until data becomes available,
// or corresponding outputStream is closed.
while ((read = dataPacketChannel.read(buffer)) > -1) {
buffer.flip();
while (buffer.hasRemaining()) {
totalProduced += encoder.write(buffer);
}
buffer.clear();
logger.trace("Read {} bytes from dataPacketChannel. {}", read, flowFilesPath);
totalRead += read;
}
// There might be remaining bytes in buffer. Make sure it's fully drained.
buffer.flip();
while (buffer.hasRemaining()) {
totalProduced += encoder.write(buffer);
}
final long totalWritten = commSession.getOutput().getBytesWritten();
logger.debug("sending data to {} has reached to its end. produced {} bytes by reading {} bytes from channel. {} bytes written in this transaction.",
flowFilesPath, totalProduced, totalRead, totalWritten);
if (totalRead != totalWritten || totalProduced != totalWritten) {
final String msg = "Sending data to %s has reached to its end, but produced : read : wrote byte sizes (%d : $d : %d) were not equal. Something went wrong.";
throw new RuntimeException(String.format(msg, flowFilesPath, totalProduced, totalRead, totalWritten));
}
transferDataLatch.countDown();
encoder.complete();
dataPacketChannel.close();
}
@Override
public void requestCompleted(HttpContext context) {
logger.debug("Sending data to {} completed.", flowFilesPath);
}
@Override
public void failed(Exception ex) {
logger.error("Sending data to {} has failed", flowFilesPath, ex);
}
@Override
public boolean isRepeatable() {
// In order to pass authentication, request has to be repeatable.
return true;
}
@Override
public void resetRequest() throws IOException {
logger.debug("Sending data request to {} has been reset...", flowFilesPath);
}
@Override
public void close() throws IOException {
logger.debug("Closing sending data request to {}", flowFilesPath);
closeSilently(outputStream);
closeSilently(dataPacketChannel);
stopExtendingTtl();
}
};
postResult = getHttpAsyncClient().execute(asyncRequestProducer, new BasicAsyncResponseConsumer(), null);
try {
// Need to wait the post request actually started so that we can write to its output stream.
if (!initConnectionLatch.await(connectTimeoutMillis, TimeUnit.MILLISECONDS)) {
throw new IOException("Awaiting initConnectionLatch has been timeout.");
}
// Started.
transferDataLatch = new CountDownLatch(1);
startExtendingTtl(transactionUrl, dataPacketChannel, null);
} catch (InterruptedException e) {
throw new IOException("Awaiting initConnectionLatch has been interrupted.", e);
}
}
public void finishTransferFlowFiles(CommunicationsSession commSession) throws IOException {
if (postResult == null) {
new IllegalStateException("Data transfer has not started yet.");
}
// No more data can be sent.
// Close PipedOutputStream so that dataPacketChannel doesn't blocked.
// If we don't close this output stream, then PipedInputStream loops infinitely at read().
commSession.getOutput().getOutputStream().close();
logger.debug("{} FinishTransferFlowFiles no more data can be sent", this);
try {
if (!transferDataLatch.await(requestExpirationMillis, TimeUnit.MILLISECONDS)) {
throw new IOException("Awaiting transferDataLatch has been timeout.");
}
} catch (InterruptedException e) {
throw new IOException("Awaiting transferDataLatch has been interrupted.", e);
}
stopExtendingTtl();
final HttpResponse response;
try {
response = postResult.get(readTimeoutMillis, TimeUnit.MILLISECONDS);
} catch (ExecutionException e) {
logger.debug("Something has happened at sending thread. {}", e.getMessage());
Throwable cause = e.getCause();
if (cause instanceof IOException) {
throw (IOException) cause;
} else {
throw new IOException(cause);
}
} catch (TimeoutException|InterruptedException e) {
throw new IOException(e);
}
int responseCode = response.getStatusLine().getStatusCode();
switch (responseCode) {
case RESPONSE_CODE_ACCEPTED :
String receivedChecksum = EntityUtils.toString(response.getEntity());
((HttpInput)commSession.getInput()).setInputStream(new ByteArrayInputStream(receivedChecksum.getBytes()));
((HttpCommunicationsSession)commSession).setChecksum(receivedChecksum);
logger.debug("receivedChecksum={}", receivedChecksum);
break;
default:
try (InputStream content = response.getEntity().getContent()) {
throw handleErrResponse(responseCode, content);
}
}
}
private void startExtendingTtl(final String transactionUrl, final Closeable stream, final CloseableHttpResponse response) {
if (ttlExtendingThread != null) {
// Already started.
return;
}
logger.debug("Starting extending TTL thread...");
extendingApiClient = new SiteToSiteRestApiClient(sslContext, proxy);
extendingApiClient.transportProtocolVersionNegotiator = this.transportProtocolVersionNegotiator;
extendingApiClient.connectTimeoutMillis = this.connectTimeoutMillis;
extendingApiClient.readTimeoutMillis = this.readTimeoutMillis;
int extendFrequency = serverTransactionTtl / 2;
ttlExtendingThread = ttlExtendTaskExecutor.scheduleWithFixedDelay(() -> {
try {
extendingApiClient.extendTransaction(transactionUrl);
} catch (Exception e) {
logger.warn("Failed to extend transaction ttl", e);
try {
// Without disconnecting, Site-to-Site client keep reading data packet,
// while server has already rollback.
this.close();
} catch (IOException ec) {
logger.warn("Failed to close", e);
}
}
}, extendFrequency, extendFrequency, TimeUnit.SECONDS);
}
private void closeSilently(final Closeable closeable) {
try {
if (closeable != null) {
closeable.close();
}
} catch (IOException e) {
logger.warn("Got an exception during closing {}: {}", closeable, e.getMessage());
if (logger.isDebugEnabled()) {
logger.warn("", e);
}
}
}
public TransactionResultEntity extendTransaction(String transactionUrl) throws IOException {
logger.debug("Sending extendTransaction request to transactionUrl: {}", transactionUrl);
final HttpPut put = createPut(transactionUrl);
put.setHeader("Accept", "application/json");
put.setHeader(HttpHeaders.PROTOCOL_VERSION, String.valueOf(transportProtocolVersionNegotiator.getVersion()));
setHandshakeProperties(put);
try (CloseableHttpResponse response = getHttpClient().execute(put)) {
int responseCode = response.getStatusLine().getStatusCode();
logger.debug("extendTransaction responseCode={}", responseCode);
try (InputStream content = response.getEntity().getContent()) {
switch (responseCode) {
case RESPONSE_CODE_OK :
return readResponse(content);
default:
throw handleErrResponse(responseCode, content);
}
}
}
}
private void stopExtendingTtl() {
if (!ttlExtendTaskExecutor.isShutdown()) {
ttlExtendTaskExecutor.shutdown();
}
if (ttlExtendingThread != null && !ttlExtendingThread.isCancelled()) {
logger.debug("Cancelling extending ttl...");
ttlExtendingThread.cancel(true);
}
closeSilently(extendingApiClient);
}
private IOException handleErrResponse(final int responseCode, final InputStream in) throws IOException {
if(in == null) {
return new IOException("Unexpected response code: " + responseCode);
}
TransactionResultEntity errEntity = readResponse(in);
ResponseCode errCode = ResponseCode.fromCode(errEntity.getResponseCode());
switch (errCode) {
case UNKNOWN_PORT:
return new UnknownPortException(errEntity.getMessage());
case PORT_NOT_IN_VALID_STATE:
return new PortNotRunningException(errEntity.getMessage());
default:
return new IOException("Unexpected response code: " + responseCode
+ " errCode:" + errCode + " errMessage:" + errEntity.getMessage());
}
}
private TransactionResultEntity readResponse(InputStream inputStream) throws IOException {
ByteArrayOutputStream bos = new ByteArrayOutputStream();
StreamUtils.copy(inputStream, bos);
String responseMessage = null;
try {
responseMessage = new String(bos.toByteArray(), "UTF-8");
logger.debug("readResponse responseMessage={}", responseMessage);
final ObjectMapper mapper = new ObjectMapper();
return mapper.readValue(responseMessage, TransactionResultEntity.class);
} catch (JsonParseException | JsonMappingException e) {
if (logger.isDebugEnabled()) {
logger.debug("Failed to parse JSON.", e);
}
TransactionResultEntity entity = new TransactionResultEntity();
entity.setResponseCode(ResponseCode.ABORT.getCode());
entity.setMessage(responseMessage);
return entity;
}
}
private String readTransactionUrl(final CloseableHttpResponse response) {
final Header locationUriIntentHeader = response.getFirstHeader(LOCATION_URI_INTENT_NAME);
logger.debug("locationUriIntentHeader={}", locationUriIntentHeader);
if (locationUriIntentHeader != null) {
if (LOCATION_URI_INTENT_VALUE.equals(locationUriIntentHeader.getValue())) {
Header transactionUrl = response.getFirstHeader(LOCATION_HEADER_NAME);
logger.debug("transactionUrl={}", transactionUrl);
if (transactionUrl != null) {
return transactionUrl.getValue();
}
}
}
return null;
}
private void setHandshakeProperties(final HttpRequestBase httpRequest) {
if(compress) httpRequest.setHeader(HANDSHAKE_PROPERTY_USE_COMPRESSION, "true");
if(requestExpirationMillis > 0) httpRequest.setHeader(HANDSHAKE_PROPERTY_REQUEST_EXPIRATION, String.valueOf(requestExpirationMillis));
if(batchCount > 0) httpRequest.setHeader(HANDSHAKE_PROPERTY_BATCH_COUNT, String.valueOf(batchCount));
if(batchSize > 0) httpRequest.setHeader(HANDSHAKE_PROPERTY_BATCH_SIZE, String.valueOf(batchSize));
if(batchDurationMillis > 0) httpRequest.setHeader(HANDSHAKE_PROPERTY_BATCH_DURATION, String.valueOf(batchDurationMillis));
}
private HttpGet createGet(final String path) {
final URI url = getUri(path);
HttpGet get = new HttpGet(url);
get.setConfig(getRequestConfig());
return get;
}
private URI getUri(String path) {
final URI url;
try {
if(HTTP_ABS_URL.matcher(path).find()){
url = new URI(path);
} else {
if(StringUtils.isEmpty(getBaseUrl())){
throw new IllegalStateException("API baseUrl is not resolved yet, call setBaseUrl or resolveBaseUrl before sending requests with relative path.");
}
url = new URI(baseUrl + path);
}
} catch (URISyntaxException e) {
throw new IllegalArgumentException(e.getMessage());
}
return url;
}
private HttpPost createPost(final String path) {
final URI url = getUri(path);
HttpPost post = new HttpPost(url);
post.setConfig(getRequestConfig());
return post;
}
private HttpPut createPut(final String path) {
final URI url = getUri(path);
HttpPut put = new HttpPut(url);
put.setConfig(getRequestConfig());
return put;
}
private HttpDelete createDelete(final String path) {
final URI url = getUri(path);
HttpDelete delete = new HttpDelete(url);
delete.setConfig(getRequestConfig());
return delete;
}
private String execute(final HttpGet get) throws IOException {
CloseableHttpClient httpClient = getHttpClient();
try (CloseableHttpResponse response = httpClient.execute(get)) {
StatusLine statusLine = response.getStatusLine();
int statusCode = statusLine.getStatusCode();
if (RESPONSE_CODE_OK != statusCode) {
throw new HttpGetFailedException(statusCode, statusLine.getReasonPhrase(), null);
}
HttpEntity entity = response.getEntity();
String responseMessage = EntityUtils.toString(entity);
return responseMessage;
}
}
public class HttpGetFailedException extends IOException {
private final int responseCode;
private final String responseMessage;
private final String explanation;
public HttpGetFailedException(final int responseCode, final String responseMessage, final String explanation) {
super("response code " + responseCode + ":" + responseMessage + " with explanation: " + explanation);
this.responseCode = responseCode;
this.responseMessage = responseMessage;
this.explanation = explanation;
}
public int getResponseCode() {
return responseCode;
}
public String getDescription() {
return !isEmpty(explanation) ? explanation : responseMessage;
}
}
private <T> T execute(final HttpGet get, final Class<T> entityClass) throws IOException {
get.setHeader("Accept", "application/json");
final String responseMessage = execute(get);
final ObjectMapper mapper = new ObjectMapper();
mapper.configure(DeserializationConfig.Feature.FAIL_ON_UNKNOWN_PROPERTIES, false);
return mapper.readValue(responseMessage, entityClass);
}
public String getBaseUrl() {
return baseUrl;
}
public void setBaseUrl(final String baseUrl) {
this.baseUrl = baseUrl;
}
public void setConnectTimeoutMillis(int connectTimeoutMillis) {
this.connectTimeoutMillis = connectTimeoutMillis;
}
public void setReadTimeoutMillis(int readTimeoutMillis) {
this.readTimeoutMillis = readTimeoutMillis;
}
public String resolveBaseUrl(String clusterUrl) {
URI clusterUri;
try {
clusterUri = new URI(clusterUrl);
} catch (URISyntaxException e) {
throw new IllegalArgumentException("Specified clusterUrl was: " + clusterUrl, e);
}
return this.resolveBaseUrl(clusterUri);
}
public String resolveBaseUrl(URI clusterUrl) {
String urlPath = clusterUrl.getPath();
if (urlPath.endsWith("/")) {
urlPath = urlPath.substring(0, urlPath.length() - 1);
}
return resolveBaseUrl(clusterUrl.getScheme(), clusterUrl.getHost(), clusterUrl.getPort(), urlPath + "-api");
}
public String resolveBaseUrl(final String scheme, final String host, final int port) {
return resolveBaseUrl(scheme, host, port, "/nifi-api");
}
public String resolveBaseUrl(final String scheme, final String host, final int port, String path) {
String baseUri = scheme + "://" + host + ":" + port + path;
this.setBaseUrl(baseUri);
return baseUri;
}
public void setCompress(boolean compress) {
this.compress = compress;
}
public void setRequestExpirationMillis(long requestExpirationMillis) {
if(requestExpirationMillis < 0) throw new IllegalArgumentException("requestExpirationMillis can't be a negative value.");
this.requestExpirationMillis = requestExpirationMillis;
}
public void setBatchCount(int batchCount) {
if(batchCount < 0) throw new IllegalArgumentException("batchCount can't be a negative value.");
this.batchCount = batchCount;
}
public void setBatchSize(long batchSize) {
if(batchSize < 0) throw new IllegalArgumentException("batchSize can't be a negative value.");
this.batchSize = batchSize;
}
public void setBatchDurationMillis(long batchDurationMillis) {
if(batchDurationMillis < 0) throw new IllegalArgumentException("batchDurationMillis can't be a negative value.");
this.batchDurationMillis = batchDurationMillis;
}
public Integer getTransactionProtocolVersion() {
return transportProtocolVersionNegotiator.getTransactionProtocolVersion();
}
public String getTrustedPeerDn() {
return this.trustedPeerDn;
}
public TransactionResultEntity commitReceivingFlowFiles(String transactionUrl, ResponseCode clientResponse, String checksum) throws IOException {
logger.debug("Sending commitReceivingFlowFiles request to transactionUrl: {}, clientResponse={}, checksum={}",
transactionUrl, clientResponse, checksum);
stopExtendingTtl();
StringBuilder urlBuilder = new StringBuilder(transactionUrl).append("?responseCode=").append(clientResponse.getCode());
if (ResponseCode.CONFIRM_TRANSACTION.equals(clientResponse)) {
urlBuilder.append("&checksum=").append(checksum);
}
HttpDelete delete = createDelete(urlBuilder.toString());
delete.setHeader("Accept", "application/json");
delete.setHeader(HttpHeaders.PROTOCOL_VERSION, String.valueOf(transportProtocolVersionNegotiator.getVersion()));
setHandshakeProperties(delete);
try (CloseableHttpResponse response = getHttpClient().execute(delete)) {
int responseCode = response.getStatusLine().getStatusCode();
logger.debug("commitReceivingFlowFiles responseCode={}", responseCode);
try (InputStream content = response.getEntity().getContent()) {
switch (responseCode) {
case RESPONSE_CODE_OK :
return readResponse(content);
case RESPONSE_CODE_BAD_REQUEST :
return readResponse(content);
default:
throw handleErrResponse(responseCode, content);
}
}
}
}
public TransactionResultEntity commitTransferFlowFiles(String transactionUrl, ResponseCode clientResponse) throws IOException {
String requestUrl = transactionUrl + "?responseCode=" + clientResponse.getCode();
logger.debug("Sending commitTransferFlowFiles request to transactionUrl: {}", requestUrl);
HttpDelete delete = createDelete(requestUrl);
delete.setHeader("Accept", "application/json");
delete.setHeader(HttpHeaders.PROTOCOL_VERSION, String.valueOf(transportProtocolVersionNegotiator.getVersion()));
setHandshakeProperties(delete);
try (CloseableHttpResponse response = getHttpClient().execute(delete)) {
int responseCode = response.getStatusLine().getStatusCode();
logger.debug("commitTransferFlowFiles responseCode={}", responseCode);
try (InputStream content = response.getEntity().getContent()) {
switch (responseCode) {
case RESPONSE_CODE_OK :
return readResponse(content);
case RESPONSE_CODE_BAD_REQUEST :
return readResponse(content);
default:
throw handleErrResponse(responseCode, content);
}
}
}
}
}

View File

@ -0,0 +1,125 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT 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.remote.client;
import org.apache.nifi.remote.PeerDescription;
import org.apache.nifi.remote.PeerStatus;
import org.apache.nifi.remote.TransferDirection;
import org.junit.Test;
import org.mockito.Mockito;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import static java.util.stream.Collectors.groupingBy;
import static java.util.stream.Collectors.reducing;
import static java.util.stream.Collectors.toMap;
import static org.junit.Assert.assertTrue;
public class TestPeerSelector {
private static final Logger logger = LoggerFactory.getLogger(TestPeerSelector.class);
private Map<String, Integer> calculateAverageSelectedCount(Set<PeerStatus> collection, List<PeerStatus> destinations) {
// Calculate hostname entry, for average calculation. Because there're multiple entry with same host name, different port.
final Map<String, Integer> hostNameCounts
= collection.stream().collect(groupingBy(p -> p.getPeerDescription().getHostname(), reducing(0, p -> 1, Integer::sum)));
// Calculate how many times each hostname is selected.
return destinations.stream().collect(groupingBy(p -> p.getPeerDescription().getHostname(), reducing(0, p -> 1, Integer::sum)))
.entrySet().stream().collect(toMap(Map.Entry::getKey, e -> {
return e.getValue() / hostNameCounts.get(e.getKey());
}));
}
@Test
public void testFormulateDestinationListForOutput() throws IOException {
final Set<PeerStatus> collection = new HashSet<>();
collection.add(new PeerStatus(new PeerDescription("HasMedium", 1111, true), 4096));
collection.add(new PeerStatus(new PeerDescription("HasLots", 2222, true), 10240));
collection.add(new PeerStatus(new PeerDescription("HasLittle", 3333, true), 1024));
collection.add(new PeerStatus(new PeerDescription("HasMedium", 4444, true), 4096));
collection.add(new PeerStatus(new PeerDescription("HasMedium", 5555, true), 4096));
PeerStatusProvider peerStatusProvider = Mockito.mock(PeerStatusProvider.class);
PeerSelector peerSelector = new PeerSelector(peerStatusProvider, null);
final List<PeerStatus> destinations = peerSelector.formulateDestinationList(collection, TransferDirection.RECEIVE);
final Map<String, Integer> selectedCounts = calculateAverageSelectedCount(collection, destinations);
logger.info("selectedCounts={}", selectedCounts);
assertTrue("HasLots should send lots", selectedCounts.get("HasLots") > selectedCounts.get("HasMedium"));
assertTrue("HasMedium should send medium", selectedCounts.get("HasMedium") > selectedCounts.get("HasLittle"));
}
@Test
public void testFormulateDestinationListForOutputHugeDifference() throws IOException {
final Set<PeerStatus> collection = new HashSet<>();
collection.add(new PeerStatus(new PeerDescription("HasLittle", 1111, true), 500));
collection.add(new PeerStatus(new PeerDescription("HasLots", 2222, true), 50000));
PeerStatusProvider peerStatusProvider = Mockito.mock(PeerStatusProvider.class);
PeerSelector peerSelector = new PeerSelector(peerStatusProvider, null);
final List<PeerStatus> destinations = peerSelector.formulateDestinationList(collection, TransferDirection.RECEIVE);
final Map<String, Integer> selectedCounts = calculateAverageSelectedCount(collection, destinations);
logger.info("selectedCounts={}", selectedCounts);
assertTrue("HasLots should send lots", selectedCounts.get("HasLots") > selectedCounts.get("HasLittle"));
}
@Test
public void testFormulateDestinationListForInputPorts() throws IOException {
final Set<PeerStatus> collection = new HashSet<>();
collection.add(new PeerStatus(new PeerDescription("HasMedium", 1111, true), 4096));
collection.add(new PeerStatus(new PeerDescription("HasLittle", 2222, true), 10240));
collection.add(new PeerStatus(new PeerDescription("HasLots", 3333, true), 1024));
collection.add(new PeerStatus(new PeerDescription("HasMedium", 4444, true), 4096));
collection.add(new PeerStatus(new PeerDescription("HasMedium", 5555, true), 4096));
PeerStatusProvider peerStatusProvider = Mockito.mock(PeerStatusProvider.class);
PeerSelector peerSelector = new PeerSelector(peerStatusProvider, null);
final List<PeerStatus> destinations = peerSelector.formulateDestinationList(collection, TransferDirection.RECEIVE);
final Map<String, Integer> selectedCounts = calculateAverageSelectedCount(collection, destinations);
logger.info("selectedCounts={}", selectedCounts);
assertTrue("HasLots should get little", selectedCounts.get("HasLots") < selectedCounts.get("HasMedium"));
assertTrue("HasMedium should get medium", selectedCounts.get("HasMedium") < selectedCounts.get("HasLittle"));
}
@Test
public void testFormulateDestinationListForInputPortsHugeDifference() throws IOException {
final Set<PeerStatus> collection = new HashSet<>();
collection.add(new PeerStatus(new PeerDescription("HasLots", 1111, true), 500));
collection.add(new PeerStatus(new PeerDescription("HasLittle", 2222, true), 50000));
PeerStatusProvider peerStatusProvider = Mockito.mock(PeerStatusProvider.class);
PeerSelector peerSelector = new PeerSelector(peerStatusProvider, null);
final List<PeerStatus> destinations = peerSelector.formulateDestinationList(collection, TransferDirection.RECEIVE);
final Map<String, Integer> selectedCounts = calculateAverageSelectedCount(collection, destinations);
logger.info("selectedCounts={}", selectedCounts);
assertTrue("HasLots should get little", selectedCounts.get("HasLots") < selectedCounts.get("HasLittle"));
}
}

View File

@ -0,0 +1,950 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT 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.remote.client.http;
import org.apache.nifi.controller.ScheduledState;
import org.apache.nifi.remote.Peer;
import org.apache.nifi.remote.Transaction;
import org.apache.nifi.remote.TransferDirection;
import org.apache.nifi.remote.client.SiteToSiteClient;
import org.apache.nifi.remote.codec.StandardFlowFileCodec;
import org.apache.nifi.remote.io.CompressionInputStream;
import org.apache.nifi.remote.io.CompressionOutputStream;
import org.apache.nifi.remote.protocol.DataPacket;
import org.apache.nifi.remote.protocol.ResponseCode;
import org.apache.nifi.remote.protocol.SiteToSiteTransportProtocol;
import org.apache.nifi.remote.protocol.http.HttpHeaders;
import org.apache.nifi.remote.util.StandardDataPacket;
import org.apache.nifi.stream.io.ByteArrayInputStream;
import org.apache.nifi.stream.io.ByteArrayOutputStream;
import org.apache.nifi.stream.io.StreamUtils;
import org.apache.nifi.web.api.dto.ControllerDTO;
import org.apache.nifi.web.api.dto.PortDTO;
import org.apache.nifi.web.api.dto.remote.PeerDTO;
import org.apache.nifi.web.api.entity.ControllerEntity;
import org.apache.nifi.web.api.entity.PeersEntity;
import org.apache.nifi.web.api.entity.TransactionResultEntity;
import org.codehaus.jackson.map.ObjectMapper;
import org.eclipse.jetty.server.Server;
import org.eclipse.jetty.servlet.ServletContextHandler;
import org.eclipse.jetty.servlet.ServletHandler;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import javax.servlet.ServletException;
import javax.servlet.ServletOutputStream;
import javax.servlet.http.HttpServlet;
import javax.servlet.http.HttpServletRequest;
import javax.servlet.http.HttpServletResponse;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.net.SocketTimeoutException;
import java.net.URI;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import static org.apache.commons.lang3.StringUtils.isEmpty;
import static org.apache.nifi.remote.protocol.http.HttpHeaders.LOCATION_HEADER_NAME;
import static org.apache.nifi.remote.protocol.http.HttpHeaders.LOCATION_URI_INTENT_NAME;
import static org.apache.nifi.remote.protocol.http.HttpHeaders.LOCATION_URI_INTENT_VALUE;
import static org.apache.nifi.remote.protocol.http.HttpHeaders.PROTOCOL_VERSION;
import static org.apache.nifi.remote.protocol.http.HttpHeaders.SERVER_SIDE_TRANSACTION_TTL;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
public class TestHttpClient {
private static Logger logger = LoggerFactory.getLogger(TestHttpClient.class);
private static Server server;
final private static AtomicBoolean isTestCaseFinished = new AtomicBoolean(false);
private static Set<PortDTO> inputPorts;
private static Set<PortDTO> outputPorts;
private static Set<PeerDTO> peers;
private static String serverChecksum;
public static class SiteInfoServlet extends HttpServlet {
@Override
protected void doGet(HttpServletRequest req, HttpServletResponse resp) throws ServletException, IOException {
final ControllerDTO controller = new ControllerDTO();
controller.setRemoteSiteHttpListeningPort(server.getURI().getPort());
controller.setId("remote-controller-id");
controller.setInstanceId("remote-instance-id");
controller.setName("Remote NiFi Flow");
controller.setSiteToSiteSecure(false);
assertNotNull("Test case should set <inputPorts> depending on the test scenario.", inputPorts);
controller.setInputPorts(inputPorts);
controller.setInputPortCount(inputPorts.size());
assertNotNull("Test case should set <outputPorts> depending on the test scenario.", outputPorts);
controller.setOutputPorts(outputPorts);
controller.setOutputPortCount(outputPorts.size());
final ControllerEntity controllerEntity = new ControllerEntity();
controllerEntity.setController(controller);
respondWithJson(resp, controllerEntity);
}
}
public static class PeersServlet extends HttpServlet {
@Override
protected void doGet(HttpServletRequest req, HttpServletResponse resp) throws ServletException, IOException {
final PeersEntity peersEntity = new PeersEntity();
assertNotNull("Test case should set <peers> depending on the test scenario.", peers);
peersEntity.setPeers(peers);
respondWithJson(resp, peersEntity);
}
}
public static class PortTransactionsServlet extends HttpServlet {
@Override
protected void doPost(HttpServletRequest req, HttpServletResponse resp) throws ServletException, IOException {
final int reqProtocolVersion = getReqProtocolVersion(req);
TransactionResultEntity entity = new TransactionResultEntity();
entity.setResponseCode(ResponseCode.PROPERTIES_OK.getCode());
entity.setMessage("A transaction is created.");
resp.setHeader(LOCATION_URI_INTENT_NAME, LOCATION_URI_INTENT_VALUE);
resp.setHeader(LOCATION_HEADER_NAME, req.getRequestURL() + "/transaction-id");
setCommonResponseHeaders(resp, reqProtocolVersion);
respondWithJson(resp, entity, HttpServletResponse.SC_CREATED);
}
}
public static class InputPortTransactionServlet extends HttpServlet {
@Override
protected void doPut(HttpServletRequest req, HttpServletResponse resp) throws ServletException, IOException {
final int reqProtocolVersion = getReqProtocolVersion(req);
final TransactionResultEntity entity = new TransactionResultEntity();
entity.setResponseCode(ResponseCode.CONTINUE_TRANSACTION.getCode());
entity.setMessage("Extended TTL.");
setCommonResponseHeaders(resp, reqProtocolVersion);
respondWithJson(resp, entity, HttpServletResponse.SC_OK);
}
@Override
protected void doDelete(HttpServletRequest req, HttpServletResponse resp) throws ServletException, IOException {
final int reqProtocolVersion = getReqProtocolVersion(req);
TransactionResultEntity entity = new TransactionResultEntity();
entity.setResponseCode(ResponseCode.TRANSACTION_FINISHED.getCode());
entity.setMessage("The transaction is finished.");
setCommonResponseHeaders(resp, reqProtocolVersion);
respondWithJson(resp, entity, HttpServletResponse.SC_OK);
}
}
public static class OutputPortTransactionServlet extends HttpServlet {
@Override
protected void doPut(HttpServletRequest req, HttpServletResponse resp) throws ServletException, IOException {
final int reqProtocolVersion = getReqProtocolVersion(req);
final TransactionResultEntity entity = new TransactionResultEntity();
entity.setResponseCode(ResponseCode.CONTINUE_TRANSACTION.getCode());
entity.setMessage("Extended TTL.");
setCommonResponseHeaders(resp, reqProtocolVersion);
respondWithJson(resp, entity, HttpServletResponse.SC_OK);
}
@Override
protected void doDelete(HttpServletRequest req, HttpServletResponse resp) throws ServletException, IOException {
final int reqProtocolVersion = getReqProtocolVersion(req);
TransactionResultEntity entity = new TransactionResultEntity();
entity.setResponseCode(ResponseCode.CONFIRM_TRANSACTION.getCode());
entity.setMessage("The transaction is confirmed.");
setCommonResponseHeaders(resp, reqProtocolVersion);
respondWithJson(resp, entity, HttpServletResponse.SC_OK);
}
}
public static class FlowFilesServlet extends HttpServlet {
@Override
protected void doPost(HttpServletRequest req, HttpServletResponse resp) throws ServletException, IOException {
final int reqProtocolVersion = getReqProtocolVersion(req);
setCommonResponseHeaders(resp, reqProtocolVersion);
DataPacket dataPacket;
while ((dataPacket = readIncomingPacket(req)) != null) {
logger.info("received {}", dataPacket);
consumeDataPacket(dataPacket);
}
logger.info("finish receiving data packets.");
assertNotNull("Test case should set <serverChecksum> depending on the test scenario.", serverChecksum);
respondWithText(resp, serverChecksum, HttpServletResponse.SC_ACCEPTED);
}
@Override
protected void doGet(HttpServletRequest req, HttpServletResponse resp) throws ServletException, IOException {
final int reqProtocolVersion = getReqProtocolVersion(req);
resp.setStatus(HttpServletResponse.SC_ACCEPTED);
resp.setContentType("application/octet-stream");
setCommonResponseHeaders(resp, reqProtocolVersion);
final OutputStream outputStream = getOutputStream(req, resp);
writeOutgoingPacket(outputStream);
writeOutgoingPacket(outputStream);
writeOutgoingPacket(outputStream);
resp.flushBuffer();
}
}
public static class FlowFilesTimeoutServlet extends FlowFilesServlet {
@Override
protected void doPost(HttpServletRequest req, HttpServletResponse resp) throws ServletException, IOException {
sleepUntilTestCaseFinish();
super.doPost(req, resp);
}
@Override
protected void doGet(HttpServletRequest req, HttpServletResponse resp) throws ServletException, IOException {
sleepUntilTestCaseFinish();
super.doGet(req, resp);
}
}
public static class FlowFilesTimeoutAfterDataExchangeServlet extends HttpServlet {
@Override
protected void doPost(HttpServletRequest req, HttpServletResponse resp) throws ServletException, IOException {
final int reqProtocolVersion = getReqProtocolVersion(req);
setCommonResponseHeaders(resp, reqProtocolVersion);
consumeDataPacket(readIncomingPacket(req));
sleepUntilTestCaseFinish();
}
@Override
protected void doGet(HttpServletRequest req, HttpServletResponse resp) throws ServletException, IOException {
final int reqProtocolVersion = getReqProtocolVersion(req);
resp.setStatus(HttpServletResponse.SC_ACCEPTED);
resp.setContentType("application/octet-stream");
setCommonResponseHeaders(resp, reqProtocolVersion);
writeOutgoingPacket(getOutputStream(req, resp));
sleepUntilTestCaseFinish();
}
}
private static void sleepUntilTestCaseFinish() {
while (!isTestCaseFinished.get()) {
try {
logger.info("Sleeping...");
Thread.sleep(1000);
} catch (InterruptedException e) {
logger.info("Got an exception while sleeping.", e);
break;
}
}
}
private static void writeOutgoingPacket(OutputStream outputStream) throws IOException {
final DataPacket packet = new DataPacketBuilder()
.contents("Example contents from server.")
.attr("Server attr 1", "Server attr 1 value")
.attr("Server attr 2", "Server attr 2 value")
.build();
new StandardFlowFileCodec().encode(packet, outputStream);
outputStream.flush();
}
private static OutputStream getOutputStream(HttpServletRequest req, HttpServletResponse resp) throws IOException {
OutputStream outputStream = resp.getOutputStream();
if (Boolean.valueOf(req.getHeader(HttpHeaders.HANDSHAKE_PROPERTY_USE_COMPRESSION))){
outputStream = new CompressionOutputStream(outputStream);
}
return outputStream;
}
private static DataPacket readIncomingPacket(HttpServletRequest req) throws IOException {
final StandardFlowFileCodec codec = new StandardFlowFileCodec();
InputStream inputStream = req.getInputStream();
if (Boolean.valueOf(req.getHeader(HttpHeaders.HANDSHAKE_PROPERTY_USE_COMPRESSION))){
inputStream = new CompressionInputStream(inputStream);
}
return codec.decode(inputStream);
}
private static int getReqProtocolVersion(HttpServletRequest req) {
final String reqProtocolVersionStr = req.getHeader(PROTOCOL_VERSION);
assertTrue(!isEmpty(reqProtocolVersionStr));
return Integer.parseInt(reqProtocolVersionStr);
}
private static void setCommonResponseHeaders(HttpServletResponse resp, int reqProtocolVersion) {
resp.setHeader(PROTOCOL_VERSION, String.valueOf(reqProtocolVersion));
resp.setHeader(SERVER_SIDE_TRANSACTION_TTL, "3");
}
private static void respondWithJson(HttpServletResponse resp, Object entity) throws IOException {
respondWithJson(resp, entity, HttpServletResponse.SC_OK);
}
private static void respondWithJson(HttpServletResponse resp, Object entity, int statusCode) throws IOException {
resp.setContentType("application/json");
resp.setStatus(statusCode);
final ServletOutputStream out = resp.getOutputStream();
new ObjectMapper().writer().writeValue(out, entity);
out.flush();
}
private static void respondWithText(HttpServletResponse resp, String result, int statusCode) throws IOException {
resp.setContentType("text/plain");
resp.setStatus(statusCode);
final ServletOutputStream out = resp.getOutputStream();
out.write(result.getBytes());
out.flush();
}
@BeforeClass
public static void setup() throws Exception {
// Create embedded Jetty server
server = new Server(0);
ServletContextHandler contextHandler = new ServletContextHandler();
contextHandler.setContextPath("/nifi-api");
server.setHandler(contextHandler);
ServletHandler servletHandler = new ServletHandler();
contextHandler.insertHandler(servletHandler);
servletHandler.addServletWithMapping(SiteInfoServlet.class, "/site-to-site");
servletHandler.addServletWithMapping(PeersServlet.class, "/site-to-site/peers");
servletHandler.addServletWithMapping(PortTransactionsServlet.class, "/site-to-site/input-ports/input-running-id/transactions");
servletHandler.addServletWithMapping(InputPortTransactionServlet.class, "/site-to-site/input-ports/input-running-id/transactions/transaction-id");
servletHandler.addServletWithMapping(FlowFilesServlet.class, "/site-to-site/input-ports/input-running-id/transactions/transaction-id/flow-files");
servletHandler.addServletWithMapping(PortTransactionsServlet.class, "/site-to-site/input-ports/input-timeout-id/transactions");
servletHandler.addServletWithMapping(InputPortTransactionServlet.class, "/site-to-site/input-ports/input-timeout-id/transactions/transaction-id");
servletHandler.addServletWithMapping(FlowFilesTimeoutServlet.class, "/site-to-site/input-ports/input-timeout-id/transactions/transaction-id/flow-files");
servletHandler.addServletWithMapping(PortTransactionsServlet.class, "/site-to-site/input-ports/input-timeout-data-ex-id/transactions");
servletHandler.addServletWithMapping(InputPortTransactionServlet.class, "/site-to-site/input-ports/input-timeout-data-ex-id/transactions/transaction-id");
servletHandler.addServletWithMapping(FlowFilesTimeoutAfterDataExchangeServlet.class, "/site-to-site/input-ports/input-timeout-data-ex-id/transactions/transaction-id/flow-files");
servletHandler.addServletWithMapping(PortTransactionsServlet.class, "/site-to-site/output-ports/output-running-id/transactions");
servletHandler.addServletWithMapping(OutputPortTransactionServlet.class, "/site-to-site/output-ports/output-running-id/transactions/transaction-id");
servletHandler.addServletWithMapping(FlowFilesServlet.class, "/site-to-site/output-ports/output-running-id/transactions/transaction-id/flow-files");
servletHandler.addServletWithMapping(PortTransactionsServlet.class, "/site-to-site/output-ports/output-timeout-id/transactions");
servletHandler.addServletWithMapping(OutputPortTransactionServlet.class, "/site-to-site/output-ports/output-timeout-id/transactions/transaction-id");
servletHandler.addServletWithMapping(FlowFilesTimeoutServlet.class, "/site-to-site/output-ports/output-timeout-id/transactions/transaction-id/flow-files");
servletHandler.addServletWithMapping(PortTransactionsServlet.class, "/site-to-site/output-ports/output-timeout-data-ex-id/transactions");
servletHandler.addServletWithMapping(OutputPortTransactionServlet.class, "/site-to-site/output-ports/output-timeout-data-ex-id/transactions/transaction-id");
servletHandler.addServletWithMapping(FlowFilesTimeoutAfterDataExchangeServlet.class, "/site-to-site/output-ports/output-timeout-data-ex-id/transactions/transaction-id/flow-files");
server.start();
int serverPort = server.getURI().getPort();
logger.info("Starting server on port {}", serverPort);
}
@AfterClass
public static void teardown() throws Exception {
logger.info("Stopping server.");
server.stop();
}
private static class DataPacketBuilder {
private final Map<String, String> attributes = new HashMap<>();
private String contents;
private DataPacketBuilder attr(final String k, final String v) {
attributes.put(k, v);
return this;
}
private DataPacketBuilder contents(final String contents) {
this.contents = contents;
return this;
}
private DataPacket build() {
byte[] bytes = contents.getBytes();
return new StandardDataPacket(attributes, new ByteArrayInputStream(bytes), bytes.length);
}
}
@Before
public void before() throws Exception {
System.setProperty("org.slf4j.simpleLogger.log.org.apache.nifi.remote", "TRACE");
System.setProperty("org.slf4j.simpleLogger.log.org.apache.nifi.remote.protocol.http.HttpClientTransaction", "DEBUG");
final URI uri = server.getURI();
final PeerDTO peer = new PeerDTO();
peer.setHostname(uri.getHost());
peer.setPort(uri.getPort());
peer.setFlowFileCount(10);
peer.setSecure(false);
isTestCaseFinished.set(false);
peers = new HashSet<>();
peers.add(peer);
inputPorts = new HashSet<>();
final PortDTO runningInputPort = new PortDTO();
runningInputPort.setId("running-input-port");
inputPorts.add(runningInputPort);
runningInputPort.setName("input-running");
runningInputPort.setId("input-running-id");
runningInputPort.setType("INPUT_PORT");
runningInputPort.setState(ScheduledState.RUNNING.name());
final PortDTO timeoutInputPort = new PortDTO();
timeoutInputPort.setId("timeout-input-port");
inputPorts.add(timeoutInputPort);
timeoutInputPort.setName("input-timeout");
timeoutInputPort.setId("input-timeout-id");
timeoutInputPort.setType("INPUT_PORT");
timeoutInputPort.setState(ScheduledState.RUNNING.name());
final PortDTO timeoutDataExInputPort = new PortDTO();
timeoutDataExInputPort.setId("timeout-dataex-input-port");
inputPorts.add(timeoutDataExInputPort);
timeoutDataExInputPort.setName("input-timeout-data-ex");
timeoutDataExInputPort.setId("input-timeout-data-ex-id");
timeoutDataExInputPort.setType("INPUT_PORT");
timeoutDataExInputPort.setState(ScheduledState.RUNNING.name());
outputPorts = new HashSet<>();
final PortDTO runningOutputPort = new PortDTO();
runningOutputPort.setId("running-output-port");
outputPorts.add(runningOutputPort);
runningOutputPort.setName("output-running");
runningOutputPort.setId("output-running-id");
runningOutputPort.setType("OUTPUT_PORT");
runningOutputPort.setState(ScheduledState.RUNNING.name());
final PortDTO timeoutOutputPort = new PortDTO();
timeoutOutputPort.setId("timeout-output-port");
outputPorts.add(timeoutOutputPort);
timeoutOutputPort.setName("output-timeout");
timeoutOutputPort.setId("output-timeout-id");
timeoutOutputPort.setType("OUTPUT_PORT");
timeoutOutputPort.setState(ScheduledState.RUNNING.name());
final PortDTO timeoutDataExOutputPort = new PortDTO();
timeoutDataExOutputPort.setId("timeout-dataex-output-port");
outputPorts.add(timeoutDataExOutputPort);
timeoutDataExOutputPort.setName("output-timeout-data-ex");
timeoutDataExOutputPort.setId("output-timeout-data-ex-id");
timeoutDataExOutputPort.setType("OUTPUT_PORT");
timeoutDataExOutputPort.setState(ScheduledState.RUNNING.name());
}
@After
public void after() throws Exception {
isTestCaseFinished.set(true);
}
private SiteToSiteClient.Builder getDefaultBuilder() {
final URI uri = server.getURI();
return new SiteToSiteClient.Builder().transportProtocol(SiteToSiteTransportProtocol.HTTP)
.url("http://" + uri.getHost() + ":" + uri.getPort() + "/nifi")
;
}
private static void consumeDataPacket(DataPacket packet) throws IOException {
final ByteArrayOutputStream bos = new ByteArrayOutputStream();
StreamUtils.copy(packet.getData(), bos);
String contents = new String(bos.toByteArray());
logger.info("received: {}, {}", contents, packet.getAttributes());
}
@Test
public void testUnkownClusterUrl() throws Exception {
final URI uri = server.getURI();
try (
SiteToSiteClient client = getDefaultBuilder()
.url("http://" + uri.getHost() + ":" + uri.getPort() + "/unkown")
.portName("input-running")
.build()
) {
final Transaction transaction = client.createTransaction(TransferDirection.SEND);
assertNull(transaction);
}
}
@Test
public void testNoAvailablePeer() throws Exception {
peers = new HashSet<>();
try (
SiteToSiteClient client = getDefaultBuilder()
.portName("input-running")
.build()
) {
final Transaction transaction = client.createTransaction(TransferDirection.SEND);
assertNull(transaction);
}
}
@Test
public void testSendUnknownPort() throws Exception {
try (
SiteToSiteClient client = getDefaultBuilder()
.portName("input-unknown")
.build()
) {
try {
client.createTransaction(TransferDirection.SEND);
fail();
} catch (IOException e) {
logger.info("Exception message: {}", e.getMessage());
assertTrue(e.getMessage().contains("Failed to determine the identifier of port"));
}
}
}
@Test
public void testSendSuccess() throws Exception {
final URI uri = server.getURI();
logger.info("uri={}", uri);
try (
SiteToSiteClient client = getDefaultBuilder()
.portName("input-running")
.build()
) {
final Transaction transaction = client.createTransaction(TransferDirection.SEND);
assertNotNull(transaction);
serverChecksum = "1071206772";
for (int i = 0; i < 20; i++) {
DataPacket packet = new DataPacketBuilder()
.contents("Example contents from client.")
.attr("Client attr 1", "Client attr 1 value")
.attr("Client attr 2", "Client attr 2 value")
.build();
transaction.send(packet);
long written = ((Peer)transaction.getCommunicant()).getCommunicationsSession().getBytesWritten();
logger.info("{}: {} bytes have been written.", i, written);
}
transaction.confirm();
transaction.complete();
}
}
@Test
public void testSendSuccessCompressed() throws Exception {
final URI uri = server.getURI();
logger.info("uri={}", uri);
try (
SiteToSiteClient client = getDefaultBuilder()
.portName("input-running")
.useCompression(true)
.build()
) {
final Transaction transaction = client.createTransaction(TransferDirection.SEND);
assertNotNull(transaction);
serverChecksum = "1071206772";
for (int i = 0; i < 20; i++) {
DataPacket packet = new DataPacketBuilder()
.contents("Example contents from client.")
.attr("Client attr 1", "Client attr 1 value")
.attr("Client attr 2", "Client attr 2 value")
.build();
transaction.send(packet);
long written = ((Peer)transaction.getCommunicant()).getCommunicationsSession().getBytesWritten();
logger.info("{}: {} bytes have been written.", i, written);
}
transaction.confirm();
transaction.complete();
}
}
@Test
public void testSendSlowClientSuccess() throws Exception {
final URI uri = server.getURI();
logger.info("uri={}", uri);
try (
SiteToSiteClient client = getDefaultBuilder()
.idleExpiration(1000, TimeUnit.MILLISECONDS)
.portName("input-running")
.build()
) {
final Transaction transaction = client.createTransaction(TransferDirection.SEND);
assertNotNull(transaction);
serverChecksum = "3882825556";
for (int i = 0; i < 3; i++) {
DataPacket packet = new DataPacketBuilder()
.contents("Example contents from client.")
.attr("Client attr 1", "Client attr 1 value")
.attr("Client attr 2", "Client attr 2 value")
.build();
transaction.send(packet);
long written = ((Peer)transaction.getCommunicant()).getCommunicationsSession().getBytesWritten();
logger.info("{} bytes have been written.", written);
Thread.sleep(50);
}
transaction.confirm();
transaction.complete();
}
}
private void completeShouldFail(Transaction transaction) throws IOException {
try {
transaction.complete();
fail("Complete operation should fail since transaction has already failed.");
} catch (IllegalStateException e) {
logger.info("An exception was thrown as expected.", e);
}
}
private void confirmShouldFail(Transaction transaction) throws IOException {
try {
transaction.confirm();
fail("Confirm operation should fail since transaction has already failed.");
} catch (IllegalStateException e) {
logger.info("An exception was thrown as expected.", e);
}
}
@Test
public void testSendTimeout() throws Exception {
final URI uri = server.getURI();
logger.info("uri={}", uri);
try (
SiteToSiteClient client = getDefaultBuilder()
.timeout(1, TimeUnit.SECONDS)
.portName("input-timeout")
.build()
) {
final Transaction transaction = client.createTransaction(TransferDirection.SEND);
assertNotNull(transaction);
DataPacket packet = new DataPacketBuilder()
.contents("Example contents from client.")
.attr("Client attr 1", "Client attr 1 value")
.attr("Client attr 2", "Client attr 2 value")
.build();
serverChecksum = "1345413116";
transaction.send(packet);
try {
transaction.confirm();
fail();
} catch (IOException e) {
logger.info("An exception was thrown as expected.", e);
assertTrue(e.getMessage().contains("TimeoutException"));
}
completeShouldFail(transaction);
}
}
@Test
public void testSendTimeoutAfterDataExchange() throws Exception {
System.setProperty("org.slf4j.simpleLogger.log.org.apache.nifi.remote.protocol.http.HttpClientTransaction", "INFO");
final URI uri = server.getURI();
logger.info("uri={}", uri);
try (
SiteToSiteClient client = getDefaultBuilder()
.idleExpiration(500, TimeUnit.MILLISECONDS)
.timeout(500, TimeUnit.MILLISECONDS)
.portName("input-timeout-data-ex")
.build()
) {
final Transaction transaction = client.createTransaction(TransferDirection.SEND);
assertNotNull(transaction);
DataPacket packet = new DataPacketBuilder()
.contents("Example contents from client.")
.attr("Client attr 1", "Client attr 1 value")
.attr("Client attr 2", "Client attr 2 value")
.build();
for(int i = 0; i < 100; i++) {
transaction.send(packet);
if (i % 10 == 0) {
logger.info("Sent {} packets...", i);
}
}
try {
confirmShouldFail(transaction);
fail("Should be timeout.");
} catch (IOException e) {
logger.info("Exception message: {}", e.getMessage());
assertTrue(e.getMessage().contains("TimeoutException"));
}
completeShouldFail(transaction);
}
}
@Test
public void testReceiveUnknownPort() throws Exception {
try (
SiteToSiteClient client = getDefaultBuilder()
.portName("output-unknown")
.build()
) {
try {
client.createTransaction(TransferDirection.RECEIVE);
fail();
} catch (IOException e) {
logger.info("Exception message: {}", e.getMessage());
assertTrue(e.getMessage().contains("Failed to determine the identifier of port"));
}
}
}
@Test
public void testReceiveSuccess() throws Exception {
final URI uri = server.getURI();
logger.info("uri={}", uri);
try (
SiteToSiteClient client = getDefaultBuilder()
.portName("output-running")
.build()
) {
final Transaction transaction = client.createTransaction(TransferDirection.RECEIVE);
assertNotNull(transaction);
DataPacket packet;
while ((packet = transaction.receive()) != null) {
consumeDataPacket(packet);
}
transaction.confirm();
transaction.complete();
}
}
@Test
public void testReceiveSuccessCompressed() throws Exception {
final URI uri = server.getURI();
logger.info("uri={}", uri);
try (
SiteToSiteClient client = getDefaultBuilder()
.portName("output-running")
.useCompression(true)
.build()
) {
final Transaction transaction = client.createTransaction(TransferDirection.RECEIVE);
assertNotNull(transaction);
DataPacket packet;
while ((packet = transaction.receive()) != null) {
consumeDataPacket(packet);
}
transaction.confirm();
transaction.complete();
}
}
@Test
public void testReceiveSlowClientSuccess() throws Exception {
final URI uri = server.getURI();
logger.info("uri={}", uri);
try (
SiteToSiteClient client = getDefaultBuilder()
.portName("output-running")
.build()
) {
final Transaction transaction = client.createTransaction(TransferDirection.RECEIVE);
assertNotNull(transaction);
DataPacket packet;
while ((packet = transaction.receive()) != null) {
consumeDataPacket(packet);
Thread.sleep(500);
}
transaction.confirm();
transaction.complete();
}
}
@Test
public void testReceiveTimeout() throws Exception {
final URI uri = server.getURI();
logger.info("uri={}", uri);
try (
SiteToSiteClient client = getDefaultBuilder()
.timeout(1, TimeUnit.SECONDS)
.portName("output-timeout")
.build()
) {
try {
client.createTransaction(TransferDirection.RECEIVE);
fail();
} catch (IOException e) {
logger.info("An exception was thrown as expected.", e);
assertTrue(e instanceof SocketTimeoutException);
}
}
}
@Test
public void testReceiveTimeoutAfterDataExchange() throws Exception {
final URI uri = server.getURI();
logger.info("uri={}", uri);
try (
SiteToSiteClient client = getDefaultBuilder()
.timeout(1, TimeUnit.SECONDS)
.portName("output-timeout-data-ex")
.build()
) {
final Transaction transaction = client.createTransaction(TransferDirection.RECEIVE);
assertNotNull(transaction);
DataPacket packet = transaction.receive();
assertNotNull(packet);
consumeDataPacket(packet);
try {
transaction.receive();
fail();
} catch (IOException e) {
logger.info("An exception was thrown as expected.", e);
assertTrue(e.getCause() instanceof SocketTimeoutException);
}
confirmShouldFail(transaction);
completeShouldFail(transaction);
}
}
}

View File

@ -1,92 +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.remote.client.socket;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import org.apache.nifi.remote.PeerStatus;
import org.apache.nifi.remote.TransferDirection;
import org.apache.nifi.remote.cluster.ClusterNodeInformation;
import org.apache.nifi.remote.cluster.NodeInformation;
import org.junit.Test;
public class TestEndpointConnectionStatePool {
@Test
public void testFormulateDestinationListForOutput() throws IOException {
final ClusterNodeInformation clusterNodeInfo = new ClusterNodeInformation();
final List<NodeInformation> collection = new ArrayList<>();
collection.add(new NodeInformation("ShouldGetMedium", 1, 1111, true, 4096));
collection.add(new NodeInformation("ShouldGetLots", 2, 2222, true, 10240));
collection.add(new NodeInformation("ShouldGetLittle", 3, 3333, true, 1024));
collection.add(new NodeInformation("ShouldGetMedium", 4, 4444, true, 4096));
collection.add(new NodeInformation("ShouldGetMedium", 5, 5555, true, 4096));
clusterNodeInfo.setNodeInformation(collection);
final List<PeerStatus> destinations = EndpointConnectionPool.formulateDestinationList(clusterNodeInfo, TransferDirection.RECEIVE);
for (final PeerStatus peerStatus : destinations) {
System.out.println(peerStatus.getPeerDescription());
}
}
@Test
public void testFormulateDestinationListForOutputHugeDifference() throws IOException {
final ClusterNodeInformation clusterNodeInfo = new ClusterNodeInformation();
final List<NodeInformation> collection = new ArrayList<>();
collection.add(new NodeInformation("ShouldGetLittle", 1, 1111, true, 500));
collection.add(new NodeInformation("ShouldGetLots", 2, 2222, true, 50000));
clusterNodeInfo.setNodeInformation(collection);
final List<PeerStatus> destinations = EndpointConnectionPool.formulateDestinationList(clusterNodeInfo, TransferDirection.RECEIVE);
for (final PeerStatus peerStatus : destinations) {
System.out.println(peerStatus.getPeerDescription());
}
}
@Test
public void testFormulateDestinationListForInputPorts() throws IOException {
final ClusterNodeInformation clusterNodeInfo = new ClusterNodeInformation();
final List<NodeInformation> collection = new ArrayList<>();
collection.add(new NodeInformation("ShouldGetMedium", 1, 1111, true, 4096));
collection.add(new NodeInformation("ShouldGetLittle", 2, 2222, true, 10240));
collection.add(new NodeInformation("ShouldGetLots", 3, 3333, true, 1024));
collection.add(new NodeInformation("ShouldGetMedium", 4, 4444, true, 4096));
collection.add(new NodeInformation("ShouldGetMedium", 5, 5555, true, 4096));
clusterNodeInfo.setNodeInformation(collection);
final List<PeerStatus> destinations = EndpointConnectionPool.formulateDestinationList(clusterNodeInfo, TransferDirection.SEND);
for (final PeerStatus peerStatus : destinations) {
System.out.println(peerStatus.getPeerDescription());
}
}
@Test
public void testFormulateDestinationListForInputPortsHugeDifference() throws IOException {
final ClusterNodeInformation clusterNodeInfo = new ClusterNodeInformation();
final List<NodeInformation> collection = new ArrayList<>();
collection.add(new NodeInformation("ShouldGetLots", 1, 1111, true, 500));
collection.add(new NodeInformation("ShouldGetLittle", 2, 2222, true, 50000));
clusterNodeInfo.setNodeInformation(collection);
final List<PeerStatus> destinations = EndpointConnectionPool.formulateDestinationList(clusterNodeInfo, TransferDirection.SEND);
for (final PeerStatus peerStatus : destinations) {
System.out.println(peerStatus.getPeerDescription());
}
}
}

View File

@ -0,0 +1,237 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT 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.remote.protocol;
import org.apache.nifi.remote.Transaction;
import org.apache.nifi.remote.TransactionCompletion;
import org.apache.nifi.remote.util.StandardDataPacket;
import org.apache.nifi.stream.io.ByteArrayInputStream;
import org.apache.nifi.stream.io.ByteArrayOutputStream;
import org.apache.nifi.stream.io.StreamUtils;
import java.io.IOException;
import java.io.UnsupportedEncodingException;
import java.util.HashMap;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
public class SiteToSiteTestUtils {
public static DataPacket createDataPacket(String contents) {
try {
byte[] bytes = contents.getBytes("UTF-8");
ByteArrayInputStream is = new ByteArrayInputStream(bytes);
return new StandardDataPacket(new HashMap<>(), is, bytes.length);
} catch (UnsupportedEncodingException e){
throw new RuntimeException(e);
}
}
public static String readContents(DataPacket packet) throws IOException {
ByteArrayOutputStream os = new ByteArrayOutputStream((int) packet.getSize());
StreamUtils.copy(packet.getData(), os);
return new String(os.toByteArray(), "UTF-8");
}
public static void execReceiveZeroFlowFile(Transaction transaction) throws IOException {
assertEquals(Transaction.TransactionState.TRANSACTION_STARTED, transaction.getState());
DataPacket packet = transaction.receive();
assertNull(packet);
transaction.confirm();
assertEquals(Transaction.TransactionState.TRANSACTION_CONFIRMED, transaction.getState());
TransactionCompletion completion = transaction.complete();
assertEquals(Transaction.TransactionState.TRANSACTION_COMPLETED, transaction.getState());
assertFalse("Should NOT be backoff", completion.isBackoff());
assertEquals(0, completion.getDataPacketsTransferred());
}
public static void execReceiveOneFlowFile(Transaction transaction) throws IOException {
assertEquals(Transaction.TransactionState.TRANSACTION_STARTED, transaction.getState());
DataPacket packet = transaction.receive();
assertNotNull(packet);
assertEquals("contents on server 1", readContents(packet));
assertEquals(Transaction.TransactionState.DATA_EXCHANGED, transaction.getState());
packet = transaction.receive();
assertNull(packet);
transaction.confirm();
assertEquals(Transaction.TransactionState.TRANSACTION_CONFIRMED, transaction.getState());
TransactionCompletion completion = transaction.complete();
assertEquals(Transaction.TransactionState.TRANSACTION_COMPLETED, transaction.getState());
assertFalse("Should NOT be backoff", completion.isBackoff());
assertEquals(1, completion.getDataPacketsTransferred());
}
public static void execReceiveTwoFlowFiles(Transaction transaction) throws IOException {
DataPacket packet = transaction.receive();
assertNotNull(packet);
assertEquals("contents on server 1", readContents(packet));
assertEquals(Transaction.TransactionState.DATA_EXCHANGED, transaction.getState());
packet = transaction.receive();
assertNotNull(packet);
assertEquals("contents on server 2", readContents(packet));
assertEquals(Transaction.TransactionState.DATA_EXCHANGED, transaction.getState());
packet = transaction.receive();
assertNull(packet);
transaction.confirm();
assertEquals(Transaction.TransactionState.TRANSACTION_CONFIRMED, transaction.getState());
TransactionCompletion completion = transaction.complete();
assertEquals(Transaction.TransactionState.TRANSACTION_COMPLETED, transaction.getState());
assertFalse("Should NOT be backoff", completion.isBackoff());
assertEquals(2, completion.getDataPacketsTransferred());
}
public static void execReceiveWithInvalidChecksum(Transaction transaction) throws IOException {
assertEquals(Transaction.TransactionState.TRANSACTION_STARTED, transaction.getState());
DataPacket packet = transaction.receive();
assertNotNull(packet);
assertEquals("contents on server 1", readContents(packet));
assertEquals(Transaction.TransactionState.DATA_EXCHANGED, transaction.getState());
packet = transaction.receive();
assertNotNull(packet);
assertEquals("contents on server 2", readContents(packet));
assertEquals(Transaction.TransactionState.DATA_EXCHANGED, transaction.getState());
packet = transaction.receive();
assertNull(packet);
try {
transaction.confirm();
fail();
} catch (IOException e){
assertTrue(e.getMessage().contains("Received a BadChecksum response"));
assertEquals(Transaction.TransactionState.ERROR, transaction.getState());
}
try {
transaction.complete();
fail("It's not confirmed.");
} catch (IllegalStateException e){
assertEquals(Transaction.TransactionState.ERROR, transaction.getState());
}
}
public static void execSendZeroFlowFile(Transaction transaction) throws IOException {
assertEquals(Transaction.TransactionState.TRANSACTION_STARTED, transaction.getState());
try {
transaction.confirm();
fail("Nothing has been sent.");
} catch (IllegalStateException e){
}
try {
transaction.complete();
fail("Nothing has been sent.");
} catch (IllegalStateException e){
}
}
public static void execSendOneFlowFile(Transaction transaction) throws IOException {
assertEquals(Transaction.TransactionState.TRANSACTION_STARTED, transaction.getState());
DataPacket packet = createDataPacket("contents on client 1");
transaction.send(packet);
transaction.confirm();
assertEquals(Transaction.TransactionState.TRANSACTION_CONFIRMED, transaction.getState());
TransactionCompletion completion = transaction.complete();
assertEquals(Transaction.TransactionState.TRANSACTION_COMPLETED, transaction.getState());
assertFalse("Should NOT be backoff", completion.isBackoff());
assertEquals(1, completion.getDataPacketsTransferred());
}
public static void execSendTwoFlowFiles(Transaction transaction) throws IOException {
assertEquals(Transaction.TransactionState.TRANSACTION_STARTED, transaction.getState());
DataPacket packet = createDataPacket("contents on client 1");
transaction.send(packet);
packet = createDataPacket("contents on client 2");
transaction.send(packet);
transaction.confirm();
assertEquals(Transaction.TransactionState.TRANSACTION_CONFIRMED, transaction.getState());
TransactionCompletion completion = transaction.complete();
assertEquals(Transaction.TransactionState.TRANSACTION_COMPLETED, transaction.getState());
assertFalse("Should NOT be backoff", completion.isBackoff());
assertEquals(2, completion.getDataPacketsTransferred());
}
public static void execSendWithInvalidChecksum(Transaction transaction) throws IOException {
assertEquals(Transaction.TransactionState.TRANSACTION_STARTED, transaction.getState());
DataPacket packet = createDataPacket("contents on client 1");
transaction.send(packet);
packet = createDataPacket("contents on client 2");
transaction.send(packet);
try {
transaction.confirm();
fail();
} catch (IOException e){
assertTrue(e.getMessage().contains("peer calculated CRC32 Checksum as Different checksum"));
assertEquals(Transaction.TransactionState.ERROR, transaction.getState());
}
try {
transaction.complete();
fail("It's not confirmed.");
} catch (IllegalStateException e){
assertEquals(Transaction.TransactionState.ERROR, transaction.getState());
}
}
public static void execSendButDestinationFull(Transaction transaction) throws IOException {
assertEquals(Transaction.TransactionState.TRANSACTION_STARTED, transaction.getState());
DataPacket packet = createDataPacket("contents on client 1");
transaction.send(packet);
packet = createDataPacket("contents on client 2");
transaction.send(packet);
transaction.confirm();
assertEquals(Transaction.TransactionState.TRANSACTION_CONFIRMED, transaction.getState());
TransactionCompletion completion = transaction.complete();
assertEquals(Transaction.TransactionState.TRANSACTION_COMPLETED, transaction.getState());
assertTrue("Should be backoff", completion.isBackoff());
assertEquals(2, completion.getDataPacketsTransferred());
}
}

View File

@ -0,0 +1,346 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT 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.remote.protocol.http;
import org.apache.nifi.events.EventReporter;
import org.apache.nifi.remote.Peer;
import org.apache.nifi.remote.PeerDescription;
import org.apache.nifi.remote.TransferDirection;
import org.apache.nifi.remote.codec.FlowFileCodec;
import org.apache.nifi.remote.codec.StandardFlowFileCodec;
import org.apache.nifi.remote.io.http.HttpCommunicationsSession;
import org.apache.nifi.remote.io.http.HttpInput;
import org.apache.nifi.remote.io.http.HttpOutput;
import org.apache.nifi.remote.protocol.CommunicationsSession;
import org.apache.nifi.remote.protocol.DataPacket;
import org.apache.nifi.remote.protocol.ResponseCode;
import org.apache.nifi.remote.util.SiteToSiteRestApiClient;
import org.apache.nifi.reporting.Severity;
import org.apache.nifi.stream.io.ByteArrayInputStream;
import org.apache.nifi.stream.io.ByteArrayOutputStream;
import org.apache.nifi.web.api.entity.TransactionResultEntity;
import org.junit.Test;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import static org.apache.nifi.remote.protocol.ResponseCode.CONFIRM_TRANSACTION;
import static org.apache.nifi.remote.protocol.SiteToSiteTestUtils.createDataPacket;
import static org.apache.nifi.remote.protocol.SiteToSiteTestUtils.execReceiveOneFlowFile;
import static org.apache.nifi.remote.protocol.SiteToSiteTestUtils.execReceiveTwoFlowFiles;
import static org.apache.nifi.remote.protocol.SiteToSiteTestUtils.execReceiveWithInvalidChecksum;
import static org.apache.nifi.remote.protocol.SiteToSiteTestUtils.execReceiveZeroFlowFile;
import static org.apache.nifi.remote.protocol.SiteToSiteTestUtils.execSendButDestinationFull;
import static org.apache.nifi.remote.protocol.SiteToSiteTestUtils.execSendOneFlowFile;
import static org.apache.nifi.remote.protocol.SiteToSiteTestUtils.execSendTwoFlowFiles;
import static org.apache.nifi.remote.protocol.SiteToSiteTestUtils.execSendWithInvalidChecksum;
import static org.apache.nifi.remote.protocol.SiteToSiteTestUtils.execSendZeroFlowFile;
import static org.apache.nifi.remote.protocol.SiteToSiteTestUtils.readContents;
import static org.junit.Assert.assertEquals;
import static org.mockito.Matchers.any;
import static org.mockito.Matchers.eq;
import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.doNothing;
import static org.mockito.Mockito.doReturn;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.verify;
public class TestHttpClientTransaction {
private Logger logger = LoggerFactory.getLogger(TestHttpClientTransaction.class);
private FlowFileCodec codec = new StandardFlowFileCodec();
private HttpClientTransaction getClientTransaction(InputStream is, OutputStream os, SiteToSiteRestApiClient apiClient, TransferDirection direction, String transactionUrl) throws IOException {
PeerDescription description = null;
String peerUrl = "";
HttpCommunicationsSession commsSession = new HttpCommunicationsSession();
((HttpInput)commsSession.getInput()).setInputStream(is);
((HttpOutput)commsSession.getOutput()).setOutputStream(os);
String clusterUrl = "";
Peer peer = new Peer(description, commsSession, peerUrl, clusterUrl);
String portId = "portId";
boolean useCompression = false;
int penaltyMillis = 1000;
EventReporter eventReporter = new EventReporter() {
@Override
public void reportEvent(Severity severity, String category, String message) {
logger.info("Reporting event... severity={}, category={}, message={}", severity, category, message);
}
};
int protocolVersion = 5;
HttpClientTransaction transaction = new HttpClientTransaction(protocolVersion, peer, direction, useCompression, portId, penaltyMillis, eventReporter);
transaction.initialize(apiClient, transactionUrl);
return transaction;
}
@Test
public void testReceiveZeroFlowFile() throws IOException {
SiteToSiteRestApiClient apiClient = mock(SiteToSiteRestApiClient.class);
final String transactionUrl = "http://www.example.com/site-to-site/input-ports/portId/transactions/transactionId";
doReturn(false).when(apiClient).openConnectionForReceive(eq(transactionUrl), any(CommunicationsSession.class));
ByteArrayInputStream serverResponse = new ByteArrayInputStream(new byte[0]);
ByteArrayOutputStream clientRequest = new ByteArrayOutputStream();
HttpClientTransaction transaction = getClientTransaction(serverResponse, clientRequest, apiClient, TransferDirection.RECEIVE, transactionUrl);
execReceiveZeroFlowFile(transaction);
assertEquals("Client sends nothing as payload to receive flow files.", 0, clientRequest.toByteArray().length);
}
@Test
public void testReceiveOneFlowFile() throws IOException {
SiteToSiteRestApiClient apiClient = mock(SiteToSiteRestApiClient.class);
final String transactionUrl = "http://www.example.com/site-to-site/input-ports/portId/transactions/transactionId";
doReturn(true).when(apiClient).openConnectionForReceive(eq(transactionUrl), any(CommunicationsSession.class));
TransactionResultEntity resultEntity = new TransactionResultEntity();
resultEntity.setResponseCode(CONFIRM_TRANSACTION.getCode());
doReturn(resultEntity).when(apiClient).commitReceivingFlowFiles(eq(transactionUrl), eq(CONFIRM_TRANSACTION), eq("3680976076"));
ByteArrayOutputStream serverResponseBos = new ByteArrayOutputStream();
codec.encode(createDataPacket("contents on server 1"), serverResponseBos);
ByteArrayInputStream serverResponse = new ByteArrayInputStream(serverResponseBos.toByteArray());
ByteArrayOutputStream clientRequest = new ByteArrayOutputStream();
HttpClientTransaction transaction = getClientTransaction(serverResponse, clientRequest, apiClient, TransferDirection.RECEIVE, transactionUrl);
execReceiveOneFlowFile(transaction);
assertEquals("Client sends nothing as payload to receive flow files.", 0, clientRequest.toByteArray().length);
verify(apiClient).commitReceivingFlowFiles(transactionUrl, CONFIRM_TRANSACTION, "3680976076");
}
@Test
public void testReceiveTwoFlowFiles() throws IOException {
SiteToSiteRestApiClient apiClient = mock(SiteToSiteRestApiClient.class);
final String transactionUrl = "http://www.example.com/site-to-site/input-ports/portId/transactions/transactionId";
doReturn(true).when(apiClient).openConnectionForReceive(eq(transactionUrl), any(CommunicationsSession.class));
TransactionResultEntity resultEntity = new TransactionResultEntity();
resultEntity.setResponseCode(CONFIRM_TRANSACTION.getCode());
doReturn(resultEntity).when(apiClient).commitReceivingFlowFiles(eq(transactionUrl), eq(CONFIRM_TRANSACTION), eq("2969091230"));
ByteArrayOutputStream serverResponseBos = new ByteArrayOutputStream();
codec.encode(createDataPacket("contents on server 1"), serverResponseBos);
codec.encode(createDataPacket("contents on server 2"), serverResponseBos);
ByteArrayInputStream serverResponse = new ByteArrayInputStream(serverResponseBos.toByteArray());
ByteArrayOutputStream clientRequest = new ByteArrayOutputStream();
HttpClientTransaction transaction = getClientTransaction(serverResponse, clientRequest, apiClient, TransferDirection.RECEIVE, transactionUrl);
execReceiveTwoFlowFiles(transaction);
assertEquals("Client sends nothing as payload to receive flow files.", 0, clientRequest.toByteArray().length);
verify(apiClient).commitReceivingFlowFiles(transactionUrl, CONFIRM_TRANSACTION, "2969091230");
}
@Test
public void testReceiveWithInvalidChecksum() throws IOException {
SiteToSiteRestApiClient apiClient = mock(SiteToSiteRestApiClient.class);
final String transactionUrl = "http://www.example.com/site-to-site/input-ports/portId/transactions/transactionId";
doReturn(true).when(apiClient).openConnectionForReceive(eq(transactionUrl), any(CommunicationsSession.class));
// The checksum is correct, but here we simulate as if it's wrong, BAD_CHECKSUM.
TransactionResultEntity resultEntity = new TransactionResultEntity();
resultEntity.setResponseCode(ResponseCode.BAD_CHECKSUM.getCode());
doReturn(resultEntity).when(apiClient).commitReceivingFlowFiles(eq(transactionUrl), eq(CONFIRM_TRANSACTION), eq("2969091230"));
ByteArrayOutputStream serverResponseBos = new ByteArrayOutputStream();
codec.encode(createDataPacket("contents on server 1"), serverResponseBos);
codec.encode(createDataPacket("contents on server 2"), serverResponseBos);
ByteArrayInputStream serverResponse = new ByteArrayInputStream(serverResponseBos.toByteArray());
ByteArrayOutputStream clientRequest = new ByteArrayOutputStream();
HttpClientTransaction transaction = getClientTransaction(serverResponse, clientRequest, apiClient, TransferDirection.RECEIVE, transactionUrl);
execReceiveWithInvalidChecksum(transaction);
assertEquals("Client sends nothing as payload to receive flow files.", 0, clientRequest.toByteArray().length);
verify(apiClient).commitReceivingFlowFiles(transactionUrl, CONFIRM_TRANSACTION, "2969091230");
}
@Test
public void testSendZeroFlowFile() throws IOException {
SiteToSiteRestApiClient apiClient = mock(SiteToSiteRestApiClient.class);
final String transactionUrl = "http://www.example.com/site-to-site/input-ports/portId/transactions/transactionId";
doNothing().when(apiClient).openConnectionForSend(eq(transactionUrl), any(CommunicationsSession.class));
ByteArrayOutputStream serverResponseBos = new ByteArrayOutputStream();
ByteArrayInputStream serverResponse = new ByteArrayInputStream(serverResponseBos.toByteArray());
ByteArrayOutputStream clientRequest = new ByteArrayOutputStream();
HttpClientTransaction transaction = getClientTransaction(serverResponse, clientRequest, apiClient, TransferDirection.SEND, transactionUrl);
execSendZeroFlowFile(transaction);
assertEquals("Client didn't send anything", 0, clientRequest.toByteArray().length);
}
@Test
public void testSendOneFlowFile() throws IOException {
SiteToSiteRestApiClient apiClient = mock(SiteToSiteRestApiClient.class);
final String transactionUrl = "http://www.example.com/site-to-site/input-ports/portId/transactions/transactionId";
doNothing().when(apiClient).openConnectionForSend(eq(transactionUrl), any(CommunicationsSession.class));
// Emulate that server returns correct checksum.
doAnswer(new Answer() {
@Override
public Object answer(InvocationOnMock invocation) throws Throwable {
HttpCommunicationsSession commSession = (HttpCommunicationsSession)invocation.getArguments()[0];
commSession.setChecksum("2946083981");
return null;
}
}).when(apiClient).finishTransferFlowFiles(any(CommunicationsSession.class));
TransactionResultEntity resultEntity = new TransactionResultEntity();
resultEntity.setResponseCode(ResponseCode.TRANSACTION_FINISHED.getCode());
doReturn(resultEntity).when(apiClient).commitTransferFlowFiles(eq(transactionUrl), eq(CONFIRM_TRANSACTION));
ByteArrayOutputStream serverResponseBos = new ByteArrayOutputStream();
ByteArrayInputStream serverResponse = new ByteArrayInputStream(serverResponseBos.toByteArray());
ByteArrayOutputStream clientRequest = new ByteArrayOutputStream();
HttpClientTransaction transaction = getClientTransaction(serverResponse, clientRequest, apiClient, TransferDirection.SEND, transactionUrl);
execSendOneFlowFile(transaction);
InputStream sentByClient = new ByteArrayInputStream(clientRequest.toByteArray());
DataPacket packetByClient = codec.decode(sentByClient);
assertEquals("contents on client 1", readContents(packetByClient));
assertEquals(-1, sentByClient.read());
verify(apiClient).commitTransferFlowFiles(transactionUrl, CONFIRM_TRANSACTION);
}
@Test
public void testSendTwoFlowFiles() throws IOException {
SiteToSiteRestApiClient apiClient = mock(SiteToSiteRestApiClient.class);
final String transactionUrl = "http://www.example.com/site-to-site/input-ports/portId/transactions/transactionId";
doNothing().when(apiClient).openConnectionForSend(eq("portId"), any(CommunicationsSession.class));
// Emulate that server returns correct checksum.
doAnswer(new Answer() {
@Override
public Object answer(InvocationOnMock invocation) throws Throwable {
HttpCommunicationsSession commSession = (HttpCommunicationsSession)invocation.getArguments()[0];
commSession.setChecksum("3359812065");
return null;
}
}).when(apiClient).finishTransferFlowFiles(any(CommunicationsSession.class));
TransactionResultEntity resultEntity = new TransactionResultEntity();
resultEntity.setResponseCode(ResponseCode.TRANSACTION_FINISHED.getCode());
doReturn(resultEntity).when(apiClient).commitTransferFlowFiles(eq(transactionUrl), eq(CONFIRM_TRANSACTION));
ByteArrayOutputStream serverResponseBos = new ByteArrayOutputStream();
ByteArrayInputStream serverResponse = new ByteArrayInputStream(serverResponseBos.toByteArray());
ByteArrayOutputStream clientRequest = new ByteArrayOutputStream();
HttpClientTransaction transaction = getClientTransaction(serverResponse, clientRequest, apiClient, TransferDirection.SEND, transactionUrl);
execSendTwoFlowFiles(transaction);
InputStream sentByClient = new ByteArrayInputStream(clientRequest.toByteArray());
DataPacket packetByClient = codec.decode(sentByClient);
assertEquals("contents on client 1", readContents(packetByClient));
packetByClient = codec.decode(sentByClient);
assertEquals("contents on client 2", readContents(packetByClient));
assertEquals(-1, sentByClient.read());
verify(apiClient).commitTransferFlowFiles(transactionUrl, CONFIRM_TRANSACTION);
}
@Test
public void testSendWithInvalidChecksum() throws IOException {
SiteToSiteRestApiClient apiClient = mock(SiteToSiteRestApiClient.class);
final String transactionUrl = "http://www.example.com/site-to-site/input-ports/portId/transactions/transactionId";
doNothing().when(apiClient).openConnectionForSend(eq(transactionUrl), any(CommunicationsSession.class));
// Emulate that server returns incorrect checksum.
doAnswer(new Answer() {
@Override
public Object answer(InvocationOnMock invocation) throws Throwable {
HttpCommunicationsSession commSession = (HttpCommunicationsSession)invocation.getArguments()[0];
commSession.setChecksum("Different checksum");
return null;
}
}).when(apiClient).finishTransferFlowFiles(any(CommunicationsSession.class));
doAnswer(new Answer() {
@Override
public Object answer(InvocationOnMock invocation) throws Throwable {
TransactionResultEntity serverResult = new TransactionResultEntity();
serverResult.setResponseCode(ResponseCode.CANCEL_TRANSACTION.getCode());
return serverResult;
}
}).when(apiClient).commitTransferFlowFiles(eq(transactionUrl), eq(ResponseCode.BAD_CHECKSUM));
ByteArrayOutputStream serverResponseBos = new ByteArrayOutputStream();
ByteArrayInputStream serverResponse = new ByteArrayInputStream(serverResponseBos.toByteArray());
ByteArrayOutputStream clientRequest = new ByteArrayOutputStream();
HttpClientTransaction transaction = getClientTransaction(serverResponse, clientRequest, apiClient, TransferDirection.SEND, transactionUrl);
execSendWithInvalidChecksum(transaction);
InputStream sentByClient = new ByteArrayInputStream(clientRequest.toByteArray());
DataPacket packetByClient = codec.decode(sentByClient);
assertEquals("contents on client 1", readContents(packetByClient));
packetByClient = codec.decode(sentByClient);
assertEquals("contents on client 2", readContents(packetByClient));
assertEquals(-1, sentByClient.read());
verify(apiClient).commitTransferFlowFiles(transactionUrl, ResponseCode.BAD_CHECKSUM);
}
@Test
public void testSendButDestinationFull() throws IOException {
SiteToSiteRestApiClient apiClient = mock(SiteToSiteRestApiClient.class);
final String transactionUrl = "http://www.example.com/site-to-site/input-ports/portId/transactions/transactionId";
doNothing().when(apiClient).openConnectionForSend(eq("portId"), any(CommunicationsSession.class));
// Emulate that server returns correct checksum.
doAnswer(new Answer() {
@Override
public Object answer(InvocationOnMock invocation) throws Throwable {
HttpCommunicationsSession commSession = (HttpCommunicationsSession)invocation.getArguments()[0];
commSession.setChecksum("3359812065");
return null;
}
}).when(apiClient).finishTransferFlowFiles(any(CommunicationsSession.class));
TransactionResultEntity resultEntity = new TransactionResultEntity();
resultEntity.setResponseCode(ResponseCode.TRANSACTION_FINISHED_BUT_DESTINATION_FULL.getCode());
doReturn(resultEntity).when(apiClient).commitTransferFlowFiles(eq(transactionUrl), eq(CONFIRM_TRANSACTION));
ByteArrayOutputStream serverResponseBos = new ByteArrayOutputStream();
ByteArrayInputStream serverResponse = new ByteArrayInputStream(serverResponseBos.toByteArray());
ByteArrayOutputStream clientRequest = new ByteArrayOutputStream();
HttpClientTransaction transaction = getClientTransaction(serverResponse, clientRequest, apiClient, TransferDirection.SEND, transactionUrl);
execSendButDestinationFull(transaction);
InputStream sentByClient = new ByteArrayInputStream(clientRequest.toByteArray());
DataPacket packetByClient = codec.decode(sentByClient);
assertEquals("contents on client 1", readContents(packetByClient));
packetByClient = codec.decode(sentByClient);
assertEquals("contents on client 2", readContents(packetByClient));
assertEquals(-1, sentByClient.read());
verify(apiClient).commitTransferFlowFiles(transactionUrl, CONFIRM_TRANSACTION);
}
}

View File

@ -0,0 +1,334 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT 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.remote.protocol.socket;
import org.apache.nifi.events.EventReporter;
import org.apache.nifi.remote.Peer;
import org.apache.nifi.remote.PeerDescription;
import org.apache.nifi.remote.Transaction;
import org.apache.nifi.remote.TransferDirection;
import org.apache.nifi.remote.codec.FlowFileCodec;
import org.apache.nifi.remote.codec.StandardFlowFileCodec;
import org.apache.nifi.remote.io.socket.SocketChannelCommunicationsSession;
import org.apache.nifi.remote.io.socket.SocketChannelInput;
import org.apache.nifi.remote.io.socket.SocketChannelOutput;
import org.apache.nifi.remote.protocol.DataPacket;
import org.apache.nifi.remote.protocol.RequestType;
import org.apache.nifi.remote.protocol.Response;
import org.apache.nifi.remote.protocol.ResponseCode;
import org.apache.nifi.stream.io.ByteArrayInputStream;
import org.apache.nifi.stream.io.ByteArrayOutputStream;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.IOException;
import static org.apache.nifi.remote.protocol.SiteToSiteTestUtils.createDataPacket;
import static org.apache.nifi.remote.protocol.SiteToSiteTestUtils.execReceiveOneFlowFile;
import static org.apache.nifi.remote.protocol.SiteToSiteTestUtils.execReceiveTwoFlowFiles;
import static org.apache.nifi.remote.protocol.SiteToSiteTestUtils.execReceiveWithInvalidChecksum;
import static org.apache.nifi.remote.protocol.SiteToSiteTestUtils.execReceiveZeroFlowFile;
import static org.apache.nifi.remote.protocol.SiteToSiteTestUtils.execSendButDestinationFull;
import static org.apache.nifi.remote.protocol.SiteToSiteTestUtils.execSendOneFlowFile;
import static org.apache.nifi.remote.protocol.SiteToSiteTestUtils.execSendTwoFlowFiles;
import static org.apache.nifi.remote.protocol.SiteToSiteTestUtils.execSendWithInvalidChecksum;
import static org.apache.nifi.remote.protocol.SiteToSiteTestUtils.execSendZeroFlowFile;
import static org.apache.nifi.remote.protocol.SiteToSiteTestUtils.readContents;
import static org.junit.Assert.assertEquals;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
public class TestSocketClientTransaction {
private Logger logger = LoggerFactory.getLogger(TestSocketClientTransaction.class);
private FlowFileCodec codec = new StandardFlowFileCodec();
private SocketClientTransaction getClientTransaction(ByteArrayInputStream bis, ByteArrayOutputStream bos, TransferDirection direction) throws IOException {
PeerDescription description = null;
String peerUrl = "";
SocketChannelCommunicationsSession commsSession = mock(SocketChannelCommunicationsSession.class);
SocketChannelInput socketIn = mock(SocketChannelInput.class);
SocketChannelOutput socketOut = mock(SocketChannelOutput.class);
when(commsSession.getInput()).thenReturn(socketIn);
when(commsSession.getOutput()).thenReturn(socketOut);
when(socketIn.getInputStream()).thenReturn(bis);
when(socketOut.getOutputStream()).thenReturn(bos);
String clusterUrl = "";
Peer peer = new Peer(description, commsSession, peerUrl, clusterUrl);
boolean useCompression = false;
int penaltyMillis = 1000;
EventReporter eventReporter = null;
int protocolVersion = 5;
String destinationId = "destinationId";
return new SocketClientTransaction(protocolVersion, destinationId, peer, codec, direction, useCompression, penaltyMillis, eventReporter);
}
@Test
public void testReceiveZeroFlowFile() throws IOException {
ByteArrayOutputStream serverResponseBos = new ByteArrayOutputStream();
DataOutputStream serverResponse = new DataOutputStream(serverResponseBos);
ResponseCode.NO_MORE_DATA.writeResponse(serverResponse);
ByteArrayInputStream bis = new ByteArrayInputStream(serverResponseBos.toByteArray());
ByteArrayOutputStream bos = new ByteArrayOutputStream();
SocketClientTransaction transaction = getClientTransaction(bis, bos, TransferDirection.RECEIVE);
execReceiveZeroFlowFile(transaction);
// Verify what client has sent.
DataInputStream sentByClient = new DataInputStream(new ByteArrayInputStream(bos.toByteArray()));
assertEquals(RequestType.RECEIVE_FLOWFILES, RequestType.readRequestType(sentByClient));
assertEquals(-1, sentByClient.read());
}
@Test
public void testReceiveOneFlowFile() throws IOException {
ByteArrayOutputStream serverResponseBos = new ByteArrayOutputStream();
DataOutputStream serverResponse = new DataOutputStream(serverResponseBos);
ResponseCode.MORE_DATA.writeResponse(serverResponse);
codec.encode(createDataPacket("contents on server 1"), serverResponse);
ResponseCode.FINISH_TRANSACTION.writeResponse(serverResponse);
ResponseCode.CONFIRM_TRANSACTION.writeResponse(serverResponse, "Checksum has been verified at server.");
ByteArrayInputStream bis = new ByteArrayInputStream(serverResponseBos.toByteArray());
ByteArrayOutputStream bos = new ByteArrayOutputStream();
SocketClientTransaction transaction = getClientTransaction(bis, bos, TransferDirection.RECEIVE);
execReceiveOneFlowFile(transaction);
// Verify what client has sent.
DataInputStream sentByClient = new DataInputStream(new ByteArrayInputStream(bos.toByteArray()));
assertEquals(RequestType.RECEIVE_FLOWFILES, RequestType.readRequestType(sentByClient));
Response confirmResponse = Response.read(sentByClient);
assertEquals(ResponseCode.CONFIRM_TRANSACTION, confirmResponse.getCode());
assertEquals("Checksum should be calculated at client", "3680976076", confirmResponse.getMessage());
Response completeResponse = Response.read(sentByClient);
assertEquals(ResponseCode.TRANSACTION_FINISHED, completeResponse.getCode());
assertEquals(-1, sentByClient.read());
}
@Test
public void testReceiveTwoFlowFiles() throws IOException {
ByteArrayOutputStream serverResponseBos = new ByteArrayOutputStream();
DataOutputStream serverResponse = new DataOutputStream(serverResponseBos);
ResponseCode.MORE_DATA.writeResponse(serverResponse);
codec.encode(createDataPacket("contents on server 1"), serverResponse);
ResponseCode.CONTINUE_TRANSACTION.writeResponse(serverResponse);
codec.encode(createDataPacket("contents on server 2"), serverResponse);
ResponseCode.FINISH_TRANSACTION.writeResponse(serverResponse);
ResponseCode.CONFIRM_TRANSACTION.writeResponse(serverResponse, "Checksum has been verified at server.");
ByteArrayInputStream bis = new ByteArrayInputStream(serverResponseBos.toByteArray());
ByteArrayOutputStream bos = new ByteArrayOutputStream();
SocketClientTransaction transaction = getClientTransaction(bis, bos, TransferDirection.RECEIVE);
assertEquals(Transaction.TransactionState.TRANSACTION_STARTED, transaction.getState());
execReceiveTwoFlowFiles(transaction);
// Verify what client has sent.
DataInputStream sentByClient = new DataInputStream(new ByteArrayInputStream(bos.toByteArray()));
assertEquals(RequestType.RECEIVE_FLOWFILES, RequestType.readRequestType(sentByClient));
Response confirmResponse = Response.read(sentByClient);
assertEquals(ResponseCode.CONFIRM_TRANSACTION, confirmResponse.getCode());
assertEquals("Checksum should be calculated at client", "2969091230", confirmResponse.getMessage());
Response completeResponse = Response.read(sentByClient);
assertEquals(ResponseCode.TRANSACTION_FINISHED, completeResponse.getCode());
assertEquals(-1, sentByClient.read());
}
@Test
public void testReceiveWithInvalidChecksum() throws IOException {
ByteArrayOutputStream serverResponseBos = new ByteArrayOutputStream();
DataOutputStream serverResponse = new DataOutputStream(serverResponseBos);
ResponseCode.MORE_DATA.writeResponse(serverResponse);
codec.encode(createDataPacket("contents on server 1"), serverResponse);
ResponseCode.CONTINUE_TRANSACTION.writeResponse(serverResponse);
codec.encode(createDataPacket("contents on server 2"), serverResponse);
ResponseCode.FINISH_TRANSACTION.writeResponse(serverResponse);
ResponseCode.BAD_CHECKSUM.writeResponse(serverResponse);
ByteArrayInputStream bis = new ByteArrayInputStream(serverResponseBos.toByteArray());
ByteArrayOutputStream bos = new ByteArrayOutputStream();
SocketClientTransaction transaction = getClientTransaction(bis, bos, TransferDirection.RECEIVE);
execReceiveWithInvalidChecksum(transaction);
// Verify what client has sent.
DataInputStream sentByClient = new DataInputStream(new ByteArrayInputStream(bos.toByteArray()));
assertEquals(RequestType.RECEIVE_FLOWFILES, RequestType.readRequestType(sentByClient));
Response confirmResponse = Response.read(sentByClient);
assertEquals(ResponseCode.CONFIRM_TRANSACTION, confirmResponse.getCode());
assertEquals("Checksum should be calculated at client", "2969091230", confirmResponse.getMessage());
assertEquals(-1, sentByClient.read());
}
@Test
public void testSendZeroFlowFile() throws IOException {
ByteArrayOutputStream serverResponseBos = new ByteArrayOutputStream();
ByteArrayInputStream bis = new ByteArrayInputStream(serverResponseBos.toByteArray());
ByteArrayOutputStream bos = new ByteArrayOutputStream();
SocketClientTransaction transaction = getClientTransaction(bis, bos, TransferDirection.SEND);
execSendZeroFlowFile(transaction);
// Verify what client has sent.
DataInputStream sentByClient = new DataInputStream(new ByteArrayInputStream(bos.toByteArray()));
assertEquals(RequestType.SEND_FLOWFILES, RequestType.readRequestType(sentByClient));
assertEquals(-1, sentByClient.read());
}
@Test
public void testSendOneFlowFile() throws IOException {
ByteArrayOutputStream serverResponseBos = new ByteArrayOutputStream();
DataOutputStream serverResponse = new DataOutputStream(serverResponseBos);
ResponseCode.CONFIRM_TRANSACTION.writeResponse(serverResponse, "2946083981");
ResponseCode.TRANSACTION_FINISHED.writeResponse(serverResponse);
ByteArrayInputStream bis = new ByteArrayInputStream(serverResponseBos.toByteArray());
ByteArrayOutputStream bos = new ByteArrayOutputStream();
SocketClientTransaction transaction = getClientTransaction(bis, bos, TransferDirection.SEND);
execSendOneFlowFile(transaction);
// Verify what client has sent.
DataInputStream sentByClient = new DataInputStream(new ByteArrayInputStream(bos.toByteArray()));
assertEquals(RequestType.SEND_FLOWFILES, RequestType.readRequestType(sentByClient));
DataPacket packetByClient = codec.decode(sentByClient);
assertEquals("contents on client 1", readContents(packetByClient));
Response endOfDataResponse = Response.read(sentByClient);
assertEquals(ResponseCode.FINISH_TRANSACTION, endOfDataResponse.getCode());
Response confirmResponse = Response.read(sentByClient);
assertEquals(ResponseCode.CONFIRM_TRANSACTION, confirmResponse.getCode());
assertEquals(-1, sentByClient.read());
}
@Test
public void testSendTwoFlowFiles() throws IOException {
ByteArrayOutputStream serverResponseBos = new ByteArrayOutputStream();
DataOutputStream serverResponse = new DataOutputStream(serverResponseBos);
ResponseCode.CONFIRM_TRANSACTION.writeResponse(serverResponse, "3359812065");
ResponseCode.TRANSACTION_FINISHED.writeResponse(serverResponse);
ByteArrayInputStream bis = new ByteArrayInputStream(serverResponseBos.toByteArray());
ByteArrayOutputStream bos = new ByteArrayOutputStream();
SocketClientTransaction transaction = getClientTransaction(bis, bos, TransferDirection.SEND);
execSendTwoFlowFiles(transaction);
// Verify what client has sent.
DataInputStream sentByClient = new DataInputStream(new ByteArrayInputStream(bos.toByteArray()));
assertEquals(RequestType.SEND_FLOWFILES, RequestType.readRequestType(sentByClient));
DataPacket packetByClient = codec.decode(sentByClient);
assertEquals("contents on client 1", readContents(packetByClient));
Response continueDataResponse = Response.read(sentByClient);
assertEquals(ResponseCode.CONTINUE_TRANSACTION, continueDataResponse.getCode());
packetByClient = codec.decode(sentByClient);
assertEquals("contents on client 2", readContents(packetByClient));
Response endOfDataResponse = Response.read(sentByClient);
assertEquals(ResponseCode.FINISH_TRANSACTION, endOfDataResponse.getCode());
Response confirmResponse = Response.read(sentByClient);
assertEquals(ResponseCode.CONFIRM_TRANSACTION, confirmResponse.getCode());
assertEquals(-1, sentByClient.read());
}
@Test
public void testSendWithInvalidChecksum() throws IOException {
ByteArrayOutputStream serverResponseBos = new ByteArrayOutputStream();
DataOutputStream serverResponse = new DataOutputStream(serverResponseBos);
ResponseCode.CONFIRM_TRANSACTION.writeResponse(serverResponse, "Different checksum");
ByteArrayInputStream bis = new ByteArrayInputStream(serverResponseBos.toByteArray());
ByteArrayOutputStream bos = new ByteArrayOutputStream();
SocketClientTransaction transaction = getClientTransaction(bis, bos, TransferDirection.SEND);
execSendWithInvalidChecksum(transaction);
// Verify what client has sent.
DataInputStream sentByClient = new DataInputStream(new ByteArrayInputStream(bos.toByteArray()));
assertEquals(RequestType.SEND_FLOWFILES, RequestType.readRequestType(sentByClient));
DataPacket packetByClient = codec.decode(sentByClient);
assertEquals("contents on client 1", readContents(packetByClient));
Response continueDataResponse = Response.read(sentByClient);
assertEquals(ResponseCode.CONTINUE_TRANSACTION, continueDataResponse.getCode());
packetByClient = codec.decode(sentByClient);
assertEquals("contents on client 2", readContents(packetByClient));
Response endOfDataResponse = Response.read(sentByClient);
assertEquals(ResponseCode.FINISH_TRANSACTION, endOfDataResponse.getCode());
Response confirmResponse = Response.read(sentByClient);
assertEquals(ResponseCode.BAD_CHECKSUM, confirmResponse.getCode());
assertEquals(-1, sentByClient.read());
}
@Test
public void testSendButDestinationFull() throws IOException {
ByteArrayOutputStream serverResponseBos = new ByteArrayOutputStream();
DataOutputStream serverResponse = new DataOutputStream(serverResponseBos);
ResponseCode.CONFIRM_TRANSACTION.writeResponse(serverResponse, "3359812065");
ResponseCode.TRANSACTION_FINISHED_BUT_DESTINATION_FULL.writeResponse(serverResponse);
ByteArrayInputStream bis = new ByteArrayInputStream(serverResponseBos.toByteArray());
ByteArrayOutputStream bos = new ByteArrayOutputStream();
SocketClientTransaction transaction = getClientTransaction(bis, bos, TransferDirection.SEND);
execSendButDestinationFull(transaction);
// Verify what client has sent.
DataInputStream sentByClient = new DataInputStream(new ByteArrayInputStream(bos.toByteArray()));
assertEquals(RequestType.SEND_FLOWFILES, RequestType.readRequestType(sentByClient));
DataPacket packetByClient = codec.decode(sentByClient);
assertEquals("contents on client 1", readContents(packetByClient));
Response continueDataResponse = Response.read(sentByClient);
assertEquals(ResponseCode.CONTINUE_TRANSACTION, continueDataResponse.getCode());
packetByClient = codec.decode(sentByClient);
assertEquals("contents on client 2", readContents(packetByClient));
Response endOfDataResponse = Response.read(sentByClient);
assertEquals(ResponseCode.FINISH_TRANSACTION, endOfDataResponse.getCode());
Response confirmResponse = Response.read(sentByClient);
assertEquals(ResponseCode.CONFIRM_TRANSACTION, confirmResponse.getCode());
assertEquals(-1, sentByClient.read());
}
}

View File

@ -1347,12 +1347,18 @@ of 576.
==== Site to Site Properties
These properties govern how this instance of NiFi communicates with remote instances of NiFi when Remote Process Groups are configured in the dataflow.
Remote Process Groups can choose transport protocol from RAW and HTTP. Properties named with _nifi.remote.input.socket.*_ are RAW transport protocol specific. Similarly, _nifi.remote.input.http.*_ are HTTP transport protocol specific properties.
|====
|*Property*|*Description*
|nifi.remote.input.socket.host|The host name that will be given out to clients to connect to this NiFi instance for Site-to-Site communication. By default, it is the value from InetAddress.getLocalHost().getHostName(). On UNIX-like operating systems, this is typically the output from the `hostname` command.
|nifi.remote.input.socket.port|The remote input socket port for Site-to-Site communication. By default, it is blank, but it must have a value in order to use Remote Process Groups.
|nifi.remote.input.secure|This indicates whether communication between this instance of NiFi and remote NiFi instances should be secure. By default, it is set to _true_. In order for secure site-to-site to work, many Security Properties (below) must also be configured.
|[line-through]#nifi.remote.input.socket.host# +
nifi.remote.input.host +
(renamed since NiFi 1.0)|The host name that will be given out to clients to connect to this NiFi instance for Site-to-Site communication. By default, it is the value from InetAddress.getLocalHost().getHostName(). On UNIX-like operating systems, this is typically the output from the `hostname` command.
|nifi.remote.input.secure|This indicates whether communication between this instance of NiFi and remote NiFi instances should be secure. By default, it is set to _false_. In order for secure site-to-site to work, in addition to change it to _true_, many Security Properties (below) must also be configured.
|nifi.remote.input.socket.port|The remote input socket port for Site-to-Site communication. By default, it is blank, but it must have a value in order to use RAW socket as transport protocol for Site-to-Site.
|nifi.remote.input.http.enabled|Specify if HTTP Site-to-Site should be enabled on this host. By default, it is _true_. HTTP non-secure Site-to-Site is enabled by default. +
Whether a Site-to-Site client uses HTTP or HTTPS is determined by _nifi.remote.input.secure_. If it is set to _true_, then requests are sent as HTTPS to _nifi.web.https.port_, if it is _false_, HTTP requests are sent to _nifi.web.http.port_.
|nifi.remote.input.http.transaction.ttl|Specify how long a transaction can stay alive on server. If a Site-to-Site client didn't proceed to next action for this period of time, the transaction is discarded from remote NiFi instance. For example, a client creates a transaction but doesn't send or receive flow files, or send or received flow files but doesn't confirm that transaction. By default, it is set to 30 seconds.|
|====
==== Web Properties

View File

@ -418,6 +418,7 @@ categorizing them by their functions.
- *PostHTTP*: Performs an HTTP POST request, sending the contents of the FlowFile as the body of the message. This is often used in conjunction
with ListenHTTP in order to transfer data between two different instances of NiFi in cases where Site-to-Site cannot be used (for instance,
when the nodes cannot access each other directly and are able to communicate through an HTTP proxy).
*Note*: HTTP is available as a link:user-guide.html#site-to-site[Site-to-Site] transport protocol in addition to the existing RAW socket transport. It also supports HTTP Proxy. Using HTTP Site-to-Site is recommended since it's more scalable, and can provide bi-directional data transfer using input/output ports with better user authentication and authorization.
- *HandleHttpRequest* / *HandleHttpResponse*: The HandleHttpRequest Processor is a Source Processor that starts an embedded HTTP(S) server
similarly to ListenHTTP. However, it does not send a response to the client. Instead, the FlowFile is sent out with the body of the HTTP request
as its contents and attributes for all of the typical Servlet parameters, headers, etc. as Attributes. The HandleHttpResponse then is able to

Binary file not shown.

After

Width:  |  Height:  |  Size: 36 KiB

View File

@ -781,7 +781,20 @@ link:administration-guide.html[Admin Guide].
This allows new capabilities to be added while still maintaining backward compatibility with all older instances. Additionally, if a vulnerability
or deficiency is ever discovered in a protocol, it allows a newer version of NiFi to forbid communication over the compromised versions of the protocol.
In order to communicate with a remote NiFi instance via Site-to-Site, simply drag a <<remote_process_group,Remote Process Group>> onto the canvas
Site-to-Site is a protocol transferring data between two NiFi instances. Both end can be a standalone NiFi or a NiFi cluster. In this section, the NiFi instance initiates the communications is called _Site-to-Site client NiFi instance_ and the other end as _Site-to-Site server NiFi instance_ to clarify what configuration needed on each NiFi instances.
A NiFi instance can be both client and server for Site-to-Site protocol, however, it can only be a client or server within a specific Site-to-Site communication. For example, if there are three NiFi instances A, B and C. A pushes data to B, and B pulls data from C. _A -- push -> B <- pull -- C_. Then B is not only a _server_ in the communication between A and B, but also a _client_ in B and C.
It is important to understand which NiFi instance will be the client or server in order to design your data flow, and configure each instance accordingly. Here is a summary of what components run on which side based on data flow direction:
- Push: a client _sends_ data to a Remote Process Group, the server _receives_ it with an Input Port
- Pull: a client _receives_ data from a Remote Process Group, the server _sends_ data through an Output Port
==== Configure Site-to-Site client NiFi instance
[[Site-to-Site_Remote_Process_Group]]
*Remote Process Group*: In order to communicate with a remote NiFi instance via Site-to-Site, simply drag a <<remote_process_group,Remote Process Group>> onto the canvas
and enter the URL of the remote NiFi instance (for more information on the components of a Remote Process Group, see
<<Remote_Group_Transmission,Remote Process Group Transmission>> section of this guide.) The URL is the same
URL you would use to go to that instance's User Interface. At that point, you can drag a connection to or from the Remote Process Group
@ -797,7 +810,17 @@ the ports shown will be the Input Ports of the remote group, as this implies tha
communicate with. For information on configuring NiFi to run securely, see the
link:administration-guide.html[Admin Guide].
In order to allow another NiFi instance to push data to your local instance, you can simply drag an <<input_port,Input Port>> onto the Root Process Group
[[Site-to-Site_Transport_Protocol]]
*Transport Protocol*: On a Remote Process Group creation or configuration dialog, you can choose Transport Protocol to use for Site-to-Site communication as shown in the following image:
image:configure-remote-process-group.png["Configure Remote Process Group", width=395]
By default, it is set to _RAW_ which uses raw socket communication using a dedicated port. _HTTP_ transport protocol is especially useful if the remote NiFi instance is in a restricted network that only allow access through HTTP(S) protocol or only accessible from a specific HTTP Proxy server. For accessing through a HTTP Proxy Server, BASIC and DIGEST authentication are supported.
==== Configure Site-to-Site server NiFi instance
[[Site-to-Site_Input_Port]]
*Input Port*: In order to allow another NiFi instance to push data to your local instance, you can simply drag an <<input_port,Input Port>> onto the Root Process Group
of your canvas. After entering a name for the port, it will be added to your flow. You can now right-click on the Input Port and choose Configure in order
to adjust the name and the number of concurrent tasks that are used for the port. If Site-to-Site is configured to run securely, you will also be given
the ability to adjust who has access to the port. If secure, only those who have been granted access to communicate with the port will be able to see
@ -806,14 +829,16 @@ that the port exists.
After being given access to a particular port, in order to see that port, the operator of a remote NiFi instance may need to right-click on their Remote
Process Group and choose to "Refresh" the flow.
Similar to an Input Port, a DataFlow Manager may choose to add an <<output_port,Output Port>> to the Root Process Group. The Output Port allows an
[[Site-to-Site_Output_Port]]
*Output Port*: Similar to an Input Port, a DataFlow Manager may choose to add an <<output_port,Output Port>> to the Root Process Group. The Output Port allows an
authorized NiFi instance to remotely connect to your instance and pull data from the Output Port. Configuring the Output Port will again allow the
DFM to control how many concurrent tasks are allowed, as well as which NiFi instances are authorized to pull data from the instance being configured.
In addition to other instances of NiFi, some other applications may use a Site-to-Site client in order to push data to or receive data from a NiFi instance.
For example, NiFi provides an Apache Storm spout and an Apache Spark Receiver that are able to pull data from NiFi's Root Group Output Ports.
If your instance of NiFi is running securely, the first time that a client establishes a connection to your instance, the client will be forbidden and
[[Site-to-Site_Access_Control]]
*Access Control*: If your instance of NiFi is running securely, the first time that a client establishes a connection to your instance, the client will be forbidden and
a request for an account for that client will automatically be generated. The client will need to be granted the 'NiFi' role in order to communicate
via Site-to-Site. For more information on managing user accounts, see the
link:administration-guide.html#controlling-levels-of-access[Controlling Levels of Access]
@ -824,8 +849,6 @@ link:administration-guide.html#site_to_site_properties[Site-to-Site Properties]
link:administration-guide.html[Admin Guide].
=== Example Dataflow
This section has described the steps required to build a dataflow. Now, to put it all together. The following example dataflow

View File

@ -41,6 +41,7 @@ public class ControllerDTO {
private Integer outputPortCount;
private Integer remoteSiteListeningPort;
private Integer remoteSiteHttpListeningPort;
private Boolean siteToSiteSecure;
private String instanceId;
private Set<PortDTO> inputPorts;
@ -149,6 +150,23 @@ public class ControllerDTO {
this.remoteSiteListeningPort = port;
}
/**
* The HTTP(S) Port on which this instance is listening for Remote Transfers of Flow Files. If this instance is not configured to receive Flow Files from remote instances, this will be null.
*
* @return a integer between 1 and 65535, or null, if not configured for remote transfer
*/
@ApiModelProperty(
value = "The HTTP(S) Port on which this instance is listening for Remote Transfers of Flow Files. If this instance is not configured to receive Flow Files from remote "
+ "instances, this will be null."
)
public Integer getRemoteSiteHttpListeningPort() {
return remoteSiteHttpListeningPort;
}
public void setRemoteSiteHttpListeningPort(Integer remoteSiteHttpListeningPort) {
this.remoteSiteHttpListeningPort = remoteSiteHttpListeningPort;
}
/**
* @return Indicates whether or not Site-to-Site communications with this instance is secure (2-way authentication)
*/
@ -274,4 +292,5 @@ public class ControllerDTO {
public void setOutputPortCount(Integer outputPortCount) {
this.outputPortCount = outputPortCount;
}
}

View File

@ -36,6 +36,11 @@ public class RemoteProcessGroupDTO extends ComponentDTO {
private String comments;
private String communicationsTimeout;
private String yieldDuration;
private String transportProtocol;
private String proxyHost;
private Integer proxyPort;
private String proxyUser;
private String proxyPassword;
private List<String> authorizationIssues;
private Boolean transmitting;
@ -288,4 +293,44 @@ public class RemoteProcessGroupDTO extends ComponentDTO {
this.flowRefreshed = flowRefreshed;
}
public String getTransportProtocol() {
return transportProtocol;
}
public void setTransportProtocol(String transportProtocol) {
this.transportProtocol = transportProtocol;
}
public String getProxyHost() {
return proxyHost;
}
public void setProxyHost(String proxyHost) {
this.proxyHost = proxyHost;
}
public Integer getProxyPort() {
return proxyPort;
}
public void setProxyPort(Integer proxyPort) {
this.proxyPort = proxyPort;
}
public String getProxyUser() {
return proxyUser;
}
public void setProxyUser(String proxyUser) {
this.proxyUser = proxyUser;
}
public String getProxyPassword() {
return proxyPassword;
}
public void setProxyPassword(String proxyPassword) {
this.proxyPassword = proxyPassword;
}
}

View File

@ -0,0 +1,78 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.web.api.dto.remote;
import com.wordnik.swagger.annotations.ApiModelProperty;
import javax.xml.bind.annotation.XmlType;
/**
* Details of a Site-to-Site peer within this NiFi.
*/
@XmlType(name = "peer")
public class PeerDTO {
private String hostname;
private int port;
private boolean secure;
private int flowFileCount;
@ApiModelProperty(
value = "The hostname of this peer."
)
public String getHostname() {
return hostname;
}
public void setHostname(String hostname) {
this.hostname = hostname;
}
@ApiModelProperty(
value = "The port number of this peer."
)
public int getPort() {
return port;
}
public void setPort(int port) {
this.port = port;
}
@ApiModelProperty(
value = "Returns if this peer connection is secure."
)
public boolean isSecure() {
return secure;
}
public void setSecure(boolean secure) {
this.secure = secure;
}
@ApiModelProperty(
value = "The number of flowFiles this peer holds."
)
public int getFlowFileCount() {
return flowFileCount;
}
public void setFlowFileCount(int flowFileCount) {
this.flowFileCount = flowFileCount;
}
}

View File

@ -0,0 +1,46 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.web.api.entity;
import org.apache.nifi.web.api.dto.remote.PeerDTO;
import javax.xml.bind.annotation.XmlRootElement;
import java.util.Collection;
/**
* A serialized representation of this class can be placed in the entity body of a request or response to or from the API.
* This particular entity holds a reference to PeerDTOs.
*/
@XmlRootElement(name = "peersEntity")
public class PeersEntity extends Entity {
private Collection<PeerDTO> peers;
/**
* The PeersDTO that is being serialized.
*
* @return The PeersDTO object
*/
public Collection<PeerDTO> getPeers() {
return peers;
}
public void setPeers(Collection<PeerDTO> peers) {
this.peers = peers;
}
}

View File

@ -0,0 +1,53 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT 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.web.api.entity;
import javax.xml.bind.annotation.XmlRootElement;
@XmlRootElement(name = "transactionResultEntity")
public class TransactionResultEntity extends Entity {
private int flowFileSent;
private int responseCode;
private String message;
public String getMessage() {
return message;
}
public void setMessage(String message) {
this.message = message;
}
public int getResponseCode() {
return responseCode;
}
public void setResponseCode(int responseCode) {
this.responseCode = responseCode;
}
public int getFlowFileSent() {
return flowFileSent;
}
public void setFlowFileSent(int flowFileSent) {
this.flowFileSent = flowFileSent;
}
}

View File

@ -40,6 +40,7 @@ public class ConnectionResponse {
private final NodeIdentifier nodeIdentifier;
private final DataFlow dataFlow;
private final Integer managerRemoteInputPort;
private final Integer managerRemoteInputHttpPort;
private final Boolean managerRemoteCommsSecure;
private final String instanceId;
private final List<NodeConnectionStatus> nodeStatuses;
@ -48,7 +49,7 @@ public class ConnectionResponse {
private volatile String coordinatorDN;
public ConnectionResponse(final NodeIdentifier nodeIdentifier, final DataFlow dataFlow,
final Integer managerRemoteInputPort, final Boolean managerRemoteCommsSecure, final String instanceId,
final Integer managerRemoteInputPort, final Integer managerRemoteInputHttpPort, final Boolean managerRemoteCommsSecure, final String instanceId,
final List<NodeConnectionStatus> nodeStatuses, final List<ComponentRevision> componentRevisions) {
if (nodeIdentifier == null) {
@ -61,6 +62,7 @@ public class ConnectionResponse {
this.tryLaterSeconds = 0;
this.rejectionReason = null;
this.managerRemoteInputPort = managerRemoteInputPort;
this.managerRemoteInputHttpPort = managerRemoteInputHttpPort;
this.managerRemoteCommsSecure = managerRemoteCommsSecure;
this.instanceId = instanceId;
this.nodeStatuses = Collections.unmodifiableList(new ArrayList<>(nodeStatuses));
@ -76,6 +78,7 @@ public class ConnectionResponse {
this.tryLaterSeconds = tryLaterSeconds;
this.rejectionReason = null;
this.managerRemoteInputPort = null;
this.managerRemoteInputHttpPort = null;
this.managerRemoteCommsSecure = null;
this.instanceId = null;
this.nodeStatuses = null;
@ -88,6 +91,7 @@ public class ConnectionResponse {
this.tryLaterSeconds = 0;
this.rejectionReason = rejectionReason;
this.managerRemoteInputPort = null;
this.managerRemoteInputHttpPort = null;
this.managerRemoteCommsSecure = null;
this.instanceId = null;
this.nodeStatuses = null;
@ -130,6 +134,10 @@ public class ConnectionResponse {
return managerRemoteInputPort;
}
public Integer getManagerRemoteInputHttpPort() {
return managerRemoteInputHttpPort;
}
public Boolean isManagerRemoteCommsSecure() {
return managerRemoteCommsSecure;
}

View File

@ -51,7 +51,8 @@ public class NodeIdentifier {
private final String apiAddress;
/**
* the port to use use for sending requests to the node's external interface
* the port to use use for sending requests to the node's external interface,
* this can be HTTP API port or HTTPS API port depending on whether //TODO: .
*/
private final int apiPort;
@ -72,24 +73,31 @@ public class NodeIdentifier {
private final String siteToSiteAddress;
/**
* the port that external clients should use to communicate with this node via Site-to-Site
* the port that external clients should use to communicate with this node via Site-to-Site RAW Socket protocol
*/
private final Integer siteToSitePort;
/**
* the port that external clients should use to communicate with this node via Site-to-Site HTTP protocol,
* this can be HTTP API port or HTTPS API port depending on whether siteToSiteSecure or not.
*/
private final Integer siteToSiteHttpApiPort;
/**
* whether or not site-to-site communications with this node are secure
*/
private Boolean siteToSiteSecure;
private final Boolean siteToSiteSecure;
private final String nodeDn;
public NodeIdentifier(final String id, final String apiAddress, final int apiPort, final String socketAddress, final int socketPort,
final String siteToSiteAddress, final Integer siteToSitePort, final boolean siteToSiteSecure) {
this(id, apiAddress, apiPort, socketAddress, socketPort, siteToSiteAddress, siteToSitePort, siteToSiteSecure, null);
final String siteToSiteAddress, final Integer siteToSitePort, final Integer siteToSiteHttpApiPort, final boolean siteToSiteSecure) {
this(id, apiAddress, apiPort, socketAddress, socketPort, siteToSiteAddress, siteToSitePort, siteToSiteHttpApiPort, siteToSiteSecure, null);
}
public NodeIdentifier(final String id, final String apiAddress, final int apiPort, final String socketAddress, final int socketPort,
final String siteToSiteAddress, final Integer siteToSitePort, final boolean siteToSiteSecure, final String dn) {
final String siteToSiteAddress, final Integer siteToSitePort, final Integer siteToSiteHttpApiPort, final boolean siteToSiteSecure, final String dn) {
if (StringUtils.isBlank(id)) {
throw new IllegalArgumentException("Node ID may not be empty or null.");
@ -113,6 +121,7 @@ public class NodeIdentifier {
this.nodeDn = dn;
this.siteToSiteAddress = siteToSiteAddress == null ? apiAddress : siteToSiteAddress;
this.siteToSitePort = siteToSitePort;
this.siteToSiteHttpApiPort = siteToSiteHttpApiPort;
this.siteToSiteSecure = siteToSiteSecure;
}
@ -128,6 +137,7 @@ public class NodeIdentifier {
this.nodeDn = null;
this.siteToSiteAddress = null;
this.siteToSitePort = null;
this.siteToSiteHttpApiPort = null;
this.siteToSiteSecure = false;
}
@ -169,6 +179,10 @@ public class NodeIdentifier {
return siteToSitePort;
}
public Integer getSiteToSiteHttpApiPort() {
return siteToSiteHttpApiPort;
}
public boolean isSiteToSiteSecure() {
return siteToSiteSecure;
}

View File

@ -34,6 +34,7 @@ public class AdaptedConnectionResponse {
private String rejectionReason;
private int tryLaterSeconds;
private Integer managerRemoteInputPort;
private Integer managerRemoteInputHttpPort;
private Boolean managerRemoteCommsSecure;
private String instanceId;
private List<NodeConnectionStatus> nodeStatuses;
@ -88,6 +89,14 @@ public class AdaptedConnectionResponse {
return managerRemoteInputPort;
}
public void setManagerRemoteInputHttpPort(Integer managerRemoteInputHttpPort) {
this.managerRemoteInputHttpPort = managerRemoteInputHttpPort;
}
public Integer getManagerRemoteInputHttpPort() {
return managerRemoteInputHttpPort;
}
public void setManagerRemoteCommsSecure(Boolean secure) {
this.managerRemoteCommsSecure = secure;
}

View File

@ -27,6 +27,8 @@ public class AdaptedNodeIdentifier {
private int socketPort;
private String siteToSiteAddress;
private Integer siteToSitePort;
private Integer siteToSiteHttpApiPort;
private boolean siteToSiteSecure;
public AdaptedNodeIdentifier() {
@ -96,4 +98,13 @@ public class AdaptedNodeIdentifier {
public void setSiteToSiteSecure(boolean siteToSiteSecure) {
this.siteToSiteSecure = siteToSiteSecure;
}
public Integer getSiteToSiteHttpApiPort() {
return siteToSiteHttpApiPort;
}
public void setSiteToSiteHttpApiPort(Integer siteToSiteHttpApiPort) {
this.siteToSiteHttpApiPort = siteToSiteHttpApiPort;
}
}

View File

@ -32,6 +32,7 @@ public class ConnectionResponseAdapter extends XmlAdapter<AdaptedConnectionRespo
aCr.setTryLaterSeconds(cr.getTryLaterSeconds());
aCr.setRejectionReason(cr.getRejectionReason());
aCr.setManagerRemoteInputPort(cr.getManagerRemoteInputPort());
aCr.setManagerRemoteInputHttpPort(cr.getManagerRemoteInputHttpPort());
aCr.setManagerRemoteCommsSecure(cr.isManagerRemoteCommsSecure());
aCr.setInstanceId(cr.getInstanceId());
aCr.setNodeConnectionStatuses(cr.getNodeConnectionStatuses());
@ -48,7 +49,7 @@ public class ConnectionResponseAdapter extends XmlAdapter<AdaptedConnectionRespo
return ConnectionResponse.createRejectionResponse(aCr.getRejectionReason());
} else {
return new ConnectionResponse(aCr.getNodeIdentifier(), aCr.getDataFlow(),
aCr.getManagerRemoteInputPort(), aCr.isManagerRemoteCommsSecure(),
aCr.getManagerRemoteInputPort(), aCr.getManagerRemoteInputHttpPort(), aCr.isManagerRemoteCommsSecure(),
aCr.getInstanceId(), aCr.getNodeConnectionStatuses(), aCr.getComponentRevisions());
}
}

View File

@ -36,6 +36,7 @@ public class NodeIdentifierAdapter extends XmlAdapter<AdaptedNodeIdentifier, Nod
aNi.setSocketPort(ni.getSocketPort());
aNi.setSiteToSiteAddress(ni.getSiteToSiteAddress());
aNi.setSiteToSitePort(ni.getSiteToSitePort());
aNi.setSiteToSiteHttpApiPort(ni.getSiteToSiteHttpApiPort());
aNi.setSiteToSiteSecure(ni.isSiteToSiteSecure());
return aNi;
}
@ -47,7 +48,7 @@ public class NodeIdentifierAdapter extends XmlAdapter<AdaptedNodeIdentifier, Nod
return null;
} else {
return new NodeIdentifier(aNi.getId(), aNi.getApiAddress(), aNi.getApiPort(), aNi.getSocketAddress(), aNi.getSocketPort(),
aNi.getSiteToSiteAddress(), aNi.getSiteToSitePort(), aNi.isSiteToSiteSecure());
aNi.getSiteToSiteAddress(), aNi.getSiteToSitePort(),aNi.getSiteToSiteHttpApiPort(), aNi.isSiteToSiteSecure());
}
}

View File

@ -36,6 +36,7 @@ public class ReconnectionRequestMessage extends ProtocolMessage {
private StandardDataFlow dataFlow;
private boolean primary;
private Integer managerRemoteSiteListeningPort;
private Integer managerRemoteSiteListeningHttpPort;
private Boolean managerRemoteSiteCommsSecure;
private String instanceId;
private List<NodeConnectionStatus> nodeStatuses;
@ -82,6 +83,14 @@ public class ReconnectionRequestMessage extends ProtocolMessage {
return managerRemoteSiteListeningPort;
}
public void setManagerRemoteSiteListeningHttpPort(Integer managerRemoteSiteListeningHttpPort) {
this.managerRemoteSiteListeningHttpPort = managerRemoteSiteListeningHttpPort;
}
public Integer getManagerRemoteSiteListeningHttpPort() {
return managerRemoteSiteListeningHttpPort;
}
public void setManagerRemoteSiteCommsSecure(final Boolean remoteSiteCommsSecure) {
this.managerRemoteSiteCommsSecure = remoteSiteCommsSecure;
}

View File

@ -45,11 +45,11 @@ public class TestJaxbProtocolUtils {
final ByteArrayOutputStream baos = new ByteArrayOutputStream();
final ConnectionResponseMessage msg = new ConnectionResponseMessage();
final NodeIdentifier nodeId = new NodeIdentifier("id", "localhost", 8000, "localhost", 8001, "localhost", 8002, true);
final NodeIdentifier nodeId = new NodeIdentifier("id", "localhost", 8000, "localhost", 8001, "localhost", 8002, 8003, true);
final DataFlow dataFlow = new StandardDataFlow(new byte[0], new byte[0]);
final List<NodeConnectionStatus> nodeStatuses = Collections.singletonList(new NodeConnectionStatus(nodeId, DisconnectionCode.NOT_YET_CONNECTED));
final List<ComponentRevision> componentRevisions = Collections.singletonList(ComponentRevision.fromRevision(new Revision(8L, "client-1", "component-1")));
msg.setConnectionResponse(new ConnectionResponse(nodeId, dataFlow, 80, false, "instance-1", nodeStatuses, componentRevisions));
msg.setConnectionResponse(new ConnectionResponse(nodeId, dataFlow, 9990, 8080, false, "instance-1", nodeStatuses, componentRevisions));
JaxbProtocolUtils.JAXB_CONTEXT.createMarshaller().marshal(msg, baos);
final Object unmarshalled = JaxbProtocolUtils.JAXB_CONTEXT.createUnmarshaller().unmarshal(new ByteArrayInputStream(baos.toByteArray()));

View File

@ -620,8 +620,8 @@ public class NodeClusterCoordinator implements ClusterCoordinator, ProtocolHandl
} else {
// there is a node with that ID and it's a different node
resolvedNodeId = new NodeIdentifier(UUID.randomUUID().toString(), proposedIdentifier.getApiAddress(), proposedIdentifier.getApiPort(),
proposedIdentifier.getSocketAddress(), proposedIdentifier.getSocketPort(),
proposedIdentifier.getSiteToSiteAddress(), proposedIdentifier.getSiteToSitePort(), proposedIdentifier.isSiteToSiteSecure());
proposedIdentifier.getSocketAddress(), proposedIdentifier.getSocketPort(), proposedIdentifier.getSiteToSiteAddress(),
proposedIdentifier.getSiteToSitePort(), proposedIdentifier.getSiteToSiteHttpApiPort(), proposedIdentifier.isSiteToSiteSecure());
logger.debug("A node already exists with ID {}. Proposed Node Identifier was {}; existing Node Identifier is {}; Resolved Node Identifier is {}",
proposedIdentifier.getId(), proposedIdentifier, getNodeIdentifier(proposedIdentifier.getId()), resolvedNodeId);
}
@ -682,7 +682,7 @@ public class NodeClusterCoordinator implements ClusterCoordinator, ProtocolHandl
// TODO: Remove the 'null' values here from the ConnectionResponse all together. These
// will no longer be needed for site-to-site once the NCM is gone.
return new ConnectionResponse(resolvedNodeIdentifier, dataFlow, null, null, instanceId,
return new ConnectionResponse(resolvedNodeIdentifier, dataFlow, null, null, null, instanceId,
new ArrayList<>(nodeStatuses.values()),
revisionManager.getAllRevisions().stream().map(rev -> ComponentRevision.fromRevision(rev)).collect(Collectors.toList()));
}
@ -690,7 +690,8 @@ public class NodeClusterCoordinator implements ClusterCoordinator, ProtocolHandl
private NodeIdentifier addRequestorDn(final NodeIdentifier nodeId, final String dn) {
return new NodeIdentifier(nodeId.getId(), nodeId.getApiAddress(), nodeId.getApiPort(),
nodeId.getSocketAddress(), nodeId.getSocketPort(),
nodeId.getSiteToSiteAddress(), nodeId.getSiteToSitePort(), nodeId.isSiteToSiteSecure(), dn);
nodeId.getSiteToSiteAddress(), nodeId.getSiteToSitePort(),
nodeId.getSiteToSiteHttpApiPort(), nodeId.isSiteToSiteSecure(), dn);
}
@Override

View File

@ -51,7 +51,7 @@ public class TestAbstractHeartbeatMonitor {
@Before
public void setup() throws Exception {
nodeId = new NodeIdentifier(UUID.randomUUID().toString(), "localhost", 9999, "localhost", 8888, "localhost", null, false);
nodeId = new NodeIdentifier(UUID.randomUUID().toString(), "localhost", 9999, "localhost", 8888, "localhost", null, null, false);
}
@After

View File

@ -36,14 +36,14 @@ public class TestProcessorEndpointMerger {
final ProcessorEndpointMerger merger = new ProcessorEndpointMerger();
final Map<String, Set<NodeIdentifier>> validationErrorMap = new HashMap<>();
final NodeIdentifier nodeId1234 = new NodeIdentifier("1234", "localhost", 9000, "localhost", 9001, "localhost", 9002, false);
final NodeIdentifier nodeId1234 = new NodeIdentifier("1234", "localhost", 9000, "localhost", 9001, "localhost", 9002, 9003, false);
final List<String> nodeValidationErrors1234 = new ArrayList<>();
nodeValidationErrors1234.add("error 1");
nodeValidationErrors1234.add("error 2");
merger.mergeValidationErrors(validationErrorMap, nodeId1234, nodeValidationErrors1234);
final NodeIdentifier nodeXyz = new NodeIdentifier("xyz", "localhost", 8000, "localhost", 8001, "localhost", 8002, false);
final NodeIdentifier nodeXyz = new NodeIdentifier("xyz", "localhost", 8000, "localhost", 8001, "localhost", 8002, 8003, false);
final List<String> nodeValidationErrorsXyz = new ArrayList<>();
nodeValidationErrorsXyz.add("error 1");

View File

@ -40,10 +40,10 @@ public class TestResponseUtils {
@Test
public void testFindLongResponseTimes() throws URISyntaxException {
final Map<NodeIdentifier, NodeResponse> responses = new HashMap<>();
final NodeIdentifier id1 = new NodeIdentifier("1", "localhost", 8000, "localhost", 8001, "localhost", 8002, false);
final NodeIdentifier id2 = new NodeIdentifier("2", "localhost", 8200, "localhost", 8201, "localhost", 8202, false);
final NodeIdentifier id3 = new NodeIdentifier("3", "localhost", 8300, "localhost", 8301, "localhost", 8302, false);
final NodeIdentifier id4 = new NodeIdentifier("4", "localhost", 8400, "localhost", 8401, "localhost", 8402, false);
final NodeIdentifier id1 = new NodeIdentifier("1", "localhost", 8000, "localhost", 8001, "localhost", 8002, 8003, false);
final NodeIdentifier id2 = new NodeIdentifier("2", "localhost", 8200, "localhost", 8201, "localhost", 8202, 8203, false);
final NodeIdentifier id3 = new NodeIdentifier("3", "localhost", 8300, "localhost", 8301, "localhost", 8302, 8303, false);
final NodeIdentifier id4 = new NodeIdentifier("4", "localhost", 8400, "localhost", 8401, "localhost", 8402, 8403, false);
final URI uri = new URI("localhost:8080");
final ClientResponse clientResponse = Mockito.mock(ClientResponse.class);

View File

@ -82,7 +82,7 @@ public class TestThreadPoolRequestReplicator {
public void testResponseRemovedWhenCompletedAndFetched() {
withReplicator(replicator -> {
final Set<NodeIdentifier> nodeIds = new HashSet<>();
nodeIds.add(new NodeIdentifier("1", "localhost", 8000, "localhost", 8001, "localhost", 8002, false));
nodeIds.add(new NodeIdentifier("1", "localhost", 8000, "localhost", 8001, "localhost", 8002, 8003, false));
final URI uri = new URI("http://localhost:8080/processors/1");
final Entity entity = new ProcessorEntity();
@ -109,7 +109,7 @@ public class TestThreadPoolRequestReplicator {
public void testLongWaitForResponse() {
withReplicator(replicator -> {
final Set<NodeIdentifier> nodeIds = new HashSet<>();
final NodeIdentifier nodeId = new NodeIdentifier("1", "localhost", 8000, "localhost", 8001, "localhost", 8002, false);
final NodeIdentifier nodeId = new NodeIdentifier("1", "localhost", 8000, "localhost", 8001, "localhost", 8002, 8003, false);
nodeIds.add(nodeId);
final URI uri = new URI("http://localhost:8080/processors/1");
final Entity entity = new ProcessorEntity();
@ -138,10 +138,10 @@ public class TestThreadPoolRequestReplicator {
public void testCompleteOnError() {
withReplicator(replicator -> {
final Set<NodeIdentifier> nodeIds = new HashSet<>();
final NodeIdentifier id1 = new NodeIdentifier("1", "localhost", 8100, "localhost", 8101, "localhost", 8102, false);
final NodeIdentifier id2 = new NodeIdentifier("2", "localhost", 8200, "localhost", 8201, "localhost", 8202, false);
final NodeIdentifier id3 = new NodeIdentifier("3", "localhost", 8300, "localhost", 8301, "localhost", 8302, false);
final NodeIdentifier id4 = new NodeIdentifier("4", "localhost", 8400, "localhost", 8401, "localhost", 8402, false);
final NodeIdentifier id1 = new NodeIdentifier("1", "localhost", 8100, "localhost", 8101, "localhost", 8102, 8103, false);
final NodeIdentifier id2 = new NodeIdentifier("2", "localhost", 8200, "localhost", 8201, "localhost", 8202, 8203, false);
final NodeIdentifier id3 = new NodeIdentifier("3", "localhost", 8300, "localhost", 8301, "localhost", 8302, 8303, false);
final NodeIdentifier id4 = new NodeIdentifier("4", "localhost", 8400, "localhost", 8401, "localhost", 8402, 8403, false);
nodeIds.add(id1);
nodeIds.add(id2);
nodeIds.add(id3);
@ -159,7 +159,7 @@ public class TestThreadPoolRequestReplicator {
@Test(timeout = 15000)
public void testMultipleRequestWithTwoPhaseCommit() {
final Set<NodeIdentifier> nodeIds = new HashSet<>();
final NodeIdentifier nodeId = new NodeIdentifier("1", "localhost", 8100, "localhost", 8101, "localhost", 8102, false);
final NodeIdentifier nodeId = new NodeIdentifier("1", "localhost", 8100, "localhost", 8101, "localhost", 8102, 8103, false);
nodeIds.add(nodeId);
final ClusterCoordinator coordinator = Mockito.mock(ClusterCoordinator.class);
@ -223,12 +223,12 @@ public class TestThreadPoolRequestReplicator {
// build a map of connection state to node ids
final Map<NodeConnectionState, List<NodeIdentifier>> nodeMap = new HashMap<>();
final List<NodeIdentifier> connectedNodes = new ArrayList<>();
connectedNodes.add(new NodeIdentifier("1", "localhost", 8100, "localhost", 8101, "localhost", 8102, false));
connectedNodes.add(new NodeIdentifier("2", "localhost", 8200, "localhost", 8201, "localhost", 8202, false));
connectedNodes.add(new NodeIdentifier("1", "localhost", 8100, "localhost", 8101, "localhost", 8102, 8103, false));
connectedNodes.add(new NodeIdentifier("2", "localhost", 8200, "localhost", 8201, "localhost", 8202, 8203, false));
nodeMap.put(NodeConnectionState.CONNECTED, connectedNodes);
final List<NodeIdentifier> otherState = new ArrayList<>();
otherState.add(new NodeIdentifier("3", "localhost", 8300, "localhost", 8301, "localhost", 8302, false));
otherState.add(new NodeIdentifier("3", "localhost", 8300, "localhost", 8301, "localhost", 8302, 8303, false));
nodeMap.put(NodeConnectionState.CONNECTING, otherState);
Mockito.when(coordinator.getConnectionStates()).thenReturn(nodeMap);
@ -280,8 +280,8 @@ public class TestThreadPoolRequestReplicator {
@Test(timeout = 15000)
public void testOneNodeRejectsTwoPhaseCommit() {
final Set<NodeIdentifier> nodeIds = new HashSet<>();
nodeIds.add(new NodeIdentifier("1", "localhost", 8100, "localhost", 8101, "localhost", 8102, false));
nodeIds.add(new NodeIdentifier("2", "localhost", 8200, "localhost", 8201, "localhost", 8202, false));
nodeIds.add(new NodeIdentifier("1", "localhost", 8100, "localhost", 8101, "localhost", 8102, 8103, false));
nodeIds.add(new NodeIdentifier("2", "localhost", 8200, "localhost", 8201, "localhost", 8202, 8203, false));
final ClusterCoordinator coordinator = createClusterCoordinator();
final AtomicInteger requestCount = new AtomicInteger(0);

View File

@ -450,8 +450,8 @@ public class TestNodeClusterCoordinator {
@Test
public void testProposedIdentifierResolvedIfConflict() {
final NodeIdentifier id1 = new NodeIdentifier("1234", "localhost", 8000, "localhost", 9000, "localhost", 10000, false);
final NodeIdentifier conflictingId = new NodeIdentifier("1234", "localhost", 8001, "localhost", 9000, "localhost", 10000, false);
final NodeIdentifier id1 = new NodeIdentifier("1234", "localhost", 8000, "localhost", 9000, "localhost", 10000, 11000, false);
final NodeIdentifier conflictingId = new NodeIdentifier("1234", "localhost", 8001, "localhost", 9000, "localhost", 10000, 11000, false);
final ConnectionRequest connectionRequest = new ConnectionRequest(id1);
final ConnectionRequestMessage crm = new ConnectionRequestMessage();
@ -484,7 +484,7 @@ public class TestNodeClusterCoordinator {
private NodeIdentifier createNodeId(final int index) {
return new NodeIdentifier(String.valueOf(index), "localhost", 8000 + index, "localhost", 9000 + index, "localhost", 10000 + index, false);
return new NodeIdentifier(String.valueOf(index), "localhost", 8000 + index, "localhost", 9000 + index, "localhost", 10000 + index, 11000 + index, false);
}
private ProtocolMessage requestConnection(final NodeIdentifier requestedNodeId, final NodeClusterCoordinator coordinator) {

View File

@ -21,6 +21,7 @@ import org.apache.nifi.connectable.Positionable;
import org.apache.nifi.controller.exception.CommunicationsException;
import org.apache.nifi.events.EventReporter;
import org.apache.nifi.remote.RemoteGroupPort;
import org.apache.nifi.remote.protocol.SiteToSiteTransportProtocol;
import java.net.URI;
import java.util.Date;
@ -158,6 +159,26 @@ public interface RemoteProcessGroup extends Authorizable, Positionable {
*/
EventReporter getEventReporter();
SiteToSiteTransportProtocol getTransportProtocol();
void setTransportProtocol(SiteToSiteTransportProtocol transportProtocol);
String getProxyHost();
void setProxyHost(String proxyHost);
Integer getProxyPort();
void setProxyPort(Integer proxyPort);
String getProxyUser();
void setProxyUser(String proxyUser);
String getProxyPassword();
void setProxyPassword(String proxyPassword);
/**
* Initiates a task in the remote process group to re-initialize, as a
* result of clustering changes

View File

@ -16,15 +16,14 @@
*/
package org.apache.nifi.remote;
import java.util.Map;
import java.util.Set;
import org.apache.nifi.connectable.Port;
import org.apache.nifi.remote.exception.BadRequestException;
import org.apache.nifi.remote.exception.NotAuthorizedException;
import org.apache.nifi.remote.exception.RequestExpiredException;
import org.apache.nifi.remote.protocol.ServerProtocol;
import java.util.Set;
public interface RootGroupPort extends Port {
boolean isTransmitting();
@ -52,20 +51,18 @@ public interface RootGroupPort extends Port {
*
* @param peer peer
* @param serverProtocol protocol
* @param requestHeaders headers
*
* @return the number of FlowFiles received
* @throws org.apache.nifi.remote.exception.NotAuthorizedException nae
* @throws org.apache.nifi.remote.exception.BadRequestException bre
* @throws org.apache.nifi.remote.exception.RequestExpiredException ree
*/
int receiveFlowFiles(Peer peer, ServerProtocol serverProtocol, Map<String, String> requestHeaders) throws NotAuthorizedException, BadRequestException, RequestExpiredException;
int receiveFlowFiles(Peer peer, ServerProtocol serverProtocol) throws NotAuthorizedException, BadRequestException, RequestExpiredException;
/**
* Transfers data to the given stream
*
* @param peer peer
* @param requestHeaders headers
* @param serverProtocol protocol
*
* @return the number of FlowFiles transferred
@ -73,6 +70,6 @@ public interface RootGroupPort extends Port {
* @throws org.apache.nifi.remote.exception.BadRequestException bre
* @throws org.apache.nifi.remote.exception.RequestExpiredException ree
*/
int transferFlowFiles(Peer peer, ServerProtocol serverProtocol, Map<String, String> requestHeaders) throws NotAuthorizedException, BadRequestException, RequestExpiredException;
int transferFlowFiles(Peer peer, ServerProtocol serverProtocol) throws NotAuthorizedException, BadRequestException, RequestExpiredException;
}

View File

@ -185,6 +185,7 @@ import org.apache.nifi.provenance.ProvenanceEventRecord;
import org.apache.nifi.provenance.ProvenanceEventRepository;
import org.apache.nifi.provenance.ProvenanceEventType;
import org.apache.nifi.provenance.StandardProvenanceEventRecord;
import org.apache.nifi.remote.HttpRemoteSiteListener;
import org.apache.nifi.remote.RemoteGroupPort;
import org.apache.nifi.remote.RemoteResourceManager;
import org.apache.nifi.remote.RemoteSiteListener;
@ -194,6 +195,7 @@ import org.apache.nifi.remote.StandardRemoteProcessGroup;
import org.apache.nifi.remote.StandardRemoteProcessGroupPortDescriptor;
import org.apache.nifi.remote.StandardRootGroupPort;
import org.apache.nifi.remote.TransferDirection;
import org.apache.nifi.remote.protocol.SiteToSiteTransportProtocol;
import org.apache.nifi.remote.protocol.socket.SocketFlowFileServerProtocol;
import org.apache.nifi.reporting.Bulletin;
import org.apache.nifi.reporting.BulletinRepository;
@ -268,7 +270,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
private final ExtensionManager extensionManager;
private final NiFiProperties properties;
private final SSLContext sslContext;
private final RemoteSiteListener externalSiteListener;
private final Set<RemoteSiteListener> externalSiteListeners = new HashSet<>();
private final AtomicReference<CounterRepository> counterRepositoryRef;
private final AtomicBoolean initialized = new AtomicBoolean(false);
private final StandardControllerServiceProvider controllerServiceProvider;
@ -289,8 +291,10 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
private final AtomicBoolean heartbeatsSuspended = new AtomicBoolean(false);
private final Integer remoteInputSocketPort;
private final Integer remoteInputHttpPort;
private final Boolean isSiteToSiteSecure;
private Integer clusterManagerRemoteSitePort = null;
private Integer clusterManagerRemoteSiteHttpPort = null;
private Boolean clusterManagerRemoteSiteCommsSecure = null;
private ProcessGroup rootGroup;
@ -398,7 +402,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
bulletinRepo,
heartbeatMonitor);
flowController.setClusterManagerRemoteSiteInfo(properties.getRemoteInputPort(), properties.isSiteToSiteSecure());
flowController.setClusterManagerRemoteSiteInfo(properties.getRemoteInputPort(), properties.getRemoteInputHttpPort(), properties.isSiteToSiteSecure());
return flowController;
}
@ -484,6 +488,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
gracefulShutdownSeconds = shutdownSecs;
remoteInputSocketPort = properties.getRemoteInputPort();
remoteInputHttpPort = properties.getRemoteInputHttpPort();
isSiteToSiteSecure = properties.isSiteToSiteSecure();
if (isSiteToSiteSecure && sslContext == null && remoteInputSocketPort != null) {
@ -503,17 +508,24 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
controllerServiceProvider.setRootProcessGroup(rootGroup);
if (remoteInputSocketPort == null) {
LOG.info("Not enabling Site-to-Site functionality because nifi.remote.input.socket.port is not set");
externalSiteListener = null;
LOG.info("Not enabling RAW Socket Site-to-Site functionality because nifi.remote.input.socket.port is not set");
} else if (isSiteToSiteSecure && sslContext == null) {
LOG.error("Unable to create Secure Site-to-Site Listener because not all required Keystore/Truststore "
+ "Properties are set. Site-to-Site functionality will be disabled until this problem is has been fixed.");
externalSiteListener = null;
} else {
// Register the SocketFlowFileServerProtocol as the appropriate resource for site-to-site Server Protocol
RemoteResourceManager.setServerProtocolImplementation(SocketFlowFileServerProtocol.RESOURCE_NAME, SocketFlowFileServerProtocol.class);
externalSiteListener = new SocketRemoteSiteListener(remoteInputSocketPort, isSiteToSiteSecure ? sslContext : null);
externalSiteListener.setRootGroup(rootGroup);
externalSiteListeners.add(new SocketRemoteSiteListener(remoteInputSocketPort, isSiteToSiteSecure ? sslContext : null));
}
if (remoteInputHttpPort == null) {
LOG.info("Not enabling HTTP(S) Site-to-Site functionality because nifi.remote.input.html.enabled is not true");
} else {
externalSiteListeners.add(HttpRemoteSiteListener.getInstance());
}
for(RemoteSiteListener listener : externalSiteListeners) {
listener.setRootGroup(rootGroup);
}
// Determine frequency for obtaining component status snapshots
@ -650,8 +662,8 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
// ContentRepository to purge superfluous files
contentRepository.cleanup();
if (externalSiteListener != null) {
externalSiteListener.start();
for(RemoteSiteListener listener : externalSiteListeners) {
listener.start();
}
notifyComponentsConfigurationRestored();
@ -1288,8 +1300,8 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
+ "will take an indeterminate amount of time to stop. Might need to kill the program manually.");
}
if (externalSiteListener != null) {
externalSiteListener.stop();
for(RemoteSiteListener listener : externalSiteListeners) {
listener.stop();
}
if (processScheduler != null) {
@ -1433,8 +1445,8 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
try {
rootGroup = group;
if (externalSiteListener != null) {
externalSiteListener.setRootGroup(group);
for(RemoteSiteListener listener : externalSiteListeners) {
listener.setRootGroup(rootGroup);
}
controllerServiceProvider.setRootProcessGroup(rootGroup);
@ -1661,6 +1673,11 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
remoteGroup.setPosition(toPosition(remoteGroupDTO.getPosition()));
remoteGroup.setCommunicationsTimeout(remoteGroupDTO.getCommunicationsTimeout());
remoteGroup.setYieldDuration(remoteGroupDTO.getYieldDuration());
remoteGroup.setTransportProtocol(SiteToSiteTransportProtocol.valueOf(remoteGroupDTO.getTransportProtocol()));
remoteGroup.setProxyHost(remoteGroupDTO.getProxyHost());
remoteGroup.setProxyPort(remoteGroupDTO.getProxyPort());
remoteGroup.setProxyUser(remoteGroupDTO.getProxyUser());
remoteGroup.setProxyPassword(remoteGroupDTO.getProxyPassword());
remoteGroup.setProcessGroup(group);
// set the input/output ports
@ -3778,10 +3795,11 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
return new ArrayList<>(history.getActions());
}
public void setClusterManagerRemoteSiteInfo(final Integer managerListeningPort, final Boolean commsSecure) {
public void setClusterManagerRemoteSiteInfo(final Integer managerListeningPort, final Integer managerListeningHttpPort, final Boolean commsSecure) {
writeLock.lock();
try {
clusterManagerRemoteSitePort = managerListeningPort;
clusterManagerRemoteSiteHttpPort = managerListeningHttpPort;
clusterManagerRemoteSiteCommsSecure = commsSecure;
} finally {
writeLock.unlock();
@ -3797,6 +3815,16 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
}
}
public Integer getClusterManagerRemoteSiteListeningHttpPort() {
readLock.lock();
try {
return clusterManagerRemoteSiteHttpPort;
} finally {
readLock.unlock();
}
}
public Boolean isClusterManagerRemoteSiteCommsSecure() {
readLock.lock();
try {
@ -3810,6 +3838,10 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
return remoteInputSocketPort;
}
public Integer getRemoteSiteListeningHttpPort() {
return remoteInputHttpPort;
}
public Boolean isRemoteSiteCommsSecure() {
return isSiteToSiteSecure;
}

View File

@ -205,7 +205,8 @@ public class StandardFlowService implements FlowService, ProtocolHandler {
this.nodeId = new NodeIdentifier(nodeUuid,
nodeApiAddress.getHostName(), nodeApiAddress.getPort(),
nodeSocketAddress.getHostName(), nodeSocketAddress.getPort(),
properties.getRemoteInputHost(), properties.getRemoteInputPort(), properties.isSiteToSiteSecure());
properties.getRemoteInputHost(), properties.getRemoteInputPort(),
properties.getRemoteInputHttpPort(), properties.isSiteToSiteSecure());
} else {
this.configuredForClustering = false;
@ -471,7 +472,7 @@ public class StandardFlowService implements FlowService, ProtocolHandler {
controller.setClustered(true, null);
clusterCoordinator.setConnected(false);
controller.setClusterManagerRemoteSiteInfo(null, null);
controller.setClusterManagerRemoteSiteInfo(null, null, null);
controller.setConnectionStatus(new NodeConnectionStatus(nodeId, DisconnectionCode.NOT_YET_CONNECTED));
/*
@ -585,7 +586,8 @@ public class StandardFlowService implements FlowService, ProtocolHandler {
// reconnect
final ConnectionResponse connectionResponse = new ConnectionResponse(getNodeId(), request.getDataFlow(),
request.getManagerRemoteSiteListeningPort(), request.isManagerRemoteSiteCommsSecure(), request.getInstanceId(),
request.getManagerRemoteSiteListeningPort(), request.getManagerRemoteSiteListeningHttpPort(),
request.isManagerRemoteSiteCommsSecure(), request.getInstanceId(),
request.getNodeConnectionStatuses(), request.getComponentRevisions());
connectionResponse.setCoordinatorDN(request.getRequestorDN());
@ -848,7 +850,7 @@ public class StandardFlowService implements FlowService, ProtocolHandler {
// mark the node as clustered
controller.setClustered(true, response.getInstanceId(), response.getCoordinatorDN());
controller.setClusterManagerRemoteSiteInfo(response.getManagerRemoteInputPort(), response.isManagerRemoteCommsSecure());
controller.setClusterManagerRemoteSiteInfo(response.getManagerRemoteInputPort(), response.getManagerRemoteInputHttpPort(), response.isManagerRemoteCommsSecure());
controller.setConnectionStatus(new NodeConnectionStatus(nodeId, NodeConnectionState.CONNECTED));

View File

@ -53,6 +53,7 @@ import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.SimpleProcessLogger;
import org.apache.nifi.remote.RemoteGroupPort;
import org.apache.nifi.remote.RootGroupPort;
import org.apache.nifi.remote.protocol.SiteToSiteTransportProtocol;
import org.apache.nifi.reporting.InitializationException;
import org.apache.nifi.reporting.ReportingInitializationContext;
import org.apache.nifi.reporting.Severity;
@ -923,7 +924,7 @@ public class StandardFlowSynchronizer implements FlowSynchronizer {
// add remote process group
final List<Element> remoteProcessGroupNodeList = getChildrenByTagName(processGroupElement, "remoteProcessGroup");
for (final Element remoteProcessGroupElement : remoteProcessGroupNodeList) {
final RemoteProcessGroupDTO remoteGroupDto = FlowFromDOMFactory.getRemoteProcessGroup(remoteProcessGroupElement);
final RemoteProcessGroupDTO remoteGroupDto = FlowFromDOMFactory.getRemoteProcessGroup(remoteProcessGroupElement, encryptor);
final RemoteProcessGroup remoteGroup = controller.createRemoteProcessGroup(remoteGroupDto.getId(), remoteGroupDto.getTargetUri());
remoteGroup.setComments(remoteGroupDto.getComments());
remoteGroup.setPosition(toPosition(remoteGroupDto.getPosition()));
@ -938,6 +939,27 @@ public class StandardFlowSynchronizer implements FlowSynchronizer {
remoteGroup.setYieldDuration(remoteGroupDto.getYieldDuration());
}
String transportProtocol = remoteGroupDto.getTransportProtocol();
if (transportProtocol != null && !transportProtocol.trim().isEmpty()) {
remoteGroup.setTransportProtocol(SiteToSiteTransportProtocol.valueOf(transportProtocol.toUpperCase()));
}
if (remoteGroupDto.getProxyHost() != null) {
remoteGroup.setProxyHost(remoteGroupDto.getProxyHost());
}
if (remoteGroupDto.getProxyPort() != null) {
remoteGroup.setProxyPort(remoteGroupDto.getProxyPort());
}
if (remoteGroupDto.getProxyUser() != null) {
remoteGroup.setProxyUser(remoteGroupDto.getProxyUser());
}
if (remoteGroupDto.getProxyPassword() != null) {
remoteGroup.setProxyPassword(remoteGroupDto.getProxyPassword());
}
final Set<RemoteProcessGroupPortDescriptor> inputPorts = new HashSet<>();
for (final Element portElement : getChildrenByTagName(remoteProcessGroupElement, "inputPort")) {
inputPorts.add(FlowFromDOMFactory.getRemoteProcessGroupPort(portElement));

View File

@ -247,6 +247,7 @@ public class TemplateUtils {
remoteProcessGroupDTO.setInputPortCount(null);
remoteProcessGroupDTO.setOutputPortCount(null);
remoteProcessGroupDTO.setTransmitting(null);
remoteProcessGroupDTO.setProxyPassword(null);
// if this remote process group has contents
if (remoteProcessGroupDTO.getContents() != null) {

View File

@ -175,7 +175,7 @@ public class FlowFromDOMFactory {
nodeList = DomUtils.getChildNodesByTagName(element, "remoteProcessGroup");
for (int i = 0; i < nodeList.getLength(); i++) {
remoteProcessGroups.add(getRemoteProcessGroup((Element) nodeList.item(i)));
remoteProcessGroups.add(getRemoteProcessGroup((Element) nodeList.item(i), encryptor));
}
nodeList = DomUtils.getChildNodesByTagName(element, "connection");
@ -246,7 +246,7 @@ public class FlowFromDOMFactory {
return dto;
}
public static RemoteProcessGroupDTO getRemoteProcessGroup(final Element element) {
public static RemoteProcessGroupDTO getRemoteProcessGroup(final Element element, final StringEncryptor encryptor) {
final RemoteProcessGroupDTO dto = new RemoteProcessGroupDTO();
dto.setId(getString(element, "id"));
dto.setName(getString(element, "name"));
@ -255,6 +255,13 @@ public class FlowFromDOMFactory {
dto.setPosition(getPosition(DomUtils.getChild(element, "position")));
dto.setCommunicationsTimeout(getString(element, "timeout"));
dto.setComments(getString(element, "comment"));
dto.setYieldDuration(getString(element, "yieldPeriod"));
dto.setTransportProtocol(getString(element, "transportProtocol"));
dto.setProxyHost(getString(element, "proxyHost"));
dto.setProxyPort(getOptionalInt(element, "proxyPort"));
dto.setProxyUser(getString(element, "proxyUser"));
String proxyPassword = decrypt(getString(element, "proxyPassword"), encryptor);
dto.setProxyPassword(proxyPassword);
return dto;
}

View File

@ -56,6 +56,7 @@ import org.apache.nifi.persistence.TemplateSerializer;
import org.apache.nifi.processor.Relationship;
import org.apache.nifi.remote.RemoteGroupPort;
import org.apache.nifi.remote.RootGroupPort;
import org.apache.nifi.util.StringUtils;
import org.w3c.dom.DOMException;
import org.w3c.dom.Document;
import org.w3c.dom.Element;
@ -244,6 +245,16 @@ public class StandardFlowSerializer implements FlowSerializer {
addTextElement(element, "timeout", remoteRef.getCommunicationsTimeout());
addTextElement(element, "yieldPeriod", remoteRef.getYieldDuration());
addTextElement(element, "transmitting", String.valueOf(remoteRef.isTransmitting()));
addTextElement(element, "transportProtocol", remoteRef.getTransportProtocol().name());
addTextElement(element, "proxyHost", remoteRef.getProxyHost());
if (remoteRef.getProxyPort() != null) {
addTextElement(element, "proxyPort", remoteRef.getProxyPort());
}
addTextElement(element, "proxyUser", remoteRef.getProxyUser());
if (!StringUtils.isEmpty(remoteRef.getProxyPassword())) {
String value = ENC_PREFIX + encryptor.encrypt(remoteRef.getProxyPassword()) + ENC_SUFFIX;
addTextElement(element, "proxyPassword", value);
}
for (final RemoteGroupPort port : remoteRef.getInputPorts()) {
if (port.hasIncomingConnection()) {

View File

@ -17,7 +17,6 @@
package org.apache.nifi.remote;
import com.sun.jersey.api.client.ClientHandlerException;
import com.sun.jersey.api.client.ClientResponse;
import com.sun.jersey.api.client.ClientResponse.Status;
import com.sun.jersey.api.client.UniformInterfaceException;
import org.apache.nifi.authorization.Resource;
@ -39,6 +38,9 @@ import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.groups.ProcessGroupCounts;
import org.apache.nifi.groups.RemoteProcessGroup;
import org.apache.nifi.groups.RemoteProcessGroupPortDescriptor;
import org.apache.nifi.remote.protocol.SiteToSiteTransportProtocol;
import org.apache.nifi.remote.protocol.http.HttpProxy;
import org.apache.nifi.remote.util.SiteToSiteRestApiClient;
import org.apache.nifi.reporting.BulletinRepository;
import org.apache.nifi.reporting.ComponentType;
import org.apache.nifi.reporting.Severity;
@ -46,13 +48,12 @@ import org.apache.nifi.util.FormatUtils;
import org.apache.nifi.util.NiFiProperties;
import org.apache.nifi.web.api.dto.ControllerDTO;
import org.apache.nifi.web.api.dto.PortDTO;
import org.apache.nifi.web.api.entity.ControllerEntity;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import javax.net.ssl.SSLContext;
import javax.ws.rs.core.Response;
import java.io.File;
import java.io.IOException;
import java.net.URI;
import java.net.URISyntaxException;
import java.util.ArrayList;
@ -82,13 +83,9 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
private static final Logger logger = LoggerFactory.getLogger(StandardRemoteProcessGroup.class);
public static final String SITE_TO_SITE_URI_PATH = "/site-to-site";
public static final String ROOT_GROUP_STATUS_URI_PATH = "/flow/process-groups/root/status";
// status codes
public static final int OK_STATUS_CODE = Status.OK.getStatusCode();
public static final int UNAUTHORIZED_STATUS_CODE = Status.UNAUTHORIZED.getStatusCode();
public static final int FORBIDDEN_STATUS_CODE = Status.FORBIDDEN.getStatusCode();
private static final int UNAUTHORIZED_STATUS_CODE = Status.UNAUTHORIZED.getStatusCode();
private static final int FORBIDDEN_STATUS_CODE = Status.FORBIDDEN.getStatusCode();
private final String id;
@ -112,6 +109,12 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
private volatile String communicationsTimeout = "30 sec";
private volatile String targetId;
private volatile String yieldDuration = "10 sec";
private volatile SiteToSiteTransportProtocol transportProtocol = SiteToSiteTransportProtocol.RAW;
private volatile String proxyHost;
private volatile Integer proxyPort;
private volatile String proxyUser;
private volatile String proxyPassword;
private final ReadWriteLock rwLock = new ReentrantReadWriteLock();
private final Lock readLock = rwLock.readLock();
@ -127,6 +130,7 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
private Long refreshContentsTimestamp = null;
private Boolean destinationSecure;
private Integer listeningPort;
private Integer listeningHttpPort;
private volatile String authorizationIssue;
@ -235,6 +239,56 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
this.targetId = targetId;
}
@Override
public void setTransportProtocol(final SiteToSiteTransportProtocol transportProtocol) {
this.transportProtocol = transportProtocol;
}
@Override
public SiteToSiteTransportProtocol getTransportProtocol() {
return transportProtocol;
}
@Override
public String getProxyHost() {
return proxyHost;
}
@Override
public void setProxyHost(String proxyHost) {
this.proxyHost = proxyHost;
}
@Override
public Integer getProxyPort() {
return proxyPort;
}
@Override
public void setProxyPort(Integer proxyPort) {
this.proxyPort = proxyPort;
}
@Override
public String getProxyUser() {
return proxyUser;
}
@Override
public void setProxyUser(String proxyUser) {
this.proxyUser = proxyUser;
}
@Override
public String getProxyPassword() {
return proxyPassword;
}
@Override
public void setProxyPassword(String proxyPassword) {
this.proxyPassword = proxyPassword;
}
/**
* @return the ID of the Root Group on the remote instance
*/
@ -696,10 +750,6 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
return parent == null ? context : getRootGroup(parent);
}
private boolean isWebApiSecure() {
return targetUri.toString().toLowerCase().startsWith("https");
}
private void refreshFlowContentsFromLocal() {
final ProcessGroup rootGroup = getRootGroup();
setName(rootGroup.getName());
@ -725,6 +775,7 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
final NiFiProperties props = NiFiProperties.getInstance();
this.destinationSecure = props.isSiteToSiteSecure();
this.listeningPort = props.getRemoteInputPort();
this.listeningHttpPort = props.getRemoteInputHttpPort();
refreshContentsTimestamp = System.currentTimeMillis();
} finally {
@ -760,20 +811,14 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
return;
}
final RemoteNiFiUtils utils = new RemoteNiFiUtils(isWebApiSecure() ? sslContext : null);
final String uriVal = apiUri.toString() + SITE_TO_SITE_URI_PATH;
URI uri;
try {
uri = new URI(uriVal);
} catch (final URISyntaxException e) {
throw new CommunicationsException("Invalid URI: " + uriVal);
}
try {
// perform the request
final ClientResponse response = utils.get(uri, getCommunicationsTimeout(TimeUnit.MILLISECONDS));
if (!Response.Status.Family.SUCCESSFUL.equals(response.getStatusInfo().getFamily())) {
final ControllerDTO dto;
try (
final SiteToSiteRestApiClient apiClient = getSiteToSiteRestApiClient();
){
dto = apiClient.getController();
} catch (IOException e) {
writeLock.lock();
try {
for (final Iterator<StandardRemoteGroupPort> iter = inputPorts.values().iterator(); iter.hasNext();) {
@ -793,15 +838,9 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
writeLock.unlock();
}
// consume the entity entirely
response.getEntity(String.class);
throw new CommunicationsException("Unable to communicate with Remote NiFi at URI " + uriVal + ". Got HTTP Error Code "
+ response.getStatus() + ": " + response.getStatusInfo().getReasonPhrase());
throw new CommunicationsException("Unable to communicate with Remote NiFi at URI " + apiUri + " due to: " + e.getMessage());
}
final ControllerEntity entity = response.getEntity(ControllerEntity.class);
final ControllerDTO dto = entity.getController();
writeLock.lock();
try {
if (dto.getInputPorts() != null) {
@ -853,6 +892,7 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
}
this.listeningPort = dto.getRemoteSiteListeningPort();
this.listeningHttpPort = dto.getRemoteSiteHttpListeningPort();
this.destinationSecure = dto.isSiteToSiteSecure();
final ProcessGroupCounts newCounts = new ProcessGroupCounts(inputPortCount, outputPortCount,
@ -867,6 +907,14 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
}
}
private SiteToSiteRestApiClient getSiteToSiteRestApiClient() {
SiteToSiteRestApiClient apiClient = new SiteToSiteRestApiClient(sslContext, new HttpProxy(proxyHost, proxyPort, proxyUser, proxyPassword));
apiClient.setBaseUrl(apiUri.toString());
apiClient.setConnectTimeoutMillis(getCommunicationsTimeout(TimeUnit.MILLISECONDS));
apiClient.setReadTimeoutMillis(getCommunicationsTimeout(TimeUnit.MILLISECONDS));
return apiClient;
}
/**
* Converts a set of ports into a set of remote process group ports.
*
@ -1075,7 +1123,7 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
public boolean isSiteToSiteEnabled() {
readLock.lock();
try {
return this.listeningPort != null;
return (this.listeningPort != null || this.listeningHttpPort != null);
} finally {
readLock.unlock();
}
@ -1090,18 +1138,16 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
@Override
public void run() {
try (
final SiteToSiteRestApiClient apiClient = getSiteToSiteRestApiClient();
){
try {
final RemoteNiFiUtils utils = new RemoteNiFiUtils(isWebApiSecure() ? sslContext : null);
final ClientResponse response = utils.get(new URI(apiUri + SITE_TO_SITE_URI_PATH), getCommunicationsTimeout(TimeUnit.MILLISECONDS));
final ControllerDTO dto = apiClient.getController();
final int statusCode = response.getStatus();
if (statusCode == OK_STATUS_CODE) {
final ControllerEntity entity = response.getEntity(ControllerEntity.class);
final ControllerDTO dto = entity.getController();
if (dto.getRemoteSiteListeningPort() == null) {
authorizationIssue = "Remote instance is not configured to allow Site-to-Site communications at this time.";
if (dto.getRemoteSiteListeningPort() == null && SiteToSiteTransportProtocol.RAW.equals(transportProtocol)) {
authorizationIssue = "Remote instance is not configured to allow RAW Site-to-Site communications at this time.";
} else if (dto.getRemoteSiteHttpListeningPort() == null && SiteToSiteTransportProtocol.HTTP.equals(transportProtocol)) {
authorizationIssue = "Remote instance is not configured to allow HTTP Site-to-Site communications at this time.";
} else {
authorizationIssue = null;
}
@ -1109,6 +1155,7 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
writeLock.lock();
try {
listeningPort = dto.getRemoteSiteListeningPort();
listeningHttpPort = dto.getRemoteSiteHttpListeningPort();
destinationSecure = dto.isSiteToSiteSecure();
} finally {
writeLock.unlock();
@ -1117,7 +1164,12 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
final String remoteInstanceId = dto.getInstanceId();
final boolean isPointingToCluster = flowController.getInstanceId().equals(remoteInstanceId);
pointsToCluster.set(isPointingToCluster);
} else if (statusCode == UNAUTHORIZED_STATUS_CODE) {
} catch (SiteToSiteRestApiClient.HttpGetFailedException e) {
if (e.getResponseCode() == UNAUTHORIZED_STATUS_CODE) {
// TODO: implement registration request
/*
try {
final ClientResponse requestAccountResponse = utils.issueRegistrationRequest(apiUri.toString());
if (Response.Status.Family.SUCCESSFUL.equals(requestAccountResponse.getStatusInfo().getFamily())) {
@ -1132,16 +1184,19 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
logger.error("", e);
}
}
*/
authorizationIssue = e.getDescription();
authorizationIssue = response.getEntity(String.class);
} else if (statusCode == FORBIDDEN_STATUS_CODE) {
authorizationIssue = response.getEntity(String.class);
} else if (e.getResponseCode() == FORBIDDEN_STATUS_CODE) {
authorizationIssue = e.getDescription();
} else {
final String message = response.getEntity(String.class);
logger.warn("{} When communicating with remote instance, got unexpected response code {}:{} with entity: {}",
new Object[]{this, response.getStatus(), response.getStatusInfo().getReasonPhrase(), message});
final String message = e.getDescription();
logger.warn("{} When communicating with remote instance, got unexpected result. {}",
new Object[]{this, e.getMessage()});
authorizationIssue = "Unable to determine Site-to-Site availability.";
}
}
} catch (final Exception e) {
logger.warn(String.format("Unable to connect to %s due to %s", StandardRemoteProcessGroup.this, e));
getEventReporter().reportEvent(Severity.WARNING, "Site to Site", String.format("Unable to connect to %s due to %s",

View File

@ -111,9 +111,11 @@ nifi.components.status.repository.buffer.size=${nifi.components.status.repositor
nifi.components.status.snapshot.frequency=${nifi.components.status.snapshot.frequency}
# Site to Site properties
nifi.remote.input.socket.host=
nifi.remote.input.host=
nifi.remote.input.secure=false
nifi.remote.input.socket.port=
nifi.remote.input.secure=true
nifi.remote.input.http.enabled=true
nifi.remote.input.http.transaction.ttl=30 sec
# web properties #
nifi.web.war.directory=${nifi.web.war.directory}

View File

@ -0,0 +1,243 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT 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.remote;
import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.remote.protocol.FlowFileTransaction;
import org.apache.nifi.remote.protocol.http.HttpFlowFileServerProtocol;
import org.apache.nifi.util.FormatUtils;
import org.apache.nifi.util.NiFiProperties;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.util.Map;
import java.util.Set;
import java.util.UUID;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.ThreadFactory;
import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;
import static org.apache.nifi.util.NiFiProperties.DEFAULT_SITE_TO_SITE_HTTP_TRANSACTION_TTL;
import static org.apache.nifi.util.NiFiProperties.SITE_TO_SITE_HTTP_TRANSACTION_TTL;
public class HttpRemoteSiteListener implements RemoteSiteListener {
private static final Logger logger = LoggerFactory.getLogger(HttpRemoteSiteListener.class);
private final int transactionTtlSec;
private static HttpRemoteSiteListener instance;
private final Map<String, TransactionWrapper> transactions = new ConcurrentHashMap<>();
private final ScheduledExecutorService taskExecutor;
private final int httpListenPort;
private ProcessGroup rootGroup;
private ScheduledFuture<?> transactionMaintenanceTask;
private HttpRemoteSiteListener() {
super();
taskExecutor = Executors.newScheduledThreadPool(1, new ThreadFactory() {
private final ThreadFactory defaultFactory = Executors.defaultThreadFactory();
@Override
public Thread newThread(final Runnable r) {
final Thread thread = defaultFactory.newThread(r);
thread.setName("Http Site-to-Site Transaction Maintenance");
thread.setDaemon(true);
return thread;
}
});
NiFiProperties properties = NiFiProperties.getInstance();
int txTtlSec;
try {
final String snapshotFrequency = properties.getProperty(SITE_TO_SITE_HTTP_TRANSACTION_TTL, DEFAULT_SITE_TO_SITE_HTTP_TRANSACTION_TTL);
txTtlSec = (int) FormatUtils.getTimeDuration(snapshotFrequency, TimeUnit.SECONDS);
} catch (final Exception e) {
txTtlSec = (int) FormatUtils.getTimeDuration(DEFAULT_SITE_TO_SITE_HTTP_TRANSACTION_TTL, TimeUnit.SECONDS);
logger.warn("Failed to parse {} due to {}, use default as {} secs.",
SITE_TO_SITE_HTTP_TRANSACTION_TTL, e.getMessage(), txTtlSec);
}
transactionTtlSec = txTtlSec;
httpListenPort = properties.getRemoteInputHttpPort() != null ? properties.getRemoteInputHttpPort() : 0;
}
public static HttpRemoteSiteListener getInstance() {
if (instance == null) {
synchronized (HttpRemoteSiteListener.class) {
if (instance == null) {
instance = new HttpRemoteSiteListener();
}
}
}
return instance;
}
private class TransactionWrapper {
private final FlowFileTransaction transaction;
private long lastCommunicationAt;
private TransactionWrapper(final FlowFileTransaction transaction) {
this.transaction = transaction;
this.lastCommunicationAt = System.currentTimeMillis();
}
private boolean isExpired() {
long elapsedMillis = System.currentTimeMillis() - lastCommunicationAt;
long elapsedSec = TimeUnit.SECONDS.convert(elapsedMillis, TimeUnit.MILLISECONDS);
return elapsedSec > transactionTtlSec;
}
private void extend() {
lastCommunicationAt = System.currentTimeMillis();
}
}
@Override
public void setRootGroup(ProcessGroup rootGroup) {
this.rootGroup = rootGroup;
}
public void setupServerProtocol(HttpFlowFileServerProtocol serverProtocol) {
serverProtocol.setRootProcessGroup(rootGroup);
}
@Override
public void start() throws IOException {
transactionMaintenanceTask = taskExecutor.scheduleWithFixedDelay(() -> {
int originalSize = transactions.size();
logger.trace("Transaction maintenance task started.");
try {
Set<String> transactionIds = transactions.keySet().stream().collect(Collectors.toSet());
transactionIds.stream().filter(tid -> !isTransactionActive(tid))
.forEach(tid -> {
cancelTransaction(tid);
});
} catch (Exception e) {
// Swallow exception so that this thread can keep working.
logger.error("An exception occurred while maintaining transactions", e);
}
logger.debug("Transaction maintenance task finished. originalSize={}, currentSize={}", originalSize, transactions.size());
}, 0, transactionTtlSec / 2, TimeUnit.SECONDS);
}
public void cancelTransaction(String transactionId) {
TransactionWrapper wrapper = transactions.remove(transactionId);
if (wrapper == null) {
logger.debug("The transaction was not found. transactionId={}", transactionId);
} else {
logger.debug("Cancel a transaction. transactionId={}", transactionId);
FlowFileTransaction t = wrapper.transaction;
if(t != null && t.getSession() != null){
logger.info("Cancel a transaction, rollback its session. transactionId={}", transactionId);
try {
t.getSession().rollback();
} catch (Exception e) {
// Swallow exception so that it can keep expiring other transactions.
logger.error("Failed to rollback. transactionId={}", transactionId, e);
}
}
}
}
@Override
public int getPort() {
return httpListenPort;
}
@Override
public void stop() {
if(transactionMaintenanceTask != null) {
logger.debug("Stopping transactionMaintenanceTask...");
transactionMaintenanceTask.cancel(true);
}
}
public String createTransaction() {
final String transactionId = UUID.randomUUID().toString();
transactions.put(transactionId, new TransactionWrapper(null));
logger.debug("Created a new transaction: {}", transactionId);
return transactionId;
}
public boolean isTransactionActive(final String transactionId) {
TransactionWrapper transaction = transactions.get(transactionId);
if (transaction == null) {
return false;
}
if (transaction.isExpired()) {
return false;
}
return true;
}
public void holdTransaction(final String transactionId, final FlowFileTransaction transaction) throws IllegalStateException {
// We don't check expiration of the transaction here, to support large file transport or slow network.
// The availability of current transaction is already checked when the HTTP request was received at SiteToSiteResource.
TransactionWrapper currentTransaction = transactions.remove(transactionId);
if (currentTransaction == null) {
logger.debug("The transaction was not found, it looks it took longer than transaction TTL.");
} else if (currentTransaction.transaction != null) {
throw new IllegalStateException("Transaction has already been processed. It can only be finalized. transactionId=" + transactionId);
}
if (transaction.getSession() == null) {
throw new IllegalStateException("Passed transaction is not associated any session yet, can not hold. transactionId=" + transactionId);
}
logger.debug("Holding a transaction: {}", transactionId);
// Server has received or sent all data, and transaction TTL count down starts here.
// However, if the client doesn't consume data fast enough, server might expire and rollback the transaction.
transactions.put(transactionId, new TransactionWrapper(transaction));
}
public FlowFileTransaction finalizeTransaction(final String transactionId) throws IllegalStateException {
if (!isTransactionActive(transactionId)){
throw new IllegalStateException("Transaction was not found or not active anymore. transactionId=" + transactionId);
}
TransactionWrapper transaction = transactions.remove(transactionId);
if (transaction == null) {
throw new IllegalStateException("Transaction was not found anymore. It's already finalized or expired. transactionId=" + transactionId);
}
if (transaction.transaction == null) {
throw new IllegalStateException("Transaction has not started yet.");
}
logger.debug("Finalized a transaction: {}", transactionId);
return transaction.transaction;
}
public void extendsTransaction(final String transactionId) throws IllegalStateException {
if (!isTransactionActive(transactionId)){
throw new IllegalStateException("Transaction was not found or not active anymore. transactionId=" + transactionId);
}
TransactionWrapper transaction = transactions.get(transactionId);
if (transaction != null) {
logger.debug("Extending transaction TTL, transactionId={}", transactionId);
transaction.extend();
}
}
public int getTransactionTtlSec() {
return transactionTtlSec;
}
}

View File

@ -16,6 +16,19 @@
*/
package org.apache.nifi.remote;
import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.remote.cluster.NodeInformant;
import org.apache.nifi.remote.exception.HandshakeException;
import org.apache.nifi.remote.io.socket.SocketChannelCommunicationsSession;
import org.apache.nifi.remote.io.socket.ssl.SSLSocketChannel;
import org.apache.nifi.remote.io.socket.ssl.SSLSocketChannelCommunicationsSession;
import org.apache.nifi.remote.protocol.CommunicationsSession;
import org.apache.nifi.remote.protocol.RequestType;
import org.apache.nifi.remote.protocol.ServerProtocol;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import javax.net.ssl.SSLContext;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.EOFException;
@ -30,24 +43,9 @@ import java.net.SocketTimeoutException;
import java.nio.channels.ServerSocketChannel;
import java.nio.channels.SocketChannel;
import java.util.Arrays;
import java.util.HashMap;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference;
import javax.net.ssl.SSLContext;
import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.remote.cluster.NodeInformant;
import org.apache.nifi.remote.exception.HandshakeException;
import org.apache.nifi.remote.io.socket.SocketChannelCommunicationsSession;
import org.apache.nifi.remote.io.socket.ssl.SSLSocketChannel;
import org.apache.nifi.remote.io.socket.ssl.SSLSocketChannelCommunicationsSession;
import org.apache.nifi.remote.protocol.CommunicationsSession;
import org.apache.nifi.remote.protocol.RequestType;
import org.apache.nifi.remote.protocol.ServerProtocol;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class SocketRemoteSiteListener implements RemoteSiteListener {
public static final String DEFAULT_FLOWFILE_PATH = "./";
@ -261,11 +259,11 @@ public class SocketRemoteSiteListener implements RemoteSiteListener {
break;
case RECEIVE_FLOWFILES:
// peer wants to receive FlowFiles, so we will transfer FlowFiles.
protocol.getPort().transferFlowFiles(peer, protocol, new HashMap<String, String>());
protocol.getPort().transferFlowFiles(peer, protocol);
break;
case SEND_FLOWFILES:
// Peer wants to send FlowFiles, so we will receive.
protocol.getPort().receiveFlowFiles(peer, protocol, new HashMap<String, String>());
protocol.getPort().receiveFlowFiles(peer, protocol);
break;
case REQUEST_PEER_LIST:
protocol.sendPeerList(peer);

View File

@ -47,6 +47,7 @@ import org.apache.nifi.remote.exception.PortNotRunningException;
import org.apache.nifi.remote.exception.ProtocolException;
import org.apache.nifi.remote.exception.UnknownPortException;
import org.apache.nifi.remote.protocol.DataPacket;
import org.apache.nifi.remote.protocol.http.HttpProxy;
import org.apache.nifi.remote.util.StandardDataPacket;
import org.apache.nifi.reporting.Severity;
import org.apache.nifi.scheduling.SchedulingStrategy;
@ -137,10 +138,13 @@ public class StandardRemoteGroupPort extends RemoteGroupPort {
.url(remoteGroup.getTargetUri().toString())
.portIdentifier(getIdentifier())
.sslContext(sslContext)
.useCompression(isUseCompression())
.eventReporter(remoteGroup.getEventReporter())
.peerPersistenceFile(getPeerPersistenceFile(getIdentifier()))
.nodePenalizationPeriod(penalizationMillis, TimeUnit.MILLISECONDS)
.timeout(remoteGroup.getCommunicationsTimeout(TimeUnit.MILLISECONDS), TimeUnit.MILLISECONDS)
.transportProtocol(remoteGroup.getTransportProtocol())
.httpProxy(new HttpProxy(remoteGroup.getProxyHost(), remoteGroup.getProxyPort(), remoteGroup.getProxyUser(), remoteGroup.getProxyPassword()))
.build();
clientRef.set(client);
}

View File

@ -50,7 +50,6 @@ import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ArrayBlockingQueue;
import java.util.concurrent.BlockingQueue;
@ -236,7 +235,8 @@ public class StandardRootGroupPort extends AbstractPort implements RootGroupPort
return;
}
session.commit();
// TODO: Comfirm this. Session.commit here is not required since it has been committed inside receiveFlowFiles/transferFlowFiles.
// session.commit();
responseQueue.add(new ProcessingResult(transferCount));
}
@ -451,7 +451,7 @@ public class StandardRootGroupPort extends AbstractPort implements RootGroupPort
}
@Override
public int receiveFlowFiles(final Peer peer, final ServerProtocol serverProtocol, final Map<String, String> requestHeaders)
public int receiveFlowFiles(final Peer peer, final ServerProtocol serverProtocol)
throws NotAuthorizedException, BadRequestException, RequestExpiredException {
if (getConnectableType() != ConnectableType.INPUT_PORT) {
throw new IllegalStateException("Cannot receive FlowFiles because this port is not an Input Port");
@ -505,7 +505,7 @@ public class StandardRootGroupPort extends AbstractPort implements RootGroupPort
}
@Override
public int transferFlowFiles(final Peer peer, final ServerProtocol serverProtocol, final Map<String, String> requestHeaders)
public int transferFlowFiles(final Peer peer, final ServerProtocol serverProtocol)
throws NotAuthorizedException, BadRequestException, RequestExpiredException {
if (getConnectableType() != ConnectableType.OUTPUT_PORT) {
throw new IllegalStateException("Cannot send FlowFiles because this port is not an Output Port");

View File

@ -0,0 +1,559 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT 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.remote.protocol;
import org.apache.nifi.connectable.Connection;
import org.apache.nifi.connectable.Port;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.flowfile.attributes.CoreAttributes;
import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.io.InputStreamCallback;
import org.apache.nifi.remote.Peer;
import org.apache.nifi.remote.PortAuthorizationResult;
import org.apache.nifi.remote.RootGroupPort;
import org.apache.nifi.remote.cluster.NodeInformant;
import org.apache.nifi.remote.codec.FlowFileCodec;
import org.apache.nifi.remote.exception.HandshakeException;
import org.apache.nifi.remote.exception.ProtocolException;
import org.apache.nifi.remote.io.CompressionInputStream;
import org.apache.nifi.remote.io.CompressionOutputStream;
import org.apache.nifi.remote.util.StandardDataPacket;
import org.apache.nifi.util.FormatUtils;
import org.apache.nifi.util.StopWatch;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import java.util.UUID;
import java.util.concurrent.TimeUnit;
import java.util.zip.CRC32;
import java.util.zip.CheckedInputStream;
import java.util.zip.CheckedOutputStream;
public abstract class AbstractFlowFileServerProtocol implements ServerProtocol {
protected ProcessGroup rootGroup;
protected RootGroupPort port;
protected boolean handshakeCompleted;
protected boolean shutdown = false;
protected FlowFileCodec negotiatedFlowFileCodec = null;
protected HandshakenProperties handshakenProperties;
protected static final long DEFAULT_BATCH_NANOS = TimeUnit.SECONDS.toNanos(5L);
protected final Logger logger = LoggerFactory.getLogger(getClass());
@Override
public void setRootProcessGroup(final ProcessGroup group) {
if (!group.isRootGroup()) {
throw new IllegalArgumentException("Specified group was not a root group.");
}
this.rootGroup = group;
}
@Override
public boolean isHandshakeSuccessful() {
return handshakeCompleted;
}
protected void validateHandshakeRequest(HandshakenProperties confirmed, final Peer peer, final Map<String, String> properties) throws HandshakeException {
Boolean useGzip = null;
for (final Map.Entry<String, String> entry : properties.entrySet()) {
final String propertyName = entry.getKey();
final String value = entry.getValue();
final HandshakeProperty property;
try {
property = HandshakeProperty.valueOf(propertyName);
} catch (final Exception e) {
throw new HandshakeException(ResponseCode.UNKNOWN_PROPERTY_NAME, "Received unknown property: " + propertyName);
}
try {
switch (property) {
case GZIP: {
useGzip = Boolean.parseBoolean(value);
confirmed.setUseGzip(useGzip);
break;
}
case REQUEST_EXPIRATION_MILLIS:
confirmed.setExpirationMillis(Long.parseLong(value));
break;
case BATCH_COUNT:
confirmed.setBatchCount(Integer.parseInt(value));
break;
case BATCH_SIZE:
confirmed.setBatchBytes(Long.parseLong(value));
break;
case BATCH_DURATION:
confirmed.setBatchDurationNanos(TimeUnit.MILLISECONDS.toNanos(Long.parseLong(value)));
break;
case PORT_IDENTIFIER: {
checkPortStatus(peer, value);
}
}
} catch (final NumberFormatException nfe) {
throw new HandshakeException(ResponseCode.ILLEGAL_PROPERTY_VALUE, "Received invalid value for property '" + property + "'; invalid value: " + value);
}
}
if (useGzip == null) {
logger.debug("Responding with ResponseCode MISSING_PROPERTY because GZIP Property missing");
throw new HandshakeException(ResponseCode.MISSING_PROPERTY, "Missing Property " + HandshakeProperty.GZIP.name());
}
}
protected void checkPortStatus(final Peer peer, String portId) throws HandshakeException {
Port receivedPort = rootGroup.getInputPort(portId);
if (receivedPort == null) {
receivedPort = rootGroup.getOutputPort(portId);
}
if (receivedPort == null) {
logger.debug("Responding with ResponseCode UNKNOWN_PORT for identifier {}", portId);
throw new HandshakeException(ResponseCode.UNKNOWN_PORT, "Received unknown port identifier: " + portId);
}
if (!(receivedPort instanceof RootGroupPort)) {
logger.debug("Responding with ResponseCode UNKNOWN_PORT for identifier {}", portId);
throw new HandshakeException(ResponseCode.UNKNOWN_PORT, "Received port identifier " + portId + ", but this Port is not a RootGroupPort");
}
this.port = (RootGroupPort) receivedPort;
final PortAuthorizationResult portAuthResult = this.port.checkUserAuthorization(peer.getCommunicationsSession().getUserDn());
if (!portAuthResult.isAuthorized()) {
logger.debug("Responding with ResponseCode UNAUTHORIZED: ", portAuthResult.getExplanation());
throw new HandshakeException(ResponseCode.UNAUTHORIZED, portAuthResult.getExplanation());
}
if (!receivedPort.isValid()) {
logger.debug("Responding with ResponseCode PORT_NOT_IN_VALID_STATE for {}", receivedPort);
throw new HandshakeException(ResponseCode.PORT_NOT_IN_VALID_STATE, "Port is not valid");
}
if (!receivedPort.isRunning()) {
logger.debug("Responding with ResponseCode PORT_NOT_IN_VALID_STATE for {}", receivedPort);
throw new HandshakeException(ResponseCode.PORT_NOT_IN_VALID_STATE, "Port not running");
}
// PORTS_DESTINATION_FULL was introduced in version 2. If version 1, just ignore this
// we we will simply not service the request but the sender will timeout
if (getVersionNegotiator().getVersion() > 1) {
for (final Connection connection : port.getConnections()) {
if (connection.getFlowFileQueue().isFull()) {
logger.debug("Responding with ResponseCode PORTS_DESTINATION_FULL for {}", port);
throw new HandshakeException(ResponseCode.PORTS_DESTINATION_FULL, "Received port identifier " + portId + ", but its destination is full");
}
}
}
}
@Override
public RootGroupPort getPort() {
return port;
}
@Override
public FlowFileCodec getPreNegotiatedCodec() {
return negotiatedFlowFileCodec;
}
@Override
public final void handshake(final Peer peer) throws IOException, HandshakeException {
if (handshakeCompleted) {
throw new IllegalStateException("Handshake has already been completed");
}
if (shutdown) {
throw new IllegalStateException("Protocol is shutdown");
}
logger.debug("{} Handshaking with {}", this, peer);
this.handshakenProperties = doHandshake(peer);
logger.debug("{} Finished handshake with {}", this, peer);
handshakeCompleted = true;
}
abstract protected HandshakenProperties doHandshake(final Peer peer) throws IOException, HandshakeException;
@Override
public int transferFlowFiles(final Peer peer, final ProcessContext context, final ProcessSession session, final FlowFileCodec codec) throws IOException, ProtocolException {
if (!handshakeCompleted) {
throw new IllegalStateException("Handshake has not been completed");
}
if (shutdown) {
throw new IllegalStateException("Protocol is shutdown");
}
logger.debug("{} Sending FlowFiles to {}", this, peer);
final CommunicationsSession commsSession = peer.getCommunicationsSession();
String remoteDn = commsSession.getUserDn();
if (remoteDn == null) {
remoteDn = "none";
}
FlowFile flowFile = session.get();
if (flowFile == null) {
// we have no data to send. Notify the peer.
logger.debug("{} No data to send to {}", this, peer);
writeTransactionResponse(true, ResponseCode.NO_MORE_DATA, commsSession);
return 0;
}
// we have data to send.
logger.debug("{} Data is available to send to {}", this, peer);
writeTransactionResponse(true, ResponseCode.MORE_DATA, commsSession);
final StopWatch stopWatch = new StopWatch(true);
long bytesSent = 0L;
final Set<FlowFile> flowFilesSent = new HashSet<>();
final CRC32 crc = new CRC32();
// send data until we reach some batch size
boolean continueTransaction = true;
final long startNanos = System.nanoTime();
String calculatedCRC = "";
OutputStream os = new DataOutputStream(commsSession.getOutput().getOutputStream());
while (continueTransaction) {
final boolean useGzip = handshakenProperties.isUseGzip();
final OutputStream flowFileOutputStream = useGzip ? new CompressionOutputStream(os) : os;
logger.debug("{} Sending {} to {}", new Object[]{this, flowFile, peer});
final CheckedOutputStream checkedOutputStream = new CheckedOutputStream(flowFileOutputStream, crc);
final StopWatch transferWatch = new StopWatch(true);
final FlowFile toSend = flowFile;
session.read(flowFile, new InputStreamCallback() {
@Override
public void process(final InputStream in) throws IOException {
final DataPacket dataPacket = new StandardDataPacket(toSend.getAttributes(), in, toSend.getSize());
codec.encode(dataPacket, checkedOutputStream);
}
});
final long transmissionMillis = transferWatch.getElapsed(TimeUnit.MILLISECONDS);
// need to close the CompressionOutputStream in order to force it write out any remaining bytes.
// Otherwise, do NOT close it because we don't want to close the underlying stream
// (CompressionOutputStream will not close the underlying stream when it's closed)
if (useGzip) {
checkedOutputStream.close();
}
flowFilesSent.add(flowFile);
bytesSent += flowFile.getSize();
String transitUriPrefix = handshakenProperties.getTransitUriPrefix();
final String transitUri = (transitUriPrefix == null) ? peer.getUrl() : transitUriPrefix + flowFile.getAttribute(CoreAttributes.UUID.key());
session.getProvenanceReporter().send(flowFile, transitUri, "Remote Host=" + peer.getHost() + ", Remote DN=" + remoteDn, transmissionMillis, false);
session.remove(flowFile);
// determine if we should check for more data on queue.
final long sendingNanos = System.nanoTime() - startNanos;
boolean poll = true;
double batchDurationNanos = handshakenProperties.getBatchDurationNanos();
if (sendingNanos >= batchDurationNanos && batchDurationNanos > 0L) {
poll = false;
}
double batchBytes = handshakenProperties.getBatchBytes();
if (bytesSent >= batchBytes && batchBytes > 0L) {
poll = false;
}
double batchCount = handshakenProperties.getBatchCount();
if (flowFilesSent.size() >= batchCount && batchCount > 0) {
poll = false;
}
if (batchDurationNanos == 0 && batchBytes == 0 && batchCount == 0) {
poll = (sendingNanos < DEFAULT_BATCH_NANOS);
}
if (poll) {
// we've not elapsed the requested sending duration, so get more data.
flowFile = session.get();
} else {
flowFile = null;
}
continueTransaction = (flowFile != null);
if (continueTransaction) {
logger.debug("{} Sending ContinueTransaction indicator to {}", this, peer);
writeTransactionResponse(true, ResponseCode.CONTINUE_TRANSACTION, commsSession);
} else {
logger.debug("{} Sending FinishTransaction indicator to {}", this, peer);
writeTransactionResponse(true, ResponseCode.FINISH_TRANSACTION, commsSession);
calculatedCRC = String.valueOf(checkedOutputStream.getChecksum().getValue());
}
}
FlowFileTransaction transaction = new FlowFileTransaction(session, context, stopWatch, bytesSent, flowFilesSent, calculatedCRC);
return commitTransferTransaction(peer, transaction);
}
protected int commitTransferTransaction(Peer peer, FlowFileTransaction transaction) throws IOException {
ProcessSession session = transaction.getSession();
Set<FlowFile> flowFilesSent = transaction.getFlowFilesSent();
// we've sent a FINISH_TRANSACTION. Now we'll wait for the peer to send a 'Confirm Transaction' response
CommunicationsSession commsSession = peer.getCommunicationsSession();
final Response transactionConfirmationResponse = readTransactionResponse(true, commsSession);
if (transactionConfirmationResponse.getCode() == ResponseCode.CONFIRM_TRANSACTION) {
// Confirm Checksum and echo back the confirmation.
logger.debug("{} Received {} from {}", this, transactionConfirmationResponse, peer);
final String receivedCRC = transactionConfirmationResponse.getMessage();
if (getVersionNegotiator().getVersion() > 3) {
String calculatedCRC = transaction.getCalculatedCRC();
if (!receivedCRC.equals(calculatedCRC)) {
writeTransactionResponse(true, ResponseCode.BAD_CHECKSUM, commsSession);
session.rollback();
throw new IOException(this + " Sent data to peer " + peer + " but calculated CRC32 Checksum as "
+ calculatedCRC + " while peer calculated CRC32 Checksum as " + receivedCRC
+ "; canceling transaction and rolling back session");
}
}
writeTransactionResponse(true, ResponseCode.CONFIRM_TRANSACTION, commsSession, "");
} else {
throw new ProtocolException("Expected to receive 'Confirm Transaction' response from peer " + peer + " but received " + transactionConfirmationResponse);
}
final String flowFileDescription = flowFilesSent.size() < 20 ? flowFilesSent.toString() : flowFilesSent.size() + " FlowFiles";
final Response transactionResponse;
try {
transactionResponse = readTransactionResponse(true, commsSession);
} catch (final IOException e) {
logger.error("{} Failed to receive a response from {} when expecting a TransactionFinished Indicator."
+ " It is unknown whether or not the peer successfully received/processed the data."
+ " Therefore, {} will be rolled back, possibly resulting in data duplication of {}",
this, peer, session, flowFileDescription);
session.rollback();
throw e;
}
logger.debug("{} received {} from {}", new Object[]{this, transactionResponse, peer});
if (transactionResponse.getCode() == ResponseCode.TRANSACTION_FINISHED_BUT_DESTINATION_FULL) {
peer.penalize(port.getIdentifier(), port.getYieldPeriod(TimeUnit.MILLISECONDS));
} else if (transactionResponse.getCode() != ResponseCode.TRANSACTION_FINISHED) {
throw new ProtocolException("After sending data, expected TRANSACTION_FINISHED response but got " + transactionResponse);
}
session.commit();
StopWatch stopWatch = transaction.getStopWatch();
long bytesSent = transaction.getBytesSent();
stopWatch.stop();
final String uploadDataRate = stopWatch.calculateDataRate(bytesSent);
final long uploadMillis = stopWatch.getDuration(TimeUnit.MILLISECONDS);
final String dataSize = FormatUtils.formatDataSize(bytesSent);
logger.info("{} Successfully sent {} ({}) to {} in {} milliseconds at a rate of {}", new Object[]{
this, flowFileDescription, dataSize, peer, uploadMillis, uploadDataRate});
return flowFilesSent.size();
}
protected Response readTransactionResponse(boolean isTransfer, CommunicationsSession commsSession) throws IOException {
final DataInputStream dis = new DataInputStream(commsSession.getInput().getInputStream());
return Response.read(dis);
}
protected final void writeTransactionResponse(boolean isTransfer, ResponseCode response, CommunicationsSession commsSession) throws IOException {
writeTransactionResponse(isTransfer, response, commsSession, null);
}
protected void writeTransactionResponse(boolean isTransfer, ResponseCode response, CommunicationsSession commsSession, String explanation) throws IOException {
final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream());
if(explanation == null){
response.writeResponse(dos);
} else {
response.writeResponse(dos, explanation);
}
}
@Override
public int receiveFlowFiles(final Peer peer, final ProcessContext context, final ProcessSession session, final FlowFileCodec codec) throws IOException, ProtocolException {
if (!handshakeCompleted) {
throw new IllegalStateException("Handshake has not been completed");
}
if (shutdown) {
throw new IllegalStateException("Protocol is shutdown");
}
logger.debug("{} receiving FlowFiles from {}", this, peer);
final CommunicationsSession commsSession = peer.getCommunicationsSession();
final DataInputStream dis = new DataInputStream(commsSession.getInput().getInputStream());
String remoteDn = commsSession.getUserDn();
if (remoteDn == null) {
remoteDn = "none";
}
final StopWatch stopWatch = new StopWatch(true);
final CRC32 crc = new CRC32();
// Peer has data. Otherwise, we would not have been called, because they would not have sent
// a SEND_FLOWFILES request to use. Just decode the bytes into FlowFiles until peer says he's
// finished sending data.
final Set<FlowFile> flowFilesReceived = new HashSet<>();
long bytesReceived = 0L;
boolean continueTransaction = true;
while (continueTransaction) {
final long startNanos = System.nanoTime();
final InputStream flowFileInputStream = handshakenProperties.isUseGzip() ? new CompressionInputStream(dis) : dis;
final CheckedInputStream checkedInputStream = new CheckedInputStream(flowFileInputStream, crc);
final DataPacket dataPacket = codec.decode(checkedInputStream);
if(dataPacket == null){
logger.debug("{} Received null dataPacket indicating the end of transaction from {}", this, peer);
break;
}
FlowFile flowFile = session.create();
flowFile = session.importFrom(dataPacket.getData(), flowFile);
flowFile = session.putAllAttributes(flowFile, dataPacket.getAttributes());
final long transferNanos = System.nanoTime() - startNanos;
final long transferMillis = TimeUnit.MILLISECONDS.convert(transferNanos, TimeUnit.NANOSECONDS);
final String sourceSystemFlowFileUuid = dataPacket.getAttributes().get(CoreAttributes.UUID.key());
flowFile = session.putAttribute(flowFile, CoreAttributes.UUID.key(), UUID.randomUUID().toString());
String transitUriPrefix = handshakenProperties.getTransitUriPrefix();
final String transitUri = (transitUriPrefix == null) ? peer.getUrl() : transitUriPrefix + sourceSystemFlowFileUuid;
session.getProvenanceReporter().receive(flowFile, transitUri, sourceSystemFlowFileUuid == null
? null : "urn:nifi:" + sourceSystemFlowFileUuid, "Remote Host=" + peer.getHost() + ", Remote DN=" + remoteDn, transferMillis);
session.transfer(flowFile, Relationship.ANONYMOUS);
flowFilesReceived.add(flowFile);
bytesReceived += flowFile.getSize();
final Response transactionResponse = readTransactionResponse(false, commsSession);
switch (transactionResponse.getCode()) {
case CONTINUE_TRANSACTION:
logger.debug("{} Received ContinueTransaction indicator from {}", this, peer);
break;
case FINISH_TRANSACTION:
logger.debug("{} Received FinishTransaction indicator from {}", this, peer);
continueTransaction = false;
break;
case CANCEL_TRANSACTION:
logger.info("{} Received CancelTransaction indicator from {} with explanation {}", this, peer, transactionResponse.getMessage());
session.rollback();
return 0;
default:
throw new ProtocolException("Received unexpected response from peer: when expecting Continue Transaction or Finish Transaction, received" + transactionResponse);
}
}
// we received a FINISH_TRANSACTION indicator. Send back a CONFIRM_TRANSACTION message
// to peer so that we can verify that the connection is still open. This is a two-phase commit,
// which helps to prevent the chances of data duplication. Without doing this, we may commit the
// session and then when we send the response back to the peer, the peer may have timed out and may not
// be listening. As a result, it will re-send the data. By doing this two-phase commit, we narrow the
// Critical Section involved in this transaction so that rather than the Critical Section being the
// time window involved in the entire transaction, it is reduced to a simple round-trip conversation.
logger.debug("{} Sending CONFIRM_TRANSACTION Response Code to {}", this, peer);
String calculatedCRC = String.valueOf(crc.getValue());
writeTransactionResponse(false, ResponseCode.CONFIRM_TRANSACTION, commsSession, calculatedCRC);
FlowFileTransaction transaction = new FlowFileTransaction(session, context, stopWatch, bytesReceived, flowFilesReceived, calculatedCRC);
return commitReceiveTransaction(peer, transaction);
}
protected int commitReceiveTransaction(Peer peer, FlowFileTransaction transaction) throws IOException {
CommunicationsSession commsSession = peer.getCommunicationsSession();
ProcessSession session = transaction.getSession();
final Response confirmTransactionResponse = readTransactionResponse(false, commsSession);
logger.debug("{} Received {} from {}", this, confirmTransactionResponse, peer);
switch (confirmTransactionResponse.getCode()) {
case CONFIRM_TRANSACTION:
break;
case BAD_CHECKSUM:
session.rollback();
throw new IOException(this + " Received a BadChecksum response from peer " + peer);
default:
throw new ProtocolException(this + " Received unexpected Response Code from peer " + peer + " : " + confirmTransactionResponse + "; expected 'Confirm Transaction' Response Code");
}
// Commit the session so that we have persisted the data
session.commit();
if (transaction.getContext().getAvailableRelationships().isEmpty()) {
// Confirm that we received the data and the peer can now discard it but that the peer should not
// send any more data for a bit
logger.debug("{} Sending TRANSACTION_FINISHED_BUT_DESTINATION_FULL to {}", this, peer);
writeTransactionResponse(false, ResponseCode.TRANSACTION_FINISHED_BUT_DESTINATION_FULL, commsSession);
} else {
// Confirm that we received the data and the peer can now discard it
logger.debug("{} Sending TRANSACTION_FINISHED to {}", this, peer);
writeTransactionResponse(false, ResponseCode.TRANSACTION_FINISHED, commsSession);
}
Set<FlowFile> flowFilesReceived = transaction.getFlowFilesSent();
long bytesReceived = transaction.getBytesSent();
StopWatch stopWatch = transaction.getStopWatch();
stopWatch.stop();
final String flowFileDescription = flowFilesReceived.size() < 20 ? flowFilesReceived.toString() : flowFilesReceived.size() + " FlowFiles";
final String uploadDataRate = stopWatch.calculateDataRate(bytesReceived);
final long uploadMillis = stopWatch.getDuration(TimeUnit.MILLISECONDS);
final String dataSize = FormatUtils.formatDataSize(bytesReceived);
logger.info("{} Successfully received {} ({}) from {} in {} milliseconds at a rate of {}", new Object[]{
this, flowFileDescription, dataSize, peer, uploadMillis, uploadDataRate});
return flowFilesReceived.size();
}
@Override
public void shutdown(final Peer peer) {
logger.debug("{} Shutting down with {}", this, peer);
shutdown = true;
}
@Override
public boolean isShutdown() {
return shutdown;
}
@Override
public void setNodeInformant(final NodeInformant nodeInformant) {
}
@Override
public long getRequestExpiration() {
return handshakenProperties.getExpirationMillis();
}
@Override
public String toString() {
String commid = handshakenProperties != null ? handshakenProperties.getCommsIdentifier() : null;
return getClass().getSimpleName() + "[CommsID=" + commid + "]";
}
}

View File

@ -0,0 +1,71 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.remote.protocol;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.util.StopWatch;
import java.util.Set;
public class FlowFileTransaction {
private final ProcessSession session;
private final ProcessContext context;
private final StopWatch stopWatch;
private final long bytesSent;
private final Set<FlowFile> flowFilesSent;
private final String calculatedCRC;
public FlowFileTransaction() {
this(null, null, new StopWatch(true), 0, null, null);
}
public FlowFileTransaction(ProcessSession session, ProcessContext context, StopWatch stopWatch, long bytesSent, Set<FlowFile> flowFilesSent, String calculatedCRC) {
this.session = session;
this.context = context;
this.stopWatch = stopWatch;
this.bytesSent = bytesSent;
this.flowFilesSent = flowFilesSent;
this.calculatedCRC = calculatedCRC;
}
public ProcessSession getSession() {
return session;
}
public StopWatch getStopWatch() {
return stopWatch;
}
public long getBytesSent() {
return bytesSent;
}
public Set<FlowFile> getFlowFilesSent() {
return flowFilesSent;
}
public String getCalculatedCRC() {
return calculatedCRC;
}
public ProcessContext getContext() {
return context;
}
}

View File

@ -0,0 +1,96 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.remote.protocol;
import org.apache.nifi.remote.exception.HandshakeException;
public class HandshakenProperties {
private String commsIdentifier;
private String transitUriPrefix = null;
private boolean useGzip;
private long expirationMillis;
private int batchCount = 0;
private long batchBytes = 0L;
private long batchDurationNanos = 0L;
public String getCommsIdentifier() {
return commsIdentifier;
}
public void setCommsIdentifier(String commsIdentifier) {
this.commsIdentifier = commsIdentifier;
}
public String getTransitUriPrefix() {
return transitUriPrefix;
}
public void setTransitUriPrefix(String transitUriPrefix) {
this.transitUriPrefix = transitUriPrefix;
}
public boolean isUseGzip() {
return useGzip;
}
public void setUseGzip(Boolean useGzip) {
this.useGzip = useGzip;
}
public long getExpirationMillis() {
return expirationMillis;
}
public void setExpirationMillis(long expirationMillis) {
this.expirationMillis = expirationMillis;
}
public int getBatchCount() {
return batchCount;
}
public void setBatchCount(int batchCount) throws HandshakeException {
if (batchCount < 0) {
throw new HandshakeException("Cannot request Batch Count less than 1; requested value: " + batchCount);
}
this.batchCount = batchCount;
}
public long getBatchBytes() {
return batchBytes;
}
public void setBatchBytes(long batchBytes) throws HandshakeException {
if (batchBytes < 0) {
throw new HandshakeException("Cannot request Batch Size less than 1; requested value: " + batchBytes);
}
this.batchBytes = batchBytes;
}
public long getBatchDurationNanos() {
return batchDurationNanos;
}
public void setBatchDurationNanos(long batchDurationNanos) throws HandshakeException {
if (batchDurationNanos < 0) {
throw new HandshakeException("Cannot request Batch Duration less than 1; requested value: " + batchDurationNanos);
}
this.batchDurationNanos = batchDurationNanos;
}
}

View File

@ -0,0 +1,30 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.remote.protocol.http;
import org.apache.nifi.remote.Peer;
import org.apache.nifi.remote.protocol.ServerProtocol;
import java.io.IOException;
public interface HttpFlowFileServerProtocol extends ServerProtocol {
int commitTransferTransaction(Peer peer, String clientChecksum) throws IOException, IllegalStateException;
int commitReceiveTransaction(Peer peer) throws IOException, IllegalStateException;
}

View File

@ -0,0 +1,223 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT 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.remote.protocol.http;
import org.apache.nifi.remote.HttpRemoteSiteListener;
import org.apache.nifi.remote.Peer;
import org.apache.nifi.remote.Transaction;
import org.apache.nifi.remote.VersionNegotiator;
import org.apache.nifi.remote.codec.FlowFileCodec;
import org.apache.nifi.remote.codec.StandardFlowFileCodec;
import org.apache.nifi.remote.exception.HandshakeException;
import org.apache.nifi.remote.io.http.HttpServerCommunicationsSession;
import org.apache.nifi.remote.protocol.AbstractFlowFileServerProtocol;
import org.apache.nifi.remote.protocol.CommunicationsSession;
import org.apache.nifi.remote.protocol.FlowFileTransaction;
import org.apache.nifi.remote.protocol.HandshakenProperties;
import org.apache.nifi.remote.protocol.RequestType;
import org.apache.nifi.remote.protocol.Response;
import org.apache.nifi.remote.protocol.ResponseCode;
import org.apache.nifi.stream.io.ByteArrayInputStream;
import org.apache.nifi.stream.io.ByteArrayOutputStream;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.IOException;
public class HttpFlowFileServerProtocolImpl extends AbstractFlowFileServerProtocol implements HttpFlowFileServerProtocol {
public static final String RESOURCE_NAME = "HttpFlowFileProtocol";
private final FlowFileCodec codec = new StandardFlowFileCodec();
private final VersionNegotiator versionNegotiator;
private final HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance();
public HttpFlowFileServerProtocolImpl(VersionNegotiator versionNegotiator) {
super();
this.versionNegotiator = versionNegotiator;
}
@Override
public FlowFileCodec negotiateCodec(final Peer peer) throws IOException {
return codec;
}
@Override
public FlowFileCodec getPreNegotiatedCodec() {
return codec;
}
@Override
protected HandshakenProperties doHandshake(Peer peer) throws IOException, HandshakeException {
HandshakenProperties confirmed = new HandshakenProperties();
HttpServerCommunicationsSession commsSession = (HttpServerCommunicationsSession) peer.getCommunicationsSession();
confirmed.setCommsIdentifier(commsSession.getTransactionId());
validateHandshakeRequest(confirmed, peer, commsSession.getHandshakeParams());
logger.debug("{} Done handshake, confirmed={}", this, confirmed);
return confirmed;
}
@Override
protected void writeTransactionResponse(boolean isTransfer, ResponseCode response, CommunicationsSession commsSession, String explanation) throws IOException {
HttpServerCommunicationsSession commSession = (HttpServerCommunicationsSession) commsSession;
commSession.setResponseCode(response);
if(isTransfer){
switch (response) {
case NO_MORE_DATA:
logger.debug("{} There's no data to send.", this);
break;
case CONTINUE_TRANSACTION:
logger.debug("{} Continue transaction... expecting more flow files.", this);
commSession.setStatus(Transaction.TransactionState.DATA_EXCHANGED);
break;
case BAD_CHECKSUM:
logger.debug("{} Received BAD_CHECKSUM.", this);
commSession.setStatus(Transaction.TransactionState.ERROR);
break;
case CONFIRM_TRANSACTION:
logger.debug("{} Transaction is confirmed.", this);
commSession.setStatus(Transaction.TransactionState.TRANSACTION_CONFIRMED);
break;
case FINISH_TRANSACTION:
logger.debug("{} transaction is completed.", this);
commSession.setStatus(Transaction.TransactionState.TRANSACTION_COMPLETED);
break;
}
} else {
switch (response) {
case CONFIRM_TRANSACTION:
logger.debug("{} Confirming transaction. checksum={}", this, explanation);
commSession.setChecksum(explanation);
commSession.setStatus(Transaction.TransactionState.DATA_EXCHANGED);
break;
case TRANSACTION_FINISHED:
case TRANSACTION_FINISHED_BUT_DESTINATION_FULL:
logger.debug("{} Transaction is completed. responseCode={}", this, response);
commSession.setStatus(Transaction.TransactionState.TRANSACTION_COMPLETED);
break;
}
}
}
@Override
protected Response readTransactionResponse(boolean isTransfer, CommunicationsSession commsSession) throws IOException {
// Returns Response based on current status.
HttpServerCommunicationsSession commSession = (HttpServerCommunicationsSession) commsSession;
ByteArrayOutputStream bos = new ByteArrayOutputStream();
Transaction.TransactionState currentStatus = commSession.getStatus();
if(isTransfer){
switch (currentStatus){
case DATA_EXCHANGED:
String clientChecksum = commSession.getChecksum();
logger.debug("readTransactionResponse. clientChecksum={}", clientChecksum);
ResponseCode.CONFIRM_TRANSACTION.writeResponse(new DataOutputStream(bos), clientChecksum);
break;
case TRANSACTION_CONFIRMED:
logger.debug("readTransactionResponse. finishing.");
ResponseCode.TRANSACTION_FINISHED.writeResponse(new DataOutputStream(bos));
break;
}
} else {
switch (currentStatus){
case TRANSACTION_STARTED:
logger.debug("readTransactionResponse. returning CONTINUE_TRANSACTION.");
// We don't know if there's more data to receive, so just continue it.
ResponseCode.CONTINUE_TRANSACTION.writeResponse(new DataOutputStream(bos));
break;
case TRANSACTION_CONFIRMED:
// Checksum was successfully validated at client side, or BAD_CHECKSUM is returned.
ResponseCode responseCode = commSession.getResponseCode();
logger.debug("readTransactionResponse. responseCode={}", responseCode);
if(responseCode.containsMessage()){
responseCode.writeResponse(new DataOutputStream(bos), "");
} else {
responseCode.writeResponse(new DataOutputStream(bos));
}
break;
}
}
ByteArrayInputStream bis = new ByteArrayInputStream(bos.toByteArray());
return Response.read(new DataInputStream(bis));
}
private int holdTransaction(Peer peer, FlowFileTransaction transaction) {
// We don't commit the session here yet,
// to avoid losing sent flow files in case some issue happens at client side while it is processing,
// hold the transaction until we confirm additional request from client.
HttpServerCommunicationsSession commSession = (HttpServerCommunicationsSession) peer.getCommunicationsSession();
String transactionId = commSession.getTransactionId();
logger.debug("{} Holding transaction. transactionId={}", this, transactionId);
transactionManager.holdTransaction(transactionId, transaction);
return transaction.getFlowFilesSent().size();
}
@Override
protected int commitTransferTransaction(Peer peer, FlowFileTransaction transaction) throws IOException {
return holdTransaction(peer, transaction);
}
public int commitTransferTransaction(Peer peer, String clientChecksum) throws IOException, IllegalStateException {
logger.debug("{} Committing the transfer transaction. peer={} clientChecksum={}", this, peer, clientChecksum);
HttpServerCommunicationsSession commSession = (HttpServerCommunicationsSession) peer.getCommunicationsSession();
String transactionId = commSession.getTransactionId();
FlowFileTransaction transaction = transactionManager.finalizeTransaction(transactionId);
commSession.setChecksum(clientChecksum);
commSession.setStatus(Transaction.TransactionState.DATA_EXCHANGED);
return super.commitTransferTransaction(peer, transaction);
}
@Override
protected int commitReceiveTransaction(Peer peer, FlowFileTransaction transaction) throws IOException {
return holdTransaction(peer, transaction);
}
public int commitReceiveTransaction(Peer peer) throws IOException, IllegalStateException {
logger.debug("{} Committing the receive transaction. peer={}", this, peer);
HttpServerCommunicationsSession commSession = (HttpServerCommunicationsSession) peer.getCommunicationsSession();
String transactionId = commSession.getTransactionId();
FlowFileTransaction transaction = transactionManager.finalizeTransaction(transactionId);
commSession.setStatus(Transaction.TransactionState.TRANSACTION_CONFIRMED);
return super.commitReceiveTransaction(peer, transaction);
}
@Override
public RequestType getRequestType(final Peer peer) throws IOException {
return null;
}
@Override
public VersionNegotiator getVersionNegotiator() {
return versionNegotiator;
}
@Override
public void sendPeerList(final Peer peer) throws IOException {
}
@Override
public String getResourceName() {
return RESOURCE_NAME;
}
}

View File

@ -34,7 +34,9 @@ import org.apache.nifi.remote.cluster.NodeInformation;
import org.apache.nifi.remote.codec.FlowFileCodec;
import org.apache.nifi.remote.exception.HandshakeException;
import org.apache.nifi.remote.protocol.CommunicationsSession;
import org.apache.nifi.remote.protocol.HandshakeProperty;
import org.apache.nifi.remote.protocol.RequestType;
import org.apache.nifi.remote.protocol.ResponseCode;
import org.apache.nifi.remote.protocol.ServerProtocol;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

View File

@ -16,106 +16,50 @@
*/
package org.apache.nifi.remote.protocol.socket;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.net.InetAddress;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import java.util.UUID;
import java.util.concurrent.TimeUnit;
import java.util.zip.CRC32;
import java.util.zip.CheckedInputStream;
import java.util.zip.CheckedOutputStream;
import org.apache.nifi.connectable.Connection;
import org.apache.nifi.connectable.Port;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.flowfile.attributes.CoreAttributes;
import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.io.InputStreamCallback;
import org.apache.nifi.remote.Peer;
import org.apache.nifi.remote.PortAuthorizationResult;
import org.apache.nifi.remote.RemoteResourceFactory;
import org.apache.nifi.remote.RootGroupPort;
import org.apache.nifi.remote.StandardVersionNegotiator;
import org.apache.nifi.remote.VersionNegotiator;
import org.apache.nifi.remote.cluster.NodeInformant;
import org.apache.nifi.remote.codec.FlowFileCodec;
import org.apache.nifi.remote.exception.HandshakeException;
import org.apache.nifi.remote.exception.ProtocolException;
import org.apache.nifi.remote.io.CompressionInputStream;
import org.apache.nifi.remote.io.CompressionOutputStream;
import org.apache.nifi.remote.protocol.AbstractFlowFileServerProtocol;
import org.apache.nifi.remote.protocol.CommunicationsSession;
import org.apache.nifi.remote.protocol.DataPacket;
import org.apache.nifi.remote.protocol.HandshakenProperties;
import org.apache.nifi.remote.protocol.RequestType;
import org.apache.nifi.remote.protocol.ServerProtocol;
import org.apache.nifi.remote.util.StandardDataPacket;
import org.apache.nifi.util.FormatUtils;
import org.apache.nifi.remote.protocol.ResponseCode;
import org.apache.nifi.util.NiFiProperties;
import org.apache.nifi.util.StopWatch;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class SocketFlowFileServerProtocol implements ServerProtocol {
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.IOException;
import java.net.InetAddress;
import java.util.HashMap;
import java.util.Map;
public class SocketFlowFileServerProtocol extends AbstractFlowFileServerProtocol {
public static final String RESOURCE_NAME = "SocketFlowFileProtocol";
private ProcessGroup rootGroup;
private String commsIdentifier;
private boolean handshakeCompleted;
private Boolean useGzip;
private long requestExpirationMillis;
private RootGroupPort port;
private boolean shutdown = false;
private FlowFileCodec negotiatedFlowFileCodec = null;
private String transitUriPrefix = null;
private int requestedBatchCount = 0;
private long requestedBatchBytes = 0L;
private long requestedBatchNanos = 0L;
private static final long DEFAULT_BATCH_NANOS = TimeUnit.SECONDS.toNanos(5L);
private final VersionNegotiator versionNegotiator = new StandardVersionNegotiator(5, 4, 3, 2, 1);
private final Logger logger = LoggerFactory.getLogger(SocketFlowFileServerProtocol.class);
@Override
public void setRootProcessGroup(final ProcessGroup group) {
if (!group.isRootGroup()) {
throw new IllegalArgumentException();
}
this.rootGroup = group;
}
protected HandshakenProperties doHandshake(Peer peer) throws IOException, HandshakeException {
@Override
public void handshake(final Peer peer) throws IOException, HandshakeException {
if (handshakeCompleted) {
throw new IllegalStateException("Handshake has already been completed");
}
if (shutdown) {
throw new IllegalStateException("Protocol is shutdown");
}
HandshakenProperties confirmed = new HandshakenProperties();
logger.debug("{} Handshaking with {}", this, peer);
final CommunicationsSession commsSession = peer.getCommunicationsSession();
final DataInputStream dis = new DataInputStream(commsSession.getInput().getInputStream());
final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream());
commsIdentifier = dis.readUTF();
confirmed.setCommsIdentifier(dis.readUTF());
if (versionNegotiator.getVersion() >= 3) {
transitUriPrefix = dis.readUTF();
String transitUriPrefix = dis.readUTF();
if (!transitUriPrefix.endsWith("/")) {
transitUriPrefix = transitUriPrefix + "/";
}
confirmed.setTransitUriPrefix(transitUriPrefix);
}
final Map<String, String> properties = new HashMap<>();
@ -128,109 +72,25 @@ public class SocketFlowFileServerProtocol implements ServerProtocol {
// evaluate the properties received
boolean responseWritten = false;
for (final Map.Entry<String, String> entry : properties.entrySet()) {
final String propertyName = entry.getKey();
final String value = entry.getValue();
final HandshakeProperty property;
try {
property = HandshakeProperty.valueOf(propertyName);
} catch (final Exception e) {
ResponseCode.UNKNOWN_PROPERTY_NAME.writeResponse(dos, "Unknown Property Name: " + propertyName);
throw new HandshakeException("Received unknown property: " + propertyName);
}
try {
switch (property) {
case GZIP: {
useGzip = Boolean.parseBoolean(value);
break;
validateHandshakeRequest(confirmed, peer, properties);
} catch (HandshakeException e) {
ResponseCode handshakeResult = e.getResponseCode();
if(handshakeResult.containsMessage()){
handshakeResult.writeResponse(dos, e.getMessage());
} else {
handshakeResult.writeResponse(dos);
}
case REQUEST_EXPIRATION_MILLIS:
requestExpirationMillis = Long.parseLong(value);
break;
case BATCH_COUNT:
requestedBatchCount = Integer.parseInt(value);
if (requestedBatchCount < 0) {
throw new HandshakeException("Cannot request Batch Count less than 1; requested value: " + value);
}
break;
case BATCH_SIZE:
requestedBatchBytes = Long.parseLong(value);
if (requestedBatchBytes < 0) {
throw new HandshakeException("Cannot request Batch Size less than 1; requested value: " + value);
}
break;
case BATCH_DURATION:
requestedBatchNanos = TimeUnit.MILLISECONDS.toNanos(Long.parseLong(value));
if (requestedBatchNanos < 0) {
throw new HandshakeException("Cannot request Batch Duration less than 1; requested value: " + value);
}
break;
case PORT_IDENTIFIER: {
Port receivedPort = rootGroup.getInputPort(value);
if (receivedPort == null) {
receivedPort = rootGroup.getOutputPort(value);
}
if (receivedPort == null) {
logger.debug("Responding with ResponseCode UNKNOWN_PORT for identifier {}", value);
ResponseCode.UNKNOWN_PORT.writeResponse(dos);
throw new HandshakeException("Received unknown port identifier: " + value);
}
if (!(receivedPort instanceof RootGroupPort)) {
logger.debug("Responding with ResponseCode UNKNOWN_PORT for identifier {}", value);
ResponseCode.UNKNOWN_PORT.writeResponse(dos);
throw new HandshakeException("Received port identifier " + value + ", but this Port is not a RootGroupPort");
}
this.port = (RootGroupPort) receivedPort;
final PortAuthorizationResult portAuthResult = this.port.checkUserAuthorization(peer.getCommunicationsSession().getUserDn());
if (!portAuthResult.isAuthorized()) {
logger.debug("Responding with ResponseCode UNAUTHORIZED: ", portAuthResult.getExplanation());
ResponseCode.UNAUTHORIZED.writeResponse(dos, portAuthResult.getExplanation());
switch (handshakeResult) {
case UNAUTHORIZED:
case PORT_NOT_IN_VALID_STATE:
case PORTS_DESTINATION_FULL:
responseWritten = true;
break;
default:
throw e;
}
if (!receivedPort.isValid()) {
logger.debug("Responding with ResponseCode PORT_NOT_IN_VALID_STATE for {}", receivedPort);
ResponseCode.PORT_NOT_IN_VALID_STATE.writeResponse(dos, "Port is not valid");
responseWritten = true;
break;
}
if (!receivedPort.isRunning()) {
logger.debug("Responding with ResponseCode PORT_NOT_IN_VALID_STATE for {}", receivedPort);
ResponseCode.PORT_NOT_IN_VALID_STATE.writeResponse(dos, "Port not running");
responseWritten = true;
break;
}
// PORTS_DESTINATION_FULL was introduced in version 2. If version 1, just ignore this
// we we will simply not service the request but the sender will timeout
if (getVersionNegotiator().getVersion() > 1) {
for (final Connection connection : port.getConnections()) {
if (connection.getFlowFileQueue().isFull()) {
logger.debug("Responding with ResponseCode PORTS_DESTINATION_FULL for {}", receivedPort);
ResponseCode.PORTS_DESTINATION_FULL.writeResponse(dos);
responseWritten = true;
break;
}
}
}
break;
}
}
} catch (final NumberFormatException nfe) {
throw new HandshakeException("Received invalid value for property '" + property + "'; invalid value: " + value);
}
}
if (useGzip == null) {
logger.debug("Responding with ResponseCode MISSING_PROPERTY because GZIP Property missing");
ResponseCode.MISSING_PROPERTY.writeResponse(dos, HandshakeProperty.GZIP.name());
throw new HandshakeException("Missing Property " + HandshakeProperty.GZIP.name());
}
// send "OK" response
@ -238,18 +98,7 @@ public class SocketFlowFileServerProtocol implements ServerProtocol {
ResponseCode.PROPERTIES_OK.writeResponse(dos);
}
logger.debug("{} Finished handshake with {}", this, peer);
handshakeCompleted = true;
}
@Override
public boolean isHandshakeSuccessful() {
return handshakeCompleted;
}
@Override
public RootGroupPort getPort() {
return port;
return confirmed;
}
@Override
@ -280,290 +129,6 @@ public class SocketFlowFileServerProtocol implements ServerProtocol {
}
}
@Override
public FlowFileCodec getPreNegotiatedCodec() {
return negotiatedFlowFileCodec;
}
@Override
public int transferFlowFiles(final Peer peer, final ProcessContext context, final ProcessSession session, final FlowFileCodec codec) throws IOException, ProtocolException {
if (!handshakeCompleted) {
throw new IllegalStateException("Handshake has not been completed");
}
if (shutdown) {
throw new IllegalStateException("Protocol is shutdown");
}
logger.debug("{} Sending FlowFiles to {}", this, peer);
final CommunicationsSession commsSession = peer.getCommunicationsSession();
final DataInputStream dis = new DataInputStream(commsSession.getInput().getInputStream());
final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream());
String remoteDn = commsSession.getUserDn();
if (remoteDn == null) {
remoteDn = "none";
}
FlowFile flowFile = session.get();
if (flowFile == null) {
// we have no data to send. Notify the peer.
logger.debug("{} No data to send to {}", this, peer);
ResponseCode.NO_MORE_DATA.writeResponse(dos);
return 0;
}
// we have data to send.
logger.debug("{} Data is available to send to {}", this, peer);
ResponseCode.MORE_DATA.writeResponse(dos);
final StopWatch stopWatch = new StopWatch(true);
long bytesSent = 0L;
final Set<FlowFile> flowFilesSent = new HashSet<>();
final CRC32 crc = new CRC32();
// send data until we reach some batch size
boolean continueTransaction = true;
final long startNanos = System.nanoTime();
String calculatedCRC = "";
while (continueTransaction) {
final OutputStream flowFileOutputStream = useGzip ? new CompressionOutputStream(dos) : dos;
logger.debug("{} Sending {} to {}", new Object[]{this, flowFile, peer});
final CheckedOutputStream checkedOutputStream = new CheckedOutputStream(flowFileOutputStream, crc);
final StopWatch transferWatch = new StopWatch(true);
final FlowFile toSend = flowFile;
session.read(flowFile, new InputStreamCallback() {
@Override
public void process(final InputStream in) throws IOException {
final DataPacket dataPacket = new StandardDataPacket(toSend.getAttributes(), in, toSend.getSize());
codec.encode(dataPacket, checkedOutputStream);
}
});
final long transmissionMillis = transferWatch.getElapsed(TimeUnit.MILLISECONDS);
// need to close the CompressionOutputStream in order to force it write out any remaining bytes.
// Otherwise, do NOT close it because we don't want to close the underlying stream
// (CompressionOutputStream will not close the underlying stream when it's closed)
if (useGzip) {
checkedOutputStream.close();
}
flowFilesSent.add(flowFile);
bytesSent += flowFile.getSize();
final String transitUri = (transitUriPrefix == null) ? peer.getUrl() : transitUriPrefix + flowFile.getAttribute(CoreAttributes.UUID.key());
session.getProvenanceReporter().send(flowFile, transitUri, "Remote Host=" + peer.getHost() + ", Remote DN=" + remoteDn, transmissionMillis, false);
session.remove(flowFile);
// determine if we should check for more data on queue.
final long sendingNanos = System.nanoTime() - startNanos;
boolean poll = true;
if (sendingNanos >= requestedBatchNanos && requestedBatchNanos > 0L) {
poll = false;
}
if (bytesSent >= requestedBatchBytes && requestedBatchBytes > 0L) {
poll = false;
}
if (flowFilesSent.size() >= requestedBatchCount && requestedBatchCount > 0) {
poll = false;
}
if (requestedBatchNanos == 0 && requestedBatchBytes == 0 && requestedBatchCount == 0) {
poll = (sendingNanos < DEFAULT_BATCH_NANOS);
}
if (poll) {
// we've not elapsed the requested sending duration, so get more data.
flowFile = session.get();
} else {
flowFile = null;
}
continueTransaction = (flowFile != null);
if (continueTransaction) {
logger.debug("{} Sending ContinueTransaction indicator to {}", this, peer);
ResponseCode.CONTINUE_TRANSACTION.writeResponse(dos);
} else {
logger.debug("{} Sending FinishTransaction indicator to {}", this, peer);
ResponseCode.FINISH_TRANSACTION.writeResponse(dos);
calculatedCRC = String.valueOf(checkedOutputStream.getChecksum().getValue());
}
}
// we've sent a FINISH_TRANSACTION. Now we'll wait for the peer to send a 'Confirm Transaction' response
final Response transactionConfirmationResponse = Response.read(dis);
if (transactionConfirmationResponse.getCode() == ResponseCode.CONFIRM_TRANSACTION) {
// Confirm Checksum and echo back the confirmation.
logger.debug("{} Received {} from {}", this, transactionConfirmationResponse, peer);
final String receivedCRC = transactionConfirmationResponse.getMessage();
if (versionNegotiator.getVersion() > 3) {
if (!receivedCRC.equals(calculatedCRC)) {
ResponseCode.BAD_CHECKSUM.writeResponse(dos);
session.rollback();
throw new IOException(this + " Sent data to peer " + peer + " but calculated CRC32 Checksum as "
+ calculatedCRC + " while peer calculated CRC32 Checksum as " + receivedCRC
+ "; canceling transaction and rolling back session");
}
}
ResponseCode.CONFIRM_TRANSACTION.writeResponse(dos, "");
} else {
throw new ProtocolException("Expected to receive 'Confirm Transaction' response from peer " + peer + " but received " + transactionConfirmationResponse);
}
final String flowFileDescription = flowFilesSent.size() < 20 ? flowFilesSent.toString() : flowFilesSent.size() + " FlowFiles";
final Response transactionResponse;
try {
transactionResponse = Response.read(dis);
} catch (final IOException e) {
logger.error("{} Failed to receive a response from {} when expecting a TransactionFinished Indicator."
+ " It is unknown whether or not the peer successfully received/processed the data."
+ " Therefore, {} will be rolled back, possibly resulting in data duplication of {}",
this, peer, session, flowFileDescription);
session.rollback();
throw e;
}
logger.debug("{} received {} from {}", new Object[]{this, transactionResponse, peer});
if (transactionResponse.getCode() == ResponseCode.TRANSACTION_FINISHED_BUT_DESTINATION_FULL) {
peer.penalize(port.getIdentifier(), port.getYieldPeriod(TimeUnit.MILLISECONDS));
} else if (transactionResponse.getCode() != ResponseCode.TRANSACTION_FINISHED) {
throw new ProtocolException("After sending data, expected TRANSACTION_FINISHED response but got " + transactionResponse);
}
session.commit();
stopWatch.stop();
final String uploadDataRate = stopWatch.calculateDataRate(bytesSent);
final long uploadMillis = stopWatch.getDuration(TimeUnit.MILLISECONDS);
final String dataSize = FormatUtils.formatDataSize(bytesSent);
logger.info("{} Successfully sent {} ({}) to {} in {} milliseconds at a rate of {}", new Object[]{
this, flowFileDescription, dataSize, peer, uploadMillis, uploadDataRate});
return flowFilesSent.size();
}
@Override
public int receiveFlowFiles(final Peer peer, final ProcessContext context, final ProcessSession session, final FlowFileCodec codec) throws IOException, ProtocolException {
if (!handshakeCompleted) {
throw new IllegalStateException("Handshake has not been completed");
}
if (shutdown) {
throw new IllegalStateException("Protocol is shutdown");
}
logger.debug("{} receiving FlowFiles from {}", this, peer);
final CommunicationsSession commsSession = peer.getCommunicationsSession();
final DataInputStream dis = new DataInputStream(commsSession.getInput().getInputStream());
final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream());
String remoteDn = commsSession.getUserDn();
if (remoteDn == null) {
remoteDn = "none";
}
final StopWatch stopWatch = new StopWatch(true);
final CRC32 crc = new CRC32();
// Peer has data. Otherwise, we would not have been called, because they would not have sent
// a SEND_FLOWFILES request to use. Just decode the bytes into FlowFiles until peer says he's
// finished sending data.
final Set<FlowFile> flowFilesReceived = new HashSet<>();
long bytesReceived = 0L;
boolean continueTransaction = true;
String calculatedCRC = "";
while (continueTransaction) {
final long startNanos = System.nanoTime();
final InputStream flowFileInputStream = useGzip ? new CompressionInputStream(dis) : dis;
final CheckedInputStream checkedInputStream = new CheckedInputStream(flowFileInputStream, crc);
final DataPacket dataPacket = codec.decode(checkedInputStream);
FlowFile flowFile = session.create();
flowFile = session.importFrom(dataPacket.getData(), flowFile);
flowFile = session.putAllAttributes(flowFile, dataPacket.getAttributes());
final long transferNanos = System.nanoTime() - startNanos;
final long transferMillis = TimeUnit.MILLISECONDS.convert(transferNanos, TimeUnit.NANOSECONDS);
final String sourceSystemFlowFileUuid = dataPacket.getAttributes().get(CoreAttributes.UUID.key());
flowFile = session.putAttribute(flowFile, CoreAttributes.UUID.key(), UUID.randomUUID().toString());
final String transitUri = (transitUriPrefix == null) ? peer.getUrl() : transitUriPrefix + sourceSystemFlowFileUuid;
session.getProvenanceReporter().receive(flowFile, transitUri, sourceSystemFlowFileUuid == null
? null : "urn:nifi:" + sourceSystemFlowFileUuid, "Remote Host=" + peer.getHost() + ", Remote DN=" + remoteDn, transferMillis);
session.transfer(flowFile, Relationship.ANONYMOUS);
flowFilesReceived.add(flowFile);
bytesReceived += flowFile.getSize();
final Response transactionResponse = Response.read(dis);
switch (transactionResponse.getCode()) {
case CONTINUE_TRANSACTION:
logger.debug("{} Received ContinueTransaction indicator from {}", this, peer);
break;
case FINISH_TRANSACTION:
logger.debug("{} Received FinishTransaction indicator from {}", this, peer);
continueTransaction = false;
calculatedCRC = String.valueOf(checkedInputStream.getChecksum().getValue());
break;
case CANCEL_TRANSACTION:
logger.info("{} Received CancelTransaction indicator from {} with explanation {}", this, peer, transactionResponse.getMessage());
session.rollback();
return 0;
default:
throw new ProtocolException("Received unexpected response from peer: when expecting Continue Transaction or Finish Transaction, received" + transactionResponse);
}
}
// we received a FINISH_TRANSACTION indicator. Send back a CONFIRM_TRANSACTION message
// to peer so that we can verify that the connection is still open. This is a two-phase commit,
// which helps to prevent the chances of data duplication. Without doing this, we may commit the
// session and then when we send the response back to the peer, the peer may have timed out and may not
// be listening. As a result, it will re-send the data. By doing this two-phase commit, we narrow the
// Critical Section involved in this transaction so that rather than the Critical Section being the
// time window involved in the entire transaction, it is reduced to a simple round-trip conversation.
logger.debug("{} Sending CONFIRM_TRANSACTION Response Code to {}", this, peer);
ResponseCode.CONFIRM_TRANSACTION.writeResponse(dos, calculatedCRC);
final Response confirmTransactionResponse = Response.read(dis);
logger.debug("{} Received {} from {}", this, confirmTransactionResponse, peer);
switch (confirmTransactionResponse.getCode()) {
case CONFIRM_TRANSACTION:
break;
case BAD_CHECKSUM:
session.rollback();
throw new IOException(this + " Received a BadChecksum response from peer " + peer);
default:
throw new ProtocolException(this + " Received unexpected Response Code from peer " + peer + " : " + confirmTransactionResponse + "; expected 'Confirm Transaction' Response Code");
}
// Commit the session so that we have persisted the data
session.commit();
if (context.getAvailableRelationships().isEmpty()) {
// Confirm that we received the data and the peer can now discard it but that the peer should not
// send any more data for a bit
logger.debug("{} Sending TRANSACTION_FINISHED_BUT_DESTINATION_FULL to {}", this, peer);
ResponseCode.TRANSACTION_FINISHED_BUT_DESTINATION_FULL.writeResponse(dos);
} else {
// Confirm that we received the data and the peer can now discard it
logger.debug("{} Sending TRANSACTION_FINISHED to {}", this, peer);
ResponseCode.TRANSACTION_FINISHED.writeResponse(dos);
}
stopWatch.stop();
final String flowFileDescription = flowFilesReceived.size() < 20 ? flowFilesReceived.toString() : flowFilesReceived.size() + " FlowFiles";
final String uploadDataRate = stopWatch.calculateDataRate(bytesReceived);
final long uploadMillis = stopWatch.getDuration(TimeUnit.MILLISECONDS);
final String dataSize = FormatUtils.formatDataSize(bytesReceived);
logger.info("{} Successfully received {} ({}) from {} in {} milliseconds at a rate of {}", new Object[]{
this, flowFileDescription, dataSize, peer, uploadMillis, uploadDataRate});
return flowFilesReceived.size();
}
@Override
public RequestType getRequestType(final Peer peer) throws IOException {
@ -581,22 +146,6 @@ public class SocketFlowFileServerProtocol implements ServerProtocol {
return requestType;
}
@Override
public VersionNegotiator getVersionNegotiator() {
return versionNegotiator;
}
@Override
public void shutdown(final Peer peer) {
logger.debug("{} Shutting down with {}", this, peer);
shutdown = true;
}
@Override
public boolean isShutdown() {
return shutdown;
}
@Override
public void sendPeerList(final Peer peer) throws IOException {
if (!handshakeCompleted) {
@ -632,17 +181,9 @@ public class SocketFlowFileServerProtocol implements ServerProtocol {
return RESOURCE_NAME;
}
@Override
public void setNodeInformant(final NodeInformant nodeInformant) {
}
@Override
public long getRequestExpiration() {
return requestExpirationMillis;
}
@Override
public String toString() {
return "SocketFlowFileServerProtocol[CommsID=" + commsIdentifier + "]";
public VersionNegotiator getVersionNegotiator() {
return versionNegotiator;
}
}

View File

@ -0,0 +1,101 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT 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.remote;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.remote.protocol.FlowFileTransaction;
import org.apache.nifi.util.NiFiProperties;
import org.junit.BeforeClass;
import org.junit.Test;
import org.mockito.Mockito;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
public class TestHttpRemoteSiteListener {
@BeforeClass
public static void setup() {
System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, "src/test/resources/nifi.properties");
System.setProperty("org.slf4j.simpleLogger.log.org.apache.nifi.remote", "DEBUG");
}
@Test
public void testNormalTransactionProgress() {
HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance();
String transactionId = transactionManager.createTransaction();
assertTrue("Transaction should be active.", transactionManager.isTransactionActive(transactionId));
ProcessSession processSession = Mockito.mock(ProcessSession.class);
FlowFileTransaction transaction = new FlowFileTransaction(processSession, null, null, 0, null, null);
transactionManager.holdTransaction(transactionId, transaction);
transaction = transactionManager.finalizeTransaction(transactionId);
assertNotNull(transaction);
assertFalse("Transaction should not be active anymore.", transactionManager.isTransactionActive(transactionId));
}
@Test
public void testDuplicatedTransactionId() {
HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance();
String transactionId = transactionManager.createTransaction();
assertTrue("Transaction should be active.", transactionManager.isTransactionActive(transactionId));
ProcessSession processSession = Mockito.mock(ProcessSession.class);
FlowFileTransaction transaction = new FlowFileTransaction(processSession, null, null, 0, null, null);
transactionManager.holdTransaction(transactionId, transaction);
try {
transactionManager.holdTransaction(transactionId, transaction);
fail("The same transaction id can't hold another transaction");
} catch (IllegalStateException e) {
}
}
@Test
public void testNoneExistingTransaction() {
HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance();
String transactionId = "does-not-exist-1";
assertFalse("Transaction should not be active.", transactionManager.isTransactionActive(transactionId));
ProcessSession processSession = Mockito.mock(ProcessSession.class);
FlowFileTransaction transaction = new FlowFileTransaction(processSession, null, null, 0, null, null);
try {
transactionManager.holdTransaction(transactionId, transaction);
} catch (IllegalStateException e) {
fail("Transaction can be held even if the transaction id is not valid anymore," +
" in order to support large file or slow network.");
}
transactionId = "does-not-exist-2";
try {
transactionManager.finalizeTransaction(transactionId);
fail("But transaction should not be finalized if it isn't active.");
} catch (IllegalStateException e) {
}
}
}

View File

@ -0,0 +1,589 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT 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.remote.protocol.http;
import org.apache.nifi.connectable.Connection;
import org.apache.nifi.controller.queue.FlowFileQueue;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.io.InputStreamCallback;
import org.apache.nifi.provenance.ProvenanceReporter;
import org.apache.nifi.remote.HttpRemoteSiteListener;
import org.apache.nifi.remote.Peer;
import org.apache.nifi.remote.PeerDescription;
import org.apache.nifi.remote.PortAuthorizationResult;
import org.apache.nifi.remote.RootGroupPort;
import org.apache.nifi.remote.StandardVersionNegotiator;
import org.apache.nifi.remote.codec.FlowFileCodec;
import org.apache.nifi.remote.codec.StandardFlowFileCodec;
import org.apache.nifi.remote.exception.HandshakeException;
import org.apache.nifi.remote.io.http.HttpInput;
import org.apache.nifi.remote.io.http.HttpServerCommunicationsSession;
import org.apache.nifi.remote.protocol.DataPacket;
import org.apache.nifi.remote.protocol.ResponseCode;
import org.apache.nifi.remote.protocol.HandshakeProperty;
import org.apache.nifi.remote.util.StandardDataPacket;
import org.apache.nifi.stream.io.ByteArrayInputStream;
import org.apache.nifi.stream.io.ByteArrayOutputStream;
import org.apache.nifi.util.NiFiProperties;
import org.junit.BeforeClass;
import org.junit.Test;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import static org.mockito.Matchers.any;
import static org.mockito.Matchers.eq;
import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.doReturn;
import static org.mockito.Mockito.mock;
public class TestHttpFlowFileServerProtocol {
@BeforeClass
public static void setup() throws Exception {
System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, "src/test/resources/nifi.properties");
System.setProperty("org.slf4j.simpleLogger.log.org.apache.nifi.remote", "DEBUG");
}
private Peer getDefaultPeer() {
return getDefaultPeer(null);
}
private Peer getDefaultPeer(final String transactionId) {
final PeerDescription description = new PeerDescription("peer-host", 8080, false);
final InputStream inputStream = new ByteArrayInputStream(new byte[]{});
final OutputStream outputStream = new ByteArrayOutputStream();
final HttpServerCommunicationsSession commsSession = new HttpServerCommunicationsSession(inputStream, outputStream, transactionId);
commsSession.putHandshakeParam(HandshakeProperty.GZIP, "false");
commsSession.putHandshakeParam(HandshakeProperty.REQUEST_EXPIRATION_MILLIS, "1234");
final String peerUrl = "http://peer-host:8080/";
final String clusterUrl = "cluster-url";
return new Peer(description, commsSession, peerUrl, clusterUrl);
}
private HttpFlowFileServerProtocol getDefaultHttpFlowFileServerProtocol() {
final StandardVersionNegotiator versionNegotiator = new StandardVersionNegotiator(5, 4, 3, 2, 1);
return new HttpFlowFileServerProtocolImpl(versionNegotiator);
}
@Test
public void testIllegalHandshakeProperty() throws Exception {
final HttpFlowFileServerProtocol serverProtocol = getDefaultHttpFlowFileServerProtocol();
final Peer peer = getDefaultPeer();
((HttpServerCommunicationsSession)peer.getCommunicationsSession()).getHandshakeParams().clear();
try {
serverProtocol.handshake(peer);
fail();
} catch (HandshakeException e) {
assertEquals(ResponseCode.MISSING_PROPERTY, e.getResponseCode());
}
assertFalse(serverProtocol.isHandshakeSuccessful());
}
@Test
public void testUnknownPort() throws Exception {
final HttpFlowFileServerProtocol serverProtocol = getDefaultHttpFlowFileServerProtocol();
final Peer peer = getDefaultPeer();
((HttpServerCommunicationsSession)peer.getCommunicationsSession())
.putHandshakeParam(HandshakeProperty.PORT_IDENTIFIER, "port-identifier");
final ProcessGroup processGroup = mock(ProcessGroup.class);
doReturn(true).when(processGroup).isRootGroup();
serverProtocol.setRootProcessGroup(processGroup);
try {
serverProtocol.handshake(peer);
fail();
} catch (HandshakeException e) {
assertEquals(ResponseCode.UNKNOWN_PORT, e.getResponseCode());
}
assertFalse(serverProtocol.isHandshakeSuccessful());
}
@Test
public void testUnauthorized() throws Exception {
final HttpFlowFileServerProtocol serverProtocol = getDefaultHttpFlowFileServerProtocol();
final Peer peer = getDefaultPeer();
((HttpServerCommunicationsSession)peer.getCommunicationsSession())
.putHandshakeParam(HandshakeProperty.PORT_IDENTIFIER, "port-identifier");
final ProcessGroup processGroup = mock(ProcessGroup.class);
final RootGroupPort port = mock(RootGroupPort.class);
final PortAuthorizationResult authResult = mock(PortAuthorizationResult.class);
doReturn(true).when(processGroup).isRootGroup();
doReturn(port).when(processGroup).getOutputPort("port-identifier");
doReturn(authResult).when(port).checkUserAuthorization(any(String.class));
serverProtocol.setRootProcessGroup(processGroup);
try {
serverProtocol.handshake(peer);
fail();
} catch (HandshakeException e) {
assertEquals(ResponseCode.UNAUTHORIZED, e.getResponseCode());
}
assertFalse(serverProtocol.isHandshakeSuccessful());
}
@Test
public void testPortNotInValidState() throws Exception {
final HttpFlowFileServerProtocol serverProtocol = getDefaultHttpFlowFileServerProtocol();
final Peer peer = getDefaultPeer();
((HttpServerCommunicationsSession)peer.getCommunicationsSession())
.putHandshakeParam(HandshakeProperty.PORT_IDENTIFIER, "port-identifier");
final ProcessGroup processGroup = mock(ProcessGroup.class);
final RootGroupPort port = mock(RootGroupPort.class);
final PortAuthorizationResult authResult = mock(PortAuthorizationResult.class);
doReturn(true).when(processGroup).isRootGroup();
doReturn(port).when(processGroup).getOutputPort("port-identifier");
doReturn(authResult).when(port).checkUserAuthorization(any(String.class));
doReturn(true).when(authResult).isAuthorized();
serverProtocol.setRootProcessGroup(processGroup);
try {
serverProtocol.handshake(peer);
fail();
} catch (HandshakeException e) {
assertEquals(ResponseCode.PORT_NOT_IN_VALID_STATE, e.getResponseCode());
}
assertFalse(serverProtocol.isHandshakeSuccessful());
}
@Test
public void testPortDestinationFull() throws Exception {
final HttpFlowFileServerProtocol serverProtocol = getDefaultHttpFlowFileServerProtocol();
final Peer peer = getDefaultPeer();
((HttpServerCommunicationsSession)peer.getCommunicationsSession())
.putHandshakeParam(HandshakeProperty.PORT_IDENTIFIER, "port-identifier");
final ProcessGroup processGroup = mock(ProcessGroup.class);
final RootGroupPort port = mock(RootGroupPort.class);
final PortAuthorizationResult authResult = mock(PortAuthorizationResult.class);
doReturn(true).when(processGroup).isRootGroup();
doReturn(port).when(processGroup).getOutputPort("port-identifier");
doReturn(authResult).when(port).checkUserAuthorization(any(String.class));
doReturn(true).when(authResult).isAuthorized();
doReturn(true).when(port).isValid();
doReturn(true).when(port).isRunning();
Set<Connection> connections = new HashSet<>();
final Connection connection = mock(Connection.class);
connections.add(connection);
doReturn(connections).when(port).getConnections();
final FlowFileQueue flowFileQueue = mock(FlowFileQueue.class);
doReturn(flowFileQueue).when(connection).getFlowFileQueue();
doReturn(true).when(flowFileQueue).isFull();
serverProtocol.setRootProcessGroup(processGroup);
try {
serverProtocol.handshake(peer);
fail();
} catch (HandshakeException e) {
assertEquals(ResponseCode.PORTS_DESTINATION_FULL, e.getResponseCode());
}
assertFalse(serverProtocol.isHandshakeSuccessful());
}
@Test
public void testShutdown() throws Exception {
final HttpFlowFileServerProtocol serverProtocol = getDefaultHttpFlowFileServerProtocol();
final Peer peer = getDefaultPeer();
serverProtocol.handshake(peer);
assertTrue(serverProtocol.isHandshakeSuccessful());
final FlowFileCodec negotiatedCoded = serverProtocol.negotiateCodec(peer);
assertTrue(negotiatedCoded instanceof StandardFlowFileCodec);
assertEquals(negotiatedCoded, serverProtocol.getPreNegotiatedCodec());
assertEquals(1234, serverProtocol.getRequestExpiration());
serverProtocol.shutdown(peer);
final ProcessContext context = null;
final ProcessSession processSession = null;
try {
serverProtocol.transferFlowFiles(peer, context, processSession, negotiatedCoded);
fail("transferFlowFiles should fail since it's already shutdown.");
} catch (IllegalStateException e) {
}
try {
serverProtocol.receiveFlowFiles(peer, context, processSession, negotiatedCoded);
fail("receiveFlowFiles should fail since it's already shutdown.");
} catch (IllegalStateException e) {
}
}
@Test
public void testTransferZeroFile() throws Exception {
final HttpFlowFileServerProtocol serverProtocol = getDefaultHttpFlowFileServerProtocol();
final Peer peer = getDefaultPeer();
serverProtocol.handshake(peer);
assertTrue(serverProtocol.isHandshakeSuccessful());
final FlowFileCodec negotiatedCoded = serverProtocol.negotiateCodec(peer);
final ProcessContext context = null;
final ProcessSession processSession = mock(ProcessSession.class);
// Execute test using mock
final int flowFileSent = serverProtocol.transferFlowFiles(peer, context, processSession, negotiatedCoded);
assertEquals(0, flowFileSent);
}
@Test
public void testTransferOneFile() throws Exception {
final HttpFlowFileServerProtocol serverProtocol = getDefaultHttpFlowFileServerProtocol();
final String transactionId = "testTransferOneFile";
final Peer peer = transferOneFile(serverProtocol, transactionId);
// Commit transaction
final int flowFileSent = serverProtocol.commitTransferTransaction(peer, "2077607535");
assertEquals(1, flowFileSent);
}
@Test
public void testTransferOneFileBadChecksum() throws Exception {
final HttpFlowFileServerProtocol serverProtocol = getDefaultHttpFlowFileServerProtocol();
final String transactionId = "testTransferOneFileBadChecksum";
final Peer peer = transferOneFile(serverProtocol, transactionId);
// Commit transaction
try {
serverProtocol.commitTransferTransaction(peer, "client-sent-wrong-checksum");
fail();
} catch (IOException e) {
assertTrue(e.getMessage().contains("CRC32 Checksum"));
}
}
private Peer transferOneFile(HttpFlowFileServerProtocol serverProtocol, String transactionId) throws IOException {
final HttpRemoteSiteListener remoteSiteListener = HttpRemoteSiteListener.getInstance();
final Peer peer = getDefaultPeer(transactionId);
final HttpServerCommunicationsSession commsSession = (HttpServerCommunicationsSession) peer.getCommunicationsSession();
commsSession.putHandshakeParam(HandshakeProperty.BATCH_COUNT, "1");
commsSession.setUserDn("unit-test");
serverProtocol.handshake(peer);
assertTrue(serverProtocol.isHandshakeSuccessful());
final FlowFileCodec negotiatedCoded = serverProtocol.negotiateCodec(peer);
final ProcessContext context = mock(ProcessContext.class);
final ProcessSession processSession = mock(ProcessSession.class);
final ProvenanceReporter provenanceReporter = mock(ProvenanceReporter.class);
final FlowFile flowFile = mock(FlowFile.class);
doReturn(flowFile).when(processSession).get();
doReturn(provenanceReporter).when(processSession).getProvenanceReporter();
doAnswer(invocation -> {
String peerUrl = (String)invocation.getArguments()[1];
String detail = (String)invocation.getArguments()[2];
assertEquals("http://peer-host:8080/", peerUrl);
assertEquals("Remote Host=peer-host, Remote DN=unit-test", detail);
return null;
}).when(provenanceReporter).send(eq(flowFile), any(String.class), any(String.class), any(Long.class), any(Boolean.class));
doAnswer(invocation -> {
InputStreamCallback callback = (InputStreamCallback)invocation.getArguments()[1];
callback.process(new java.io.ByteArrayInputStream("Server content".getBytes()));
return null;
}).when(processSession).read(any(FlowFile.class), any(InputStreamCallback.class));
// Execute test using mock
int flowFileSent = serverProtocol.transferFlowFiles(peer, context, processSession, negotiatedCoded);
assertEquals(1, flowFileSent);
assertTrue(remoteSiteListener.isTransactionActive(transactionId));
return peer;
}
@Test
public void testTransferTwoFiles() throws Exception {
final HttpRemoteSiteListener remoteSiteListener = HttpRemoteSiteListener.getInstance();
final HttpFlowFileServerProtocol serverProtocol = getDefaultHttpFlowFileServerProtocol();
final String transactionId = "testTransferTwoFiles";
final Peer peer = getDefaultPeer(transactionId);
final HttpServerCommunicationsSession commsSession = (HttpServerCommunicationsSession) peer.getCommunicationsSession();
commsSession.putHandshakeParam(HandshakeProperty.BATCH_COUNT, "2");
commsSession.setUserDn("unit-test");
serverProtocol.handshake(peer);
assertTrue(serverProtocol.isHandshakeSuccessful());
final FlowFileCodec negotiatedCoded = serverProtocol.negotiateCodec(peer);
final ProcessContext context = mock(ProcessContext.class);
final ProcessSession processSession = mock(ProcessSession.class);
final ProvenanceReporter provenanceReporter = mock(ProvenanceReporter.class);
final FlowFile flowFile1 = mock(FlowFile.class);
final FlowFile flowFile2 = mock(FlowFile.class);
doReturn(flowFile1)
.doReturn(flowFile2)
.when(processSession).get();
doReturn(provenanceReporter).when(processSession).getProvenanceReporter();
doAnswer(invocation -> {
String peerUrl = (String)invocation.getArguments()[1];
String detail = (String)invocation.getArguments()[2];
assertEquals("http://peer-host:8080/", peerUrl);
assertEquals("Remote Host=peer-host, Remote DN=unit-test", detail);
return null;
}).when(provenanceReporter).send(eq(flowFile1), any(String.class), any(String.class), any(Long.class), any(Boolean.class));
doReturn(provenanceReporter).when(processSession).getProvenanceReporter();
doAnswer(invocation -> {
String peerUrl = (String)invocation.getArguments()[1];
String detail = (String)invocation.getArguments()[2];
assertEquals("http://peer-host:8080/", peerUrl);
assertEquals("Remote Host=peer-host, Remote DN=unit-test", detail);
return null;
}).when(provenanceReporter).send(eq(flowFile2), any(String.class), any(String.class), any(Long.class), any(Boolean.class));
doAnswer(invocation -> {
InputStreamCallback callback = (InputStreamCallback)invocation.getArguments()[1];
callback.process(new java.io.ByteArrayInputStream("Server content".getBytes()));
return null;
}).when(processSession).read(any(FlowFile.class), any(InputStreamCallback.class));
// Execute test using mock
int flowFileSent = serverProtocol.transferFlowFiles(peer, context, processSession, negotiatedCoded);
assertEquals(2, flowFileSent);
assertTrue(remoteSiteListener.isTransactionActive(transactionId));
// Commit transaction
flowFileSent = serverProtocol.commitTransferTransaction(peer, "2747386400");
assertEquals(2, flowFileSent);
}
private DataPacket createClientDataPacket() {
final String contents = "Content from client.";
final byte[] bytes = contents.getBytes();
final InputStream in = new ByteArrayInputStream(bytes);
Map<String, String> attributes = new HashMap<>();
attributes.put("client-attr-1", "client-attr-1-value");
attributes.put("client-attr-2", "client-attr-2-value");
return new StandardDataPacket(attributes, in, bytes.length);
}
@Test
public void testReceiveZeroFile() throws Exception {
final HttpFlowFileServerProtocol serverProtocol = getDefaultHttpFlowFileServerProtocol();
final Peer peer = getDefaultPeer("testReceiveZeroFile");
final HttpServerCommunicationsSession commsSession = (HttpServerCommunicationsSession) peer.getCommunicationsSession();
commsSession.setUserDn("unit-test");
serverProtocol.handshake(peer);
assertTrue(serverProtocol.isHandshakeSuccessful());
final FlowFileCodec negotiatedCoded = serverProtocol.negotiateCodec(peer);
final ProcessContext context = null;
final ProcessSession processSession = mock(ProcessSession.class);
final InputStream httpInputStream = new ByteArrayInputStream(new byte[]{});
((HttpInput)commsSession.getInput()).setInputStream(httpInputStream);
// Execute test using mock
final int flowFileReceived = serverProtocol.receiveFlowFiles(peer, context, processSession, negotiatedCoded);
assertEquals(0, flowFileReceived);
}
@Test
public void testReceiveOneFile() throws Exception {
final HttpFlowFileServerProtocol serverProtocol = getDefaultHttpFlowFileServerProtocol();
final String transactionId = "testReceiveOneFile";
final Peer peer = getDefaultPeer(transactionId);
final HttpServerCommunicationsSession commsSession = (HttpServerCommunicationsSession) peer.getCommunicationsSession();
receiveOneFile(serverProtocol, transactionId, peer);
// Commit transaction
commsSession.setResponseCode(ResponseCode.CONFIRM_TRANSACTION);
final int flowFileReceived = serverProtocol.commitReceiveTransaction(peer);
assertEquals(1, flowFileReceived);
}
@Test
public void testReceiveOneFileBadChecksum() throws Exception {
final HttpFlowFileServerProtocol serverProtocol = getDefaultHttpFlowFileServerProtocol();
final String transactionId = "testReceiveOneFileBadChecksum";
final Peer peer = getDefaultPeer(transactionId);
final HttpServerCommunicationsSession commsSession = (HttpServerCommunicationsSession) peer.getCommunicationsSession();
receiveOneFile(serverProtocol, transactionId, peer);
// Commit transaction
commsSession.setResponseCode(ResponseCode.BAD_CHECKSUM);
try {
serverProtocol.commitReceiveTransaction(peer);
fail();
} catch (IOException e) {
assertTrue(e.getMessage().contains("Received a BadChecksum response"));
}
}
private void receiveOneFile(HttpFlowFileServerProtocol serverProtocol, String transactionId, Peer peer) throws IOException {
final HttpRemoteSiteListener remoteSiteListener = HttpRemoteSiteListener.getInstance();
final HttpServerCommunicationsSession commsSession = (HttpServerCommunicationsSession) peer.getCommunicationsSession();
commsSession.putHandshakeParam(HandshakeProperty.BATCH_COUNT, "1");
commsSession.setUserDn("unit-test");
serverProtocol.handshake(peer);
assertTrue(serverProtocol.isHandshakeSuccessful());
final FlowFileCodec negotiatedCoded = serverProtocol.negotiateCodec(peer);
final ProcessContext context = mock(ProcessContext.class);
final ProcessSession processSession = mock(ProcessSession.class);
final ProvenanceReporter provenanceReporter = mock(ProvenanceReporter.class);
final FlowFile flowFile = mock(FlowFile.class);
DataPacket dataPacket = createClientDataPacket();
final ByteArrayOutputStream testDataOs = new ByteArrayOutputStream();
negotiatedCoded.encode(dataPacket, testDataOs);
final InputStream httpInputStream = new ByteArrayInputStream(testDataOs.toByteArray());
((HttpInput)commsSession.getInput()).setInputStream(httpInputStream);
doAnswer(invocation -> {
InputStream is = (InputStream) invocation.getArguments()[0];
for (int b; (b = is.read()) >= 0;) {
// consume stream.
}
return flowFile;
}).when(processSession).importFrom(any(InputStream.class), any(FlowFile.class));
// AbstractFlowFileServerProtocol adopts builder pattern and putAttribute is the last execution
// which returns flowFile instance used later.
doReturn(flowFile).when(processSession).putAttribute(any(FlowFile.class), any(String.class), any(String.class));
doReturn(provenanceReporter).when(processSession).getProvenanceReporter();
doAnswer(invocation -> {
String peerUrl = (String)invocation.getArguments()[1];
String detail = (String)invocation.getArguments()[3];
assertEquals("http://peer-host:8080/", peerUrl);
assertEquals("Remote Host=peer-host, Remote DN=unit-test", detail);
return null;
}).when(provenanceReporter)
.receive(any(FlowFile.class), any(String.class), any(String.class), any(String.class), any(Long.class));
Set<Relationship> relations = new HashSet<>();
final Relationship relationship = new Relationship.Builder().build();
relations.add(relationship);
doReturn(relations).when(context).getAvailableRelationships();
// Execute test using mock
int flowFileReceived = serverProtocol.receiveFlowFiles(peer, context, processSession, negotiatedCoded);
assertEquals(1, flowFileReceived);
assertTrue(remoteSiteListener.isTransactionActive(transactionId));
}
@Test
public void testReceiveTwoFiles() throws Exception {
final HttpRemoteSiteListener remoteSiteListener = HttpRemoteSiteListener.getInstance();
final HttpFlowFileServerProtocol serverProtocol = getDefaultHttpFlowFileServerProtocol();
final String transactionId = "testReceiveTwoFile";
final Peer peer = getDefaultPeer(transactionId);
final HttpServerCommunicationsSession commsSession = (HttpServerCommunicationsSession) peer.getCommunicationsSession();
commsSession.putHandshakeParam(HandshakeProperty.BATCH_COUNT, "2");
commsSession.setUserDn("unit-test");
serverProtocol.handshake(peer);
assertTrue(serverProtocol.isHandshakeSuccessful());
final FlowFileCodec negotiatedCoded = serverProtocol.negotiateCodec(peer);
final ProcessContext context = mock(ProcessContext.class);
final ProcessSession processSession = mock(ProcessSession.class);
final ProvenanceReporter provenanceReporter = mock(ProvenanceReporter.class);
final FlowFile flowFile1 = mock(FlowFile.class);
final FlowFile flowFile2 = mock(FlowFile.class);
final ByteArrayOutputStream testDataOs = new ByteArrayOutputStream();
negotiatedCoded.encode(createClientDataPacket(), testDataOs);
negotiatedCoded.encode(createClientDataPacket(), testDataOs);
final InputStream httpInputStream = new ByteArrayInputStream(testDataOs.toByteArray());
((HttpInput)commsSession.getInput()).setInputStream(httpInputStream);
doAnswer(invocation -> {
InputStream is = (InputStream) invocation.getArguments()[0];
for (int b; (b = is.read()) >= 0;) {
// consume stream.
}
return flowFile1;
}).when(processSession).importFrom(any(InputStream.class), any(FlowFile.class));
// AbstractFlowFileServerProtocol adopts builder pattern and putAttribute is the last execution
// which returns flowFile instance used later.
doReturn(flowFile1)
.doReturn(flowFile2)
.when(processSession).putAttribute(any(FlowFile.class), any(String.class), any(String.class));
doReturn(provenanceReporter).when(processSession).getProvenanceReporter();
doAnswer(invocation -> {
String peerUrl = (String)invocation.getArguments()[1];
String detail = (String)invocation.getArguments()[3];
assertEquals("http://peer-host:8080/", peerUrl);
assertEquals("Remote Host=peer-host, Remote DN=unit-test", detail);
return null;
}).when(provenanceReporter)
.receive(any(FlowFile.class), any(String.class), any(String.class), any(String.class), any(Long.class));
Set<Relationship> relations = new HashSet<>();
doReturn(relations).when(context).getAvailableRelationships();
// Execute test using mock
int flowFileReceived = serverProtocol.receiveFlowFiles(peer, context, processSession, negotiatedCoded);
assertEquals(2, flowFileReceived);
assertTrue(remoteSiteListener.isTransactionActive(transactionId));
// Commit transaction
commsSession.setResponseCode(ResponseCode.CONFIRM_TRANSACTION);
flowFileReceived = serverProtocol.commitReceiveTransaction(peer);
assertEquals(2, flowFileReceived);
}
}

View File

@ -2338,9 +2338,11 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
// determine the site to site configuration
if (isClustered()) {
controllerDTO.setRemoteSiteListeningPort(controllerFacade.getClusterManagerRemoteSiteListeningPort());
controllerDTO.setRemoteSiteHttpListeningPort(controllerFacade.getClusterManagerRemoteSiteListeningHttpPort());
controllerDTO.setSiteToSiteSecure(controllerFacade.isClusterManagerRemoteSiteCommsSecure());
} else {
controllerDTO.setRemoteSiteListeningPort(controllerFacade.getRemoteSiteListeningPort());
controllerDTO.setRemoteSiteHttpListeningPort(controllerFacade.getRemoteSiteListeningHttpPort());
controllerDTO.setSiteToSiteSecure(controllerFacade.isRemoteSiteCommsSecure());
}

View File

@ -103,7 +103,7 @@ public abstract class ApplicationResource {
@Context
private HttpContext httpContext;
private NiFiProperties properties;
protected NiFiProperties properties;
private RequestReplicator requestReplicator;
private ClusterCoordinator clusterCoordinator;

View File

@ -127,6 +127,7 @@ import org.apache.nifi.reporting.Bulletin;
import org.apache.nifi.reporting.ReportingTask;
import org.apache.nifi.scheduling.SchedulingStrategy;
import org.apache.nifi.util.FormatUtils;
import org.apache.nifi.util.StringUtils;
import org.apache.nifi.web.FlowModification;
import org.apache.nifi.web.Revision;
import org.apache.nifi.web.api.dto.PropertyDescriptorDTO.AllowableValueDTO;
@ -172,6 +173,7 @@ public final class DtoFactory {
return Collator.getInstance(Locale.US).compare(class1.getSimpleName(), class2.getSimpleName());
}
};
public static final String SENSITIVE_VALUE_MASK = "********";
private ControllerServiceProvider controllerServiceProvider;
private EntityFactory entityFactory;
@ -1140,7 +1142,7 @@ public final class DtoFactory {
// determine the property value - don't include sensitive properties
String propertyValue = entry.getValue();
if (propertyValue != null && descriptor.isSensitive()) {
propertyValue = "********";
propertyValue = SENSITIVE_VALUE_MASK;
}
// set the property value
@ -1204,7 +1206,7 @@ public final class DtoFactory {
// determine the property value - don't include sensitive properties
String propertyValue = entry.getValue();
if (propertyValue != null && descriptor.isSensitive()) {
propertyValue = "********";
propertyValue = SENSITIVE_VALUE_MASK;
}
// set the property value
@ -1292,7 +1294,7 @@ public final class DtoFactory {
// determine the property value - don't include sensitive properties
String propertyValue = entry.getValue();
if (propertyValue != null && descriptor.isSensitive()) {
propertyValue = "********";
propertyValue = SENSITIVE_VALUE_MASK;
}
// set the property value
@ -1395,6 +1397,13 @@ public final class DtoFactory {
dto.setTargetUri(group.getTargetUri().toString());
dto.setFlowRefreshed(group.getLastRefreshTime());
dto.setContents(contents);
dto.setTransportProtocol(group.getTransportProtocol().name());
dto.setProxyHost(group.getProxyHost());
dto.setProxyPort(group.getProxyPort());
dto.setProxyUser(group.getProxyUser());
if (!StringUtils.isEmpty(group.getProxyPassword())) {
dto.setProxyPassword(SENSITIVE_VALUE_MASK);
}
// only specify the secure flag if we know the target system has site to site enabled
if (group.isSiteToSiteEnabled()) {
@ -2273,7 +2282,7 @@ public final class DtoFactory {
// determine the property value - don't include sensitive properties
String propertyValue = entry.getValue();
if (propertyValue != null && descriptor.isSensitive()) {
propertyValue = "********";
propertyValue = SENSITIVE_VALUE_MASK;
}
// set the property value
@ -2600,6 +2609,11 @@ public final class DtoFactory {
copy.setInactiveRemoteOutputPortCount(original.getInactiveRemoteOutputPortCount());
copy.setParentGroupId(original.getParentGroupId());
copy.setTargetUri(original.getTargetUri());
copy.setTransportProtocol(original.getTransportProtocol());
copy.setProxyHost(original.getProxyHost());
copy.setProxyPort(original.getProxyPort());
copy.setProxyUser(original.getProxyUser());
copy.setProxyPassword(original.getProxyPassword());
copy.setContents(copyContents);

View File

@ -693,6 +693,17 @@ public class ControllerFacade implements Authorizable {
return flowController.getClusterManagerRemoteSiteListeningPort();
}
/**
* Returns the http(s) port that the Cluster Manager is listening on for
* Site-to-Site communications
*
* @return the socket port that the Cluster Manager is listening on for
* Site-to-Site communications
*/
public Integer getClusterManagerRemoteSiteListeningHttpPort() {
return flowController.getClusterManagerRemoteSiteListeningHttpPort();
}
/**
* Indicates whether or not Site-to-Site communications with the Cluster
* Manager are secure
@ -715,6 +726,17 @@ public class ControllerFacade implements Authorizable {
return flowController.getRemoteSiteListeningPort();
}
/**
* Returns the http(s) port that the local instance is listening on for
* Site-to-Site communications
*
* @return the socket port that the local instance is listening on for
* Site-to-Site communications
*/
public Integer getRemoteSiteListeningHttpPort() {
return flowController.getRemoteSiteListeningHttpPort();
}
/**
* Indicates whether or not Site-to-Site communications with the local
* instance are secure

View File

@ -18,11 +18,14 @@ package org.apache.nifi.web.dao.impl;
import org.apache.nifi.connectable.Position;
import org.apache.nifi.controller.FlowController;
import org.apache.nifi.controller.exception.ValidationException;
import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.groups.RemoteProcessGroup;
import org.apache.nifi.remote.RemoteGroupPort;
import org.apache.nifi.remote.protocol.SiteToSiteTransportProtocol;
import org.apache.nifi.util.FormatUtils;
import org.apache.nifi.web.ResourceNotFoundException;
import org.apache.nifi.web.api.dto.DtoFactory;
import org.apache.nifi.web.api.dto.RemoteProcessGroupDTO;
import org.apache.nifi.web.api.dto.RemoteProcessGroupPortDTO;
import org.apache.nifi.web.dao.RemoteProcessGroupDAO;
@ -34,6 +37,8 @@ import java.util.List;
import java.util.Set;
import java.util.regex.Matcher;
import static org.apache.nifi.util.StringUtils.isEmpty;
public class StandardRemoteProcessGroupDAO extends ComponentDAO implements RemoteProcessGroupDAO {
private static final Logger logger = LoggerFactory.getLogger(StandardRemoteProcessGroupDAO.class);
@ -78,11 +83,8 @@ public class StandardRemoteProcessGroupDAO extends ComponentDAO implements Remot
// create the remote process group
RemoteProcessGroup remoteProcessGroup = flowController.createRemoteProcessGroup(remoteProcessGroupDTO.getId(), rawTargetUri);
// update the remote process group
if (isNotNull(remoteProcessGroupDTO.getPosition())) {
remoteProcessGroup.setPosition(new Position(remoteProcessGroupDTO.getPosition().getX(), remoteProcessGroupDTO.getPosition().getY()));
}
remoteProcessGroup.setComments(remoteProcessGroupDTO.getComments());
// set other properties
updateRemoteProcessGroup(remoteProcessGroup, remoteProcessGroupDTO);
// get the group to add the remote process group to
group.addRemoteProcessGroup(remoteProcessGroup);
@ -134,10 +136,19 @@ public class StandardRemoteProcessGroupDAO extends ComponentDAO implements Remot
}
// validate the proposed configuration
validateProposedRemoteProcessGroupConfiguration(remoteProcessGroupDto);
final List<String> requestValidation = validateProposedRemoteProcessGroupConfiguration(remoteProcessGroupDto);
// ensure there was no validation errors
if (!requestValidation.isEmpty()) {
throw new ValidationException(requestValidation);
}
// if any remote group properties are changing, verify update
if (isAnyNotNull(remoteProcessGroupDto.getYieldDuration(), remoteProcessGroupDto.getCommunicationsTimeout())) {
if (isAnyNotNull(remoteProcessGroupDto.getYieldDuration(),
remoteProcessGroupDto.getCommunicationsTimeout(),
remoteProcessGroupDto.getProxyHost(),
remoteProcessGroupDto.getProxyPort(),
remoteProcessGroupDto.getProxyUser(),
remoteProcessGroupDto.getProxyPassword())) {
remoteProcessGroup.verifyCanUpdate();
}
}
@ -182,7 +193,12 @@ public class StandardRemoteProcessGroupDAO extends ComponentDAO implements Remot
}
// validate the proposed configuration
validateProposedRemoteProcessGroupPortConfiguration(port, remoteProcessGroupPortDto);
final List<String> requestValidation = validateProposedRemoteProcessGroupPortConfiguration(port, remoteProcessGroupPortDto);
// ensure there was no validation errors
if (!requestValidation.isEmpty()) {
throw new ValidationException(requestValidation);
}
// verify update when appropriate
if (isAnyNotNull(remoteProcessGroupPortDto.getConcurrentlySchedulableTaskCount(), remoteProcessGroupPortDto.getUseCompression())) {
@ -222,7 +238,33 @@ public class StandardRemoteProcessGroupDAO extends ComponentDAO implements Remot
validationErrors.add("Yield duration is not a valid time duration (ie 30 sec, 5 min)");
}
}
String proxyPassword = remoteProcessGroupDTO.getProxyPassword();
String proxyUser = remoteProcessGroupDTO.getProxyUser();
String proxyHost = remoteProcessGroupDTO.getProxyHost();
if (isNotNull(remoteProcessGroupDTO.getProxyPort())) {
if (isEmpty(proxyHost)) {
validationErrors.add("Proxy port was specified, but proxy host was empty.");
}
}
if (!isEmpty(proxyUser)) {
if (isEmpty(proxyHost)) {
validationErrors.add("Proxy user name was specified, but proxy host was empty.");
}
if (isEmpty(proxyPassword)) {
validationErrors.add("User password should be specified if Proxy server needs user authentication.");
}
}
if (!isEmpty(proxyPassword)) {
if (isEmpty(proxyHost)) {
validationErrors.add("Proxy user password was specified, but proxy host was empty.");
}
if (isEmpty(proxyPassword)) {
validationErrors.add("User name should be specified if Proxy server needs user authentication.");
}
}
return validationErrors;
}
@ -297,7 +339,12 @@ public class StandardRemoteProcessGroupDAO extends ComponentDAO implements Remot
@Override
public RemoteProcessGroup updateRemoteProcessGroup(RemoteProcessGroupDTO remoteProcessGroupDTO) {
RemoteProcessGroup remoteProcessGroup = locateRemoteProcessGroup(remoteProcessGroupDTO.getId());
return updateRemoteProcessGroup(remoteProcessGroup, remoteProcessGroupDTO);
}
private RemoteProcessGroup updateRemoteProcessGroup(RemoteProcessGroup remoteProcessGroup, RemoteProcessGroupDTO remoteProcessGroupDTO) {
// verify the update request
verifyUpdate(remoteProcessGroup, remoteProcessGroupDTO);
@ -306,6 +353,12 @@ public class StandardRemoteProcessGroupDAO extends ComponentDAO implements Remot
final String comments = remoteProcessGroupDTO.getComments();
final String communicationsTimeout = remoteProcessGroupDTO.getCommunicationsTimeout();
final String yieldDuration = remoteProcessGroupDTO.getYieldDuration();
final String proxyHost = remoteProcessGroupDTO.getProxyHost();
final Integer proxyPort = remoteProcessGroupDTO.getProxyPort();
final String proxyUser = remoteProcessGroupDTO.getProxyUser();
final String proxyPassword = remoteProcessGroupDTO.getProxyPassword();
final String transportProtocol = remoteProcessGroupDTO.getTransportProtocol();
if (isNotNull(name)) {
remoteProcessGroup.setName(name);
@ -322,6 +375,22 @@ public class StandardRemoteProcessGroupDAO extends ComponentDAO implements Remot
if (isNotNull(remoteProcessGroupDTO.getPosition())) {
remoteProcessGroup.setPosition(new Position(remoteProcessGroupDTO.getPosition().getX(), remoteProcessGroupDTO.getPosition().getY()));
}
if (isNotNull(transportProtocol)) {
remoteProcessGroup.setTransportProtocol(SiteToSiteTransportProtocol.valueOf(transportProtocol.toUpperCase()));
// No null check because these proxy settings have to be clear if not specified.
// But when user Enable/Disable transmission, only isTransmitting is sent.
// To prevent clearing these values in that case, set these only if transportProtocol is sent,
// assuming UI sends transportProtocol always for update.
remoteProcessGroup.setProxyHost(proxyHost);
remoteProcessGroup.setProxyPort(proxyPort);
remoteProcessGroup.setProxyUser(proxyUser);
// Keep using current password when null or "********" was sent.
// Passing other values updates the password,
// specify empty String to clear password.
if (isNotNull(proxyPassword) && !DtoFactory.SENSITIVE_VALUE_MASK.equals(proxyPassword)) {
remoteProcessGroup.setProxyPassword(proxyPassword);
}
}
final Boolean isTransmitting = remoteProcessGroupDTO.isTransmitting();
if (isNotNull(isTransmitting)) {

View File

@ -0,0 +1,516 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT 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.web.api;
import org.apache.nifi.remote.HttpRemoteSiteListener;
import org.apache.nifi.remote.Peer;
import org.apache.nifi.remote.RootGroupPort;
import org.apache.nifi.remote.VersionNegotiator;
import org.apache.nifi.remote.exception.HandshakeException;
import org.apache.nifi.remote.io.http.HttpServerCommunicationsSession;
import org.apache.nifi.remote.protocol.ResponseCode;
import org.apache.nifi.remote.protocol.http.HttpFlowFileServerProtocol;
import org.apache.nifi.remote.protocol.http.HttpHeaders;
import org.apache.nifi.util.NiFiProperties;
import org.apache.nifi.web.NiFiServiceFacade;
import org.apache.nifi.web.api.dto.ControllerDTO;
import org.apache.nifi.web.api.entity.ControllerEntity;
import org.apache.nifi.web.api.entity.PeersEntity;
import org.apache.nifi.web.api.entity.TransactionResultEntity;
import org.apache.nifi.web.api.request.ClientIdParameter;
import org.junit.BeforeClass;
import org.junit.Test;
import javax.servlet.ServletContext;
import javax.servlet.http.HttpServletRequest;
import javax.servlet.http.HttpServletResponse;
import javax.ws.rs.core.Response;
import javax.ws.rs.core.StreamingOutput;
import javax.ws.rs.core.UriBuilder;
import javax.ws.rs.core.UriInfo;
import java.io.InputStream;
import java.net.URI;
import java.net.URISyntaxException;
import java.net.URL;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import static org.mockito.Matchers.any;
import static org.mockito.Matchers.eq;
import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.doReturn;
import static org.mockito.Mockito.doThrow;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.spy;
public class TestSiteToSiteResource {
@BeforeClass
public static void setup() throws Exception {
final URL resource = TestSiteToSiteResource.class.getResource("/site-to-site/nifi.properties");
final String propertiesFile = resource.toURI().getPath();
System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, propertiesFile);
}
@Test
public void testGetControllerForOlderVersion() throws Exception {
final HttpServletRequest req = mock(HttpServletRequest.class);
final NiFiServiceFacade serviceFacade = mock(NiFiServiceFacade.class);
final ControllerEntity controllerEntity = new ControllerEntity();
final ControllerDTO controller = new ControllerDTO();
controllerEntity.setController(controller);
controller.setRemoteSiteHttpListeningPort(8080);
controller.setRemoteSiteListeningPort(9990);
doReturn(controller).when(serviceFacade).getController();
final SiteToSiteResource resource = new SiteToSiteResource();
resource.setProperties(NiFiProperties.getInstance());
resource.setServiceFacade(serviceFacade);
final Response response = resource.getController(req);
ControllerEntity resultEntity = (ControllerEntity)response.getEntity();
assertEquals(200, response.getStatus());
assertNull("remoteSiteHttpListeningPort should be null since older version doesn't recognize this field" +
" and throws JSON mapping exception.", resultEntity.getController().getRemoteSiteHttpListeningPort());
assertEquals("Other fields should be retained.", new Integer(9990), controllerEntity.getController().getRemoteSiteListeningPort());
}
@Test
public void testGetController() throws Exception {
final HttpServletRequest req = createCommonHttpServletRequest();
final NiFiServiceFacade serviceFacade = mock(NiFiServiceFacade.class);
final ControllerEntity controllerEntity = new ControllerEntity();
final ControllerDTO controller = new ControllerDTO();
controllerEntity.setController(controller);
controller.setRemoteSiteHttpListeningPort(8080);
controller.setRemoteSiteListeningPort(9990);
doReturn(controller).when(serviceFacade).getController();
final SiteToSiteResource resource = new SiteToSiteResource();
resource.setProperties(NiFiProperties.getInstance());
resource.setServiceFacade(serviceFacade);
final Response response = resource.getController(req);
ControllerEntity resultEntity = (ControllerEntity)response.getEntity();
assertEquals(200, response.getStatus());
assertEquals("remoteSiteHttpListeningPort should be retained", new Integer(8080), resultEntity.getController().getRemoteSiteHttpListeningPort());
assertEquals("Other fields should be retained.", new Integer(9990), controllerEntity.getController().getRemoteSiteListeningPort());
}
private HttpServletRequest createCommonHttpServletRequest() {
final HttpServletRequest req = mock(HttpServletRequest.class);
doReturn("1").when(req).getHeader(eq(HttpHeaders.PROTOCOL_VERSION));
return req;
}
@Test
public void testPeers() throws Exception {
final HttpServletRequest req = createCommonHttpServletRequest();
final NiFiServiceFacade serviceFacade = mock(NiFiServiceFacade.class);
final SiteToSiteResource resource = new SiteToSiteResource();
resource.setProperties(NiFiProperties.getInstance());
resource.setServiceFacade(serviceFacade);
final Response response = resource.getPeers(null, req);
PeersEntity resultEntity = (PeersEntity) response.getEntity();
assertEquals(200, response.getStatus());
assertEquals(1, resultEntity.getPeers().size());
}
@Test
public void testPeersVersionWasNotSpecified() throws Exception {
final HttpServletRequest req = mock(HttpServletRequest.class);
final NiFiServiceFacade serviceFacade = mock(NiFiServiceFacade.class);
final SiteToSiteResource resource = new SiteToSiteResource();
resource.setProperties(NiFiProperties.getInstance());
resource.setServiceFacade(serviceFacade);
final Response response = resource.getPeers(null, req);
TransactionResultEntity resultEntity = (TransactionResultEntity) response.getEntity();
assertEquals(400, response.getStatus());
assertEquals(ResponseCode.ABORT.getCode(), resultEntity.getResponseCode());
}
@Test
public void testPeersVersionNegotiationDowngrade() throws Exception {
final HttpServletRequest req = mock(HttpServletRequest.class);
doReturn("999").when(req).getHeader(eq(HttpHeaders.PROTOCOL_VERSION));
final NiFiServiceFacade serviceFacade = mock(NiFiServiceFacade.class);
final SiteToSiteResource resource = new SiteToSiteResource();
resource.setProperties(NiFiProperties.getInstance());
resource.setServiceFacade(serviceFacade);
final Response response = resource.getPeers(null, req);
PeersEntity resultEntity = (PeersEntity) response.getEntity();
assertEquals(200, response.getStatus());
assertEquals(1, resultEntity.getPeers().size());
assertEquals(new Integer(1), response.getMetadata().getFirst(HttpHeaders.PROTOCOL_VERSION));
}
@Test
public void testCreateTransactionPortNotFound() throws Exception {
final HttpServletRequest req = createCommonHttpServletRequest();
final NiFiServiceFacade serviceFacade = mock(NiFiServiceFacade.class);
final SiteToSiteResource resource = spySiteToSiteResource(serviceFacade);
final HttpFlowFileServerProtocol serverProtocol = mockHttpFlowFileServerProtocol(resource);
doThrow(new HandshakeException(ResponseCode.UNKNOWN_PORT, "Not found.")).when(serverProtocol).handshake(any());
final ClientIdParameter clientId = new ClientIdParameter("client-id");
final ServletContext context = null;
final UriInfo uriInfo = null;
final InputStream inputStream = null;
final Response response = resource.createPortTransaction(clientId, "input-ports", "port-id", req, context, uriInfo, inputStream);
TransactionResultEntity resultEntity = (TransactionResultEntity) response.getEntity();
assertEquals(404, response.getStatus());
assertEquals(ResponseCode.UNKNOWN_PORT.getCode(), resultEntity.getResponseCode());
}
@Test
public void testCreateTransactionPortNotInValidState() throws Exception {
final HttpServletRequest req = createCommonHttpServletRequest();
final NiFiServiceFacade serviceFacade = mock(NiFiServiceFacade.class);
final SiteToSiteResource resource = spySiteToSiteResource(serviceFacade);
final HttpFlowFileServerProtocol serverProtocol = mockHttpFlowFileServerProtocol(resource);
doThrow(new HandshakeException(ResponseCode.PORT_NOT_IN_VALID_STATE, "Not in valid state.")).when(serverProtocol).handshake(any());
final ClientIdParameter clientId = new ClientIdParameter("client-id");
final ServletContext context = null;
final UriInfo uriInfo = null;
final InputStream inputStream = null;
final Response response = resource.createPortTransaction(clientId, "input-ports", "port-id", req, context, uriInfo, inputStream);
TransactionResultEntity resultEntity = (TransactionResultEntity) response.getEntity();
assertEquals(503, response.getStatus());
assertEquals(ResponseCode.PORT_NOT_IN_VALID_STATE.getCode(), resultEntity.getResponseCode());
}
@Test
public void testCreateTransactionUnauthorized() throws Exception {
final HttpServletRequest req = createCommonHttpServletRequest();
final NiFiServiceFacade serviceFacade = mock(NiFiServiceFacade.class);
final SiteToSiteResource resource = spySiteToSiteResource(serviceFacade);
final HttpFlowFileServerProtocol serverProtocol = mockHttpFlowFileServerProtocol(resource);
doThrow(new HandshakeException(ResponseCode.UNAUTHORIZED, "Unauthorized.")).when(serverProtocol).handshake(any());
final ClientIdParameter clientId = new ClientIdParameter("client-id");
final ServletContext context = null;
final UriInfo uriInfo = null;
final InputStream inputStream = null;
final Response response = resource.createPortTransaction(clientId, "input-ports", "port-id", req, context, uriInfo, inputStream);
TransactionResultEntity resultEntity = (TransactionResultEntity) response.getEntity();
assertEquals(401, response.getStatus());
assertEquals(ResponseCode.UNAUTHORIZED.getCode(), resultEntity.getResponseCode());
}
private UriInfo mockUriInfo(final String locationUriStr) throws URISyntaxException {
final UriInfo uriInfo = mock(UriInfo.class);
final UriBuilder uriBuilder = mock(UriBuilder.class);
final URI locationUri = new URI(locationUriStr);
doReturn(uriBuilder).when(uriInfo).getBaseUriBuilder();
doReturn(uriBuilder).when(uriBuilder).path(any(String.class));
doReturn(locationUri).when(uriBuilder).build();
return uriInfo;
}
@Test
public void testCreateTransaction() throws Exception {
final HttpServletRequest req = createCommonHttpServletRequest();
final NiFiServiceFacade serviceFacade = mock(NiFiServiceFacade.class);
final SiteToSiteResource resource = spySiteToSiteResource(serviceFacade);
mockHttpFlowFileServerProtocol(resource);
final String locationUriStr = "http://localhost:8080/nifi-api/site-to-site/input-ports/port-id/transactions/transaction-id";
final ClientIdParameter clientId = new ClientIdParameter("client-id");
final ServletContext context = null;
final UriInfo uriInfo = mockUriInfo(locationUriStr);
final InputStream inputStream = null;
final Response response = resource.createPortTransaction(clientId, "input-ports", "port-id", req, context, uriInfo, inputStream);
TransactionResultEntity resultEntity = (TransactionResultEntity) response.getEntity();
assertEquals(201, response.getStatus());
assertEquals(ResponseCode.PROPERTIES_OK.getCode(), resultEntity.getResponseCode());
assertEquals(locationUriStr, response.getMetadata().getFirst(HttpHeaders.LOCATION_HEADER_NAME).toString());
}
@Test
public void testExtendTransaction() throws Exception {
final HttpServletRequest req = createCommonHttpServletRequest();
final NiFiServiceFacade serviceFacade = mock(NiFiServiceFacade.class);
final SiteToSiteResource resource = spySiteToSiteResource(serviceFacade);
mockHttpFlowFileServerProtocol(resource);
final String locationUriStr = "http://localhost:8080/nifi-api/site-to-site/input-ports/port-id/transactions/transaction-id";
final ClientIdParameter clientId = new ClientIdParameter("client-id");
final ServletContext context = null;
final HttpServletResponse res = null;
final UriInfo uriInfo = mockUriInfo(locationUriStr);
final InputStream inputStream = null;
final HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance();
final String transactionId = transactionManager.createTransaction();
final Response response = resource.extendPortTransactionTTL(clientId, "input-ports", "port-id", transactionId, req, res, context, uriInfo, inputStream);
transactionManager.cancelTransaction(transactionId);
TransactionResultEntity resultEntity = (TransactionResultEntity) response.getEntity();
assertEquals(200, response.getStatus());
assertEquals(ResponseCode.CONTINUE_TRANSACTION.getCode(), resultEntity.getResponseCode());
}
@Test
public void testReceiveFlowFiles() throws Exception {
final HttpServletRequest req = createCommonHttpServletRequest();
final NiFiServiceFacade serviceFacade = mock(NiFiServiceFacade.class);
final SiteToSiteResource resource = spySiteToSiteResource(serviceFacade);
final HttpFlowFileServerProtocol serverProtocol = mockHttpFlowFileServerProtocol(resource);
final RootGroupPort port = mock(RootGroupPort.class);
doReturn(port).when(serverProtocol).getPort();
doAnswer(invocation -> {
Peer peer = (Peer) invocation.getArguments()[0];
((HttpServerCommunicationsSession)peer.getCommunicationsSession()).setChecksum("server-checksum");
return 7;
}).when(port).receiveFlowFiles(any(Peer.class), any());
final ClientIdParameter clientId = new ClientIdParameter("client-id");
final ServletContext context = null;
final InputStream inputStream = null;
final HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance();
final String transactionId = transactionManager.createTransaction();
final Response response = resource.receiveFlowFiles(clientId, "port-id", transactionId, req, context, inputStream);
transactionManager.cancelTransaction(transactionId);
final Object entity = response.getEntity();
assertEquals(202, response.getStatus());
assertEquals("server-checksum", entity);
}
@Test
public void testReceiveZeroFlowFiles() throws Exception {
final HttpServletRequest req = createCommonHttpServletRequest();
final NiFiServiceFacade serviceFacade = mock(NiFiServiceFacade.class);
final SiteToSiteResource resource = spySiteToSiteResource(serviceFacade);
final HttpFlowFileServerProtocol serverProtocol = mockHttpFlowFileServerProtocol(resource);
final RootGroupPort port = mock(RootGroupPort.class);
doReturn(port).when(serverProtocol).getPort();
doAnswer(invocation -> 0).when(port).receiveFlowFiles(any(Peer.class), any());
final ClientIdParameter clientId = new ClientIdParameter("client-id");
final ServletContext context = null;
final InputStream inputStream = null;
final HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance();
final String transactionId = transactionManager.createTransaction();
final Response response = resource.receiveFlowFiles(clientId, "port-id", transactionId, req, context, inputStream);
transactionManager.cancelTransaction(transactionId);
assertEquals(400, response.getStatus());
}
@Test
public void testCommitInputPortTransaction() throws Exception {
final HttpServletRequest req = createCommonHttpServletRequest();
final NiFiServiceFacade serviceFacade = mock(NiFiServiceFacade.class);
final SiteToSiteResource resource = spySiteToSiteResource(serviceFacade);
mockHttpFlowFileServerProtocol(resource);
final ClientIdParameter clientId = new ClientIdParameter("client-id");
final ServletContext context = null;
final InputStream inputStream = null;
final HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance();
final String transactionId = transactionManager.createTransaction();
final Response response = resource.commitInputPortTransaction(clientId, ResponseCode.CONFIRM_TRANSACTION.getCode(), "port-id", transactionId, req, context, inputStream);
transactionManager.cancelTransaction(transactionId);
TransactionResultEntity resultEntity = (TransactionResultEntity) response.getEntity();
assertEquals(200, response.getStatus());
assertEquals(ResponseCode.CONFIRM_TRANSACTION.getCode(), resultEntity.getResponseCode());
}
@Test
public void testTransferFlowFiles() throws Exception {
final HttpServletRequest req = createCommonHttpServletRequest();
final NiFiServiceFacade serviceFacade = mock(NiFiServiceFacade.class);
final SiteToSiteResource resource = spySiteToSiteResource(serviceFacade);
mockHttpFlowFileServerProtocol(resource);
final ClientIdParameter clientId = new ClientIdParameter("client-id");
final ServletContext context = null;
final HttpServletResponse res = null;
final InputStream inputStream = null;
final HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance();
final String transactionId = transactionManager.createTransaction();
final Response response = resource.transferFlowFiles(clientId, "port-id", transactionId, req, res, context, inputStream);
transactionManager.cancelTransaction(transactionId);
final Object entity = response.getEntity();
assertEquals(202, response.getStatus());
assertTrue(entity instanceof StreamingOutput);
}
@Test
public void testCommitOutputPortTransaction() throws Exception {
final HttpServletRequest req = createCommonHttpServletRequest();
final NiFiServiceFacade serviceFacade = mock(NiFiServiceFacade.class);
final SiteToSiteResource resource = spySiteToSiteResource(serviceFacade);
mockHttpFlowFileServerProtocol(resource);
final ClientIdParameter clientId = new ClientIdParameter("client-id");
final ServletContext context = null;
final InputStream inputStream = null;
final HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance();
final String transactionId = transactionManager.createTransaction();
final Response response = resource.commitOutputPortTransaction(clientId, ResponseCode.CONFIRM_TRANSACTION.getCode(),
"client-checksum", "port-id", transactionId, req, context, inputStream);
transactionManager.cancelTransaction(transactionId);
TransactionResultEntity resultEntity = (TransactionResultEntity) response.getEntity();
assertEquals(200, response.getStatus());
assertEquals(ResponseCode.CONFIRM_TRANSACTION.getCode(), resultEntity.getResponseCode());
}
@Test
public void testCommitOutputPortTransactionBadChecksum() throws Exception {
final HttpServletRequest req = createCommonHttpServletRequest();
final NiFiServiceFacade serviceFacade = mock(NiFiServiceFacade.class);
final SiteToSiteResource resource = spySiteToSiteResource(serviceFacade);
final HttpFlowFileServerProtocol serverProtocol = mockHttpFlowFileServerProtocol(resource);
doThrow(new HandshakeException(ResponseCode.BAD_CHECKSUM, "Bad checksum.")).when(serverProtocol).commitTransferTransaction(any(), any());
final ClientIdParameter clientId = new ClientIdParameter("client-id");
final ServletContext context = null;
final InputStream inputStream = null;
final HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance();
final String transactionId = transactionManager.createTransaction();
final Response response = resource.commitOutputPortTransaction(clientId, ResponseCode.CONFIRM_TRANSACTION.getCode(),
"client-checksum", "port-id", transactionId, req, context, inputStream);
transactionManager.cancelTransaction(transactionId);
TransactionResultEntity resultEntity = (TransactionResultEntity) response.getEntity();
assertEquals(400, response.getStatus());
assertEquals(ResponseCode.BAD_CHECKSUM.getCode(), resultEntity.getResponseCode());
}
private HttpFlowFileServerProtocol mockHttpFlowFileServerProtocol(SiteToSiteResource resource) {
final HttpFlowFileServerProtocol serverProtocol = mock(HttpFlowFileServerProtocol.class);
doReturn(serverProtocol).when(resource).getHttpFlowFileServerProtocol(any(VersionNegotiator.class));
return serverProtocol;
}
private SiteToSiteResource spySiteToSiteResource(NiFiServiceFacade serviceFacade) {
final SiteToSiteResource resource = spy(SiteToSiteResource.class);
resource.setProperties(NiFiProperties.getInstance());
resource.setServiceFacade(serviceFacade);
return resource;
}
}

View File

@ -73,7 +73,7 @@ nifi.components.status.snapshot.frequency=10 secs
# Site to Site properties
#For testing purposes. Default value should actually be empty!
nifi.remote.input.socket.host=
nifi.remote.input.host=
nifi.remote.input.socket.port=
nifi.remote.input.secure=false

View File

@ -0,0 +1,174 @@
# Licensed to the Apache Software Foundation (ASF) under one or more
# contributor license agreements. See the NOTICE file distributed with
# this work for additional information regarding copyright ownership.
# The ASF licenses this file to You under the Apache License, Version 2.0
# (the "License"); you may not use this file except in compliance with
# the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
# Core Properties #
nifi.version=${nifi.version}
nifi.flow.configuration.file=${nifi.flow.configuration.file}
nifi.flow.configuration.archive.dir=${nifi.flow.configuration.archive.dir}
nifi.flowcontroller.autoResumeState=${nifi.flowcontroller.autoResumeState}
nifi.flowcontroller.graceful.shutdown.period=${nifi.flowcontroller.graceful.shutdown.period}
nifi.flowservice.writedelay.interval=${nifi.flowservice.writedelay.interval}
nifi.administrative.yield.duration=${nifi.administrative.yield.duration}
# If a component has no work to do (is "bored"), how long should we wait before checking again for work?
nifi.bored.yield.duration=${nifi.bored.yield.duration}
nifi.authorizer.configuration.file=${nifi.authorizer.configuration.file}
nifi.login.identity.provider.configuration.file=${nifi.login.identity.provider.configuration.file}
nifi.templates.directory=${nifi.templates.directory}
nifi.ui.banner.text=${nifi.ui.banner.text}
nifi.ui.autorefresh.interval=${nifi.ui.autorefresh.interval}
nifi.nar.library.directory=${nifi.nar.library.directory}
nifi.nar.working.directory=${nifi.nar.working.directory}
nifi.documentation.working.directory=${nifi.documentation.working.directory}
####################
# State Management #
####################
nifi.state.management.configuration.file=${nifi.state.management.configuration.file}
# The ID of the local state provider
nifi.state.management.provider.local=${nifi.state.management.provider.local}
# The ID of the cluster-wide state provider. This will be ignored if NiFi is not clustered but must be populated if running in a cluster.
nifi.state.management.provider.cluster=${nifi.state.management.provider.cluster}
# Specifies whether or not this instance of NiFi should run an embedded ZooKeeper server
nifi.state.management.embedded.zookeeper.start=${nifi.state.management.embedded.zookeeper.start}
# Properties file that provides the ZooKeeper properties to use if <nifi.state.management.embedded.zookeeper.start> is set to true
nifi.state.management.embedded.zookeeper.properties=${nifi.state.management.embedded.zookeeper.properties}
# H2 Settings
nifi.database.directory=${nifi.database.directory}
nifi.h2.url.append=${nifi.h2.url.append}
# FlowFile Repository
nifi.flowfile.repository.implementation=${nifi.flowfile.repository.implementation}
nifi.flowfile.repository.directory=${nifi.flowfile.repository.directory}
nifi.flowfile.repository.partitions=${nifi.flowfile.repository.partitions}
nifi.flowfile.repository.checkpoint.interval=${nifi.flowfile.repository.checkpoint.interval}
nifi.flowfile.repository.always.sync=${nifi.flowfile.repository.always.sync}
nifi.swap.manager.implementation=${nifi.swap.manager.implementation}
nifi.queue.swap.threshold=${nifi.queue.swap.threshold}
nifi.swap.in.period=${nifi.swap.in.period}
nifi.swap.in.threads=${nifi.swap.in.threads}
nifi.swap.out.period=${nifi.swap.out.period}
nifi.swap.out.threads=${nifi.swap.out.threads}
# Content Repository
nifi.content.repository.implementation=${nifi.content.repository.implementation}
nifi.content.claim.max.appendable.size=${nifi.content.claim.max.appendable.size}
nifi.content.claim.max.flow.files=${nifi.content.claim.max.flow.files}
nifi.content.repository.directory.default=${nifi.content.repository.directory.default}
nifi.content.repository.archive.max.retention.period=${nifi.content.repository.archive.max.retention.period}
nifi.content.repository.archive.max.usage.percentage=${nifi.content.repository.archive.max.usage.percentage}
nifi.content.repository.archive.enabled=${nifi.content.repository.archive.enabled}
nifi.content.repository.always.sync=${nifi.content.repository.always.sync}
nifi.content.viewer.url=${nifi.content.viewer.url}
# Provenance Repository Properties
nifi.provenance.repository.implementation=${nifi.provenance.repository.implementation}
# Persistent Provenance Repository Properties
nifi.provenance.repository.directory.default=${nifi.provenance.repository.directory.default}
nifi.provenance.repository.max.storage.time=${nifi.provenance.repository.max.storage.time}
nifi.provenance.repository.max.storage.size=${nifi.provenance.repository.max.storage.size}
nifi.provenance.repository.rollover.time=${nifi.provenance.repository.rollover.time}
nifi.provenance.repository.rollover.size=${nifi.provenance.repository.rollover.size}
nifi.provenance.repository.query.threads=${nifi.provenance.repository.query.threads}
nifi.provenance.repository.index.threads=${nifi.provenance.repository.index.threads}
nifi.provenance.repository.compress.on.rollover=${nifi.provenance.repository.compress.on.rollover}
nifi.provenance.repository.always.sync=${nifi.provenance.repository.always.sync}
nifi.provenance.repository.journal.count=${nifi.provenance.repository.journal.count}
# Comma-separated list of fields. Fields that are not indexed will not be searchable. Valid fields are:
# EventType, FlowFileUUID, Filename, TransitURI, ProcessorID, AlternateIdentifierURI, Relationship, Details
nifi.provenance.repository.indexed.fields=${nifi.provenance.repository.indexed.fields}
# FlowFile Attributes that should be indexed and made searchable. Some examples to consider are filename, uuid, mime.type
nifi.provenance.repository.indexed.attributes=${nifi.provenance.repository.indexed.attributes}
# Large values for the shard size will result in more Java heap usage when searching the Provenance Repository
# but should provide better performance
nifi.provenance.repository.index.shard.size=${nifi.provenance.repository.index.shard.size}
# Indicates the maximum length that a FlowFile attribute can be when retrieving a Provenance Event from
# the repository. If the length of any attribute exceeds this value, it will be truncated when the event is retrieved.
nifi.provenance.repository.max.attribute.length=${nifi.provenance.repository.max.attribute.length}
# Volatile Provenance Respository Properties
nifi.provenance.repository.buffer.size=${nifi.provenance.repository.buffer.size}
# Component Status Repository
nifi.components.status.repository.implementation=${nifi.components.status.repository.implementation}
nifi.components.status.repository.buffer.size=${nifi.components.status.repository.buffer.size}
nifi.components.status.snapshot.frequency=${nifi.components.status.snapshot.frequency}
# Site to Site properties
nifi.remote.input.host=
nifi.remote.input.secure=false
nifi.remote.input.socket.port=
nifi.remote.input.http.enabled=true
nifi.remote.input.http.transaction.ttl=30 sec
# web properties #
nifi.web.war.directory=${nifi.web.war.directory}
nifi.web.http.host=${nifi.web.http.host}
nifi.web.http.port=8080
nifi.web.https.host=${nifi.web.https.host}
nifi.web.https.port=${nifi.web.https.port}
nifi.web.jetty.working.directory=${nifi.jetty.work.dir}
nifi.web.jetty.threads=${nifi.web.jetty.threads}
# security properties #
nifi.sensitive.props.key=
nifi.sensitive.props.algorithm=${nifi.sensitive.props.algorithm}
nifi.sensitive.props.provider=${nifi.sensitive.props.provider}
nifi.security.keystore=${nifi.security.keystore}
nifi.security.keystoreType=${nifi.security.keystoreType}
nifi.security.keystorePasswd=${nifi.security.keystorePasswd}
nifi.security.keyPasswd=${nifi.security.keyPasswd}
nifi.security.truststore=${nifi.security.truststore}
nifi.security.truststoreType=${nifi.security.truststoreType}
nifi.security.truststorePasswd=${nifi.security.truststorePasswd}
nifi.security.needClientAuth=${nifi.security.needClientAuth}
nifi.security.user.authorizer=${nifi.security.user.authorizer}
nifi.security.user.login.identity.provider=${nifi.security.user.login.identity.provider}
nifi.security.ocsp.responder.url=${nifi.security.ocsp.responder.url}
nifi.security.ocsp.responder.certificate=${nifi.security.ocsp.responder.certificate}
# cluster common properties (all nodes must have same values) #
nifi.cluster.protocol.heartbeat.interval=${nifi.cluster.protocol.heartbeat.interval}
nifi.cluster.protocol.is.secure=${nifi.cluster.protocol.is.secure}
# cluster node properties (only configure for cluster nodes) #
nifi.cluster.is.node=${nifi.cluster.is.node}
nifi.cluster.node.address=${nifi.cluster.node.address}
nifi.cluster.node.protocol.port=${nifi.cluster.node.protocol.port}
nifi.cluster.node.protocol.threads=${nifi.cluster.node.protocol.threads}
nifi.cluster.node.event.history.size=${nifi.cluster.node.event.history.size}
nifi.cluster.node.connection.timeout=${nifi.cluster.node.connection.timeout}
nifi.cluster.node.read.timeout=${nifi.cluster.node.read.timeout}
nifi.cluster.firewall.file=${nifi.cluster.firewall.file}
# How long a request should be allowed to hold a 'lock' on a component. #
nifi.cluster.request.replication.claim.timeout=${nifi.cluster.request.replication.claim.timeout}
# zookeeper properties, used for cluster management #
nifi.zookeeper.connect.string=${nifi.zookeeper.connect.string}
nifi.zookeeper.connect.timeout=${nifi.zookeeper.connect.timeout}
nifi.zookeeper.session.timeout=${nifi.zookeeper.session.timeout}
nifi.zookeeper.root.node=${nifi.zookeeper.root.node}
# kerberos #
nifi.kerberos.krb5.file=${nifi.kerberos.krb5.file}
nifi.kerberos.service.principal=${nifi.kerberos.service.principal}
nifi.kerberos.keytab.location=${nifi.kerberos.keytab.location}
nifi.kerberos.authentication.expiration=${nifi.kerberos.authentication.expiration}

View File

@ -23,5 +23,77 @@
<input id="new-remote-process-group-uri" type="text" placeholder="https://remotehost:8080/nifi"/>
</div>
</div>
<div class="setting">
<div class="remote-process-group-timeout-setting">
<div class="setting-name">
Communications timeout
<img class="setting-icon icon-info" src="images/iconInfo.png" alt="Info" title="When communication with this remote process group takes longer than this amount of time, it will timeout."/>
</div>
<div class="setting-field">
<input type="text" class="small-setting-input" id="new-remote-process-group-timeout"/>
</div>
</div>
<div class="remote-process-group-yield-duration-setting">
<div class="setting-name">
Yield duration
<img class="setting-icon icon-info" src="images/iconInfo.png" alt="Info" title="When communication with this remote process group fails, it will not be scheduled again until this amount of time elapses."/>
</div>
<div class="setting-field">
<input type="text" class="small-setting-input" id="new-remote-process-group-yield-duration"/>
</div>
</div>
<div class="clear"></div>
</div>
<div class="setting">
<div class="setting-name">
Transport Protocol
<img class="setting-icon icon-info" src="images/iconInfo.png" alt="Info" title="Specify the transport protocol to use for this Remote Process Group communication."/>
</div>
<div class="setting-field">
<div id="new-remote-process-group-transport-protocol-combo"></div>
</div>
</div>
<div class="setting">
<div class="remote-process-group-proxy-host-setting">
<div class="setting-name">
HTTP Proxy server hostname
<img class="setting-icon icon-info" src="images/iconInfo.png" alt="Info" title="Specify the proxy server's hostname to use. If not specified, HTTP traffics are sent directly to the target NiFi instance."/>
</div>
<div class="setting-field">
<input type="text" class="small-setting-input" id="new-remote-process-group-proxy-host"/>
</div>
</div>
<div class="remote-process-group-proxy-port-setting">
<div class="setting-name">
HTTP Proxy server port
<img class="setting-icon icon-info" src="images/iconInfo.png" alt="Info" title="Specify the proxy server's port number, optional. If not specified, default port 80 will be used."/>
</div>
<div class="setting-field">
<input type="text" class="small-setting-input" id="new-remote-process-group-proxy-port"/>
</div>
</div>
<div class="clear"></div>
</div>
<div class="setting">
<div class="remote-process-group-proxy-user-setting">
<div class="setting-name">
HTTP Proxy user
<img class="setting-icon icon-info" src="images/iconInfo.png" alt="Info" title="Specify an user name to connect to the proxy server, optional."/>
</div>
<div class="setting-field">
<input type="text" class="small-setting-input" id="new-remote-process-group-proxy-user"/>
</div>
</div>
<div class="remote-process-group-proxy-password-setting">
<div class="setting-name">
HTTP Proxy password
<img class="setting-icon icon-info" src="images/iconInfo.png" alt="Info" title="Specify an user password to connect to the proxy server, optional."/>
</div>
<div class="setting-field">
<input type="password" class="small-setting-input" id="new-remote-process-group-proxy-password"/>
</div>
</div>
<div class="clear"></div>
</div>
</div>
</div>

View File

@ -36,19 +36,76 @@
</div>
</div>
<div class="setting">
<div class="setting-name">Communications timeout</div>
<div class="remote-process-group-timeout-setting">
<div class="setting-name">
Communications timeout
<img class="setting-icon icon-info" src="images/iconInfo.png" alt="Info" title="When communication with this remote process group takes longer than this amount of time, it will timeout."/>
</div>
<div class="setting-field">
<input type="text" id="remote-process-group-timeout"/>
<input type="text" class="small-setting-input" id="remote-process-group-timeout"/>
</div>
</div>
<div class="setting">
<div class="remote-process-group-yield-duration-setting">
<div class="setting-name">
Yield duration
<img class="setting-icon icon-info" src="images/iconInfo.png" alt="Info" title="When communication with this remote process group fails, it will not be scheduled again until this amount of time elapses."/>
</div>
<div class="setting-field">
<input type="text" id="remote-process-group-yield-duration"/>
<input type="text" class="small-setting-input" id="remote-process-group-yield-duration"/>
</div>
</div>
<div class="clear"></div>
</div>
<div class="setting">
<div class="setting-name">
Transport Protocol
<img class="setting-icon icon-info" src="images/iconInfo.png" alt="Info" title="Specify the transport protocol to use for this Remote Process Group communication."/>
</div>
<div class="setting-field">
<div id="remote-process-group-transport-protocol-combo"></div>
</div>
</div>
<div class="setting">
<div class="remote-process-group-proxy-host-setting">
<div class="setting-name">
HTTP Proxy server hostname
<img class="setting-icon icon-info" src="images/iconInfo.png" alt="Info" title="Specify the proxy server's hostname to use. If not specified, HTTP traffics are sent directly to the target NiFi instance."/>
</div>
<div class="setting-field">
<input type="text" class="small-setting-input" id="remote-process-group-proxy-host"/>
</div>
</div>
<div class="remote-process-group-proxy-port-setting">
<div class="setting-name">
HTTP Proxy server port
<img class="setting-icon icon-info" src="images/iconInfo.png" alt="Info" title="Specify the proxy server's port number, optional. If not specified, default port 80 will be used."/>
</div>
<div class="setting-field">
<input type="text" class="small-setting-input" id="remote-process-group-proxy-port"/>
</div>
</div>
<div class="clear"></div>
</div>
<div class="setting">
<div class="remote-process-group-proxy-user-setting">
<div class="setting-name">
HTTP Proxy user
<img class="setting-icon icon-info" src="images/iconInfo.png" alt="Info" title="Specify an user name to connect to the proxy server, optional."/>
</div>
<div class="setting-field">
<input type="text" class="small-setting-input" id="remote-process-group-proxy-user"/>
</div>
</div>
<div class="remote-process-group-proxy-password-setting">
<div class="setting-name">
HTTP Proxy password
<img class="setting-icon icon-info" src="images/iconInfo.png" alt="Info" title="Specify an user password to connect to the proxy server, optional."/>
</div>
<div class="setting-field">
<input type="password" class="small-setting-input" id="remote-process-group-proxy-password"/>
</div>
</div>
<div class="clear"></div>
</div>
</div>
</div>

View File

@ -36,12 +36,16 @@
</div>
</div>
<div class="setting">
<div class="setting-name">Communications timeout</div>
<div class="remote-process-group-timeout-setting">
<div class="setting-name">
Communications timeout
<img class="setting-icon icon-info" src="images/iconInfo.png" alt="Info" title="When communication with this remote process group takes longer than this amount of time, it will timeout."/>
</div>
<div class="setting-field">
<span id="read-only-remote-process-group-timeout"></span>
</div>
</div>
<div class="setting">
<div class="remote-process-group-yield-duration-setting">
<div class="setting-name">
Yield duration
<img class="setting-icon icon-info" src="images/iconInfo.png" alt="Info" title="When communication with this remote process group fails, it will not be scheduled again until this amount of time elapses."/>
@ -50,5 +54,58 @@
<span id="read-only-remote-process-group-yield-duration"></span>
</div>
</div>
<div class="clear"></div>
</div>
<div class="setting">
<div class="setting-name">
Transport Protocol
<img class="setting-icon icon-info" src="images/iconInfo.png" alt="Info" title="Transport protocol to use for this Remote Process Group communication."/>
</div>
<div class="setting-field">
<div id="read-only-remote-process-group-transport-protocol"></div>
</div>
</div>
<div class="setting">
<div class="remote-process-group-proxy-host-setting">
<div class="setting-name">
HTTP Proxy server hostname
<img class="setting-icon icon-info" src="images/iconInfo.png" alt="Info" title="Specify the proxy server's hostname to use. If not specified, HTTP traffics are sent directly to the target NiFi instance."/>
</div>
<div class="setting-field">
<span id="read-only-remote-process-group-proxy-host"></span>
</div>
</div>
<div class="remote-process-group-proxy-port-setting">
<div class="setting-name">
HTTP Proxy server port
<img class="setting-icon icon-info" src="images/iconInfo.png" alt="Info" title="Specify the proxy server's port number, optional. If not specified, default port 80 will be used."/>
</div>
<div class="setting-field">
<span id="read-only-remote-process-group-proxy-port"></span>
</div>
</div>
<div class="clear"></div>
</div>
<div class="setting">
<div class="remote-process-group-proxy-user-setting">
<div class="setting-name">
HTTP Proxy user
<img class="setting-icon icon-info" src="images/iconInfo.png" alt="Info" title="Specify an user name to connect to the proxy server, optional."/>
</div>
<div class="setting-field">
<span id="read-only-remote-process-group-proxy-user"></span>
</div>
</div>
<div class="remote-process-group-proxy-password-setting">
<div class="setting-name">
HTTP Proxy password
<img class="setting-icon icon-info" src="images/iconInfo.png" alt="Info" title="Specify an user password to connect to the proxy server, optional."/>
</div>
<div class="setting-field">
<span id="read-only-remote-process-group-proxy-password"></span>
</div>
</div>
<div class="clear"></div>
</div>
</div>
</div>

View File

@ -43,14 +43,25 @@
#new-remote-process-group-dialog {
z-index: 1301;
display: none;
width: 350px;
height: 150px;
width: 400px;
height: 370px;
}
#new-remote-process-group-dialog .small-setting-input {
width: 160px;
}
#new-remote-process-group-transport-protocol-combo {
width: 160px;
height: 18px;
line-height: 18px;
}
#new-remote-process-group-uri {
width: 320px;
width: 370px;
}
#new-template-dialog {
z-index: 1301;
display: none;

Some files were not shown because too many files have changed in this diff Show More