This commit is contained in:
Jenn Barnabee 2015-02-24 13:45:49 -05:00
commit 4ac6dfd5e5
129 changed files with 5471 additions and 1830 deletions

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.remote;
import java.util.concurrent.TimeUnit;
/**
* A model object for referring to a remote destination (i.e., a Port) for site-to-site communications
*/
public interface RemoteDestination {
/**
* Returns the identifier of the remote destination
*
* @return
*/
String getIdentifier();
/**
* Returns the human-readable name of the remote destination
* @return
*/
String getName();
/**
* Returns the amount of time that system should pause sending to a particular node if unable to
* send data to or receive data from this endpoint
* @param timeUnit
* @return
*/
long getYieldPeriod(TimeUnit timeUnit);
/**
* Returns whether or not compression should be used when transferring data to or receiving
* data from the remote endpoint
* @return
*/
boolean isUseCompression();
}

View File

@ -0,0 +1,45 @@
<?xml version="1.0"?>
<project
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
<modelVersion>4.0.0</modelVersion>
<parent>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-commons</artifactId>
<version>0.0.2-incubating-SNAPSHOT</version>
</parent>
<artifactId>nifi-site-to-site-client</artifactId>
<dependencies>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-api</artifactId>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-utils</artifactId>
</dependency>
<!-- <dependency> <groupId>com.sun.jersey</groupId> <artifactId>jersey-client</artifactId>
</dependency> <dependency> <groupId>org.apache.nifi</groupId> <artifactId>nifi-web-utils</artifactId>
</dependency> -->
<dependency>
<groupId>org.codehaus.jackson</groupId>
<artifactId>jackson-mapper-asl</artifactId>
<version>1.9.13</version>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-client-dto</artifactId>
<version>0.0.2-incubating-SNAPSHOT</version>
</dependency>
<dependency>
<groupId>junit</groupId>
<artifactId>junit</artifactId>
<scope>test</scope>
</dependency>
</dependencies>
</project>

View File

@ -0,0 +1,47 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.remote;
/**
* Represents the remote entity that the client is communicating with
*/
public interface Communicant {
/**
* Returns the NiFi site-to-site URL for the remote NiFi instance
* @return
*/
String getUrl();
/**
* The Host of the remote NiFi instance
* @return
*/
String getHost();
/**
* The Port that the remote NiFi instance is listening on for site-to-site communications
* @return
*/
int getPort();
/**
* The distinguished name that the remote NiFi instance has provided in its certificate if
* using secure communications, or <code>null</code> if the Distinguished Name is unknown
* @return
*/
String getDistinguishedName();
}

View File

