diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/remote/RemoteDestination.java b/nifi/nifi-api/src/main/java/org/apache/nifi/remote/RemoteDestination.java new file mode 100644 index 0000000000..f718581575 --- /dev/null +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/remote/RemoteDestination.java @@ -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; + +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 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(); +} diff --git a/nifi/nifi-commons/nifi-site-to-site-client/pom.xml b/nifi/nifi-commons/nifi-site-to-site-client/pom.xml new file mode 100644 index 0000000000..3fc00a2038 --- /dev/null +++ b/nifi/nifi-commons/nifi-site-to-site-client/pom.xml @@ -0,0 +1,43 @@ + + + 4.0.0 + + + org.apache.nifi + nifi-commons + 0.0.2-incubating-SNAPSHOT + + + nifi-site-to-site-client + + + + org.apache.nifi + nifi-api + + + org.apache.nifi + nifi-utils + + + com.sun.jersey + jersey-client + + + org.apache.nifi + nifi-client-dto + 0.0.2-incubating-SNAPSHOT + + + org.apache.nifi + nifi-web-utils + + + + junit + junit + test + + + diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/AbstractCommunicationsSession.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/AbstractCommunicationsSession.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/AbstractCommunicationsSession.java rename to nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/AbstractCommunicationsSession.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/Peer.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/Peer.java similarity index 81% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/Peer.java rename to nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/Peer.java index 2422fe144a..29af777173 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/Peer.java +++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/Peer.java @@ -20,29 +20,37 @@ import java.io.IOException; import java.net.URI; import org.apache.nifi.remote.protocol.CommunicationsSession; +import org.apache.nifi.stream.io.NullOutputStream; +import org.apache.nifi.stream.io.StreamUtils; public class Peer { private final CommunicationsSession commsSession; private final String url; + private final String clusterUrl; private final String host; private long penalizationExpiration = 0L; private boolean closed = false; - public Peer(final CommunicationsSession commsSession, final String url) { + public Peer(final CommunicationsSession commsSession, final String peerUrl, final String clusterUrl) { this.commsSession = commsSession; - this.url = url; + this.url = peerUrl; + this.clusterUrl = clusterUrl; try { - this.host = new URI(url).getHost(); + this.host = new URI(peerUrl).getHost(); } catch (final Exception e) { - throw new IllegalArgumentException("Invalid URL: " + url); + throw new IllegalArgumentException("Invalid URL: " + peerUrl); } } public String getUrl() { return url; } + + public String getClusterUrl() { + return clusterUrl; + } public CommunicationsSession getCommunicationsSession() { return commsSession; @@ -51,8 +59,12 @@ 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 - commsSession.close(); + // Consume the InputStream so that it doesn't linger on the Peer's outgoing socket buffer + try { + StreamUtils.copy(commsSession.getInput().getInputStream(), new NullOutputStream()); + } finally { + commsSession.close(); + } } public void penalize(final long millis) { diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/PeerStatus.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/PeerStatus.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/PeerStatus.java rename to nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/PeerStatus.java diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/RemoteResourceInitiator.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/RemoteResourceInitiator.java new file mode 100644 index 0000000000..8eb5d8d9b2 --- /dev/null +++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/RemoteResourceInitiator.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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; + +public class RemoteResourceInitiator { + public static final int RESOURCE_OK = 20; + public static final int DIFFERENT_RESOURCE_VERSION = 21; + public static final int ABORT = 255; + + + 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 + dos.writeUTF(resource.getResourceName()); + final VersionNegotiator negotiator = resource.getVersionNegotiator(); + dos.writeInt(negotiator.getVersion()); + dos.flush(); + + // wait for response from server. + final int statusCode = dis.read(); + switch (statusCode) { + case RESOURCE_OK: // server accepted our proposal of codec name/version + return resource; + case DIFFERENT_RESOURCE_VERSION: // server accepted our proposal of codec name but not the 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: + throw new HandshakeException("Remote destination aborted connection with message: " + dis.readUTF()); + default: + return null; // Unable to negotiate codec + } + } +} diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/Transaction.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/Transaction.java new file mode 100644 index 0000000000..cc16625e2b --- /dev/null +++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/Transaction.java @@ -0,0 +1,200 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.remote; + +import java.io.IOException; + +import org.apache.nifi.remote.protocol.DataPacket; + + +/** + *

+ * Provides a transaction for performing site-to-site data transfers. + *

+ * + *

+ * 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. + *

+ * + *

+ * The general flow of execute of a Transaction is as follows: + *

    + *
  1. Create the transaction as described above.
  2. + *
  3. 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. Note: receive() should not be + * called a second time without first fully consuming the stream from the previous Packet that was received.
  4. + *
  5. Confirm the transaction via the {@link #confirm()} method.
  6. + *
  7. Either complete the transaction via the {@link #complete(boolean)} method or cancel the transaction + * via the {@link #cancel()} method.
  8. + *
+ *

+ * + *

+ * 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: + *

+ *

+ * + *

+ * 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()}. + *

+ * + *

+ * The Transaction class should not be assumed to be thread-safe. + *

+ */ +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; + + /** + * 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; + + /** + *

+ * Confirms the data that was sent or received by comparing CRC32's of the data sent and the data received. + *

+ * + *

+ * 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. + *

+ * + *

+ * 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}). + *

+ * + *

+ * 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)}. + *

+ * + * @throws IOException + */ + void confirm() throws IOException; + + /** + *

+ * Completes the transaction and indicates to both the sender and receiver that the data transfer was + * successful. If receiving data, this method can also optionally request that the sender back off sending + * data for a short period of time. This is used, for instance, to apply backpressure or to notify the sender + * that the receiver is not ready to receive data and made not service another request in the short term. + *

+ * + * @param requestBackoff if true and the TransferDirection is RECEIVE, indicates to sender that it + * should back off sending data for a short period of time. If false or if the TransferDirection of + * this Transaction is SEND, then this argument is ignored. + * + * @throws IOException + */ + void complete(boolean requestBackoff) throws IOException; + + /** + *

+ * 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. + *

+ * + * @param explanation an explanation to tell the other party why the transaction was canceled. + * @throws IOException + */ + void cancel(final String explanation) throws IOException; + + + /** + *

+ * 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. + *

+ */ + void error(); + + + /** + * Returns the current state of the Transaction. + * @return + * @throws IOException + */ + TransactionState getState() throws IOException; + + + 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; + } +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/TransferDirection.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/TransferDirection.java similarity index 75% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/TransferDirection.java rename to nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/TransferDirection.java index 56432d50f2..45029a4ca3 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/TransferDirection.java +++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/TransferDirection.java @@ -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; } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/VersionedRemoteResource.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/VersionedRemoteResource.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/VersionedRemoteResource.java rename to nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/VersionedRemoteResource.java diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClient.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClient.java new file mode 100644 index 0000000000..fa94b81255 --- /dev/null +++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClient.java @@ -0,0 +1,443 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.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.protocol.DataPacket; + +/** + *

+ * 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). + *

+ * + *

+ * 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. + *

+ * + *

+ * 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). + *

+ * + *

+ * 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. + *

+ * + *

+ * 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. + *

+ */ +public interface SiteToSiteClient extends Closeable { + + /** + * 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. + * + * + * @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 + * @throws IOException + */ + Transaction createTransaction(TransferDirection direction) throws IOException; + + /** + *

+ * 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. + *

+ * + *

+ * 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. + *

+ * + * @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; + + /** + *

+ * Returns the configuration object that was built by the Builder + *

+ * @return + */ + SiteToSiteClientConfig getConfig(); + + /** + *

+ * 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. + *

+ */ + public static class Builder { + private String url; + private long timeoutNanos = TimeUnit.SECONDS.toNanos(30); + private long penalizationNanos = TimeUnit.SECONDS.toNanos(3); + 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; + + /** + * 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; + } + + /** + * 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; + } + + + /** + * Builds a new SiteToSiteClient that can be used to send and receive data with remote instances of NiFi + * @return + */ + 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"); + } + + final SiteToSiteClientConfig config = new SiteToSiteClientConfig() { + + @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 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 new SocketClient(config); + } + + /** + * Returns the configured URL for the remote NiFi instance + * @return + */ + public String getUrl() { + return url; + } + + /** + * Returns the communications timeout in nanoseconds + * @return + */ + public long getTimeout(final TimeUnit timeUnit) { + return timeUnit.convert(timeoutNanos, 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; + } + } +} diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClientConfig.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClientConfig.java new file mode 100644 index 0000000000..37c48f856c --- /dev/null +++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClientConfig.java @@ -0,0 +1,114 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.util.concurrent.TimeUnit; + +import javax.net.ssl.SSLContext; + +import org.apache.nifi.events.EventReporter; +import org.apache.nifi.remote.protocol.DataPacket; + +public interface SiteToSiteClientConfig { + + /** + * 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 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 EventReporter that is to be used by clients to report events + * @return + */ + EventReporter getEventReporter(); + + /** + * 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(); +} diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionState.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionState.java new file mode 100644 index 0000000000..f4ac727401 --- /dev/null +++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionState.java @@ -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 EndpointConnectionState { + private final Peer peer; + private final SocketClientProtocol socketClientProtocol; + private final FlowFileCodec codec; + private volatile long lastUsed; + + public EndpointConnectionState(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; + } +} diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionStatePool.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionStatePool.java new file mode 100644 index 0000000000..8c23e28813 --- /dev/null +++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionStatePool.java @@ -0,0 +1,835 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +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.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.PeerStatusCache; +import org.apache.nifi.remote.util.RemoteNiFiUtils; +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 EndpointConnectionStatePool { + 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(EndpointConnectionStatePool.class); + + private final BlockingQueue connectionStateQueue = new LinkedBlockingQueue<>(); + private final ConcurrentMap 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 peerStatuses; + private volatile long peerRefreshTime = 0L; + private volatile PeerStatusCache peerStatusCache; + private final Set activeCommsChannels = new HashSet<>(); + + private final File peersFile; + private final EventReporter eventReporter; + private final SSLContext sslContext; + private final ScheduledExecutorService taskExecutor; + + 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 inputPortMap = new HashMap<>(); // map input port name to identifier + private final Map outputPortMap = new HashMap<>(); // map output port name to identifier + + private volatile int commsTimeout; + + public EndpointConnectionStatePool(final String clusterUrl, final int commsTimeoutMillis, final EventReporter eventReporter, final File persistenceFile) { + this(clusterUrl, commsTimeoutMillis, null, eventReporter, persistenceFile); + } + + public EndpointConnectionStatePool(final String clusterUrl, final int commsTimeoutMillis, final SSLContext sslContext, final EventReporter eventReporter, final File persistenceFile) { + 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.sslContext = sslContext; + this.peersFile = persistenceFile; + this.eventReporter = eventReporter; + this.commsTimeout = commsTimeoutMillis; + + Set 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); + } + + + public EndpointConnectionState getEndpointConnectionState(final RemoteDestination remoteDestination, final TransferDirection direction) throws IOException, HandshakeException, PortNotRunningException, UnknownPortException, ProtocolException { + return getEndpointConnectionState(remoteDestination, direction, null); + } + + + + public EndpointConnectionState getEndpointConnectionState(final RemoteDestination remoteDestination, 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; + EndpointConnectionState connectionState; + Peer peer = null; + + final List addBack = new ArrayList<>(); + try { + do { + final PeerStatus peerStatus = getNextPeerStatus(direction); + if ( peerStatus == null ) { + return null; + } + + connectionState = connectionStateQueue.poll(); + logger.debug("{} Connection State for {} = {}", this, clusterUrl, connectionState); + + if ( connectionState == null && !addBack.isEmpty() ) { + // all available connections have been penalized. + return null; + } + + if ( connectionState != null && connectionState.getPeer().isPenalized() ) { + // 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(connectionState); + continue; + } + + // if we can't get an existing ConnectionState, create one + if ( connectionState == null ) { + protocol = new SocketClientProtocol(); + protocol.setDestination(remoteDestination); + + try { + commsSession = establishSiteToSiteConnection(peerStatus); + final DataInputStream dis = new DataInputStream(commsSession.getInput().getInputStream()); + final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream()); + try { + RemoteResourceInitiator.initiateResourceNegotiation(protocol, dis, dos); + } catch (final HandshakeException e) { + try { + commsSession.close(); + } catch (final IOException ioe) { + throw e; + } + } + } catch (final IOException e) { + } + + + final String peerUrl = "nifi://" + peerStatus.getHostname() + ":" + peerStatus.getPort(); + peer = new Peer(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 { + protocol.handshake(peer); + + // handle error cases + if ( protocol.isDestinationFull() ) { + logger.warn("{} {} indicates that port's destination is full; penalizing peer", this, peer); + penalize(peer, remoteDestination.getYieldPeriod(TimeUnit.MILLISECONDS)); + connectionStateQueue.offer(connectionState); + continue; + } else if ( protocol.isPortInvalid() ) { + penalize(peer, remoteDestination.getYieldPeriod(TimeUnit.MILLISECONDS)); + cleanup(protocol, peer); + throw new PortNotRunningException(peer.toString() + " indicates that port " + remoteDestination.getIdentifier() + " is not running"); + } else if ( protocol.isPortUnknown() ) { + penalize(peer, remoteDestination.getYieldPeriod(TimeUnit.MILLISECONDS)); + cleanup(protocol, peer); + throw new UnknownPortException(peer.toString() + " indicates that port " + remoteDestination.getIdentifier() + " is not known"); + } + + // negotiate the FlowFileCodec to use + codec = protocol.negotiateCodec(peer); + } catch (final PortNotRunningException | UnknownPortException e) { + throw e; + } catch (final Exception e) { + penalize(peer, remoteDestination.getYieldPeriod(TimeUnit.MILLISECONDS)); + 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; + } + + connectionState = new EndpointConnectionState(peer, protocol, codec); + } else { + final long lastTimeUsed = connectionState.getLastTimeUsed(); + final long millisSinceLastUse = System.currentTimeMillis() - lastTimeUsed; + + if ( commsTimeout > 0L && millisSinceLastUse >= commsTimeout ) { + cleanup(connectionState.getSocketClientProtocol(), connectionState.getPeer()); + connectionState = null; + } else { + codec = connectionState.getCodec(); + peer = connectionState.getPeer(); + commsSession = peer.getCommunicationsSession(); + protocol = connectionState.getSocketClientProtocol(); + } + } + } while ( connectionState == null || codec == null || commsSession == null || protocol == null ); + } finally { + if ( !addBack.isEmpty() ) { + connectionStateQueue.addAll(addBack); + } + } + + return connectionState; + } + + + public boolean offer(final EndpointConnectionState endpointConnectionState) { + final Peer peer = endpointConnectionState.getPeer(); + if ( peer == null ) { + return false; + } + + final String url = peer.getUrl(); + if ( url == null ) { + return false; + } + + return connectionStateQueue.offer(endpointConnectionState); + } + + /** + * 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) { + String host; + int port; + try { + final URI uri = new URI(peer.getUrl()); + host = uri.getHost(); + port = uri.getPort(); + } catch (final URISyntaxException e) { + host = peer.getHost(); + port = -1; + } + + final PeerStatus status = new PeerStatus(host, port, true, 1); + Long expiration = peerTimeoutExpirations.get(status); + if ( expiration == null ) { + expiration = Long.valueOf(0L); + } + + final long newExpiration = Math.max(expiration, System.currentTimeMillis() + penalizationMillis); + peerTimeoutExpirations.put(status, Long.valueOf(newExpiration)); + } + + 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 PeerStatus getNextPeerStatus(final TransferDirection direction) { + List peerList = peerStatuses; + if ( (peerList == null || peerList.isEmpty() || System.currentTimeMillis() > peerRefreshTime + PEER_REFRESH_PERIOD) && peerRefreshLock.tryLock() ) { + try { + 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 createPeerStatusList(final TransferDirection direction) throws IOException, HandshakeException, UnknownPortException, PortNotRunningException { + final Set statuses = getPeerStatuses(); + if ( statuses == null ) { + return new ArrayList<>(); + } + + final ClusterNodeInformation clusterNodeInfo = new ClusterNodeInformation(); + final List nodeInfos = new ArrayList<>(); + for ( final PeerStatus peerStatus : statuses ) { + final NodeInformation nodeInfo = new NodeInformation(peerStatus.getHostname(), peerStatus.getPort(), 0, peerStatus.isSecure(), peerStatus.getFlowFileCount()); + nodeInfos.add(nodeInfo); + } + clusterNodeInfo.setNodeInformation(nodeInfos); + return formulateDestinationList(clusterNodeInfo, direction); + } + + + private Set 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 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 fetchRemotePeerStatuses() throws IOException, HandshakeException, UnknownPortException, PortNotRunningException { + final String hostname = clusterUrl.getHost(); + final int port = getSiteToSitePort(); + + final CommunicationsSession commsSession = establishSiteToSiteConnection(hostname, port); + final Peer peer = new Peer(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); + clientProtocol.handshake(peer, null); + final Set 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 statuses) { + if ( peersFile == null ) { + return; + } + + 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 recoverPersistedPeerStatuses(final File file) throws IOException { + if (!file.exists()) { + return null; + } + + final Set 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 CommunicationsSession establishSiteToSiteConnection(final PeerStatus peerStatus) throws IOException { + return establishSiteToSiteConnection(peerStatus.getHostname(), peerStatus.getPort()); + } + + private CommunicationsSession establishSiteToSiteConnection(final String hostname, final int port) throws IOException { + if ( siteToSiteSecure == null ) { + throw new IOException("Remote NiFi instance " + clusterUrl + " is not currently configured to accept site-to-site connections"); + } + + 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 formulateDestinationList(final ClusterNodeInformation clusterNodeInfo, final TransferDirection direction) { + final Collection nodeInfoSet = clusterNodeInfo.getNodeInformation(); + final int numDestinations = Math.max(128, nodeInfoSet.size()); + final Map 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 destinations = new ArrayList<>(totalEntries); + for (int i=0; i < totalEntries; i++) { + destinations.add(null); + } + for ( final Map.Entry 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 ) { + status = new PeerStatus(nodeInfo.getHostname(), nodeInfo.getSiteToSitePort(), nodeInfo.isSiteToSiteSecure(), 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 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 FlowFiles"); + } + logger.info(distributionDescription.toString()); + + // Jumble the list of destinations. + return destinations; + } + + + private void cleanupExpiredSockets() { + final List states = new ArrayList<>(); + + EndpointConnectionState state; + while ((state = connectionStateQueue.poll()) != null) { + // If the socket has not been used in 10 seconds, shut it down. + final long lastUsed = state.getLastTimeUsed(); + if ( lastUsed < System.currentTimeMillis() - 10000L ) { + try { + state.getSocketClientProtocol().shutdown(state.getPeer()); + } catch (final Exception e) { + logger.debug("Failed to shut down {} using {} due to {}", + new Object[] {state.getSocketClientProtocol(), state.getPeer(), e} ); + } + + cleanup(state.getSocketClientProtocol(), state.getPeer()); + } else { + states.add(state); + } + } + + connectionStateQueue.addAll(states); + } + + public void shutdown() { + taskExecutor.shutdown(); + peerTimeoutExpirations.clear(); + + for ( final CommunicationsSession commsSession : activeCommsChannels ) { + commsSession.interrupt(); + } + + EndpointConnectionState state; + while ( (state = connectionStateQueue.poll()) != null) { + cleanup(state.getSocketClientProtocol(), state.getPeer()); + } + } + + public void terminate(final EndpointConnectionState state) { + cleanup(state.getSocketClientProtocol(), state.getPeer()); + } + + private void refreshPeers() { + final PeerStatusCache existingCache = peerStatusCache; + if (existingCache != null && (existingCache.getTimestamp() + PEER_CACHE_MILLIS > System.currentTimeMillis())) { + return; + } + + try { + final Set 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 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 RemoteNiFiUtils utils = new RemoteNiFiUtils(webInterfaceSecure ? sslContext : null); + final ControllerDTO controller = utils.getController(URI.create(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 null 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; + } + + /** + * 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(); + return controller.isSiteToSiteSecure(); + } +} diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/SocketClient.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/SocketClient.java new file mode 100644 index 0000000000..0494d04fc8 --- /dev/null +++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/SocketClient.java @@ -0,0 +1,184 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.concurrent.TimeUnit; + +import org.apache.nifi.remote.RemoteDestination; +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.exception.HandshakeException; +import org.apache.nifi.remote.exception.PortNotRunningException; +import org.apache.nifi.remote.exception.ProtocolException; +import org.apache.nifi.remote.exception.UnknownPortException; +import org.apache.nifi.remote.protocol.DataPacket; +import org.apache.nifi.util.ObjectHolder; + +public class SocketClient implements SiteToSiteClient { + private final SiteToSiteClientConfig config; + private final EndpointConnectionStatePool pool; + private final boolean compress; + private final String portName; + private final long penalizationNanos; + private volatile String portIdentifier; + + public SocketClient(final SiteToSiteClientConfig config) { + pool = new EndpointConnectionStatePool(config.getUrl(), (int) config.getTimeout(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; + } + + if ( direction == TransferDirection.SEND ) { + return pool.getInputPortIdentifier(this.portName); + } else { + return pool.getOutputPortIdentifier(this.portName); + } + } + + + @Override + public Transaction createTransaction(final TransferDirection direction) throws IOException { + final String portId = getPortIdentifier(TransferDirection.SEND); + + if ( portId == null ) { + throw new IOException("Could not find Port with name " + portName + " for remote NiFi instance"); + } + + final RemoteDestination remoteDestination = new RemoteDestination() { + @Override + public String getIdentifier() { + return portId; + } + + @Override + public long getYieldPeriod(final TimeUnit timeUnit) { + return timeUnit.convert(penalizationNanos, TimeUnit.NANOSECONDS); + } + + @Override + public boolean isUseCompression() { + return compress; + } + }; + + final EndpointConnectionState connectionState; + try { + connectionState = pool.getEndpointConnectionState(remoteDestination, direction); + } catch (final ProtocolException | HandshakeException | PortNotRunningException | UnknownPortException e) { + throw new IOException(e); + } + + 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 connectionStateRef = new ObjectHolder<>(connectionState); + return new Transaction() { + @Override + public void confirm() throws IOException { + transaction.confirm(); + } + + @Override + public void complete(final boolean requestBackoff) throws IOException { + try { + transaction.complete(requestBackoff); + } finally { + final EndpointConnectionState 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 EndpointConnectionState state = connectionStateRef.get(); + if ( state != null ) { + pool.terminate(connectionState); + connectionStateRef.set(null); + } + } + } + + @Override + public void error() { + try { + transaction.error(); + } finally { + final EndpointConnectionState 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 DataPacket receive() throws IOException { + return transaction.receive(); + } + + @Override + public TransactionState getState() throws IOException { + return transaction.getState(); + } + + }; + } + + + @Override + public void close() throws IOException { + pool.shutdown(); + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/cluster/AdaptedNodeInformation.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/cluster/AdaptedNodeInformation.java similarity index 97% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/cluster/AdaptedNodeInformation.java rename to nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/cluster/AdaptedNodeInformation.java index 0092f7a5d5..6ca5812c6f 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/cluster/AdaptedNodeInformation.java +++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/cluster/AdaptedNodeInformation.java @@ -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 { diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/cluster/ClusterNodeInformation.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/cluster/ClusterNodeInformation.java similarity index 98% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/cluster/ClusterNodeInformation.java rename to nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/cluster/ClusterNodeInformation.java index 5751c32d39..1bc83b9a77 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/cluster/ClusterNodeInformation.java +++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/cluster/ClusterNodeInformation.java @@ -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; diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/cluster/NodeInformation.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/cluster/NodeInformation.java similarity index 98% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/cluster/NodeInformation.java rename to nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/cluster/NodeInformation.java index 848eb7e065..2041268693 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/cluster/NodeInformation.java +++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/cluster/NodeInformation.java @@ -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 { diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/cluster/NodeInformationAdapter.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/cluster/NodeInformationAdapter.java similarity index 94% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/cluster/NodeInformationAdapter.java rename to nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/cluster/NodeInformationAdapter.java index 630631fcc1..440463cbd7 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/cluster/NodeInformationAdapter.java +++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/cluster/NodeInformationAdapter.java @@ -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 { @Override diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/codec/FlowFileCodec.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/codec/FlowFileCodec.java similarity index 61% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/codec/FlowFileCodec.java rename to nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/codec/FlowFileCodec.java index b4206b318d..1380e1b50f 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/codec/FlowFileCodec.java +++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/codec/FlowFileCodec.java @@ -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; /** *

@@ -44,36 +43,29 @@ public interface FlowFileCodec extends VersionedRemoteResource { public List 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 null if the stream + * @return the DataPacket that was created, or null 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; } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/codec/StandardFlowFileCodec.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/codec/StandardFlowFileCodec.java similarity index 58% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/codec/StandardFlowFileCodec.java rename to nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/codec/StandardFlowFileCodec.java index d18a4ee9f2..6fd92de266 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/codec/StandardFlowFileCodec.java +++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/codec/StandardFlowFileCodec.java @@ -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 attributes = flowFile.getAttributes(); + final Map attributes = dataPacket.getAttributes(); out.writeInt(attributes.size()); for ( final Map.Entry entry : attributes.entrySet() ) { writeString(entry.getKey(), out); writeString(entry.getValue(), out); } - out.writeLong(flowFile.getSize()); + out.writeLong(dataPacket.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); - } - - encodedOut.flush(); - } - }); - - return flowFile; + final InputStream in = dataPacket.getData(); + StreamUtils.copy(in, encodedOut); + encodedOut.flush(); } @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,43 +81,16 @@ public class StandardFlowFileCodec implements FlowFileCodec { throw new ProtocolException("FlowFile exceeds maximum number of attributes with a total of " + numAttributes); } - try { - final Map attributes = new HashMap<>(numAttributes); - for (int i=0; i < numAttributes; i++) { - final String attrName = readString(in); - final String attrValue = readString(in); - attributes.put(attrName, attrValue); - } - - 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; + final Map attributes = new HashMap<>(numAttributes); + for (int i=0; i < numAttributes; i++) { + final String attrName = readString(in); + final String attrValue = readString(in); + attributes.put(attrName, attrValue); } + + final long numBytes = in.readLong(); + + return new StandardDataPacket(attributes, stream, numBytes); } private void writeString(final String val, final DataOutputStream out) throws IOException { diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/exception/HandshakeException.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/exception/HandshakeException.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/exception/HandshakeException.java rename to nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/exception/HandshakeException.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/exception/PortNotRunningException.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/exception/PortNotRunningException.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/exception/PortNotRunningException.java rename to nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/exception/PortNotRunningException.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/exception/ProtocolException.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/exception/ProtocolException.java similarity index 93% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/exception/ProtocolException.java rename to nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/exception/ProtocolException.java index 0f50b98808..e12348a700 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/exception/ProtocolException.java +++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/exception/ProtocolException.java @@ -16,7 +16,9 @@ */ package org.apache.nifi.remote.exception; -public class ProtocolException extends Exception { +import java.io.IOException; + +public class ProtocolException extends IOException { private static final long serialVersionUID = 5763900324505818495L; diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/exception/UnknownPortException.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/exception/UnknownPortException.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/exception/UnknownPortException.java rename to nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/exception/UnknownPortException.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelCommunicationsSession.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelCommunicationsSession.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelCommunicationsSession.java rename to nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelCommunicationsSession.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelInput.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelInput.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelInput.java rename to nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelInput.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelOutput.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelOutput.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelOutput.java rename to nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelOutput.java diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelCommunicationsSession.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelCommunicationsSession.java new file mode 100644 index 0000000000..dca1d847e5 --- /dev/null +++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelCommunicationsSession.java @@ -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.remote.io.socket.ssl; + +import java.io.IOException; + +import org.apache.nifi.remote.AbstractCommunicationsSession; + +public class SSLSocketChannelCommunicationsSession extends AbstractCommunicationsSession { + private final SSLSocketChannel channel; + private final SSLSocketChannelInput request; + private final SSLSocketChannelOutput response; + + public SSLSocketChannelCommunicationsSession(final SSLSocketChannel channel, final String uri) { + super(uri); + request = new SSLSocketChannelInput(channel); + response = new SSLSocketChannelOutput(channel); + this.channel = channel; + } + + @Override + public SSLSocketChannelInput getInput() { + return request; + } + + @Override + public SSLSocketChannelOutput getOutput() { + return response; + } + + @Override + public void setTimeout(final int millis) throws IOException { + channel.setTimeout(millis); + } + + @Override + public int getTimeout() throws IOException { + return channel.getTimeout(); + } + + @Override + public void close() throws IOException { + channel.close(); + } + + @Override + public boolean isClosed() { + return channel.isClosed(); + } + + @Override + public boolean isDataAvailable() { + try { + return request.isDataAvailable(); + } catch (final Exception e) { + return false; + } + } + + @Override + public long getBytesWritten() { + return response.getBytesWritten(); + } + + @Override + public long getBytesRead() { + return request.getBytesRead(); + } + + @Override + public void interrupt() { + channel.interrupt(); + } + + @Override + public String toString() { + return super.toString() + "[SSLSocketChannel=" + channel + "]"; + } +} diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelInput.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelInput.java new file mode 100644 index 0000000000..60ef33f24e --- /dev/null +++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelInput.java @@ -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.io.socket.ssl; + +import java.io.IOException; +import java.io.InputStream; + +import org.apache.nifi.stream.io.BufferedInputStream; +import org.apache.nifi.stream.io.ByteCountingInputStream; +import org.apache.nifi.remote.protocol.CommunicationsInput; + +public class SSLSocketChannelInput implements CommunicationsInput { + private final SSLSocketChannelInputStream in; + private final ByteCountingInputStream countingIn; + private final InputStream bufferedIn; + + public SSLSocketChannelInput(final SSLSocketChannel socketChannel) { + in = new SSLSocketChannelInputStream(socketChannel); + countingIn = new ByteCountingInputStream(in); + this.bufferedIn = new BufferedInputStream(countingIn); + } + + @Override + public InputStream getInputStream() throws IOException { + return bufferedIn; + } + + public boolean isDataAvailable() throws IOException { + return bufferedIn.available() > 0; + } + + @Override + public long getBytesRead() { + return countingIn.getBytesRead(); + } +} diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelOutput.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelOutput.java new file mode 100644 index 0000000000..dc3d68fbbd --- /dev/null +++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelOutput.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.remote.io.socket.ssl; + +import java.io.IOException; +import java.io.OutputStream; + +import org.apache.nifi.stream.io.BufferedOutputStream; +import org.apache.nifi.stream.io.ByteCountingOutputStream; +import org.apache.nifi.remote.protocol.CommunicationsOutput; + +public class SSLSocketChannelOutput implements CommunicationsOutput { + private final OutputStream out; + private final ByteCountingOutputStream countingOut; + + public SSLSocketChannelOutput(final SSLSocketChannel channel) { + countingOut = new ByteCountingOutputStream(new SSLSocketChannelOutputStream(channel)); + out = new BufferedOutputStream(countingOut); + } + + @Override + public OutputStream getOutputStream() throws IOException { + return out; + } + + @Override + public long getBytesWritten() { + return countingOut.getBytesWritten(); + } +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/protocol/ClientProtocol.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/ClientProtocol.java similarity index 92% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/protocol/ClientProtocol.java rename to nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/ClientProtocol.java index 32274eb9ac..befbdaa38c 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/protocol/ClientProtocol.java +++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/ClientProtocol.java @@ -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; @@ -46,6 +48,12 @@ public interface ClientProtocol extends VersionedRemoteResource { boolean isReadyForFileTransfer(); + + + + Transaction startTransaction(Peer peer, FlowFileCodec codec, TransferDirection direction) throws IOException; + + /** * returns true if remote instance indicates that the port is * invalid diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/protocol/CommunicationsInput.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/CommunicationsInput.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/protocol/CommunicationsInput.java rename to nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/CommunicationsInput.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/protocol/CommunicationsOutput.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/CommunicationsOutput.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/protocol/CommunicationsOutput.java rename to nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/CommunicationsOutput.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/protocol/CommunicationsSession.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/CommunicationsSession.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/protocol/CommunicationsSession.java rename to nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/CommunicationsSession.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/HandshakeProperty.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/DataPacket.java similarity index 79% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/HandshakeProperty.java rename to nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/DataPacket.java index c4519cd68e..f4fa4d02f3 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/HandshakeProperty.java +++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/DataPacket.java @@ -14,10 +14,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.nifi.remote.protocol.socket; +package org.apache.nifi.remote.protocol; -public enum HandshakeProperty { - GZIP, - PORT_IDENTIFIER, - REQUEST_EXPIRATION_MILLIS; +import java.io.InputStream; +import java.util.Map; + +public interface DataPacket { + + Map getAttributes(); + + InputStream getData(); + + long getSize(); } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/protocol/RequestType.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/RequestType.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/protocol/RequestType.java rename to nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/RequestType.java diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/HandshakeProperty.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/HandshakeProperty.java new file mode 100644 index 0000000000..41dc2763c7 --- /dev/null +++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/HandshakeProperty.java @@ -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; +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/Response.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/Response.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/Response.java rename to nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/Response.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/ResponseCode.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/ResponseCode.java similarity index 99% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/ResponseCode.java rename to nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/ResponseCode.java index 0e588cd35d..8860e730de 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/ResponseCode.java +++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/ResponseCode.java @@ -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 diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.java new file mode 100644 index 0000000000..5f194f8f9e --- /dev/null +++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.java @@ -0,0 +1,437 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.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 properties = new HashMap<>(); + properties.put(HandshakeProperty.GZIP, String.valueOf(useCompression)); + + if ( destinationId != null ) { + properties.put(HandshakeProperty.PORT_IDENTIFIER, destination.getIdentifier()); + } + + 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 + "/"; + } + } + + dos.writeInt(properties.size()); + for ( final Map.Entry 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 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 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(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(), peer, codec, + direction, useCompression, (int) destination.getYieldPeriod(TimeUnit.MILLISECONDS)); + } + + + @Override + public void 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 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.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 = ""; + } + + 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(); + + // We want to apply backpressure if the outgoing connections are full. I.e., there are no available relationships. + final boolean applyBackpressure = context.getAvailableRelationships().isEmpty(); + + transaction.complete(applyBackpressure); + logger.debug("{} Sending TRANSACTION_FINISHED_BUT_DESTINATION_FULL to {}", this, peer); + + if ( flowFilesReceived.isEmpty() ) { + return; + } + + 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 }); + } + + + @Override + public void transferFlowFiles(final Peer peer, final ProcessContext context, final ProcessSession session, final FlowFileCodec codec) throws IOException, ProtocolException { + FlowFile flowFile = session.get(); + if (flowFile == null) { + return; + } + + 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 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(false); + + 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}); + } 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 + "]"; + } +} diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientTransaction.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientTransaction.java new file mode 100644 index 0000000000..edb360eb86 --- /dev/null +++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientTransaction.java @@ -0,0 +1,357 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.OutputStream; +import java.util.zip.CRC32; +import java.util.zip.CheckedInputStream; +import java.util.zip.CheckedOutputStream; + +import org.apache.nifi.remote.Peer; +import org.apache.nifi.remote.Transaction; +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.protocol.DataPacket; +import org.apache.nifi.remote.protocol.RequestType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class SocketClientTransaction implements Transaction { + private static final Logger logger = LoggerFactory.getLogger(SocketClientTransaction.class); + + + 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 boolean dataAvailable = false; + private int transfers = 0; + private TransactionState state; + + SocketClientTransaction(final int protocolVersion, final Peer peer, final FlowFileCodec codec, + final TransferDirection direction, final boolean useCompression, final int penaltyMillis) throws IOException { + this.protocolVersion = protocolVersion; + 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 { + 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", 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 DataPacket packet = codec.decode(new CheckedInputStream(dis, crc)); + + if ( packet == null ) { + this.dataAvailable = false; + } else { + transfers++; + } + + this.state = TransactionState.DATA_EXCHANGED; + return packet; + } catch (final Exception e) { + error(); + throw e; + } + } + + + @Override + public void send(DataPacket dataPacket) throws IOException { + 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 out = new CheckedOutputStream(dos, 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++; + this.state = TransactionState.DATA_EXCHANGED; + } 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 ? "" : explanation); + state = TransactionState.TRANSACTION_CANCELED; + } catch (final IOException ioe) { + error(); + throw ioe; + } + } + + + @Override + public void complete(boolean requestBackoff) throws IOException { + 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); + } + + if ( direction == TransferDirection.RECEIVE ) { + if ( transfers == 0 ) { + state = TransactionState.TRANSACTION_COMPLETED; + return; + } + + if ( requestBackoff ) { + // Confirm that we received the data and the peer can now discard it but that the peer should not + // send any more data for a bit + logger.debug("{} Sending TRANSACTION_FINISHED_BUT_DESTINATION_FULL to {}", this, peer); + ResponseCode.TRANSACTION_FINISHED_BUT_DESTINATION_FULL.writeResponse(dos); + } else { + // Confirm that we received the data and the peer can now discard it + logger.debug("{} Sending TRANSACTION_FINISHED to {}", this, peer); + ResponseCode.TRANSACTION_FINISHED.writeResponse(dos); + } + + 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(penaltyMillis); + } else if ( transactionResponse.getCode() != ResponseCode.TRANSACTION_FINISHED ) { + throw new ProtocolException("After sending data, expected TRANSACTION_FINISHED response but got " + transactionResponse); + } + + state = TransactionState.TRANSACTION_COMPLETED; + } + } catch (final Exception e) { + error(); + throw e; + } + } + + + @Override + public void confirm() throws IOException { + 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 Exception e) { + error(); + throw e; + } + } + + @Override + public void error() { + this.state = TransactionState.ERROR; + } + + @Override + public TransactionState getState() { + return state; + } + +} diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/util/PeerStatusCache.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/util/PeerStatusCache.java new file mode 100644 index 0000000000..6dab77b399 --- /dev/null +++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/util/PeerStatusCache.java @@ -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 statuses; + private final long timestamp; + + public PeerStatusCache(final Set statuses) { + this(statuses, System.currentTimeMillis()); + } + + public PeerStatusCache(final Set statuses, final long timestamp) { + this.statuses = statuses; + this.timestamp = timestamp; + } + + public Set getStatuses() { + return statuses; + } + + public long getTimestamp() { + return timestamp; + } +} diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/util/RemoteNiFiUtils.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/util/RemoteNiFiUtils.java new file mode 100644 index 0000000000..b2dbdcddca --- /dev/null +++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/util/RemoteNiFiUtils.java @@ -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.util; + +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 queryParams) throws ClientHandlerException, UniformInterfaceException { + // perform the request + WebResource webResource = client.resource(uri); + if ( queryParams != null ) { + for ( final Map.Entry 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 null 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 null 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); + } +} diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/util/StandardDataPacket.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/util/StandardDataPacket.java new file mode 100644 index 0000000000..bd1b50c5a4 --- /dev/null +++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/util/StandardDataPacket.java @@ -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 attributes; + private final InputStream stream; + private final long size; + + public StandardDataPacket(final Map attributes, final InputStream stream, final long size) { + this.attributes = attributes; + this.stream = new MinimumLengthInputStream(new LimitingInputStream(stream, size), size); + this.size = size; + } + + public Map getAttributes() { + return attributes; + } + + public InputStream getData() { + return stream; + } + + public long getSize() { + return size; + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/test/java/org/apache/nifi/remote/TestStandardRemoteGroupPort.java b/nifi/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/socket/TestEndpointConnectionStatePool.java similarity index 82% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/test/java/org/apache/nifi/remote/TestStandardRemoteGroupPort.java rename to nifi/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/socket/TestEndpointConnectionStatePool.java index e074010db1..d8899ea686 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/test/java/org/apache/nifi/remote/TestStandardRemoteGroupPort.java +++ b/nifi/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/socket/TestEndpointConnectionStatePool.java @@ -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,7 +39,7 @@ public class TestStandardRemoteGroupPort { collection.add(new NodeInformation("ShouldGetMedium", 5, 5555, true, 4096)); clusterNodeInfo.setNodeInformation(collection); - final List destinations = StandardRemoteGroupPort.formulateDestinationList(clusterNodeInfo, ConnectableType.REMOTE_OUTPUT_PORT); + final List destinations = EndpointConnectionStatePool.formulateDestinationList(clusterNodeInfo, TransferDirection.SEND); for ( final PeerStatus peerStatus : destinations ) { System.out.println(peerStatus.getHostname() + ":" + peerStatus.getPort()); } @@ -55,7 +53,7 @@ public class TestStandardRemoteGroupPort { collection.add(new NodeInformation("ShouldGetLots", 2, 2222, true, 50000)); clusterNodeInfo.setNodeInformation(collection); - final List destinations = StandardRemoteGroupPort.formulateDestinationList(clusterNodeInfo, ConnectableType.REMOTE_OUTPUT_PORT); + final List destinations = EndpointConnectionStatePool.formulateDestinationList(clusterNodeInfo, TransferDirection.SEND); for ( final PeerStatus peerStatus : destinations ) { System.out.println(peerStatus.getHostname() + ":" + peerStatus.getPort()); } @@ -75,7 +73,7 @@ public class TestStandardRemoteGroupPort { collection.add(new NodeInformation("ShouldGetMedium", 5, 5555, true, 4096)); clusterNodeInfo.setNodeInformation(collection); - final List destinations = StandardRemoteGroupPort.formulateDestinationList(clusterNodeInfo, ConnectableType.REMOTE_INPUT_PORT); + final List destinations = EndpointConnectionStatePool.formulateDestinationList(clusterNodeInfo, TransferDirection.SEND); for ( final PeerStatus peerStatus : destinations ) { System.out.println(peerStatus.getHostname() + ":" + peerStatus.getPort()); } @@ -89,7 +87,7 @@ public class TestStandardRemoteGroupPort { collection.add(new NodeInformation("ShouldGetLittle", 2, 2222, true, 50000)); clusterNodeInfo.setNodeInformation(collection); - final List destinations = StandardRemoteGroupPort.formulateDestinationList(clusterNodeInfo, ConnectableType.REMOTE_INPUT_PORT); + final List destinations = EndpointConnectionStatePool.formulateDestinationList(clusterNodeInfo, TransferDirection.SEND); for ( final PeerStatus peerStatus : destinations ) { System.out.println(peerStatus.getHostname() + ":" + peerStatus.getPort()); } diff --git a/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/LimitingInputStream.java b/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/LimitingInputStream.java new file mode 100644 index 0000000000..421d579c2a --- /dev/null +++ b/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/LimitingInputStream.java @@ -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; + } +} diff --git a/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/MinimumLengthInputStream.java b/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/MinimumLengthInputStream.java new file mode 100644 index 0000000000..2e9359956c --- /dev/null +++ b/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/MinimumLengthInputStream.java @@ -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; + } + +} diff --git a/nifi/nifi-commons/pom.xml b/nifi/nifi-commons/pom.xml index ec0bb62096..768dfd99c7 100644 --- a/nifi/nifi-commons/pom.xml +++ b/nifi/nifi-commons/pom.xml @@ -35,5 +35,6 @@ nifi-web-utils nifi-processor-utilities nifi-write-ahead-log + nifi-site-to-site-client diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/pom.xml index 4a75a34482..3d9f3ace98 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/pom.xml @@ -73,6 +73,10 @@ org.apache.nifi nifi-site-to-site + + org.apache.nifi + nifi-site-to-site-client + org.apache.commons commons-compress diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/ClusterManager.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/ClusterManager.java index 0fcac8c18f..3a1dfb2195 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/ClusterManager.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/ClusterManager.java @@ -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. diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManager.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManager.java index 656759a82b..4d5455f55b 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManager.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManager.java @@ -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 diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/pom.xml index d088c5ce01..4b4536ebcb 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/pom.xml @@ -43,6 +43,10 @@ org.apache.nifi nifi-api + + org.apache.nifi + nifi-site-to-site-client + org.apache.commons commons-lang3 diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/RemoteProcessGroup.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/RemoteProcessGroup.java index e0cca64abb..ac41cba585 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/RemoteProcessGroup.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/RemoteProcessGroup.java @@ -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; @@ -27,7 +26,7 @@ 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; +import org.apache.nifi.remote.client.socket.EndpointConnectionStatePool; public interface RemoteProcessGroup { @@ -81,6 +80,8 @@ public interface RemoteProcessGroup { void setYieldDuration(final String yieldDuration); String getYieldDuration(); + + EndpointConnectionStatePool getConnectionPool(); /** * Sets the timeout using the TimePeriod format (e.g., "30 secs", "1 min") @@ -106,15 +107,6 @@ public interface RemoteProcessGroup { */ String getCommunicationsTimeout(); - /** - * @return the port that the remote instance is listening on for - * site-to-site communication, or null 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 +203,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 +219,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 getPeerStatuses(); } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/RemoteGroupPort.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/RemoteGroupPort.java index d4ad374ff5..f08277c7e9 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/RemoteGroupPort.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/RemoteGroupPort.java @@ -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(); } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/cluster/NodeInformant.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/cluster/NodeInformant.java similarity index 95% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/cluster/NodeInformant.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/cluster/NodeInformant.java index 987ff65cd5..e46ff5c12e 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/cluster/NodeInformant.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/cluster/NodeInformant.java @@ -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 { diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/protocol/ServerProtocol.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/protocol/ServerProtocol.java index 0d18f2e303..011853461f 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/protocol/ServerProtocol.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/protocol/ServerProtocol.java @@ -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; diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml index df5c845a94..1ff5fbe90b 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml @@ -63,6 +63,10 @@ org.apache.nifi nifi-client-dto + + org.apache.nifi + nifi-site-to-site-client + org.quartz-scheduler quartz diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java index 2c1b0857f2..db0aeb7b07 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java @@ -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,8 @@ 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.remote.client.socket.EndpointConnectionStatePool; +import org.apache.nifi.remote.util.RemoteNiFiUtils; import org.apache.nifi.reporting.BulletinRepository; import org.apache.nifi.reporting.Severity; import org.apache.nifi.util.FormatUtils; @@ -108,7 +84,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 +125,12 @@ 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 EndpointConnectionStatePool endpointConnectionPool; private final ScheduledExecutorService backgroundThreadExecutor; public StandardRemoteProcessGroup(final String id, final String targetUri, final ProcessGroup processGroup, @@ -200,72 +172,13 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup { } }; - final Runnable socketCleanup = new Runnable() { - @Override - public void run() { - final Set 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 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 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); - } - } - } - }; - + endpointConnectionPool = new EndpointConnectionStatePool(getTargetUri().toString(), getCommunicationsTimeout(TimeUnit.MILLISECONDS), + sslContext, eventReporter, getPeerPersistenceFile()); + 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); } @Override @@ -287,6 +200,7 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup { @Override public void shutdown() { backgroundThreadExecutor.shutdown(); + endpointConnectionPool.shutdown(); } @Override @@ -858,7 +772,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 +912,6 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup { return descriptor; } - /** - * @return the port that the remote instance is listening on for - * site-to-site communication, or null 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 +1136,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 +1147,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(); @@ -1385,6 +1220,11 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup { public String getYieldDuration() { return yieldDuration; } + + @Override + public EndpointConnectionStatePool getConnectionPool() { + return endpointConnectionPool; + } @Override public void verifyCanDelete() { @@ -1487,135 +1327,9 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup { } } - @Override - public Set 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 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 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 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 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 recoverPersistedPeerStatuses(final File file) throws IOException { - if (!file.exists()) { - return null; - } - - final Set 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 statuses; - private final long timestamp; - - public PeerStatusCache(final Set statuses) { - this(statuses, System.currentTimeMillis()); - } - - public PeerStatusCache(final Set statuses, final long timestamp) { - this.statuses = statuses; - this.timestamp = timestamp; - } - - public Set getStatuses() { - return statuses; - } - - public long getTimestamp() { - return timestamp; - } - } } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/.gitignore b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/.gitignore index ea8c4bf7f3..d9d66d8144 100755 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/.gitignore +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/.gitignore @@ -1 +1,2 @@ /target +/bin/ diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/controller/util/RemoteProcessGroupUtils.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/controller/util/RemoteProcessGroupUtils.class new file mode 100644 index 0000000000..a6951d4f6d Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/controller/util/RemoteProcessGroupUtils.class differ diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/AbstractCommunicationsSession.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/AbstractCommunicationsSession.class new file mode 100644 index 0000000000..2e868ea92f Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/AbstractCommunicationsSession.class differ diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/RemoteResourceFactory.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/RemoteResourceFactory.class new file mode 100644 index 0000000000..9c6e821b64 Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/RemoteResourceFactory.class differ diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/RemoteResourceManager.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/RemoteResourceManager.class new file mode 100644 index 0000000000..bb3fc776c1 Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/RemoteResourceManager.class differ diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/RemoteSiteListener.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/RemoteSiteListener.class new file mode 100644 index 0000000000..a56b5ba135 Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/RemoteSiteListener.class differ diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/SocketRemoteSiteListener.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/SocketRemoteSiteListener.class new file mode 100644 index 0000000000..9780f75671 Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/SocketRemoteSiteListener.class differ diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort$EndpointConnectionState.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort$EndpointConnectionState.class new file mode 100644 index 0000000000..f184d643c6 Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort$EndpointConnectionState.class differ diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.class new file mode 100644 index 0000000000..0740bb4500 Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.class differ diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRootGroupPort$FlowFileRequest.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRootGroupPort$FlowFileRequest.class new file mode 100644 index 0000000000..d517458b5c Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRootGroupPort$FlowFileRequest.class differ diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRootGroupPort$ProcessingResult.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRootGroupPort$ProcessingResult.class new file mode 100644 index 0000000000..1cf5ceb6b6 Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRootGroupPort$ProcessingResult.class differ diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRootGroupPort$StandardPortAuthorizationResult.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRootGroupPort$StandardPortAuthorizationResult.class new file mode 100644 index 0000000000..3ad7542431 Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRootGroupPort$StandardPortAuthorizationResult.class differ diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRootGroupPort.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRootGroupPort.class new file mode 100644 index 0000000000..4db4735a92 Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRootGroupPort.class differ diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/codec/StandardFlowFileCodec.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/codec/StandardFlowFileCodec.class new file mode 100644 index 0000000000..e49ffe8e68 Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/codec/StandardFlowFileCodec.class differ diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/exception/UnsupportedCodecException.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/exception/UnsupportedCodecException.class new file mode 100644 index 0000000000..39dd49a3c4 Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/exception/UnsupportedCodecException.class differ diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelCommunicationsSession.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelCommunicationsSession.class new file mode 100644 index 0000000000..b415421c4c Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelCommunicationsSession.class differ diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelInput.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelInput.class new file mode 100644 index 0000000000..551097ec55 Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelInput.class differ diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelOutput.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelOutput.class new file mode 100644 index 0000000000..6913767a10 Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelOutput.class differ diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelCommunicationsSession.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelCommunicationsSession.class new file mode 100644 index 0000000000..f6e9f20cb1 Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelCommunicationsSession.class differ diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelInput.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelInput.class new file mode 100644 index 0000000000..41fe366305 Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelInput.class differ diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelOutput.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelOutput.class new file mode 100644 index 0000000000..9b9cdc0182 Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelOutput.class differ diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/ClusterManagerServerProtocol.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/ClusterManagerServerProtocol.class new file mode 100644 index 0000000000..2ef1c39a92 Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/ClusterManagerServerProtocol.class differ diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/HandshakeProperty.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/HandshakeProperty.class new file mode 100644 index 0000000000..fad8245ea1 Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/HandshakeProperty.class differ diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/Response.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/Response.class new file mode 100644 index 0000000000..27ec8d3236 Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/Response.class differ diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/ResponseCode.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/ResponseCode.class new file mode 100644 index 0000000000..4673aec38f Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/ResponseCode.class differ diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.class new file mode 100644 index 0000000000..ac7e1b9535 Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.class differ diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/SocketFlowFileServerProtocol.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/SocketFlowFileServerProtocol.class new file mode 100644 index 0000000000..933ef4d29a Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/SocketFlowFileServerProtocol.class differ diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/test/java/org/apache/nifi/remote/TestStandardRemoteGroupPort.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/test/java/org/apache/nifi/remote/TestStandardRemoteGroupPort.class new file mode 100644 index 0000000000..0e77276519 Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/test/java/org/apache/nifi/remote/TestStandardRemoteGroupPort.class differ diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/pom.xml index f989f66848..08bf5902ba 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/pom.xml @@ -59,6 +59,10 @@ org.apache.nifi nifi-utils + + org.apache.nifi + nifi-site-to-site-client + org.apache.httpcomponents httpclient diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/RemoteResourceFactory.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/RemoteResourceFactory.java index 49d3c3c793..2b27de27fe 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/RemoteResourceFactory.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/RemoteResourceFactory.java @@ -26,48 +26,8 @@ import org.apache.nifi.remote.exception.HandshakeException; import org.apache.nifi.remote.protocol.ClientProtocol; import org.apache.nifi.remote.protocol.ServerProtocol; -public class RemoteResourceFactory { +public class RemoteResourceFactory extends RemoteResourceInitiator { - public static final int RESOURCE_OK = 20; - public static final int DIFFERENT_RESOURCE_VERSION = 21; - public static final int ABORT = 255; - - - 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 - dos.writeUTF(resource.getResourceName()); - final VersionNegotiator negotiator = resource.getVersionNegotiator(); - dos.writeInt(negotiator.getVersion()); - dos.flush(); - - // wait for response from server. - final int statusCode = dis.read(); - switch (statusCode) { - case RESOURCE_OK: // server accepted our proposal of codec name/version - return resource; - case DIFFERENT_RESOURCE_VERSION: // server accepted our proposal of codec name but not the 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: - throw new HandshakeException("Remote destination aborted connection with message: " + dis.readUTF()); - default: - return null; // Unable to negotiate codec - } - } - - @SuppressWarnings("unchecked") public static T receiveCodecNegotiation(final DataInputStream dis, final DataOutputStream dos) throws IOException, HandshakeException { final String codecName = dis.readUTF(); @@ -96,6 +56,14 @@ public class RemoteResourceFactory { } } + public static void rejectCodecNegotiation(final DataInputStream dis, final DataOutputStream dos, final String explanation) throws IOException { + dis.readUTF(); // read codec name + dis.readInt(); // read codec version + + dos.write(ABORT); + dos.writeUTF(explanation); + dos.flush(); + } @SuppressWarnings("unchecked") public static T receiveClientProtocolNegotiation(final DataInputStream dis, final DataOutputStream dos) throws IOException, HandshakeException { diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/SocketRemoteSiteListener.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/SocketRemoteSiteListener.java index ec169ad570..3295956a75 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/SocketRemoteSiteListener.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/SocketRemoteSiteListener.java @@ -35,8 +35,8 @@ import java.util.concurrent.atomic.AtomicReference; import javax.net.ssl.SSLContext; -import org.apache.nifi.cluster.NodeInformant; import org.apache.nifi.groups.ProcessGroup; +import org.apache.nifi.remote.cluster.NodeInformant; import org.apache.nifi.remote.exception.HandshakeException; import org.apache.nifi.remote.io.socket.SocketChannelCommunicationsSession; import org.apache.nifi.remote.io.socket.ssl.SSLSocketChannel; @@ -122,6 +122,9 @@ public class SocketRemoteSiteListener implements RemoteSiteListener { } LOG.trace("Got connection"); + if ( stopped.get() ) { + return; + } final Socket socket = acceptedSocket; final SocketChannel socketChannel = socket.getChannel(); final Thread thread = new Thread(new Runnable() { @@ -198,7 +201,7 @@ public class SocketRemoteSiteListener implements RemoteSiteListener { protocol.setRootProcessGroup(rootGroup.get()); protocol.setNodeInformant(nodeInformant); - peer = new Peer(commsSession, peerUri); + peer = new Peer(commsSession, peerUri, "nifi://localhost:" + getPort()); LOG.debug("Handshaking...."); protocol.handshake(peer); diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.java index b0d88d4695..a51cdba3c6 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.java @@ -16,90 +16,59 @@ */ package org.apache.nifi.remote; -import java.io.DataInputStream; -import java.io.DataOutputStream; import java.io.IOException; -import java.net.InetSocketAddress; -import java.net.URI; -import java.net.URISyntaxException; -import java.nio.channels.SocketChannel; import java.util.ArrayList; import java.util.Collection; -import java.util.HashMap; import java.util.HashSet; -import java.util.List; -import java.util.Map; import java.util.Set; -import java.util.concurrent.BlockingQueue; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; import javax.net.ssl.SSLContext; -import javax.security.cert.CertificateExpiredException; -import javax.security.cert.CertificateNotYetValidException; -import org.apache.nifi.cluster.ClusterNodeInformation; -import org.apache.nifi.cluster.NodeInformation; import org.apache.nifi.components.ValidationResult; import org.apache.nifi.connectable.ConnectableType; import org.apache.nifi.connectable.Connection; -import org.apache.nifi.controller.AbstractPort; import org.apache.nifi.controller.ProcessScheduler; import org.apache.nifi.groups.ProcessGroup; import org.apache.nifi.groups.RemoteProcessGroup; import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.Relationship; +import org.apache.nifi.remote.client.socket.EndpointConnectionState; +import org.apache.nifi.remote.client.socket.EndpointConnectionStatePool; import org.apache.nifi.remote.codec.FlowFileCodec; -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.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.ClientProtocol; import org.apache.nifi.remote.protocol.CommunicationsSession; import org.apache.nifi.remote.protocol.socket.SocketClientProtocol; import org.apache.nifi.reporting.Severity; import org.apache.nifi.scheduling.SchedulingStrategy; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.sun.jersey.api.client.ClientHandlerException; - -public class StandardRemoteGroupPort extends AbstractPort implements RemoteGroupPort { +public class StandardRemoteGroupPort extends RemoteGroupPort { public static final String USER_AGENT = "NiFi-Site-to-Site"; public static final String CONTENT_TYPE = "application/octet-stream"; public static final int GZIP_COMPRESSION_LEVEL = 1; - public static final long PEER_REFRESH_PERIOD = 60000L; private static final String CATEGORY = "Site to Site"; private static final Logger logger = LoggerFactory.getLogger(StandardRemoteGroupPort.class); private final RemoteProcessGroup remoteGroup; - private final SSLContext sslContext; private final AtomicBoolean useCompression = new AtomicBoolean(false); private final AtomicBoolean targetExists = new AtomicBoolean(true); private final AtomicBoolean targetRunning = new AtomicBoolean(true); - private final AtomicLong peerIndex = new AtomicLong(0L); + private final TransferDirection transferDirection; - private volatile List peerStatuses; - private volatile long peerRefreshTime = 0L; - private final ReentrantLock peerRefreshLock = new ReentrantLock(); - - private final ConcurrentMap> endpointConnectionMap = new ConcurrentHashMap<>(); - private final ConcurrentMap peerTimeoutExpirations = new ConcurrentHashMap<>(); + private final EndpointConnectionStatePool connectionStatePool; private final Set activeCommsChannels = new HashSet<>(); private final Lock interruptLock = new ReentrantLock(); @@ -113,8 +82,10 @@ public class StandardRemoteGroupPort extends AbstractPort implements RemoteGroup super(id, name, processGroup, type, scheduler); this.remoteGroup = remoteGroup; - this.sslContext = sslContext; + this.transferDirection = direction; setScheduldingPeriod(MINIMUM_SCHEDULING_NANOS + " nanos"); + + connectionStatePool = remoteGroup.getConnectionPool(); } @Override @@ -133,25 +104,10 @@ public class StandardRemoteGroupPort extends AbstractPort implements RemoteGroup @Override public void shutdown() { - super.shutdown(); - - peerTimeoutExpirations.clear(); + super.shutdown(); interruptLock.lock(); try { this.shutdown = true; - - for ( final CommunicationsSession commsSession : activeCommsChannels ) { - commsSession.interrupt(); - } - - for ( final BlockingQueue queue : endpointConnectionMap.values() ) { - EndpointConnectionState state; - while ( (state = queue.poll()) != null) { - cleanup(state.getSocketClientProtocol(), state.getPeer()); - } - } - - endpointConnectionMap.clear(); } finally { interruptLock.unlock(); } @@ -170,35 +126,6 @@ public class StandardRemoteGroupPort extends AbstractPort implements RemoteGroup } - void cleanupSockets() { - final List states = new ArrayList<>(); - - for ( final BlockingQueue queue : endpointConnectionMap.values() ) { - states.clear(); - - EndpointConnectionState state; - while ((state = queue.poll()) != null) { - // If the socket has not been used in 10 seconds, shut it down. - final long lastUsed = state.getLastTimeUsed(); - if ( lastUsed < System.currentTimeMillis() - 10000L ) { - try { - state.getSocketClientProtocol().shutdown(state.getPeer()); - } catch (final Exception e) { - logger.debug("Failed to shut down {} using {} due to {}", - new Object[] {state.getSocketClientProtocol(), state.getPeer(), e} ); - } - - cleanup(state.getSocketClientProtocol(), state.getPeer()); - } else { - states.add(state); - } - } - - queue.addAll(states); - } - } - - @Override public void onTrigger(final ProcessContext context, final ProcessSession session) { if ( !remoteGroup.isTransmitting() ) { @@ -212,137 +139,46 @@ public class StandardRemoteGroupPort extends AbstractPort implements RemoteGroup } String url = getRemoteProcessGroup().getTargetUri().toString(); - Peer peer = null; - final PeerStatus peerStatus = getNextPeerStatus(); - if ( peerStatus == null ) { + + final EndpointConnectionState connectionState; + try { + connectionState = connectionStatePool.getEndpointConnectionState(this, transferDirection); + } catch (final PortNotRunningException e) { + context.yield(); + this.targetRunning.set(false); + final String message = String.format("%s failed to communicate with %s because the remote instance indicates that the port is not in a valid state", this, url); + logger.error(message); + remoteGroup.getEventReporter().reportEvent(Severity.ERROR, CATEGORY, message); + return; + } catch (final UnknownPortException e) { + context.yield(); + this.targetExists.set(false); + final String message = String.format("%s failed to communicate with %s because the remote instance indicates that the port no longer exists", this, url); + logger.error(message); + remoteGroup.getEventReporter().reportEvent(Severity.ERROR, CATEGORY, message); + return; + } catch (final HandshakeException | IOException e) { + final String message = String.format("%s failed to communicate with %s due to %s", this, url, e.toString()); + logger.error(message); + if ( logger.isDebugEnabled() ) { + logger.error("", e); + } + remoteGroup.getEventReporter().reportEvent(Severity.ERROR, CATEGORY, message); + session.rollback(); + return; + } + + if ( connectionState == null ) { logger.debug("{} Unable to determine the next peer to communicate with; all peers must be penalized, so yielding context", this); context.yield(); return; } - url = "nifi://" + peerStatus.getHostname() + ":" + peerStatus.getPort(); + FlowFileCodec codec = connectionState.getCodec(); + SocketClientProtocol protocol = connectionState.getSocketClientProtocol(); + final Peer peer = connectionState.getPeer(); + url = peer.getUrl(); - // - // Attempt to get a connection state that already exists for this URL. - // - BlockingQueue connectionStateQueue = endpointConnectionMap.get(url); - if ( connectionStateQueue == null ) { - connectionStateQueue = new LinkedBlockingQueue<>(); - BlockingQueue existingQueue = endpointConnectionMap.putIfAbsent(url, connectionStateQueue); - if ( existingQueue != null ) { - connectionStateQueue = existingQueue; - } - } - - FlowFileCodec codec = null; - CommunicationsSession commsSession = null; - SocketClientProtocol protocol = null; - EndpointConnectionState connectionState; - - do { - connectionState = connectionStateQueue.poll(); - logger.debug("{} Connection State for {} = {}", this, url, connectionState); - - // if we can't get an existing ConnectionState, create one - if ( connectionState == null ) { - protocol = new SocketClientProtocol(); - protocol.setPort(this); - - try { - commsSession = establishSiteToSiteConnection(peerStatus); - final DataInputStream dis = new DataInputStream(commsSession.getInput().getInputStream()); - final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream()); - try { - RemoteResourceFactory.initiateResourceNegotiation(protocol, dis, dos); - } catch (final HandshakeException e) { - try { - commsSession.close(); - } catch (final IOException ioe) { - final String message = String.format("%s unable to close communications session %s due to %s; resources may not be appropriately cleaned up", - this, commsSession, ioe.toString()); - logger.error(message); - remoteGroup.getEventReporter().reportEvent(Severity.ERROR, CATEGORY, message); - } - } - } catch (final IOException e) { - final String message = String.format("%s failed to communicate with %s due to %s", this, peer == null ? url : peer, e.toString()); - logger.error(message); - if ( logger.isDebugEnabled() ) { - logger.error("", e); - } - remoteGroup.getEventReporter().reportEvent(Severity.ERROR, CATEGORY, message); - session.rollback(); - return; - } - - - peer = new Peer(commsSession, url); - - // perform handshake - try { - protocol.handshake(peer); - - // handle error cases - if ( protocol.isDestinationFull() ) { - logger.warn("{} {} indicates that port's destination is full; penalizing peer", this, peer); - penalize(peer); - cleanup(protocol, peer); - return; - } else if ( protocol.isPortInvalid() ) { - penalize(peer); - context.yield(); - cleanup(protocol, peer); - this.targetRunning.set(false); - final String message = String.format("%s failed to communicate with %s because the remote instance indicates that the port is not in a valid state", this, peer); - logger.error(message); - remoteGroup.getEventReporter().reportEvent(Severity.ERROR, CATEGORY, message); - return; - } else if ( protocol.isPortUnknown() ) { - penalize(peer); - context.yield(); - cleanup(protocol, peer); - this.targetExists.set(false); - final String message = String.format("%s failed to communicate with %s because the remote instance indicates that the port no longer exists", this, peer); - logger.error(message); - remoteGroup.getEventReporter().reportEvent(Severity.ERROR, CATEGORY, message); - return; - } - - // negotiate the FlowFileCodec to use - codec = protocol.negotiateCodec(peer); - } catch (final Exception e) { - penalize(peer); - cleanup(protocol, peer); - - final String message = String.format("%s failed to communicate with %s due to %s", this, peer == null ? url : peer, e.toString()); - logger.error(message); - if ( logger.isDebugEnabled() ) { - logger.error("", e); - } - remoteGroup.getEventReporter().reportEvent(Severity.ERROR, CATEGORY, message); - session.rollback(); - return; - } - - connectionState = new EndpointConnectionState(peer, protocol, codec); - } else { - final long lastTimeUsed = connectionState.getLastTimeUsed(); - final long millisSinceLastUse = System.currentTimeMillis() - lastTimeUsed; - final long timeoutMillis = remoteGroup.getCommunicationsTimeout(TimeUnit.MILLISECONDS); - - if ( timeoutMillis > 0L && millisSinceLastUse >= timeoutMillis ) { - cleanup(connectionState.getSocketClientProtocol(), connectionState.getPeer()); - connectionState = null; - } else { - codec = connectionState.getCodec(); - peer = connectionState.getPeer(); - commsSession = peer.getCommunicationsSession(); - protocol = connectionState.getSocketClientProtocol(); - } - } - } while ( connectionState == null || codec == null || commsSession == null || protocol == null ); - - try { interruptLock.lock(); try { @@ -361,11 +197,6 @@ public class StandardRemoteGroupPort extends AbstractPort implements RemoteGroup receiveFlowFiles(peer, protocol, context, session, codec); } - if ( peer.isPenalized() ) { - logger.debug("{} {} was penalized", this, peer); - penalize(peer); - } - interruptLock.lock(); try { if ( shutdown ) { @@ -380,12 +211,12 @@ public class StandardRemoteGroupPort extends AbstractPort implements RemoteGroup session.commit(); connectionState.setLastTimeUsed(); - connectionStateQueue.add(connectionState); + connectionStatePool.offer(connectionState); } catch (final TransmissionDisabledException e) { cleanup(protocol, peer); session.rollback(); } catch (final Exception e) { - penalize(peer); + connectionStatePool.penalize(peer, getYieldPeriod(TimeUnit.MILLISECONDS)); final String message = String.format("%s failed to communicate with %s (%s) due to %s", this, peer == null ? url : peer, protocol, e.toString()); logger.error(message); @@ -401,34 +232,6 @@ public class StandardRemoteGroupPort extends AbstractPort implements RemoteGroup } - /** - * Updates internal state map to penalize a PeerStatus that points to the specified peer - * @param peer - */ - private void penalize(final Peer peer) { - String host; - int port; - try { - final URI uri = new URI(peer.getUrl()); - host = uri.getHost(); - port = uri.getPort(); - } catch (final URISyntaxException e) { - host = peer.getHost(); - port = -1; - } - - final PeerStatus status = new PeerStatus(host, port, true, 1); - Long expiration = peerTimeoutExpirations.get(status); - if ( expiration == null ) { - expiration = Long.valueOf(0L); - } - - final long penalizationMillis = getYieldPeriod(TimeUnit.MILLISECONDS); - final long newExpiration = Math.max(expiration, System.currentTimeMillis() + penalizationMillis); - peerTimeoutExpirations.put(status, Long.valueOf(newExpiration)); - } - - private void cleanup(final SocketClientProtocol protocol, final Peer peer) { if ( protocol != null && peer != null ) { try { @@ -457,108 +260,6 @@ public class StandardRemoteGroupPort extends AbstractPort implements RemoteGroup return remoteGroup.getYieldDuration(); } - public CommunicationsSession establishSiteToSiteConnection(final PeerStatus peerStatus) throws IOException { - final String destinationUri = "nifi://" + peerStatus.getHostname() + ":" + peerStatus.getPort(); - - CommunicationsSession commsSession = null; - try { - if ( peerStatus.isSecure() ) { - if ( sslContext == null ) { - throw new IOException("Unable to communicate with " + peerStatus.getHostname() + ":" + peerStatus.getPort() + " because it requires Secure Site-to-Site communications, but this instance is not configured for secure communications"); - } - - final SSLSocketChannel socketChannel = new SSLSocketChannel(sslContext, peerStatus.getHostname(), peerStatus.getPort(), 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(peerStatus.getHostname(), peerStatus.getPort())); - 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; - } - - private PeerStatus getNextPeerStatus() { - List peerList = peerStatuses; - if ( (peerList == null || peerList.isEmpty() || System.currentTimeMillis() > peerRefreshTime + PEER_REFRESH_PERIOD) && peerRefreshLock.tryLock() ) { - try { - try { - peerList = createPeerStatusList(); - } catch (final IOException | BadRequestException | HandshakeException | UnknownPortException | PortNotRunningException | ClientHandlerException 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); - } - remoteGroup.getEventReporter().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 createPeerStatusList() throws IOException, BadRequestException, HandshakeException, UnknownPortException, PortNotRunningException { - final Set statuses = remoteGroup.getPeerStatuses(); - if ( statuses == null ) { - return new ArrayList<>(); - } - - final ClusterNodeInformation clusterNodeInfo = new ClusterNodeInformation(); - final List nodeInfos = new ArrayList<>(); - for ( final PeerStatus peerStatus : statuses ) { - final NodeInformation nodeInfo = new NodeInformation(peerStatus.getHostname(), peerStatus.getPort(), 0, peerStatus.isSecure(), peerStatus.getFlowFileCount()); - nodeInfos.add(nodeInfo); - } - clusterNodeInfo.setNodeInformation(nodeInfos); - return formulateDestinationList(clusterNodeInfo); - } private void transferFlowFiles(final Peer peer, final ClientProtocol protocol, final ProcessContext context, final ProcessSession session, final FlowFileCodec codec) throws IOException, ProtocolException { protocol.transferFlowFiles(peer, context, session, codec); @@ -568,70 +269,6 @@ public class StandardRemoteGroupPort extends AbstractPort implements RemoteGroup protocol.receiveFlowFiles(peer, context, session, codec); } - private List formulateDestinationList(final ClusterNodeInformation clusterNodeInfo) throws IOException { - return formulateDestinationList(clusterNodeInfo, getConnectableType()); - } - - static List formulateDestinationList(final ClusterNodeInformation clusterNodeInfo, final ConnectableType connectableType) { - final Collection nodeInfoSet = clusterNodeInfo.getNodeInformation(); - final int numDestinations = Math.max(128, nodeInfoSet.size()); - final Map 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 = (connectableType == ConnectableType.REMOTE_INPUT_PORT) ? (1 - percentageOfFlowFiles) : percentageOfFlowFiles; - final int entries = Math.max(1, (int) (numDestinations * relativeWeighting)); - - entryCountMap.put(nodeInfo, Math.max(1, entries)); - totalEntries += entries; - } - - final List destinations = new ArrayList<>(totalEntries); - for (int i=0; i < totalEntries; i++) { - destinations.add(null); - } - for ( final Map.Entry 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 ) { - status = new PeerStatus(nodeInfo.getHostname(), nodeInfo.getSiteToSitePort(), nodeInfo.isSiteToSiteSecure(), 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 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 FlowFiles"); - } - logger.info(distributionDescription.toString()); - - // Jumble the list of destinations. - return destinations; - } - - @Override public boolean getTargetExists() { return targetExists.get(); @@ -717,40 +354,6 @@ public class StandardRemoteGroupPort extends AbstractPort implements RemoteGroup } - private static class EndpointConnectionState { - private final Peer peer; - private final SocketClientProtocol socketClientProtocol; - private final FlowFileCodec codec; - private volatile long lastUsed; - - private EndpointConnectionState(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; - } - } - - @Override public SchedulingStrategy getSchedulingStrategy() { return SchedulingStrategy.TIMER_DRIVEN; diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/ClusterManagerServerProtocol.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/ClusterManagerServerProtocol.java index a526f4c364..391d52b55d 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/ClusterManagerServerProtocol.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/ClusterManagerServerProtocol.java @@ -21,9 +21,6 @@ import java.io.DataOutputStream; import java.io.IOException; import java.util.Collection; -import org.apache.nifi.cluster.ClusterNodeInformation; -import org.apache.nifi.cluster.NodeInformant; -import org.apache.nifi.cluster.NodeInformation; import org.apache.nifi.groups.ProcessGroup; import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; @@ -31,12 +28,14 @@ import org.apache.nifi.remote.Peer; import org.apache.nifi.remote.RootGroupPort; import org.apache.nifi.remote.StandardVersionNegotiator; import org.apache.nifi.remote.VersionNegotiator; +import org.apache.nifi.remote.cluster.ClusterNodeInformation; +import org.apache.nifi.remote.cluster.NodeInformant; +import org.apache.nifi.remote.cluster.NodeInformation; import org.apache.nifi.remote.codec.FlowFileCodec; import org.apache.nifi.remote.exception.HandshakeException; import org.apache.nifi.remote.protocol.CommunicationsSession; import org.apache.nifi.remote.protocol.RequestType; import org.apache.nifi.remote.protocol.ServerProtocol; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.java deleted file mode 100644 index d4b4f61542..0000000000 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.java +++ /dev/null @@ -1,510 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.remote.protocol.socket; - -import java.io.DataInputStream; -import java.io.DataOutputStream; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Map; -import java.util.Set; -import java.util.UUID; -import java.util.concurrent.TimeUnit; -import java.util.zip.CRC32; -import java.util.zip.CheckedInputStream; -import java.util.zip.CheckedOutputStream; - -import org.apache.nifi.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.remote.Peer; -import org.apache.nifi.remote.PeerStatus; -import org.apache.nifi.remote.RemoteGroupPort; -import org.apache.nifi.remote.RemoteResourceFactory; -import org.apache.nifi.remote.StandardVersionNegotiator; -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.io.CompressionInputStream; -import org.apache.nifi.remote.io.CompressionOutputStream; -import org.apache.nifi.remote.protocol.ClientProtocol; -import org.apache.nifi.remote.protocol.CommunicationsSession; -import org.apache.nifi.remote.protocol.RequestType; -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(4, 3, 2, 1); - - - private RemoteGroupPort port; - private boolean useCompression; - - 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 static final long BATCH_SEND_NANOS = TimeUnit.SECONDS.toNanos(5L); // send batches of up to 5 seconds - - public SocketClientProtocol() { - } - - public void setPort(final RemoteGroupPort port) { - this.port = port; - this.useCompression = port.isUseCompression(); - } - - @Override - public void handshake(final Peer peer) 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 properties = new HashMap<>(); - properties.put(HandshakeProperty.GZIP, String.valueOf(useCompression)); - properties.put(HandshakeProperty.PORT_IDENTIFIER, port.getIdentifier()); - properties.put(HandshakeProperty.REQUEST_EXPIRATION_MILLIS, String.valueOf( - port.getRemoteProcessGroup().getCommunicationsTimeout(TimeUnit.MILLISECONDS)) ); - - final CommunicationsSession commsSession = peer.getCommunicationsSession(); - commsSession.setTimeout(port.getRemoteProcessGroup().getCommunicationsTimeout(TimeUnit.MILLISECONDS)); - 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 + "/"; - } - } - - dos.writeInt(properties.size()); - for ( final Map.Entry 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 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 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(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) RemoteResourceFactory.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 void receiveFlowFiles(final Peer peer, final ProcessContext context, final ProcessSession session, final FlowFileCodec codec) throws IOException, ProtocolException { - if ( !handshakeComplete ) { - throw new IllegalStateException("Handshake has not been performed"); - } - if ( !readyForFileTransfer ) { - throw new IllegalStateException("Cannot receive files; handshake resolution was " + handshakeResponse); - } - - logger.debug("{} Receiving FlowFiles from {}", this, peer); - final CommunicationsSession commsSession = peer.getCommunicationsSession(); - final DataInputStream dis = new DataInputStream(commsSession.getInput().getInputStream()); - final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream()); - String userDn = commsSession.getUserDn(); - if ( userDn == null ) { - userDn = "none"; - } - - // Indicate that we would like to have some data - RequestType.RECEIVE_FLOWFILES.writeRequestType(dos); - dos.flush(); - - // Determine if Peer will send us data or has no data to send us - final Response dataAvailableCode = Response.read(dis); - switch (dataAvailableCode.getCode()) { - case MORE_DATA: - logger.debug("{} {} Indicates that data is available", this, peer); - break; - case NO_MORE_DATA: - logger.debug("{} No data available from {}", peer); - return; - default: - throw new ProtocolException("Got unexpected response when asking for data: " + dataAvailableCode); - } - - final StopWatch stopWatch = new StopWatch(true); - final Set flowFilesReceived = new HashSet<>(); - long bytesReceived = 0L; - final CRC32 crc = new CRC32(); - - // Peer has data. Decode the bytes into FlowFiles until peer says he's finished sending data. - boolean continueTransaction = true; - String calculatedCRC = ""; - while (continueTransaction) { - final InputStream flowFileInputStream = useCompression ? new CompressionInputStream(dis) : dis; - final CheckedInputStream checkedIn = new CheckedInputStream(flowFileInputStream, crc); - - final long startNanos = System.nanoTime(); - FlowFile flowFile = codec.decode(checkedIn, session); - final long transmissionNanos = System.nanoTime() - startNanos; - final long transmissionMillis = TimeUnit.MILLISECONDS.convert(transmissionNanos, TimeUnit.NANOSECONDS); - - final String sourceFlowFileIdentifier = flowFile.getAttribute(CoreAttributes.UUID.key()); - flowFile = session.putAttribute(flowFile, CoreAttributes.UUID.key(), UUID.randomUUID().toString()); - - final String transitUri = (transitUriPrefix == null) ? peer.getUrl() : transitUriPrefix + sourceFlowFileIdentifier; - session.getProvenanceReporter().receive(flowFile, transitUri, "urn:nifi:" + sourceFlowFileIdentifier, "Remote Host=" + peer.getHost() + ", Remote DN=" + userDn, transmissionMillis); - - session.transfer(flowFile, Relationship.ANONYMOUS); - bytesReceived += flowFile.getSize(); - flowFilesReceived.add(flowFile); - logger.debug("{} Received {} from {}", this, flowFile, peer); - - final Response transactionCode = Response.read(dis); - switch (transactionCode.getCode()) { - case CONTINUE_TRANSACTION: - logger.trace("{} Received ContinueTransaction indicator from {}", this, peer); - break; - case FINISH_TRANSACTION: - logger.trace("{} Received FinishTransaction indicator from {}", this, peer); - continueTransaction = false; - calculatedCRC = String.valueOf(checkedIn.getChecksum().getValue()); - break; - default: - throw new ProtocolException("Received unexpected response from peer: when expecting Continue Transaction or Finish Transaction, received" + transactionCode); - } - } - - // 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); - ResponseCode.CONFIRM_TRANSACTION.writeResponse(dos, calculatedCRC); - - final Response confirmTransactionResponse = Response.read(dis); - logger.trace("{} Received {} from {}", this, confirmTransactionResponse, peer); - - switch (confirmTransactionResponse.getCode()) { - case CONFIRM_TRANSACTION: - break; - case BAD_CHECKSUM: - session.rollback(); - throw new IOException(this + " Received a BadChecksum response from peer " + peer); - default: - throw new ProtocolException(this + " Received unexpected Response from peer " + peer + " : " + confirmTransactionResponse + "; expected 'Confirm Transaction' Response Code"); - } - - // Commit the session so that we have persisted the data - session.commit(); - - if ( context.getAvailableRelationships().isEmpty() ) { - // Confirm that we received the data and the peer can now discard it but that the peer should not - // send any more data for a bit - logger.debug("{} Sending TRANSACTION_FINISHED_BUT_DESTINATION_FULL to {}", this, peer); - ResponseCode.TRANSACTION_FINISHED_BUT_DESTINATION_FULL.writeResponse(dos); - } else { - // Confirm that we received the data and the peer can now discard it - logger.debug("{} Sending TRANSACTION_FINISHED to {}", this, peer); - ResponseCode.TRANSACTION_FINISHED.writeResponse(dos); - } - - stopWatch.stop(); - final String flowFileDescription = flowFilesReceived.size() < 20 ? flowFilesReceived.toString() : flowFilesReceived.size() + " FlowFiles"; - final String uploadDataRate = stopWatch.calculateDataRate(bytesReceived); - final long uploadMillis = stopWatch.getDuration(TimeUnit.MILLISECONDS); - final String dataSize = FormatUtils.formatDataSize(bytesReceived); - logger.info("{} Successfully receveied {} ({}) from {} in {} milliseconds at a rate of {}", new Object[] { - this, flowFileDescription, dataSize, peer, uploadMillis, uploadDataRate}); - } - - @Override - public void transferFlowFiles(final Peer peer, final ProcessContext context, final ProcessSession session, final FlowFileCodec codec) throws IOException, ProtocolException { - if ( !handshakeComplete ) { - throw new IllegalStateException("Handshake has not been performed"); - } - if ( !readyForFileTransfer ) { - throw new IllegalStateException("Cannot transfer files; handshake resolution was " + handshakeResponse); - } - - FlowFile flowFile = session.get(); - if ( flowFile == null ) { - return; - } - - logger.debug("{} Sending FlowFiles to {}", this, peer); - final CommunicationsSession commsSession = peer.getCommunicationsSession(); - final DataInputStream dis = new DataInputStream(commsSession.getInput().getInputStream()); - final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream()); - String userDn = commsSession.getUserDn(); - if ( userDn == null ) { - userDn = "none"; - } - - // Indicate that we would like to have some data - RequestType.SEND_FLOWFILES.writeRequestType(dos); - dos.flush(); - - final StopWatch stopWatch = new StopWatch(true); - final CRC32 crc = new CRC32(); - - long bytesSent = 0L; - final Set flowFilesSent = new HashSet<>(); - boolean continueTransaction = true; - String calculatedCRC = ""; - final long startSendingNanos = System.nanoTime(); - while (continueTransaction) { - final OutputStream flowFileOutputStream = useCompression ? new CompressionOutputStream(dos) : dos; - logger.debug("{} Sending {} to {}", this, flowFile, peer); - - final CheckedOutputStream checkedOutStream = new CheckedOutputStream(flowFileOutputStream, crc); - - final long startNanos = System.nanoTime(); - flowFile = codec.encode(flowFile, session, checkedOutStream); - final long transferNanos = System.nanoTime() - startNanos; - final long transferMillis = TimeUnit.MILLISECONDS.convert(transferNanos, TimeUnit.NANOSECONDS); - - // 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 ( useCompression ) { - checkedOutStream.close(); - } - - 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); - if ( continueTransaction ) { - logger.debug("{} Sent CONTINUE_TRANSACTION indicator to {}", this, peer); - ResponseCode.CONTINUE_TRANSACTION.writeResponse(dos); - } else { - logger.debug("{} Sent FINISH_TRANSACTION indicator to {}", this, peer); - ResponseCode.FINISH_TRANSACTION.writeResponse(dos); - - calculatedCRC = String.valueOf( checkedOutStream.getChecksum().getValue() ); - } - } - - // we've sent a FINISH_TRANSACTION. Now we'll wait for the peer to send a 'Confirm Transaction' response - final Response transactionConfirmationResponse = Response.read(dis); - if ( transactionConfirmationResponse.getCode() == ResponseCode.CONFIRM_TRANSACTION ) { - // Confirm checksum and echo back the confirmation. - logger.trace("{} Received {} from {}", this, transactionConfirmationResponse, peer); - final String receivedCRC = transactionConfirmationResponse.getMessage(); - - if ( versionNegotiator.getVersion() > 3 ) { - if ( !receivedCRC.equals(calculatedCRC) ) { - ResponseCode.BAD_CHECKSUM.writeResponse(dos); - session.rollback(); - throw new IOException(this + " Sent data to peer " + peer + " but calculated CRC32 Checksum as " + calculatedCRC + " while peer calculated CRC32 Checksum as " + receivedCRC + "; canceling transaction and rolling back session"); - } - } - - ResponseCode.CONFIRM_TRANSACTION.writeResponse(dos, ""); - } else { - throw new ProtocolException("Expected to receive 'Confirm Transaction' response from peer " + peer + " but received " + transactionConfirmationResponse); - } - - final String flowFileDescription = (flowFilesSent.size() < 20) ? flowFilesSent.toString() : flowFilesSent.size() + " FlowFiles"; - - final Response transactionResponse; - try { - transactionResponse = Response.read(dis); - } catch (final IOException e) { - logger.error("{} Failed to receive a response from {} when expecting a TransactionFinished Indicator." + - " It is unknown whether or not the peer successfully received/processed the data." + - " Therefore, {} will be rolled back, possibly resulting in data duplication of {}", - this, peer, session, flowFileDescription); - session.rollback(); - throw e; - } - - logger.debug("{} Received {} from {}", this, transactionResponse, peer); - if ( transactionResponse.getCode() == ResponseCode.TRANSACTION_FINISHED_BUT_DESTINATION_FULL ) { - peer.penalize(port.getYieldPeriod(TimeUnit.MILLISECONDS)); - } else if ( transactionResponse.getCode() != ResponseCode.TRANSACTION_FINISHED ) { - throw new ProtocolException("After sending data, expected TRANSACTION_FINISHED response but got " + transactionResponse); - } - - // 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(); - - logger.info("{} Successfully sent {} ({}) to {} in {} milliseconds at a rate of {}", new Object[] { - this, flowFileDescription, dataSize, peer, uploadMillis, uploadDataRate}); - } - - @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 + "]"; - } -} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketFlowFileServerProtocol.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketFlowFileServerProtocol.java index 5edd4f9579..eb22b0e837 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketFlowFileServerProtocol.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketFlowFileServerProtocol.java @@ -32,7 +32,6 @@ import java.util.zip.CRC32; import java.util.zip.CheckedInputStream; import java.util.zip.CheckedOutputStream; -import org.apache.nifi.cluster.NodeInformant; import org.apache.nifi.connectable.Connection; import org.apache.nifi.connectable.Port; import org.apache.nifi.flowfile.FlowFile; @@ -41,24 +40,27 @@ import org.apache.nifi.groups.ProcessGroup; import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.Relationship; +import org.apache.nifi.processor.io.InputStreamCallback; import org.apache.nifi.remote.Peer; import org.apache.nifi.remote.PortAuthorizationResult; import org.apache.nifi.remote.RemoteResourceFactory; import org.apache.nifi.remote.RootGroupPort; import org.apache.nifi.remote.StandardVersionNegotiator; import org.apache.nifi.remote.VersionNegotiator; +import org.apache.nifi.remote.cluster.NodeInformant; import org.apache.nifi.remote.codec.FlowFileCodec; import org.apache.nifi.remote.exception.HandshakeException; import org.apache.nifi.remote.exception.ProtocolException; import org.apache.nifi.remote.io.CompressionInputStream; import org.apache.nifi.remote.io.CompressionOutputStream; import org.apache.nifi.remote.protocol.CommunicationsSession; +import org.apache.nifi.remote.protocol.DataPacket; import org.apache.nifi.remote.protocol.RequestType; import org.apache.nifi.remote.protocol.ServerProtocol; +import org.apache.nifi.remote.util.StandardDataPacket; import org.apache.nifi.util.FormatUtils; import org.apache.nifi.util.NiFiProperties; import org.apache.nifi.util.StopWatch; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -76,10 +78,14 @@ public class SocketFlowFileServerProtocol implements ServerProtocol { private FlowFileCodec negotiatedFlowFileCodec = null; private String transitUriPrefix = null; - private final VersionNegotiator versionNegotiator = new StandardVersionNegotiator(4, 3, 2, 1); + private int requestedBatchCount = 0; + private long requestedBatchBytes = 0L; + private long requestedBatchNanos = 0L; + private static final long DEFAULT_BATCH_NANOS = TimeUnit.SECONDS.toNanos(5L); + + private final VersionNegotiator versionNegotiator = new StandardVersionNegotiator(5, 4, 3, 2, 1); private final Logger logger = LoggerFactory.getLogger(SocketFlowFileServerProtocol.class); - private static final long BATCH_NANOS = TimeUnit.SECONDS.toNanos(5L); // send batches of up to 5 seconds @Override @@ -135,68 +141,90 @@ public class SocketFlowFileServerProtocol implements ServerProtocol { throw new HandshakeException("Received unknown property: " + propertyName); } - switch (property) { - case GZIP: { - useGzip = Boolean.parseBoolean(value); - break; - } - case REQUEST_EXPIRATION_MILLIS: - requestExpirationMillis = Long.parseLong(value); - break; - case PORT_IDENTIFIER: { - Port receivedPort = rootGroup.getInputPort(value); - if ( receivedPort == null ) { - receivedPort = rootGroup.getOutputPort(value); - } - if ( receivedPort == null ) { - logger.debug("Responding with ResponseCode UNKNOWN_PORT for identifier {}", value); - ResponseCode.UNKNOWN_PORT.writeResponse(dos); - throw new HandshakeException("Received unknown port identifier: " + value); - } - if ( !(receivedPort instanceof RootGroupPort) ) { - logger.debug("Responding with ResponseCode UNKNOWN_PORT for identifier {}", value); - ResponseCode.UNKNOWN_PORT.writeResponse(dos); - throw new HandshakeException("Received port identifier " + value + ", but this Port is not a RootGroupPort"); - } - - this.port = (RootGroupPort) receivedPort; - final PortAuthorizationResult portAuthResult = this.port.checkUserAuthorization(peer.getCommunicationsSession().getUserDn()); - if ( !portAuthResult.isAuthorized() ) { - logger.debug("Responding with ResponseCode UNAUTHORIZED: ", portAuthResult.getExplanation()); - ResponseCode.UNAUTHORIZED.writeResponse(dos, portAuthResult.getExplanation()); - responseWritten = true; + try { + switch (property) { + case GZIP: { + useGzip = Boolean.parseBoolean(value); break; } - - if ( !receivedPort.isValid() ) { - logger.debug("Responding with ResponseCode PORT_NOT_IN_VALID_STATE for {}", receivedPort); - ResponseCode.PORT_NOT_IN_VALID_STATE.writeResponse(dos, "Port is not valid"); - responseWritten = true; + case REQUEST_EXPIRATION_MILLIS: + requestExpirationMillis = Long.parseLong(value); break; - } - - if ( !receivedPort.isRunning() ) { - logger.debug("Responding with ResponseCode PORT_NOT_IN_VALID_STATE for {}", receivedPort); - ResponseCode.PORT_NOT_IN_VALID_STATE.writeResponse(dos, "Port not running"); - responseWritten = true; + case BATCH_COUNT: + requestedBatchCount = Integer.parseInt(value); + if ( requestedBatchCount < 0 ) { + throw new HandshakeException("Cannot request Batch Count less than 1; requested value: " + value); + } break; - } - - // PORTS_DESTINATION_FULL was introduced in version 2. If version 1, just ignore this - // we we will simply not service the request but the sender will timeout - if ( getVersionNegotiator().getVersion() > 1 ) { - for ( final Connection connection : port.getConnections() ) { - if ( connection.getFlowFileQueue().isFull() ) { - logger.debug("Responding with ResponseCode PORTS_DESTINATION_FULL for {}", receivedPort); - ResponseCode.PORTS_DESTINATION_FULL.writeResponse(dos); - responseWritten = true; - break; + case BATCH_SIZE: + requestedBatchBytes = Long.parseLong(value); + if ( requestedBatchBytes < 0 ) { + throw new HandshakeException("Cannot request Batch Size less than 1; requested value: " + value); + } + break; + case BATCH_DURATION: + requestedBatchNanos = TimeUnit.MILLISECONDS.toNanos(Long.parseLong(value)); + if ( requestedBatchNanos < 0 ) { + throw new HandshakeException("Cannot request Batch Duration less than 1; requested value: " + value); + } + break; + case PORT_IDENTIFIER: { + Port receivedPort = rootGroup.getInputPort(value); + if ( receivedPort == null ) { + receivedPort = rootGroup.getOutputPort(value); + } + if ( receivedPort == null ) { + logger.debug("Responding with ResponseCode UNKNOWN_PORT for identifier {}", value); + ResponseCode.UNKNOWN_PORT.writeResponse(dos); + throw new HandshakeException("Received unknown port identifier: " + value); + } + if ( !(receivedPort instanceof RootGroupPort) ) { + logger.debug("Responding with ResponseCode UNKNOWN_PORT for identifier {}", value); + ResponseCode.UNKNOWN_PORT.writeResponse(dos); + throw new HandshakeException("Received port identifier " + value + ", but this Port is not a RootGroupPort"); + } + + this.port = (RootGroupPort) receivedPort; + final PortAuthorizationResult portAuthResult = this.port.checkUserAuthorization(peer.getCommunicationsSession().getUserDn()); + if ( !portAuthResult.isAuthorized() ) { + logger.debug("Responding with ResponseCode UNAUTHORIZED: ", portAuthResult.getExplanation()); + ResponseCode.UNAUTHORIZED.writeResponse(dos, portAuthResult.getExplanation()); + responseWritten = true; + break; + } + + if ( !receivedPort.isValid() ) { + logger.debug("Responding with ResponseCode PORT_NOT_IN_VALID_STATE for {}", receivedPort); + ResponseCode.PORT_NOT_IN_VALID_STATE.writeResponse(dos, "Port is not valid"); + responseWritten = true; + break; + } + + if ( !receivedPort.isRunning() ) { + logger.debug("Responding with ResponseCode PORT_NOT_IN_VALID_STATE for {}", receivedPort); + ResponseCode.PORT_NOT_IN_VALID_STATE.writeResponse(dos, "Port not running"); + responseWritten = true; + break; + } + + // PORTS_DESTINATION_FULL was introduced in version 2. If version 1, just ignore this + // we we will simply not service the request but the sender will timeout + if ( getVersionNegotiator().getVersion() > 1 ) { + for ( final Connection connection : port.getConnections() ) { + if ( connection.getFlowFileQueue().isFull() ) { + logger.debug("Responding with ResponseCode PORTS_DESTINATION_FULL for {}", receivedPort); + ResponseCode.PORTS_DESTINATION_FULL.writeResponse(dos); + responseWritten = true; + break; + } } } + + break; } - - break; } + } catch (final NumberFormatException nfe) { + throw new HandshakeException("Received invalid value for property '" + property + "'; invalid value: " + value); } } @@ -205,11 +233,6 @@ public class SocketFlowFileServerProtocol implements ServerProtocol { ResponseCode.MISSING_PROPERTY.writeResponse(dos, HandshakeProperty.GZIP.name()); throw new HandshakeException("Missing Property " + HandshakeProperty.GZIP.name()); } - if ( port == null ) { - logger.debug("Responding with ResponseCode MISSING_PROPERTY because Port Identifier property is missing"); - ResponseCode.MISSING_PROPERTY.writeResponse(dos, HandshakeProperty.PORT_IDENTIFIER.name()); - throw new HandshakeException("Missing Property " + HandshakeProperty.PORT_IDENTIFIER.name()); - } // send "OK" response if ( !responseWritten ) { @@ -244,6 +267,10 @@ public class SocketFlowFileServerProtocol implements ServerProtocol { final DataInputStream dis = new DataInputStream(commsSession.getInput().getInputStream()); final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream()); + if ( port == null ) { + RemoteResourceFactory.rejectCodecNegotiation(dis, dos, "Cannot transfer FlowFiles because no port was specified"); + } + // Negotiate the FlowFileCodec to use. try { negotiatedFlowFileCodec = RemoteResourceFactory.receiveCodecNegotiation(dis, dos); @@ -306,7 +333,16 @@ public class SocketFlowFileServerProtocol implements ServerProtocol { final CheckedOutputStream checkedOutputStream = new CheckedOutputStream(flowFileOutputStream, crc); final StopWatch transferWatch = new StopWatch(true); - flowFile = codec.encode(flowFile, session, checkedOutputStream); + + final FlowFile toSend = flowFile; + session.read(flowFile, new InputStreamCallback() { + @Override + public void process(final InputStream in) throws IOException { + final DataPacket dataPacket = new StandardDataPacket(toSend.getAttributes(), in, toSend.getSize()); + codec.encode(dataPacket, checkedOutputStream); + } + }); + final long transmissionMillis = transferWatch.getElapsed(TimeUnit.MILLISECONDS); // need to close the CompressionOutputStream in order to force it write out any remaining bytes. @@ -323,8 +359,25 @@ public class SocketFlowFileServerProtocol implements ServerProtocol { session.getProvenanceReporter().send(flowFile, transitUri, "Remote Host=" + peer.getHost() + ", Remote DN=" + remoteDn, transmissionMillis, false); session.remove(flowFile); + // determine if we should check for more data on queue. final long sendingNanos = System.nanoTime() - startNanos; - if ( sendingNanos < BATCH_NANOS ) { + boolean poll = true; + if ( sendingNanos >= requestedBatchNanos && requestedBatchNanos > 0L ) { + poll = false; + } + if ( bytesSent >= requestedBatchBytes && requestedBatchBytes > 0L ) { + poll = false; + } + if ( flowFilesSent.size() >= requestedBatchCount && requestedBatchCount > 0 ) { + poll = false; + } + + if ( requestedBatchNanos == 0 && requestedBatchBytes == 0 && requestedBatchCount == 0 ) { + poll = (sendingNanos < DEFAULT_BATCH_NANOS); + } + + if ( poll ) { + // we've not elapsed the requested sending duration, so get more data. flowFile = session.get(); } else { flowFile = null; @@ -429,7 +482,11 @@ public class SocketFlowFileServerProtocol implements ServerProtocol { final InputStream flowFileInputStream = useGzip ? new CompressionInputStream(dis) : dis; final CheckedInputStream checkedInputStream = new CheckedInputStream(flowFileInputStream, crc); - FlowFile flowFile = codec.decode(checkedInputStream, session); + final DataPacket dataPacket = codec.decode(checkedInputStream); + FlowFile flowFile = session.create(); + flowFile = session.importFrom(dataPacket.getData(), flowFile); + flowFile = session.putAllAttributes(flowFile, dataPacket.getAttributes()); + final long transferNanos = System.nanoTime() - startNanos; final long transferMillis = TimeUnit.MILLISECONDS.convert(transferNanos, TimeUnit.NANOSECONDS); final String sourceSystemFlowFileUuid = flowFile.getAttribute(CoreAttributes.UUID.key()); @@ -451,6 +508,10 @@ public class SocketFlowFileServerProtocol implements ServerProtocol { continueTransaction = false; calculatedCRC = String.valueOf(checkedInputStream.getChecksum().getValue()); break; + case CANCEL_TRANSACTION: + logger.info("{} Received CancelTransaction indicator from {} with explanation {}", this, peer, transactionResponse.getMessage()); + session.rollback(); + return 0; default: throw new ProtocolException("Received unexpected response from peer: when expecting Continue Transaction or Finish Transaction, received" + transactionResponse); } diff --git a/nifi/pom.xml b/nifi/pom.xml index 8eda68297d..eb395c9074 100644 --- a/nifi/pom.xml +++ b/nifi/pom.xml @@ -624,6 +624,11 @@ nifi-utils 0.0.2-incubating-SNAPSHOT + + org.apache.nifi + nifi-site-to-site-client + 0.0.2-incubating-SNAPSHOT + org.apache.nifi nifi-web-utils