@ -18,32 +18,51 @@ package org.apache.nifi.remote;
import java.io.IOException;
import java.net.URI;
import java.util.HashMap;
import java.util.Map;
import org.apache.nifi.remote.protocol.CommunicationsSession;
public class Peer {
public class Peer implements Communicant {
private final PeerDescription description;
private final CommunicationsSession commsSession;
private final String url;
private final String clusterUrl;
private final String host;
private long penalizationExpiration = 0L;
private final int port;
private final Map<String, Long> penaltyExpirationMap = new HashMap<>();
private boolean closed = false;
public Peer(final CommunicationsSession commsSession, final String url) {
public Peer(final PeerDescription description, final CommunicationsSession commsSession, final String peerUrl, final String clusterUrl) {
this.description = description;
this.commsSession = commsSession;
this.url = url;
this.url = peerUrl;
this.clusterUrl = clusterUrl;
try {
this.host = new URI(url).getHost();
final URI uri = new URI(peerUrl);
this.port = uri.getPort();
this.host = uri.getHost();
} catch (final Exception e) {
throw new IllegalArgumentException("Invalid URL: " + url);
throw new IllegalArgumentException("Invalid URL: " + peerUrl);
}
}
public PeerDescription getDescription() {
return description;
}
@Override
public String getUrl() {
return url;
}
public String getClusterUrl() {
return clusterUrl;
}
public CommunicationsSession getCommunicationsSession() {
return commsSession;
}
@ -51,22 +70,39 @@ public class Peer {
public void close() throws IOException {
this.closed = true;
// TODO: Consume the InputStream so that it doesn't linger on the Peer's outgoing socket buffer
// Consume the InputStream so that it doesn't linger on the Peer's outgoing socket buffer
try {
commsSession.getInput().consume();
} finally {
commsSession.close();
}
public void penalize(final long millis) {
penalizationExpiration = Math.max(penalizationExpiration, System.currentTimeMillis() + millis);
}
public boolean isPenalized() {
return penalizationExpiration > System.currentTimeMillis();
/**
* Penalizes this peer for the given destination only for the provided number of milliseconds
* @param destinationId
* @param millis
*/
public void penalize(final String destinationId, final long millis) {
final Long currentPenalty = penaltyExpirationMap.get(destinationId);
final long proposedPenalty = System.currentTimeMillis() + millis;
if ( currentPenalty == null || proposedPenalty > currentPenalty ) {
penaltyExpirationMap.put(destinationId, proposedPenalty);
}
}
public boolean isPenalized(final String destinationId) {
final Long currentPenalty = penaltyExpirationMap.get(destinationId);
return (currentPenalty != null && currentPenalty > System.currentTimeMillis());
}
public boolean isClosed() {
return closed;
}
@Override
public String getHost() {
return host;
}
@ -98,10 +134,18 @@ public class Peer {
sb.append("Peer[url=").append(url);
if (closed) {
sb.append(",CLOSED");
} else if (isPenalized()) {
sb.append(",PENALIZED");
}
sb.append("]");
return sb.toString();
}
@Override
public int getPort() {
return port;
}
@Override
public String getDistinguishedName() {
return commsSession.getUserDn();
}
}

View File

@ -0,0 +1,79 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF 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;
public class PeerDescription {
private final String hostname;
private final int port;
private final boolean secure;
public PeerDescription(final String hostname, final int port, final boolean secure) {
this.hostname = hostname;
this.port = port;
this.secure = secure;
}
public String getHostname() {
return hostname;
}
public int getPort() {
return port;
}
public boolean isSecure() {
return secure;
}
@Override
public String toString() {
return "PeerDescription[hostname=" + hostname + ", port=" + port + ", secure=" + secure + "]";
}
@Override
public int hashCode() {
final int prime = 31;
int result = 1;
result = prime * result + ((hostname == null) ? 0 : hostname.hashCode());
result = prime * result + port;
return result;
}
@Override
public boolean equals(final Object obj) {
if (this == obj) {
return true;
}
if (obj == null) {
return false;
}
if (getClass() != obj.getClass()) {
return false;
}
final PeerDescription other = (PeerDescription) obj;
if (hostname == null) {
if (other.hostname != null) {
return false;
}
} else if (!hostname.equals(other.hostname)) {
return false;
}
return port == other.port;
}
}

View File

@ -17,29 +17,16 @@
package org.apache.nifi.remote;
public class PeerStatus {
private final String hostname;
private final int port;
private final boolean secure;
private final PeerDescription description;
private final int numFlowFiles;
public PeerStatus(final String hostname, final int port, final boolean secure, final int numFlowFiles) {
this.hostname = hostname;
this.port = port;
this.secure = secure;
public PeerStatus(final PeerDescription description, final int numFlowFiles) {
this.description = description;
this.numFlowFiles = numFlowFiles;
}
public String getHostname() {
return hostname;
}
public int getPort() {
return port;
}
public boolean isSecure() {
return secure;
public PeerDescription getPeerDescription() {
return description;
}
public int getFlowFileCount() {
@ -48,12 +35,13 @@ public class PeerStatus {
@Override
public String toString() {
return "PeerStatus[hostname=" + hostname + ",port=" + port + ",secure=" + secure + ",flowFileCount=" + numFlowFiles + "]";
return "PeerStatus[hostname=" + description.getHostname() + ",port=" + description.getPort() +
",secure=" + description.isSecure() + ",flowFileCount=" + numFlowFiles + "]";
}
@Override
public int hashCode() {
return 9824372 + hostname.hashCode() + port;
return 9824372 + description.getHostname().hashCode() + description.getPort() * 41;
}
@Override
@ -67,6 +55,6 @@ public class PeerStatus {
}
final PeerStatus other = (PeerStatus) obj;
return port == other.port && hostname.equals(other.hostname);
return description.equals(other.getPeerDescription());
}
}

View File

@ -0,0 +1,73 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.remote;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.IOException;
import org.apache.nifi.remote.exception.HandshakeException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class RemoteResourceInitiator {
public static final int RESOURCE_OK = 20;
public static final int DIFFERENT_RESOURCE_VERSION = 21;
public static final int ABORT = 255;
private static final Logger logger = LoggerFactory.getLogger(RemoteResourceInitiator.class);
public static VersionedRemoteResource initiateResourceNegotiation(final VersionedRemoteResource resource, final DataInputStream dis, final DataOutputStream dos) throws IOException, HandshakeException {
// Write the classname of the RemoteStreamCodec, followed by its version
logger.debug("Negotiating resource; proposal is {}", resource);
dos.writeUTF(resource.getResourceName());
final VersionNegotiator negotiator = resource.getVersionNegotiator();
dos.writeInt(negotiator.getVersion());
dos.flush();
// wait for response from server.
logger.debug("Receiving response from remote instance");
final int statusCode = dis.read();
switch (statusCode) {
case RESOURCE_OK: // server accepted our proposal of codec name/version
logger.debug("Response was RESOURCE_OK");
return resource;
case DIFFERENT_RESOURCE_VERSION: // server accepted our proposal of codec name but not the version
logger.debug("Response was DIFFERENT_RESOURCE_VERSION");
// Get server's preferred version
final int newVersion = dis.readInt();
// Determine our new preferred version that is no greater than the server's preferred version.
final Integer newPreference = negotiator.getPreferredVersion(newVersion);
// If we could not agree with server on a version, fail now.
if ( newPreference == null ) {
throw new HandshakeException("Could not agree on version for " + resource);
}
negotiator.setVersion(newPreference);
// Attempt negotiation of resource based on our new preferred version.
return initiateResourceNegotiation(resource, dis, dos);
case ABORT:
logger.debug("Response was ABORT");
throw new HandshakeException("Remote destination aborted connection with message: " + dis.readUTF());
default:
logger.debug("Response was {}; unable to negotiate codec", statusCode);
return null; // Unable to negotiate codec
}
}
}

View File

@ -0,0 +1,214 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF 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 java.io.IOException;
import java.util.Map;
import org.apache.nifi.remote.protocol.DataPacket;
/**
* <p>
* Provides a transaction for performing site-to-site data transfers.
* </p>
*
* <p>
* A Transaction is created by calling the
* {@link org.apache.nifi.remote.client.SiteToSiteClient#createTransaction(TransferDirection) createTransaction(TransferDirection)}
* method of a {@link org.apache.nifi.remote.client.SiteToSiteClient SiteToSiteClient}. The resulting Transaction
* can be used to either send or receive data but not both. A new Transaction must be created in order perform the
* other operation.
* </p>
*
* <p>
* The general flow of execute of a Transaction is as follows:
* <ol>
* <li>Create the transaction as described above.</li>
* <li>Send data via the {@link #send(DataPacket)} method or receive data via the {@link #receive()} method. This method
* will be called 1 or more times. In the case of receive, this method should be called until the method returns {@code null},
* signifying that the remote instance is finished sending data. <b>Note:</b> <code>receive()</code> should not be
* called a second time without first fully consuming the stream from the previous Packet that was received.</li>
* <li>Confirm the transaction via the {@link #confirm()} method.</li>
* <li>Either complete the transaction via the {@link #complete(boolean)} method or cancel the transaction
* via the {@link #cancel()} method.</li>
* </ol>
* </p>
*
* <p>
* It is important that the Transaction be terminated in order to free the resources held
* by the Transaction. If a Transaction is not terminated, its resources will not be freed and
* if the Transaction holds connections from a connection pool, the connections in that pool
* will eventually become exhausted. A Transaction is terminated by calling one of the following
* methods:
* <ul>
* <li>{@link #complete(boolean)}</li>
* <li>{@link #cancel()}</li>
* <li>{@link #error()}</li>
* </ul>
* </p>
*
* <p>
* If at any point an IOException is thrown from one of the methods of the Transaction, that Transaction
* is automatically closed via a call to {@link #error()}.
* </p>
*
* <p>
* The Transaction class should not be assumed to be thread-safe.
* </p>
*/
public interface Transaction {
/**
* Sends information to the remote NiFi instance.
*
* @param dataPacket the data packet to send
* @throws IOException
*/
void send(DataPacket dataPacket) throws IOException;
/**
* Sends the given byte array as the content of a {@link DataPacket} along with the
* provided attributes
*
* @param content
* @param attributes
* @throws IOException
*/
void send(byte[] content, Map<String, String> attributes) throws IOException;
/**
* Retrieves information from the remote NiFi instance, if any is available. If no data is available, will return
* {@code null}. It is important to consume all data from the remote NiFi instance before attempting to
* call {@link #confirm()}. This is because the sender is always responsible for determining when the Transaction
* has finished. This is done in order to prevent the need for a round-trip network request to receive data for
* each data packet.
*
* @return the DataPacket received, or {@code null} if there is no more data to receive.
* @throws IOException
*/
DataPacket receive() throws IOException;
/**
* <p>
* Confirms the data that was sent or received by comparing CRC32's of the data sent and the data received.
* </p>
*
* <p>
* Even if the protocol being used to send the data is reliable and guarantees ordering of packets (such as TCP),
* it is still required that we confirm the transaction before completing the transaction. This is done as
* "safety net" or a defensive programming technique. Mistakes happen, and this mechanism helps to ensure that if
* a bug exists somewhere along the line that we do not end up sending or receiving corrupt data. If the
* CRC32 of the sender and the CRC32 of the receiver do not match, an IOException will be thrown and both the
* sender and receiver will cancel the transaction automatically.
* </p>
*
* <p>
* If the {@link TransferDirection} of this Transaction is RECEIVE, this method will throw an Exception unless
* all data from the remote instance has been consumed (i.e., a call to {@link #receive()} returns {@code null}).
* </p>
*
* <p>
* If the {@link TransferDirection} of this Transaction is SEND, calling this method dictates that no more data will be
* sent in this transaction. I.e., there will be no more calls to {@link #send(DataPacket)}.
* </p>
*
* @throws IOException
*/
void confirm() throws IOException;
/**
* <p>
* Completes the transaction and indicates to both the sender and receiver that the data transfer was
* successful.
* </p>
*
* @throws IOException
*
* @return a TransactionCompletion that contains details about the Transaction
*/
TransactionCompletion complete() throws IOException;
/**
* <p>
* Cancels this transaction, indicating to the sender that the data has not been successfully received so that
* the sender can retry or handle however is appropriate.
* </p>
*
* @param explanation an explanation to tell the other party why the transaction was canceled.
* @throws IOException
*/
void cancel(final String explanation) throws IOException;
/**
* <p>
* Sets the TransactionState of the Transaction to {@link TransactionState#ERROR}, and closes
* the Transaction. The underlying connection should not be returned to a connection pool in this case.
* </p>
*/
void error();
/**
* Returns the current state of the Transaction.
* @return
* @throws IOException
*/
TransactionState getState() throws IOException;
/**
* Returns a Communicant that represents the other side of this Transaction (i.e.,
* the remote NiFi instance)
* @return
*/
Communicant getCommunicant();
public enum TransactionState {
/**
* Transaction has been started but no data has been sent or received.
*/
TRANSACTION_STARTED,
/**
* Transaction has been started and data has been sent or received.
*/
DATA_EXCHANGED,
/**
* Data that has been transferred has been confirmed via its CRC. Transaction is
* ready to be completed.
*/
TRANSACTION_CONFIRMED,
/**
* Transaction has been successfully completed.
*/
TRANSACTION_COMPLETED,
/**
* The Transaction has been canceled.
*/
TRANSACTION_CANCELED,
/**
* The Transaction ended in an error.
*/
ERROR;
}
}

View File

@ -0,0 +1,63 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF 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 java.io.InputStream;
import java.util.concurrent.TimeUnit;
import org.apache.nifi.remote.protocol.DataPacket;
/**
* A TransactionCompletion provides information about a {@link Transaction} that has completed successfully.
*/
public interface TransactionCompletion {
/**
* When a sending to a NiFi instance, the server may accept the content sent to it
* but indicate that its queues are full and that the client should backoff sending
* data for a bit. This method returns <code>true</code> if the server did in fact
* request that, <code>false</code> otherwise.
* @return
*/
boolean isBackoff();
/**
* Returns the number of Data Packets that were sent to or received from the remote
* NiFi instance in the Transaction
* @return
*/
int getDataPacketsTransferred();
/**
* Returns the number of bytes of DataPacket content that were sent to or received from
* the remote NiFI instance in the Transaction. Note that this is different than the number
* of bytes actually transferred between the client and server, as it does not take into
* account the attributes or protocol-specific information that is exchanged but rather
* takes into account only the data in the {@link InputStream} of the {@link DataPacket}
* @return
*/
long getBytesTransferred();
/**
* Returns the amount of time that the Transaction took, from the time that the Transaction
* was created to the time that the Transaction was completed.
* @param timeUnit
* @return
*/
long getDuration(TimeUnit timeUnit);
}

View File

@ -16,8 +16,19 @@
*/
package org.apache.nifi.remote;
public enum TransferDirection {
/**
* An enumeration for specifying the direction in which data should be transferred between a client
* and a remote NiFi instance.
*/
public enum TransferDirection {
/**
* The client is to send data to the remote instance.
*/
SEND,
/**
* The client is to receive data from the remote instance.
*/
RECEIVE;
}

View File

@ -0,0 +1,519 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF 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.io.Closeable;
import java.io.File;
import java.io.IOException;
import java.io.Serializable;
import java.util.concurrent.TimeUnit;
import javax.net.ssl.SSLContext;
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 instance of NiFi
* (or NiFi cluster) and retrieving data from a remote instance of NiFi (or NiFi cluster).
* </p>
*
* <p>
* When configuring the client via the {@link SiteToSiteClient.Builder}, the Builder must
* be provided the URL of the remote NiFi instance. If the URL points to a standalone instance
* of NiFi, all interaction will take place with that instance of NiFi. However, if the URL
* points to the NiFi Cluster Manager of a cluster, the client will automatically handle load
* balancing the transactions across the different nodes in the cluster.
* </p>
*
* <p>
* The SiteToSiteClient provides a {@link Transaction} through which all interaction with the
* remote instance takes place. After data has been exchanged or it is determined that no data
* is available, the Transaction can then be canceled (via the {@link Transaction#cancel(String)}
* method) or can be completed (via the {@link Transaction#complete(boolean)} method).
* </p>
*
* <p>
* An instance of SiteToSiteClient can be obtained by constructing a new instance of the
* {@link SiteToSiteClient.Builder} class, calling the appropriate methods to configured the
* client as desired, and then calling the {@link SiteToSiteClient.Builder#build() build()} method.
* </p>
*
* <p>
* The SiteToSiteClient itself is immutable once constructed and is thread-safe. Many threads can
* share access to the same client. However, the {@link Transaction} that is created by the client
* is not thread safe and should not be shared among threads.
* </p>
*/
public interface SiteToSiteClient extends Closeable {
/**
* <p>
* Creates a new Transaction that can be used to either send data to a remote NiFi instance
* or receive data from a remote NiFi instance, depending on the value passed for the {@code direction} argument.
* </p>
*
* <p>
* <b>Note:</b> If all of the nodes are penalized (See {@link Builder#nodePenalizationPeriod(long, TimeUnit)}), then
* this method will return <code>null</code>.
* </p>
*
* @param direction specifies which direction the data should be transferred. A value of {@link TransferDirection#SEND}
* indicates that this Transaction will send data to the remote instance; a value of {@link TransferDirection#RECEIVE} indicates
* that this Transaction will be used to receive data from the remote instance.
*
* @return a Transaction to use for sending or receiving data, or <code>null</code> if all nodes are penalized.
* @throws IOException
*/
Transaction createTransaction(TransferDirection direction) throws HandshakeException, PortNotRunningException, ProtocolException, UnknownPortException, IOException;
/**
* <p>
* Returns {@code true} if site-to-site communications with the remote instance are secure,
* {@code false} if site-to-site communications with the remote instance are not secure. Whether or not
* communications are secure depends on the server, not the client.
* </p>
*
* <p>
* In order to determine whether the server is configured for secure communications, the client may have
* to query the server's RESTful interface. Doing so could result in an IOException.
* </p>
*
* @return
* @throws IOException if unable to query the remote instance's RESTful interface or if the remote
* instance is not configured to allow site-to-site communications
*/
boolean isSecure() throws IOException;
/**
* <p>
* Returns the configuration object that was built by the Builder
* </p>
* @return
*/
SiteToSiteClientConfig getConfig();
/**
* <p>
* The Builder is the mechanism by which all configuration is passed to the SiteToSiteClient.
* Once constructed, the SiteToSiteClient cannot be reconfigured (i.e., it is immutable). If
* a change in configuration should be desired, the client should be {@link Closeable#close() closed}
* and a new client created.
* </p>
*/
public static class Builder implements Serializable {
private static final long serialVersionUID = -4954962284343090219L;
private String url;
private long timeoutNanos = TimeUnit.SECONDS.toNanos(30);
private long penalizationNanos = TimeUnit.SECONDS.toNanos(3);
private long idleExpirationNanos = TimeUnit.SECONDS.toNanos(30L);
private SSLContext sslContext;
private EventReporter eventReporter;
private File peerPersistenceFile;
private boolean useCompression;
private String portName;
private String portIdentifier;
private int batchCount;
private long batchSize;
private long batchNanos;
/**
* Populates the builder with values from the provided config
* @param config
* @return
*/
public Builder fromConfig(final SiteToSiteClientConfig config) {
this.url = config.getUrl();
this.timeoutNanos = config.getTimeout(TimeUnit.NANOSECONDS);
this.penalizationNanos = config.getPenalizationPeriod(TimeUnit.NANOSECONDS);
this.idleExpirationNanos = config.getIdleConnectionExpiration(TimeUnit.NANOSECONDS);
this.sslContext = config.getSslContext();
this.eventReporter = config.getEventReporter();
this.peerPersistenceFile = config.getPeerPersistenceFile();
this.useCompression = config.isUseCompression();
this.portName = config.getPortName();
this.portIdentifier = config.getPortIdentifier();
this.batchCount = config.getPreferredBatchCount();
this.batchSize = config.getPreferredBatchSize();
this.batchNanos = config.getPreferredBatchDuration(TimeUnit.NANOSECONDS);
return this;
}
/**
* Specifies the URL of the remote NiFi instance. If this URL points to the Cluster Manager of
* a NiFi cluster, data transfer to and from nodes will be automatically load balanced across
* the different nodes.
*
* @param url
* @return
*/
public Builder url(final String url) {
this.url = url;
return this;
}
/**
* Specifies the communications timeouts to use when interacting with the remote instances. The
* default value is 30 seconds.
*
* @param timeout
* @param unit
* @return
*/
public Builder timeout(final long timeout, final TimeUnit unit) {
this.timeoutNanos = unit.toNanos(timeout);
return this;
}
/**
* Specifies the amount of time that a connection can remain idle in the connection pool before it
* is "expired" and shutdown. The default value is 30 seconds.
*
* @param timeout
* @param unit
* @return
*/
public Builder idleExpiration(final long timeout, final TimeUnit unit) {
this.idleExpirationNanos = unit.toNanos(timeout);
return this;
}
/**
* If there is a problem communicating with a node (i.e., any node in the remote NiFi cluster
* or the remote instance of NiFi if it is standalone), specifies how long the client should
* wait before attempting to communicate with that node again. While a particular node is penalized,
* all other nodes in the remote cluster (if any) will still be available for communication.
* The default value is 3 seconds.
*
* @param period
* @param unit
* @return
*/
public Builder nodePenalizationPeriod(final long period, final TimeUnit unit) {
this.penalizationNanos = unit.toNanos(period);
return this;
}
/**
* Specifies the SSL Context to use when communicating with the remote NiFi instance(s). If not
* specified, communications will not be secure. The remote instance of NiFi always determines
* whether or not Site-to-Site communications are secure (i.e., the client will always use
* secure or non-secure communications, depending on what the server dictates).
*
* @param sslContext
* @return
*/
public Builder sslContext(final SSLContext sslContext) {
this.sslContext = sslContext;
return this;
}
/**
* Provides an EventReporter that can be used by the client in order to report any events that
* could be of interest when communicating with the remote instance. The EventReporter provided
* must be threadsafe.
*
* @param eventReporter
* @return
*/
public Builder eventReporter(final EventReporter eventReporter) {
this.eventReporter = eventReporter;
return this;
}
/**
* Specifies a file that the client can write to in order to persist the list of nodes in the
* remote cluster and recover the list of nodes upon restart. This allows the client to function
* if the remote Cluster Manager is unavailable, even after a restart of the client software.
* If not specified, the list of nodes will not be persisted and a failure of the Cluster Manager
* will result in not being able to communicate with the remote instance if a new client
* is created.
*
* @param peerPersistenceFile
* @return
*/
public Builder peerPersistenceFile(final File peerPersistenceFile) {
this.peerPersistenceFile = peerPersistenceFile;
return this;
}
/**
* Specifies whether or not data should be compressed before being transferred to or from the
* remote instance.
*
* @param compress
* @return
*/
public Builder useCompression(final boolean compress) {
this.useCompression = compress;
return this;
}
/**
* Specifies the name of the port to communicate with. Either the port name or the port identifier
* must be specified.
*
* @param portName
* @return
*/
public Builder portName(final String portName) {
this.portName = portName;
return this;
}
/**
* Specifies the unique identifier of the port to communicate with. If it is known, this is preferred over providing
* the port name, as the port name may change.
*
* @param portIdentifier
* @return
*/
public Builder portIdentifier(final String portIdentifier) {
this.portIdentifier = portIdentifier;
return this;
}
/**
* When pulling data from a NiFi instance, the sender chooses how large a Transaction is. However,
* the client has the ability to request a particular batch size/duration. This method specifies
* the preferred number of {@link DataPacket}s to include in a Transaction.
*
* @return
*/
public Builder requestBatchCount(final int count) {
this.batchCount = count;
return this;
}
/**
* When pulling data from a NiFi instance, the sender chooses how large a Transaction is. However,
* the client has the ability to request a particular batch size/duration. This method specifies
* the preferred number of bytes to include in a Transaction.
*
* @return
*/
public Builder requestBatchSize(final long bytes) {
this.batchSize = bytes;
return this;
}
/**
* When pulling data from a NiFi instance, the sender chooses how large a Transaction is. However,
* the client has the ability to request a particular batch size/duration. This method specifies
* the preferred amount of time that a Transaction should span.
*
* @return
*/
public Builder requestBatchDuration(final long value, final TimeUnit unit) {
this.batchNanos = unit.toNanos(value);
return this;
}
/**
* Returns a {@link SiteToSiteClientConfig} for the configured values but does not create a SiteToSiteClient
* @return
*/
public SiteToSiteClientConfig buildConfig() {
final SiteToSiteClientConfig config = new SiteToSiteClientConfig() {
private static final long serialVersionUID = 1323119754841633818L;
@Override
public boolean isUseCompression() {
return Builder.this.isUseCompression();
}
@Override
public String getUrl() {
return Builder.this.getUrl();
}
@Override
public long getTimeout(final TimeUnit timeUnit) {
return Builder.this.getTimeout(timeUnit);
}
@Override
public long getIdleConnectionExpiration(final TimeUnit timeUnit) {
return Builder.this.getIdleConnectionExpiration(timeUnit);
}
@Override
public SSLContext getSslContext() {
return Builder.this.getSslContext();
}
@Override
public String getPortName() {
return Builder.this.getPortName();
}
@Override
public String getPortIdentifier() {
return Builder.this.getPortIdentifier();
}
@Override
public long getPenalizationPeriod(final TimeUnit timeUnit) {
return Builder.this.getPenalizationPeriod(timeUnit);
}
@Override
public File getPeerPersistenceFile() {
return Builder.this.getPeerPersistenceFile();
}
@Override
public EventReporter getEventReporter() {
return Builder.this.getEventReporter();
}
@Override
public long getPreferredBatchDuration(final TimeUnit timeUnit) {
return timeUnit.convert(Builder.this.batchNanos, TimeUnit.NANOSECONDS);
}
@Override
public long getPreferredBatchSize() {
return Builder.this.batchSize;
}
@Override
public int getPreferredBatchCount() {
return Builder.this.batchCount;
}
};
return config;
}
/**
* Builds a new SiteToSiteClient that can be used to send and receive data with remote instances of NiFi
* @return
*
* @throws IllegalStateException if either the url is not set or neither the port name nor port identifier
* is set.
*/
public SiteToSiteClient build() {
if ( url == null ) {
throw new IllegalStateException("Must specify URL to build Site-to-Site client");
}
if ( portName == null && portIdentifier == null ) {
throw new IllegalStateException("Must specify either Port Name or Port Identifier to builder Site-to-Site client");
}
return new SocketClient(buildConfig());
}
/**
* Returns the configured URL for the remote NiFi instance
* @return
*/
public String getUrl() {
return url;
}
/**
* Returns the communications timeout
* @return
*/
public long getTimeout(final TimeUnit timeUnit) {
return timeUnit.convert(timeoutNanos, TimeUnit.NANOSECONDS);
}
/**
* Returns the amount of of time that a connection can remain idle in the connection
* pool before being shutdown
* @param timeUnit
* @return
*/
public long getIdleConnectionExpiration(final TimeUnit timeUnit) {
return timeUnit.convert(idleExpirationNanos, TimeUnit.NANOSECONDS);
}
/**
* Returns the amount of time that a particular node will be ignored after a
* communications error with that node occurs
* @param timeUnit
* @return
*/
public long getPenalizationPeriod(TimeUnit timeUnit) {
return timeUnit.convert(penalizationNanos, TimeUnit.NANOSECONDS);
}
/**
* Returns the SSL Context that is configured for this builder
* @return
*/
public SSLContext getSslContext() {
return sslContext;
}
/**
* Returns the EventReporter that is to be used by clients to report events
* @return
*/
public EventReporter getEventReporter() {
return eventReporter;
}
/**
* Returns the file that is to be used for persisting the nodes of a remote cluster, if any.
* @return
*/
public File getPeerPersistenceFile() {
return peerPersistenceFile;
}
/**
* Returns a boolean indicating whether or not compression will be used to transfer data
* to and from the remote instance
* @return
*/
public boolean isUseCompression() {
return useCompression;
}
/**
* Returns the name of the port that the client is to communicate with.
* @return
*/
public String getPortName() {
return portName;
}
/**
* Returns the identifier of the port that the client is to communicate with.
* @return
*/
public String getPortIdentifier() {
return portIdentifier;
}
}
}

View File

@ -0,0 +1,124 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF 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.io.File;
import java.io.Serializable;
import java.util.concurrent.TimeUnit;
import javax.net.ssl.SSLContext;
import org.apache.nifi.events.EventReporter;
import org.apache.nifi.remote.protocol.DataPacket;
public interface SiteToSiteClientConfig extends Serializable {
/**
* Returns the configured URL for the remote NiFi instance
* @return
*/
String getUrl();
/**
* Returns the communications timeout in nanoseconds
* @return
*/
long getTimeout(final TimeUnit timeUnit);
/**
* Returns the amount of time that a connection can remain idle before it is
* "expired" and shut down
* @param timeUnit
* @return
*/
long getIdleConnectionExpiration(TimeUnit timeUnit);
/**
* Returns the amount of time that a particular node will be ignored after a
* communications error with that node occurs
* @param timeUnit
* @return
*/
long getPenalizationPeriod(TimeUnit timeUnit);
/**
* Returns the SSL Context that is configured for this builder
* @return
*/
SSLContext getSslContext();
/**
* Returns the file that is to be used for persisting the nodes of a remote cluster, if any.
* @return
*/
File getPeerPersistenceFile();
/**
* Returns a boolean indicating whether or not compression will be used to transfer data
* to and from the remote instance
* @return
*/
boolean isUseCompression();
/**
* Returns the name of the port that the client is to communicate with.
* @return
*/
String getPortName();
/**
* Returns the identifier of the port that the client is to communicate with.
* @return
*/
String getPortIdentifier();
/**
* When pulling data from a NiFi instance, the sender chooses how large a Transaction is. However,
* the client has the ability to request a particular batch size/duration. This returns the maximum
* amount of time that we will request a NiFi instance to send data to us in a Transaction.
*
* @param timeUnit
* @return
*/
long getPreferredBatchDuration(TimeUnit timeUnit);
/**
* When pulling data from a NiFi instance, the sender chooses how large a Transaction is. However,
* the client has the ability to request a particular batch size/duration. This returns the maximum
* number of bytes that we will request a NiFi instance to send data to us in a Transaction.
*
* @return
*/
long getPreferredBatchSize();
/**
* When pulling data from a NiFi instance, the sender chooses how large a Transaction is. However,
* the client has the ability to request a particular batch size/duration. This returns the maximum
* number of {@link DataPacket}s that we will request a NiFi instance to send data to us in a Transaction.
*
* @return
*/
int getPreferredBatchCount();
/**
* Returns the EventReporter that is to be used by clients to report events
* @return
*/
EventReporter getEventReporter();
}

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.socket;
import org.apache.nifi.remote.Peer;
import org.apache.nifi.remote.codec.FlowFileCodec;
import org.apache.nifi.remote.protocol.socket.SocketClientProtocol;
public class EndpointConnection {
private final Peer peer;
private final SocketClientProtocol socketClientProtocol;
private final FlowFileCodec codec;
private volatile long lastUsed;
public EndpointConnection(final Peer peer, final SocketClientProtocol socketClientProtocol, final FlowFileCodec codec) {
this.peer = peer;
this.socketClientProtocol = socketClientProtocol;
this.codec = codec;
}
public FlowFileCodec getCodec() {
return codec;
}
public SocketClientProtocol getSocketClientProtocol() {
return socketClientProtocol;
}
public Peer getPeer() {
return peer;
}
public void setLastTimeUsed() {
lastUsed = System.currentTimeMillis();
}
public long getLastTimeUsed() {
return lastUsed;
}
}

View File

@ -0,0 +1,970 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF 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.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.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 javax.security.cert.CertificateExpiredException;
import javax.security.cert.CertificateNotYetValidException;
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.RemoteDestination;
import org.apache.nifi.remote.RemoteResourceInitiator;
import org.apache.nifi.remote.TransferDirection;
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;
import org.apache.nifi.remote.exception.ProtocolException;
import org.apache.nifi.remote.exception.TransmissionDisabledException;
import org.apache.nifi.remote.exception.UnknownPortException;
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.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;
public class EndpointConnectionPool {
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);
private static final long PEER_CACHE_MILLIS = TimeUnit.MILLISECONDS.convert(1, TimeUnit.MINUTES);
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 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);
}
public EndpointConnectionPool(final String clusterUrl, final RemoteDestination remoteDestination, final int commsTimeoutMillis, final int idleExpirationMillis,
final SSLContext sslContext, final EventReporter eventReporter, final File persistenceFile)
{
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.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;
}
// 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.
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("NiFi Site-to-Site Connection Pool Maintenance");
return thread;
}
});
taskExecutor.scheduleWithFixedDelay(new Runnable() {
@Override
public void run() {
refreshPeers();
}
}, 0, 5, TimeUnit.SECONDS);
taskExecutor.scheduleWithFixedDelay(new Runnable() {
@Override
public void run() {
cleanupExpiredSockets();
}
}, 5, 5, TimeUnit.SECONDS);
}
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());
}
}
public EndpointConnection getEndpointConnection(final TransferDirection direction) throws IOException, HandshakeException, PortNotRunningException, UnknownPortException, ProtocolException {
return getEndpointConnection(direction, null);
}
public EndpointConnection getEndpointConnection(final TransferDirection direction, final SiteToSiteClientConfig config) throws IOException, HandshakeException, PortNotRunningException, UnknownPortException, ProtocolException {
//
// Attempt to get a connection state that already exists for this URL.
//
FlowFileCodec codec = null;
CommunicationsSession commsSession = null;
SocketClientProtocol protocol = null;
EndpointConnection connection;
Peer peer = null;
logger.debug("{} getting next peer status", this);
final PeerStatus peerStatus = getNextPeerStatus(direction);
logger.debug("{} next peer status = {}", this, peerStatus);
if ( peerStatus == null ) {
return null;
}
final PeerDescription peerDescription = peerStatus.getPeerDescription();
BlockingQueue<EndpointConnection> connectionQueue = connectionQueueMap.get(peerStatus);
if ( connectionQueue == null ) {
connectionQueue = new LinkedBlockingQueue<>();
BlockingQueue<EndpointConnection> existing = connectionQueueMap.putIfAbsent(peerDescription, connectionQueue);
if ( existing != null ) {
connectionQueue = existing;
}
}
final List<EndpointConnection> addBack = new ArrayList<>();
try {
do {
connection = connectionQueue.poll();
logger.debug("{} Connection State for {} = {}", this, clusterUrl, connection);
final String portId = getPortIdentifier(direction);
if ( connection == null && !addBack.isEmpty() ) {
// all available connections have been penalized.
logger.debug("{} all Connections for {} are penalized; returning no Connection", this, portId);
return null;
}
if ( connection != null && connection.getPeer().isPenalized(portId) ) {
// we have a connection, but it's penalized. We want to add it back to the queue
// when we've found one to use.
addBack.add(connection);
continue;
}
// if we can't get an existing Connection, create one
if ( connection == null ) {
logger.debug("{} No Connection available for Port {}; creating new Connection", this, portId);
protocol = new SocketClientProtocol();
protocol.setDestination(new IdEnrichedRemoteDestination(remoteDestination, portId));
final long penalizationMillis = remoteDestination.getYieldPeriod(TimeUnit.MILLISECONDS);
try {
logger.debug("{} Establishing site-to-site connection with {}", this, peerStatus);
commsSession = establishSiteToSiteConnection(peerStatus);
} catch (final IOException ioe) {
penalize(peerStatus.getPeerDescription(), penalizationMillis);
throw ioe;
}
final DataInputStream dis = new DataInputStream(commsSession.getInput().getInputStream());
final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream());
try {
logger.debug("{} Negotiating protocol", this);
RemoteResourceInitiator.initiateResourceNegotiation(protocol, dis, dos);
} catch (final HandshakeException e) {
try {
commsSession.close();
} catch (final IOException ioe) {
throw e;
}
}
final String peerUrl = "nifi://" + peerDescription.getHostname() + ":" + peerDescription.getPort();
peer = new Peer(peerDescription, commsSession, peerUrl, clusterUrl.toString());
// set properties based on config
if ( config != null ) {
protocol.setTimeout((int) config.getTimeout(TimeUnit.MILLISECONDS));
protocol.setPreferredBatchCount(config.getPreferredBatchCount());
protocol.setPreferredBatchSize(config.getPreferredBatchSize());
protocol.setPreferredBatchDuration(config.getPreferredBatchDuration(TimeUnit.MILLISECONDS));
}
// perform handshake
try {
logger.debug("{} performing handshake", this);
protocol.handshake(peer);
// handle error cases
if ( protocol.isDestinationFull() ) {
logger.warn("{} {} indicates that port's destination is full; penalizing peer", this, peer);
penalize(peer, penalizationMillis);
connectionQueue.offer(connection);
continue;
} else if ( protocol.isPortInvalid() ) {
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);
cleanup(protocol, peer);
throw new UnknownPortException(peer.toString() + " indicates that port " + portId + " is not known");
}
// negotiate the FlowFileCodec to use
logger.debug("{} negotiating codec", this);
codec = protocol.negotiateCodec(peer);
logger.debug("{} negotiated codec is {}", this, codec);
} catch (final PortNotRunningException | UnknownPortException e) {
throw e;
} catch (final Exception e) {
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());
logger.error(message);
if ( logger.isDebugEnabled() ) {
logger.error("", e);
}
throw e;
}
connection = new EndpointConnection(peer, protocol, codec);
} else {
final long lastTimeUsed = connection.getLastTimeUsed();
final long millisSinceLastUse = System.currentTimeMillis() - lastTimeUsed;
if ( commsTimeout > 0L && millisSinceLastUse >= commsTimeout ) {
cleanup(connection.getSocketClientProtocol(), connection.getPeer());
connection = null;
} else {
codec = connection.getCodec();
peer = connection.getPeer();
commsSession = peer.getCommunicationsSession();
protocol = connection.getSocketClientProtocol();
}
}
} while ( connection == null || codec == null || commsSession == null || protocol == null );
} finally {
if ( !addBack.isEmpty() ) {
connectionQueue.addAll(addBack);
}
}
activeConnections.add(connection);
return connection;
}
public boolean offer(final EndpointConnection endpointConnection) {
final Peer peer = endpointConnection.getPeer();
if ( peer == null ) {
return false;
}
final String url = peer.getUrl();
if ( url == null ) {
return false;
}
final BlockingQueue<EndpointConnection> connectionQueue = connectionQueueMap.get(peer.getDescription());
if ( connectionQueue == null ) {
return false;
}
activeConnections.remove(endpointConnection);
if ( shutdown ) {
terminate(endpointConnection);
return false;
} else {
endpointConnection.setLastTimeUsed();
return connectionQueue.offer(endpointConnection);
}
}
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
*/
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 {
protocol.shutdown(peer);
} catch (final TransmissionDisabledException e) {
// User disabled transmission.... do nothing.
logger.debug(this + " Transmission Disabled by User");
} catch (IOException e1) {
}
}
if ( peer != null ) {
try {
peer.close();
} catch (final TransmissionDisabledException e) {
// User disabled transmission.... do nothing.
logger.debug(this + " Transmission Disabled by User");
} catch (IOException e1) {
}
}
}
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());
logger.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);
return (expirationEnd == null ? false : expirationEnd > System.currentTimeMillis() );
}
private List<PeerStatus> createPeerStatusList(final TransferDirection direction) throws IOException, HandshakeException, UnknownPortException, PortNotRunningException {
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, HandshakeException, UnknownPortException, PortNotRunningException {
final String hostname = clusterUrl.getHost();
final Integer port = getSiteToSitePort();
if ( port == null ) {
throw new IOException("Remote instance of NiFi is not configured to allow site-to-site communications");
}
final PeerDescription clusterPeerDescription = new PeerDescription(hostname, port, clusterUrl.toString().startsWith("https://"));
final CommunicationsSession commsSession = establishSiteToSiteConnection(hostname, port);
final Peer peer = new Peer(clusterPeerDescription, commsSession, "nifi://" + hostname + ":" + port, clusterUrl.toString());
final SocketClientProtocol clientProtocol = new SocketClientProtocol();
final DataInputStream dis = new DataInputStream(commsSession.getInput().getInputStream());
final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream());
RemoteResourceInitiator.initiateResourceNegotiation(clientProtocol, dis, dos);
clientProtocol.setTimeout(commsTimeout);
if (clientProtocol.getVersionNegotiator().getVersion() < 5) {
String portId = getPortIdentifier(TransferDirection.RECEIVE);
if ( portId == null ) {
portId = getPortIdentifier(TransferDirection.SEND);
}
if ( portId == null ) {
peer.close();
throw new IOException("Failed to determine the identifier of port " + remoteDestination.getName());
}
clientProtocol.handshake(peer, portId);
} else {
clientProtocol.handshake(peer, null);
}
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());
logger.warn(message);
if (logger.isDebugEnabled()) {
logger.warn("", e);
}
}
try {
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());
logger.warn(message);
if (logger.isDebugEnabled()) {
logger.warn("", e);
}
}
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) {
logger.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(), 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 String destinationUri = "nifi://" + hostname + ":" + port;
CommunicationsSession commsSession = null;
try {
if ( siteToSiteSecure ) {
if ( sslContext == null ) {
throw new IOException("Unable to communicate with " + hostname + ":" + port + " because it requires Secure Site-to-Site communications, but this instance is not configured for secure communications");
}
final SSLSocketChannel socketChannel = new SSLSocketChannel(sslContext, hostname, port, true);
socketChannel.connect();
commsSession = new SSLSocketChannelCommunicationsSession(socketChannel, destinationUri);
try {
commsSession.setUserDn(socketChannel.getDn());
} catch (final CertificateNotYetValidException | CertificateExpiredException ex) {
throw new IOException(ex);
}
} else {
final SocketChannel socketChannel = SocketChannel.open(new InetSocketAddress(hostname, port));
commsSession = new SocketChannelCommunicationsSession(socketChannel, destinationUri);
}
commsSession.getOutput().getOutputStream().write(CommunicationsSession.MAGIC_BYTES);
commsSession.setUri(destinationUri);
} catch (final IOException ioe) {
if ( commsSession != null ) {
commsSession.close();
}
throw ioe;
}
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.RECEIVE) ? (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.getHostname(), 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 / (double) 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<>();
EndpointConnection connection;
while ((connection = connectionQueue.poll()) != null) {
// If the socket has not been used in 10 seconds, shut it down.
final long lastUsed = connection.getLastTimeUsed();
if ( lastUsed < System.currentTimeMillis() - idleExpirationMillis ) {
try {
connection.getSocketClientProtocol().shutdown(connection.getPeer());
} catch (final Exception e) {
logger.debug("Failed to shut down {} using {} due to {}",
new Object[] {connection.getSocketClientProtocol(), connection.getPeer(), e} );
}
terminate(connection);
} else {
connections.add(connection);
}
}
connectionQueue.addAll(connections);
}
}
public void shutdown() {
shutdown = true;
taskExecutor.shutdown();
peerTimeoutExpirations.clear();
for ( final EndpointConnection conn : activeConnections ) {
conn.getPeer().getCommunicationsSession().interrupt();
}
for ( final BlockingQueue<EndpointConnection> connectionQueue : connectionQueueMap.values() ) {
EndpointConnection state;
while ( (state = connectionQueue.poll()) != null) {
cleanup(state.getSocketClientProtocol(), state.getPeer());
}
}
}
public void terminate(final EndpointConnection connection) {
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) {
logger.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 + "]";
}
/**
* Returns {@code true} if the remote instance is configured for secure site-to-site communications,
* {@code false} otherwise.
*
* @return
* @throws IOException
*/
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;
private final String identifier;
public IdEnrichedRemoteDestination(final RemoteDestination original, final String identifier) {
this.original = original;
this.identifier = identifier;
}
@Override
public String getIdentifier() {
return identifier;
}
@Override
public String getName() {
return original.getName();
}
@Override
public long getYieldPeriod(final TimeUnit timeUnit) {
return original.getYieldPeriod(timeUnit);
}
@Override
public boolean isUseCompression() {
return original.isUseCompression();
}
}
}

View File

@ -0,0 +1,218 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF 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.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.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 {
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;
private final long penalizationNanos;
private volatile String portIdentifier;
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());
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();
}
private String getPortIdentifier(final TransferDirection direction) throws IOException {
final String id = this.portIdentifier;
if ( id != null ) {
return id;
}
final String portId;
if ( direction == TransferDirection.SEND ) {
portId = pool.getInputPortIdentifier(this.portName);
} else {
portId = pool.getOutputPortIdentifier(this.portName);
}
if (portId == null) {
logger.debug("Unable to resolve port [{}] to an identifier", portName);
} else {
logger.debug("Resolved port [{}] to identifier [{}]", portName, portId);
}
return portId;
}
private RemoteDestination createRemoteDestination(final String portId, final String portName) {
return new RemoteDestination() {
@Override
public String getIdentifier() {
return portId;
}
@Override
public String getName() {
return portName;
}
@Override
public long getYieldPeriod(final TimeUnit timeUnit) {
return timeUnit.convert(penalizationNanos, TimeUnit.NANOSECONDS);
}
@Override
public boolean isUseCompression() {
return compress;
}
};
}
@Override
public Transaction createTransaction(final TransferDirection direction) throws IOException {
if ( closed ) {
throw new IllegalStateException("Client is closed");
}
final String portId = getPortIdentifier(direction);
if ( portId == null ) {
throw new IOException("Could not find Port with name '" + portName + "' for remote NiFi instance");
}
final EndpointConnection connectionState = pool.getEndpointConnection(direction, getConfig());
if ( connectionState == null ) {
return null;
}
final Transaction transaction = connectionState.getSocketClientProtocol().startTransaction(
connectionState.getPeer(), connectionState.getCodec(), direction);
// 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);
return new Transaction() {
@Override
public void confirm() throws IOException {
transaction.confirm();
}
@Override
public TransactionCompletion complete() throws IOException {
try {
return transaction.complete();
} finally {
final EndpointConnection state = connectionStateRef.get();
if ( state != null ) {
pool.offer(connectionState);
connectionStateRef.set(null);
}
}
}
@Override
public void cancel(final String explanation) throws IOException {
try {
transaction.cancel(explanation);
} finally {
final EndpointConnection state = connectionStateRef.get();
if ( state != null ) {
pool.terminate(connectionState);
connectionStateRef.set(null);
}
}
}
@Override
public void error() {
try {
transaction.error();
} finally {
final EndpointConnection state = connectionStateRef.get();
if ( state != null ) {
pool.terminate(connectionState);
connectionStateRef.set(null);
}
}
}
@Override
public void send(final DataPacket dataPacket) throws IOException {
transaction.send(dataPacket);
}
@Override
public void send(final byte[] content, final Map<String, String> attributes) throws IOException {
transaction.send(content, attributes);
}
@Override
public DataPacket receive() throws IOException {
return transaction.receive();
}
@Override
public TransactionState getState() throws IOException {
return transaction.getState();
}
@Override
public Communicant getCommunicant() {
return transaction.getCommunicant();
}
};
}
@Override
public void close() throws IOException {
closed = true;
pool.shutdown();
}
}

View File

@ -14,7 +14,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.cluster;
package org.apache.nifi.remote.cluster;
public class AdaptedNodeInformation {

View File

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

View File

@ -14,7 +14,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.cluster;
package org.apache.nifi.remote.cluster;
public class NodeInformation {

View File

@ -14,10 +14,12 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.cluster;
package org.apache.nifi.remote.cluster;
import javax.xml.bind.annotation.adapters.XmlAdapter;
import org.apache.nifi.remote.cluster.NodeInformation;
public class NodeInformationAdapter extends XmlAdapter<AdaptedNodeInformation, NodeInformation> {
@Override

View File

@ -21,11 +21,10 @@ import java.io.InputStream;
import java.io.OutputStream;
import java.util.List;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.remote.VersionedRemoteResource;
import org.apache.nifi.remote.exception.ProtocolException;
import org.apache.nifi.remote.exception.TransmissionDisabledException;
import org.apache.nifi.remote.protocol.DataPacket;
/**
* <p>
@ -44,36 +43,29 @@ public interface FlowFileCodec extends VersionedRemoteResource {
public List<Integer> getSupportedVersions();
/**
* Encodes a FlowFile and its content as a single stream of data and writes
* that stream to the output. If checksum is not null, it will be calculated
* as the stream is read
* Encodes a DataPacket and its content as a single stream of data and writes
* that stream to the output.
*
* @param flowFile the FlowFile to encode
* @param session a session that can be used to transactionally create and
* transfer flow files
* @param dataPacket the data to serialize
* @param outStream the stream to write the data to
*
* @return the updated FlowFile
*
* @throws IOException
* @throws IOException if there is a communications issue
* @throws TransmissionDisabledException if a user terminates the connection
*/
FlowFile encode(FlowFile flowFile, ProcessSession session, OutputStream outStream) throws IOException, TransmissionDisabledException;
void encode(DataPacket dataPacket, OutputStream outStream) throws IOException, TransmissionDisabledException;
/**
* Decodes the contents of the InputStream, interpreting the data to
* determine the next FlowFile's attributes and content, as well as their
* destinations. If not null, checksum will be used to calculate the
* checksum as the data is read.
* determine the next DataPacket's attributes and content.
*
* @param stream an InputStream containing FlowFiles' contents, attributes,
* and destinations
* @param session
* @param stream an InputStream containing DataPacket's content and attributes
*
* @return the FlowFile that was created, or <code>null</code> if the stream
* @return the DataPacket that was created, or <code>null</code> if the stream
* was out of data
*
* @throws IOException
* @throws ProtocolException if the input is malformed
* @throws TransmissionDisabledException if a user terminates the connection
*/
FlowFile decode(InputStream stream, ProcessSession session) throws IOException, ProtocolException, TransmissionDisabledException;
DataPacket decode(InputStream stream) throws IOException, ProtocolException, TransmissionDisabledException;
}

View File

@ -26,14 +26,12 @@ import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.stream.io.StreamUtils;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.io.InputStreamCallback;
import org.apache.nifi.processor.io.OutputStreamCallback;
import org.apache.nifi.remote.StandardVersionNegotiator;
import org.apache.nifi.remote.VersionNegotiator;
import org.apache.nifi.remote.exception.ProtocolException;
import org.apache.nifi.remote.protocol.DataPacket;
import org.apache.nifi.remote.util.StandardDataPacket;
import org.apache.nifi.stream.io.StreamUtils;
public class StandardFlowFileCodec implements FlowFileCodec {
public static final int MAX_NUM_ATTRIBUTES = 25000;
@ -47,37 +45,26 @@ public class StandardFlowFileCodec implements FlowFileCodec {
}
@Override
public FlowFile encode(final FlowFile flowFile, final ProcessSession session, final OutputStream encodedOut) throws IOException {
public void encode(final DataPacket dataPacket, final OutputStream encodedOut) throws IOException {
final DataOutputStream out = new DataOutputStream(encodedOut);
final Map<String, String> attributes = flowFile.getAttributes();
final Map<String, String> attributes = dataPacket.getAttributes();
out.writeInt(attributes.size());
for ( final Map.Entry<String, String> entry : attributes.entrySet() ) {
writeString(entry.getKey(), out);
writeString(entry.getValue(), out);
}
out.writeLong(flowFile.getSize());
session.read(flowFile, new InputStreamCallback() {
@Override
public void process(final InputStream in) throws IOException {
final byte[] buffer = new byte[8192];
int len;
while ( (len = in.read(buffer)) > 0 ) {
encodedOut.write(buffer, 0, len);
}
out.writeLong(dataPacket.getSize());
final InputStream in = dataPacket.getData();
StreamUtils.copy(in, encodedOut);
encodedOut.flush();
}
});
return flowFile;
}
@Override
public FlowFile decode(final InputStream stream, final ProcessSession session) throws IOException, ProtocolException {
public DataPacket decode(final InputStream stream) throws IOException, ProtocolException {
final DataInputStream in = new DataInputStream(stream);
final int numAttributes;
@ -94,7 +81,6 @@ public class StandardFlowFileCodec implements FlowFileCodec {
throw new ProtocolException("FlowFile exceeds maximum number of attributes with a total of " + numAttributes);
}
try {
final Map<String, String> attributes = new HashMap<>(numAttributes);
for (int i=0; i < numAttributes; i++) {
final String attrName = readString(in);
@ -104,33 +90,7 @@ public class StandardFlowFileCodec implements FlowFileCodec {
final long numBytes = in.readLong();
FlowFile flowFile = session.create();
flowFile = session.putAllAttributes(flowFile, attributes);
flowFile = session.write(flowFile, new OutputStreamCallback() {
@Override
public void process(final OutputStream out) throws IOException {
int len;
long size = 0;
final byte[] buffer = new byte[8192];
while ( size < numBytes && (len = in.read(buffer, 0, (int) Math.min(buffer.length, numBytes - size))) > 0 ) {
out.write(buffer, 0, len);
size += len;
}
if ( size != numBytes ) {
throw new EOFException("Expected " + numBytes + " bytes but received only " + size);
}
}
});
return flowFile;
} catch (final EOFException e) {
session.rollback();
// we throw the general IOException here because we did not expect to hit EOFException
throw e;
}
return new StandardDataPacket(attributes, stream, numBytes);
}
private void writeString(final String val, final DataOutputStream out) throws IOException {

View File

@ -16,8 +16,15 @@
*/
package org.apache.nifi.remote.exception;
public class HandshakeException extends Exception {
import java.io.IOException;
/**
* A HandshakeException occurs when the client and the remote NiFi instance do not agree
* on some condition during the handshake. For example, if the NiFi instance does not recognize
* one of the parameters that the client passes during the Handshaking phase.
*/
public class HandshakeException extends IOException {
private static final long serialVersionUID = 178192341908726L;
public HandshakeException(final String message) {

View File

@ -16,8 +16,12 @@
*/
package org.apache.nifi.remote.exception;
public class PortNotRunningException extends Exception {
/**
* PortNotRunningException occurs when the remote NiFi instance reports
* that the Port that the client is attempting to communicate with is not
* currently running and therefore communications with that Port are not allowed.
*/
public class PortNotRunningException extends ProtocolException {
private static final long serialVersionUID = -2790940982005516375L;
public PortNotRunningException(final String message) {

View File

@ -16,7 +16,13 @@
*/
package org.apache.nifi.remote.exception;
public class ProtocolException extends Exception {
import java.io.IOException;
/**
* A ProtocolException occurs when unexpected data is received, for example
* an invalid Response Code.
*/
public class ProtocolException extends IOException {
private static final long serialVersionUID = 5763900324505818495L;

View File

@ -16,8 +16,11 @@
*/
package org.apache.nifi.remote.exception;
public class UnknownPortException extends Exception {
/**
* An UnknownPortException indicates that the remote NiFi instance has reported that
* the endpoint that the client attempted to communicate with does not exist.
*/
public class UnknownPortException extends ProtocolException {
private static final long serialVersionUID = -2790940982005516375L;
public UnknownPortException(final String message) {

View File

@ -63,4 +63,9 @@ public class SocketChannelInput implements CommunicationsInput {
public void interrupt() {
interruptableIn.interrupt();
}
@Override
public void consume() throws IOException {
socketIn.consume();
}
}

View File

@ -47,4 +47,9 @@ public class SSLSocketChannelInput implements CommunicationsInput {
public long getBytesRead() {
return countingIn.getBytesRead();
}
@Override
public void consume() throws IOException {
in.consume();
}
}

View File

@ -23,6 +23,8 @@ 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;
import org.apache.nifi.remote.TransferDirection;
import org.apache.nifi.remote.VersionedRemoteResource;
import org.apache.nifi.remote.codec.FlowFileCodec;
import org.apache.nifi.remote.exception.HandshakeException;
@ -38,14 +40,20 @@ public interface ClientProtocol extends VersionedRemoteResource {
FlowFileCodec negotiateCodec(Peer peer) throws IOException, ProtocolException;
void receiveFlowFiles(Peer peer, ProcessContext context, ProcessSession session, FlowFileCodec codec) throws IOException, ProtocolException;
int receiveFlowFiles(Peer peer, ProcessContext context, ProcessSession session, FlowFileCodec codec) throws IOException, ProtocolException;
void transferFlowFiles(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;
/**
* returns <code>true</code> if remote instance indicates that the port is
* invalid

View File

@ -21,6 +21,12 @@ import java.io.InputStream;
public interface CommunicationsInput {
/**
* Reads all data currently on the socket and throws it away
* @throws IOException
*/
void consume() throws IOException;
InputStream getInputStream() throws IOException;
long getBytesRead();

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.protocol;
import java.io.InputStream;
import java.util.Map;
/**
* Represents a piece of data that is to be sent to or that was received from a NiFi instance.
*/
public interface DataPacket {
/**
* The key-value attributes that are to be associated with the data
* @return
*/
Map<String, String> getAttributes();
/**
* An InputStream from which the content can be read
* @return
*/
InputStream getData();
/**
* The length of the InputStream.
* @return
*/
long getSize();
}

View File

@ -0,0 +1,61 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.remote.protocol.socket;
/**
* Enumeration of Properties that can be used for the Site-to-Site Socket Protocol.
*/
public enum HandshakeProperty {
/**
* Boolean value indicating whether or not the contents of a FlowFile should be
* GZipped when transferred.
*/
GZIP,
/**
* The unique identifier of the port to communicate with
*/
PORT_IDENTIFIER,
/**
* Indicates the number of milliseconds after the request was made that the client
* will wait for a response. If no response has been received by the time this value
* expires, the server can move on without attempting to service the request because
* the client will have already disconnected.
*/
REQUEST_EXPIRATION_MILLIS,
/**
* The preferred number of FlowFiles that the server should send to the client
* when pulling data. This property was introduced in version 5 of the protocol.
*/
BATCH_COUNT,
/**
* The preferred number of bytes that the server should send to the client when
* pulling data. This property was introduced in version 5 of the protocol.
*/
BATCH_SIZE,
/**
* The preferred amount of time that the server should send data to the client
* when pulling data. This property was introduced in version 5 of the protocol.
* Value is in milliseconds.
*/
BATCH_DURATION;
}

View File

@ -40,6 +40,7 @@ public enum ResponseCode {
CONFIRM_TRANSACTION(12, "Confirm Transaction", true), // "Explanation" of this code is the checksum
TRANSACTION_FINISHED(13, "Transaction Finished", false),
TRANSACTION_FINISHED_BUT_DESTINATION_FULL(14, "Transaction Finished But Destination is Full", false),
CANCEL_TRANSACTION(15, "Cancel Transaction", true),
BAD_CHECKSUM(19, "Bad Checksum", false),
// data availability indicators

View File

@ -0,0 +1,438 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF 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 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.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;
import org.apache.nifi.remote.RemoteDestination;
import org.apache.nifi.remote.RemoteResourceInitiator;
import org.apache.nifi.remote.StandardVersionNegotiator;
import org.apache.nifi.remote.Transaction;
import org.apache.nifi.remote.TransferDirection;
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.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.RequestType;
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;
public class SocketClientProtocol implements ClientProtocol {
private final VersionNegotiator versionNegotiator = new StandardVersionNegotiator(5, 4, 3, 2, 1);
private RemoteDestination destination;
private boolean useCompression = false;
private String commsIdentifier;
private boolean handshakeComplete = false;
private final Logger logger = LoggerFactory.getLogger(SocketClientProtocol.class);
private Response handshakeResponse = null;
private boolean readyForFileTransfer = false;
private String transitUriPrefix = null;
private int timeoutMillis = 30000;
private int batchCount;
private long batchSize;
private long batchMillis;
private static final long BATCH_SEND_NANOS = TimeUnit.SECONDS.toNanos(5L); // send batches of up to 5 seconds
public SocketClientProtocol() {
}
public void setPreferredBatchCount(final int count) {
this.batchCount = count;
}
public void setPreferredBatchSize(final long bytes) {
this.batchSize = bytes;
}
public void setPreferredBatchDuration(final long millis) {
this.batchMillis = millis;
}
public void setDestination(final RemoteDestination destination) {
this.destination = destination;
this.useCompression = destination.isUseCompression();
}
public void setTimeout(final int timeoutMillis) {
this.timeoutMillis = timeoutMillis;
}
@Override
public void handshake(final Peer peer) throws IOException, HandshakeException {
handshake(peer, destination.getIdentifier());
}
public void handshake(final Peer peer, final String destinationId) throws IOException, HandshakeException {
if ( handshakeComplete ) {
throw new IllegalStateException("Handshake has already been completed");
}
commsIdentifier = UUID.randomUUID().toString();
logger.debug("{} handshaking with {}", this, peer);
final Map<HandshakeProperty, String> properties = new HashMap<>();
properties.put(HandshakeProperty.GZIP, String.valueOf(useCompression));
if ( destinationId != null ) {
properties.put(HandshakeProperty.PORT_IDENTIFIER, destinationId);
}
properties.put(HandshakeProperty.REQUEST_EXPIRATION_MILLIS, String.valueOf(timeoutMillis) );
if ( versionNegotiator.getVersion() >= 5 ) {
if ( batchCount > 0 ) {
properties.put(HandshakeProperty.BATCH_COUNT, String.valueOf(batchCount));
}
if ( batchSize > 0L ) {
properties.put(HandshakeProperty.BATCH_SIZE, String.valueOf(batchSize));
}
if ( batchMillis > 0L ) {
properties.put(HandshakeProperty.BATCH_DURATION, String.valueOf(batchMillis));
}
}
final CommunicationsSession commsSession = peer.getCommunicationsSession();
commsSession.setTimeout(timeoutMillis);
final DataInputStream dis = new DataInputStream(commsSession.getInput().getInputStream());
final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream());
dos.writeUTF(commsIdentifier);
if ( versionNegotiator.getVersion() >= 3 ) {
dos.writeUTF(peer.getUrl());
transitUriPrefix = peer.getUrl();
if ( !transitUriPrefix.endsWith("/") ) {
transitUriPrefix = transitUriPrefix + "/";
}
}
logger.debug("Handshaking with properties {}", properties);
dos.writeInt(properties.size());
for ( final Map.Entry<HandshakeProperty, String> entry : properties.entrySet() ) {
dos.writeUTF(entry.getKey().name());
dos.writeUTF(entry.getValue());
}
dos.flush();
try {
handshakeResponse = Response.read(dis);
} catch (final ProtocolException e) {
throw new HandshakeException(e);
}
switch (handshakeResponse.getCode()) {
case PORT_NOT_IN_VALID_STATE:
case UNKNOWN_PORT:
case PORTS_DESTINATION_FULL:
break;
case PROPERTIES_OK:
readyForFileTransfer = true;
break;
default:
logger.error("{} received unexpected response {} from {} when negotiating Codec", new Object[] {
this, handshakeResponse, peer});
peer.close();
throw new HandshakeException("Received unexpected response " + handshakeResponse);
}
logger.debug("{} Finished handshake with {}", this, peer);
handshakeComplete = true;
}
public boolean isReadyForFileTransfer() {
return readyForFileTransfer;
}
public boolean isPortInvalid() {
if ( !handshakeComplete ) {
throw new IllegalStateException("Handshake has not completed successfully");
}
return handshakeResponse.getCode() == ResponseCode.PORT_NOT_IN_VALID_STATE;
}
public boolean isPortUnknown() {
if ( !handshakeComplete ) {
throw new IllegalStateException("Handshake has not completed successfully");
}
return handshakeResponse.getCode() == ResponseCode.UNKNOWN_PORT;
}
public boolean isDestinationFull() {
if ( !handshakeComplete ) {
throw new IllegalStateException("Handshake has not completed successfully");
}
return handshakeResponse.getCode() == ResponseCode.PORTS_DESTINATION_FULL;
}
@Override
public Set<PeerStatus> getPeerStatuses(final Peer peer) throws IOException {
if ( !handshakeComplete ) {
throw new IllegalStateException("Handshake has not been performed");
}
logger.debug("{} Get Peer Statuses from {}", this, peer);
final CommunicationsSession commsSession = peer.getCommunicationsSession();
final DataInputStream dis = new DataInputStream(commsSession.getInput().getInputStream());
final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream());
RequestType.REQUEST_PEER_LIST.writeRequestType(dos);
dos.flush();
final int numPeers = dis.readInt();
final Set<PeerStatus> peers = new HashSet<>(numPeers);
for (int i=0; i < numPeers; i++) {
final String hostname = dis.readUTF();
final int port = dis.readInt();
final boolean secure = dis.readBoolean();
final int flowFileCount = dis.readInt();
peers.add(new PeerStatus(new PeerDescription(hostname, port, secure), flowFileCount));
}
logger.debug("{} Received {} Peer Statuses from {}", this, peers.size(), peer);
return peers;
}
@Override
public FlowFileCodec negotiateCodec(final Peer peer) throws IOException, ProtocolException {
if ( !handshakeComplete ) {
throw new IllegalStateException("Handshake has not been performed");
}
logger.debug("{} Negotiating Codec with {}", this, peer);
final CommunicationsSession commsSession = peer.getCommunicationsSession();
final DataInputStream dis = new DataInputStream(commsSession.getInput().getInputStream());
final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream());
RequestType.NEGOTIATE_FLOWFILE_CODEC.writeRequestType(dos);
FlowFileCodec codec = new StandardFlowFileCodec();
try {
codec = (FlowFileCodec) RemoteResourceInitiator.initiateResourceNegotiation(codec, dis, dos);
} catch (HandshakeException e) {
throw new ProtocolException(e.toString());
}
logger.debug("{} negotiated FlowFileCodec {} with {}", new Object[] {this, codec, commsSession});
return codec;
}
@Override
public Transaction startTransaction(final Peer peer, final FlowFileCodec codec, final TransferDirection direction) throws IOException, ProtocolException {
if ( !handshakeComplete ) {
throw new IllegalStateException("Handshake has not been performed");
}
if ( !readyForFileTransfer ) {
throw new IllegalStateException("Cannot start transaction; handshake resolution was " + handshakeResponse);
}
return new SocketClientTransaction(versionNegotiator.getVersion(), destination.getIdentifier(), peer, codec,
direction, useCompression, (int) destination.getYieldPeriod(TimeUnit.MILLISECONDS));
}
@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;
}
@Override
public void shutdown(final Peer peer) throws IOException {
readyForFileTransfer = false;
final CommunicationsSession commsSession = peer.getCommunicationsSession();
final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream());
logger.debug("{} Shutting down with {}", this, peer);
// Indicate that we would like to have some data
RequestType.SHUTDOWN.writeRequestType(dos);
dos.flush();
}
@Override
public String getResourceName() {
return "SocketFlowFileProtocol";
}
@Override
public String toString() {
return "SocketClientProtocol[CommsID=" + commsIdentifier + "]";
}
}

View File

@ -0,0 +1,399 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF 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 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.remote.Communicant;
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.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class SocketClientTransaction implements Transaction {
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 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) throws IOException {
this.protocolVersion = protocolVersion;
this.destinationId = destinationId;
this.peer = peer;
this.codec = codec;
this.direction = direction;
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;
initialize();
}
private void initialize() throws IOException {
try {
if ( direction == TransferDirection.RECEIVE ) {
// Indicate that we would like to have some data
RequestType.RECEIVE_FLOWFILES.writeRequestType(dos);
dos.flush();
final Response dataAvailableCode = Response.read(dis);
switch (dataAvailableCode.getCode()) {
case MORE_DATA:
logger.debug("{} {} Indicates that data is available", this, peer);
this.dataAvailable = true;
break;
case NO_MORE_DATA:
logger.debug("{} No data available from {}", peer);
this.dataAvailable = false;
return;
default:
throw new ProtocolException("Got unexpected response when asking for data: " + dataAvailableCode);
}
} else {
// Indicate that we would like to have some data
RequestType.SEND_FLOWFILES.writeRequestType(dos);
dos.flush();
}
} catch (final Exception e) {
error();
throw e;
}
}
@Override
public DataPacket receive() throws IOException {
try {
try {
if ( state != TransactionState.DATA_EXCHANGED && state != TransactionState.TRANSACTION_STARTED) {
throw new IllegalStateException("Cannot receive data because Transaction State is " + state);
}
if ( direction == TransferDirection.SEND ) {
throw new IllegalStateException("Attempting to receive data 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 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;
} 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;
}
}
@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 because Transaction State is " + state);
}
if ( direction == TransferDirection.RECEIVE ) {
throw new IllegalStateException("Attempting to send data 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 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, 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 expected confirmation of transaction", peer);
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,57 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.remote.protocol.socket;
import java.util.concurrent.TimeUnit;
import org.apache.nifi.remote.TransactionCompletion;
public class SocketClientTransactionCompletion 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) {
this.backoff = backoff;
this.dataPacketsTransferred = dataPacketsTransferred;
this.bytesTransferred = bytesTransferred;
this.durationNanos = durationNanos;
}
@Override
public boolean isBackoff() {
return backoff;
}
@Override
public int getDataPacketsTransferred() {
return dataPacketsTransferred;
}
@Override
public long getBytesTransferred() {
return bytesTransferred;
}
@Override
public long getDuration(final TimeUnit timeUnit) {
return timeUnit.convert(durationNanos, TimeUnit.NANOSECONDS);
}
}

View File

@ -0,0 +1,98 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF 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,43 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF 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.util.Set;
import org.apache.nifi.remote.PeerStatus;
public class PeerStatusCache {
private final Set<PeerStatus> statuses;
private final long timestamp;
public PeerStatusCache(final Set<PeerStatus> statuses) {
this(statuses, System.currentTimeMillis());
}
public PeerStatusCache(final Set<PeerStatus> statuses, final long timestamp) {
this.statuses = statuses;
this.timestamp = timestamp;
}
public Set<PeerStatus> getStatuses() {
return statuses;
}
public long getTimestamp() {
return timestamp;
}
}

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 java.io.InputStream;
import java.util.Map;
import org.apache.nifi.remote.protocol.DataPacket;
import org.apache.nifi.stream.io.LimitingInputStream;
import org.apache.nifi.stream.io.MinimumLengthInputStream;
public class StandardDataPacket implements DataPacket {
private final Map<String, String> attributes;
private final InputStream stream;
private final long size;
public StandardDataPacket(final Map<String, String> attributes, final InputStream stream, final long size) {
this.attributes = attributes;
this.stream = new MinimumLengthInputStream(new LimitingInputStream(stream, size), size);
this.size = size;
}
public Map<String, String> getAttributes() {
return attributes;
}
public InputStream getData() {
return stream;
}
public long getSize() {
return size;
}
}

View File

@ -14,21 +14,19 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.remote;
package org.apache.nifi.remote.client.socket;
import org.apache.nifi.remote.StandardRemoteGroupPort;
import org.apache.nifi.remote.PeerStatus;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import org.apache.nifi.cluster.ClusterNodeInformation;
import org.apache.nifi.cluster.NodeInformation;
import org.apache.nifi.connectable.ConnectableType;
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 TestStandardRemoteGroupPort {
public class TestEndpointConnectionStatePool {
@Test
public void testFormulateDestinationListForOutput() throws IOException {
@ -41,9 +39,9 @@ public class TestStandardRemoteGroupPort {
collection.add(new NodeInformation("ShouldGetMedium", 5, 5555, true, 4096));
clusterNodeInfo.setNodeInformation(collection);
final List<PeerStatus> destinations = StandardRemoteGroupPort.formulateDestinationList(clusterNodeInfo, ConnectableType.REMOTE_OUTPUT_PORT);
final List<PeerStatus> destinations = EndpointConnectionPool.formulateDestinationList(clusterNodeInfo, TransferDirection.SEND);
for ( final PeerStatus peerStatus : destinations ) {
System.out.println(peerStatus.getHostname() + ":" + peerStatus.getPort());
System.out.println(peerStatus.getPeerDescription());
}
}
@ -55,9 +53,9 @@ public class TestStandardRemoteGroupPort {
collection.add(new NodeInformation("ShouldGetLots", 2, 2222, true, 50000));
clusterNodeInfo.setNodeInformation(collection);
final List<PeerStatus> destinations = StandardRemoteGroupPort.formulateDestinationList(clusterNodeInfo, ConnectableType.REMOTE_OUTPUT_PORT);
final List<PeerStatus> destinations = EndpointConnectionPool.formulateDestinationList(clusterNodeInfo, TransferDirection.SEND);
for ( final PeerStatus peerStatus : destinations ) {
System.out.println(peerStatus.getHostname() + ":" + peerStatus.getPort());
System.out.println(peerStatus.getPeerDescription());
}
}
@ -75,9 +73,9 @@ public class TestStandardRemoteGroupPort {
collection.add(new NodeInformation("ShouldGetMedium", 5, 5555, true, 4096));
clusterNodeInfo.setNodeInformation(collection);
final List<PeerStatus> destinations = StandardRemoteGroupPort.formulateDestinationList(clusterNodeInfo, ConnectableType.REMOTE_INPUT_PORT);
final List<PeerStatus> destinations = EndpointConnectionPool.formulateDestinationList(clusterNodeInfo, TransferDirection.SEND);
for ( final PeerStatus peerStatus : destinations ) {
System.out.println(peerStatus.getHostname() + ":" + peerStatus.getPort());
System.out.println(peerStatus.getPeerDescription());
}
}
@ -89,9 +87,9 @@ public class TestStandardRemoteGroupPort {
collection.add(new NodeInformation("ShouldGetLittle", 2, 2222, true, 50000));
clusterNodeInfo.setNodeInformation(collection);
final List<PeerStatus> destinations = StandardRemoteGroupPort.formulateDestinationList(clusterNodeInfo, ConnectableType.REMOTE_INPUT_PORT);
final List<PeerStatus> destinations = EndpointConnectionPool.formulateDestinationList(clusterNodeInfo, TransferDirection.SEND);
for ( final PeerStatus peerStatus : destinations ) {
System.out.println(peerStatus.getHostname() + ":" + peerStatus.getPort());
System.out.println(peerStatus.getPeerDescription());
}
}
}

View File

@ -0,0 +1,105 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF 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.ByteArrayInputStream;
import java.io.IOException;
import java.io.InputStream;
import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import org.apache.nifi.remote.Transaction;
import org.apache.nifi.remote.TransferDirection;
import org.apache.nifi.remote.client.SiteToSiteClient;
import org.apache.nifi.remote.protocol.DataPacket;
import org.apache.nifi.remote.util.StandardDataPacket;
import org.apache.nifi.stream.io.StreamUtils;
import org.junit.Assert;
import org.junit.Ignore;
import org.junit.Test;
public class TestSiteToSiteClient {
@Test
@Ignore("For local testing only; not really a unit test but a manual test")
public void testReceive() throws IOException {
System.setProperty("org.slf4j.simpleLogger.log.org.apache.nifi.remote", "DEBUG");
final SiteToSiteClient client = new SiteToSiteClient.Builder()
.url("http://localhost:8080/nifi")
.portName("cba")
.requestBatchCount(10)
.build();
try {
for (int i=0; i < 1000; i++) {
final Transaction transaction = client.createTransaction(TransferDirection.RECEIVE);
Assert.assertNotNull(transaction);
DataPacket packet;
while (true) {
packet = transaction.receive();
if ( packet == null ) {
break;
}
final InputStream in = packet.getData();
final long size = packet.getSize();
final byte[] buff = new byte[(int) size];
StreamUtils.fillBuffer(in, buff);
}
transaction.confirm();
transaction.complete();
}
} finally {
client.close();
}
}
@Test
@Ignore("For local testing only; not really a unit test but a manual test")
public void testSend() throws IOException {
System.setProperty("org.slf4j.simpleLogger.log.org.apache.nifi.remote", "DEBUG");
final SiteToSiteClient client = new SiteToSiteClient.Builder()
.url("http://localhost:8080/nifi")
.portName("input")
.build();
try {
final Transaction transaction = client.createTransaction(TransferDirection.SEND);
Assert.assertNotNull(transaction);
final Map<String, String> attrs = new HashMap<>();
attrs.put("site-to-site", "yes, please!");
final byte[] bytes = "Hello".getBytes();
final ByteArrayInputStream bais = new ByteArrayInputStream(bytes);
final DataPacket packet = new StandardDataPacket(attrs, bais, bytes.length);
transaction.send(packet);
transaction.confirm();
transaction.complete();
} finally {
client.close();
}
}
}

View File

@ -142,7 +142,7 @@ public class CompressionInputStream extends InputStream {
return -1;
}
return buffer[bufferIndex++];
return buffer[bufferIndex++] & 0xFF;
}
@Override

View File

@ -44,6 +44,16 @@ public class SocketChannelInputStream extends InputStream {
this.timeoutMillis = timeoutMillis;
}
public void consume() throws IOException {
final byte[] b = new byte[4096];
final ByteBuffer buffer = ByteBuffer.wrap(b);
int bytesRead;
do {
bytesRead = channel.read(buffer);
buffer.flip();
} while ( bytesRead > 0 );
}
@Override
public int read() throws IOException {
if (bufferedByte != null) {

View File

@ -258,6 +258,16 @@ public class SSLSocketChannel implements Closeable {
}
}
public void consume() throws IOException {
final byte[] b = new byte[4096];
final ByteBuffer buffer = ByteBuffer.wrap(b);
int readCount;
do {
readCount = channel.read(buffer);
buffer.flip();
} while (readCount > 0);
}
private int readData(final ByteBuffer dest) throws IOException {
final long startTime = System.currentTimeMillis();

View File

@ -27,6 +27,10 @@ public class SSLSocketChannelInputStream extends InputStream {
this.channel = channel;
}
public void consume() throws IOException {
channel.consume();
}
@Override
public int read() throws IOException {
return channel.read();

View File

@ -0,0 +1,111 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.stream.io;
import java.io.IOException;
import java.io.InputStream;
public class LimitingInputStream extends InputStream {
private final InputStream in;
private final long limit;
private long bytesRead = 0;
public LimitingInputStream(final InputStream in, final long limit) {
this.in = in;
this.limit = limit;
}
@Override
public int read() throws IOException {
if (bytesRead >= limit) {
return -1;
}
final int val = in.read();
if (val > -1) {
bytesRead++;
}
return val;
}
@Override
public int read(final byte[] b) throws IOException {
if (bytesRead >= limit) {
return -1;
}
final int maxToRead = (int) Math.min(b.length, limit - bytesRead);
final int val = in.read(b, 0, maxToRead);
if (val > 0) {
bytesRead += val;
}
return val;
}
@Override
public int read(byte[] b, int off, int len) throws IOException {
if (bytesRead >= limit) {
return -1;
}
final int maxToRead = (int) Math.min(len, limit - bytesRead);
final int val = in.read(b, off, maxToRead);
if (val > 0) {
bytesRead += val;
}
return val;
}
@Override
public long skip(final long n) throws IOException {
final long skipped = in.skip(Math.min(n, limit - bytesRead));
bytesRead += skipped;
return skipped;
}
@Override
public int available() throws IOException {
return in.available();
}
@Override
public void close() throws IOException {
in.close();
}
@Override
public void mark(int readlimit) {
in.mark(readlimit);
}
@Override
public boolean markSupported() {
return in.markSupported();
}
@Override
public void reset() throws IOException {
in.reset();
}
public long getLimit() {
return limit;
}
}

View File

@ -0,0 +1,93 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.stream.io;
import java.io.EOFException;
import java.io.FilterInputStream;
import java.io.IOException;
import java.io.InputStream;
/**
* An InputStream that will throw EOFException if the underlying InputStream runs out of data before reaching the
* configured minimum amount of data
*/
public class MinimumLengthInputStream extends FilterInputStream {
private final long minLength;
private long consumedCount = 0L;
public MinimumLengthInputStream(final InputStream in, final long minLength) {
super(in);
this.minLength = minLength;
}
@Override
public int read() throws IOException {
final int b = super.read();
if ( b < 0 && consumedCount < minLength ) {
throw new EOFException();
}
if ( b >= 0 ) {
consumedCount++;
}
return b;
}
@Override
public int read(byte[] b) throws IOException {
return read(b, 0, b.length);
}
public int read(byte[] b, int off, int len) throws IOException {
final int num = super.read(b, off, len);
if ( num < 0 && consumedCount < minLength ) {
throw new EOFException();
}
if ( num >= 0 ) {
consumedCount += num;
}
return num;
}
@Override
public long skip(final long n) throws IOException {
long skipped = super.skip(n);
if ( skipped < 1 ) {
final int b = super.read();
if ( b >= 0 ) {
skipped = 1;
}
}
if ( skipped < 0 && consumedCount < minLength ) {
throw new EOFException();
}
if ( skipped >= 0 ) {
consumedCount += skipped;
}
return skipped;
}
}

View File

@ -35,5 +35,6 @@
<module>nifi-web-utils</module>
<module>nifi-processor-utilities</module>
<module>nifi-write-ahead-log</module>
<module>nifi-site-to-site-client</module>
</modules>
</project>

View File

@ -0,0 +1,19 @@
<!--
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements. See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to You under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License. You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
-->
# nifi-external
The nifi-external module is a location where components can be developed by the NiFi team
that are not intended to be used directly by NiFi but are to be used within other frameworks
in order to integrate with NiFi.

View File

@ -0,0 +1,37 @@
<?xml version="1.0" encoding="UTF-8"?>
<!--
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements. See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to You under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License. You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
-->
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
<modelVersion>4.0.0</modelVersion>
<parent>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-external</artifactId>
<version>0.0.2-incubating-SNAPSHOT</version>
</parent>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-spark-receiver</artifactId>
<dependencies>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-streaming_2.10</artifactId>
<version>1.2.0</version>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-site-to-site-client</artifactId>
</dependency>
</dependencies>
</project>

View File

@ -14,10 +14,27 @@
* 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.spark;
public enum HandshakeProperty {
GZIP,
PORT_IDENTIFIER,
REQUEST_EXPIRATION_MILLIS;
import java.util.Map;
/**
* <p>
* The NiFiDataPacket provides a packaging around a NiFi FlowFile. It wraps both a FlowFile's
* content and its attributes so that they can be processed by Spark
* </p>
*/
public interface NiFiDataPacket {
/**
* Returns the contents of a NiFi FlowFile
* @return
*/
byte[] getContent();
/**
* Returns a Map of attributes that are associated with the NiFi FlowFile
* @return
*/
Map<String, String> getAttributes();
}

View File

@ -0,0 +1,198 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.spark;
import java.io.IOException;
import java.io.InputStream;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import org.apache.nifi.remote.Transaction;
import org.apache.nifi.remote.TransferDirection;
import org.apache.nifi.remote.client.SiteToSiteClient;
import org.apache.nifi.remote.client.SiteToSiteClientConfig;
import org.apache.nifi.remote.protocol.DataPacket;
import org.apache.nifi.stream.io.StreamUtils;
import org.apache.spark.storage.StorageLevel;
import org.apache.spark.streaming.receiver.Receiver;
/**
* <p>
* The <code>NiFiReceiver</code> is a Reliable Receiver that provides a way to pull data
* from Apache NiFi so that it can be processed by Spark Streaming. The NiFi Receiver connects
* to NiFi instance provided in the config and requests data from
* the OutputPort that is named. In NiFi, when an OutputPort is added to the root process group,
* it acts as a queue of data for remote clients. This receiver is then able to pull that data
* from NiFi reliably.
* </p>
*
* <p>
* It is important to note that if pulling data from a NiFi cluster, the URL that should be used
* is that of the NiFi Cluster Manager. The Receiver will automatically handle determining the nodes
* in that cluster and pull from those nodes as appropriate.
* </p>
*
* <p>
* In order to use the NiFiReceiver, you will need to first build a {@link SiteToSiteClientConfig} to provide
* to the constructor. This can be achieved by using the {@link SiteToSiteClient.Builder}.
* Below is an example snippet of driver code to pull data from NiFi that is running on localhost:8080. This
* example assumes that NiFi exposes and OutputPort on the root group named "Data For Spark".
* Additionally, it assumes that the data that it will receive from this OutputPort is text
* data, as it will map the byte array received from NiFi to a UTF-8 Encoded string.
* </p>
*
* <code>
* <pre>
* Pattern SPACE = Pattern.compile(" ");
*
* // Build a Site-to-site client config
* SiteToSiteClientConfig config = new SiteToSiteClient.Builder()
* .setUrl("http://localhost:8080/nifi")
* .setPortName("Data For Spark")
* .buildConfig();
*
* SparkConf sparkConf = new SparkConf().setAppName("NiFi-Spark Streaming example");
* JavaStreamingContext ssc = new JavaStreamingContext(sparkConf, new Duration(1000L));
*
* // Create a JavaReceiverInputDStream using a NiFi receiver so that we can pull data from
* // specified Port
* JavaReceiverInputDStream<NiFiDataPacket> packetStream =
* ssc.receiverStream(new NiFiReceiver(clientConfig, StorageLevel.MEMORY_ONLY()));
*
* // Map the data from NiFi to text, ignoring the attributes
* JavaDStream<String> text = packetStream.map(new Function<NiFiDataPacket, String>() {
* public String call(final NiFiDataPacket dataPacket) throws Exception {
* return new String(dataPacket.getContent(), StandardCharsets.UTF_8);
* }
* });
*
* // Split the words by spaces
* JavaDStream<String> words = text.flatMap(new FlatMapFunction<String, String>() {
* public Iterable<String> call(final String text) throws Exception {
* return Arrays.asList(SPACE.split(text));
* }
* });
*
* // Map each word to the number 1, then aggregate by key
* JavaPairDStream<String, Integer> wordCounts = words.mapToPair(
* new PairFunction<String, String, Integer>() {
* public Tuple2<String, Integer> call(String s) {
* return new Tuple2<String, Integer>(s, 1);
* }
* }).reduceByKey(new Function2<Integer, Integer, Integer>() {
* public Integer call(Integer i1, Integer i2) {
* return i1 + i2;
* }
* }
* );
*
* // print the results
* wordCounts.print();
* ssc.start();
* ssc.awaitTermination();
* </pre>
* </code>
*/
public class NiFiReceiver extends Receiver<NiFiDataPacket> {
private static final long serialVersionUID = 3067274587595578836L;
private final SiteToSiteClientConfig clientConfig;
public NiFiReceiver(final SiteToSiteClientConfig clientConfig, final StorageLevel storageLevel) {
super(storageLevel);
this.clientConfig = clientConfig;
}
@Override
public void onStart() {
final Thread thread = new Thread(new ReceiveRunnable());
thread.setDaemon(true);
thread.setName("NiFi Receiver");
thread.start();
}
@Override
public void onStop() {
}
class ReceiveRunnable implements Runnable {
public ReceiveRunnable() {
}
public void run() {
try {
final SiteToSiteClient client = new SiteToSiteClient.Builder().fromConfig(clientConfig).build();
try {
while ( !isStopped() ) {
final Transaction transaction = client.createTransaction(TransferDirection.RECEIVE);
DataPacket dataPacket = transaction.receive();
if ( dataPacket == null ) {
transaction.confirm();
transaction.complete();
// no data available. Wait a bit and try again
try {
Thread.sleep(1000L);
} catch (InterruptedException e) {}
continue;
}
final List<NiFiDataPacket> dataPackets = new ArrayList<NiFiDataPacket>();
do {
// Read the data into a byte array and wrap it along with the attributes
// into a NiFiDataPacket.
final InputStream inStream = dataPacket.getData();
final byte[] data = new byte[(int) dataPacket.getSize()];
StreamUtils.fillBuffer(inStream, data);
final Map<String, String> attributes = dataPacket.getAttributes();
final NiFiDataPacket NiFiDataPacket = new NiFiDataPacket() {
public byte[] getContent() {
return data;
}
public Map<String, String> getAttributes() {
return attributes;
}
};
dataPackets.add(NiFiDataPacket);
dataPacket = transaction.receive();
} while ( dataPacket != null );
// Confirm transaction to verify the data
transaction.confirm();
store(dataPackets.iterator());
transaction.complete();
}
} finally {
try {
client.close();
} catch (final IOException ioe) {
reportError("Failed to close client", ioe);
}
}
} catch (final IOException ioe) {
restart("Failed to receive data from NiFi", ioe);
}
}
}
}

View File

@ -0,0 +1,29 @@
<?xml version="1.0" encoding="UTF-8"?>
<!--
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements. See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to You under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License. You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
-->
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
<modelVersion>4.0.0</modelVersion>
<parent>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi</artifactId>
<version>0.0.2-incubating-SNAPSHOT</version>
</parent>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-external</artifactId>
<packaging>pom</packaging>
<modules>
<module>nifi-spark-receiver</module>
</modules>
</project>

View File

@ -73,6 +73,10 @@
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-site-to-site</artifactId>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-site-to-site-client</artifactId>
</dependency>
<dependency>
<groupId>org.apache.commons</groupId>
<artifactId>commons-compress</artifactId>

View File

@ -16,15 +16,17 @@
*/
package org.apache.nifi.cluster.manager;
import org.apache.nifi.cluster.manager.exception.PrimaryRoleAssignmentException;
import org.apache.nifi.cluster.manager.exception.IllegalNodeDisconnectionException;
import org.apache.nifi.cluster.manager.exception.UnknownNodeException;
import org.apache.nifi.cluster.manager.exception.IneligiblePrimaryNodeException;
import org.apache.nifi.cluster.manager.exception.IllegalNodeDeletionException;
import org.apache.nifi.cluster.manager.exception.IllegalNodeReconnectionException;
import org.apache.nifi.cluster.manager.exception.NodeDisconnectionException;
import org.apache.nifi.cluster.NodeInformant;
import java.util.List;
import java.util.Set;
import org.apache.nifi.cluster.event.Event;
import org.apache.nifi.cluster.manager.exception.IllegalNodeDeletionException;
import org.apache.nifi.cluster.manager.exception.IllegalNodeDisconnectionException;
import org.apache.nifi.cluster.manager.exception.IllegalNodeReconnectionException;
import org.apache.nifi.cluster.manager.exception.IneligiblePrimaryNodeException;
import org.apache.nifi.cluster.manager.exception.NodeDisconnectionException;
import org.apache.nifi.cluster.manager.exception.PrimaryRoleAssignmentException;
import org.apache.nifi.cluster.manager.exception.UnknownNodeException;
import org.apache.nifi.cluster.node.Node;
import org.apache.nifi.cluster.node.Node.Status;
import org.apache.nifi.cluster.protocol.ConnectionRequest;
@ -33,11 +35,9 @@ import org.apache.nifi.cluster.protocol.Heartbeat;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.controller.status.ProcessGroupStatus;
import org.apache.nifi.diagnostics.SystemDiagnostics;
import org.apache.nifi.remote.cluster.NodeInformant;
import org.apache.nifi.reporting.BulletinRepository;
import java.util.List;
import java.util.Set;
/**
* Defines the interface for a ClusterManager. The cluster manager is a
* threadsafe centralized manager for a cluster. Members of a cluster are nodes.

View File

@ -63,11 +63,10 @@ import javax.xml.validation.Schema;
import javax.xml.validation.SchemaFactory;
import javax.xml.validation.Validator;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.admin.service.AuditService;
import org.apache.nifi.cluster.BulletinsPayload;
import org.apache.nifi.cluster.ClusterNodeInformation;
import org.apache.nifi.cluster.HeartbeatPayload;
import org.apache.nifi.cluster.NodeInformation;
import org.apache.nifi.cluster.context.ClusterContext;
import org.apache.nifi.cluster.context.ClusterContextImpl;
import org.apache.nifi.cluster.event.Event;
@ -117,6 +116,7 @@ import org.apache.nifi.cluster.protocol.message.NodeBulletinsMessage;
import org.apache.nifi.cluster.protocol.message.PrimaryRoleAssignmentMessage;
import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
import org.apache.nifi.cluster.protocol.message.ProtocolMessage.MessageType;
import org.apache.nifi.cluster.protocol.message.ReconnectionFailureMessage;
import org.apache.nifi.cluster.protocol.message.ReconnectionRequestMessage;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.controller.ControllerService;
@ -155,6 +155,8 @@ import org.apache.nifi.processor.StandardValidationContextFactory;
import org.apache.nifi.remote.RemoteResourceManager;
import org.apache.nifi.remote.RemoteSiteListener;
import org.apache.nifi.remote.SocketRemoteSiteListener;
import org.apache.nifi.remote.cluster.ClusterNodeInformation;
import org.apache.nifi.remote.cluster.NodeInformation;
import org.apache.nifi.remote.protocol.socket.ClusterManagerServerProtocol;
import org.apache.nifi.reporting.Bulletin;
import org.apache.nifi.reporting.BulletinRepository;
@ -191,7 +193,6 @@ import org.apache.nifi.web.api.entity.ProvenanceEventEntity;
import org.apache.nifi.web.api.entity.RemoteProcessGroupEntity;
import org.apache.nifi.web.api.entity.RemoteProcessGroupsEntity;
import org.apache.nifi.web.util.WebUtils;
import org.apache.commons.lang3.StringUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.w3c.dom.DOMException;
@ -203,8 +204,6 @@ import org.xml.sax.SAXParseException;
import com.sun.jersey.api.client.ClientResponse;
import org.apache.nifi.cluster.protocol.message.ReconnectionFailureMessage;
/**
* Provides a cluster manager implementation. The manager federates incoming
* HTTP client requests to the nodes' external API using the HTTP protocol. The

View File

@ -42,6 +42,10 @@
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-api</artifactId>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-site-to-site-client</artifactId>
</dependency>
<dependency>
<groupId>org.apache.commons</groupId>
<artifactId>commons-lang3</artifactId>

View File

@ -94,7 +94,7 @@ public class StandardFunnel implements Funnel {
position = new AtomicReference<>(new Position(0D, 0D));
scheduledState = new AtomicReference<>(ScheduledState.STOPPED);
penalizationPeriod = new AtomicReference<>("30 sec");
yieldPeriod = new AtomicReference<>("1 sec");
yieldPeriod = new AtomicReference<>("250 millis");
yieldExpiration = new AtomicLong(0L);
schedulingPeriod = new AtomicReference<>("0 millis");
schedulingNanos = new AtomicLong(30000);

View File

@ -16,7 +16,6 @@
*/
package org.apache.nifi.groups;
import java.io.IOException;
import java.net.URI;
import java.util.Date;
import java.util.Set;
@ -25,9 +24,7 @@ import java.util.concurrent.TimeUnit;
import org.apache.nifi.connectable.Position;
import org.apache.nifi.controller.exception.CommunicationsException;
import org.apache.nifi.events.EventReporter;
import org.apache.nifi.remote.PeerStatus;
import org.apache.nifi.remote.RemoteGroupPort;
import org.apache.nifi.remote.protocol.CommunicationsSession;
public interface RemoteProcessGroup {
@ -106,15 +103,6 @@ public interface RemoteProcessGroup {
*/
String getCommunicationsTimeout();
/**
* @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
*/
Integer getListeningPort() throws IOException;
/**
* Indicates whether or not the RemoteProcessGroup is currently scheduled to
* transmit data
@ -211,11 +199,6 @@ public interface RemoteProcessGroup {
*/
void removeNonExistentPort(final RemoteGroupPort port);
/**
*
* @return @throws IOException
*/
CommunicationsSession establishSiteToSiteConnection() throws IOException;
/**
* Called whenever RemoteProcessGroup is removed from the flow, so that any
@ -232,24 +215,4 @@ public interface RemoteProcessGroup {
void verifyCanStopTransmitting();
void verifyCanUpdate();
/**
* Returns a set of PeerStatus objects that describe the different peers
* that we can communicate with for this RemoteProcessGroup.
*
* If the destination is a cluster, this set will contain PeerStatuses for
* each of the nodes in the cluster.
*
* If the destination is a standalone instance, this set will contain just a
* PeerStatus for the destination.
*
* Once the PeerStatuses have been obtained, they may be cached by this
* RemoteProcessGroup for some amount of time.
*
* If unable to obtain the PeerStatuses or no peer status has yet been
* obtained, will return null.
*
* @return
*/
Set<PeerStatus> getPeerStatuses();
}

View File

@ -16,20 +16,28 @@
*/
package org.apache.nifi.remote;
import org.apache.nifi.connectable.ConnectableType;
import org.apache.nifi.connectable.Port;
import org.apache.nifi.controller.AbstractPort;
import org.apache.nifi.controller.ProcessScheduler;
import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.groups.RemoteProcessGroup;
public interface RemoteGroupPort extends Port {
public abstract class RemoteGroupPort extends AbstractPort implements Port, RemoteDestination {
RemoteProcessGroup getRemoteProcessGroup();
public RemoteGroupPort(String id, String name, ProcessGroup processGroup, ConnectableType type, ProcessScheduler scheduler) {
super(id, name, processGroup, type, scheduler);
}
TransferDirection getTransferDirection();
public abstract RemoteProcessGroup getRemoteProcessGroup();
boolean isUseCompression();
public abstract TransferDirection getTransferDirection();
void setUseCompression(boolean useCompression);
public abstract boolean isUseCompression();
boolean getTargetExists();
public abstract void setUseCompression(boolean useCompression);
boolean isTargetRunning();
public abstract boolean getTargetExists();
public abstract boolean isTargetRunning();
}

View File

@ -14,7 +14,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.cluster;
package org.apache.nifi.remote.cluster;
public interface NodeInformant {

View File

@ -18,13 +18,13 @@ package org.apache.nifi.remote.protocol;
import java.io.IOException;
import org.apache.nifi.cluster.NodeInformant;
import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.remote.Peer;
import org.apache.nifi.remote.RootGroupPort;
import org.apache.nifi.remote.VersionedRemoteResource;
import org.apache.nifi.remote.cluster.NodeInformant;
import org.apache.nifi.remote.codec.FlowFileCodec;
import org.apache.nifi.remote.exception.BadRequestException;
import org.apache.nifi.remote.exception.HandshakeException;

View File

@ -63,6 +63,10 @@
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-client-dto</artifactId>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-site-to-site-client</artifactId>
</dependency>
<dependency>
<groupId>org.quartz-scheduler</groupId>
<artifactId>quartz</artifactId>

View File

@ -48,6 +48,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
import javax.net.ssl.SSLContext;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.admin.service.UserService;
import org.apache.nifi.annotation.lifecycle.OnAdded;
import org.apache.nifi.annotation.lifecycle.OnRemoved;
@ -128,14 +129,12 @@ import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.groups.RemoteProcessGroup;
import org.apache.nifi.groups.RemoteProcessGroupPortDescriptor;
import org.apache.nifi.groups.StandardProcessGroup;
import org.apache.nifi.stream.io.StreamUtils;
import org.apache.nifi.logging.LogLevel;
import org.apache.nifi.logging.LogRepository;
import org.apache.nifi.logging.LogRepositoryFactory;
import org.apache.nifi.logging.ProcessorLog;
import org.apache.nifi.logging.ProcessorLogObserver;
import org.apache.nifi.nar.ExtensionManager;
import org.apache.nifi.nar.NarClassLoader;
import org.apache.nifi.nar.NarCloseable;
import org.apache.nifi.nar.NarThreadContextClassLoader;
import org.apache.nifi.processor.Processor;
@ -165,6 +164,7 @@ import org.apache.nifi.reporting.EventAccess;
import org.apache.nifi.reporting.ReportingTask;
import org.apache.nifi.reporting.Severity;
import org.apache.nifi.scheduling.SchedulingStrategy;
import org.apache.nifi.stream.io.StreamUtils;
import org.apache.nifi.util.FormatUtils;
import org.apache.nifi.util.NiFiProperties;
import org.apache.nifi.util.ReflectionUtils;
@ -184,7 +184,6 @@ import org.apache.nifi.web.api.dto.RemoteProcessGroupDTO;
import org.apache.nifi.web.api.dto.RemoteProcessGroupPortDTO;
import org.apache.nifi.web.api.dto.TemplateDTO;
import org.apache.nifi.web.api.dto.status.StatusHistoryDTO;
import org.apache.commons.lang3.StringUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

View File

@ -21,6 +21,7 @@ import java.util.Date;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.Callable;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
@ -34,8 +35,9 @@ import org.apache.nifi.controller.tasks.ContinuallyRunProcessorTask;
import org.apache.nifi.controller.tasks.ReportingTaskWrapper;
import org.apache.nifi.encrypt.StringEncryptor;
import org.apache.nifi.engine.FlowEngine;
import org.apache.nifi.processor.StandardProcessContext;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.util.FormatUtils;
import org.quartz.CronExpression;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -130,13 +132,16 @@ public class QuartzSchedulingAgent implements SchedulingAgent {
final List<AtomicBoolean> triggers = new ArrayList<>();
for (int i = 0; i < connectable.getMaxConcurrentTasks(); i++) {
final Runnable continuallyRunTask;
final Callable<Boolean> continuallyRunTask;
if (connectable.getConnectableType() == ConnectableType.PROCESSOR) {
final ProcessorNode procNode = (ProcessorNode) connectable;
ContinuallyRunProcessorTask runnableTask = new ContinuallyRunProcessorTask(this, procNode, flowController, contextFactory, scheduleState, encryptor);
final StandardProcessContext standardProcContext = new StandardProcessContext(procNode, flowController, encryptor);
ContinuallyRunProcessorTask runnableTask = new ContinuallyRunProcessorTask(this, procNode, flowController, contextFactory, scheduleState, standardProcContext);
continuallyRunTask = runnableTask;
} else {
continuallyRunTask = new ContinuallyRunConnectableTask(contextFactory, connectable, scheduleState, encryptor);
final ConnectableProcessContext connProcContext = new ConnectableProcessContext(connectable, encryptor);
continuallyRunTask = new ContinuallyRunConnectableTask(contextFactory, connectable, scheduleState, connProcContext);
}
final AtomicBoolean canceled = new AtomicBoolean(false);
@ -147,7 +152,13 @@ public class QuartzSchedulingAgent implements SchedulingAgent {
return;
}
continuallyRunTask.run();
try {
continuallyRunTask.call();
} catch (final RuntimeException re) {
throw re;
} catch (final Exception e) {
throw new ProcessException(e);
}
if (canceled.get()) {
return;

View File

@ -16,9 +16,10 @@
*/
package org.apache.nifi.controller.scheduling;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.HashSet;
import java.util.Set;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
@ -27,7 +28,7 @@ public class ScheduleState {
private final AtomicInteger activeThreadCount = new AtomicInteger(0);
private final AtomicBoolean scheduled = new AtomicBoolean(false);
private final List<ScheduledFuture<?>> futures = new ArrayList<>();
private final Set<ScheduledFuture<?>> futures = new HashSet<ScheduledFuture<?>>();
private final AtomicBoolean mustCallOnStoppedMethods = new AtomicBoolean(false);
private volatile long lastStopTime = -1;
@ -79,12 +80,17 @@ public class ScheduleState {
*
* @param newFutures
*/
public void setFutures(final List<ScheduledFuture<?>> newFutures) {
public synchronized void setFutures(final Collection<ScheduledFuture<?>> newFutures) {
futures.clear();
futures.addAll(newFutures);
}
public List<ScheduledFuture<?>> getFutures() {
return Collections.unmodifiableList(futures);
public synchronized void replaceFuture(final ScheduledFuture<?> oldFuture, final ScheduledFuture<?> newFuture) {
futures.remove(oldFuture);
futures.add(newFuture);
}
public synchronized Set<ScheduledFuture<?>> getFutures() {
return Collections.unmodifiableSet(futures);
}
}

View File

@ -18,8 +18,10 @@ package org.apache.nifi.controller.scheduling;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.Callable;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.nifi.connectable.Connectable;
import org.apache.nifi.connectable.ConnectableType;
@ -31,14 +33,16 @@ import org.apache.nifi.controller.tasks.ContinuallyRunProcessorTask;
import org.apache.nifi.controller.tasks.ReportingTaskWrapper;
import org.apache.nifi.encrypt.StringEncryptor;
import org.apache.nifi.engine.FlowEngine;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.StandardProcessContext;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.util.FormatUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class TimerDrivenSchedulingAgent implements SchedulingAgent {
private static final Logger logger = LoggerFactory.getLogger(TimerDrivenSchedulingAgent.class);
private static final long NO_WORK_YIELD_NANOS = TimeUnit.MILLISECONDS.toNanos(10L);
private final FlowController flowController;
private final FlowEngine flowEngine;
@ -72,20 +76,105 @@ public class TimerDrivenSchedulingAgent implements SchedulingAgent {
logger.info("{} started.", taskNode.getReportingTask());
}
@Override
public void schedule(final Connectable connectable, final ScheduleState scheduleState) {
final Runnable runnable;
if (connectable.getConnectableType() == ConnectableType.PROCESSOR) {
final ProcessorNode procNode = (ProcessorNode) connectable;
ContinuallyRunProcessorTask runnableTask = new ContinuallyRunProcessorTask(this, procNode, flowController, contextFactory, scheduleState, encryptor);
runnable = runnableTask;
} else {
runnable = new ContinuallyRunConnectableTask(contextFactory, connectable, scheduleState, encryptor);
}
final List<ScheduledFuture<?>> futures = new ArrayList<>();
for (int i = 0; i < connectable.getMaxConcurrentTasks(); i++) {
final ScheduledFuture<?> future = flowEngine.scheduleWithFixedDelay(runnable, 0L, connectable.getSchedulingPeriod(TimeUnit.NANOSECONDS), TimeUnit.NANOSECONDS);
final Callable<Boolean> continuallyRunTask;
final ProcessContext processContext;
// Determine the task to run and create it.
if (connectable.getConnectableType() == ConnectableType.PROCESSOR) {
final ProcessorNode procNode = (ProcessorNode) connectable;
final StandardProcessContext standardProcContext = new StandardProcessContext(procNode, flowController, encryptor);
final ContinuallyRunProcessorTask runnableTask = new ContinuallyRunProcessorTask(this, procNode, flowController,
contextFactory, scheduleState, standardProcContext);
continuallyRunTask = runnableTask;
processContext = standardProcContext;
} else {
processContext = new ConnectableProcessContext(connectable, encryptor);
continuallyRunTask = new ContinuallyRunConnectableTask(contextFactory, connectable, scheduleState, processContext);
}
final AtomicReference<ScheduledFuture<?>> futureRef = new AtomicReference<>();
final Runnable yieldDetectionRunnable = new Runnable() {
@Override
public void run() {
// Call the continually run task. It will return a boolean indicating whether or not we should yield
// based on a lack of work for to do for the component.
final boolean shouldYield;
try {
shouldYield = continuallyRunTask.call();
} catch (final RuntimeException re) {
throw re;
} catch (final Exception e) {
throw new ProcessException(e);
}
// If the component is yielded, cancel its future and re-submit it to run again
// after the yield has expired.
final long newYieldExpiration = connectable.getYieldExpiration();
if ( newYieldExpiration > System.currentTimeMillis() ) {
final long yieldMillis = System.currentTimeMillis() - newYieldExpiration;
final ScheduledFuture<?> scheduledFuture = futureRef.get();
if ( scheduledFuture == null ) {
return;
}
// If we are able to cancel the future, create a new one and update the ScheduleState so that it has
// an accurate accounting of which futures are outstanding; we must then also update the futureRef
// so that we can do this again the next time that the component is yielded.
if (scheduledFuture.cancel(false)) {
final long yieldNanos = TimeUnit.MILLISECONDS.toNanos(yieldMillis);
final ScheduledFuture<?> newFuture = flowEngine.scheduleWithFixedDelay(this, yieldNanos,
connectable.getSchedulingPeriod(TimeUnit.NANOSECONDS), TimeUnit.NANOSECONDS);
synchronized (scheduleState) {
if ( scheduleState.isScheduled() ) {
scheduleState.replaceFuture(scheduledFuture, newFuture);
futureRef.set(newFuture);
}
}
}
} else if ( shouldYield ) {
// Component itself didn't yield but there was no work to do, so the framework will choose
// to yield the component automatically for a short period of time.
final ScheduledFuture<?> scheduledFuture = futureRef.get();
if ( scheduledFuture == null ) {
return;
}
// If we are able to cancel the future, create a new one and update the ScheduleState so that it has
// an accurate accounting of which futures are outstanding; we must then also update the futureRef
// so that we can do this again the next time that the component is yielded.
if (scheduledFuture.cancel(false)) {
final ScheduledFuture<?> newFuture = flowEngine.scheduleWithFixedDelay(this, NO_WORK_YIELD_NANOS,
connectable.getSchedulingPeriod(TimeUnit.NANOSECONDS), TimeUnit.NANOSECONDS);
synchronized (scheduleState) {
if ( scheduleState.isScheduled() ) {
scheduleState.replaceFuture(scheduledFuture, newFuture);
futureRef.set(newFuture);
}
}
}
}
}
};
// Schedule the task to run
final ScheduledFuture<?> future = flowEngine.scheduleWithFixedDelay(yieldDetectionRunnable, 0L,
connectable.getSchedulingPeriod(TimeUnit.NANOSECONDS), TimeUnit.NANOSECONDS);
// now that we have the future, set the atomic reference so that if the component is yielded we
// are able to then cancel this future.
futureRef.set(future);
// Keep track of the futures so that we can update the ScheduleState.
futures.add(future);
}

View File

@ -16,16 +16,16 @@
*/
package org.apache.nifi.controller.tasks;
import java.util.concurrent.Callable;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.nifi.annotation.lifecycle.OnStopped;
import org.apache.nifi.connectable.Connectable;
import org.apache.nifi.controller.repository.StandardProcessSessionFactory;
import org.apache.nifi.controller.scheduling.ConnectableProcessContext;
import org.apache.nifi.controller.scheduling.ProcessContextFactory;
import org.apache.nifi.controller.scheduling.ScheduleState;
import org.apache.nifi.encrypt.StringEncryptor;
import org.apache.nifi.nar.NarCloseable;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSessionFactory;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.util.Connectables;
@ -33,28 +33,33 @@ import org.apache.nifi.util.ReflectionUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class ContinuallyRunConnectableTask implements Runnable {
/**
* Continually runs a Connectable as long as the processor has work to do. {@link #call()} will return
* <code>true</code> if the Connectable should be yielded, <code>false</code> otherwise.
*/
public class ContinuallyRunConnectableTask implements Callable<Boolean> {
private static final Logger logger = LoggerFactory.getLogger(ContinuallyRunConnectableTask.class);
private final Connectable connectable;
private final ScheduleState scheduleState;
private final ProcessSessionFactory sessionFactory;
private final ConnectableProcessContext processContext;
private final ProcessContext processContext;
public ContinuallyRunConnectableTask(final ProcessContextFactory contextFactory, final Connectable connectable, final ScheduleState scheduleState, final StringEncryptor encryptor) {
public ContinuallyRunConnectableTask(final ProcessContextFactory contextFactory, final Connectable connectable, final ScheduleState scheduleState, final ProcessContext processContext) {
this.connectable = connectable;
this.scheduleState = scheduleState;
this.sessionFactory = new StandardProcessSessionFactory(contextFactory.newProcessContext(connectable, new AtomicLong(0L)));
this.processContext = new ConnectableProcessContext(connectable, encryptor);
this.processContext = processContext;
}
@SuppressWarnings("deprecation")
@Override
public void run() {
@SuppressWarnings("deprecation")
public Boolean call() {
if (!scheduleState.isScheduled()) {
return;
return false;
}
// Connectable should run if the following conditions are met:
// 1. It's an Input Port or or is a Remote Input Port or has incoming FlowFiles queued
// 2. Any relationship is available (since there's only 1
@ -62,8 +67,9 @@ public class ContinuallyRunConnectableTask implements Runnable {
// it means the same thing)
// 3. It is not yielded.
final boolean triggerWhenEmpty = connectable.isTriggerWhenEmpty();
boolean flowFilesQueued = true;
final boolean shouldRun = (connectable.getYieldExpiration() < System.currentTimeMillis())
&& (triggerWhenEmpty || Connectables.flowFilesQueued(connectable)) && (connectable.getRelationships().isEmpty() || Connectables.anyRelationshipAvailable(connectable));
&& (triggerWhenEmpty || (flowFilesQueued = Connectables.flowFilesQueued(connectable))) && (connectable.getRelationships().isEmpty() || Connectables.anyRelationshipAvailable(connectable));
if (shouldRun) {
scheduleState.incrementActiveThreadCount();
@ -92,6 +98,12 @@ public class ContinuallyRunConnectableTask implements Runnable {
scheduleState.decrementActiveThreadCount();
}
} else if (!flowFilesQueued) {
// FlowFiles must be queued in order to run but there are none queued;
// yield for just a bit.
return true;
}
return true;
}
}

View File

@ -17,6 +17,7 @@
package org.apache.nifi.controller.tasks;
import java.io.IOException;
import java.util.concurrent.Callable;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;
@ -31,7 +32,6 @@ import org.apache.nifi.controller.repository.StandardProcessSessionFactory;
import org.apache.nifi.controller.scheduling.ProcessContextFactory;
import org.apache.nifi.controller.scheduling.ScheduleState;
import org.apache.nifi.controller.scheduling.SchedulingAgent;
import org.apache.nifi.encrypt.StringEncryptor;
import org.apache.nifi.logging.ProcessorLog;
import org.apache.nifi.nar.NarCloseable;
import org.apache.nifi.processor.ProcessSessionFactory;
@ -43,7 +43,12 @@ import org.apache.nifi.util.ReflectionUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class ContinuallyRunProcessorTask implements Runnable {
/**
* Continually runs a processor as long as the processor has work to do. {@link #call()} will return
* <code>true</code> if the processor should be yielded, <code>false</code> otherwise.
*/
public class ContinuallyRunProcessorTask implements Callable<Boolean> {
private static final Logger logger = LoggerFactory.getLogger(ContinuallyRunProcessorTask.class);
@ -56,7 +61,8 @@ public class ContinuallyRunProcessorTask implements Runnable {
private final int numRelationships;
public ContinuallyRunProcessorTask(final SchedulingAgent schedulingAgent, final ProcessorNode procNode,
final FlowController flowController, final ProcessContextFactory contextFactory, final ScheduleState scheduleState, final StringEncryptor encryptor) {
final FlowController flowController, final ProcessContextFactory contextFactory, final ScheduleState scheduleState,
final StandardProcessContext processContext) {
this.schedulingAgent = schedulingAgent;
this.procNode = procNode;
@ -65,28 +71,28 @@ public class ContinuallyRunProcessorTask implements Runnable {
this.flowController = flowController;
context = contextFactory.newProcessContext(procNode, new AtomicLong(0L));
this.processContext = new StandardProcessContext(procNode, flowController, encryptor);
this.processContext = processContext;
}
@SuppressWarnings("deprecation")
@Override
public void run() {
@SuppressWarnings("deprecation")
public Boolean call() {
// make sure processor is not yielded
boolean shouldRun = (procNode.getYieldExpiration() < System.currentTimeMillis());
if (!shouldRun) {
return;
return false;
}
// make sure that either we're not clustered or this processor runs on all nodes or that this is the primary node
shouldRun = !procNode.isIsolated() || !flowController.isClustered() || flowController.isPrimary();
if (!shouldRun) {
return;
return false;
}
// make sure that either proc has incoming FlowFiles or has no incoming connections or is annotated with @TriggerWhenEmpty
shouldRun = procNode.isTriggerWhenEmpty() || !procNode.hasIncomingConnection() || Connectables.flowFilesQueued(procNode);
if (!shouldRun) {
return;
return true;
}
if (numRelationships > 0) {
@ -109,7 +115,7 @@ public class ContinuallyRunProcessorTask implements Runnable {
}
if (!shouldRun) {
return;
return false;
}
scheduleState.incrementActiveThreadCount();
@ -124,11 +130,11 @@ public class ContinuallyRunProcessorTask implements Runnable {
invocationCount++;
if (!batch) {
return;
return false;
}
if (System.nanoTime() > finishNanos) {
return;
return false;
}
shouldRun = procNode.isTriggerWhenEmpty() || !procNode.hasIncomingConnection() || Connectables.flowFilesQueued(procNode);
@ -180,6 +186,8 @@ public class ContinuallyRunProcessorTask implements Runnable {
logger.error("", e);
}
}
return false;
}
}

View File

@ -0,0 +1,216 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF 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 java.io.IOException;
import java.net.URI;
import java.net.URISyntaxException;
import java.util.Map;
import javax.net.ssl.SSLContext;
import javax.ws.rs.core.MediaType;
import org.apache.nifi.web.api.dto.ControllerDTO;
import org.apache.nifi.web.api.entity.ControllerEntity;
import org.apache.nifi.web.util.WebUtils;
import com.sun.jersey.api.client.Client;
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.config.ClientConfig;
import com.sun.jersey.api.client.UniformInterfaceException;
import com.sun.jersey.api.client.WebResource;
import com.sun.jersey.core.util.MultivaluedMapImpl;
/**
*
*/
public class RemoteNiFiUtils {
public static final String CONTROLLER_URI_PATH = "/controller";
private static final int CONNECT_TIMEOUT = 10000;
private static final int READ_TIMEOUT = 10000;
private final Client client;
public RemoteNiFiUtils(final SSLContext sslContext) {
this.client = getClient(sslContext);
}
/**
* Gets the content at the specified URI.
*
* @param uri
* @param timeoutMillis
* @return
* @throws ClientHandlerException
* @throws UniformInterfaceException
*/
public ClientResponse get(final URI uri, final int timeoutMillis) throws ClientHandlerException, UniformInterfaceException {
return get(uri, timeoutMillis, null);
}
/**
* Gets the content at the specified URI using the given query parameters.
*
* @param uri
* @param timeoutMillis
* @param queryParams
* @return
* @throws ClientHandlerException
* @throws UniformInterfaceException
*/
public ClientResponse get(final URI uri, final int timeoutMillis, final Map<String, String> queryParams) throws ClientHandlerException, UniformInterfaceException {
// perform the request
WebResource webResource = client.resource(uri);
if ( queryParams != null ) {
for ( final Map.Entry<String, String> queryEntry : queryParams.entrySet() ) {
webResource = webResource.queryParam(queryEntry.getKey(), queryEntry.getValue());
}
}
webResource.setProperty(ClientConfig.PROPERTY_READ_TIMEOUT, timeoutMillis);
webResource.setProperty(ClientConfig.PROPERTY_CONNECT_TIMEOUT, timeoutMillis);
return webResource.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
}
/**
* Performs a HEAD request to the specified URI.
*
* @param uri
* @param timeoutMillis
* @return
* @throws ClientHandlerException
* @throws UniformInterfaceException
*/
public ClientResponse head(final URI uri, final int timeoutMillis) throws ClientHandlerException, UniformInterfaceException {
// perform the request
WebResource webResource = client.resource(uri);
webResource.setProperty(ClientConfig.PROPERTY_READ_TIMEOUT, timeoutMillis);
webResource.setProperty(ClientConfig.PROPERTY_CONNECT_TIMEOUT, timeoutMillis);
return webResource.head();
}
/**
* Gets a client based on the specified URI.
*
* @param uri
* @return
*/
private Client getClient(final SSLContext sslContext) {
final Client client;
if (sslContext == null) {
client = WebUtils.createClient(null);
} else {
client = WebUtils.createClient(null, sslContext);
}
client.setReadTimeout(READ_TIMEOUT);
client.setConnectTimeout(CONNECT_TIMEOUT);
return client;
}
/**
* Returns the port on which the remote instance is listening for Flow File transfers, or <code>null</code> if the remote instance
* is not configured to use Site-to-Site transfers.
*
* @param uri the base URI of the remote instance. This should include the path only to the nifi-api level, as well as the protocol, host, and port.
* @param timeoutMillis
* @return
* @throws IOException
*/
public Integer getRemoteListeningPort(final String uri, final int timeoutMillis) throws IOException {
try {
final URI uriObject = new URI(uri + CONTROLLER_URI_PATH);
return getRemoteListeningPort(uriObject, timeoutMillis);
} catch (URISyntaxException e) {
throw new IOException("Unable to establish connection to remote host because URI is invalid: " + uri);
}
}
public String getRemoteRootGroupId(final String uri, final int timeoutMillis) throws IOException {
try {
final URI uriObject = new URI(uri + CONTROLLER_URI_PATH);
return getRemoteRootGroupId(uriObject, timeoutMillis);
} catch (URISyntaxException e) {
throw new IOException("Unable to establish connection to remote host because URI is invalid: " + uri);
}
}
public String getRemoteInstanceId(final String uri, final int timeoutMillis) throws IOException {
try {
final URI uriObject = new URI(uri + CONTROLLER_URI_PATH);
return getController(uriObject, timeoutMillis).getInstanceId();
} catch (URISyntaxException e) {
throw new IOException("Unable to establish connection to remote host because URI is invalid: " + uri);
}
}
/**
* Returns the port on which the remote instance is listening for Flow File transfers, or <code>null</code> if the remote instance
* is not configured to use Site-to-Site transfers.
*
* @param uri the full URI to fetch, including the path.
* @return
* @throws IOException
*/
private Integer getRemoteListeningPort(final URI uri, final int timeoutMillis) throws IOException {
return getController(uri, timeoutMillis).getRemoteSiteListeningPort();
}
private String getRemoteRootGroupId(final URI uri, final int timeoutMillis) throws IOException {
return getController(uri, timeoutMillis).getId();
}
public ControllerDTO getController(final URI uri, final int timeoutMillis) throws IOException {
final ClientResponse response = get(uri, timeoutMillis);
if (Status.OK.getStatusCode() == response.getStatusInfo().getStatusCode()) {
final ControllerEntity entity = response.getEntity(ControllerEntity.class);
return entity.getController();
} else {
final String responseMessage = response.getEntity(String.class);
throw new IOException("Got HTTP response Code " + response.getStatusInfo().getStatusCode() + ": " + response.getStatusInfo().getReasonPhrase() + " with explanation: " + responseMessage);
}
}
/**
* Issues a registration request on behalf of the current user.
*
* @param baseApiUri
* @return
*/
public ClientResponse issueRegistrationRequest(String baseApiUri) {
final URI uri = URI.create(String.format("%s/%s", baseApiUri, "/controller/users"));
// set up the query params
MultivaluedMapImpl entity = new MultivaluedMapImpl();
entity.add("justification", "A Remote instance of NiFi has attempted to create a reference to this NiFi. This action must be approved first.");
// create the web resource
WebResource webResource = client.resource(uri);
// get the client utils and make the request
return webResource.accept(MediaType.APPLICATION_JSON).type(MediaType.APPLICATION_FORM_URLENCODED).entity(entity).post(ClientResponse.class);
}
}

View File

@ -18,21 +18,9 @@ package org.apache.nifi.remote;
import static java.util.Objects.requireNonNull;
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.util.ArrayList;
import java.util.Date;
import java.util.HashMap;
@ -49,11 +37,8 @@ import java.util.concurrent.atomic.AtomicReference;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.regex.Pattern;
import javax.net.ssl.SSLContext;
import javax.security.cert.CertificateExpiredException;
import javax.security.cert.CertificateNotYetValidException;
import javax.ws.rs.core.Response;
import org.apache.nifi.connectable.ConnectableType;
@ -64,7 +49,6 @@ import org.apache.nifi.controller.FlowController;
import org.apache.nifi.controller.ProcessScheduler;
import org.apache.nifi.controller.ScheduledState;
import org.apache.nifi.controller.exception.CommunicationsException;
import org.apache.nifi.controller.util.RemoteProcessGroupUtils;
import org.apache.nifi.engine.FlowEngine;
import org.apache.nifi.events.BulletinFactory;
import org.apache.nifi.events.EventReporter;
@ -72,16 +56,6 @@ 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.stream.io.BufferedOutputStream;
import org.apache.nifi.remote.exception.BadRequestException;
import org.apache.nifi.remote.exception.HandshakeException;
import org.apache.nifi.remote.exception.PortNotRunningException;
import org.apache.nifi.remote.exception.UnknownPortException;
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.socket.SocketClientProtocol;
import org.apache.nifi.reporting.BulletinRepository;
import org.apache.nifi.reporting.Severity;
import org.apache.nifi.util.FormatUtils;
@ -108,7 +82,6 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
public static final String CONTROLLER_URI_PATH = "/controller";
public static final String ROOT_GROUP_STATUS_URI_PATH = "/controller/process-groups/root/status";
public static final long LISTENING_PORT_REFRESH_MILLIS = TimeUnit.MILLISECONDS.convert(10, TimeUnit.MINUTES);
// status codes
public static final int OK_STATUS_CODE = Status.OK.getStatusCode();
@ -150,15 +123,11 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
private ProcessGroupCounts counts = new ProcessGroupCounts(0, 0, 0, 0, 0, 0, 0, 0);
private Long refreshContentsTimestamp = null;
private Integer listeningPort;
private long listeningPortRetrievalTime = 0L;
private Boolean destinationSecure;
private Integer listeningPort;
private volatile String authorizationIssue;
private volatile PeerStatusCache peerStatusCache;
private static final long PEER_CACHE_MILLIS = TimeUnit.MILLISECONDS.convert(1, TimeUnit.MINUTES);
private final ScheduledExecutorService backgroundThreadExecutor;
public StandardRemoteProcessGroup(final String id, final String targetUri, final ProcessGroup processGroup,
@ -200,72 +169,9 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
}
};
final Runnable socketCleanup = new Runnable() {
@Override
public void run() {
final Set<StandardRemoteGroupPort> ports = new HashSet<>();
readLock.lock();
try {
ports.addAll(inputPorts.values());
ports.addAll(outputPorts.values());
} finally {
readLock.unlock();
}
for (final StandardRemoteGroupPort port : ports) {
port.cleanupSockets();
}
}
};
try {
final File peersFile = getPeerPersistenceFile();
this.peerStatusCache = new PeerStatusCache(recoverPersistedPeerStatuses(peersFile), peersFile.lastModified());
} catch (final IOException e) {
logger.error("{} Failed to recover persisted Peer Statuses due to {}", this, e);
}
final Runnable refreshPeers = new Runnable() {
@Override
public void run() {
final PeerStatusCache existingCache = peerStatusCache;
if (existingCache != null && (existingCache.getTimestamp() + PEER_CACHE_MILLIS > System.currentTimeMillis())) {
return;
}
Set<RemoteGroupPort> ports = getInputPorts();
if (ports.isEmpty()) {
ports = getOutputPorts();
}
if (ports.isEmpty()){
return;
}
// it doesn't really matter which port we use. Since we are just getting the Peer Status,
// if the server indicates that the port cannot receive data for whatever reason, we will
// simply ignore the error.
final RemoteGroupPort port = ports.iterator().next();
try {
final Set<PeerStatus> statuses = fetchRemotePeerStatuses(port);
peerStatusCache = new PeerStatusCache(statuses);
logger.info("{} Successfully refreshed Peer Status; remote instance consists of {} peers", StandardRemoteProcessGroup.this, statuses.size());
} catch (Exception e) {
logger.warn("{} Unable to refresh Remote Group's peers due to {}", StandardRemoteProcessGroup.this, e);
if (logger.isDebugEnabled()) {
logger.warn("", e);
}
}
}
};
final Runnable checkAuthorizations = new InitializationTask();
backgroundThreadExecutor = new FlowEngine(1, "Remote Process Group " + id + ": " + targetUri);
backgroundThreadExecutor.scheduleWithFixedDelay(checkAuthorizations, 0L, 30L, TimeUnit.SECONDS);
backgroundThreadExecutor.scheduleWithFixedDelay(refreshPeers, 0, 5, TimeUnit.SECONDS);
backgroundThreadExecutor.scheduleWithFixedDelay(socketCleanup, 10L, 10L, TimeUnit.SECONDS);
backgroundThreadExecutor.scheduleWithFixedDelay(checkAuthorizations, 5L, 30L, TimeUnit.SECONDS);
}
@Override
@ -858,7 +764,7 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
return;
}
final RemoteProcessGroupUtils utils = new RemoteProcessGroupUtils(isWebApiSecure() ? sslContext : null);
final RemoteNiFiUtils utils = new RemoteNiFiUtils(isWebApiSecure() ? sslContext : null);
final String uriVal = apiUri.toString() + CONTROLLER_URI_PATH;
URI uri;
try {
@ -998,39 +904,6 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
return descriptor;
}
/**
* @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
*/
@Override
public Integer getListeningPort() throws IOException {
Integer listeningPort;
readLock.lock();
try {
listeningPort = this.listeningPort;
if (listeningPort != null && this.listeningPortRetrievalTime > System.currentTimeMillis() - LISTENING_PORT_REFRESH_MILLIS) {
return listeningPort;
}
} finally {
readLock.unlock();
}
final RemoteProcessGroupUtils utils = new RemoteProcessGroupUtils(isWebApiSecure() ? sslContext : null);
listeningPort = utils.getRemoteListeningPort(apiUri.toString(), getCommunicationsTimeout(TimeUnit.MILLISECONDS));
writeLock.lock();
try {
this.listeningPort = listeningPort;
this.listeningPortRetrievalTime = System.currentTimeMillis();
} finally {
writeLock.unlock();
}
return listeningPort;
}
@Override
public boolean isTransmitting() {
return transmitting.get();
@ -1255,52 +1128,6 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
}
}
@Override
public CommunicationsSession establishSiteToSiteConnection() throws IOException {
final URI uri = apiUri;
final String destinationUri = uri.toString();
CommunicationsSession commsSession = null;
try {
if (isSecure()) {
if (sslContext == null) {
throw new IOException("Unable to communicate with " + getTargetUri() + " because it requires Secure Site-to-Site communications, but this instance is not configured for secure communications");
}
final Integer listeningPort = getListeningPort();
if (listeningPort == null) {
throw new IOException("Remote instance is not configured to allow incoming Site-to-Site connections");
}
final SSLSocketChannel socketChannel = new SSLSocketChannel(sslContext, uri.getHost(), listeningPort, true);
socketChannel.connect();
commsSession = new SSLSocketChannelCommunicationsSession(socketChannel, destinationUri);
try {
commsSession.setUserDn(socketChannel.getDn());
} catch (final CertificateNotYetValidException | CertificateExpiredException ex) {
throw new IOException(ex);
}
} else {
final SocketChannel socketChannel = SocketChannel.open(new InetSocketAddress(uri.getHost(), getListeningPort()));
commsSession = new SocketChannelCommunicationsSession(socketChannel, destinationUri);
}
commsSession.getOutput().getOutputStream().write(CommunicationsSession.MAGIC_BYTES);
commsSession.setUri("nifi://" + uri.getHost() + ":" + uri.getPort());
} catch (final IOException e) {
if (commsSession != null) {
try {
commsSession.close();
} catch (final IOException ignore) {
}
}
throw e;
}
return commsSession;
}
@Override
public EventReporter getEventReporter() {
@ -1312,7 +1139,7 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
@Override
public void run() {
try {
final RemoteProcessGroupUtils utils = new RemoteProcessGroupUtils(isWebApiSecure() ? sslContext : null);
final RemoteNiFiUtils utils = new RemoteNiFiUtils(isWebApiSecure() ? sslContext : null);
final ClientResponse response = utils.get(new URI(apiUri + CONTROLLER_URI_PATH), getCommunicationsTimeout(TimeUnit.MILLISECONDS));
final int statusCode = response.getStatus();
@ -1487,135 +1314,9 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
}
}
@Override
public 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.getHostname(), status.getPort(), status.isSecure(), 1);
equalizedSet.add(equalizedStatus);
}
return equalizedSet;
}
return cache.getStatuses();
}
private Set<PeerStatus> fetchRemotePeerStatuses(final RemoteGroupPort port) throws IOException, HandshakeException, UnknownPortException, PortNotRunningException, BadRequestException {
final CommunicationsSession commsSession = establishSiteToSiteConnection();
final Peer peer = new Peer(commsSession, "nifi://" + getTargetUri().getHost() + ":" + getListeningPort());
final SocketClientProtocol clientProtocol = new SocketClientProtocol();
clientProtocol.setPort(port);
final DataInputStream dis = new DataInputStream(commsSession.getInput().getInputStream());
final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream());
try {
RemoteResourceFactory.initiateResourceNegotiation(clientProtocol, dis, dos);
} catch (final HandshakeException e) {
throw new BadRequestException(e.toString());
}
clientProtocol.handshake(peer);
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());
logger.warn(message);
if (logger.isDebugEnabled()) {
logger.warn("", e);
}
getEventReporter().reportEvent(Severity.WARNING, "Site to Site", message);
}
try {
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());
logger.warn(message);
if (logger.isDebugEnabled()) {
logger.warn("", e);
}
getEventReporter().reportEvent(Severity.WARNING, "Site to Site", message);
}
return peerStatuses;
}
private File getPeerPersistenceFile() {
final File stateDir = NiFiProperties.getInstance().getPersistentStateDirectory();
return new File(stateDir, getIdentifier() + ".peers");
}
private void persistPeerStatuses(final Set<PeerStatus> statuses) {
final File peersFile = getPeerPersistenceFile();
try (final OutputStream fos = new FileOutputStream(peersFile);
final OutputStream out = new BufferedOutputStream(fos)) {
for (final PeerStatus status : statuses) {
final String line = status.getHostname() + ":" + status.getPort() + ":" + status.isSecure() + "\n";
out.write(line.getBytes(StandardCharsets.UTF_8));
}
} catch (final IOException e) {
logger.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(), 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(hostname, port, secure, 1));
}
}
return statuses;
}
private static class PeerStatusCache {
private final Set<PeerStatus> statuses;
private final long timestamp;
public PeerStatusCache(final Set<PeerStatus> statuses) {
this(statuses, System.currentTimeMillis());
}
public PeerStatusCache(final Set<PeerStatus> statuses, final long timestamp) {
this.statuses = statuses;
this.timestamp = timestamp;
}
public Set<PeerStatus> getStatuses() {
return statuses;
}
public long getTimestamp() {
return timestamp;
}
}
}

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