mirror of https://github.com/apache/nifi.git
NIFI-5516: Implement Load-Balanced Connections
Refactoring StandardFlowFileQueue to have an AbstractFlowFileQueue Refactored more into AbstractFlowFileQueue Added documentation, cleaned up code some Refactored FlowFileQueue so that there is SwappablePriorityQueue Several unit tests written Added REST API Endpoint to allow PUT to update connection to use load balancing or not. When enabling load balancing, though, I saw the queue size go from 9 to 18. Then was only able to process 9 FlowFiles. Bug fixes Code refactoring Added integration tests, bug fixes Refactored clients to use NIO Bug fixes. Appears to finally be working with NIO Client!!!!! NIFI-5516: Refactored some code from NioAsyncLoadBalanceClient to LoadBalanceSession Bug fixes and allowed load balancing socket connections to be reused Implemented ability to compress Nothing, Attributes, or Content + Attributes when performing load-balancing Added flag to ConnectionDTO to indicate Load Balance Status Updated Diagnostics DTO for connections Store state about cluster topology in NodeClusterCoordinator so that the state is known upon restart Code cleanup Fixed checkstyle and unit tests NIFI-5516: Updating logic for Cluster Node Firewall so that the node's identity comes from its certificate, not from whatever it says it is. NIFI-5516: FIxed missing License headers NIFI-5516: Some minor code cleanup NIFI-5516: Adddressed review feedback; Bug fixes; some code cleanup. Changed dependency on nifi-registry from SNAPSHOT to official 0.3.0 release NIFI-5516: Take backpressure configuration into account NIFI-5516: Fixed ConnectionDiagnosticsSnapshot to include node identifier NIFI-5516: Addressed review feedback This closes #2947
This commit is contained in:
parent
5872eb3c4a
commit
619f1ffe8f
|
@ -63,6 +63,10 @@ public class QueueSize {
|
|||
return new QueueSize(objectCount + other.getObjectCount(), totalSizeBytes + other.getByteCount());
|
||||
}
|
||||
|
||||
public QueueSize add(final int count, final long bytes) {
|
||||
return new QueueSize(objectCount + count, totalSizeBytes + bytes);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "QueueSize[FlowFiles=" + objectCount + ", ContentSize=" + NumberFormat.getNumberInstance().format(totalSizeBytes) + " Bytes]";
|
||||
|
|
|
@ -121,5 +121,9 @@ public interface FlowFile extends Comparable<FlowFile> {
|
|||
}
|
||||
return key;
|
||||
}
|
||||
|
||||
public static boolean isValid(final String key) {
|
||||
return key != null && !key.trim().isEmpty();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -202,6 +202,13 @@ public abstract class NiFiProperties {
|
|||
public static final String FLOW_ELECTION_MAX_WAIT_TIME = "nifi.cluster.flow.election.max.wait.time";
|
||||
public static final String FLOW_ELECTION_MAX_CANDIDATES = "nifi.cluster.flow.election.max.candidates";
|
||||
|
||||
// cluster load balance properties
|
||||
public static final String LOAD_BALANCE_ADDRESS = "nifi.cluster.load.balance.address";
|
||||
public static final String LOAD_BALANCE_PORT = "nifi.cluster.load.balance.port";
|
||||
public static final String LOAD_BALANCE_CONNECTIONS_PER_NODE = "nifi.cluster.load.balance.connections.per.node";
|
||||
public static final String LOAD_BALANCE_MAX_THREAD_COUNT = "nifi.cluster.load.balance.max.thread.count";
|
||||
public static final String LOAD_BALANCE_COMMS_TIMEOUT = "nifi.cluster.load.balance.comms.timeout";
|
||||
|
||||
// zookeeper properties
|
||||
public static final String ZOOKEEPER_CONNECT_STRING = "nifi.zookeeper.connect.string";
|
||||
public static final String ZOOKEEPER_CONNECT_TIMEOUT = "nifi.zookeeper.connect.timeout";
|
||||
|
@ -285,6 +292,13 @@ public abstract class NiFiProperties {
|
|||
public static final String DEFAULT_REQUEST_REPLICATION_CLAIM_TIMEOUT = "15 secs";
|
||||
public static final String DEFAULT_FLOW_ELECTION_MAX_WAIT_TIME = "5 mins";
|
||||
|
||||
// cluster load balance defaults
|
||||
public static final int DEFAULT_LOAD_BALANCE_PORT = 6342;
|
||||
public static final int DEFAULT_LOAD_BALANCE_CONNECTIONS_PER_NODE = 4;
|
||||
public static final int DEFAULT_LOAD_BALANCE_MAX_THREAD_COUNT = 8;
|
||||
public static final String DEFAULT_LOAD_BALANCE_COMMS_TIMEOUT = "30 sec";
|
||||
|
||||
|
||||
// state management defaults
|
||||
public static final String DEFAULT_STATE_MANAGEMENT_CONFIG_FILE = "conf/state-management.xml";
|
||||
|
||||
|
@ -734,6 +748,23 @@ public abstract class NiFiProperties {
|
|||
}
|
||||
}
|
||||
|
||||
public InetSocketAddress getClusterLoadBalanceAddress() {
|
||||
try {
|
||||
String address = getProperty(LOAD_BALANCE_ADDRESS);
|
||||
if (StringUtils.isBlank(address)) {
|
||||
address = getProperty(CLUSTER_NODE_ADDRESS);
|
||||
}
|
||||
if (StringUtils.isBlank(address)) {
|
||||
address = "localhost";
|
||||
}
|
||||
|
||||
final int port = getIntegerProperty(LOAD_BALANCE_PORT, DEFAULT_LOAD_BALANCE_PORT);
|
||||
return InetSocketAddress.createUnresolved(address, port);
|
||||
} catch (final Exception e) {
|
||||
throw new RuntimeException("Invalid load balance address/port due to: " + e, e);
|
||||
}
|
||||
}
|
||||
|
||||
public Integer getClusterNodeProtocolPort() {
|
||||
try {
|
||||
return Integer.parseInt(getProperty(CLUSTER_NODE_PROTOCOL_PORT));
|
||||
|
|
|
@ -16,6 +16,9 @@
|
|||
*/
|
||||
package org.apache.nifi.stream.io;
|
||||
|
||||
import org.apache.nifi.stream.io.exception.BytePatternNotFoundException;
|
||||
import org.apache.nifi.stream.io.util.NonThreadSafeCircularBuffer;
|
||||
|
||||
import java.io.EOFException;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
|
@ -23,9 +26,6 @@ import java.io.OutputStream;
|
|||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.nifi.stream.io.exception.BytePatternNotFoundException;
|
||||
import org.apache.nifi.stream.io.util.NonThreadSafeCircularBuffer;
|
||||
|
||||
public class StreamUtils {
|
||||
|
||||
public static long copy(final InputStream source, final OutputStream destination) throws IOException {
|
||||
|
@ -101,6 +101,34 @@ public class StreamUtils {
|
|||
return bytesRead;
|
||||
}
|
||||
|
||||
/**
|
||||
* Reads <code>byteCount</code> bytes of data from the given InputStream, writing to the provided byte array.
|
||||
*
|
||||
* @param source the InputStream to read from
|
||||
* @param destination the destination for the data
|
||||
* @param byteCount the number of bytes to copy
|
||||
*
|
||||
* @throws IllegalArgumentException if the given byte array is smaller than <code>byteCount</code> elements.
|
||||
* @throws EOFException if the InputStream does not have <code>byteCount</code> bytes in the InputStream
|
||||
* @throws IOException if unable to read from the InputStream
|
||||
*/
|
||||
public static void read(final InputStream source, final byte[] destination, final int byteCount) throws IOException {
|
||||
if (destination.length < byteCount) {
|
||||
throw new IllegalArgumentException();
|
||||
}
|
||||
|
||||
int bytesRead = 0;
|
||||
int len;
|
||||
while (bytesRead < byteCount) {
|
||||
len = source.read(destination, bytesRead, byteCount - bytesRead);
|
||||
if (len < 0) {
|
||||
throw new EOFException("Expected to consume " + byteCount + " bytes but consumed only " + bytesRead);
|
||||
}
|
||||
|
||||
bytesRead += len;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Copies data from in to out until either we are out of data (returns null) or we hit one of the byte patterns identified by the <code>stoppers</code> parameter (returns the byte pattern
|
||||
* matched). The bytes in the stopper will be copied.
|
||||
|
|
|
@ -3934,6 +3934,13 @@ from the remote node before considering the communication with the node a failur
|
|||
to the cluster. It provides an additional layer of security. This value is blank by default, meaning that no firewall file is to be used.
|
||||
|`nifi.cluster.flow.election.max.wait.time`|Specifies the amount of time to wait before electing a Flow as the "correct" Flow. If the number of Nodes that have voted is equal to the number specified by the `nifi.cluster.flow.election.max.candidates` property, the cluster will not wait this long. The default value is `5 mins`. Note that the time starts as soon as the first vote is cast.
|
||||
|`nifi.cluster.flow.election.max.candidates`|Specifies the number of Nodes required in the cluster to cause early election of Flows. This allows the Nodes in the cluster to avoid having to wait a long time before starting processing if we reach at least this number of nodes in the cluster.
|
||||
|`nifi.cluster.flow.election.max.wait.time`|Specifies the amount of time to wait before electing a Flow as the "correct" Flow. If the number of Nodes that have voted is equal to the number specified
|
||||
by the `nifi.cluster.flow.election.max.candidates` property, the cluster will not wait this long. The default value is `5 mins`. Note that the time starts as soon as the first vote is cast.
|
||||
|`nifi.cluster.flow.election.max.candidates`|Specifies the number of Nodes required in the cluster to cause early election of Flows. This allows the Nodes in the cluster to avoid having to wait a
|
||||
long time before starting processing if we reach at least this number of nodes in the cluster.
|
||||
|`nifi.cluster.load.balance.port`|Specifies the port to listen on for incoming connections for load balancing data across the cluster. The default value is `6342`.
|
||||
|`nifi.cluster.load.balance.host`|Specifies the hostname to listen on for incoming connections for load balancing data across the cluster. If not specified, will default to the value used by the `nifi
|
||||
.cluster.node.address` property.
|
||||
|====
|
||||
|
||||
[[claim_management]]
|
||||
|
|
|
@ -16,18 +16,17 @@
|
|||
*/
|
||||
package org.apache.nifi.controller.queue;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
import java.util.Queue;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.apache.nifi.controller.repository.FlowFileRecord;
|
||||
import org.apache.nifi.controller.repository.SwapSummary;
|
||||
import org.apache.nifi.flowfile.FlowFilePrioritizer;
|
||||
import org.apache.nifi.processor.FlowFileFilter;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
public interface FlowFileQueue {
|
||||
|
||||
/**
|
||||
|
@ -59,8 +58,6 @@ public interface FlowFileQueue {
|
|||
*/
|
||||
void purgeSwapFiles();
|
||||
|
||||
int getSwapFileCount();
|
||||
|
||||
/**
|
||||
* Resets the comparator used by this queue to maintain order.
|
||||
*
|
||||
|
@ -108,33 +105,21 @@ public interface FlowFileQueue {
|
|||
*/
|
||||
boolean isActiveQueueEmpty();
|
||||
|
||||
/**
|
||||
* Returns a QueueSize that represents all FlowFiles that are 'unacknowledged'. A FlowFile
|
||||
* is considered to be unacknowledged if it has been pulled from the queue by some component
|
||||
* but the session that pulled the FlowFile has not yet been committed or rolled back.
|
||||
*
|
||||
* @return a QueueSize that represents all FlowFiles that are 'unacknowledged'.
|
||||
*/
|
||||
QueueSize getUnacknowledgedQueueSize();
|
||||
|
||||
QueueSize getActiveQueueSize();
|
||||
|
||||
QueueSize getSwapQueueSize();
|
||||
|
||||
void acknowledge(FlowFileRecord flowFile);
|
||||
|
||||
void acknowledge(Collection<FlowFileRecord> flowFiles);
|
||||
|
||||
/**
|
||||
* @return <code>true</code> if at least one FlowFile is unacknowledged, <code>false</code> if all FlowFiles that have been dequeued have been acknowledged
|
||||
*/
|
||||
boolean isUnacknowledgedFlowFile();
|
||||
|
||||
/**
|
||||
* @return true if maximum queue size has been reached or exceeded; false
|
||||
* otherwise
|
||||
*/
|
||||
boolean isFull();
|
||||
|
||||
boolean isAnyActiveFlowFilePenalized();
|
||||
|
||||
boolean isAllActiveFlowFilesPenalized();
|
||||
|
||||
/**
|
||||
* places the given file into the queue
|
||||
*
|
||||
|
@ -163,18 +148,6 @@ public interface FlowFileQueue {
|
|||
*/
|
||||
List<FlowFileRecord> poll(int maxResults, Set<FlowFileRecord> expiredRecords);
|
||||
|
||||
/**
|
||||
* Drains flow files from the given source queue into the given destination
|
||||
* list.
|
||||
*
|
||||
* @param sourceQueue queue to drain from
|
||||
* @param destination Collection to drain to
|
||||
* @param maxResults max number to drain
|
||||
* @param expiredRecords for expired records
|
||||
* @return size (bytes) of flow files drained from queue
|
||||
*/
|
||||
long drainQueue(Queue<FlowFileRecord> sourceQueue, List<FlowFileRecord> destination, int maxResults, Set<FlowFileRecord> expiredRecords);
|
||||
|
||||
List<FlowFileRecord> poll(FlowFileFilter filter, Set<FlowFileRecord> expiredRecords);
|
||||
|
||||
String getFlowFileExpiration();
|
||||
|
@ -187,7 +160,7 @@ public interface FlowFileQueue {
|
|||
* Initiates a request to drop all FlowFiles in this queue. This method returns
|
||||
* a DropFlowFileStatus that can be used to determine the current state of the request.
|
||||
* Additionally, the DropFlowFileStatus provides a request identifier that can then be
|
||||
* passed to the {@link #getDropFlowFileStatus(String)} and {@link #cancelDropFlowFileStatus(String)}
|
||||
* passed to the {@link #getDropFlowFileStatus(String)} and {@link #cancelDropFlowFileRequest(String)}
|
||||
* methods in order to obtain the status later or cancel a request
|
||||
*
|
||||
* @param requestIdentifier the identifier of the Drop FlowFile Request
|
||||
|
@ -200,7 +173,7 @@ public interface FlowFileQueue {
|
|||
|
||||
/**
|
||||
* Returns the current status of a Drop FlowFile Request that was initiated via the
|
||||
* {@link #dropFlowFiles()} method that has the given identifier
|
||||
* {@link #dropFlowFiles(String, String)} method that has the given identifier
|
||||
*
|
||||
* @param requestIdentifier the identifier of the Drop FlowFile Request
|
||||
* @return the status for the request with the given identifier, or <code>null</code> if no
|
||||
|
@ -244,7 +217,7 @@ public interface FlowFileQueue {
|
|||
ListFlowFileStatus listFlowFiles(String requestIdentifier, int maxResults);
|
||||
|
||||
/**
|
||||
* Returns the current status of a List FlowFile Request that was initiated via the {@link #listFlowFiles(String)}
|
||||
* Returns the current status of a List FlowFile Request that was initiated via the {@link #listFlowFiles(String, int)}
|
||||
* method that has the given identifier
|
||||
*
|
||||
* @param requestIdentifier the identifier of the Drop FlowFile Request
|
||||
|
@ -282,4 +255,32 @@ public interface FlowFileQueue {
|
|||
* @throws IllegalStateException if the queue is not in a state in which a listing can be performed
|
||||
*/
|
||||
void verifyCanList() throws IllegalStateException;
|
||||
|
||||
/**
|
||||
* Returns diagnostic information about the queue
|
||||
*/
|
||||
QueueDiagnostics getQueueDiagnostics();
|
||||
|
||||
void lock();
|
||||
|
||||
void unlock();
|
||||
|
||||
void setLoadBalanceStrategy(LoadBalanceStrategy strategy, String partitioningAttribute);
|
||||
|
||||
LoadBalanceStrategy getLoadBalanceStrategy();
|
||||
|
||||
void setLoadBalanceCompression(LoadBalanceCompression compression);
|
||||
|
||||
LoadBalanceCompression getLoadBalanceCompression();
|
||||
|
||||
String getPartitioningAttribute();
|
||||
|
||||
void startLoadBalancing();
|
||||
|
||||
void stopLoadBalancing();
|
||||
|
||||
/**
|
||||
* @return <code>true</code> if the queue is actively transferring data to another node, <code>false</code> otherwise
|
||||
*/
|
||||
boolean isActivelyLoadBalancing();
|
||||
}
|
||||
|
|
|
@ -0,0 +1,35 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.nifi.controller.queue;
|
||||
|
||||
public enum LoadBalanceCompression {
|
||||
/**
|
||||
* FlowFiles will not be compressed
|
||||
*/
|
||||
DO_NOT_COMPRESS,
|
||||
|
||||
/**
|
||||
* FlowFiles' attributes will be compressed, but the FlowFiles' contents will not be
|
||||
*/
|
||||
COMPRESS_ATTRIBUTES_ONLY,
|
||||
|
||||
/**
|
||||
* FlowFiles' attributes and content will be compressed
|
||||
*/
|
||||
COMPRESS_ATTRIBUTES_AND_CONTENT;
|
||||
}
|
|
@ -0,0 +1,41 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.nifi.controller.queue;
|
||||
|
||||
public enum LoadBalanceStrategy {
|
||||
/**
|
||||
* Do not load balance FlowFiles between nodes in the cluster.
|
||||
*/
|
||||
DO_NOT_LOAD_BALANCE,
|
||||
|
||||
/**
|
||||
* Determine which node to send a given FlowFile to based on the value of a user-specified FlowFile Attribute.
|
||||
* All FlowFiles that have the same value for said Attribute will be sent to the same node in the cluster.
|
||||
*/
|
||||
PARTITION_BY_ATTRIBUTE,
|
||||
|
||||
/**
|
||||
* FlowFiles will be distributed to nodes in the cluster in a Round-Robin fashion.
|
||||
*/
|
||||
ROUND_ROBIN,
|
||||
|
||||
/**
|
||||
* All FlowFiles will be sent to the same node. Which node they are sent to is not defined.
|
||||
*/
|
||||
SINGLE_NODE;
|
||||
}
|
|
@ -0,0 +1,69 @@
|
|||
/*
|
||||
* 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.controller.queue;
|
||||
|
||||
import org.apache.nifi.controller.repository.FlowFileRecord;
|
||||
|
||||
import java.util.Collection;
|
||||
|
||||
public interface LoadBalancedFlowFileQueue extends FlowFileQueue {
|
||||
/**
|
||||
* Adds the given FlowFiles to this queue, as they have been received from another node in the cluster
|
||||
* @param flowFiles the FlowFiles received from the peer
|
||||
*/
|
||||
void receiveFromPeer(Collection<FlowFileRecord> flowFiles);
|
||||
|
||||
/**
|
||||
* Distributes the given FlowFiles to the appropriate partitions. Unlike the {@link #putAll(Collection)} method,
|
||||
* this does not alter the size of the FlowFile Queue itself, as it is intended only to place the FlowFiles into
|
||||
* their appropriate partitions
|
||||
*
|
||||
* @param flowFiles the FlowFiles to distribute
|
||||
*/
|
||||
void distributeToPartitions(Collection<FlowFileRecord> flowFiles);
|
||||
|
||||
/**
|
||||
* Notifies the queue that the given FlowFiles have been successfully transferred to another node
|
||||
* @param flowFiles the FlowFiles that were transferred
|
||||
*/
|
||||
void onTransfer(Collection<FlowFileRecord> flowFiles);
|
||||
|
||||
/**
|
||||
* Notifies the queue the a transaction containing the given FlowFiles was aborted
|
||||
* @param flowFiles the FlowFiles in the transaction
|
||||
*/
|
||||
void onAbort(Collection<FlowFileRecord> flowFiles);
|
||||
|
||||
/**
|
||||
* Handles updating the repositories for the given FlowFiles, which have been expired
|
||||
* @param flowFiles the expired FlowFiles
|
||||
*/
|
||||
void handleExpiredRecords(Collection<FlowFileRecord> flowFiles);
|
||||
|
||||
/**
|
||||
* There are times when we want to ensure that if a node in the cluster reaches the point where backpressure is engaged, that we
|
||||
* honor that backpressure and do not attempt to load balance from a different node in the cluster to that node. There are other times
|
||||
* when we may want to push data to the remote node even though it has already reached its backpressure threshold. This method indicates
|
||||
* whether or not we want to propagate that backpressure indicator across the cluster.
|
||||
*
|
||||
* @return <code>true</code> if backpressure on Node A should prevent Node B from sending to it, <code>false</code> if Node B should send to Node A
|
||||
* even when backpressure is engaged on Node A.
|
||||
*/
|
||||
boolean isPropagateBackpressureAcrossNodes();
|
||||
|
||||
}
|
|
@ -0,0 +1,32 @@
|
|||
/*
|
||||
* 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.controller.queue;
|
||||
|
||||
public interface LocalQueuePartitionDiagnostics {
|
||||
QueueSize getUnacknowledgedQueueSize();
|
||||
|
||||
QueueSize getActiveQueueSize();
|
||||
|
||||
QueueSize getSwapQueueSize();
|
||||
|
||||
int getSwapFileCount();
|
||||
|
||||
boolean isAnyActiveFlowFilePenalized();
|
||||
|
||||
boolean isAllActiveFlowFilesPenalized();
|
||||
}
|
|
@ -0,0 +1,28 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.nifi.controller.queue;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
public interface QueueDiagnostics {
|
||||
|
||||
LocalQueuePartitionDiagnostics getLocalQueuePartitionDiagnostics();
|
||||
|
||||
List<RemoteQueuePartitionDiagnostics> getRemoteQueuePartitionDiagnostics();
|
||||
|
||||
}
|
|
@ -0,0 +1,30 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.nifi.controller.queue;
|
||||
|
||||
public interface RemoteQueuePartitionDiagnostics {
|
||||
String getNodeIdentifier();
|
||||
|
||||
QueueSize getUnacknowledgedQueueSize();
|
||||
|
||||
QueueSize getActiveQueueSize();
|
||||
|
||||
QueueSize getSwapQueueSize();
|
||||
|
||||
int getSwapFileCount();
|
||||
}
|
|
@ -18,6 +18,7 @@ package org.apache.nifi.controller.repository;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.nifi.controller.queue.FlowFileQueue;
|
||||
|
||||
|
@ -44,11 +45,12 @@ public interface FlowFileSwapManager {
|
|||
*
|
||||
* @param flowFiles the FlowFiles to swap out to external storage
|
||||
* @param flowFileQueue the queue that the FlowFiles belong to
|
||||
* @param partitionName the name of the partition within the queue, or <code>null</code> if the queue is not partitioned
|
||||
* @return the location of the externally stored swap file
|
||||
*
|
||||
* @throws IOException if unable to swap the FlowFiles out
|
||||
*/
|
||||
String swapOut(List<FlowFileRecord> flowFiles, FlowFileQueue flowFileQueue) throws IOException;
|
||||
String swapOut(List<FlowFileRecord> flowFiles, FlowFileQueue flowFileQueue, final String partitionName) throws IOException;
|
||||
|
||||
/**
|
||||
* Recovers the FlowFiles from the swap file that lives at the given location. This action
|
||||
|
@ -82,11 +84,32 @@ public interface FlowFileSwapManager {
|
|||
* Determines swap files that exist for the given FlowFileQueue
|
||||
*
|
||||
* @param flowFileQueue the queue for which the FlowFiles should be recovered
|
||||
* @param partitionName the partition within the FlowFileQueue to recover, or <code>null</code> if the queue is not partitioned
|
||||
*
|
||||
* @return all swap locations that have been identified for the given queue, in the order that they should
|
||||
* be swapped back in
|
||||
*/
|
||||
List<String> recoverSwapLocations(FlowFileQueue flowFileQueue) throws IOException;
|
||||
List<String> recoverSwapLocations(FlowFileQueue flowFileQueue, String partitionName) throws IOException;
|
||||
|
||||
/**
|
||||
* Determines the names of each of the Partitions for which there are swap files for the given queue
|
||||
*
|
||||
* @param queue the queue to which the FlowFiles belong
|
||||
*
|
||||
* @return the Set of names of all Partitions for which there are swap files
|
||||
* @throws IOException if unable to read the information from the underlying storage
|
||||
*/
|
||||
Set<String> getSwappedPartitionNames(FlowFileQueue queue) throws IOException;
|
||||
|
||||
/**
|
||||
* Updates the name of the partition that owns a given swap file
|
||||
*
|
||||
* @param swapLocation the location of the swap file
|
||||
* @param newPartitionName the new name of the new partition that owns the swap file
|
||||
* @return the new swap location
|
||||
* @throws IOException if unable to rename the swap file
|
||||
*/
|
||||
String changePartitionName(String swapLocation, String newPartitionName) throws IOException;
|
||||
|
||||
/**
|
||||
* Parses the contents of the swap file at the given location and provides a SwapSummary that provides
|
||||
|
|
|
@ -27,6 +27,9 @@ import java.util.Set;
|
|||
*/
|
||||
@XmlType(name = "connection")
|
||||
public class ConnectionDTO extends ComponentDTO {
|
||||
public static final String LOAD_BALANCE_NOT_CONFIGURED = "LOAD_BALANCE_NOT_CONFIGURED";
|
||||
public static final String LOAD_BALANCE_INACTIVE = "LOAD_BALANCE_INACTIVE";
|
||||
public static final String LOAD_BALANCE_ACTIVE = "LOAD_BALANCE_ACTIVE";
|
||||
|
||||
private ConnectableDTO source;
|
||||
private ConnectableDTO destination;
|
||||
|
@ -42,6 +45,11 @@ public class ConnectionDTO extends ComponentDTO {
|
|||
private List<String> prioritizers;
|
||||
private List<PositionDTO> bends;
|
||||
|
||||
private String loadBalanceStrategy;
|
||||
private String loadBalancePartitionAttribute;
|
||||
private String loadBalanceCompression;
|
||||
private String loadBalanceStatus;
|
||||
|
||||
/**
|
||||
* The source of this connection.
|
||||
*
|
||||
|
@ -231,6 +239,47 @@ public class ConnectionDTO extends ComponentDTO {
|
|||
this.prioritizers = prioritizers;
|
||||
}
|
||||
|
||||
@ApiModelProperty(value = "How to load balance the data in this Connection across the nodes in the cluster.",
|
||||
allowableValues = "DO_NOT_LOAD_BALANCE, PARTITION_BY_ATTRIBUTE, ROUND_ROBIN, SINGLE_NODE")
|
||||
public String getLoadBalanceStrategy() {
|
||||
return loadBalanceStrategy;
|
||||
}
|
||||
|
||||
public void setLoadBalanceStrategy(String loadBalanceStrategy) {
|
||||
this.loadBalanceStrategy = loadBalanceStrategy;
|
||||
}
|
||||
|
||||
@ApiModelProperty(value = "The FlowFile Attribute to use for determining which node a FlowFile will go to if the Load Balancing Strategy is set to PARTITION_BY_ATTRIBUTE")
|
||||
public String getLoadBalancePartitionAttribute() {
|
||||
return loadBalancePartitionAttribute;
|
||||
}
|
||||
|
||||
public void setLoadBalancePartitionAttribute(String partitionAttribute) {
|
||||
this.loadBalancePartitionAttribute = partitionAttribute;
|
||||
}
|
||||
|
||||
@ApiModelProperty(value = "Whether or not data should be compressed when being transferred between nodes in the cluster.",
|
||||
allowableValues = "DO_NOT_COMPRESS, COMPRESS_ATTRIBUTES_ONLY, COMPRESS_ATTRIBUTES_AND_CONTENT")
|
||||
public String getLoadBalanceCompression() {
|
||||
return loadBalanceCompression;
|
||||
}
|
||||
|
||||
public void setLoadBalanceCompression(String compression) {
|
||||
this.loadBalanceCompression = compression;
|
||||
}
|
||||
|
||||
@ApiModelProperty(value = "The current status of the Connection's Load Balancing Activities. Status can indicate that Load Balancing is not configured for the connection, that Load Balancing " +
|
||||
"is configured but inactive (not currently transferring data to another node), or that Load Balancing is configured and actively transferring data to another node.",
|
||||
allowableValues = LOAD_BALANCE_NOT_CONFIGURED + ", " + LOAD_BALANCE_INACTIVE + ", " + LOAD_BALANCE_ACTIVE,
|
||||
readOnly = true)
|
||||
public String getLoadBalanceStatus() {
|
||||
return loadBalanceStatus;
|
||||
}
|
||||
|
||||
public void setLoadBalanceStatus(String status) {
|
||||
this.loadBalanceStatus = status;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "ConnectionDTO [id: " + getId() + "]";
|
||||
|
|
|
@ -14,135 +14,44 @@
|
|||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.nifi.web.api.dto.diagnostics;
|
||||
|
||||
import javax.xml.bind.annotation.XmlType;
|
||||
|
||||
import io.swagger.annotations.ApiModelProperty;
|
||||
import org.apache.nifi.web.api.dto.ConnectionDTO;
|
||||
|
||||
import io.swagger.annotations.ApiModelProperty;
|
||||
import javax.xml.bind.annotation.XmlType;
|
||||
import java.util.List;
|
||||
|
||||
@XmlType(name = "connectionDiagnostics")
|
||||
@XmlType(name="connectionDiagnostics")
|
||||
public class ConnectionDiagnosticsDTO {
|
||||
private ConnectionDTO connection;
|
||||
private int totalFlowFileCount;
|
||||
private long totalByteCount;
|
||||
private int activeQueueFlowFileCount;
|
||||
private long activeQueueByteCount;
|
||||
private int swapFlowFileCount;
|
||||
private long swapByteCount;
|
||||
private int swapFiles;
|
||||
private int inFlightFlowFileCount;
|
||||
private long inFlightByteCount;
|
||||
private Boolean allActiveQueueFlowFilesPenalized;
|
||||
private Boolean anyActiveQueueFlowFilesPenalized;
|
||||
private ConnectionDiagnosticsSnapshotDTO aggregateSnapshot;
|
||||
private List<ConnectionDiagnosticsSnapshotDTO> nodeSnapshots;
|
||||
|
||||
@ApiModelProperty("Information about the Connection")
|
||||
@ApiModelProperty(value = "Details about the connection", readOnly = true)
|
||||
public ConnectionDTO getConnection() {
|
||||
return connection;
|
||||
}
|
||||
|
||||
public void setConnection(ConnectionDTO connection) {
|
||||
public void setConnection(final ConnectionDTO connection) {
|
||||
this.connection = connection;
|
||||
}
|
||||
|
||||
@ApiModelProperty("Total number of FlowFiles owned by the Connection")
|
||||
public int getTotalFlowFileCount() {
|
||||
return totalFlowFileCount;
|
||||
@ApiModelProperty(value = "Aggregate values for all nodes in the cluster, or for this instance if not clustered", readOnly = true)
|
||||
public ConnectionDiagnosticsSnapshotDTO getAggregateSnapshot() {
|
||||
return aggregateSnapshot;
|
||||
}
|
||||
|
||||
public void setTotalFlowFileCount(int totalFlowFileCount) {
|
||||
this.totalFlowFileCount = totalFlowFileCount;
|
||||
public void setAggregateSnapshot(final ConnectionDiagnosticsSnapshotDTO aggregateSnapshot) {
|
||||
this.aggregateSnapshot = aggregateSnapshot;
|
||||
}
|
||||
|
||||
@ApiModelProperty("Total number of bytes that make up the content for the FlowFiles owned by this Connection")
|
||||
public long getTotalByteCount() {
|
||||
return totalByteCount;
|
||||
@ApiModelProperty(value = "A list of values for each node in the cluster, if clustered.", readOnly = true)
|
||||
public List<ConnectionDiagnosticsSnapshotDTO> getNodeSnapshots() {
|
||||
return nodeSnapshots;
|
||||
}
|
||||
|
||||
public void setTotalByteCount(long totalByteCount) {
|
||||
this.totalByteCount = totalByteCount;
|
||||
}
|
||||
|
||||
@ApiModelProperty("Total number of FlowFiles that exist in the Connection's Active Queue, immediately available to be offered up to a component")
|
||||
public int getActiveQueueFlowFileCount() {
|
||||
return activeQueueFlowFileCount;
|
||||
}
|
||||
|
||||
public void setActiveQueueFlowFileCount(int activeQueueFlowFileCount) {
|
||||
this.activeQueueFlowFileCount = activeQueueFlowFileCount;
|
||||
}
|
||||
|
||||
@ApiModelProperty("Total number of bytes that make up the content for the FlowFiles that are present in the Connection's Active Queue")
|
||||
public long getActiveQueueByteCount() {
|
||||
return activeQueueByteCount;
|
||||
}
|
||||
|
||||
public void setActiveQueueByteCount(long activeQueueByteCount) {
|
||||
this.activeQueueByteCount = activeQueueByteCount;
|
||||
}
|
||||
|
||||
@ApiModelProperty("The total number of FlowFiles that are swapped out for this Connection")
|
||||
public int getSwapFlowFileCount() {
|
||||
return swapFlowFileCount;
|
||||
}
|
||||
|
||||
public void setSwapFlowFileCount(int swapFlowFileCount) {
|
||||
this.swapFlowFileCount = swapFlowFileCount;
|
||||
}
|
||||
|
||||
@ApiModelProperty("Total number of bytes that make up the content for the FlowFiles that are swapped out to disk for the Connection")
|
||||
public long getSwapByteCount() {
|
||||
return swapByteCount;
|
||||
}
|
||||
|
||||
public void setSwapByteCount(long swapByteCount) {
|
||||
this.swapByteCount = swapByteCount;
|
||||
}
|
||||
|
||||
@ApiModelProperty("The number of Swap Files that exist for this Connection")
|
||||
public int getSwapFiles() {
|
||||
return swapFiles;
|
||||
}
|
||||
|
||||
public void setSwapFiles(int swapFiles) {
|
||||
this.swapFiles = swapFiles;
|
||||
}
|
||||
|
||||
@ApiModelProperty("The number of In-Flight FlowFiles for this Connection. These are FlowFiles that belong to the connection but are currently being operated on by a Processor, Port, etc.")
|
||||
public int getInFlightFlowFileCount() {
|
||||
return inFlightFlowFileCount;
|
||||
}
|
||||
|
||||
public void setInFlightFlowFileCount(int inFlightFlowFileCount) {
|
||||
this.inFlightFlowFileCount = inFlightFlowFileCount;
|
||||
}
|
||||
|
||||
@ApiModelProperty("The number bytes that make up the content of the FlowFiles that are In-Flight")
|
||||
public long getInFlightByteCount() {
|
||||
return inFlightByteCount;
|
||||
}
|
||||
|
||||
public void setInFlightByteCount(long inFlightByteCount) {
|
||||
this.inFlightByteCount = inFlightByteCount;
|
||||
}
|
||||
|
||||
@ApiModelProperty("Whether or not all of the FlowFiles in the Active Queue are penalized")
|
||||
public Boolean getAllActiveQueueFlowFilesPenalized() {
|
||||
return allActiveQueueFlowFilesPenalized;
|
||||
}
|
||||
|
||||
public void setAllActiveQueueFlowFilesPenalized(Boolean allFlowFilesPenalized) {
|
||||
this.allActiveQueueFlowFilesPenalized = allFlowFilesPenalized;
|
||||
}
|
||||
|
||||
@ApiModelProperty("Whether or not any of the FlowFiles in the Active Queue are penalized")
|
||||
public Boolean getAnyActiveQueueFlowFilesPenalized() {
|
||||
return anyActiveQueueFlowFilesPenalized;
|
||||
}
|
||||
|
||||
public void setAnyActiveQueueFlowFilesPenalized(Boolean anyFlowFilesPenalized) {
|
||||
this.anyActiveQueueFlowFilesPenalized = anyFlowFilesPenalized;
|
||||
public void setNodeSnapshots(final List<ConnectionDiagnosticsSnapshotDTO> nodeSnapshots) {
|
||||
this.nodeSnapshots = nodeSnapshots;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,76 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.nifi.web.api.dto.diagnostics;
|
||||
|
||||
import io.swagger.annotations.ApiModelProperty;
|
||||
|
||||
import javax.xml.bind.annotation.XmlType;
|
||||
import java.util.List;
|
||||
|
||||
@XmlType(name = "connectionDiagnosticsSnapshot")
|
||||
public class ConnectionDiagnosticsSnapshotDTO {
|
||||
private int totalFlowFileCount;
|
||||
private long totalByteCount;
|
||||
private String nodeIdentifier;
|
||||
private LocalQueuePartitionDTO localQueuePartition;
|
||||
private List<RemoteQueuePartitionDTO> remoteQueuePartitions;
|
||||
|
||||
@ApiModelProperty("Total number of FlowFiles owned by the Connection")
|
||||
public int getTotalFlowFileCount() {
|
||||
return totalFlowFileCount;
|
||||
}
|
||||
|
||||
public void setTotalFlowFileCount(int totalFlowFileCount) {
|
||||
this.totalFlowFileCount = totalFlowFileCount;
|
||||
}
|
||||
|
||||
@ApiModelProperty("Total number of bytes that make up the content for the FlowFiles owned by this Connection")
|
||||
public long getTotalByteCount() {
|
||||
return totalByteCount;
|
||||
}
|
||||
|
||||
public void setTotalByteCount(long totalByteCount) {
|
||||
this.totalByteCount = totalByteCount;
|
||||
}
|
||||
|
||||
@ApiModelProperty("The Node Identifier that this information pertains to")
|
||||
public String getNodeIdentifier() {
|
||||
return nodeIdentifier;
|
||||
}
|
||||
|
||||
public void setNodeIdentifier(final String nodeIdentifier) {
|
||||
this.nodeIdentifier = nodeIdentifier;
|
||||
}
|
||||
|
||||
@ApiModelProperty("The local queue partition, from which components can pull FlowFiles on this node.")
|
||||
public LocalQueuePartitionDTO getLocalQueuePartition() {
|
||||
return localQueuePartition;
|
||||
}
|
||||
|
||||
public void setLocalQueuePartition(LocalQueuePartitionDTO localQueuePartition) {
|
||||
this.localQueuePartition = localQueuePartition;
|
||||
}
|
||||
|
||||
public List<RemoteQueuePartitionDTO> getRemoteQueuePartitions() {
|
||||
return remoteQueuePartitions;
|
||||
}
|
||||
|
||||
public void setRemoteQueuePartitions(List<RemoteQueuePartitionDTO> remoteQueuePartitions) {
|
||||
this.remoteQueuePartitions = remoteQueuePartitions;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,136 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.nifi.web.api.dto.diagnostics;
|
||||
|
||||
import io.swagger.annotations.ApiModelProperty;
|
||||
|
||||
import javax.xml.bind.annotation.XmlType;
|
||||
|
||||
@XmlType(name = "localQueuePartition")
|
||||
public class LocalQueuePartitionDTO {
|
||||
private int totalFlowFileCount;
|
||||
private long totalByteCount;
|
||||
private int activeQueueFlowFileCount;
|
||||
private long activeQueueByteCount;
|
||||
private int swapFlowFileCount;
|
||||
private long swapByteCount;
|
||||
private int swapFiles;
|
||||
private int inFlightFlowFileCount;
|
||||
private long inFlightByteCount;
|
||||
private Boolean allActiveQueueFlowFilesPenalized;
|
||||
private Boolean anyActiveQueueFlowFilesPenalized;
|
||||
|
||||
@ApiModelProperty("Total number of FlowFiles owned by the Connection")
|
||||
public int getTotalFlowFileCount() {
|
||||
return totalFlowFileCount;
|
||||
}
|
||||
|
||||
public void setTotalFlowFileCount(int totalFlowFileCount) {
|
||||
this.totalFlowFileCount = totalFlowFileCount;
|
||||
}
|
||||
|
||||
@ApiModelProperty("Total number of bytes that make up the content for the FlowFiles owned by this Connection")
|
||||
public long getTotalByteCount() {
|
||||
return totalByteCount;
|
||||
}
|
||||
|
||||
public void setTotalByteCount(long totalByteCount) {
|
||||
this.totalByteCount = totalByteCount;
|
||||
}
|
||||
|
||||
@ApiModelProperty("Total number of FlowFiles that exist in the Connection's Active Queue, immediately available to be offered up to a component")
|
||||
public int getActiveQueueFlowFileCount() {
|
||||
return activeQueueFlowFileCount;
|
||||
}
|
||||
|
||||
public void setActiveQueueFlowFileCount(int activeQueueFlowFileCount) {
|
||||
this.activeQueueFlowFileCount = activeQueueFlowFileCount;
|
||||
}
|
||||
|
||||
@ApiModelProperty("Total number of bytes that make up the content for the FlowFiles that are present in the Connection's Active Queue")
|
||||
public long getActiveQueueByteCount() {
|
||||
return activeQueueByteCount;
|
||||
}
|
||||
|
||||
public void setActiveQueueByteCount(long activeQueueByteCount) {
|
||||
this.activeQueueByteCount = activeQueueByteCount;
|
||||
}
|
||||
|
||||
@ApiModelProperty("The total number of FlowFiles that are swapped out for this Connection")
|
||||
public int getSwapFlowFileCount() {
|
||||
return swapFlowFileCount;
|
||||
}
|
||||
|
||||
public void setSwapFlowFileCount(int swapFlowFileCount) {
|
||||
this.swapFlowFileCount = swapFlowFileCount;
|
||||
}
|
||||
|
||||
@ApiModelProperty("Total number of bytes that make up the content for the FlowFiles that are swapped out to disk for the Connection")
|
||||
public long getSwapByteCount() {
|
||||
return swapByteCount;
|
||||
}
|
||||
|
||||
public void setSwapByteCount(long swapByteCount) {
|
||||
this.swapByteCount = swapByteCount;
|
||||
}
|
||||
|
||||
@ApiModelProperty("The number of Swap Files that exist for this Connection")
|
||||
public int getSwapFiles() {
|
||||
return swapFiles;
|
||||
}
|
||||
|
||||
public void setSwapFiles(int swapFiles) {
|
||||
this.swapFiles = swapFiles;
|
||||
}
|
||||
|
||||
@ApiModelProperty("The number of In-Flight FlowFiles for this Connection. These are FlowFiles that belong to the connection but are currently being operated on by a Processor, Port, etc.")
|
||||
public int getInFlightFlowFileCount() {
|
||||
return inFlightFlowFileCount;
|
||||
}
|
||||
|
||||
public void setInFlightFlowFileCount(int inFlightFlowFileCount) {
|
||||
this.inFlightFlowFileCount = inFlightFlowFileCount;
|
||||
}
|
||||
|
||||
@ApiModelProperty("The number bytes that make up the content of the FlowFiles that are In-Flight")
|
||||
public long getInFlightByteCount() {
|
||||
return inFlightByteCount;
|
||||
}
|
||||
|
||||
public void setInFlightByteCount(long inFlightByteCount) {
|
||||
this.inFlightByteCount = inFlightByteCount;
|
||||
}
|
||||
|
||||
@ApiModelProperty("Whether or not all of the FlowFiles in the Active Queue are penalized")
|
||||
public Boolean getAllActiveQueueFlowFilesPenalized() {
|
||||
return allActiveQueueFlowFilesPenalized;
|
||||
}
|
||||
|
||||
public void setAllActiveQueueFlowFilesPenalized(Boolean allFlowFilesPenalized) {
|
||||
this.allActiveQueueFlowFilesPenalized = allFlowFilesPenalized;
|
||||
}
|
||||
|
||||
@ApiModelProperty("Whether or not any of the FlowFiles in the Active Queue are penalized")
|
||||
public Boolean getAnyActiveQueueFlowFilesPenalized() {
|
||||
return anyActiveQueueFlowFilesPenalized;
|
||||
}
|
||||
|
||||
public void setAnyActiveQueueFlowFilesPenalized(Boolean anyFlowFilesPenalized) {
|
||||
this.anyActiveQueueFlowFilesPenalized = anyFlowFilesPenalized;
|
||||
}
|
||||
}
|
|
@ -17,15 +17,13 @@
|
|||
|
||||
package org.apache.nifi.web.api.dto.diagnostics;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
import javax.xml.bind.annotation.XmlType;
|
||||
|
||||
import io.swagger.annotations.ApiModelProperty;
|
||||
import org.apache.nifi.web.api.dto.ProcessorDTO;
|
||||
import org.apache.nifi.web.api.dto.status.ProcessorStatusDTO;
|
||||
|
||||
import io.swagger.annotations.ApiModelProperty;
|
||||
import javax.xml.bind.annotation.XmlType;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
@XmlType(name = "processorDiagnostics")
|
||||
public class ProcessorDiagnosticsDTO {
|
||||
|
|
|
@ -0,0 +1,126 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.nifi.web.api.dto.diagnostics;
|
||||
|
||||
import io.swagger.annotations.ApiModelProperty;
|
||||
|
||||
import javax.xml.bind.annotation.XmlType;
|
||||
|
||||
@XmlType(name = "remoteQueuePartition")
|
||||
public class RemoteQueuePartitionDTO {
|
||||
private String nodeId;
|
||||
private int totalFlowFileCount;
|
||||
private long totalByteCount;
|
||||
private int activeQueueFlowFileCount;
|
||||
private long activeQueueByteCount;
|
||||
private int swapFlowFileCount;
|
||||
private long swapByteCount;
|
||||
private int swapFiles;
|
||||
private int inFlightFlowFileCount;
|
||||
private long inFlightByteCount;
|
||||
|
||||
@ApiModelProperty("The Node Identifier that this queue partition is sending to")
|
||||
public String getNodeIdentifier() {
|
||||
return nodeId;
|
||||
}
|
||||
|
||||
public void setNodeIdentifier(String nodeId) {
|
||||
this.nodeId = nodeId;
|
||||
}
|
||||
|
||||
@ApiModelProperty("Total number of FlowFiles owned by the Connection")
|
||||
public int getTotalFlowFileCount() {
|
||||
return totalFlowFileCount;
|
||||
}
|
||||
|
||||
public void setTotalFlowFileCount(int totalFlowFileCount) {
|
||||
this.totalFlowFileCount = totalFlowFileCount;
|
||||
}
|
||||
|
||||
@ApiModelProperty("Total number of bytes that make up the content for the FlowFiles owned by this Connection")
|
||||
public long getTotalByteCount() {
|
||||
return totalByteCount;
|
||||
}
|
||||
|
||||
public void setTotalByteCount(long totalByteCount) {
|
||||
this.totalByteCount = totalByteCount;
|
||||
}
|
||||
|
||||
@ApiModelProperty("Total number of FlowFiles that exist in the Connection's Active Queue, immediately available to be offered up to a component")
|
||||
public int getActiveQueueFlowFileCount() {
|
||||
return activeQueueFlowFileCount;
|
||||
}
|
||||
|
||||
public void setActiveQueueFlowFileCount(int activeQueueFlowFileCount) {
|
||||
this.activeQueueFlowFileCount = activeQueueFlowFileCount;
|
||||
}
|
||||
|
||||
@ApiModelProperty("Total number of bytes that make up the content for the FlowFiles that are present in the Connection's Active Queue")
|
||||
public long getActiveQueueByteCount() {
|
||||
return activeQueueByteCount;
|
||||
}
|
||||
|
||||
public void setActiveQueueByteCount(long activeQueueByteCount) {
|
||||
this.activeQueueByteCount = activeQueueByteCount;
|
||||
}
|
||||
|
||||
@ApiModelProperty("The total number of FlowFiles that are swapped out for this Connection")
|
||||
public int getSwapFlowFileCount() {
|
||||
return swapFlowFileCount;
|
||||
}
|
||||
|
||||
public void setSwapFlowFileCount(int swapFlowFileCount) {
|
||||
this.swapFlowFileCount = swapFlowFileCount;
|
||||
}
|
||||
|
||||
@ApiModelProperty("Total number of bytes that make up the content for the FlowFiles that are swapped out to disk for the Connection")
|
||||
public long getSwapByteCount() {
|
||||
return swapByteCount;
|
||||
}
|
||||
|
||||
public void setSwapByteCount(long swapByteCount) {
|
||||
this.swapByteCount = swapByteCount;
|
||||
}
|
||||
|
||||
@ApiModelProperty("The number of Swap Files that exist for this Connection")
|
||||
public int getSwapFiles() {
|
||||
return swapFiles;
|
||||
}
|
||||
|
||||
public void setSwapFiles(int swapFiles) {
|
||||
this.swapFiles = swapFiles;
|
||||
}
|
||||
|
||||
@ApiModelProperty("The number of In-Flight FlowFiles for this Connection. These are FlowFiles that belong to the connection but are currently being operated on by a Processor, Port, etc.")
|
||||
public int getInFlightFlowFileCount() {
|
||||
return inFlightFlowFileCount;
|
||||
}
|
||||
|
||||
public void setInFlightFlowFileCount(int inFlightFlowFileCount) {
|
||||
this.inFlightFlowFileCount = inFlightFlowFileCount;
|
||||
}
|
||||
|
||||
@ApiModelProperty("The number bytes that make up the content of the FlowFiles that are In-Flight")
|
||||
public long getInFlightByteCount() {
|
||||
return inFlightByteCount;
|
||||
}
|
||||
|
||||
public void setInFlightByteCount(long inFlightByteCount) {
|
||||
this.inFlightByteCount = inFlightByteCount;
|
||||
}
|
||||
}
|
|
@ -17,11 +17,6 @@
|
|||
|
||||
package org.apache.nifi.cluster.coordination;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.nifi.cluster.coordination.node.DisconnectionCode;
|
||||
import org.apache.nifi.cluster.coordination.node.NodeConnectionState;
|
||||
import org.apache.nifi.cluster.coordination.node.NodeConnectionStatus;
|
||||
|
@ -31,6 +26,11 @@ import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
|||
import org.apache.nifi.reporting.Severity;
|
||||
import org.apache.nifi.services.FlowService;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* <p>
|
||||
* Responsible for coordinating nodes in the cluster
|
||||
|
@ -127,12 +127,12 @@ public interface ClusterCoordinator {
|
|||
* <code>true</code> if the node is blocked, <code>false</code> if the node is
|
||||
* allowed through the firewall or if there is no firewall configured
|
||||
*
|
||||
* @param hostname the hostname of the node that is attempting to connect to the cluster
|
||||
* @param nodeIdentities the identities of the node that is attempting to connect to the cluster
|
||||
*
|
||||
* @return <code>true</code> if the node is blocked, <code>false</code> if the node is
|
||||
* allowed through the firewall or if there is no firewall configured
|
||||
*/
|
||||
boolean isBlockedByFirewall(String hostname);
|
||||
boolean isBlockedByFirewall(Set<String> nodeIdentities);
|
||||
|
||||
/**
|
||||
* Reports that some event occurred that is relevant to the cluster
|
||||
|
@ -244,4 +244,16 @@ public interface ClusterCoordinator {
|
|||
* @throws IOException thrown when it failed to communicate with the cluster coordinator.
|
||||
*/
|
||||
Map<NodeIdentifier, NodeWorkload> getClusterWorkload() throws IOException;
|
||||
|
||||
/**
|
||||
* Registers the given event listener so that it is notified whenever a cluster topology event occurs
|
||||
* @param eventListener the event listener to notify
|
||||
*/
|
||||
void registerEventListener(ClusterTopologyEventListener eventListener);
|
||||
|
||||
/**
|
||||
* Stops notifying the given listener when cluster topology events occurs
|
||||
* @param eventListener the event listener to stop notifying
|
||||
*/
|
||||
void unregisterEventListener(ClusterTopologyEventListener eventListener);
|
||||
}
|
||||
|
|
|
@ -0,0 +1,29 @@
|
|||
/*
|
||||
* 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.cluster.coordination;
|
||||
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
|
||||
public interface ClusterTopologyEventListener {
|
||||
|
||||
void onNodeAdded(NodeIdentifier nodeId);
|
||||
|
||||
void onNodeRemoved(NodeIdentifier nodeId);
|
||||
|
||||
void onLocalNodeIdentifierSet(NodeIdentifier localNodeId);
|
||||
}
|
|
@ -16,11 +16,14 @@
|
|||
*/
|
||||
package org.apache.nifi.cluster.protocol;
|
||||
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
|
||||
import javax.xml.bind.annotation.XmlAccessType;
|
||||
import javax.xml.bind.annotation.XmlAccessorType;
|
||||
import javax.xml.bind.annotation.XmlRootElement;
|
||||
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* A node identifier denoting the coordinates of a flow controller that is
|
||||
|
@ -63,10 +66,20 @@ public class NodeIdentifier {
|
|||
private final String socketAddress;
|
||||
|
||||
/**
|
||||
* the port to use use for sending requests to the node's internal interface
|
||||
* the port to use for sending requests to the node's internal interface
|
||||
*/
|
||||
private final int socketPort;
|
||||
|
||||
/**
|
||||
* The IP or hostname to use for sending FlowFiles when load balancing a connection
|
||||
*/
|
||||
private final String loadBalanceAddress;
|
||||
|
||||
/**
|
||||
* the port to use for sending FlowFiles when load balancing a connection
|
||||
*/
|
||||
private final int loadBalancePort;
|
||||
|
||||
/**
|
||||
* the IP or hostname that external clients should use to communicate with this node via Site-to-Site
|
||||
*/
|
||||
|
@ -89,15 +102,20 @@ public class NodeIdentifier {
|
|||
private final Boolean siteToSiteSecure;
|
||||
|
||||
|
||||
private final String nodeDn;
|
||||
private final Set<String> nodeIdentities;
|
||||
|
||||
public NodeIdentifier(final String id, final String apiAddress, final int apiPort, final String socketAddress, final int socketPort,
|
||||
final String siteToSiteAddress, final Integer siteToSitePort, final Integer siteToSiteHttpApiPort, final boolean siteToSiteSecure) {
|
||||
this(id, apiAddress, apiPort, socketAddress, socketPort, siteToSiteAddress, siteToSitePort, siteToSiteHttpApiPort, siteToSiteSecure, null);
|
||||
final String siteToSiteAddress, final Integer siteToSitePort, final Integer siteToSiteHttpApiPort, final boolean siteToSiteSecure) {
|
||||
this(id, apiAddress, apiPort, socketAddress, socketPort, socketAddress, 6342, siteToSiteAddress, siteToSitePort, siteToSiteHttpApiPort, siteToSiteSecure, null);
|
||||
}
|
||||
|
||||
public NodeIdentifier(final String id, final String apiAddress, final int apiPort, final String socketAddress, final int socketPort,
|
||||
final String siteToSiteAddress, final Integer siteToSitePort, final Integer siteToSiteHttpApiPort, final boolean siteToSiteSecure, final String dn) {
|
||||
public NodeIdentifier(final String id, final String apiAddress, final int apiPort, final String socketAddress, final int socketPort, final String loadBalanceAddress, final int loadBalancePort,
|
||||
final String siteToSiteAddress, final Integer siteToSitePort, final Integer siteToSiteHttpApiPort, final boolean siteToSiteSecure) {
|
||||
this(id, apiAddress, apiPort, socketAddress, socketPort, loadBalanceAddress, loadBalancePort, siteToSiteAddress, siteToSitePort, siteToSiteHttpApiPort, siteToSiteSecure, null);
|
||||
}
|
||||
|
||||
public NodeIdentifier(final String id, final String apiAddress, final int apiPort, final String socketAddress, final int socketPort, final String loadBalanceAddress, final int loadBalancePort,
|
||||
final String siteToSiteAddress, final Integer siteToSitePort, final Integer siteToSiteHttpApiPort, final boolean siteToSiteSecure, final Set<String> nodeIdentities) {
|
||||
|
||||
if (StringUtils.isBlank(id)) {
|
||||
throw new IllegalArgumentException("Node ID may not be empty or null.");
|
||||
|
@ -109,6 +127,7 @@ public class NodeIdentifier {
|
|||
|
||||
validatePort(apiPort);
|
||||
validatePort(socketPort);
|
||||
validatePort(loadBalancePort);
|
||||
if (siteToSitePort != null) {
|
||||
validatePort(siteToSitePort);
|
||||
}
|
||||
|
@ -118,7 +137,9 @@ public class NodeIdentifier {
|
|||
this.apiPort = apiPort;
|
||||
this.socketAddress = socketAddress;
|
||||
this.socketPort = socketPort;
|
||||
this.nodeDn = dn;
|
||||
this.loadBalanceAddress = loadBalanceAddress;
|
||||
this.loadBalancePort = loadBalancePort;
|
||||
this.nodeIdentities = nodeIdentities == null ? Collections.emptySet() : Collections.unmodifiableSet(new HashSet<>(nodeIdentities));
|
||||
this.siteToSiteAddress = siteToSiteAddress == null ? apiAddress : siteToSiteAddress;
|
||||
this.siteToSitePort = siteToSitePort;
|
||||
this.siteToSiteHttpApiPort = siteToSiteHttpApiPort;
|
||||
|
@ -134,7 +155,9 @@ public class NodeIdentifier {
|
|||
this.apiPort = 0;
|
||||
this.socketAddress = null;
|
||||
this.socketPort = 0;
|
||||
this.nodeDn = null;
|
||||
this.loadBalanceAddress = null;
|
||||
this.loadBalancePort = 0;
|
||||
this.nodeIdentities = Collections.emptySet();
|
||||
this.siteToSiteAddress = null;
|
||||
this.siteToSitePort = null;
|
||||
this.siteToSiteHttpApiPort = null;
|
||||
|
@ -145,8 +168,8 @@ public class NodeIdentifier {
|
|||
return id;
|
||||
}
|
||||
|
||||
public String getDN() {
|
||||
return nodeDn;
|
||||
public Set<String> getNodeIdentities() {
|
||||
return nodeIdentities;
|
||||
}
|
||||
|
||||
public String getApiAddress() {
|
||||
|
@ -165,6 +188,14 @@ public class NodeIdentifier {
|
|||
return socketPort;
|
||||
}
|
||||
|
||||
public String getLoadBalanceAddress() {
|
||||
return loadBalanceAddress;
|
||||
}
|
||||
|
||||
public int getLoadBalancePort() {
|
||||
return loadBalancePort;
|
||||
}
|
||||
|
||||
private void validatePort(final int port) {
|
||||
if (port < 1 || port > 65535) {
|
||||
throw new IllegalArgumentException("Port must be inclusively in the range [1, 65535]. Port given: " + port);
|
||||
|
@ -235,6 +266,12 @@ public class NodeIdentifier {
|
|||
if (this.socketPort != other.socketPort) {
|
||||
return false;
|
||||
}
|
||||
if (!this.loadBalanceAddress.equals(other.loadBalanceAddress)) {
|
||||
return false;
|
||||
}
|
||||
if (this.loadBalancePort != other.loadBalancePort) {
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
|
|
@ -18,6 +18,8 @@ package org.apache.nifi.cluster.protocol;
|
|||
|
||||
import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
|
||||
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* A handler for processing protocol messages.
|
||||
*
|
||||
|
@ -30,11 +32,12 @@ public interface ProtocolHandler {
|
|||
* should be returned.
|
||||
*
|
||||
* @param msg a message
|
||||
* @param nodeIdentities the set of identities for this node
|
||||
* @return a response or null, if no response is necessary
|
||||
*
|
||||
* @throws ProtocolException if the message could not be processed
|
||||
*/
|
||||
ProtocolMessage handle(ProtocolMessage msg) throws ProtocolException;
|
||||
ProtocolMessage handle(ProtocolMessage msg, Set<String> nodeIdentities) throws ProtocolException;
|
||||
|
||||
/**
|
||||
* @param msg a message
|
||||
|
|
|
@ -16,15 +16,6 @@
|
|||
*/
|
||||
package org.apache.nifi.cluster.protocol.impl;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.net.Socket;
|
||||
import java.security.cert.CertificateException;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.CopyOnWriteArrayList;
|
||||
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.cluster.protocol.ProtocolContext;
|
||||
import org.apache.nifi.cluster.protocol.ProtocolException;
|
||||
|
@ -49,6 +40,22 @@ import org.apache.nifi.util.StopWatch;
|
|||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import javax.net.ssl.SSLPeerUnverifiedException;
|
||||
import javax.net.ssl.SSLSession;
|
||||
import javax.net.ssl.SSLSocket;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.net.Socket;
|
||||
import java.security.cert.Certificate;
|
||||
import java.security.cert.CertificateException;
|
||||
import java.security.cert.X509Certificate;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.Set;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.CopyOnWriteArrayList;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* Implements a listener for protocol messages sent over unicast socket.
|
||||
*
|
||||
|
@ -82,7 +89,6 @@ public class SocketProtocolListener extends SocketListener implements ProtocolLi
|
|||
|
||||
@Override
|
||||
public void start() throws IOException {
|
||||
|
||||
if (super.isRunning()) {
|
||||
throw new IllegalStateException("Instance is already started.");
|
||||
}
|
||||
|
@ -92,7 +98,6 @@ public class SocketProtocolListener extends SocketListener implements ProtocolLi
|
|||
|
||||
@Override
|
||||
public void stop() throws IOException {
|
||||
|
||||
if (super.isRunning() == false) {
|
||||
throw new IOException("Instance is already stopped.");
|
||||
}
|
||||
|
@ -128,8 +133,6 @@ public class SocketProtocolListener extends SocketListener implements ProtocolLi
|
|||
final String requestId = UUID.randomUUID().toString();
|
||||
logger.debug("Received request {} from {}", requestId, hostname);
|
||||
|
||||
String requestorDn = getRequestorDN(socket);
|
||||
|
||||
// unmarshall message
|
||||
final ProtocolMessageUnmarshaller<ProtocolMessage> unmarshaller = protocolContext.createUnmarshaller();
|
||||
final ByteCountingInputStream countingIn = new ByteCountingInputStream(socket.getInputStream());
|
||||
|
@ -151,7 +154,7 @@ public class SocketProtocolListener extends SocketListener implements ProtocolLi
|
|||
}
|
||||
}
|
||||
|
||||
request.setRequestorDN(requestorDn);
|
||||
final Set<String> nodeIdentities = getCertificateIdentities(socket);
|
||||
|
||||
// dispatch message to handler
|
||||
ProtocolHandler desiredHandler = null;
|
||||
|
@ -168,7 +171,7 @@ public class SocketProtocolListener extends SocketListener implements ProtocolLi
|
|||
logger.error("Received request of type {} but none of the following Protocol Handlers were able to process the request: {}", request.getType(), handlers);
|
||||
throw new ProtocolException("No handler assigned to handle message type: " + request.getType());
|
||||
} else {
|
||||
final ProtocolMessage response = desiredHandler.handle(request);
|
||||
final ProtocolMessage response = desiredHandler.handle(request, nodeIdentities);
|
||||
if (response != null) {
|
||||
try {
|
||||
logger.debug("Sending response for request {}", requestId);
|
||||
|
@ -218,11 +221,32 @@ public class SocketProtocolListener extends SocketListener implements ProtocolLi
|
|||
}
|
||||
}
|
||||
|
||||
private String getRequestorDN(Socket socket) {
|
||||
try {
|
||||
return CertificateUtils.extractPeerDNFromSSLSocket(socket);
|
||||
} catch (CertificateException e) {
|
||||
throw new ProtocolException(e);
|
||||
private Set<String> getCertificateIdentities(final Socket socket) throws IOException {
|
||||
if (socket instanceof SSLSocket) {
|
||||
try {
|
||||
final SSLSession sslSession = ((SSLSocket) socket).getSession();
|
||||
return getCertificateIdentities(sslSession);
|
||||
} catch (CertificateException e) {
|
||||
throw new IOException("Could not extract Subject Alternative Names from client's certificate", e);
|
||||
}
|
||||
} else {
|
||||
return Collections.emptySet();
|
||||
}
|
||||
}
|
||||
|
||||
private Set<String> getCertificateIdentities(final SSLSession sslSession) throws CertificateException, SSLPeerUnverifiedException {
|
||||
final Certificate[] certs = sslSession.getPeerCertificates();
|
||||
if (certs == null || certs.length == 0) {
|
||||
throw new SSLPeerUnverifiedException("No certificates found");
|
||||
}
|
||||
|
||||
final X509Certificate cert = CertificateUtils.convertAbstractX509Certificate(certs[0]);
|
||||
cert.checkValidity();
|
||||
|
||||
final Set<String> identities = CertificateUtils.getSubjectAlternativeNames(cert).stream()
|
||||
.map(CertificateUtils::extractUsername)
|
||||
.collect(Collectors.toSet());
|
||||
|
||||
return identities;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -25,6 +25,8 @@ public class AdaptedNodeIdentifier {
|
|||
private int apiPort;
|
||||
private String socketAddress;
|
||||
private int socketPort;
|
||||
private String loadBalanceAddress;
|
||||
private int loadBalancePort;
|
||||
private String siteToSiteAddress;
|
||||
private Integer siteToSitePort;
|
||||
private Integer siteToSiteHttpApiPort;
|
||||
|
@ -74,6 +76,22 @@ public class AdaptedNodeIdentifier {
|
|||
this.socketPort = socketPort;
|
||||
}
|
||||
|
||||
public String getLoadBalanceAddress() {
|
||||
return loadBalanceAddress;
|
||||
}
|
||||
|
||||
public void setLoadBalanceAddress(final String loadBalanceAddress) {
|
||||
this.loadBalanceAddress = loadBalanceAddress;
|
||||
}
|
||||
|
||||
public int getLoadBalancePort() {
|
||||
return loadBalancePort;
|
||||
}
|
||||
|
||||
public void setLoadBalancePort(final int loadBalancePort) {
|
||||
this.loadBalancePort = loadBalancePort;
|
||||
}
|
||||
|
||||
public String getSiteToSiteAddress() {
|
||||
return siteToSiteAddress;
|
||||
}
|
||||
|
|
|
@ -34,6 +34,8 @@ public class NodeIdentifierAdapter extends XmlAdapter<AdaptedNodeIdentifier, Nod
|
|||
aNi.setApiPort(ni.getApiPort());
|
||||
aNi.setSocketAddress(ni.getSocketAddress());
|
||||
aNi.setSocketPort(ni.getSocketPort());
|
||||
aNi.setLoadBalanceAddress(ni.getLoadBalanceAddress());
|
||||
aNi.setLoadBalancePort(ni.getLoadBalancePort());
|
||||
aNi.setSiteToSiteAddress(ni.getSiteToSiteAddress());
|
||||
aNi.setSiteToSitePort(ni.getSiteToSitePort());
|
||||
aNi.setSiteToSiteHttpApiPort(ni.getSiteToSiteHttpApiPort());
|
||||
|
@ -47,7 +49,7 @@ public class NodeIdentifierAdapter extends XmlAdapter<AdaptedNodeIdentifier, Nod
|
|||
if (aNi == null) {
|
||||
return null;
|
||||
} else {
|
||||
return new NodeIdentifier(aNi.getId(), aNi.getApiAddress(), aNi.getApiPort(), aNi.getSocketAddress(), aNi.getSocketPort(),
|
||||
return new NodeIdentifier(aNi.getId(), aNi.getApiAddress(), aNi.getApiPort(), aNi.getSocketAddress(), aNi.getSocketPort(), aNi.getLoadBalanceAddress(), aNi.getLoadBalancePort(),
|
||||
aNi.getSiteToSiteAddress(), aNi.getSiteToSitePort(),aNi.getSiteToSiteHttpApiPort(), aNi.isSiteToSiteSecure());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -18,8 +18,6 @@ package org.apache.nifi.cluster.protocol.message;
|
|||
|
||||
public abstract class ProtocolMessage {
|
||||
|
||||
private volatile String requestorDN;
|
||||
|
||||
public static enum MessageType {
|
||||
CONNECTION_REQUEST,
|
||||
CONNECTION_RESPONSE,
|
||||
|
@ -42,21 +40,4 @@ public abstract class ProtocolMessage {
|
|||
|
||||
public abstract MessageType getType();
|
||||
|
||||
/**
|
||||
* Sets the DN of the entity making the request
|
||||
*
|
||||
* @param dn dn of the entity making the request
|
||||
*/
|
||||
public void setRequestorDN(final String dn) {
|
||||
this.requestorDN = dn;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the DN of the entity that made the request, if using a secure
|
||||
* socket. Otherwise, returns <code>null</code>
|
||||
*/
|
||||
public String getRequestorDN() {
|
||||
return requestorDN;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -27,17 +27,17 @@
|
|||
<util:constant static-field="org.apache.nifi.cluster.protocol.jaxb.message.JaxbProtocolUtils.JAXB_CONTEXT"/>
|
||||
</constructor-arg>
|
||||
</bean>
|
||||
|
||||
|
||||
<!-- socket configuration -->
|
||||
<bean id="protocolSocketConfiguration" class="org.apache.nifi.cluster.protocol.spring.SocketConfigurationFactoryBean">
|
||||
<property name="properties" ref="nifiProperties"/>
|
||||
</bean>
|
||||
|
||||
|
||||
<!-- server socket configuration -->
|
||||
<bean id="protocolServerSocketConfiguration" class="org.apache.nifi.cluster.protocol.spring.ServerSocketConfigurationFactoryBean">
|
||||
<property name="properties" ref="nifiProperties"/>
|
||||
</bean>
|
||||
|
||||
|
||||
<!-- cluster manager protocol sender -->
|
||||
<bean id="clusterCoordinationProtocolSender" class="org.apache.nifi.cluster.protocol.impl.StandardClusterCoordinationProtocolSender">
|
||||
<constructor-arg ref="protocolSocketConfiguration"/>
|
||||
|
@ -49,13 +49,13 @@
|
|||
<bean factory-bean="nifiProperties" factory-method="getClusterNodeConnectionTimeout"/>
|
||||
</property>
|
||||
</bean>
|
||||
|
||||
|
||||
<!-- cluster manager sender/listener -->
|
||||
<bean id="clusterCoordinationProtocolSenderListener" class="org.apache.nifi.cluster.protocol.impl.ClusterCoordinationProtocolSenderListener">
|
||||
<constructor-arg ref="clusterCoordinationProtocolSender"/>
|
||||
<constructor-arg ref="protocolListener"/>
|
||||
</bean>
|
||||
|
||||
|
||||
<!-- node protocol sender -->
|
||||
<!--
|
||||
<bean id="nodeProtocolSender" class="org.apache.nifi.cluster.coordination.node.CuratorNodeProtocolSender">
|
||||
|
@ -69,7 +69,7 @@
|
|||
<constructor-arg ref="protocolContext"/>
|
||||
<constructor-arg ref="leaderElectionManager"/>
|
||||
</bean>
|
||||
|
||||
|
||||
<!-- protocol listener -->
|
||||
<bean id="protocolListener" class="org.apache.nifi.cluster.protocol.impl.SocketProtocolListener">
|
||||
<constructor-arg index="0">
|
||||
|
@ -81,7 +81,7 @@
|
|||
<constructor-arg ref="protocolServerSocketConfiguration" index="2"/>
|
||||
<constructor-arg ref="protocolContext" index="3"/>
|
||||
</bean>
|
||||
|
||||
|
||||
<!-- node sender/listener -->
|
||||
<bean id="nodeProtocolSenderListener" class="org.apache.nifi.cluster.protocol.impl.NodeProtocolSenderListener">
|
||||
<constructor-arg ref="nodeProtocolSender"/>
|
||||
|
|
|
@ -16,12 +16,14 @@
|
|||
*/
|
||||
package org.apache.nifi.cluster.protocol.impl.testutils;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import org.apache.nifi.cluster.protocol.ProtocolException;
|
||||
import org.apache.nifi.cluster.protocol.ProtocolHandler;
|
||||
import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class DelayedProtocolHandler implements ProtocolHandler {
|
||||
|
@ -34,7 +36,7 @@ public class DelayedProtocolHandler implements ProtocolHandler {
|
|||
}
|
||||
|
||||
@Override
|
||||
public ProtocolMessage handle(ProtocolMessage msg) throws ProtocolException {
|
||||
public ProtocolMessage handle(ProtocolMessage msg, Set<String> nodeIdentities) throws ProtocolException {
|
||||
try {
|
||||
messages.add(msg);
|
||||
Thread.sleep(delay);
|
||||
|
|
|
@ -16,12 +16,14 @@
|
|||
*/
|
||||
package org.apache.nifi.cluster.protocol.impl.testutils;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import org.apache.nifi.cluster.protocol.ProtocolException;
|
||||
import org.apache.nifi.cluster.protocol.ProtocolHandler;
|
||||
import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class ReflexiveProtocolHandler implements ProtocolHandler {
|
||||
|
@ -29,7 +31,7 @@ public class ReflexiveProtocolHandler implements ProtocolHandler {
|
|||
private List<ProtocolMessage> messages = new ArrayList<>();
|
||||
|
||||
@Override
|
||||
public ProtocolMessage handle(ProtocolMessage msg) throws ProtocolException {
|
||||
public ProtocolMessage handle(ProtocolMessage msg, Set<String> nodeIdentities) throws ProtocolException {
|
||||
messages.add(msg);
|
||||
return msg;
|
||||
}
|
||||
|
|
|
@ -29,6 +29,7 @@ import org.apache.nifi.util.StopWatch;
|
|||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ScheduledFuture;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
@ -198,7 +199,7 @@ public abstract class AbstractHeartbeatMonitor implements HeartbeatMonitor {
|
|||
final NodeIdentifier nodeId = heartbeat.getNodeIdentifier();
|
||||
|
||||
// Do not process heartbeat if it's blocked by firewall.
|
||||
if (clusterCoordinator.isBlockedByFirewall(nodeId.getSocketAddress())) {
|
||||
if (clusterCoordinator.isBlockedByFirewall(Collections.singleton(nodeId.getSocketAddress()))) {
|
||||
clusterCoordinator.reportEvent(nodeId, Severity.WARNING, "Firewall blocked received heartbeat. Issuing disconnection request.");
|
||||
|
||||
// request node to disconnect
|
||||
|
|
|
@ -16,16 +16,6 @@
|
|||
*/
|
||||
package org.apache.nifi.cluster.coordination.heartbeat;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.nifi.cluster.coordination.ClusterCoordinator;
|
||||
import org.apache.nifi.cluster.coordination.node.NodeConnectionState;
|
||||
import org.apache.nifi.cluster.coordination.node.NodeConnectionStatus;
|
||||
|
@ -46,6 +36,17 @@ import org.apache.nifi.util.NiFiProperties;
|
|||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* Uses Apache ZooKeeper to advertise the address to send heartbeats to, and
|
||||
* then relies on the NiFi Cluster Protocol to receive heartbeat messages from
|
||||
|
@ -134,7 +135,7 @@ public class ClusterProtocolHeartbeatMonitor extends AbstractHeartbeatMonitor im
|
|||
}
|
||||
|
||||
@Override
|
||||
public ProtocolMessage handle(final ProtocolMessage msg) throws ProtocolException {
|
||||
public ProtocolMessage handle(final ProtocolMessage msg, Set<String> nodeIds) throws ProtocolException {
|
||||
switch (msg.getType()) {
|
||||
case HEARTBEAT:
|
||||
return handleHeartbeat((HeartbeatMessage) msg);
|
||||
|
|
|
@ -16,13 +16,19 @@
|
|||
*/
|
||||
package org.apache.nifi.cluster.coordination.node;
|
||||
|
||||
import com.fasterxml.jackson.core.JsonFactory;
|
||||
import com.fasterxml.jackson.core.JsonGenerator;
|
||||
import com.fasterxml.jackson.core.JsonParser;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import org.apache.commons.collections4.queue.CircularFifoQueue;
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.cluster.coordination.ClusterCoordinator;
|
||||
import org.apache.nifi.cluster.coordination.ClusterTopologyEventListener;
|
||||
import org.apache.nifi.cluster.coordination.flow.FlowElection;
|
||||
import org.apache.nifi.cluster.coordination.http.HttpResponseMapper;
|
||||
import org.apache.nifi.cluster.coordination.http.StandardHttpResponseMapper;
|
||||
import org.apache.nifi.cluster.coordination.http.replication.RequestCompletionCallback;
|
||||
import org.apache.nifi.cluster.coordination.node.state.NodeIdentifierDescriptor;
|
||||
import org.apache.nifi.cluster.event.Event;
|
||||
import org.apache.nifi.cluster.event.NodeEvent;
|
||||
import org.apache.nifi.cluster.exception.NoClusterCoordinatorException;
|
||||
|
@ -49,8 +55,14 @@ import org.apache.nifi.cluster.protocol.message.NodeStatusChangeMessage;
|
|||
import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
|
||||
import org.apache.nifi.cluster.protocol.message.ProtocolMessage.MessageType;
|
||||
import org.apache.nifi.cluster.protocol.message.ReconnectionRequestMessage;
|
||||
import org.apache.nifi.components.state.Scope;
|
||||
import org.apache.nifi.components.state.StateManager;
|
||||
import org.apache.nifi.components.state.StateManagerProvider;
|
||||
import org.apache.nifi.components.state.StateMap;
|
||||
import org.apache.nifi.controller.leader.election.LeaderElectionManager;
|
||||
import org.apache.nifi.controller.state.manager.StandardStateManagerProvider;
|
||||
import org.apache.nifi.events.EventReporter;
|
||||
import org.apache.nifi.registry.VariableRegistry;
|
||||
import org.apache.nifi.reporting.Severity;
|
||||
import org.apache.nifi.services.FlowService;
|
||||
import org.apache.nifi.util.NiFiProperties;
|
||||
|
@ -59,6 +71,7 @@ import org.slf4j.Logger;
|
|||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.StringWriter;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
|
@ -69,6 +82,7 @@ import java.util.Set;
|
|||
import java.util.UUID;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import java.util.concurrent.CopyOnWriteArrayList;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.function.Supplier;
|
||||
import java.util.regex.Pattern;
|
||||
|
@ -93,6 +107,7 @@ public class NodeClusterCoordinator implements ClusterCoordinator, ProtocolHandl
|
|||
private final AtomicLong latestUpdateId = new AtomicLong(-1);
|
||||
private final FlowElection flowElection;
|
||||
private final NodeProtocolSender nodeProtocolSender;
|
||||
private final StateManager stateManager;
|
||||
|
||||
private volatile FlowService flowService;
|
||||
private volatile boolean connected;
|
||||
|
@ -102,9 +117,18 @@ public class NodeClusterCoordinator implements ClusterCoordinator, ProtocolHandl
|
|||
private final ConcurrentMap<NodeIdentifier, NodeConnectionStatus> nodeStatuses = new ConcurrentHashMap<>();
|
||||
private final ConcurrentMap<NodeIdentifier, CircularFifoQueue<NodeEvent>> nodeEvents = new ConcurrentHashMap<>();
|
||||
|
||||
private final List<ClusterTopologyEventListener> eventListeners = new CopyOnWriteArrayList<>();
|
||||
|
||||
public NodeClusterCoordinator(final ClusterCoordinationProtocolSenderListener senderListener, final EventReporter eventReporter, final LeaderElectionManager leaderElectionManager,
|
||||
final FlowElection flowElection, final ClusterNodeFirewall firewall, final RevisionManager revisionManager, final NiFiProperties nifiProperties,
|
||||
final NodeProtocolSender nodeProtocolSender) throws IOException {
|
||||
this(senderListener, eventReporter, leaderElectionManager, flowElection, firewall, revisionManager, nifiProperties, nodeProtocolSender,
|
||||
StandardStateManagerProvider.create(nifiProperties, VariableRegistry.EMPTY_REGISTRY));
|
||||
}
|
||||
|
||||
public NodeClusterCoordinator(final ClusterCoordinationProtocolSenderListener senderListener, final EventReporter eventReporter, final LeaderElectionManager leaderElectionManager,
|
||||
final FlowElection flowElection, final ClusterNodeFirewall firewall, final RevisionManager revisionManager, final NiFiProperties nifiProperties,
|
||||
final NodeProtocolSender nodeProtocolSender) {
|
||||
final NodeProtocolSender nodeProtocolSender, final StateManagerProvider stateManagerProvider) throws IOException {
|
||||
this.senderListener = senderListener;
|
||||
this.flowService = null;
|
||||
this.eventReporter = eventReporter;
|
||||
|
@ -114,10 +138,98 @@ public class NodeClusterCoordinator implements ClusterCoordinator, ProtocolHandl
|
|||
this.leaderElectionManager = leaderElectionManager;
|
||||
this.flowElection = flowElection;
|
||||
this.nodeProtocolSender = nodeProtocolSender;
|
||||
this.stateManager = stateManagerProvider.getStateManager("Cluster Coordinator");
|
||||
|
||||
recoverState();
|
||||
|
||||
senderListener.addHandler(this);
|
||||
}
|
||||
|
||||
private void recoverState() throws IOException {
|
||||
final StateMap stateMap = stateManager.getState(Scope.LOCAL);
|
||||
if (stateMap == null) {
|
||||
logger.debug("No state to restore");
|
||||
return;
|
||||
}
|
||||
|
||||
final ObjectMapper mapper = new ObjectMapper();
|
||||
final JsonFactory jsonFactory = new JsonFactory();
|
||||
jsonFactory.setCodec(mapper);
|
||||
|
||||
final Map<NodeIdentifier, NodeConnectionStatus> connectionStatusMap = new HashMap<>();
|
||||
NodeIdentifier localNodeId = null;
|
||||
|
||||
final Map<String, String> state = stateMap.toMap();
|
||||
for (final Map.Entry<String, String> entry : state.entrySet()) {
|
||||
final String nodeUuid = entry.getKey();
|
||||
final String nodeIdentifierJson = entry.getValue();
|
||||
logger.debug("Recovering state for {} = {}", nodeUuid, nodeIdentifierJson);
|
||||
|
||||
try (final JsonParser jsonParser = jsonFactory.createParser(nodeIdentifierJson)) {
|
||||
final NodeIdentifierDescriptor nodeIdDesc = jsonParser.readValueAs(NodeIdentifierDescriptor.class);
|
||||
final NodeIdentifier nodeId = nodeIdDesc.toNodeIdentifier();
|
||||
|
||||
connectionStatusMap.put(nodeId, new NodeConnectionStatus(nodeId, DisconnectionCode.NOT_YET_CONNECTED));
|
||||
if (nodeIdDesc.isLocalNodeIdentifier()) {
|
||||
if (localNodeId == null) {
|
||||
localNodeId = nodeId;
|
||||
} else {
|
||||
logger.warn("When recovering state, determined that tgwo Node Identifiers claim to be the local Node Identifier: {} and {}. Will ignore both of these and wait until " +
|
||||
"connecting to cluster to determine which Node Identiifer is the local Node Identifier", localNodeId, nodeId);
|
||||
localNodeId = null;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (!connectionStatusMap.isEmpty()) {
|
||||
resetNodeStatuses(connectionStatusMap);
|
||||
}
|
||||
|
||||
if (localNodeId != null) {
|
||||
logger.debug("Recovered state indicating that Local Node Identifier is {}", localNodeId);
|
||||
setLocalNodeIdentifier(localNodeId);
|
||||
}
|
||||
}
|
||||
|
||||
private void storeState() {
|
||||
final ObjectMapper mapper = new ObjectMapper();
|
||||
final JsonFactory jsonFactory = new JsonFactory();
|
||||
jsonFactory.setCodec(mapper);
|
||||
|
||||
try {
|
||||
final Map<String, String> stateMap = new HashMap<>();
|
||||
|
||||
final NodeIdentifier localNodeId = getLocalNodeIdentifier();
|
||||
for (final NodeIdentifier nodeId : getNodeIdentifiers()) {
|
||||
final boolean isLocalId = nodeId.equals(localNodeId);
|
||||
final NodeIdentifierDescriptor descriptor = NodeIdentifierDescriptor.fromNodeIdentifier(nodeId, isLocalId);
|
||||
|
||||
try (final StringWriter writer = new StringWriter()) {
|
||||
final JsonGenerator jsonGenerator = jsonFactory.createGenerator(writer);
|
||||
jsonGenerator.writeObject(descriptor);
|
||||
|
||||
final String serializedDescriptor = writer.toString();
|
||||
stateMap.put(nodeId.getId(), serializedDescriptor);
|
||||
}
|
||||
}
|
||||
|
||||
stateManager.setState(stateMap, Scope.LOCAL);
|
||||
logger.debug("Stored the following state as the Cluster Topology: {}", stateMap);
|
||||
} catch (final Exception e) {
|
||||
logger.warn("Failed to store cluster topology to local State Manager. Upon restart of NiFi, the cluster topology may not be accurate until joining the cluster.", e);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
public void registerEventListener(final ClusterTopologyEventListener eventListener) {
|
||||
this.eventListeners.add(eventListener);
|
||||
}
|
||||
|
||||
public void unregisterEventListener(final ClusterTopologyEventListener eventListener) {
|
||||
this.eventListeners.remove(eventListener);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void shutdown() {
|
||||
if (closed) {
|
||||
|
@ -136,8 +248,13 @@ public class NodeClusterCoordinator implements ClusterCoordinator, ProtocolHandl
|
|||
|
||||
@Override
|
||||
public void setLocalNodeIdentifier(final NodeIdentifier nodeId) {
|
||||
if (nodeId == null || nodeId.equals(this.nodeId)) {
|
||||
return;
|
||||
}
|
||||
|
||||
this.nodeId = nodeId;
|
||||
nodeStatuses.computeIfAbsent(nodeId, id -> new NodeConnectionStatus(id, DisconnectionCode.NOT_YET_CONNECTED));
|
||||
eventListeners.forEach(listener -> listener.onLocalNodeIdentifierSet(nodeId));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -170,7 +287,7 @@ public class NodeClusterCoordinator implements ClusterCoordinator, ProtocolHandl
|
|||
return localNodeId;
|
||||
}
|
||||
|
||||
private String getElectedActiveCoordinatorAddress() throws IOException {
|
||||
private String getElectedActiveCoordinatorAddress() {
|
||||
return leaderElectionManager.getLeader(ClusterRoles.CLUSTER_COORDINATOR);
|
||||
}
|
||||
|
||||
|
@ -185,11 +302,62 @@ public class NodeClusterCoordinator implements ClusterCoordinator, ProtocolHandl
|
|||
final NodeConnectionStatus proposedStatus = entry.getValue();
|
||||
|
||||
if (proposedStatus.getState() == NodeConnectionState.REMOVED) {
|
||||
nodeStatuses.remove(nodeId);
|
||||
removeNode(nodeId);
|
||||
} else {
|
||||
nodeStatuses.put(nodeId, proposedStatus);
|
||||
updateNodeStatus(nodeId, proposedStatus, false);
|
||||
}
|
||||
}
|
||||
|
||||
storeState();
|
||||
}
|
||||
|
||||
private NodeConnectionStatus removeNode(final NodeIdentifier nodeId) {
|
||||
final NodeConnectionStatus status = nodeStatuses.remove(nodeId);
|
||||
nodeEvents.remove(nodeId);
|
||||
if (status != null) {
|
||||
onNodeRemoved(nodeId);
|
||||
}
|
||||
|
||||
return status;
|
||||
}
|
||||
|
||||
private boolean removeNodeConditionally(final NodeIdentifier nodeId, final NodeConnectionStatus expectedStatus) {
|
||||
final boolean removed = nodeStatuses.remove(nodeId, expectedStatus);
|
||||
if (removed) {
|
||||
nodeEvents.remove(nodeId);
|
||||
onNodeRemoved(nodeId);
|
||||
}
|
||||
|
||||
return removed;
|
||||
}
|
||||
|
||||
private NodeConnectionStatus updateNodeStatus(final NodeIdentifier nodeId, final NodeConnectionStatus updatedStatus) {
|
||||
return updateNodeStatus(nodeId, updatedStatus, true);
|
||||
}
|
||||
|
||||
private NodeConnectionStatus updateNodeStatus(final NodeIdentifier nodeId, final NodeConnectionStatus updatedStatus, final boolean storeState) {
|
||||
final NodeConnectionStatus evictedStatus = nodeStatuses.put(nodeId, updatedStatus);
|
||||
if (evictedStatus == null) {
|
||||
onNodeAdded(nodeId, storeState);
|
||||
}
|
||||
|
||||
return evictedStatus;
|
||||
}
|
||||
|
||||
private boolean updateNodeStatusConditionally(final NodeIdentifier nodeId, final NodeConnectionStatus expectedStatus, final NodeConnectionStatus updatedStatus) {
|
||||
final boolean updated;
|
||||
if (expectedStatus == null) {
|
||||
final NodeConnectionStatus existingValue = nodeStatuses.putIfAbsent(nodeId, updatedStatus);
|
||||
updated = existingValue == null;
|
||||
|
||||
if (updated) {
|
||||
onNodeAdded(nodeId, true);
|
||||
}
|
||||
} else {
|
||||
updated = nodeStatuses.replace(nodeId, expectedStatus, updatedStatus);
|
||||
}
|
||||
|
||||
return updated;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -228,17 +396,21 @@ public class NodeClusterCoordinator implements ClusterCoordinator, ProtocolHandl
|
|||
|
||||
if (currentStatus == null) {
|
||||
if (newStatus.getState() == NodeConnectionState.REMOVED) {
|
||||
return nodeStatuses.remove(nodeId, currentStatus);
|
||||
return removeNodeConditionally(nodeId, currentStatus);
|
||||
} else {
|
||||
final NodeConnectionStatus existingValue = nodeStatuses.putIfAbsent(nodeId, newStatus);
|
||||
return existingValue == null;
|
||||
return updateNodeStatusConditionally(nodeId, null, newStatus);
|
||||
}
|
||||
}
|
||||
|
||||
if (newStatus.getState() == NodeConnectionState.REMOVED) {
|
||||
return nodeStatuses.remove(nodeId, currentStatus);
|
||||
if (removeNodeConditionally(nodeId, currentStatus)) {
|
||||
storeState();
|
||||
return true;
|
||||
} else {
|
||||
return false;
|
||||
}
|
||||
} else {
|
||||
return nodeStatuses.replace(nodeId, currentStatus, newStatus);
|
||||
return updateNodeStatusConditionally(nodeId, currentStatus, newStatus);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -348,9 +520,23 @@ public class NodeClusterCoordinator implements ClusterCoordinator, ProtocolHandl
|
|||
@Override
|
||||
public void removeNode(final NodeIdentifier nodeId, final String userDn) {
|
||||
reportEvent(nodeId, Severity.INFO, "User " + userDn + " requested that node be removed from cluster");
|
||||
nodeStatuses.remove(nodeId);
|
||||
nodeEvents.remove(nodeId);
|
||||
notifyOthersOfNodeStatusChange(new NodeConnectionStatus(nodeId, NodeConnectionState.REMOVED));
|
||||
removeNode(nodeId);
|
||||
|
||||
storeState();
|
||||
}
|
||||
|
||||
private void onNodeRemoved(final NodeIdentifier nodeId) {
|
||||
eventListeners.stream().forEach(listener -> listener.onNodeRemoved(nodeId));
|
||||
}
|
||||
|
||||
private void onNodeAdded(final NodeIdentifier nodeId, final boolean storeState) {
|
||||
if (storeState) {
|
||||
storeState();
|
||||
}
|
||||
|
||||
|
||||
eventListeners.stream().forEach(listener -> listener.onNodeAdded(nodeId));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -381,8 +567,18 @@ public class NodeClusterCoordinator implements ClusterCoordinator, ProtocolHandl
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean isBlockedByFirewall(final String hostname) {
|
||||
return firewall != null && !firewall.isPermissible(hostname);
|
||||
public boolean isBlockedByFirewall(final Set<String> nodeIdentities) {
|
||||
if (firewall == null) {
|
||||
return false;
|
||||
}
|
||||
|
||||
for (final String nodeId : nodeIdentities) {
|
||||
if (firewall.isPermissible(nodeId)) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -455,28 +651,21 @@ public class NodeClusterCoordinator implements ClusterCoordinator, ProtocolHandl
|
|||
}
|
||||
|
||||
private NodeIdentifier getElectedActiveCoordinatorNode(final boolean warnOnError) {
|
||||
final String electedNodeAddress;
|
||||
String electedNodeAddress;
|
||||
try {
|
||||
electedNodeAddress = getElectedActiveCoordinatorAddress();
|
||||
} catch (final NoClusterCoordinatorException ncce) {
|
||||
logger.debug("There is currently no elected active Cluster Coordinator");
|
||||
return null;
|
||||
} catch (final IOException ioe) {
|
||||
if (warnOnError) {
|
||||
logger.warn("Failed to determine which node is elected active Cluster Coordinator. There may be no coordinator currently: " + ioe);
|
||||
if (logger.isDebugEnabled()) {
|
||||
logger.warn("", ioe);
|
||||
}
|
||||
}
|
||||
|
||||
return null;
|
||||
}
|
||||
|
||||
if (electedNodeAddress == null) {
|
||||
if (electedNodeAddress == null || electedNodeAddress.trim().isEmpty()) {
|
||||
logger.debug("There is currently no elected active Cluster Coordinator");
|
||||
return null;
|
||||
}
|
||||
|
||||
electedNodeAddress = electedNodeAddress.trim();
|
||||
|
||||
final int colonLoc = electedNodeAddress.indexOf(':');
|
||||
if (colonLoc < 1) {
|
||||
if (warnOnError) {
|
||||
|
@ -519,6 +708,7 @@ public class NodeClusterCoordinator implements ClusterCoordinator, ProtocolHandl
|
|||
|
||||
final NodeConnectionStatus existingStatus = this.nodeStatuses.putIfAbsent(connectionStatus.getNodeIdentifier(), connectionStatus);
|
||||
if (existingStatus == null) {
|
||||
onNodeAdded(connectionStatus.getNodeIdentifier(), true);
|
||||
return connectionStatus.getNodeIdentifier();
|
||||
} else {
|
||||
return existingStatus.getNodeIdentifier();
|
||||
|
@ -594,7 +784,7 @@ public class NodeClusterCoordinator implements ClusterCoordinator, ProtocolHandl
|
|||
// this method is called when something occurs that causes this node to change the status of the
|
||||
// node in question. We only use comparisons against the current value when we receive an update
|
||||
// about a node status from a different node, since those may be received out-of-order.
|
||||
final NodeConnectionStatus currentStatus = nodeStatuses.put(nodeId, status);
|
||||
final NodeConnectionStatus currentStatus = updateNodeStatus(nodeId, status);
|
||||
final NodeConnectionState currentState = currentStatus == null ? null : currentStatus.getState();
|
||||
logger.info("Status of {} changed from {} to {}", nodeId, currentStatus, status);
|
||||
logger.debug("State of cluster nodes is now {}", nodeStatuses);
|
||||
|
@ -741,10 +931,10 @@ public class NodeClusterCoordinator implements ClusterCoordinator, ProtocolHandl
|
|||
}
|
||||
|
||||
@Override
|
||||
public ProtocolMessage handle(final ProtocolMessage protocolMessage) throws ProtocolException {
|
||||
public ProtocolMessage handle(final ProtocolMessage protocolMessage, final Set<String> nodeIdentities) throws ProtocolException {
|
||||
switch (protocolMessage.getType()) {
|
||||
case CONNECTION_REQUEST:
|
||||
return handleConnectionRequest((ConnectionRequestMessage) protocolMessage);
|
||||
return handleConnectionRequest((ConnectionRequestMessage) protocolMessage, nodeIdentities);
|
||||
case NODE_STATUS_CHANGE:
|
||||
handleNodeStatusChange((NodeStatusChangeMessage) protocolMessage);
|
||||
return null;
|
||||
|
@ -790,9 +980,11 @@ public class NodeClusterCoordinator implements ClusterCoordinator, ProtocolHandl
|
|||
|
||||
// Either remove the value from the map or update the map depending on the connection state
|
||||
if (statusChangeMessage.getNodeConnectionStatus().getState() == NodeConnectionState.REMOVED) {
|
||||
nodeStatuses.remove(nodeId, oldStatus);
|
||||
if (removeNodeConditionally(nodeId, oldStatus)) {
|
||||
storeState();
|
||||
}
|
||||
} else {
|
||||
nodeStatuses.put(nodeId, updatedStatus);
|
||||
updateNodeStatus(nodeId, updatedStatus);
|
||||
}
|
||||
|
||||
logger.info("Status of {} changed from {} to {}", statusChangeMessage.getNodeId(), oldStatus, updatedStatus);
|
||||
|
@ -838,6 +1030,7 @@ public class NodeClusterCoordinator implements ClusterCoordinator, ProtocolHandl
|
|||
// there is no node with that ID
|
||||
resolvedNodeId = proposedIdentifier;
|
||||
logger.debug("No existing node with ID {}; resolved node ID is as-proposed", proposedIdentifier.getId());
|
||||
onNodeAdded(resolvedNodeId, true);
|
||||
} else if (existingStatus.getNodeIdentifier().logicallyEquals(proposedIdentifier)) {
|
||||
// there is a node with that ID but it's the same node.
|
||||
resolvedNodeId = proposedIdentifier;
|
||||
|
@ -854,28 +1047,37 @@ public class NodeClusterCoordinator implements ClusterCoordinator, ProtocolHandl
|
|||
return resolvedNodeId;
|
||||
}
|
||||
|
||||
private ConnectionResponseMessage handleConnectionRequest(final ConnectionRequestMessage requestMessage) {
|
||||
private ConnectionResponseMessage handleConnectionRequest(final ConnectionRequestMessage requestMessage, final Set<String> nodeIdentities) {
|
||||
final NodeIdentifier proposedIdentifier = requestMessage.getConnectionRequest().getProposedNodeIdentifier();
|
||||
final NodeIdentifier withRequestorDn = addRequestorDn(proposedIdentifier, requestMessage.getRequestorDN());
|
||||
final NodeIdentifier withNodeIdentities = addNodeIdentities(proposedIdentifier, nodeIdentities);
|
||||
final DataFlow dataFlow = requestMessage.getConnectionRequest().getDataFlow();
|
||||
final ConnectionRequest requestWithDn = new ConnectionRequest(withRequestorDn, dataFlow);
|
||||
final ConnectionRequest requestWithNodeIdentities = new ConnectionRequest(withNodeIdentities, dataFlow);
|
||||
|
||||
// Resolve Node identifier.
|
||||
final NodeIdentifier resolvedNodeId = resolveNodeId(proposedIdentifier);
|
||||
|
||||
if (isBlockedByFirewall(nodeIdentities)) {
|
||||
// if the socket address is not listed in the firewall, then return a null response
|
||||
logger.info("Firewall blocked connection request from node " + resolvedNodeId + " with Node Identities " + nodeIdentities);
|
||||
final ConnectionResponse response = ConnectionResponse.createBlockedByFirewallResponse();
|
||||
final ConnectionResponseMessage responseMessage = new ConnectionResponseMessage();
|
||||
responseMessage.setConnectionResponse(response);
|
||||
return responseMessage;
|
||||
}
|
||||
|
||||
if (requireElection) {
|
||||
final DataFlow electedDataFlow = flowElection.castVote(dataFlow, withRequestorDn);
|
||||
final DataFlow electedDataFlow = flowElection.castVote(dataFlow, withNodeIdentities);
|
||||
if (electedDataFlow == null) {
|
||||
logger.info("Received Connection Request from {}; responding with Flow Election In Progress message", withRequestorDn);
|
||||
logger.info("Received Connection Request from {}; responding with Flow Election In Progress message", withNodeIdentities);
|
||||
return createFlowElectionInProgressResponse();
|
||||
} else {
|
||||
logger.info("Received Connection Request from {}; responding with DataFlow that was elected", withRequestorDn);
|
||||
return createConnectionResponse(requestWithDn, resolvedNodeId, electedDataFlow);
|
||||
logger.info("Received Connection Request from {}; responding with DataFlow that was elected", withNodeIdentities);
|
||||
return createConnectionResponse(requestWithNodeIdentities, resolvedNodeId, electedDataFlow);
|
||||
}
|
||||
}
|
||||
|
||||
logger.info("Received Connection Request from {}; responding with my DataFlow", withRequestorDn);
|
||||
return createConnectionResponse(requestWithDn, resolvedNodeId);
|
||||
logger.info("Received Connection Request from {}; responding with my DataFlow", withNodeIdentities);
|
||||
return createConnectionResponse(requestWithNodeIdentities, resolvedNodeId);
|
||||
}
|
||||
|
||||
private ConnectionResponseMessage createFlowElectionInProgressResponse() {
|
||||
|
@ -901,15 +1103,6 @@ public class NodeClusterCoordinator implements ClusterCoordinator, ProtocolHandl
|
|||
|
||||
|
||||
private ConnectionResponseMessage createConnectionResponse(final ConnectionRequest request, final NodeIdentifier resolvedNodeIdentifier, final DataFlow clusterDataFlow) {
|
||||
if (isBlockedByFirewall(resolvedNodeIdentifier.getSocketAddress())) {
|
||||
// if the socket address is not listed in the firewall, then return a null response
|
||||
logger.info("Firewall blocked connection request from node " + resolvedNodeIdentifier);
|
||||
final ConnectionResponse response = ConnectionResponse.createBlockedByFirewallResponse();
|
||||
final ConnectionResponseMessage responseMessage = new ConnectionResponseMessage();
|
||||
responseMessage.setConnectionResponse(response);
|
||||
return responseMessage;
|
||||
}
|
||||
|
||||
if (clusterDataFlow == null) {
|
||||
final ConnectionResponseMessage responseMessage = new ConnectionResponseMessage();
|
||||
responseMessage.setConnectionResponse(new ConnectionResponse(5, "The cluster dataflow is not yet available"));
|
||||
|
@ -936,11 +1129,12 @@ public class NodeClusterCoordinator implements ClusterCoordinator, ProtocolHandl
|
|||
}
|
||||
|
||||
|
||||
private NodeIdentifier addRequestorDn(final NodeIdentifier nodeId, final String dn) {
|
||||
private NodeIdentifier addNodeIdentities(final NodeIdentifier nodeId, final Set<String> nodeIdentities) {
|
||||
return new NodeIdentifier(nodeId.getId(), nodeId.getApiAddress(), nodeId.getApiPort(),
|
||||
nodeId.getSocketAddress(), nodeId.getSocketPort(),
|
||||
nodeId.getLoadBalanceAddress(), nodeId.getLoadBalancePort(),
|
||||
nodeId.getSiteToSiteAddress(), nodeId.getSiteToSitePort(),
|
||||
nodeId.getSiteToSiteHttpApiPort(), nodeId.isSiteToSiteSecure(), dn);
|
||||
nodeId.getSiteToSiteHttpApiPort(), nodeId.isSiteToSiteSecure(), nodeIdentities);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -0,0 +1,167 @@
|
|||
/*
|
||||
* 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.cluster.coordination.node.state;
|
||||
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
import java.util.Set;
|
||||
|
||||
public class NodeIdentifierDescriptor {
|
||||
private String id;
|
||||
private String apiAddress;
|
||||
private int apiPort;
|
||||
private String socketAddress;
|
||||
private int socketPort;
|
||||
private String loadBalanceAddress;
|
||||
private int loadBalancePort;
|
||||
private String siteToSiteAddress;
|
||||
private Integer siteToSitePort;
|
||||
private Integer siteToSiteHttpApiPort;
|
||||
private Boolean siteToSiteSecure;
|
||||
private Set<String> nodeIdentities;
|
||||
private boolean localNodeIdentifier;
|
||||
|
||||
public String getId() {
|
||||
return id;
|
||||
}
|
||||
|
||||
public void setId(final String id) {
|
||||
this.id = id;
|
||||
}
|
||||
|
||||
public String getApiAddress() {
|
||||
return apiAddress;
|
||||
}
|
||||
|
||||
public void setApiAddress(final String apiAddress) {
|
||||
this.apiAddress = apiAddress;
|
||||
}
|
||||
|
||||
public int getApiPort() {
|
||||
return apiPort;
|
||||
}
|
||||
|
||||
public void setApiPort(final int apiPort) {
|
||||
this.apiPort = apiPort;
|
||||
}
|
||||
|
||||
public String getSocketAddress() {
|
||||
return socketAddress;
|
||||
}
|
||||
|
||||
public void setSocketAddress(final String socketAddress) {
|
||||
this.socketAddress = socketAddress;
|
||||
}
|
||||
|
||||
public int getSocketPort() {
|
||||
return socketPort;
|
||||
}
|
||||
|
||||
public void setSocketPort(final int socketPort) {
|
||||
this.socketPort = socketPort;
|
||||
}
|
||||
|
||||
public String getLoadBalanceAddress() {
|
||||
return loadBalanceAddress;
|
||||
}
|
||||
|
||||
public void setLoadBalanceAddress(final String loadBalanceAddress) {
|
||||
this.loadBalanceAddress = loadBalanceAddress;
|
||||
}
|
||||
|
||||
public int getLoadBalancePort() {
|
||||
return loadBalancePort;
|
||||
}
|
||||
|
||||
public void setLoadBalancePort(final int loadBalancePort) {
|
||||
this.loadBalancePort = loadBalancePort;
|
||||
}
|
||||
|
||||
public String getSiteToSiteAddress() {
|
||||
return siteToSiteAddress;
|
||||
}
|
||||
|
||||
public void setSiteToSiteAddress(final String siteToSiteAddress) {
|
||||
this.siteToSiteAddress = siteToSiteAddress;
|
||||
}
|
||||
|
||||
public Integer getSiteToSitePort() {
|
||||
return siteToSitePort;
|
||||
}
|
||||
|
||||
public void setSiteToSitePort(final Integer siteToSitePort) {
|
||||
this.siteToSitePort = siteToSitePort;
|
||||
}
|
||||
|
||||
public Integer getSiteToSiteHttpApiPort() {
|
||||
return siteToSiteHttpApiPort;
|
||||
}
|
||||
|
||||
public void setSiteToSiteHttpApiPort(final Integer siteToSiteHttpApiPort) {
|
||||
this.siteToSiteHttpApiPort = siteToSiteHttpApiPort;
|
||||
}
|
||||
|
||||
public Boolean getSiteToSiteSecure() {
|
||||
return siteToSiteSecure;
|
||||
}
|
||||
|
||||
public void setSiteToSiteSecure(final Boolean siteToSiteSecure) {
|
||||
this.siteToSiteSecure = siteToSiteSecure;
|
||||
}
|
||||
|
||||
public Set<String> getNodeIdentities() {
|
||||
return nodeIdentities;
|
||||
}
|
||||
|
||||
public void setNodeIdentities(final Set<String> nodeIdentities) {
|
||||
this.nodeIdentities = Collections.unmodifiableSet(new HashSet<>(nodeIdentities));
|
||||
}
|
||||
|
||||
public boolean isLocalNodeIdentifier() {
|
||||
return localNodeIdentifier;
|
||||
}
|
||||
|
||||
public void setLocalNodeIdentifier(final boolean localNodeIdentifier) {
|
||||
this.localNodeIdentifier = localNodeIdentifier;
|
||||
}
|
||||
|
||||
public static NodeIdentifierDescriptor fromNodeIdentifier(final NodeIdentifier nodeId, final boolean localNodeId) {
|
||||
final NodeIdentifierDescriptor descriptor = new NodeIdentifierDescriptor();
|
||||
descriptor.setId(nodeId.getId());
|
||||
descriptor.setApiAddress(nodeId.getApiAddress());
|
||||
descriptor.setApiPort(nodeId.getApiPort());
|
||||
descriptor.setSocketAddress(nodeId.getSocketAddress());
|
||||
descriptor.setSocketPort(nodeId.getSocketPort());
|
||||
descriptor.setSiteToSiteAddress(nodeId.getSiteToSiteAddress());
|
||||
descriptor.setSiteToSitePort(nodeId.getSiteToSitePort());
|
||||
descriptor.setSiteToSiteHttpApiPort(nodeId.getSiteToSiteHttpApiPort());
|
||||
descriptor.setSiteToSiteSecure(nodeId.isSiteToSiteSecure());
|
||||
descriptor.setNodeIdentities(nodeId.getNodeIdentities());
|
||||
descriptor.setLoadBalanceAddress(nodeId.getLoadBalanceAddress());
|
||||
descriptor.setLoadBalancePort(nodeId.getLoadBalancePort());
|
||||
descriptor.setLocalNodeIdentifier(localNodeId);
|
||||
return descriptor;
|
||||
}
|
||||
|
||||
public NodeIdentifier toNodeIdentifier() {
|
||||
return new NodeIdentifier(getId(), getApiAddress(), getApiPort(), getSocketAddress(), getSocketPort(), getLoadBalanceAddress(), getLoadBalancePort(),
|
||||
getSiteToSiteAddress(), getSiteToSitePort(), getSiteToSiteHttpApiPort(), getSiteToSiteSecure(), getNodeIdentities());
|
||||
}
|
||||
}
|
|
@ -17,10 +17,12 @@
|
|||
package org.apache.nifi.cluster.manager;
|
||||
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.web.api.dto.ConnectionDTO;
|
||||
import org.apache.nifi.web.api.dto.status.ConnectionStatusDTO;
|
||||
import org.apache.nifi.web.api.entity.ConnectionEntity;
|
||||
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
||||
public class ConnectionEntityMerger implements ComponentEntityMerger<ConnectionEntity>, ComponentEntityStatusMerger<ConnectionStatusDTO> {
|
||||
|
||||
|
@ -33,6 +35,22 @@ public class ConnectionEntityMerger implements ComponentEntityMerger<ConnectionE
|
|||
mergeStatus(clientEntity.getStatus(), clientEntity.getPermissions().getCanRead(), entry.getValue().getStatus(), entry.getValue().getPermissions().getCanRead(), entry.getKey());
|
||||
}
|
||||
}
|
||||
|
||||
// If Load Balancing is configured but client entity indicates that data is not being transferred, we need to check if any other
|
||||
// node is actively transferring data. If Client Entity is transferring data, we already know the correct value for the Status,
|
||||
// and if the Connection is not configured for Load Balancing, then we also know the correct value, so no need to look at all of
|
||||
// the values of the other nodes.
|
||||
if (clientEntity.getComponent() != null && ConnectionDTO.LOAD_BALANCE_INACTIVE.equals(clientEntity.getComponent().getLoadBalanceStatus())) {
|
||||
final boolean anyActive = entityMap.values().stream()
|
||||
.map(ConnectionEntity::getComponent)
|
||||
.filter(Objects::nonNull)
|
||||
.map(ConnectionDTO::getLoadBalanceStatus)
|
||||
.anyMatch(status -> status.equals(ConnectionDTO.LOAD_BALANCE_ACTIVE));
|
||||
|
||||
if (anyActive) {
|
||||
clientEntity.getComponent().setLoadBalanceStatus(ConnectionDTO.LOAD_BALANCE_ACTIVE);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -17,22 +17,27 @@
|
|||
|
||||
package org.apache.nifi.cluster.manager;
|
||||
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.web.api.dto.diagnostics.ConnectionDiagnosticsDTO;
|
||||
import org.apache.nifi.web.api.dto.diagnostics.ConnectionDiagnosticsSnapshotDTO;
|
||||
import org.apache.nifi.web.api.dto.diagnostics.ControllerServiceDiagnosticsDTO;
|
||||
import org.apache.nifi.web.api.dto.diagnostics.JVMDiagnosticsSnapshotDTO;
|
||||
import org.apache.nifi.web.api.dto.diagnostics.LocalQueuePartitionDTO;
|
||||
import org.apache.nifi.web.api.dto.diagnostics.NodeJVMDiagnosticsSnapshotDTO;
|
||||
import org.apache.nifi.web.api.dto.diagnostics.ProcessorDiagnosticsDTO;
|
||||
import org.apache.nifi.web.api.dto.diagnostics.RemoteQueuePartitionDTO;
|
||||
import org.apache.nifi.web.api.dto.diagnostics.ThreadDumpDTO;
|
||||
import org.apache.nifi.web.api.entity.ControllerServiceEntity;
|
||||
import org.apache.nifi.web.api.entity.ProcessorDiagnosticsEntity;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.web.api.dto.diagnostics.ControllerServiceDiagnosticsDTO;
|
||||
import org.apache.nifi.web.api.dto.diagnostics.JVMDiagnosticsSnapshotDTO;
|
||||
import org.apache.nifi.web.api.dto.diagnostics.NodeJVMDiagnosticsSnapshotDTO;
|
||||
import org.apache.nifi.web.api.dto.diagnostics.ProcessorDiagnosticsDTO;
|
||||
import org.apache.nifi.web.api.dto.diagnostics.ThreadDumpDTO;
|
||||
import org.apache.nifi.web.api.entity.ControllerServiceEntity;
|
||||
import org.apache.nifi.web.api.entity.ProcessorDiagnosticsEntity;
|
||||
|
||||
public class ProcessorDiagnosticsEntityMerger implements ComponentEntityMerger<ProcessorDiagnosticsEntity> {
|
||||
private final long componentStatusSnapshotMillis;
|
||||
|
||||
|
@ -46,6 +51,11 @@ public class ProcessorDiagnosticsEntityMerger implements ComponentEntityMerger<P
|
|||
|
||||
final List<NodeJVMDiagnosticsSnapshotDTO> nodeJvmDiagnosticsSnapshots = new ArrayList<>(entityMap.size());
|
||||
|
||||
// Merge connection diagnostics
|
||||
mergeConnectionDiagnostics(clientEntity, entityMap, entity -> entity.getComponent().getIncomingConnections());
|
||||
mergeConnectionDiagnostics(clientEntity, entityMap, entity -> entity.getComponent().getOutgoingConnections());
|
||||
|
||||
|
||||
// Merge the Processor Statuses and create a separate NodeJVMDiagnosticsSnapshotDTO for each. We do both of these
|
||||
// together simply because we are already iterating over the entityMap and we have to create the Node-specific JVM diagnostics
|
||||
// before we start merging the values, in the second iteration over the map.
|
||||
|
@ -99,7 +109,7 @@ public class ProcessorDiagnosticsEntityMerger implements ComponentEntityMerger<P
|
|||
|
||||
// Merge permissions on referenced controller services
|
||||
final Map<String, ControllerServiceEntity> serviceEntityById = clientDto.getReferencedControllerServices().stream()
|
||||
.map(diagnosticsDto -> diagnosticsDto.getControllerService())
|
||||
.map(ControllerServiceDiagnosticsDTO::getControllerService)
|
||||
.collect(Collectors.toMap(ControllerServiceEntity::getId, Function.identity()));
|
||||
|
||||
for (final Map.Entry<NodeIdentifier, ProcessorDiagnosticsEntity> entry : entityMap.entrySet()) {
|
||||
|
@ -114,6 +124,129 @@ public class ProcessorDiagnosticsEntityMerger implements ComponentEntityMerger<P
|
|||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void mergeConnectionDiagnostics(final ProcessorDiagnosticsEntity clientEntity, final Map<NodeIdentifier, ProcessorDiagnosticsEntity> entityMap,
|
||||
final Function<ProcessorDiagnosticsEntity, Set<ConnectionDiagnosticsDTO>> extractConnections) {
|
||||
|
||||
final Map<String, List<ConnectionDiagnosticsSnapshotDTO>> snapshotByConnectionId = new HashMap<>();
|
||||
final Map<String, ConnectionDiagnosticsDTO> connectionById = new HashMap<>();
|
||||
|
||||
for (final Map.Entry<NodeIdentifier, ProcessorDiagnosticsEntity> entry : entityMap.entrySet()) {
|
||||
final NodeIdentifier nodeId = entry.getKey();
|
||||
final ProcessorDiagnosticsEntity entity = entry.getValue();
|
||||
|
||||
final Set<ConnectionDiagnosticsDTO> connections = extractConnections.apply(entity);
|
||||
for (final ConnectionDiagnosticsDTO connectionDiagnostics : connections) {
|
||||
final String connectionId = connectionDiagnostics.getConnection().getId();
|
||||
final ConnectionDiagnosticsSnapshotDTO snapshot = connectionDiagnostics.getAggregateSnapshot();
|
||||
|
||||
snapshot.setNodeIdentifier(nodeId.getApiAddress() + ":" + nodeId.getApiPort());
|
||||
|
||||
final List<ConnectionDiagnosticsSnapshotDTO> snapshots = snapshotByConnectionId.computeIfAbsent(connectionId, id -> new ArrayList<>());
|
||||
snapshots.add(snapshot);
|
||||
|
||||
if (entity == clientEntity){
|
||||
connectionById.put(connectionId, connectionDiagnostics);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
for (final Map.Entry<String, List<ConnectionDiagnosticsSnapshotDTO>> entry : snapshotByConnectionId.entrySet()) {
|
||||
final String connectionId = entry.getKey();
|
||||
final List<ConnectionDiagnosticsSnapshotDTO> snapshots = entry.getValue();
|
||||
|
||||
final ConnectionDiagnosticsDTO dto = connectionById.get(connectionId);
|
||||
dto.setNodeSnapshots(snapshots);
|
||||
|
||||
dto.setAggregateSnapshot(mergeConnectionSnapshots(snapshots));
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
||||
private ConnectionDiagnosticsSnapshotDTO mergeConnectionSnapshots(final List<ConnectionDiagnosticsSnapshotDTO> snapshots) {
|
||||
final ConnectionDiagnosticsSnapshotDTO aggregate = new ConnectionDiagnosticsSnapshotDTO();
|
||||
|
||||
final Map<String, List<RemoteQueuePartitionDTO>> remotePartitionsByNodeId = new HashMap<>();
|
||||
|
||||
final LocalQueuePartitionDTO localPartition = new LocalQueuePartitionDTO();
|
||||
localPartition.setActiveQueueByteCount(0);
|
||||
localPartition.setActiveQueueFlowFileCount(0);
|
||||
localPartition.setAllActiveQueueFlowFilesPenalized(true); // set to true because we will update this value by AND'ing it with the snapshot value
|
||||
localPartition.setAnyActiveQueueFlowFilesPenalized(false); // set to false because we will update this value by OR'ing it with the snapshot value
|
||||
localPartition.setInFlightByteCount(0);
|
||||
localPartition.setInFlightFlowFileCount(0);
|
||||
localPartition.setSwapByteCount(0);
|
||||
localPartition.setSwapFiles(0);
|
||||
localPartition.setSwapFlowFileCount(0);
|
||||
localPartition.setTotalByteCount(0);
|
||||
localPartition.setTotalFlowFileCount(0);
|
||||
|
||||
aggregate.setTotalByteCount(0L);
|
||||
aggregate.setTotalFlowFileCount(0);
|
||||
aggregate.setLocalQueuePartition(localPartition);
|
||||
|
||||
for (final ConnectionDiagnosticsSnapshotDTO snapshot : snapshots) {
|
||||
aggregate.setTotalByteCount(aggregate.getTotalByteCount() + snapshot.getTotalByteCount());
|
||||
aggregate.setTotalFlowFileCount(aggregate.getTotalFlowFileCount() + snapshot.getTotalFlowFileCount());
|
||||
|
||||
final LocalQueuePartitionDTO snapshotLocalPartition = snapshot.getLocalQueuePartition();
|
||||
localPartition.setActiveQueueByteCount(localPartition.getActiveQueueByteCount() + snapshotLocalPartition.getActiveQueueByteCount());
|
||||
localPartition.setActiveQueueFlowFileCount(localPartition.getActiveQueueFlowFileCount() + snapshotLocalPartition.getActiveQueueFlowFileCount());
|
||||
localPartition.setAllActiveQueueFlowFilesPenalized(localPartition.getAllActiveQueueFlowFilesPenalized() && snapshotLocalPartition.getAllActiveQueueFlowFilesPenalized());
|
||||
localPartition.setAnyActiveQueueFlowFilesPenalized(localPartition.getAnyActiveQueueFlowFilesPenalized() || snapshotLocalPartition.getAnyActiveQueueFlowFilesPenalized());
|
||||
localPartition.setInFlightByteCount(localPartition.getInFlightByteCount() + snapshotLocalPartition.getInFlightByteCount());
|
||||
localPartition.setInFlightFlowFileCount(localPartition.getInFlightFlowFileCount() + snapshotLocalPartition.getInFlightFlowFileCount());
|
||||
localPartition.setSwapByteCount(localPartition.getSwapByteCount() + snapshotLocalPartition.getSwapByteCount());
|
||||
localPartition.setSwapFiles(localPartition.getSwapFiles() + snapshotLocalPartition.getSwapFiles());
|
||||
localPartition.setSwapFlowFileCount(localPartition.getSwapFlowFileCount() + snapshotLocalPartition.getSwapFlowFileCount());
|
||||
localPartition.setTotalByteCount(localPartition.getTotalByteCount() + snapshotLocalPartition.getTotalByteCount());
|
||||
localPartition.setTotalFlowFileCount(localPartition.getTotalFlowFileCount() + snapshotLocalPartition.getTotalFlowFileCount());
|
||||
|
||||
for (final RemoteQueuePartitionDTO remoteQueuePartition : snapshot.getRemoteQueuePartitions()) {
|
||||
final String nodeId = remoteQueuePartition.getNodeIdentifier();
|
||||
final List<RemoteQueuePartitionDTO> partitionsForNodeId = remotePartitionsByNodeId.computeIfAbsent(nodeId, key -> new ArrayList<>());
|
||||
partitionsForNodeId.add(remoteQueuePartition);
|
||||
}
|
||||
}
|
||||
|
||||
final List<RemoteQueuePartitionDTO> mergedRemoteQueuePartitions = new ArrayList<>();
|
||||
for (final List<RemoteQueuePartitionDTO> partitions : remotePartitionsByNodeId.values()) {
|
||||
final RemoteQueuePartitionDTO merged = mergeRemoteQueuePartitions(partitions);
|
||||
mergedRemoteQueuePartitions.add(merged);
|
||||
}
|
||||
|
||||
aggregate.setRemoteQueuePartitions(mergedRemoteQueuePartitions);
|
||||
|
||||
return aggregate;
|
||||
}
|
||||
|
||||
private RemoteQueuePartitionDTO mergeRemoteQueuePartitions(final List<RemoteQueuePartitionDTO> partitions) {
|
||||
final RemoteQueuePartitionDTO merged = new RemoteQueuePartitionDTO();
|
||||
merged.setActiveQueueByteCount(0);
|
||||
merged.setActiveQueueFlowFileCount(0);
|
||||
merged.setInFlightByteCount(0);
|
||||
merged.setInFlightFlowFileCount(0);
|
||||
merged.setSwapByteCount(0);
|
||||
merged.setSwapFiles(0);
|
||||
merged.setSwapFlowFileCount(0);
|
||||
merged.setTotalByteCount(0);
|
||||
merged.setTotalFlowFileCount(0);
|
||||
|
||||
for (final RemoteQueuePartitionDTO partition : partitions) {
|
||||
merged.setActiveQueueByteCount(merged.getActiveQueueByteCount() + partition.getActiveQueueByteCount());
|
||||
merged.setActiveQueueFlowFileCount(merged.getActiveQueueFlowFileCount() + partition.getActiveQueueFlowFileCount());
|
||||
merged.setInFlightByteCount(merged.getInFlightByteCount() + partition.getInFlightByteCount());
|
||||
merged.setInFlightFlowFileCount(merged.getInFlightFlowFileCount() + partition.getInFlightFlowFileCount());
|
||||
merged.setSwapByteCount(merged.getSwapByteCount() + partition.getSwapByteCount());
|
||||
merged.setSwapFiles(merged.getSwapFiles() + partition.getSwapFiles());
|
||||
merged.setSwapFlowFileCount(merged.getSwapFlowFileCount() + partition.getSwapFlowFileCount());
|
||||
merged.setTotalByteCount(merged.getTotalByteCount() + partition.getTotalByteCount());
|
||||
merged.setTotalFlowFileCount(merged.getTotalFlowFileCount() + partition.getTotalFlowFileCount());
|
||||
merged.setNodeIdentifier(partition.getNodeIdentifier());
|
||||
}
|
||||
|
||||
return merged;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -218,7 +218,7 @@ public class TestPopularVoteFlowElection {
|
|||
}
|
||||
|
||||
private NodeIdentifier createNodeId(final int index) {
|
||||
return new NodeIdentifier(UUID.randomUUID().toString(), "localhost", 9000 + index, "localhost", 9000 + index, "localhost", 9000 + index, 9000 + index, true);
|
||||
return new NodeIdentifier(UUID.randomUUID().toString(), "localhost", 9000 + index, "localhost", 9000 + index, "localhost", 9000 + index, "localhost", 9000 + index, 9000 + index, true);
|
||||
}
|
||||
|
||||
private DataFlow createDataFlow(final byte[] flow) {
|
||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.nifi.cluster.coordination.heartbeat;
|
|||
|
||||
import org.apache.nifi.cluster.ReportedEvent;
|
||||
import org.apache.nifi.cluster.coordination.ClusterCoordinator;
|
||||
import org.apache.nifi.cluster.coordination.ClusterTopologyEventListener;
|
||||
import org.apache.nifi.cluster.coordination.node.DisconnectionCode;
|
||||
import org.apache.nifi.cluster.coordination.node.NodeConnectionState;
|
||||
import org.apache.nifi.cluster.coordination.node.NodeConnectionStatus;
|
||||
|
@ -55,7 +56,7 @@ public class TestAbstractHeartbeatMonitor {
|
|||
@Before
|
||||
public void setup() throws Exception {
|
||||
System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, "src/test/resources/conf/nifi.properties");
|
||||
nodeId = new NodeIdentifier(UUID.randomUUID().toString(), "localhost", 9999, "localhost", 8888, "localhost", null, null, false);
|
||||
nodeId = new NodeIdentifier(UUID.randomUUID().toString(), "localhost", 9999, "localhost", 8888, "localhost", 777, "localhost", null, null, false);
|
||||
}
|
||||
|
||||
@After
|
||||
|
@ -136,7 +137,7 @@ public class TestAbstractHeartbeatMonitor {
|
|||
@Test
|
||||
public void testDisconnectionOfTerminatedNodeDueToLackOfHeartbeat() throws Exception {
|
||||
final NodeIdentifier nodeId1 = nodeId;
|
||||
final NodeIdentifier nodeId2 = new NodeIdentifier(UUID.randomUUID().toString(), "localhost", 7777, "localhost", 6666, "localhost", null, null, false);
|
||||
final NodeIdentifier nodeId2 = new NodeIdentifier(UUID.randomUUID().toString(), "localhost", 7777, "localhost", 6666, "localhost", 5555, "localhost", null, null, false);
|
||||
|
||||
final ClusterCoordinatorAdapter adapter = new ClusterCoordinatorAdapter();
|
||||
final TestFriendlyHeartbeatMonitor monitor = createMonitor(adapter);
|
||||
|
@ -272,7 +273,7 @@ public class TestAbstractHeartbeatMonitor {
|
|||
}
|
||||
|
||||
@Override
|
||||
public synchronized boolean isBlockedByFirewall(String hostname) {
|
||||
public synchronized boolean isBlockedByFirewall(Set<String> nodeIds) {
|
||||
return false;
|
||||
}
|
||||
|
||||
|
@ -369,6 +370,14 @@ public class TestAbstractHeartbeatMonitor {
|
|||
public Map<NodeIdentifier, NodeWorkload> getClusterWorkload() throws IOException {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void registerEventListener(final ClusterTopologyEventListener eventListener) {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void unregisterEventListener(final ClusterTopologyEventListener eventListener) {
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -64,7 +64,8 @@ class StandardHttpResponseMapperSpec extends Specification {
|
|||
int n = it.node
|
||||
def response = Mock(Response)
|
||||
mockToRequestEntity.put response, it
|
||||
new NodeResponse(new NodeIdentifier("cluster-node-$n", 'addr', n, 'sktaddr', n * 10, 'stsaddr', n * 100, n * 1000, false, null), "get", requestUri, response, 500L, requestId)
|
||||
new NodeResponse(new NodeIdentifier("cluster-node-$n", 'addr', n, 'sktaddr', n * 10, 'sktaddr', n * 10, 'stsaddr', n * 100, n * 1000, false, null), "get", requestUri, response, 500L,
|
||||
requestId)
|
||||
} as Set
|
||||
|
||||
when:
|
||||
|
@ -102,7 +103,8 @@ class StandardHttpResponseMapperSpec extends Specification {
|
|||
++n
|
||||
def response = Mock(Response)
|
||||
mockToRequestEntity.put response, it
|
||||
new NodeResponse(new NodeIdentifier("cluster-node-$n", 'addr', n, 'sktaddr', n * 10, 'stsaddr', n * 100, n * 1000, false, null), "get", requestUri, response, 500L, requestId)
|
||||
new NodeResponse(new NodeIdentifier("cluster-node-$n", 'addr', n, 'sktaddr', n * 10, 'sktaddr', n * 11, 'stsaddr', n * 100, n * 1000, false, null), "get", requestUri, response, 500L,
|
||||
requestId)
|
||||
} as Set
|
||||
|
||||
when:
|
||||
|
|
|
@ -39,7 +39,7 @@ public class CurrentUserEndpointMergerTest {
|
|||
|
||||
@Test
|
||||
public void testMergeUserPermissions() {
|
||||
final NodeIdentifier nodeId1 = new NodeIdentifier("1", "localhost", 9000, "localhost", 9001, "localhost", 9002, 9003, false);
|
||||
final NodeIdentifier nodeId1 = new NodeIdentifier("1", "localhost", 9000, "localhost", 9001, "localhost", 9006, "localhost", 9002, 9003, false);
|
||||
final CurrentUserEntity userNode1 = new CurrentUserEntity();
|
||||
userNode1.setControllerPermissions(buildPermissions(true, false));
|
||||
userNode1.setCountersPermissions(buildPermissions(true, true));
|
||||
|
@ -55,7 +55,7 @@ public class CurrentUserEndpointMergerTest {
|
|||
componentRestrictionsNode1.add(buildComponentRestriction(RequiredPermission.READ_FILESYSTEM, true, true));
|
||||
userNode1.setComponentRestrictionPermissions(componentRestrictionsNode1);
|
||||
|
||||
final NodeIdentifier nodeId2 = new NodeIdentifier("2", "localhost", 8000, "localhost", 8001, "localhost", 8002, 8003, false);
|
||||
final NodeIdentifier nodeId2 = new NodeIdentifier("2", "localhost", 8000, "localhost", 8001, "localhost", 9006,"localhost", 8002, 8003, false);
|
||||
final CurrentUserEntity userNode2 = new CurrentUserEntity();
|
||||
userNode2.setControllerPermissions(buildPermissions(false, true));
|
||||
userNode2.setCountersPermissions(buildPermissions(true, false));
|
||||
|
|
|
@ -57,7 +57,8 @@ class StatusHistoryEndpointMergerSpec extends Specification {
|
|||
++n
|
||||
def response = Mock(Response)
|
||||
mockToRequestEntity.put response, it
|
||||
new NodeResponse(new NodeIdentifier("cluster-node-$n", 'addr', n, 'sktaddr', n * 10, 'stsaddr', n * 100, n * 1000, false, null), "get", requestUri, response, 500L, requestId)
|
||||
new NodeResponse(new NodeIdentifier("cluster-node-$n", 'addr', n, 'sktaddr', n * 10, null, n * 10, 'stsaddr', n * 100, n * 1000, false, null),
|
||||
"GET", requestUri, response, 500L, requestId)
|
||||
} as Set
|
||||
|
||||
when:
|
||||
|
|
|
@ -16,24 +16,6 @@
|
|||
*/
|
||||
package org.apache.nifi.cluster.coordination.node;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.assertNotSame;
|
||||
import static org.junit.Assert.assertNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import org.apache.nifi.cluster.coordination.flow.FlowElection;
|
||||
import org.apache.nifi.cluster.manager.exception.IllegalNodeDisconnectionException;
|
||||
import org.apache.nifi.cluster.protocol.ConnectionRequest;
|
||||
|
@ -47,8 +29,12 @@ import org.apache.nifi.cluster.protocol.message.ConnectionResponseMessage;
|
|||
import org.apache.nifi.cluster.protocol.message.NodeStatusChangeMessage;
|
||||
import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
|
||||
import org.apache.nifi.cluster.protocol.message.ReconnectionRequestMessage;
|
||||
import org.apache.nifi.components.state.Scope;
|
||||
import org.apache.nifi.components.state.StateManager;
|
||||
import org.apache.nifi.components.state.StateManagerProvider;
|
||||
import org.apache.nifi.events.EventReporter;
|
||||
import org.apache.nifi.services.FlowService;
|
||||
import org.apache.nifi.state.MockStateMap;
|
||||
import org.apache.nifi.util.NiFiProperties;
|
||||
import org.apache.nifi.web.revision.RevisionManager;
|
||||
import org.junit.Assert;
|
||||
|
@ -58,11 +44,33 @@ import org.mockito.Mockito;
|
|||
import org.mockito.invocation.InvocationOnMock;
|
||||
import org.mockito.stubbing.Answer;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.assertNotSame;
|
||||
import static org.junit.Assert.assertNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.mockito.Matchers.any;
|
||||
import static org.mockito.Matchers.anyString;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
public class TestNodeClusterCoordinator {
|
||||
|
||||
private NodeClusterCoordinator coordinator;
|
||||
private ClusterCoordinationProtocolSenderListener senderListener;
|
||||
private List<NodeConnectionStatus> nodeStatuses;
|
||||
private StateManagerProvider stateManagerProvider;
|
||||
|
||||
private NiFiProperties createProperties() {
|
||||
final Map<String,String> addProps = new HashMap<>();
|
||||
|
@ -76,12 +84,18 @@ public class TestNodeClusterCoordinator {
|
|||
|
||||
senderListener = Mockito.mock(ClusterCoordinationProtocolSenderListener.class);
|
||||
nodeStatuses = Collections.synchronizedList(new ArrayList<>());
|
||||
stateManagerProvider = Mockito.mock(StateManagerProvider.class);
|
||||
|
||||
final StateManager stateManager = Mockito.mock(StateManager.class);
|
||||
when(stateManager.getState(any(Scope.class))).thenReturn(new MockStateMap(Collections.emptyMap(), 1));
|
||||
when(stateManagerProvider.getStateManager(anyString())).thenReturn(stateManager);
|
||||
|
||||
|
||||
final EventReporter eventReporter = Mockito.mock(EventReporter.class);
|
||||
final RevisionManager revisionManager = Mockito.mock(RevisionManager.class);
|
||||
Mockito.when(revisionManager.getAllRevisions()).thenReturn(Collections.emptyList());
|
||||
when(revisionManager.getAllRevisions()).thenReturn(Collections.emptyList());
|
||||
|
||||
coordinator = new NodeClusterCoordinator(senderListener, eventReporter, null, new FirstVoteWinsFlowElection(), null, revisionManager, createProperties(), null) {
|
||||
coordinator = new NodeClusterCoordinator(senderListener, eventReporter, null, new FirstVoteWinsFlowElection(), null, revisionManager, createProperties(), null, stateManagerProvider) {
|
||||
@Override
|
||||
void notifyOthersOfNodeStatusChange(NodeConnectionStatus updatedStatus, boolean notifyAllNodes, boolean waitForCoordinator) {
|
||||
nodeStatuses.add(updatedStatus);
|
||||
|
@ -90,7 +104,7 @@ public class TestNodeClusterCoordinator {
|
|||
|
||||
final FlowService flowService = Mockito.mock(FlowService.class);
|
||||
final StandardDataFlow dataFlow = new StandardDataFlow(new byte[50], new byte[50], new byte[50], new HashSet<>());
|
||||
Mockito.when(flowService.createDataFlow()).thenReturn(dataFlow);
|
||||
when(flowService.createDataFlow()).thenReturn(dataFlow);
|
||||
coordinator.setFlowService(flowService);
|
||||
}
|
||||
|
||||
|
@ -130,14 +144,14 @@ public class TestNodeClusterCoordinator {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testTryAgainIfNoFlowServiceSet() {
|
||||
public void testTryAgainIfNoFlowServiceSet() throws IOException {
|
||||
final ClusterCoordinationProtocolSenderListener senderListener = Mockito.mock(ClusterCoordinationProtocolSenderListener.class);
|
||||
final EventReporter eventReporter = Mockito.mock(EventReporter.class);
|
||||
final RevisionManager revisionManager = Mockito.mock(RevisionManager.class);
|
||||
Mockito.when(revisionManager.getAllRevisions()).thenReturn(Collections.emptyList());
|
||||
when(revisionManager.getAllRevisions()).thenReturn(Collections.emptyList());
|
||||
|
||||
final NodeClusterCoordinator coordinator = new NodeClusterCoordinator(senderListener, eventReporter, null, new FirstVoteWinsFlowElection(),
|
||||
null, revisionManager, createProperties(), null) {
|
||||
null, revisionManager, createProperties(), null, stateManagerProvider) {
|
||||
@Override
|
||||
void notifyOthersOfNodeStatusChange(NodeConnectionStatus updatedStatus, boolean notifyAllNodes, boolean waitForCoordinator) {
|
||||
}
|
||||
|
@ -150,7 +164,7 @@ public class TestNodeClusterCoordinator {
|
|||
|
||||
coordinator.setConnected(true);
|
||||
|
||||
final ProtocolMessage protocolResponse = coordinator.handle(requestMsg);
|
||||
final ProtocolMessage protocolResponse = coordinator.handle(requestMsg, Collections.emptySet());
|
||||
assertNotNull(protocolResponse);
|
||||
assertTrue(protocolResponse instanceof ConnectionResponseMessage);
|
||||
|
||||
|
@ -164,7 +178,7 @@ public class TestNodeClusterCoordinator {
|
|||
final ClusterCoordinationProtocolSenderListener senderListener = Mockito.mock(ClusterCoordinationProtocolSenderListener.class);
|
||||
final AtomicReference<ReconnectionRequestMessage> requestRef = new AtomicReference<>();
|
||||
|
||||
Mockito.when(senderListener.requestReconnection(Mockito.any(ReconnectionRequestMessage.class))).thenAnswer(new Answer<Object>() {
|
||||
when(senderListener.requestReconnection(any(ReconnectionRequestMessage.class))).thenAnswer(new Answer<Object>() {
|
||||
@Override
|
||||
public Object answer(InvocationOnMock invocation) throws Throwable {
|
||||
final ReconnectionRequestMessage msg = invocation.getArgumentAt(0, ReconnectionRequestMessage.class);
|
||||
|
@ -175,10 +189,10 @@ public class TestNodeClusterCoordinator {
|
|||
|
||||
final EventReporter eventReporter = Mockito.mock(EventReporter.class);
|
||||
final RevisionManager revisionManager = Mockito.mock(RevisionManager.class);
|
||||
Mockito.when(revisionManager.getAllRevisions()).thenReturn(Collections.emptyList());
|
||||
when(revisionManager.getAllRevisions()).thenReturn(Collections.emptyList());
|
||||
|
||||
final NodeClusterCoordinator coordinator = new NodeClusterCoordinator(senderListener, eventReporter, null, new FirstVoteWinsFlowElection(),
|
||||
null, revisionManager, createProperties(), null) {
|
||||
null, revisionManager, createProperties(), null, stateManagerProvider) {
|
||||
@Override
|
||||
void notifyOthersOfNodeStatusChange(NodeConnectionStatus updatedStatus, boolean notifyAllNodes, boolean waitForCoordinator) {
|
||||
}
|
||||
|
@ -186,7 +200,7 @@ public class TestNodeClusterCoordinator {
|
|||
|
||||
final FlowService flowService = Mockito.mock(FlowService.class);
|
||||
final StandardDataFlow dataFlow = new StandardDataFlow(new byte[50], new byte[50], new byte[50], new HashSet<>());
|
||||
Mockito.when(flowService.createDataFlowFromController()).thenReturn(dataFlow);
|
||||
when(flowService.createDataFlowFromController()).thenReturn(dataFlow);
|
||||
coordinator.setFlowService(flowService);
|
||||
coordinator.setConnected(true);
|
||||
|
||||
|
@ -232,7 +246,7 @@ public class TestNodeClusterCoordinator {
|
|||
@Test(timeout = 5000)
|
||||
public void testStatusChangesReplicated() throws InterruptedException, IOException {
|
||||
final RevisionManager revisionManager = Mockito.mock(RevisionManager.class);
|
||||
Mockito.when(revisionManager.getAllRevisions()).thenReturn(Collections.emptyList());
|
||||
when(revisionManager.getAllRevisions()).thenReturn(Collections.emptyList());
|
||||
|
||||
// Create a connection request message and send to the coordinator
|
||||
final NodeIdentifier requestedNodeId = createNodeId(1);
|
||||
|
@ -397,7 +411,7 @@ public class TestNodeClusterCoordinator {
|
|||
final NodeStatusChangeMessage msg = new NodeStatusChangeMessage();
|
||||
msg.setNodeId(nodeId1);
|
||||
msg.setNodeConnectionStatus(oldStatus);
|
||||
coordinator.handle(msg);
|
||||
coordinator.handle(msg, Collections.emptySet());
|
||||
|
||||
// Ensure that no status change message was send
|
||||
Thread.sleep(1000);
|
||||
|
@ -413,7 +427,7 @@ public class TestNodeClusterCoordinator {
|
|||
final ConnectionRequestMessage crm = new ConnectionRequestMessage();
|
||||
crm.setConnectionRequest(connectionRequest);
|
||||
|
||||
final ProtocolMessage response = coordinator.handle(crm);
|
||||
final ProtocolMessage response = coordinator.handle(crm, Collections.emptySet());
|
||||
assertNotNull(response);
|
||||
assertTrue(response instanceof ConnectionResponseMessage);
|
||||
final ConnectionResponseMessage responseMessage = (ConnectionResponseMessage) response;
|
||||
|
@ -424,7 +438,7 @@ public class TestNodeClusterCoordinator {
|
|||
final ConnectionRequestMessage crm2 = new ConnectionRequestMessage();
|
||||
crm2.setConnectionRequest(conRequest2);
|
||||
|
||||
final ProtocolMessage conflictingResponse = coordinator.handle(crm2);
|
||||
final ProtocolMessage conflictingResponse = coordinator.handle(crm2, Collections.emptySet());
|
||||
assertNotNull(conflictingResponse);
|
||||
assertTrue(conflictingResponse instanceof ConnectionResponseMessage);
|
||||
final ConnectionResponseMessage conflictingResponseMessage = (ConnectionResponseMessage) conflictingResponse;
|
||||
|
@ -446,7 +460,7 @@ public class TestNodeClusterCoordinator {
|
|||
final ConnectionRequest request = new ConnectionRequest(requestedNodeId, new StandardDataFlow(new byte[0], new byte[0], new byte[0], new HashSet<>()));
|
||||
final ConnectionRequestMessage requestMsg = new ConnectionRequestMessage();
|
||||
requestMsg.setConnectionRequest(request);
|
||||
return coordinator.handle(requestMsg);
|
||||
return coordinator.handle(requestMsg, Collections.emptySet());
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -219,7 +219,7 @@ public class ClusterConnectionIT {
|
|||
cluster.waitUntilAllNodesConnected(10, TimeUnit.SECONDS);
|
||||
final Node coordinator = cluster.waitForClusterCoordinator(10, TimeUnit.SECONDS);
|
||||
|
||||
final NodeIdentifier node4NotReallyInCluster = new NodeIdentifier(UUID.randomUUID().toString(), "localhost", 9283, "localhost", 9284, "localhost", 9285, null, false, null);
|
||||
final NodeIdentifier node4NotReallyInCluster = new NodeIdentifier(UUID.randomUUID().toString(), "localhost", 9283, "localhost", 9284, "localhost", 9286, "localhost", 9285, null, false, null);
|
||||
|
||||
final Map<NodeIdentifier, NodeConnectionStatus> replacementStatuses = new HashMap<>();
|
||||
replacementStatuses.put(node1.getIdentifier(), new NodeConnectionStatus(node1.getIdentifier(), DisconnectionCode.USER_DISCONNECTED));
|
||||
|
|
|
@ -17,17 +17,6 @@
|
|||
|
||||
package org.apache.nifi.cluster.integration;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.net.ServerSocket;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.ScheduledExecutorService;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import org.apache.commons.lang3.builder.HashCodeBuilder;
|
||||
import org.apache.nifi.authorization.Authorizer;
|
||||
import org.apache.nifi.bundle.Bundle;
|
||||
|
@ -73,6 +62,18 @@ import org.apache.nifi.web.revision.RevisionManager;
|
|||
import org.junit.Assert;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.net.ServerSocket;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.ScheduledExecutorService;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
public class Node {
|
||||
private final NodeIdentifier nodeId;
|
||||
private final NiFiProperties nodeProperties;
|
||||
|
@ -133,7 +134,7 @@ public class Node {
|
|||
|
||||
|
||||
private static NodeIdentifier createNodeId() {
|
||||
return new NodeIdentifier(UUID.randomUUID().toString(), "localhost", createPort(), "localhost", createPort(), "localhost", null, null, false, null);
|
||||
return new NodeIdentifier(UUID.randomUUID().toString(), "localhost", createPort(), "localhost", createPort(), "localhost", createPort(), "localhost", null, null, false, null);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -296,8 +297,13 @@ public class Node {
|
|||
}
|
||||
|
||||
final ClusterCoordinationProtocolSenderListener protocolSenderListener = new ClusterCoordinationProtocolSenderListener(createCoordinatorProtocolSender(), protocolListener);
|
||||
return new NodeClusterCoordinator(protocolSenderListener, eventReporter, electionManager, flowElection, null,
|
||||
revisionManager, nodeProperties, protocolSender);
|
||||
try {
|
||||
return new NodeClusterCoordinator(protocolSenderListener, eventReporter, electionManager, flowElection, null,
|
||||
revisionManager, nodeProperties, protocolSender);
|
||||
} catch (IOException e) {
|
||||
Assert.fail(e.toString());
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -63,6 +63,6 @@ class ConnectionEntityMergerSpec extends Specification {
|
|||
}
|
||||
|
||||
def createNodeIdentifier(int id) {
|
||||
new NodeIdentifier("cluster-node-$id", 'addr', id, 'sktaddr', id * 10, 'stsaddr', id * 100, id * 1000, false, null)
|
||||
new NodeIdentifier("cluster-node-$id", 'addr', id, 'sktaddr', id * 10, null, id * 10, 'stsaddr', id * 100, id * 1000, false, null)
|
||||
}
|
||||
}
|
||||
|
|
|
@ -147,6 +147,6 @@ class ControllerServiceEntityMergerSpec extends Specification {
|
|||
}
|
||||
|
||||
def createNodeIdentifier(int id) {
|
||||
new NodeIdentifier("cluster-node-$id", 'addr', id, 'sktaddr', id * 10, 'stsaddr', id * 100, id * 1000, false, null)
|
||||
new NodeIdentifier("cluster-node-$id", 'addr', id, 'sktaddr', id * 10, null, id * 10, 'stsaddr', id * 100, id * 1000, false, null)
|
||||
}
|
||||
}
|
||||
|
|
|
@ -55,6 +55,6 @@ class LabelEntityMergerSpec extends Specification {
|
|||
}
|
||||
|
||||
def createNodeIdentifier(int id) {
|
||||
new NodeIdentifier("cluster-node-$id", 'addr', id, 'sktaddr', id * 10, 'stsaddr', id * 100, id * 1000, false, null)
|
||||
new NodeIdentifier("cluster-node-$id", 'addr', id, 'sktaddr', id * 10, null, id * 10, 'stsaddr', id * 100, id * 1000, false, null)
|
||||
}
|
||||
}
|
||||
|
|
|
@ -15,11 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.nifi.controller;
|
||||
|
||||
import org.apache.nifi.controller.queue.DropFlowFileState;
|
||||
import org.apache.nifi.controller.queue.DropFlowFileStatus;
|
||||
import org.apache.nifi.controller.queue.QueueSize;
|
||||
package org.apache.nifi.controller.queue;
|
||||
|
||||
public class DropFlowFileRequest implements DropFlowFileStatus {
|
||||
private final String identifier;
|
||||
|
@ -53,7 +49,7 @@ public class DropFlowFileRequest implements DropFlowFileStatus {
|
|||
return originalSize;
|
||||
}
|
||||
|
||||
void setOriginalSize(final QueueSize originalSize) {
|
||||
public void setOriginalSize(final QueueSize originalSize) {
|
||||
this.originalSize = originalSize;
|
||||
}
|
||||
|
||||
|
@ -62,7 +58,7 @@ public class DropFlowFileRequest implements DropFlowFileStatus {
|
|||
return currentSize;
|
||||
}
|
||||
|
||||
void setCurrentSize(final QueueSize queueSize) {
|
||||
public void setCurrentSize(final QueueSize queueSize) {
|
||||
this.currentSize = queueSize;
|
||||
}
|
||||
|
||||
|
@ -71,7 +67,7 @@ public class DropFlowFileRequest implements DropFlowFileStatus {
|
|||
return droppedSize;
|
||||
}
|
||||
|
||||
void setDroppedSize(final QueueSize droppedSize) {
|
||||
public void setDroppedSize(final QueueSize droppedSize) {
|
||||
this.droppedSize = droppedSize;
|
||||
}
|
||||
|
||||
|
@ -90,17 +86,17 @@ public class DropFlowFileRequest implements DropFlowFileStatus {
|
|||
return failureReason;
|
||||
}
|
||||
|
||||
synchronized void setState(final DropFlowFileState state) {
|
||||
public synchronized void setState(final DropFlowFileState state) {
|
||||
setState(state, null);
|
||||
}
|
||||
|
||||
synchronized void setState(final DropFlowFileState state, final String explanation) {
|
||||
public synchronized void setState(final DropFlowFileState state, final String explanation) {
|
||||
this.state = state;
|
||||
this.failureReason = explanation;
|
||||
this.lastUpdated = System.currentTimeMillis();
|
||||
}
|
||||
|
||||
synchronized boolean cancel() {
|
||||
public synchronized boolean cancel() {
|
||||
if (this.state == DropFlowFileState.COMPLETE || this.state == DropFlowFileState.CANCELED) {
|
||||
return false;
|
||||
}
|
|
@ -18,6 +18,8 @@ package org.apache.nifi.controller.repository;
|
|||
|
||||
import org.apache.nifi.controller.repository.claim.ContentClaim;
|
||||
|
||||
import java.util.Optional;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
|
@ -25,23 +27,37 @@ public class ContentNotFoundException extends RuntimeException {
|
|||
|
||||
private static final long serialVersionUID = 19048239082L;
|
||||
private final transient ContentClaim claim;
|
||||
private final transient FlowFileRecord flowFile;
|
||||
|
||||
public ContentNotFoundException(final ContentClaim claim) {
|
||||
super("Could not find content for " + claim);
|
||||
this.claim = claim;
|
||||
this.flowFile = null;
|
||||
}
|
||||
|
||||
public ContentNotFoundException(final ContentClaim claim, final Throwable t) {
|
||||
super("Could not find content for " + claim, t);
|
||||
this.claim = claim;
|
||||
this.flowFile = null;
|
||||
}
|
||||
|
||||
public ContentNotFoundException(final ContentClaim claim, final String message) {
|
||||
super("Could not find content for " + claim + ": " + message);
|
||||
this.claim = claim;
|
||||
this.flowFile = null;
|
||||
}
|
||||
|
||||
public ContentNotFoundException(final FlowFileRecord flowFile, final ContentClaim claim, final String message) {
|
||||
super("Could not find content for " + claim + ": " + message);
|
||||
this.claim = claim;
|
||||
this.flowFile = flowFile;
|
||||
}
|
||||
|
||||
public ContentClaim getMissingClaim() {
|
||||
return claim;
|
||||
}
|
||||
|
||||
public Optional<FlowFileRecord> getFlowFile() {
|
||||
return Optional.ofNullable(flowFile);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -29,18 +29,16 @@ import org.apache.nifi.authorization.Resource;
|
|||
import org.apache.nifi.authorization.resource.Authorizable;
|
||||
import org.apache.nifi.authorization.user.NiFiUser;
|
||||
import org.apache.nifi.controller.ProcessScheduler;
|
||||
import org.apache.nifi.controller.StandardFlowFileQueue;
|
||||
import org.apache.nifi.controller.queue.ConnectionEventListener;
|
||||
import org.apache.nifi.controller.queue.FlowFileQueue;
|
||||
import org.apache.nifi.controller.queue.FlowFileQueueFactory;
|
||||
import org.apache.nifi.controller.queue.LoadBalanceStrategy;
|
||||
import org.apache.nifi.controller.repository.FlowFileRecord;
|
||||
import org.apache.nifi.controller.repository.FlowFileRepository;
|
||||
import org.apache.nifi.controller.repository.FlowFileSwapManager;
|
||||
import org.apache.nifi.controller.repository.claim.ResourceClaimManager;
|
||||
import org.apache.nifi.events.EventReporter;
|
||||
import org.apache.nifi.groups.ProcessGroup;
|
||||
import org.apache.nifi.processor.FlowFileFilter;
|
||||
import org.apache.nifi.processor.Relationship;
|
||||
import org.apache.nifi.provenance.ProvenanceEventRepository;
|
||||
import org.apache.nifi.remote.RemoteGroupPort;
|
||||
import org.apache.nifi.scheduling.SchedulingStrategy;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
|
@ -60,7 +58,7 @@ import java.util.stream.Collectors;
|
|||
* one or more relationships that map the source component to the destination
|
||||
* component.
|
||||
*/
|
||||
public final class StandardConnection implements Connection {
|
||||
public final class StandardConnection implements Connection, ConnectionEventListener {
|
||||
|
||||
private final String id;
|
||||
private final AtomicReference<ProcessGroup> processGroup;
|
||||
|
@ -69,13 +67,16 @@ public final class StandardConnection implements Connection {
|
|||
private final Connectable source;
|
||||
private final AtomicReference<Connectable> destination;
|
||||
private final AtomicReference<Collection<Relationship>> relationships;
|
||||
private final StandardFlowFileQueue flowFileQueue;
|
||||
private final AtomicInteger labelIndex = new AtomicInteger(1);
|
||||
private final AtomicLong zIndex = new AtomicLong(0L);
|
||||
private final AtomicReference<String> versionedComponentId = new AtomicReference<>();
|
||||
private final ProcessScheduler scheduler;
|
||||
private final FlowFileQueueFactory flowFileQueueFactory;
|
||||
private final boolean clustered;
|
||||
private final int hashCode;
|
||||
|
||||
private volatile FlowFileQueue flowFileQueue;
|
||||
|
||||
private StandardConnection(final Builder builder) {
|
||||
id = builder.id;
|
||||
name = new AtomicReference<>(builder.name);
|
||||
|
@ -85,9 +86,10 @@ public final class StandardConnection implements Connection {
|
|||
destination = new AtomicReference<>(builder.destination);
|
||||
relationships = new AtomicReference<>(Collections.unmodifiableCollection(builder.relationships));
|
||||
scheduler = builder.scheduler;
|
||||
flowFileQueue = new StandardFlowFileQueue(id, this, builder.flowFileRepository, builder.provenanceRepository, builder.resourceClaimManager,
|
||||
scheduler, builder.swapManager, builder.eventReporter, builder.queueSwapThreshold,
|
||||
builder.defaultBackPressureObjectThreshold, builder.defaultBackPressureDataSizeThreshold);
|
||||
flowFileQueueFactory = builder.flowFileQueueFactory;
|
||||
clustered = builder.clustered;
|
||||
|
||||
flowFileQueue = flowFileQueueFactory.createFlowFileQueue(LoadBalanceStrategy.DO_NOT_LOAD_BALANCE, null, this);
|
||||
hashCode = new HashCodeBuilder(7, 67).append(id).toHashCode();
|
||||
}
|
||||
|
||||
|
@ -147,6 +149,20 @@ public final class StandardConnection implements Connection {
|
|||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public void triggerDestinationEvent() {
|
||||
if (getDestination().getSchedulingStrategy() == SchedulingStrategy.EVENT_DRIVEN) {
|
||||
scheduler.registerEvent(getDestination());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void triggerSourceEvent() {
|
||||
if (getSource().getSchedulingStrategy() == SchedulingStrategy.EVENT_DRIVEN) {
|
||||
scheduler.registerEvent(getSource());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Authorizable getSourceAuthorizable() {
|
||||
final Connectable sourceConnectable = getSource();
|
||||
|
@ -297,7 +313,7 @@ public final class StandardConnection implements Connection {
|
|||
throw new IllegalStateException("Cannot change destination of Connection because the current destination is running");
|
||||
}
|
||||
|
||||
if (getFlowFileQueue().getUnacknowledgedQueueSize().getObjectCount() > 0) {
|
||||
if (getFlowFileQueue().isUnacknowledgedFlowFile()) {
|
||||
throw new IllegalStateException("Cannot change destination of Connection because FlowFiles from this Connection are currently held by " + previousDestination);
|
||||
}
|
||||
|
||||
|
@ -354,7 +370,7 @@ public final class StandardConnection implements Connection {
|
|||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "Connection[Source ID=" + id + ",Dest ID=" + getDestination().getIdentifier() + "]";
|
||||
return "Connection[ID=" + getIdentifier() + ", Source ID=" + getSource().getIdentifier() + ", Dest ID=" + getDestination().getIdentifier() + "]";
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -386,14 +402,8 @@ public final class StandardConnection implements Connection {
|
|||
private Connectable source;
|
||||
private Connectable destination;
|
||||
private Collection<Relationship> relationships;
|
||||
private FlowFileSwapManager swapManager;
|
||||
private EventReporter eventReporter;
|
||||
private FlowFileRepository flowFileRepository;
|
||||
private ProvenanceEventRepository provenanceRepository;
|
||||
private ResourceClaimManager resourceClaimManager;
|
||||
private int queueSwapThreshold;
|
||||
private Long defaultBackPressureObjectThreshold;
|
||||
private String defaultBackPressureDataSizeThreshold;
|
||||
private FlowFileQueueFactory flowFileQueueFactory;
|
||||
private boolean clustered = false;
|
||||
|
||||
public Builder(final ProcessScheduler scheduler) {
|
||||
this.scheduler = scheduler;
|
||||
|
@ -440,43 +450,13 @@ public final class StandardConnection implements Connection {
|
|||
return this;
|
||||
}
|
||||
|
||||
public Builder swapManager(final FlowFileSwapManager swapManager) {
|
||||
this.swapManager = swapManager;
|
||||
public Builder flowFileQueueFactory(final FlowFileQueueFactory flowFileQueueFactory) {
|
||||
this.flowFileQueueFactory = flowFileQueueFactory;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder eventReporter(final EventReporter eventReporter) {
|
||||
this.eventReporter = eventReporter;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder flowFileRepository(final FlowFileRepository flowFileRepository) {
|
||||
this.flowFileRepository = flowFileRepository;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder provenanceRepository(final ProvenanceEventRepository provenanceRepository) {
|
||||
this.provenanceRepository = provenanceRepository;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder resourceClaimManager(final ResourceClaimManager resourceClaimManager) {
|
||||
this.resourceClaimManager = resourceClaimManager;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder queueSwapThreshold(final int queueSwapThreshold) {
|
||||
this.queueSwapThreshold = queueSwapThreshold;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder defaultBackPressureObjectThreshold(final long defaultBackPressureObjectThreshold) {
|
||||
this.defaultBackPressureObjectThreshold = defaultBackPressureObjectThreshold;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder defaultBackPressureDataSizeThreshold(final String defaultBackPressureDataSizeThreshold) {
|
||||
this.defaultBackPressureDataSizeThreshold = defaultBackPressureDataSizeThreshold;
|
||||
public Builder clustered(final boolean clustered) {
|
||||
this.clustered = clustered;
|
||||
return this;
|
||||
}
|
||||
|
||||
|
@ -487,17 +467,8 @@ public final class StandardConnection implements Connection {
|
|||
if (destination == null) {
|
||||
throw new IllegalStateException("Cannot build a Connection without a Destination");
|
||||
}
|
||||
if (swapManager == null) {
|
||||
throw new IllegalStateException("Cannot build a Connection without a FlowFileSwapManager");
|
||||
}
|
||||
if (flowFileRepository == null) {
|
||||
throw new IllegalStateException("Cannot build a Connection without a FlowFile Repository");
|
||||
}
|
||||
if (provenanceRepository == null) {
|
||||
throw new IllegalStateException("Cannot build a Connection without a Provenance Repository");
|
||||
}
|
||||
if (resourceClaimManager == null) {
|
||||
throw new IllegalStateException("Cannot build a Connection without a Resource Claim Manager");
|
||||
if (flowFileQueueFactory == null) {
|
||||
throw new IllegalStateException("Cannot build a Connection without a FlowFileQueueFactory");
|
||||
}
|
||||
|
||||
if (relationships == null) {
|
||||
|
|
|
@ -16,28 +16,6 @@
|
|||
*/
|
||||
package org.apache.nifi.controller;
|
||||
|
||||
import java.io.BufferedInputStream;
|
||||
import java.io.BufferedOutputStream;
|
||||
import java.io.DataInputStream;
|
||||
import java.io.DataOutputStream;
|
||||
import java.io.EOFException;
|
||||
import java.io.File;
|
||||
import java.io.FileInputStream;
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.FileOutputStream;
|
||||
import java.io.FilenameFilter;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
import java.nio.file.Path;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
import java.util.UUID;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import org.apache.nifi.controller.queue.FlowFileQueue;
|
||||
import org.apache.nifi.controller.repository.FlowFileRecord;
|
||||
import org.apache.nifi.controller.repository.FlowFileRepository;
|
||||
|
@ -58,6 +36,33 @@ import org.apache.nifi.util.NiFiProperties;
|
|||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.BufferedInputStream;
|
||||
import java.io.BufferedOutputStream;
|
||||
import java.io.DataInputStream;
|
||||
import java.io.DataOutputStream;
|
||||
import java.io.EOFException;
|
||||
import java.io.File;
|
||||
import java.io.FileInputStream;
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.FileOutputStream;
|
||||
import java.io.FilenameFilter;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
import java.util.Set;
|
||||
import java.util.UUID;
|
||||
import java.util.regex.Pattern;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
/**
|
||||
* <p>
|
||||
* An implementation of the {@link FlowFileSwapManager} that swaps FlowFiles
|
||||
|
@ -66,9 +71,8 @@ import org.slf4j.LoggerFactory;
|
|||
*/
|
||||
public class FileSystemSwapManager implements FlowFileSwapManager {
|
||||
|
||||
public static final int MINIMUM_SWAP_COUNT = 10000;
|
||||
private static final Pattern SWAP_FILE_PATTERN = Pattern.compile("\\d+-.+\\.swap");
|
||||
private static final Pattern TEMP_SWAP_FILE_PATTERN = Pattern.compile("\\d+-.+\\.swap\\.part");
|
||||
private static final Pattern SWAP_FILE_PATTERN = Pattern.compile("\\d+-.+?(\\..*?)?\\.swap");
|
||||
private static final Pattern TEMP_SWAP_FILE_PATTERN = Pattern.compile("\\d+-.+?(\\..*?)?\\.swap\\.part");
|
||||
|
||||
public static final int SWAP_ENCODING_VERSION = 10;
|
||||
public static final String EVENT_CATEGORY = "Swap FlowFiles";
|
||||
|
@ -106,13 +110,18 @@ public class FileSystemSwapManager implements FlowFileSwapManager {
|
|||
this.flowFileRepository = initializationContext.getFlowFileRepository();
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public String swapOut(final List<FlowFileRecord> toSwap, final FlowFileQueue flowFileQueue) throws IOException {
|
||||
public String swapOut(final List<FlowFileRecord> toSwap, final FlowFileQueue flowFileQueue, final String partitionName) throws IOException {
|
||||
if (toSwap == null || toSwap.isEmpty()) {
|
||||
return null;
|
||||
}
|
||||
|
||||
final File swapFile = new File(storageDirectory, System.currentTimeMillis() + "-" + flowFileQueue.getIdentifier() + "-" + UUID.randomUUID().toString() + ".swap");
|
||||
final String swapFilePrefix = System.currentTimeMillis() + "-" + flowFileQueue.getIdentifier() + "-" + UUID.randomUUID().toString();
|
||||
final String swapFileBaseName = partitionName == null ? swapFilePrefix : swapFilePrefix + "." + partitionName;
|
||||
final String swapFileName = swapFileBaseName + ".swap";
|
||||
|
||||
final File swapFile = new File(storageDirectory, swapFileName);
|
||||
final File swapTempFile = new File(swapFile.getParentFile(), swapFile.getName() + ".part");
|
||||
final String swapLocation = swapFile.getAbsolutePath();
|
||||
|
||||
|
@ -185,8 +194,55 @@ public class FileSystemSwapManager implements FlowFileSwapManager {
|
|||
}
|
||||
}
|
||||
|
||||
private String getOwnerQueueIdentifier(final File swapFile) {
|
||||
final String[] splits = swapFile.getName().split("-");
|
||||
if (splits.length > 6) {
|
||||
final String queueIdentifier = splits[1] + "-" + splits[2] + "-" + splits[3] + "-" + splits[4] + "-" + splits[5];
|
||||
return queueIdentifier;
|
||||
}
|
||||
|
||||
return null;
|
||||
}
|
||||
|
||||
private String getOwnerPartition(final File swapFile) {
|
||||
final String filename = swapFile.getName();
|
||||
final int indexOfDot = filename.indexOf(".");
|
||||
if (indexOfDot < 1) {
|
||||
return null;
|
||||
}
|
||||
|
||||
final int lastIndexOfDot = filename.lastIndexOf(".");
|
||||
if (lastIndexOfDot == indexOfDot) {
|
||||
return null;
|
||||
}
|
||||
|
||||
return filename.substring(indexOfDot + 1, lastIndexOfDot);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<String> recoverSwapLocations(final FlowFileQueue flowFileQueue) throws IOException {
|
||||
public Set<String> getSwappedPartitionNames(final FlowFileQueue queue) {
|
||||
final File[] swapFiles = storageDirectory.listFiles(new FilenameFilter() {
|
||||
@Override
|
||||
public boolean accept(final File dir, final String name) {
|
||||
return SWAP_FILE_PATTERN.matcher(name).matches();
|
||||
}
|
||||
});
|
||||
|
||||
if (swapFiles == null) {
|
||||
return Collections.emptySet();
|
||||
}
|
||||
|
||||
final String queueId = queue.getIdentifier();
|
||||
|
||||
return Stream.of(swapFiles)
|
||||
.filter(swapFile -> queueId.equals(getOwnerQueueIdentifier(swapFile)))
|
||||
.map(this::getOwnerPartition)
|
||||
.filter(Objects::nonNull)
|
||||
.collect(Collectors.toSet());
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<String> recoverSwapLocations(final FlowFileQueue flowFileQueue, final String partitionName) throws IOException {
|
||||
final File[] swapFiles = storageDirectory.listFiles(new FilenameFilter() {
|
||||
@Override
|
||||
public boolean accept(final File dir, final String name) {
|
||||
|
@ -212,15 +268,21 @@ public class FileSystemSwapManager implements FlowFileSwapManager {
|
|||
}
|
||||
|
||||
// split the filename by dashes. The old filenaming scheme was "<timestamp>-<randomuuid>.swap" but the new naming scheme is
|
||||
// "<timestamp>-<queue identifier>-<random uuid>.swap". If we have two dashes, then we can just check if the queue ID is equal
|
||||
// to the id of the queue given and if not we can just move on.
|
||||
final String[] splits = swapFile.getName().split("-");
|
||||
if (splits.length > 6) {
|
||||
final String queueIdentifier = splits[1] + "-" + splits[2] + "-" + splits[3] + "-" + splits[4] + "-" + splits[5];
|
||||
if (queueIdentifier.equals(flowFileQueue.getIdentifier())) {
|
||||
swapLocations.add(swapFile.getAbsolutePath());
|
||||
// "<timestamp>-<queue identifier>-<random uuid>.[partition name.]swap".
|
||||
final String ownerQueueId = getOwnerQueueIdentifier(swapFile);
|
||||
if (ownerQueueId != null) {
|
||||
if (!ownerQueueId.equals(flowFileQueue.getIdentifier())) {
|
||||
continue;
|
||||
}
|
||||
|
||||
if (partitionName != null) {
|
||||
final String ownerPartition = getOwnerPartition(swapFile);
|
||||
if (!partitionName.equals(ownerPartition)) {
|
||||
continue;
|
||||
}
|
||||
}
|
||||
|
||||
swapLocations.add(swapFile.getAbsolutePath());
|
||||
continue;
|
||||
}
|
||||
|
||||
|
@ -357,4 +419,28 @@ public class FileSystemSwapManager implements FlowFileSwapManager {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public String changePartitionName(final String swapLocation, final String newPartitionName) throws IOException {
|
||||
final File existingFile = new File(swapLocation);
|
||||
if (!existingFile.exists()) {
|
||||
throw new FileNotFoundException("Could not change name of partition for swap location " + swapLocation + " because no swap file exists at that location");
|
||||
}
|
||||
|
||||
final String existingFilename = existingFile.getName();
|
||||
|
||||
final String newFilename;
|
||||
final int dotIndex = existingFilename.indexOf(".");
|
||||
if (dotIndex < 0) {
|
||||
newFilename = existingFilename + "." + newPartitionName + ".swap";
|
||||
} else {
|
||||
newFilename = existingFilename.substring(0, dotIndex) + "." + newPartitionName + ".swap";
|
||||
}
|
||||
|
||||
final File newFile = new File(existingFile.getParentFile(), newFilename);
|
||||
// Use Files.move and convert to Path's instead of File.rename so that we get an IOException on failure that describes why we failed.
|
||||
Files.move(existingFile.toPath(), newFile.toPath());
|
||||
|
||||
return newFile.getAbsolutePath();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -35,6 +35,8 @@ import org.apache.nifi.authorization.resource.DataAuthorizable;
|
|||
import org.apache.nifi.authorization.resource.ProvenanceDataAuthorizable;
|
||||
import org.apache.nifi.authorization.resource.ResourceFactory;
|
||||
import org.apache.nifi.authorization.user.NiFiUser;
|
||||
import org.apache.nifi.authorization.util.IdentityMapping;
|
||||
import org.apache.nifi.authorization.util.IdentityMappingUtil;
|
||||
import org.apache.nifi.bundle.Bundle;
|
||||
import org.apache.nifi.bundle.BundleCoordinate;
|
||||
import org.apache.nifi.cluster.coordination.ClusterCoordinator;
|
||||
|
@ -76,8 +78,23 @@ import org.apache.nifi.controller.label.Label;
|
|||
import org.apache.nifi.controller.label.StandardLabel;
|
||||
import org.apache.nifi.controller.leader.election.LeaderElectionManager;
|
||||
import org.apache.nifi.controller.leader.election.LeaderElectionStateChangeListener;
|
||||
import org.apache.nifi.controller.queue.ConnectionEventListener;
|
||||
import org.apache.nifi.controller.queue.FlowFileQueue;
|
||||
import org.apache.nifi.controller.queue.FlowFileQueueFactory;
|
||||
import org.apache.nifi.controller.queue.LoadBalanceStrategy;
|
||||
import org.apache.nifi.controller.queue.QueueSize;
|
||||
import org.apache.nifi.controller.queue.StandardFlowFileQueue;
|
||||
import org.apache.nifi.controller.queue.clustered.ContentRepositoryFlowFileAccess;
|
||||
import org.apache.nifi.controller.queue.clustered.SocketLoadBalancedFlowFileQueue;
|
||||
import org.apache.nifi.controller.queue.clustered.client.StandardLoadBalanceFlowFileCodec;
|
||||
import org.apache.nifi.controller.queue.clustered.client.async.nio.NioAsyncLoadBalanceClientFactory;
|
||||
import org.apache.nifi.controller.queue.clustered.client.async.nio.NioAsyncLoadBalanceClientRegistry;
|
||||
import org.apache.nifi.controller.queue.clustered.client.async.nio.NioAsyncLoadBalanceClientTask;
|
||||
import org.apache.nifi.controller.queue.clustered.server.ClusterLoadBalanceAuthorizer;
|
||||
import org.apache.nifi.controller.queue.clustered.server.ConnectionLoadBalanceServer;
|
||||
import org.apache.nifi.controller.queue.clustered.server.LoadBalanceAuthorizer;
|
||||
import org.apache.nifi.controller.queue.clustered.server.LoadBalanceProtocol;
|
||||
import org.apache.nifi.controller.queue.clustered.server.StandardLoadBalanceProtocol;
|
||||
import org.apache.nifi.controller.reporting.ReportingTaskInstantiationException;
|
||||
import org.apache.nifi.controller.reporting.ReportingTaskProvider;
|
||||
import org.apache.nifi.controller.reporting.StandardReportingInitializationContext;
|
||||
|
@ -243,6 +260,7 @@ import java.io.InputStream;
|
|||
import java.io.OutputStream;
|
||||
import java.lang.management.GarbageCollectorMXBean;
|
||||
import java.lang.management.ManagementFactory;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.net.URL;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
|
@ -324,6 +342,11 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
private final VariableRegistry variableRegistry;
|
||||
private final ConcurrentMap<String, ControllerServiceNode> rootControllerServices = new ConcurrentHashMap<>();
|
||||
|
||||
private final ConnectionLoadBalanceServer loadBalanceServer;
|
||||
private final NioAsyncLoadBalanceClientRegistry loadBalanceClientRegistry;
|
||||
private final FlowEngine loadBalanceClientThreadPool;
|
||||
private final Set<NioAsyncLoadBalanceClientTask> loadBalanceClientTasks = new HashSet<>();
|
||||
|
||||
private final ConcurrentMap<String, ProcessorNode> allProcessors = new ConcurrentHashMap<>();
|
||||
private final ConcurrentMap<String, ProcessGroup> allProcessGroups = new ConcurrentHashMap<>();
|
||||
private final ConcurrentMap<String, Connection> allConnections = new ConcurrentHashMap<>();
|
||||
|
@ -673,8 +696,40 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
|
||||
leaderElectionManager.start();
|
||||
heartbeatMonitor.start();
|
||||
|
||||
final InetSocketAddress loadBalanceAddress = nifiProperties.getClusterLoadBalanceAddress();
|
||||
// Setup Load Balancing Server
|
||||
final EventReporter eventReporter = createEventReporter(bulletinRepository);
|
||||
final List<IdentityMapping> identityMappings = IdentityMappingUtil.getIdentityMappings(nifiProperties);
|
||||
final LoadBalanceAuthorizer authorizeConnection = new ClusterLoadBalanceAuthorizer(clusterCoordinator, eventReporter);
|
||||
final LoadBalanceProtocol loadBalanceProtocol = new StandardLoadBalanceProtocol(flowFileRepo, contentRepository, provenanceRepository, this, authorizeConnection);
|
||||
|
||||
final int numThreads = nifiProperties.getIntegerProperty(NiFiProperties.LOAD_BALANCE_MAX_THREAD_COUNT, NiFiProperties.DEFAULT_LOAD_BALANCE_MAX_THREAD_COUNT);
|
||||
final String timeoutPeriod = nifiProperties.getProperty(NiFiProperties.LOAD_BALANCE_COMMS_TIMEOUT, NiFiProperties.DEFAULT_LOAD_BALANCE_COMMS_TIMEOUT);
|
||||
final int timeoutMillis = (int) FormatUtils.getTimeDuration(timeoutPeriod, TimeUnit.MILLISECONDS);
|
||||
|
||||
loadBalanceServer = new ConnectionLoadBalanceServer(loadBalanceAddress.getHostName(), loadBalanceAddress.getPort(), sslContext,
|
||||
numThreads, loadBalanceProtocol, eventReporter, timeoutMillis);
|
||||
|
||||
|
||||
final int connectionsPerNode = nifiProperties.getIntegerProperty(NiFiProperties.LOAD_BALANCE_CONNECTIONS_PER_NODE, NiFiProperties.DEFAULT_LOAD_BALANCE_CONNECTIONS_PER_NODE);
|
||||
final NioAsyncLoadBalanceClientFactory asyncClientFactory = new NioAsyncLoadBalanceClientFactory(sslContext, timeoutMillis, new ContentRepositoryFlowFileAccess(contentRepository),
|
||||
eventReporter, new StandardLoadBalanceFlowFileCodec());
|
||||
loadBalanceClientRegistry = new NioAsyncLoadBalanceClientRegistry(asyncClientFactory, connectionsPerNode);
|
||||
|
||||
final int loadBalanceClientThreadCount = nifiProperties.getIntegerProperty(NiFiProperties.LOAD_BALANCE_MAX_THREAD_COUNT, NiFiProperties.DEFAULT_LOAD_BALANCE_MAX_THREAD_COUNT);
|
||||
loadBalanceClientThreadPool = new FlowEngine(loadBalanceClientThreadCount, "Load-Balanced Client", true);
|
||||
|
||||
for (int i=0; i < loadBalanceClientThreadCount; i++) {
|
||||
final NioAsyncLoadBalanceClientTask clientTask = new NioAsyncLoadBalanceClientTask(loadBalanceClientRegistry, clusterCoordinator, eventReporter);
|
||||
loadBalanceClientTasks.add(clientTask);
|
||||
loadBalanceClientThreadPool.submit(clientTask);
|
||||
}
|
||||
} else {
|
||||
loadBalanceClientRegistry = null;
|
||||
heartbeater = null;
|
||||
loadBalanceServer = null;
|
||||
loadBalanceClientThreadPool = null;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -775,6 +830,10 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
listener.start();
|
||||
}
|
||||
|
||||
if (loadBalanceServer != null) {
|
||||
loadBalanceServer.start();
|
||||
}
|
||||
|
||||
notifyComponentsConfigurationRestored();
|
||||
|
||||
timerDrivenEngineRef.get().scheduleWithFixedDelay(new Runnable() {
|
||||
|
@ -940,11 +999,19 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
startConnectablesAfterInitialization.clear();
|
||||
startRemoteGroupPortsAfterInitialization.clear();
|
||||
}
|
||||
|
||||
for (final Connection connection : getRootGroup().findAllConnections()) {
|
||||
connection.getFlowFileQueue().startLoadBalancing();
|
||||
}
|
||||
} finally {
|
||||
writeLock.unlock("onFlowInitialized");
|
||||
}
|
||||
}
|
||||
|
||||
public boolean isStartAfterInitialization(final Connectable component) {
|
||||
return startConnectablesAfterInitialization.contains(component) || startRemoteGroupPortsAfterInitialization.contains(component);
|
||||
}
|
||||
|
||||
private ContentRepository createContentRepository(final NiFiProperties properties) throws InstantiationException, IllegalAccessException, ClassNotFoundException {
|
||||
final String implementationClassName = properties.getProperty(NiFiProperties.CONTENT_REPOSITORY_IMPLEMENTATION, DEFAULT_CONTENT_REPO_IMPLEMENTATION);
|
||||
if (implementationClassName == null) {
|
||||
|
@ -1040,20 +1107,35 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
swapManager.initialize(initializationContext);
|
||||
}
|
||||
|
||||
return builder.id(requireNonNull(id).intern())
|
||||
final FlowFileQueueFactory flowFileQueueFactory = new FlowFileQueueFactory() {
|
||||
@Override
|
||||
public FlowFileQueue createFlowFileQueue(final LoadBalanceStrategy loadBalanceStrategy, final String partitioningAttribute, final ConnectionEventListener eventListener) {
|
||||
final FlowFileQueue flowFileQueue;
|
||||
|
||||
if (clusterCoordinator == null) {
|
||||
flowFileQueue = new StandardFlowFileQueue(id, eventListener, flowFileRepository, provenanceRepository, resourceClaimManager, processScheduler, swapManager,
|
||||
eventReporter, nifiProperties.getQueueSwapThreshold(), nifiProperties.getDefaultBackPressureObjectThreshold(), nifiProperties.getDefaultBackPressureDataSizeThreshold());
|
||||
} else {
|
||||
flowFileQueue = new SocketLoadBalancedFlowFileQueue(id, eventListener, processScheduler, flowFileRepository, provenanceRepository, contentRepository, resourceClaimManager,
|
||||
clusterCoordinator, loadBalanceClientRegistry, swapManager, nifiProperties.getQueueSwapThreshold(), eventReporter);
|
||||
|
||||
flowFileQueue.setBackPressureObjectThreshold(nifiProperties.getDefaultBackPressureObjectThreshold());
|
||||
flowFileQueue.setBackPressureDataSizeThreshold(nifiProperties.getDefaultBackPressureDataSizeThreshold());
|
||||
}
|
||||
|
||||
return flowFileQueue;
|
||||
}
|
||||
};
|
||||
|
||||
final Connection connection = builder.id(requireNonNull(id).intern())
|
||||
.name(name == null ? null : name.intern())
|
||||
.relationships(relationships)
|
||||
.source(requireNonNull(source))
|
||||
.destination(destination)
|
||||
.swapManager(swapManager)
|
||||
.queueSwapThreshold(nifiProperties.getQueueSwapThreshold())
|
||||
.defaultBackPressureObjectThreshold(nifiProperties.getDefaultBackPressureObjectThreshold())
|
||||
.defaultBackPressureDataSizeThreshold(nifiProperties.getDefaultBackPressureDataSizeThreshold())
|
||||
.eventReporter(eventReporter)
|
||||
.resourceClaimManager(resourceClaimManager)
|
||||
.flowFileRepository(flowFileRepository)
|
||||
.provenanceRepository(provenanceRepository)
|
||||
.flowFileQueueFactory(flowFileQueueFactory)
|
||||
.build();
|
||||
|
||||
return connection;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -1561,6 +1643,11 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
zooKeeperStateServer.shutdown();
|
||||
}
|
||||
|
||||
if (loadBalanceClientThreadPool != null) {
|
||||
loadBalanceClientThreadPool.shutdownNow();
|
||||
}
|
||||
loadBalanceClientTasks.forEach(NioAsyncLoadBalanceClientTask::stop);
|
||||
|
||||
// Trigger any processors' methods marked with @OnShutdown to be called
|
||||
getRootGroup().shutdown();
|
||||
|
||||
|
@ -1606,6 +1693,14 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
listener.stop();
|
||||
}
|
||||
|
||||
if (loadBalanceServer != null) {
|
||||
loadBalanceServer.stop();
|
||||
}
|
||||
|
||||
if (loadBalanceClientRegistry != null) {
|
||||
loadBalanceClientRegistry.stop();
|
||||
}
|
||||
|
||||
if (processScheduler != null) {
|
||||
processScheduler.shutdown();
|
||||
}
|
||||
|
@ -2226,6 +2321,13 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
queue.setPriorities(newPrioritizers);
|
||||
}
|
||||
|
||||
final String loadBalanceStrategyName = connectionDTO.getLoadBalanceStrategy();
|
||||
if (loadBalanceStrategyName != null) {
|
||||
final LoadBalanceStrategy loadBalanceStrategy = LoadBalanceStrategy.valueOf(loadBalanceStrategyName);
|
||||
final String partitioningAttribute = connectionDTO.getLoadBalancePartitionAttribute();
|
||||
queue.setLoadBalanceStrategy(loadBalanceStrategy, partitioningAttribute);
|
||||
}
|
||||
|
||||
connection.setProcessGroup(group);
|
||||
group.addConnection(connection);
|
||||
}
|
||||
|
@ -2737,6 +2839,10 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
|
||||
public void onConnectionAdded(final Connection connection) {
|
||||
allConnections.put(connection.getIdentifier(), connection);
|
||||
|
||||
if (isInitialized()) {
|
||||
connection.getFlowFileQueue().startLoadBalancing();
|
||||
}
|
||||
}
|
||||
|
||||
public void onConnectionRemoved(final Connection connection) {
|
||||
|
@ -3494,6 +3600,19 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
}
|
||||
}
|
||||
|
||||
public void stopTransmitting(final RemoteGroupPort remoteGroupPort) {
|
||||
writeLock.lock();
|
||||
try {
|
||||
if (initialized.get()) {
|
||||
remoteGroupPort.getRemoteProcessGroup().stopTransmitting(remoteGroupPort);
|
||||
} else {
|
||||
startRemoteGroupPortsAfterInitialization.remove(remoteGroupPort);
|
||||
}
|
||||
} finally {
|
||||
writeLock.unlock("stopTransmitting");
|
||||
}
|
||||
}
|
||||
|
||||
public void stopProcessor(final String parentGroupId, final String processorId) {
|
||||
final ProcessGroup group = lookupGroup(parentGroupId);
|
||||
final ProcessorNode node = group.getProcessor(processorId);
|
||||
|
@ -4344,10 +4463,11 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
leaderElectionManager.start();
|
||||
stateManagerProvider.enableClusterProvider();
|
||||
|
||||
loadBalanceClientRegistry.start();
|
||||
|
||||
heartbeat();
|
||||
} else {
|
||||
stateManagerProvider.disableClusterProvider();
|
||||
|
||||
setPrimary(false);
|
||||
}
|
||||
|
||||
|
@ -4369,6 +4489,8 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
|
||||
/**
|
||||
* @return true if this instance is the primary node in the cluster; false
|
||||
* otherwise
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -202,6 +202,7 @@ public class StandardFlowService implements FlowService, ProtocolHandler {
|
|||
|
||||
final InetSocketAddress nodeApiAddress = nifiProperties.getNodeApiAddress();
|
||||
final InetSocketAddress nodeSocketAddress = nifiProperties.getClusterNodeProtocolAddress();
|
||||
final InetSocketAddress loadBalanceAddress = nifiProperties.getClusterLoadBalanceAddress();
|
||||
|
||||
String nodeUuid = null;
|
||||
final StateManager stateManager = controller.getStateManagerProvider().getStateManager(CLUSTER_NODE_CONFIG);
|
||||
|
@ -217,6 +218,7 @@ public class StandardFlowService implements FlowService, ProtocolHandler {
|
|||
this.nodeId = new NodeIdentifier(nodeUuid,
|
||||
nodeApiAddress.getHostName(), nodeApiAddress.getPort(),
|
||||
nodeSocketAddress.getHostName(), nodeSocketAddress.getPort(),
|
||||
loadBalanceAddress.getHostName(), loadBalanceAddress.getPort(),
|
||||
nifiProperties.getRemoteInputHost(), nifiProperties.getRemoteInputPort(),
|
||||
nifiProperties.getRemoteInputHttpPort(), nifiProperties.isSiteToSiteSecure());
|
||||
|
||||
|
@ -388,7 +390,7 @@ public class StandardFlowService implements FlowService, ProtocolHandler {
|
|||
}
|
||||
|
||||
@Override
|
||||
public ProtocolMessage handle(final ProtocolMessage request) throws ProtocolException {
|
||||
public ProtocolMessage handle(final ProtocolMessage request, final Set<String> nodeIdentities) throws ProtocolException {
|
||||
final long startNanos = System.nanoTime();
|
||||
try {
|
||||
switch (request.getType()) {
|
||||
|
|
|
@ -16,34 +16,6 @@
|
|||
*/
|
||||
package org.apache.nifi.controller;
|
||||
|
||||
import java.io.ByteArrayInputStream;
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.net.URL;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
import java.nio.file.StandardOpenOption;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.zip.GZIPInputStream;
|
||||
|
||||
import javax.xml.XMLConstants;
|
||||
import javax.xml.parsers.DocumentBuilder;
|
||||
import javax.xml.parsers.DocumentBuilderFactory;
|
||||
import javax.xml.parsers.ParserConfigurationException;
|
||||
import javax.xml.validation.Schema;
|
||||
import javax.xml.validation.SchemaFactory;
|
||||
|
||||
import org.apache.commons.collections4.CollectionUtils;
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.authorization.Authorizer;
|
||||
|
@ -58,6 +30,8 @@ import org.apache.nifi.connectable.Connectable;
|
|||
import org.apache.nifi.connectable.ConnectableType;
|
||||
import org.apache.nifi.connectable.Connection;
|
||||
import org.apache.nifi.connectable.Funnel;
|
||||
import org.apache.nifi.controller.queue.LoadBalanceCompression;
|
||||
import org.apache.nifi.controller.queue.LoadBalanceStrategy;
|
||||
import org.apache.nifi.connectable.Port;
|
||||
import org.apache.nifi.connectable.Position;
|
||||
import org.apache.nifi.connectable.Size;
|
||||
|
@ -127,6 +101,33 @@ import org.w3c.dom.Node;
|
|||
import org.w3c.dom.NodeList;
|
||||
import org.xml.sax.SAXException;
|
||||
|
||||
import javax.xml.XMLConstants;
|
||||
import javax.xml.parsers.DocumentBuilder;
|
||||
import javax.xml.parsers.DocumentBuilderFactory;
|
||||
import javax.xml.parsers.ParserConfigurationException;
|
||||
import javax.xml.validation.Schema;
|
||||
import javax.xml.validation.SchemaFactory;
|
||||
import java.io.ByteArrayInputStream;
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.net.URL;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
import java.nio.file.StandardOpenOption;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.zip.GZIPInputStream;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class StandardFlowSynchronizer implements FlowSynchronizer {
|
||||
|
@ -837,9 +838,10 @@ public class StandardFlowSynchronizer implements FlowSynchronizer {
|
|||
final ProcessorDTO dto = FlowFromDOMFactory.getProcessor(processorElement, encryptor);
|
||||
final ProcessorNode procNode = processGroup.getProcessor(dto.getId());
|
||||
|
||||
final ScheduledState procState = getScheduledState(procNode, controller);
|
||||
updateNonFingerprintedProcessorSettings(procNode, dto);
|
||||
|
||||
if (!procNode.getScheduledState().name().equals(dto.getState())) {
|
||||
if (!procState.name().equals(dto.getState())) {
|
||||
try {
|
||||
switch (ScheduledState.valueOf(dto.getState())) {
|
||||
case DISABLED:
|
||||
|
@ -855,9 +857,9 @@ public class StandardFlowSynchronizer implements FlowSynchronizer {
|
|||
controller.startProcessor(procNode.getProcessGroupIdentifier(), procNode.getIdentifier(), false);
|
||||
break;
|
||||
case STOPPED:
|
||||
if (procNode.getScheduledState() == ScheduledState.DISABLED) {
|
||||
if (procState == ScheduledState.DISABLED) {
|
||||
procNode.getProcessGroup().enableProcessor(procNode);
|
||||
} else if (procNode.getScheduledState() == ScheduledState.RUNNING) {
|
||||
} else if (procState == ScheduledState.RUNNING) {
|
||||
controller.stopProcessor(procNode.getProcessGroupIdentifier(), procNode.getIdentifier());
|
||||
}
|
||||
break;
|
||||
|
@ -882,7 +884,9 @@ public class StandardFlowSynchronizer implements FlowSynchronizer {
|
|||
final PortDTO dto = FlowFromDOMFactory.getPort(portElement);
|
||||
final Port port = processGroup.getInputPort(dto.getId());
|
||||
|
||||
if (!port.getScheduledState().name().equals(dto.getState())) {
|
||||
final ScheduledState portState = getScheduledState(port, controller);
|
||||
|
||||
if (!portState.name().equals(dto.getState())) {
|
||||
switch (ScheduledState.valueOf(dto.getState())) {
|
||||
case DISABLED:
|
||||
// switch processor do disabled. This means we have to stop it (if it's already stopped, this method does nothing),
|
||||
|
@ -896,9 +900,9 @@ public class StandardFlowSynchronizer implements FlowSynchronizer {
|
|||
controller.startConnectable(port);
|
||||
break;
|
||||
case STOPPED:
|
||||
if (port.getScheduledState() == ScheduledState.DISABLED) {
|
||||
if (portState == ScheduledState.DISABLED) {
|
||||
port.getProcessGroup().enableInputPort(port);
|
||||
} else if (port.getScheduledState() == ScheduledState.RUNNING) {
|
||||
} else if (portState == ScheduledState.RUNNING) {
|
||||
controller.stopConnectable(port);
|
||||
}
|
||||
break;
|
||||
|
@ -911,7 +915,9 @@ public class StandardFlowSynchronizer implements FlowSynchronizer {
|
|||
final PortDTO dto = FlowFromDOMFactory.getPort(portElement);
|
||||
final Port port = processGroup.getOutputPort(dto.getId());
|
||||
|
||||
if (!port.getScheduledState().name().equals(dto.getState())) {
|
||||
final ScheduledState portState = getScheduledState(port, controller);
|
||||
|
||||
if (!portState.name().equals(dto.getState())) {
|
||||
switch (ScheduledState.valueOf(dto.getState())) {
|
||||
case DISABLED:
|
||||
// switch processor do disabled. This means we have to stop it (if it's already stopped, this method does nothing),
|
||||
|
@ -925,9 +931,9 @@ public class StandardFlowSynchronizer implements FlowSynchronizer {
|
|||
controller.startConnectable(port);
|
||||
break;
|
||||
case STOPPED:
|
||||
if (port.getScheduledState() == ScheduledState.DISABLED) {
|
||||
if (portState == ScheduledState.DISABLED) {
|
||||
port.getProcessGroup().enableOutputPort(port);
|
||||
} else if (port.getScheduledState() == ScheduledState.RUNNING) {
|
||||
} else if (portState == ScheduledState.RUNNING) {
|
||||
controller.stopConnectable(port);
|
||||
}
|
||||
break;
|
||||
|
@ -951,12 +957,14 @@ public class StandardFlowSynchronizer implements FlowSynchronizer {
|
|||
continue;
|
||||
}
|
||||
|
||||
final ScheduledState portState = getScheduledState(inputPort, controller);
|
||||
|
||||
if (portDescriptor.isTransmitting()) {
|
||||
if (inputPort.getScheduledState() != ScheduledState.RUNNING && inputPort.getScheduledState() != ScheduledState.STARTING) {
|
||||
rpg.startTransmitting(inputPort);
|
||||
if (portState != ScheduledState.RUNNING && portState != ScheduledState.STARTING) {
|
||||
controller.startTransmitting(inputPort);
|
||||
}
|
||||
} else if (inputPort.getScheduledState() != ScheduledState.STOPPED && inputPort.getScheduledState() != ScheduledState.STOPPING) {
|
||||
rpg.stopTransmitting(inputPort);
|
||||
} else if (portState != ScheduledState.STOPPED && portState != ScheduledState.STOPPING) {
|
||||
controller.stopTransmitting(inputPort);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -970,12 +978,14 @@ public class StandardFlowSynchronizer implements FlowSynchronizer {
|
|||
continue;
|
||||
}
|
||||
|
||||
final ScheduledState portState = getScheduledState(outputPort, controller);
|
||||
|
||||
if (portDescriptor.isTransmitting()) {
|
||||
if (outputPort.getScheduledState() != ScheduledState.RUNNING && outputPort.getScheduledState() != ScheduledState.STARTING) {
|
||||
rpg.startTransmitting(outputPort);
|
||||
if (portState != ScheduledState.RUNNING && portState != ScheduledState.STARTING) {
|
||||
controller.startTransmitting(outputPort);
|
||||
}
|
||||
} else if (outputPort.getScheduledState() != ScheduledState.STOPPED && outputPort.getScheduledState() != ScheduledState.STOPPING) {
|
||||
rpg.stopTransmitting(outputPort);
|
||||
} else if (portState != ScheduledState.STOPPED && portState != ScheduledState.STOPPING) {
|
||||
controller.stopTransmitting(outputPort);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -1073,6 +1083,17 @@ public class StandardFlowSynchronizer implements FlowSynchronizer {
|
|||
return processGroup;
|
||||
}
|
||||
|
||||
private <T extends Connectable & Triggerable> ScheduledState getScheduledState(final T component, final FlowController flowController) {
|
||||
final ScheduledState componentState = component.getScheduledState();
|
||||
if (componentState == ScheduledState.STOPPED) {
|
||||
if (flowController.isStartAfterInitialization(component)) {
|
||||
return ScheduledState.RUNNING;
|
||||
}
|
||||
}
|
||||
|
||||
return componentState;
|
||||
}
|
||||
|
||||
private Position toPosition(final PositionDTO dto) {
|
||||
return new Position(dto.getX(), dto.getY());
|
||||
}
|
||||
|
@ -1499,6 +1520,14 @@ public class StandardFlowSynchronizer implements FlowSynchronizer {
|
|||
connection.getFlowFileQueue().setFlowFileExpiration(dto.getFlowFileExpiration());
|
||||
}
|
||||
|
||||
if (dto.getLoadBalanceStrategy() != null) {
|
||||
connection.getFlowFileQueue().setLoadBalanceStrategy(LoadBalanceStrategy.valueOf(dto.getLoadBalanceStrategy()), dto.getLoadBalancePartitionAttribute());
|
||||
}
|
||||
|
||||
if (dto.getLoadBalanceCompression() != null) {
|
||||
connection.getFlowFileQueue().setLoadBalanceCompression(LoadBalanceCompression.valueOf(dto.getLoadBalanceCompression()));
|
||||
}
|
||||
|
||||
processGroup.addConnection(connection);
|
||||
}
|
||||
|
||||
|
|
|
@ -1426,9 +1426,14 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
|
|||
|
||||
@Override
|
||||
public synchronized int getTerminatedThreadCount() {
|
||||
return (int) activeThreads.values().stream()
|
||||
.filter(ActiveTask::isTerminated)
|
||||
.count();
|
||||
int count = 0;
|
||||
for (final ActiveTask task : activeThreads.values()) {
|
||||
if (task.isTerminated()) {
|
||||
count++;
|
||||
}
|
||||
}
|
||||
|
||||
return count;
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -0,0 +1,460 @@
|
|||
/*
|
||||
* 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.controller.queue;
|
||||
|
||||
import org.apache.nifi.controller.ProcessScheduler;
|
||||
import org.apache.nifi.controller.repository.FlowFileRecord;
|
||||
import org.apache.nifi.controller.repository.FlowFileRepository;
|
||||
import org.apache.nifi.controller.repository.RepositoryRecord;
|
||||
import org.apache.nifi.controller.repository.claim.ContentClaim;
|
||||
import org.apache.nifi.controller.repository.claim.ResourceClaim;
|
||||
import org.apache.nifi.controller.repository.claim.ResourceClaimManager;
|
||||
import org.apache.nifi.flowfile.FlowFile;
|
||||
import org.apache.nifi.flowfile.attributes.CoreAttributes;
|
||||
import org.apache.nifi.processor.DataUnit;
|
||||
import org.apache.nifi.provenance.ProvenanceEventBuilder;
|
||||
import org.apache.nifi.provenance.ProvenanceEventRecord;
|
||||
import org.apache.nifi.provenance.ProvenanceEventRepository;
|
||||
import org.apache.nifi.provenance.ProvenanceEventType;
|
||||
import org.apache.nifi.util.FormatUtils;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
|
||||
public abstract class AbstractFlowFileQueue implements FlowFileQueue {
|
||||
private static final Logger logger = LoggerFactory.getLogger(AbstractFlowFileQueue.class);
|
||||
private final String identifier;
|
||||
private final FlowFileRepository flowFileRepository;
|
||||
private final ProvenanceEventRepository provRepository;
|
||||
private final ResourceClaimManager resourceClaimManager;
|
||||
private final ProcessScheduler scheduler;
|
||||
|
||||
private final AtomicReference<TimePeriod> expirationPeriod = new AtomicReference<>(new TimePeriod("0 mins", 0L));
|
||||
private final AtomicReference<MaxQueueSize> maxQueueSize = new AtomicReference<>(new MaxQueueSize("1 GB", 1024 * 1024 * 1024, 10000));
|
||||
|
||||
private final ConcurrentMap<String, ListFlowFileRequest> listRequestMap = new ConcurrentHashMap<>();
|
||||
private final ConcurrentMap<String, DropFlowFileRequest> dropRequestMap = new ConcurrentHashMap<>();
|
||||
|
||||
private LoadBalanceStrategy loadBalanceStrategy = LoadBalanceStrategy.DO_NOT_LOAD_BALANCE;
|
||||
private String partitioningAttribute = null;
|
||||
|
||||
private LoadBalanceCompression compression = LoadBalanceCompression.DO_NOT_COMPRESS;
|
||||
|
||||
|
||||
public AbstractFlowFileQueue(final String identifier, final ProcessScheduler scheduler,
|
||||
final FlowFileRepository flowFileRepo, final ProvenanceEventRepository provRepo, final ResourceClaimManager resourceClaimManager) {
|
||||
this.identifier = identifier;
|
||||
this.scheduler = scheduler;
|
||||
this.flowFileRepository = flowFileRepo;
|
||||
this.provRepository = provRepo;
|
||||
this.resourceClaimManager = resourceClaimManager;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getIdentifier() {
|
||||
return identifier;
|
||||
}
|
||||
|
||||
protected ProcessScheduler getScheduler() {
|
||||
return scheduler;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getFlowFileExpiration() {
|
||||
return expirationPeriod.get().getPeriod();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getFlowFileExpiration(final TimeUnit timeUnit) {
|
||||
return (int) timeUnit.convert(expirationPeriod.get().getMillis(), TimeUnit.MILLISECONDS);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setFlowFileExpiration(final String flowExpirationPeriod) {
|
||||
final long millis = FormatUtils.getTimeDuration(flowExpirationPeriod, TimeUnit.MILLISECONDS);
|
||||
if (millis < 0) {
|
||||
throw new IllegalArgumentException("FlowFile Expiration Period must be positive");
|
||||
}
|
||||
|
||||
expirationPeriod.set(new TimePeriod(flowExpirationPeriod, millis));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setBackPressureObjectThreshold(final long threshold) {
|
||||
boolean updated = false;
|
||||
while (!updated) {
|
||||
MaxQueueSize maxSize = getMaxQueueSize();
|
||||
final MaxQueueSize updatedSize = new MaxQueueSize(maxSize.getMaxSize(), maxSize.getMaxBytes(), threshold);
|
||||
updated = maxQueueSize.compareAndSet(maxSize, updatedSize);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getBackPressureObjectThreshold() {
|
||||
return getMaxQueueSize().getMaxCount();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setBackPressureDataSizeThreshold(final String maxDataSize) {
|
||||
final long maxBytes = DataUnit.parseDataSize(maxDataSize, DataUnit.B).longValue();
|
||||
|
||||
boolean updated = false;
|
||||
while (!updated) {
|
||||
MaxQueueSize maxSize = getMaxQueueSize();
|
||||
final MaxQueueSize updatedSize = new MaxQueueSize(maxDataSize, maxBytes, maxSize.getMaxCount());
|
||||
updated = maxQueueSize.compareAndSet(maxSize, updatedSize);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getBackPressureDataSizeThreshold() {
|
||||
return getMaxQueueSize().getMaxSize();
|
||||
}
|
||||
|
||||
private MaxQueueSize getMaxQueueSize() {
|
||||
return maxQueueSize.get();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isFull() {
|
||||
final MaxQueueSize maxSize = getMaxQueueSize();
|
||||
|
||||
// Check if max size is set
|
||||
if (maxSize.getMaxBytes() <= 0 && maxSize.getMaxCount() <= 0) {
|
||||
return false;
|
||||
}
|
||||
|
||||
final QueueSize queueSize = size();
|
||||
if (maxSize.getMaxCount() > 0 && queueSize.getObjectCount() >= maxSize.getMaxCount()) {
|
||||
return true;
|
||||
}
|
||||
|
||||
if (maxSize.getMaxBytes() > 0 && queueSize.getByteCount() >= maxSize.getMaxBytes()) {
|
||||
return true;
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public ListFlowFileStatus listFlowFiles(final String requestIdentifier, final int maxResults) {
|
||||
// purge any old requests from the map just to keep it clean. But if there are very few requests, which is usually the case, then don't bother
|
||||
if (listRequestMap.size() > 10) {
|
||||
final List<String> toDrop = new ArrayList<>();
|
||||
for (final Map.Entry<String, ListFlowFileRequest> entry : listRequestMap.entrySet()) {
|
||||
final ListFlowFileRequest request = entry.getValue();
|
||||
final boolean completed = request.getState() == ListFlowFileState.COMPLETE || request.getState() == ListFlowFileState.FAILURE;
|
||||
|
||||
if (completed && System.currentTimeMillis() - request.getLastUpdated() > TimeUnit.MINUTES.toMillis(5L)) {
|
||||
toDrop.add(entry.getKey());
|
||||
}
|
||||
}
|
||||
|
||||
for (final String requestId : toDrop) {
|
||||
listRequestMap.remove(requestId);
|
||||
}
|
||||
}
|
||||
|
||||
// numSteps = 1 for each swap location + 1 for active queue + 1 for swap queue.
|
||||
final ListFlowFileRequest listRequest = new ListFlowFileRequest(requestIdentifier, maxResults, size());
|
||||
|
||||
final Thread t = new Thread(new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
int position = 0;
|
||||
int resultCount = 0;
|
||||
final List<FlowFileSummary> summaries = new ArrayList<>();
|
||||
|
||||
// Create an ArrayList that contains all of the contents of the active queue.
|
||||
// We do this so that we don't have to hold the lock any longer than absolutely necessary.
|
||||
// We cannot simply pull the first 'maxResults' records from the queue, however, because the
|
||||
// Iterator provided by PriorityQueue does not return records in order. So we would have to either
|
||||
// use a writeLock and 'pop' the first 'maxResults' records off the queue or use a read lock and
|
||||
// do a shallow copy of the queue. The shallow copy is generally quicker because it doesn't have to do
|
||||
// the sorting to put the records back. So even though this has an expensive of Java Heap to create the
|
||||
// extra collection, we are making this trade-off to avoid locking the queue any longer than required.
|
||||
final List<FlowFileRecord> allFlowFiles = getListableFlowFiles();
|
||||
final QueuePrioritizer prioritizer = new QueuePrioritizer(getPriorities());
|
||||
|
||||
listRequest.setState(ListFlowFileState.CALCULATING_LIST);
|
||||
|
||||
// sort the FlowFileRecords so that we have the list in the same order as on the queue.
|
||||
allFlowFiles.sort(prioritizer);
|
||||
|
||||
for (final FlowFileRecord flowFile : allFlowFiles) {
|
||||
summaries.add(summarize(flowFile, ++position));
|
||||
if (summaries.size() >= maxResults) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
logger.debug("{} Finished listing FlowFiles for active queue with a total of {} results", this, resultCount);
|
||||
listRequest.setFlowFileSummaries(summaries);
|
||||
listRequest.setState(ListFlowFileState.COMPLETE);
|
||||
}
|
||||
}, "List FlowFiles for Connection " + getIdentifier());
|
||||
t.setDaemon(true);
|
||||
t.start();
|
||||
|
||||
listRequestMap.put(requestIdentifier, listRequest);
|
||||
return listRequest;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ListFlowFileStatus getListFlowFileStatus(final String requestIdentifier) {
|
||||
return listRequestMap.get(requestIdentifier);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ListFlowFileStatus cancelListFlowFileRequest(final String requestIdentifier) {
|
||||
logger.info("Canceling ListFlowFile Request with ID {}", requestIdentifier);
|
||||
final ListFlowFileRequest request = listRequestMap.remove(requestIdentifier);
|
||||
if (request != null) {
|
||||
request.cancel();
|
||||
}
|
||||
|
||||
return request;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return all FlowFiles that should be listed in response to a List Queue request
|
||||
*/
|
||||
protected abstract List<FlowFileRecord> getListableFlowFiles();
|
||||
|
||||
|
||||
@Override
|
||||
public DropFlowFileStatus dropFlowFiles(final String requestIdentifier, final String requestor) {
|
||||
logger.info("Initiating drop of FlowFiles from {} on behalf of {} (request identifier={})", this, requestor, requestIdentifier);
|
||||
|
||||
// purge any old requests from the map just to keep it clean. But if there are very requests, which is usually the case, then don't bother
|
||||
if (dropRequestMap.size() > 10) {
|
||||
final List<String> toDrop = new ArrayList<>();
|
||||
for (final Map.Entry<String, DropFlowFileRequest> entry : dropRequestMap.entrySet()) {
|
||||
final DropFlowFileRequest request = entry.getValue();
|
||||
final boolean completed = request.getState() == DropFlowFileState.COMPLETE || request.getState() == DropFlowFileState.FAILURE;
|
||||
|
||||
if (completed && System.currentTimeMillis() - request.getLastUpdated() > TimeUnit.MINUTES.toMillis(5L)) {
|
||||
toDrop.add(entry.getKey());
|
||||
}
|
||||
}
|
||||
|
||||
for (final String requestId : toDrop) {
|
||||
dropRequestMap.remove(requestId);
|
||||
}
|
||||
}
|
||||
|
||||
final DropFlowFileRequest dropRequest = new DropFlowFileRequest(requestIdentifier);
|
||||
final QueueSize originalSize = size();
|
||||
dropRequest.setCurrentSize(originalSize);
|
||||
dropRequest.setOriginalSize(originalSize);
|
||||
if (originalSize.getObjectCount() == 0) {
|
||||
dropRequest.setDroppedSize(originalSize);
|
||||
dropRequest.setState(DropFlowFileState.COMPLETE);
|
||||
dropRequestMap.put(requestIdentifier, dropRequest);
|
||||
return dropRequest;
|
||||
}
|
||||
|
||||
final Thread t = new Thread(new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
dropFlowFiles(dropRequest, requestor);
|
||||
}
|
||||
}, "Drop FlowFiles for Connection " + getIdentifier());
|
||||
t.setDaemon(true);
|
||||
t.start();
|
||||
|
||||
dropRequestMap.put(requestIdentifier, dropRequest);
|
||||
|
||||
return dropRequest;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public DropFlowFileRequest cancelDropFlowFileRequest(final String requestIdentifier) {
|
||||
final DropFlowFileRequest request = dropRequestMap.remove(requestIdentifier);
|
||||
if (request == null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
request.cancel();
|
||||
return request;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DropFlowFileStatus getDropFlowFileStatus(final String requestIdentifier) {
|
||||
return dropRequestMap.get(requestIdentifier);
|
||||
}
|
||||
|
||||
/**
|
||||
* Synchronously drops all FlowFiles in the queue
|
||||
*
|
||||
* @param dropRequest the request
|
||||
* @param requestor the identity of the user/agent who made the request
|
||||
*/
|
||||
protected abstract void dropFlowFiles(final DropFlowFileRequest dropRequest, final String requestor);
|
||||
|
||||
@Override
|
||||
public void verifyCanList() throws IllegalStateException {
|
||||
}
|
||||
|
||||
|
||||
protected FlowFileSummary summarize(final FlowFile flowFile, final int position) {
|
||||
// extract all of the information that we care about into new variables rather than just
|
||||
// wrapping the FlowFile object with a FlowFileSummary object. We do this because we want to
|
||||
// be able to hold many FlowFileSummary objects in memory and if we just wrap the FlowFile object,
|
||||
// we will end up holding the entire FlowFile (including all Attributes) in the Java heap as well,
|
||||
// which can be problematic if we expect them to be swapped out.
|
||||
final String uuid = flowFile.getAttribute(CoreAttributes.UUID.key());
|
||||
final String filename = flowFile.getAttribute(CoreAttributes.FILENAME.key());
|
||||
final long size = flowFile.getSize();
|
||||
final Long lastQueuedTime = flowFile.getLastQueueDate();
|
||||
final long lineageStart = flowFile.getLineageStartDate();
|
||||
final boolean penalized = flowFile.isPenalized();
|
||||
|
||||
return new FlowFileSummary() {
|
||||
@Override
|
||||
public String getUuid() {
|
||||
return uuid;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getFilename() {
|
||||
return filename;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getPosition() {
|
||||
return position;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getSize() {
|
||||
return size;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getLastQueuedTime() {
|
||||
return lastQueuedTime == null ? 0L : lastQueuedTime;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getLineageStartDate() {
|
||||
return lineageStart;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isPenalized() {
|
||||
return penalized;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
protected QueueSize drop(final List<FlowFileRecord> flowFiles, final String requestor) throws IOException {
|
||||
// Create a Provenance Event and a FlowFile Repository record for each FlowFile
|
||||
final List<ProvenanceEventRecord> provenanceEvents = new ArrayList<>(flowFiles.size());
|
||||
final List<RepositoryRecord> flowFileRepoRecords = new ArrayList<>(flowFiles.size());
|
||||
for (final FlowFileRecord flowFile : flowFiles) {
|
||||
provenanceEvents.add(createDropProvenanceEvent(flowFile, requestor));
|
||||
flowFileRepoRecords.add(createDeleteRepositoryRecord(flowFile));
|
||||
}
|
||||
|
||||
long dropContentSize = 0L;
|
||||
for (final FlowFileRecord flowFile : flowFiles) {
|
||||
dropContentSize += flowFile.getSize();
|
||||
final ContentClaim contentClaim = flowFile.getContentClaim();
|
||||
if (contentClaim == null) {
|
||||
continue;
|
||||
}
|
||||
|
||||
final ResourceClaim resourceClaim = contentClaim.getResourceClaim();
|
||||
if (resourceClaim == null) {
|
||||
continue;
|
||||
}
|
||||
|
||||
resourceClaimManager.decrementClaimantCount(resourceClaim);
|
||||
}
|
||||
|
||||
provRepository.registerEvents(provenanceEvents);
|
||||
flowFileRepository.updateRepository(flowFileRepoRecords);
|
||||
return new QueueSize(flowFiles.size(), dropContentSize);
|
||||
}
|
||||
|
||||
private ProvenanceEventRecord createDropProvenanceEvent(final FlowFileRecord flowFile, final String requestor) {
|
||||
final ProvenanceEventBuilder builder = provRepository.eventBuilder();
|
||||
builder.fromFlowFile(flowFile);
|
||||
builder.setEventType(ProvenanceEventType.DROP);
|
||||
builder.setLineageStartDate(flowFile.getLineageStartDate());
|
||||
builder.setComponentId(getIdentifier());
|
||||
builder.setComponentType("Connection");
|
||||
builder.setAttributes(flowFile.getAttributes(), Collections.emptyMap());
|
||||
builder.setDetails("FlowFile Queue emptied by " + requestor);
|
||||
builder.setSourceQueueIdentifier(getIdentifier());
|
||||
|
||||
final ContentClaim contentClaim = flowFile.getContentClaim();
|
||||
if (contentClaim != null) {
|
||||
final ResourceClaim resourceClaim = contentClaim.getResourceClaim();
|
||||
builder.setPreviousContentClaim(resourceClaim.getContainer(), resourceClaim.getSection(), resourceClaim.getId(), contentClaim.getOffset(), flowFile.getSize());
|
||||
}
|
||||
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
private RepositoryRecord createDeleteRepositoryRecord(final FlowFileRecord flowFile) {
|
||||
return new DropFlowFileRepositoryRecord(this, flowFile);
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized void setLoadBalanceStrategy(final LoadBalanceStrategy strategy, final String partitioningAttribute) {
|
||||
if (strategy == LoadBalanceStrategy.PARTITION_BY_ATTRIBUTE && !FlowFile.KeyValidator.isValid(partitioningAttribute)) {
|
||||
throw new IllegalArgumentException("Cannot set Load Balance Strategy to " + strategy + " without providing a valid Partitioning Attribute");
|
||||
}
|
||||
|
||||
this.loadBalanceStrategy = strategy;
|
||||
this.partitioningAttribute = partitioningAttribute;
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized String getPartitioningAttribute() {
|
||||
return partitioningAttribute;
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized LoadBalanceStrategy getLoadBalanceStrategy() {
|
||||
return loadBalanceStrategy;
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized void setLoadBalanceCompression(final LoadBalanceCompression compression) {
|
||||
this.compression = compression;
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized LoadBalanceCompression getLoadBalanceCompression() {
|
||||
return compression;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,84 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.nifi.controller.queue;
|
||||
|
||||
import org.apache.nifi.controller.repository.FlowFileRecord;
|
||||
import org.apache.nifi.controller.repository.FlowFileSwapManager;
|
||||
import org.apache.nifi.events.EventReporter;
|
||||
|
||||
import java.util.Collection;
|
||||
import java.util.Set;
|
||||
|
||||
public class BlockingSwappablePriorityQueue extends SwappablePriorityQueue {
|
||||
private final Object monitor = new Object();
|
||||
|
||||
public BlockingSwappablePriorityQueue(final FlowFileSwapManager swapManager, final int swapThreshold, final EventReporter eventReporter, final FlowFileQueue flowFileQueue,
|
||||
final DropFlowFileAction dropAction, final String partitionName) {
|
||||
|
||||
super(swapManager, swapThreshold, eventReporter, flowFileQueue, dropAction, partitionName);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void put(final FlowFileRecord flowFile) {
|
||||
super.put(flowFile);
|
||||
|
||||
synchronized (monitor) {
|
||||
monitor.notify();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void putAll(final Collection<FlowFileRecord> flowFiles) {
|
||||
super.putAll(flowFiles);
|
||||
|
||||
synchronized (monitor) {
|
||||
monitor.notifyAll();
|
||||
}
|
||||
}
|
||||
|
||||
public FlowFileRecord poll(final Set<FlowFileRecord> expiredRecords, final long expirationMillis, final long waitMillis) throws InterruptedException {
|
||||
final long maxTimestamp = System.currentTimeMillis() + waitMillis;
|
||||
|
||||
synchronized (monitor) {
|
||||
FlowFileRecord flowFile = null;
|
||||
do {
|
||||
flowFile = super.poll(expiredRecords, expirationMillis);
|
||||
if (flowFile != null) {
|
||||
return flowFile;
|
||||
}
|
||||
|
||||
monitor.wait(waitMillis);
|
||||
} while (System.currentTimeMillis() < maxTimestamp);
|
||||
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void inheritQueueContents(final FlowFileQueueContents queueContents) {
|
||||
// We have to override this method and synchronize on monitor before calling super.inheritQueueContents.
|
||||
// If we don't do this, then our super class will obtain the write lock and call putAll, which will cause
|
||||
// us to synchronize on monitor AFTER obtaining the write lock (WriteLock then monitor).
|
||||
// If poll() is then called, we will synchronize on monitor, THEN attempt to obtain the write lock (monitor then WriteLock),
|
||||
// which would cause a deadlock.
|
||||
synchronized (monitor) {
|
||||
super.inheritQueueContents(queueContents);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,24 @@
|
|||
/*
|
||||
* 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.controller.queue;
|
||||
|
||||
public interface ConnectionEventListener {
|
||||
void triggerSourceEvent();
|
||||
|
||||
void triggerDestinationEvent();
|
||||
}
|
|
@ -0,0 +1,27 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.nifi.controller.queue;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.nifi.controller.repository.FlowFileRecord;
|
||||
|
||||
public interface DropFlowFileAction {
|
||||
QueueSize drop(List<FlowFileRecord> flowFiles, String requestor) throws IOException;
|
||||
}
|
|
@ -0,0 +1,91 @@
|
|||
/*
|
||||
* 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.controller.queue;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.nifi.controller.repository.FlowFileRecord;
|
||||
import org.apache.nifi.controller.repository.RepositoryRecord;
|
||||
import org.apache.nifi.controller.repository.RepositoryRecordType;
|
||||
import org.apache.nifi.controller.repository.claim.ContentClaim;
|
||||
|
||||
public class DropFlowFileRepositoryRecord implements RepositoryRecord {
|
||||
private final FlowFileQueue queue;
|
||||
private final FlowFileRecord flowFile;
|
||||
|
||||
public DropFlowFileRepositoryRecord(final FlowFileQueue queue, final FlowFileRecord flowFile) {
|
||||
this.queue = queue;
|
||||
this.flowFile = flowFile;
|
||||
}
|
||||
|
||||
@Override
|
||||
public FlowFileQueue getDestination() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public FlowFileQueue getOriginalQueue() {
|
||||
return queue;
|
||||
}
|
||||
|
||||
@Override
|
||||
public RepositoryRecordType getType() {
|
||||
return RepositoryRecordType.DELETE;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ContentClaim getCurrentClaim() {
|
||||
return flowFile.getContentClaim();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ContentClaim getOriginalClaim() {
|
||||
return flowFile.getContentClaim();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getCurrentClaimOffset() {
|
||||
return flowFile.getContentClaimOffset();
|
||||
}
|
||||
|
||||
@Override
|
||||
public FlowFileRecord getCurrent() {
|
||||
return flowFile;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isAttributesChanged() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isMarkedForAbort() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getSwapLocation() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<ContentClaim> getTransientClaims() {
|
||||
return Collections.emptyList();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,46 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.nifi.controller.queue;
|
||||
|
||||
import org.apache.nifi.controller.repository.FlowFileRecord;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
public class FlowFileQueueContents {
|
||||
private final List<String> swapLocations;
|
||||
private final List<FlowFileRecord> activeFlowFiles;
|
||||
private final QueueSize swapSize;
|
||||
|
||||
public FlowFileQueueContents(final List<FlowFileRecord> activeFlowFiles, final List<String> swapLocations, final QueueSize swapSize) {
|
||||
this.activeFlowFiles = activeFlowFiles;
|
||||
this.swapLocations = swapLocations;
|
||||
this.swapSize = swapSize;
|
||||
}
|
||||
|
||||
public List<FlowFileRecord> getActiveFlowFiles() {
|
||||
return activeFlowFiles;
|
||||
}
|
||||
|
||||
public List<String> getSwapLocations() {
|
||||
return swapLocations;
|
||||
}
|
||||
|
||||
public QueueSize getSwapSize() {
|
||||
return swapSize;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,22 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.nifi.controller.queue;
|
||||
|
||||
public interface FlowFileQueueFactory {
|
||||
FlowFileQueue createFlowFileQueue(LoadBalanceStrategy loadBalanceStrategy, String partitioningAttribute, ConnectionEventListener connectionEventListener);
|
||||
}
|
|
@ -0,0 +1,94 @@
|
|||
/*
|
||||
* 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.controller.queue;
|
||||
|
||||
public class FlowFileQueueSize {
|
||||
private final int activeQueueCount;
|
||||
private final long activeQueueBytes;
|
||||
private final int swappedCount;
|
||||
private final long swappedBytes;
|
||||
private final int swapFiles;
|
||||
private final int unacknowledgedCount;
|
||||
private final long unacknowledgedBytes;
|
||||
|
||||
public FlowFileQueueSize(final int activeQueueCount, final long activeQueueBytes, final int swappedCount, final long swappedBytes, final int swapFileCount,
|
||||
final int unacknowledgedCount, final long unacknowledgedBytes) {
|
||||
this.activeQueueCount = activeQueueCount;
|
||||
this.activeQueueBytes = activeQueueBytes;
|
||||
this.swappedCount = swappedCount;
|
||||
this.swappedBytes = swappedBytes;
|
||||
this.swapFiles = swapFileCount;
|
||||
this.unacknowledgedCount = unacknowledgedCount;
|
||||
this.unacknowledgedBytes = unacknowledgedBytes;
|
||||
}
|
||||
|
||||
public int getSwappedCount() {
|
||||
return swappedCount;
|
||||
}
|
||||
|
||||
public long getSwappedBytes() {
|
||||
return swappedBytes;
|
||||
}
|
||||
|
||||
public int getSwapFileCount() {
|
||||
return swapFiles;
|
||||
}
|
||||
|
||||
public int getActiveCount() {
|
||||
return activeQueueCount;
|
||||
}
|
||||
|
||||
public long getActiveBytes() {
|
||||
return activeQueueBytes;
|
||||
}
|
||||
|
||||
public int getUnacknowledgedCount() {
|
||||
return unacknowledgedCount;
|
||||
}
|
||||
|
||||
public long getUnacknowledgedBytes() {
|
||||
return unacknowledgedBytes;
|
||||
}
|
||||
|
||||
public boolean isEmpty() {
|
||||
return activeQueueCount == 0 && swappedCount == 0 && unacknowledgedCount == 0;
|
||||
}
|
||||
|
||||
public QueueSize toQueueSize() {
|
||||
return new QueueSize(activeQueueCount + swappedCount + unacknowledgedCount, activeQueueBytes + swappedBytes + unacknowledgedBytes);
|
||||
}
|
||||
|
||||
public QueueSize activeQueueSize() {
|
||||
return new QueueSize(activeQueueCount, activeQueueBytes);
|
||||
}
|
||||
|
||||
public QueueSize unacknowledgedQueueSize() {
|
||||
return new QueueSize(unacknowledgedCount, unacknowledgedBytes);
|
||||
}
|
||||
|
||||
public QueueSize swapQueueSize() {
|
||||
return new QueueSize(swappedCount, swappedBytes);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "FlowFile Queue Size[ ActiveQueue=[" + activeQueueCount + ", " + activeQueueBytes +
|
||||
" Bytes], Swap Queue=[" + swappedCount + ", " + swappedBytes +
|
||||
" Bytes], Swap Files=[" + swapFiles + "], Unacknowledged=[" + unacknowledgedCount + ", " + unacknowledgedBytes + " Bytes] ]";
|
||||
}
|
||||
}
|
|
@ -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.controller.queue;
|
||||
|
||||
public class MaxQueueSize {
|
||||
private final String maxSize;
|
||||
private final long maxBytes;
|
||||
private final long maxCount;
|
||||
|
||||
public MaxQueueSize(final String maxSize, final long maxBytes, final long maxCount) {
|
||||
this.maxSize = maxSize;
|
||||
this.maxBytes = maxBytes;
|
||||
this.maxCount = maxCount;
|
||||
}
|
||||
|
||||
public String getMaxSize() {
|
||||
return maxSize;
|
||||
}
|
||||
|
||||
public long getMaxBytes() {
|
||||
return maxBytes;
|
||||
}
|
||||
|
||||
public long getMaxCount() {
|
||||
return maxCount;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return maxCount + " Objects/" + maxSize;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,29 @@
|
|||
/*
|
||||
* 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.controller.queue;
|
||||
|
||||
public class NopConnectionEventListener implements ConnectionEventListener {
|
||||
@Override
|
||||
public void triggerSourceEvent() {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void triggerDestinationEvent() {
|
||||
|
||||
}
|
||||
}
|
|
@ -0,0 +1,90 @@
|
|||
/*
|
||||
* 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.controller.queue;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.nifi.controller.repository.FlowFileRecord;
|
||||
import org.apache.nifi.controller.repository.claim.ContentClaim;
|
||||
import org.apache.nifi.flowfile.FlowFilePrioritizer;
|
||||
|
||||
public class QueuePrioritizer implements Comparator<FlowFileRecord>, Serializable {
|
||||
private static final long serialVersionUID = 1L;
|
||||
private final transient List<FlowFilePrioritizer> prioritizers = new ArrayList<>();
|
||||
|
||||
public QueuePrioritizer(final List<FlowFilePrioritizer> priorities) {
|
||||
if (null != priorities) {
|
||||
prioritizers.addAll(priorities);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compare(final FlowFileRecord f1, final FlowFileRecord f2) {
|
||||
int returnVal = 0;
|
||||
final boolean f1Penalized = f1.isPenalized();
|
||||
final boolean f2Penalized = f2.isPenalized();
|
||||
|
||||
if (f1Penalized && !f2Penalized) {
|
||||
return 1;
|
||||
} else if (!f1Penalized && f2Penalized) {
|
||||
return -1;
|
||||
}
|
||||
|
||||
if (f1Penalized && f2Penalized) {
|
||||
if (f1.getPenaltyExpirationMillis() < f2.getPenaltyExpirationMillis()) {
|
||||
return -1;
|
||||
} else if (f1.getPenaltyExpirationMillis() > f2.getPenaltyExpirationMillis()) {
|
||||
return 1;
|
||||
}
|
||||
}
|
||||
|
||||
if (!prioritizers.isEmpty()) {
|
||||
for (final FlowFilePrioritizer prioritizer : prioritizers) {
|
||||
returnVal = prioritizer.compare(f1, f2);
|
||||
if (returnVal != 0) {
|
||||
return returnVal;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
final ContentClaim claim1 = f1.getContentClaim();
|
||||
final ContentClaim claim2 = f2.getContentClaim();
|
||||
|
||||
// put the one without a claim first
|
||||
if (claim1 == null && claim2 != null) {
|
||||
return -1;
|
||||
} else if (claim1 != null && claim2 == null) {
|
||||
return 1;
|
||||
} else if (claim1 != null && claim2 != null) {
|
||||
final int claimComparison = claim1.compareTo(claim2);
|
||||
if (claimComparison != 0) {
|
||||
return claimComparison;
|
||||
}
|
||||
|
||||
final int claimOffsetComparison = Long.compare(f1.getContentClaimOffset(), f2.getContentClaimOffset());
|
||||
if (claimOffsetComparison != 0) {
|
||||
return claimOffsetComparison;
|
||||
}
|
||||
}
|
||||
|
||||
return Long.compare(f1.getId(), f2.getId());
|
||||
}
|
||||
}
|
|
@ -0,0 +1,213 @@
|
|||
/*
|
||||
* 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.controller.queue;
|
||||
|
||||
import org.apache.nifi.controller.ProcessScheduler;
|
||||
import org.apache.nifi.controller.repository.FlowFileRecord;
|
||||
import org.apache.nifi.controller.repository.FlowFileRepository;
|
||||
import org.apache.nifi.controller.repository.FlowFileSwapManager;
|
||||
import org.apache.nifi.controller.repository.SwapSummary;
|
||||
import org.apache.nifi.controller.repository.claim.ResourceClaimManager;
|
||||
import org.apache.nifi.events.EventReporter;
|
||||
import org.apache.nifi.flowfile.FlowFilePrioritizer;
|
||||
import org.apache.nifi.processor.FlowFileFilter;
|
||||
import org.apache.nifi.provenance.ProvenanceEventRepository;
|
||||
import org.apache.nifi.util.concurrency.TimedLock;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||
|
||||
/**
|
||||
* A FlowFileQueue is used to queue FlowFile objects that are awaiting further
|
||||
* processing. Must be thread safe.
|
||||
*
|
||||
*/
|
||||
public class StandardFlowFileQueue extends AbstractFlowFileQueue implements FlowFileQueue {
|
||||
|
||||
private final SwappablePriorityQueue queue;
|
||||
private final ConnectionEventListener eventListener;
|
||||
|
||||
private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(true);
|
||||
private final FlowFileSwapManager swapManager;
|
||||
private final TimedLock writeLock;
|
||||
|
||||
|
||||
public StandardFlowFileQueue(final String identifier, final ConnectionEventListener eventListener, final FlowFileRepository flowFileRepo, final ProvenanceEventRepository provRepo,
|
||||
final ResourceClaimManager resourceClaimManager, final ProcessScheduler scheduler, final FlowFileSwapManager swapManager, final EventReporter eventReporter,
|
||||
final int swapThreshold, final long defaultBackPressureObjectThreshold, final String defaultBackPressureDataSizeThreshold) {
|
||||
|
||||
super(identifier, scheduler, flowFileRepo, provRepo, resourceClaimManager);
|
||||
this.swapManager = swapManager;
|
||||
this.queue = new SwappablePriorityQueue(swapManager, swapThreshold, eventReporter, this, this::drop, null);
|
||||
this.eventListener = eventListener;
|
||||
|
||||
writeLock = new TimedLock(this.lock.writeLock(), getIdentifier() + " Write Lock", 100);
|
||||
|
||||
setBackPressureDataSizeThreshold(defaultBackPressureDataSizeThreshold);
|
||||
setBackPressureObjectThreshold(defaultBackPressureObjectThreshold);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void startLoadBalancing() {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void stopLoadBalancing() {
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isActivelyLoadBalancing() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setPriorities(final List<FlowFilePrioritizer> newPriorities) {
|
||||
queue.setPriorities(newPriorities);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<FlowFilePrioritizer> getPriorities() {
|
||||
return queue.getPriorities();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected List<FlowFileRecord> getListableFlowFiles() {
|
||||
return queue.getActiveFlowFiles();
|
||||
}
|
||||
|
||||
@Override
|
||||
public QueueDiagnostics getQueueDiagnostics() {
|
||||
return new StandardQueueDiagnostics(queue.getQueueDiagnostics(), Collections.emptyList());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void put(final FlowFileRecord file) {
|
||||
queue.put(file);
|
||||
|
||||
eventListener.triggerDestinationEvent();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void putAll(final Collection<FlowFileRecord> files) {
|
||||
queue.putAll(files);
|
||||
|
||||
eventListener.triggerDestinationEvent();
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public FlowFileRecord poll(final Set<FlowFileRecord> expiredRecords) {
|
||||
// First check if we have any records Pre-Fetched.
|
||||
final long expirationMillis = getFlowFileExpiration(TimeUnit.MILLISECONDS);
|
||||
return queue.poll(expiredRecords, expirationMillis);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public List<FlowFileRecord> poll(int maxResults, final Set<FlowFileRecord> expiredRecords) {
|
||||
return queue.poll(maxResults, expiredRecords, getFlowFileExpiration(TimeUnit.MILLISECONDS));
|
||||
}
|
||||
|
||||
|
||||
|
||||
@Override
|
||||
public void acknowledge(final FlowFileRecord flowFile) {
|
||||
queue.acknowledge(flowFile);
|
||||
|
||||
eventListener.triggerSourceEvent();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void acknowledge(final Collection<FlowFileRecord> flowFiles) {
|
||||
queue.acknowledge(flowFiles);
|
||||
|
||||
eventListener.triggerSourceEvent();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isUnacknowledgedFlowFile() {
|
||||
return queue.isUnacknowledgedFlowFile();
|
||||
}
|
||||
|
||||
@Override
|
||||
public QueueSize size() {
|
||||
return queue.size();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isEmpty() {
|
||||
return queue.getFlowFileQueueSize().isEmpty();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isActiveQueueEmpty() {
|
||||
final FlowFileQueueSize queueSize = queue.getFlowFileQueueSize();
|
||||
return queueSize.getActiveCount() == 0 && queueSize.getSwappedCount() == 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<FlowFileRecord> poll(final FlowFileFilter filter, final Set<FlowFileRecord> expiredRecords) {
|
||||
return queue.poll(filter, expiredRecords, getFlowFileExpiration(TimeUnit.MILLISECONDS));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void purgeSwapFiles() {
|
||||
swapManager.purge();
|
||||
}
|
||||
|
||||
@Override
|
||||
public SwapSummary recoverSwappedFlowFiles() {
|
||||
return queue.recoverSwappedFlowFiles();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "FlowFileQueue[id=" + getIdentifier() + "]";
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public FlowFileRecord getFlowFile(final String flowFileUuid) throws IOException {
|
||||
return queue.getFlowFile(flowFileUuid);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
protected void dropFlowFiles(final DropFlowFileRequest dropRequest, final String requestor) {
|
||||
queue.dropFlowFiles(dropRequest, requestor);
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Lock the queue so that other threads are unable to interact with the queue
|
||||
*/
|
||||
public void lock() {
|
||||
writeLock.lock();
|
||||
}
|
||||
|
||||
/**
|
||||
* Unlock the queue
|
||||
*/
|
||||
public void unlock() {
|
||||
writeLock.unlock("external unlock");
|
||||
}
|
||||
}
|
|
@ -0,0 +1,60 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.nifi.controller.queue;
|
||||
|
||||
public class StandardLocalQueuePartitionDiagnostics implements LocalQueuePartitionDiagnostics {
|
||||
private final FlowFileQueueSize queueSize;
|
||||
private final boolean anyPenalized;
|
||||
private final boolean allPenalized;
|
||||
|
||||
public StandardLocalQueuePartitionDiagnostics(final FlowFileQueueSize queueSize, final boolean anyPenalized, final boolean allPenalized) {
|
||||
this.queueSize = queueSize;
|
||||
this.anyPenalized = anyPenalized;
|
||||
this.allPenalized = allPenalized;
|
||||
}
|
||||
|
||||
@Override
|
||||
public QueueSize getUnacknowledgedQueueSize() {
|
||||
return new QueueSize(queueSize.getUnacknowledgedCount(), queueSize.getUnacknowledgedCount());
|
||||
}
|
||||
|
||||
@Override
|
||||
public QueueSize getActiveQueueSize() {
|
||||
return new QueueSize(queueSize.getActiveCount(), queueSize.getActiveBytes());
|
||||
}
|
||||
|
||||
@Override
|
||||
public QueueSize getSwapQueueSize() {
|
||||
return new QueueSize(queueSize.getSwappedCount(), queueSize.getSwappedBytes());
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getSwapFileCount() {
|
||||
return queueSize.getSwapFileCount();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isAnyActiveFlowFilePenalized() {
|
||||
return anyPenalized;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isAllActiveFlowFilesPenalized() {
|
||||
return allPenalized;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,40 @@
|
|||
/*
|
||||
* 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.controller.queue;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
public class StandardQueueDiagnostics implements QueueDiagnostics {
|
||||
final LocalQueuePartitionDiagnostics localQueuePartitionDiagnostics;
|
||||
final List<RemoteQueuePartitionDiagnostics> remoteQueuePartitionDiagnostics;
|
||||
|
||||
public StandardQueueDiagnostics(final LocalQueuePartitionDiagnostics localQueuePartitionDiagnostics, final List<RemoteQueuePartitionDiagnostics> remoteQueuePartitionDiagnostics) {
|
||||
this.localQueuePartitionDiagnostics = localQueuePartitionDiagnostics;
|
||||
this.remoteQueuePartitionDiagnostics = remoteQueuePartitionDiagnostics;
|
||||
}
|
||||
|
||||
@Override
|
||||
public LocalQueuePartitionDiagnostics getLocalQueuePartitionDiagnostics() {
|
||||
return localQueuePartitionDiagnostics;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<RemoteQueuePartitionDiagnostics> getRemoteQueuePartitionDiagnostics() {
|
||||
return remoteQueuePartitionDiagnostics;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,53 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.nifi.controller.queue;
|
||||
|
||||
public class StandardRemoteQueuePartitionDiagnostics implements RemoteQueuePartitionDiagnostics {
|
||||
private final String nodeId;
|
||||
private final FlowFileQueueSize queueSize;
|
||||
|
||||
public StandardRemoteQueuePartitionDiagnostics(final String nodeId, final FlowFileQueueSize queueSize) {
|
||||
this.nodeId = nodeId;
|
||||
this.queueSize = queueSize;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getNodeIdentifier() {
|
||||
return nodeId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public QueueSize getUnacknowledgedQueueSize() {
|
||||
return new QueueSize(queueSize.getUnacknowledgedCount(), queueSize.getUnacknowledgedCount());
|
||||
}
|
||||
|
||||
@Override
|
||||
public QueueSize getActiveQueueSize() {
|
||||
return new QueueSize(queueSize.getActiveCount(), queueSize.getActiveBytes());
|
||||
}
|
||||
|
||||
@Override
|
||||
public QueueSize getSwapQueueSize() {
|
||||
return new QueueSize(queueSize.getSwappedCount(), queueSize.getSwappedBytes());
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getSwapFileCount() {
|
||||
return queueSize.getSwapFileCount();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,990 @@
|
|||
/*
|
||||
* 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.controller.queue;
|
||||
|
||||
import org.apache.nifi.controller.repository.FlowFileRecord;
|
||||
import org.apache.nifi.controller.repository.FlowFileSwapManager;
|
||||
import org.apache.nifi.controller.repository.IncompleteSwapFileException;
|
||||
import org.apache.nifi.controller.repository.SwapContents;
|
||||
import org.apache.nifi.controller.repository.SwapSummary;
|
||||
import org.apache.nifi.controller.repository.claim.ResourceClaim;
|
||||
import org.apache.nifi.controller.swap.StandardSwapSummary;
|
||||
import org.apache.nifi.events.EventReporter;
|
||||
import org.apache.nifi.flowfile.FlowFile;
|
||||
import org.apache.nifi.flowfile.FlowFilePrioritizer;
|
||||
import org.apache.nifi.flowfile.attributes.CoreAttributes;
|
||||
import org.apache.nifi.processor.FlowFileFilter;
|
||||
import org.apache.nifi.processor.FlowFileFilter.FlowFileFilterResult;
|
||||
import org.apache.nifi.reporting.Severity;
|
||||
import org.apache.nifi.util.concurrency.TimedLock;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.PriorityQueue;
|
||||
import java.util.Queue;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||
|
||||
|
||||
public class SwappablePriorityQueue {
|
||||
private static final Logger logger = LoggerFactory.getLogger(SwappablePriorityQueue.class);
|
||||
private static final int SWAP_RECORD_POLL_SIZE = 10_000;
|
||||
private static final int MAX_EXPIRED_RECORDS_PER_ITERATION = 10_000;
|
||||
|
||||
private final int swapThreshold;
|
||||
private final FlowFileSwapManager swapManager;
|
||||
private final EventReporter eventReporter;
|
||||
private final FlowFileQueue flowFileQueue;
|
||||
private final DropFlowFileAction dropAction;
|
||||
private final List<FlowFilePrioritizer> priorities = new ArrayList<>();
|
||||
private final String swapPartitionName;
|
||||
|
||||
private final List<String> swapLocations = new ArrayList<>();
|
||||
private final AtomicReference<FlowFileQueueSize> size = new AtomicReference<>(new FlowFileQueueSize(0, 0L, 0, 0L, 0, 0, 0L));
|
||||
private final TimedLock readLock;
|
||||
private final TimedLock writeLock;
|
||||
|
||||
// We keep an "active queue" and a "swap queue" that both are able to hold records in heap. When
|
||||
// FlowFiles are added to this FlowFileQueue, we first check if we are in "swap mode" and if so
|
||||
// we add to the 'swap queue' instead of the 'active queue'. The code would be much simpler if we
|
||||
// eliminated the 'swap queue' and instead just used the active queue and swapped out the 10,000
|
||||
// lowest priority FlowFiles from that. However, doing that would cause problems with the ordering
|
||||
// of FlowFiles. If we swap out some FlowFiles, and then allow a new FlowFile to be written to the
|
||||
// active queue, then we would end up processing the newer FlowFile before the swapped FlowFile. By
|
||||
// keeping these separate, we are able to guarantee that FlowFiles are swapped in in the same order
|
||||
// that they are swapped out.
|
||||
// Guarded by lock.
|
||||
private PriorityQueue<FlowFileRecord> activeQueue;
|
||||
private ArrayList<FlowFileRecord> swapQueue;
|
||||
private boolean swapMode = false;
|
||||
|
||||
public SwappablePriorityQueue(final FlowFileSwapManager swapManager, final int swapThreshold, final EventReporter eventReporter, final FlowFileQueue flowFileQueue,
|
||||
final DropFlowFileAction dropAction, final String swapPartitionName) {
|
||||
this.swapManager = swapManager;
|
||||
this.swapThreshold = swapThreshold;
|
||||
|
||||
this.activeQueue = new PriorityQueue<>(20, new QueuePrioritizer(Collections.emptyList()));
|
||||
this.swapQueue = new ArrayList<>();
|
||||
this.eventReporter = eventReporter;
|
||||
this.flowFileQueue = flowFileQueue;
|
||||
this.dropAction = dropAction;
|
||||
this.swapPartitionName = swapPartitionName;
|
||||
|
||||
final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(true);
|
||||
readLock = new TimedLock(lock.readLock(), flowFileQueue.getIdentifier() + " Read Lock", 100);
|
||||
writeLock = new TimedLock(lock.writeLock(), flowFileQueue.getIdentifier() + " Write Lock", 100);
|
||||
}
|
||||
|
||||
private String getQueueIdentifier() {
|
||||
return flowFileQueue.getIdentifier();
|
||||
}
|
||||
|
||||
public synchronized List<FlowFilePrioritizer> getPriorities() {
|
||||
readLock.lock();
|
||||
try {
|
||||
return Collections.unmodifiableList(priorities);
|
||||
} finally {
|
||||
readLock.unlock("getPriorities");
|
||||
}
|
||||
}
|
||||
|
||||
public void setPriorities(final List<FlowFilePrioritizer> newPriorities) {
|
||||
writeLock.lock();
|
||||
try {
|
||||
priorities.clear();
|
||||
priorities.addAll(newPriorities);
|
||||
|
||||
final PriorityQueue<FlowFileRecord> newQueue = new PriorityQueue<>(Math.max(20, activeQueue.size()), new QueuePrioritizer(newPriorities));
|
||||
newQueue.addAll(activeQueue);
|
||||
activeQueue = newQueue;
|
||||
} finally {
|
||||
writeLock.unlock("setPriorities");
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
public LocalQueuePartitionDiagnostics getQueueDiagnostics() {
|
||||
readLock.lock();
|
||||
try {
|
||||
final boolean anyPenalized = !activeQueue.isEmpty() && activeQueue.peek().isPenalized();
|
||||
final boolean allPenalized = anyPenalized && activeQueue.stream().anyMatch(FlowFileRecord::isPenalized);
|
||||
|
||||
return new StandardLocalQueuePartitionDiagnostics(getFlowFileQueueSize(), anyPenalized, allPenalized);
|
||||
} finally {
|
||||
readLock.unlock("getQueueDiagnostics");
|
||||
}
|
||||
}
|
||||
|
||||
public List<FlowFileRecord> getActiveFlowFiles() {
|
||||
readLock.lock();
|
||||
try {
|
||||
return new ArrayList<>(activeQueue);
|
||||
} finally {
|
||||
readLock.unlock("getActiveFlowFiles");
|
||||
}
|
||||
}
|
||||
|
||||
public boolean isUnacknowledgedFlowFile() {
|
||||
return getFlowFileQueueSize().getUnacknowledgedCount() > 0;
|
||||
}
|
||||
|
||||
/**
|
||||
* This method MUST be called with the write lock held
|
||||
*/
|
||||
private void writeSwapFilesIfNecessary() {
|
||||
if (swapQueue.size() < SWAP_RECORD_POLL_SIZE) {
|
||||
return;
|
||||
}
|
||||
|
||||
migrateSwapToActive();
|
||||
|
||||
final int numSwapFiles = swapQueue.size() / SWAP_RECORD_POLL_SIZE;
|
||||
|
||||
int originalSwapQueueCount = swapQueue.size();
|
||||
long originalSwapQueueBytes = 0L;
|
||||
for (final FlowFileRecord flowFile : swapQueue) {
|
||||
originalSwapQueueBytes += flowFile.getSize();
|
||||
}
|
||||
|
||||
// Create a new Priority queue with the prioritizers that are set, but reverse the
|
||||
// prioritizers because we want to pull the lowest-priority FlowFiles to swap out
|
||||
final PriorityQueue<FlowFileRecord> tempQueue = new PriorityQueue<>(activeQueue.size() + swapQueue.size(), Collections.reverseOrder(new QueuePrioritizer(getPriorities())));
|
||||
tempQueue.addAll(activeQueue);
|
||||
tempQueue.addAll(swapQueue);
|
||||
|
||||
long bytesSwappedOut = 0L;
|
||||
int flowFilesSwappedOut = 0;
|
||||
final List<String> swapLocations = new ArrayList<>(numSwapFiles);
|
||||
for (int i = 0; i < numSwapFiles; i++) {
|
||||
// Create a new swap file for the next SWAP_RECORD_POLL_SIZE records
|
||||
final List<FlowFileRecord> toSwap = new ArrayList<>(SWAP_RECORD_POLL_SIZE);
|
||||
for (int j = 0; j < SWAP_RECORD_POLL_SIZE; j++) {
|
||||
final FlowFileRecord flowFile = tempQueue.poll();
|
||||
toSwap.add(flowFile);
|
||||
bytesSwappedOut += flowFile.getSize();
|
||||
flowFilesSwappedOut++;
|
||||
}
|
||||
|
||||
try {
|
||||
Collections.reverse(toSwap); // currently ordered in reverse priority order based on the ordering of the temp queue.
|
||||
final String swapLocation = swapManager.swapOut(toSwap, flowFileQueue, swapPartitionName);
|
||||
swapLocations.add(swapLocation);
|
||||
} catch (final IOException ioe) {
|
||||
tempQueue.addAll(toSwap); // if we failed, we must add the FlowFiles back to the queue.
|
||||
|
||||
final int objectCount = getFlowFileCount();
|
||||
logger.error("FlowFile Queue with identifier {} has {} FlowFiles queued up. Attempted to spill FlowFile information over to disk in order to avoid exhausting "
|
||||
+ "the Java heap space but failed to write information to disk due to {}", getQueueIdentifier(), objectCount, ioe.toString());
|
||||
logger.error("", ioe);
|
||||
if (eventReporter != null) {
|
||||
eventReporter.reportEvent(Severity.ERROR, "Failed to Overflow to Disk", "Flowfile Queue with identifier " + getQueueIdentifier() + " has " + objectCount +
|
||||
" queued up. Attempted to spill FlowFile information over to disk in order to avoid exhausting the Java heap space but failed to write information to disk. "
|
||||
+ "See logs for more information.");
|
||||
}
|
||||
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
// Pull any records off of the temp queue that won't fit back on the active queue, and add those to the
|
||||
// swap queue. Then add the records back to the active queue.
|
||||
swapQueue.clear();
|
||||
long updatedSwapQueueBytes = 0L;
|
||||
while (tempQueue.size() > swapThreshold) {
|
||||
final FlowFileRecord record = tempQueue.poll();
|
||||
swapQueue.add(record);
|
||||
updatedSwapQueueBytes += record.getSize();
|
||||
}
|
||||
|
||||
Collections.reverse(swapQueue); // currently ordered in reverse priority order based on the ordering of the temp queue
|
||||
|
||||
// replace the contents of the active queue, since we've merged it with the swap queue.
|
||||
activeQueue.clear();
|
||||
FlowFileRecord toRequeue;
|
||||
long activeQueueBytes = 0L;
|
||||
while ((toRequeue = tempQueue.poll()) != null) {
|
||||
activeQueue.offer(toRequeue);
|
||||
activeQueueBytes += toRequeue.getSize();
|
||||
}
|
||||
|
||||
boolean updated = false;
|
||||
while (!updated) {
|
||||
final FlowFileQueueSize originalSize = getFlowFileQueueSize();
|
||||
|
||||
final int addedSwapRecords = swapQueue.size() - originalSwapQueueCount;
|
||||
final long addedSwapBytes = updatedSwapQueueBytes - originalSwapQueueBytes;
|
||||
|
||||
final FlowFileQueueSize newSize = new FlowFileQueueSize(activeQueue.size(), activeQueueBytes,
|
||||
originalSize.getSwappedCount() + addedSwapRecords + flowFilesSwappedOut,
|
||||
originalSize.getSwappedBytes() + addedSwapBytes + bytesSwappedOut,
|
||||
originalSize.getSwapFileCount() + numSwapFiles,
|
||||
originalSize.getUnacknowledgedCount(), originalSize.getUnacknowledgedBytes());
|
||||
updated = updateSize(originalSize, newSize);
|
||||
|
||||
if (updated) {
|
||||
logIfNegative(originalSize, newSize, "swap");
|
||||
}
|
||||
}
|
||||
|
||||
this.swapLocations.addAll(swapLocations);
|
||||
}
|
||||
|
||||
private int getFlowFileCount() {
|
||||
final FlowFileQueueSize size = getFlowFileQueueSize();
|
||||
return size.getActiveCount() + size.getSwappedCount() + size.getUnacknowledgedCount();
|
||||
}
|
||||
|
||||
/**
|
||||
* If there are FlowFiles waiting on the swap queue, move them to the active
|
||||
* queue until we meet our threshold. This prevents us from having to swap
|
||||
* them to disk & then back out.
|
||||
*
|
||||
* This method MUST be called with the writeLock held.
|
||||
*/
|
||||
private void migrateSwapToActive() {
|
||||
// Migrate as many FlowFiles as we can from the Swap Queue to the Active Queue, so that we don't
|
||||
// have to swap them out & then swap them back in.
|
||||
// If we don't do this, we could get into a situation where we have potentially thousands of FlowFiles
|
||||
// sitting on the Swap Queue but not getting processed because there aren't enough to be swapped out.
|
||||
// In particular, this can happen if the queue is typically filled with surges.
|
||||
// For example, if the queue has 25,000 FlowFiles come in, it may process 20,000 of them and leave
|
||||
// 5,000 sitting on the Swap Queue. If it then takes an hour for an additional 5,000 FlowFiles to come in,
|
||||
// those FlowFiles sitting on the Swap Queue will sit there for an hour, waiting to be swapped out and
|
||||
// swapped back in again.
|
||||
// Calling this method when records are polled prevents this condition by migrating FlowFiles from the
|
||||
// Swap Queue to the Active Queue. However, we don't do this if there are FlowFiles already swapped out
|
||||
// to disk, because we want them to be swapped back in in the same order that they were swapped out.
|
||||
|
||||
final int activeQueueSize = activeQueue.size();
|
||||
if (activeQueueSize > 0 && activeQueueSize > swapThreshold - SWAP_RECORD_POLL_SIZE) {
|
||||
return;
|
||||
}
|
||||
|
||||
// If there are swap files waiting to be swapped in, swap those in first. We do this in order to ensure that those that
|
||||
// were swapped out first are then swapped back in first. If we instead just immediately migrated the FlowFiles from the
|
||||
// swap queue to the active queue, and we never run out of FlowFiles in the active queue (because destination cannot
|
||||
// keep up with queue), we will end up always processing the new FlowFiles first instead of the FlowFiles that arrived
|
||||
// first.
|
||||
if (!swapLocations.isEmpty()) {
|
||||
swapIn();
|
||||
return;
|
||||
}
|
||||
|
||||
// this is the most common condition (nothing is swapped out), so do the check first and avoid the expense
|
||||
// of other checks for 99.999% of the cases.
|
||||
final FlowFileQueueSize size = getFlowFileQueueSize();
|
||||
if (size.getSwappedCount() == 0 && swapQueue.isEmpty()) {
|
||||
return;
|
||||
}
|
||||
|
||||
if (size.getSwappedCount() > swapQueue.size()) {
|
||||
// we already have FlowFiles swapped out, so we won't migrate the queue; we will wait for
|
||||
// the files to be swapped back in first
|
||||
return;
|
||||
}
|
||||
|
||||
int recordsMigrated = 0;
|
||||
long bytesMigrated = 0L;
|
||||
final Iterator<FlowFileRecord> swapItr = swapQueue.iterator();
|
||||
while (activeQueue.size() < swapThreshold && swapItr.hasNext()) {
|
||||
final FlowFileRecord toMigrate = swapItr.next();
|
||||
activeQueue.add(toMigrate);
|
||||
bytesMigrated += toMigrate.getSize();
|
||||
recordsMigrated++;
|
||||
swapItr.remove();
|
||||
}
|
||||
|
||||
if (recordsMigrated > 0) {
|
||||
incrementActiveQueueSize(recordsMigrated, bytesMigrated);
|
||||
incrementSwapQueueSize(-recordsMigrated, -bytesMigrated, 0);
|
||||
}
|
||||
|
||||
if (size.getSwappedCount() == 0) {
|
||||
swapMode = false;
|
||||
}
|
||||
}
|
||||
|
||||
private void swapIn() {
|
||||
final String swapLocation = swapLocations.get(0);
|
||||
boolean partialContents = false;
|
||||
SwapContents swapContents;
|
||||
try {
|
||||
swapContents = swapManager.swapIn(swapLocation, flowFileQueue);
|
||||
swapLocations.remove(0);
|
||||
} catch (final IncompleteSwapFileException isfe) {
|
||||
logger.error("Failed to swap in all FlowFiles from Swap File {}; Swap File ended prematurely. The records that were present will still be swapped in", swapLocation);
|
||||
logger.error("", isfe);
|
||||
swapContents = isfe.getPartialContents();
|
||||
partialContents = true;
|
||||
swapLocations.remove(0);
|
||||
} catch (final FileNotFoundException fnfe) {
|
||||
logger.error("Failed to swap in FlowFiles from Swap File {} because the Swap File can no longer be found", swapLocation);
|
||||
if (eventReporter != null) {
|
||||
eventReporter.reportEvent(Severity.ERROR, "Swap File", "Failed to swap in FlowFiles from Swap File " + swapLocation + " because the Swap File can no longer be found");
|
||||
}
|
||||
|
||||
swapLocations.remove(0);
|
||||
return;
|
||||
} catch (final IOException ioe) {
|
||||
logger.error("Failed to swap in FlowFiles from Swap File {}; Swap File appears to be corrupt!", swapLocation);
|
||||
logger.error("", ioe);
|
||||
if (eventReporter != null) {
|
||||
eventReporter.reportEvent(Severity.ERROR, "Swap File", "Failed to swap in FlowFiles from Swap File " +
|
||||
swapLocation + "; Swap File appears to be corrupt! Some FlowFiles in the queue may not be accessible. See logs for more information.");
|
||||
}
|
||||
|
||||
// We do not remove the Swap File from swapLocations because the IOException may be recoverable later. For instance, the file may be on a network
|
||||
// drive and we may have connectivity problems, etc.
|
||||
return;
|
||||
} catch (final Throwable t) {
|
||||
logger.error("Failed to swap in FlowFiles from Swap File {}", swapLocation, t);
|
||||
|
||||
// We do not remove the Swap File from swapLocations because this is an unexpected failure that may be retry-able. For example, if there were
|
||||
// an OOME, etc. then we don't want to he queue to still reflect that the data is around but never swap it in. By leaving the Swap File
|
||||
// in swapLocations, we will continue to retry.
|
||||
throw t;
|
||||
}
|
||||
|
||||
final QueueSize swapSize = swapContents.getSummary().getQueueSize();
|
||||
final long contentSize = swapSize.getByteCount();
|
||||
final int flowFileCount = swapSize.getObjectCount();
|
||||
incrementSwapQueueSize(-flowFileCount, -contentSize, -1);
|
||||
|
||||
if (partialContents) {
|
||||
// if we have partial results, we need to calculate the content size of the flowfiles
|
||||
// actually swapped back in.
|
||||
long contentSizeSwappedIn = 0L;
|
||||
for (final FlowFileRecord swappedIn : swapContents.getFlowFiles()) {
|
||||
contentSizeSwappedIn += swappedIn.getSize();
|
||||
}
|
||||
|
||||
incrementActiveQueueSize(swapContents.getFlowFiles().size(), contentSizeSwappedIn);
|
||||
logger.debug("Swapped in partial contents containing {} FlowFiles ({} bytes) from {}", swapContents.getFlowFiles().size(), contentSizeSwappedIn, swapLocation);
|
||||
} else {
|
||||
// we swapped in the whole swap file. We can just use the info that we got from the summary.
|
||||
incrementActiveQueueSize(flowFileCount, contentSize);
|
||||
logger.debug("Successfully swapped in Swap File {}", swapLocation);
|
||||
}
|
||||
|
||||
activeQueue.addAll(swapContents.getFlowFiles());
|
||||
}
|
||||
|
||||
public QueueSize size() {
|
||||
return getFlowFileQueueSize().toQueueSize();
|
||||
}
|
||||
|
||||
public boolean isEmpty() {
|
||||
return getFlowFileQueueSize().isEmpty();
|
||||
}
|
||||
|
||||
public boolean isActiveQueueEmpty() {
|
||||
final FlowFileQueueSize queueSize = getFlowFileQueueSize();
|
||||
return queueSize.getActiveCount() == 0 && queueSize.getSwappedCount() == 0;
|
||||
}
|
||||
|
||||
public void acknowledge(final FlowFileRecord flowFile) {
|
||||
logger.debug("{} Acknowledging {}", this, flowFile);
|
||||
incrementUnacknowledgedQueueSize(-1, -flowFile.getSize());
|
||||
}
|
||||
|
||||
public void acknowledge(final Collection<FlowFileRecord> flowFiles) {
|
||||
logger.debug("{} Acknowledging {}", this, flowFiles);
|
||||
final long totalSize = flowFiles.stream().mapToLong(FlowFileRecord::getSize).sum();
|
||||
incrementUnacknowledgedQueueSize(-flowFiles.size(), -totalSize);
|
||||
}
|
||||
|
||||
|
||||
public void put(final FlowFileRecord flowFile) {
|
||||
writeLock.lock();
|
||||
try {
|
||||
if (swapMode || activeQueue.size() >= swapThreshold) {
|
||||
swapQueue.add(flowFile);
|
||||
incrementSwapQueueSize(1, flowFile.getSize(), 0);
|
||||
swapMode = true;
|
||||
writeSwapFilesIfNecessary();
|
||||
} else {
|
||||
incrementActiveQueueSize(1, flowFile.getSize());
|
||||
activeQueue.add(flowFile);
|
||||
}
|
||||
|
||||
logger.debug("{} put to {}", flowFile, this);
|
||||
} finally {
|
||||
writeLock.unlock("put(FlowFileRecord)");
|
||||
}
|
||||
}
|
||||
|
||||
public void putAll(final Collection<FlowFileRecord> flowFiles) {
|
||||
final int numFiles = flowFiles.size();
|
||||
long bytes = 0L;
|
||||
for (final FlowFile flowFile : flowFiles) {
|
||||
bytes += flowFile.getSize();
|
||||
}
|
||||
|
||||
writeLock.lock();
|
||||
try {
|
||||
if (swapMode || activeQueue.size() >= swapThreshold - numFiles) {
|
||||
swapQueue.addAll(flowFiles);
|
||||
incrementSwapQueueSize(numFiles, bytes, 0);
|
||||
swapMode = true;
|
||||
writeSwapFilesIfNecessary();
|
||||
} else {
|
||||
incrementActiveQueueSize(numFiles, bytes);
|
||||
activeQueue.addAll(flowFiles);
|
||||
}
|
||||
|
||||
logger.debug("{} put to {}", flowFiles, this);
|
||||
} finally {
|
||||
writeLock.unlock("putAll");
|
||||
}
|
||||
}
|
||||
|
||||
public FlowFileRecord poll(final Set<FlowFileRecord> expiredRecords, final long expirationMillis) {
|
||||
FlowFileRecord flowFile;
|
||||
|
||||
// First check if we have any records Pre-Fetched.
|
||||
writeLock.lock();
|
||||
try {
|
||||
flowFile = doPoll(expiredRecords, expirationMillis);
|
||||
|
||||
if (flowFile != null) {
|
||||
logger.debug("{} poll() returning {}", this, flowFile);
|
||||
incrementUnacknowledgedQueueSize(1, flowFile.getSize());
|
||||
}
|
||||
|
||||
return flowFile;
|
||||
} finally {
|
||||
writeLock.unlock("poll(Set)");
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
private FlowFileRecord doPoll(final Set<FlowFileRecord> expiredRecords, final long expirationMillis) {
|
||||
FlowFileRecord flowFile;
|
||||
boolean isExpired;
|
||||
|
||||
migrateSwapToActive();
|
||||
|
||||
long expiredBytes = 0L;
|
||||
do {
|
||||
flowFile = this.activeQueue.poll();
|
||||
|
||||
isExpired = isExpired(flowFile, expirationMillis);
|
||||
if (isExpired) {
|
||||
expiredRecords.add(flowFile);
|
||||
expiredBytes += flowFile.getSize();
|
||||
flowFile = null;
|
||||
|
||||
if (expiredRecords.size() >= MAX_EXPIRED_RECORDS_PER_ITERATION) {
|
||||
break;
|
||||
}
|
||||
} else if (flowFile != null && flowFile.isPenalized()) {
|
||||
this.activeQueue.add(flowFile);
|
||||
flowFile = null;
|
||||
break;
|
||||
}
|
||||
|
||||
if (flowFile != null) {
|
||||
incrementActiveQueueSize(-1, -flowFile.getSize());
|
||||
}
|
||||
} while (isExpired);
|
||||
|
||||
if (!expiredRecords.isEmpty()) {
|
||||
incrementActiveQueueSize(-expiredRecords.size(), -expiredBytes);
|
||||
}
|
||||
|
||||
return flowFile;
|
||||
}
|
||||
|
||||
public List<FlowFileRecord> poll(int maxResults, final Set<FlowFileRecord> expiredRecords, final long expirationMillis) {
|
||||
final List<FlowFileRecord> records = new ArrayList<>(Math.min(1, maxResults));
|
||||
|
||||
// First check if we have any records Pre-Fetched.
|
||||
writeLock.lock();
|
||||
try {
|
||||
doPoll(records, maxResults, expiredRecords, expirationMillis);
|
||||
} finally {
|
||||
writeLock.unlock("poll(int, Set)");
|
||||
}
|
||||
|
||||
if (!records.isEmpty()) {
|
||||
logger.debug("{} poll() returning {}", this, records);
|
||||
}
|
||||
|
||||
return records;
|
||||
}
|
||||
|
||||
public List<FlowFileRecord> poll(final FlowFileFilter filter, final Set<FlowFileRecord> expiredRecords, final long expirationMillis) {
|
||||
long bytesPulled = 0L;
|
||||
int flowFilesPulled = 0;
|
||||
|
||||
writeLock.lock();
|
||||
try {
|
||||
migrateSwapToActive();
|
||||
|
||||
final List<FlowFileRecord> selectedFlowFiles = new ArrayList<>();
|
||||
final List<FlowFileRecord> unselected = new ArrayList<>();
|
||||
|
||||
while (true) {
|
||||
FlowFileRecord flowFile = this.activeQueue.poll();
|
||||
if (flowFile == null) {
|
||||
break;
|
||||
}
|
||||
|
||||
final boolean isExpired = isExpired(flowFile, expirationMillis);
|
||||
if (isExpired) {
|
||||
expiredRecords.add(flowFile);
|
||||
bytesPulled += flowFile.getSize();
|
||||
flowFilesPulled++;
|
||||
|
||||
if (expiredRecords.size() >= MAX_EXPIRED_RECORDS_PER_ITERATION) {
|
||||
break;
|
||||
} else {
|
||||
continue;
|
||||
}
|
||||
} else if (flowFile.isPenalized()) {
|
||||
this.activeQueue.add(flowFile);
|
||||
break; // just stop searching because the rest are all penalized.
|
||||
}
|
||||
|
||||
final FlowFileFilterResult result = filter.filter(flowFile);
|
||||
if (result.isAccept()) {
|
||||
bytesPulled += flowFile.getSize();
|
||||
flowFilesPulled++;
|
||||
|
||||
incrementUnacknowledgedQueueSize(1, flowFile.getSize());
|
||||
selectedFlowFiles.add(flowFile);
|
||||
} else {
|
||||
unselected.add(flowFile);
|
||||
}
|
||||
|
||||
if (!result.isContinue()) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
this.activeQueue.addAll(unselected);
|
||||
incrementActiveQueueSize(-flowFilesPulled, -bytesPulled);
|
||||
|
||||
if (!selectedFlowFiles.isEmpty()) {
|
||||
logger.debug("{} poll() returning {}", this, selectedFlowFiles);
|
||||
}
|
||||
|
||||
return selectedFlowFiles;
|
||||
} finally {
|
||||
writeLock.unlock("poll(Filter, Set)");
|
||||
}
|
||||
}
|
||||
|
||||
private void doPoll(final List<FlowFileRecord> records, int maxResults, final Set<FlowFileRecord> expiredRecords, final long expirationMillis) {
|
||||
migrateSwapToActive();
|
||||
|
||||
final long bytesDrained = drainQueue(activeQueue, records, maxResults, expiredRecords, expirationMillis);
|
||||
|
||||
long expiredBytes = 0L;
|
||||
for (final FlowFileRecord record : expiredRecords) {
|
||||
expiredBytes += record.getSize();
|
||||
}
|
||||
|
||||
incrementActiveQueueSize(-(expiredRecords.size() + records.size()), -bytesDrained);
|
||||
incrementUnacknowledgedQueueSize(records.size(), bytesDrained - expiredBytes);
|
||||
}
|
||||
|
||||
|
||||
protected boolean isExpired(final FlowFile flowFile, final long expirationMillis) {
|
||||
return isLaterThan(getExpirationDate(flowFile, expirationMillis));
|
||||
}
|
||||
|
||||
private boolean isLaterThan(final Long maxAge) {
|
||||
if (maxAge == null) {
|
||||
return false;
|
||||
}
|
||||
return maxAge < System.currentTimeMillis();
|
||||
}
|
||||
|
||||
private Long getExpirationDate(final FlowFile flowFile, final long expirationMillis) {
|
||||
if (flowFile == null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
if (expirationMillis <= 0) {
|
||||
return null;
|
||||
} else {
|
||||
final long entryDate = flowFile.getEntryDate();
|
||||
final long expirationDate = entryDate + expirationMillis;
|
||||
return expirationDate;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
private long drainQueue(final Queue<FlowFileRecord> sourceQueue, final List<FlowFileRecord> destination, int maxResults, final Set<FlowFileRecord> expiredRecords, final long expirationMillis) {
|
||||
long drainedSize = 0L;
|
||||
FlowFileRecord pulled;
|
||||
|
||||
while (destination.size() < maxResults && (pulled = sourceQueue.poll()) != null) {
|
||||
if (isExpired(pulled, expirationMillis)) {
|
||||
expiredRecords.add(pulled);
|
||||
if (expiredRecords.size() >= MAX_EXPIRED_RECORDS_PER_ITERATION) {
|
||||
break;
|
||||
}
|
||||
} else {
|
||||
if (pulled.isPenalized()) {
|
||||
sourceQueue.add(pulled);
|
||||
break;
|
||||
}
|
||||
destination.add(pulled);
|
||||
}
|
||||
drainedSize += pulled.getSize();
|
||||
}
|
||||
return drainedSize;
|
||||
}
|
||||
|
||||
|
||||
public FlowFileRecord getFlowFile(final String flowFileUuid) {
|
||||
if (flowFileUuid == null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
readLock.lock();
|
||||
try {
|
||||
// read through all of the FlowFiles in the queue, looking for the FlowFile with the given ID
|
||||
for (final FlowFileRecord flowFile : activeQueue) {
|
||||
if (flowFileUuid.equals(flowFile.getAttribute(CoreAttributes.UUID.key()))) {
|
||||
return flowFile;
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
readLock.unlock("getFlowFile");
|
||||
}
|
||||
|
||||
return null;
|
||||
}
|
||||
|
||||
|
||||
public void dropFlowFiles(final DropFlowFileRequest dropRequest, final String requestor) {
|
||||
final String requestIdentifier = dropRequest.getRequestIdentifier();
|
||||
|
||||
writeLock.lock();
|
||||
try {
|
||||
dropRequest.setState(DropFlowFileState.DROPPING_FLOWFILES);
|
||||
logger.debug("For DropFlowFileRequest {}, original size is {}", requestIdentifier, size());
|
||||
|
||||
try {
|
||||
final List<FlowFileRecord> activeQueueRecords = new ArrayList<>(activeQueue);
|
||||
|
||||
QueueSize droppedSize;
|
||||
try {
|
||||
if (dropRequest.getState() == DropFlowFileState.CANCELED) {
|
||||
logger.info("Cancel requested for DropFlowFileRequest {}", requestIdentifier);
|
||||
return;
|
||||
}
|
||||
|
||||
droppedSize = dropAction.drop(activeQueueRecords, requestor);
|
||||
logger.debug("For DropFlowFileRequest {}, Dropped {} from active queue", requestIdentifier, droppedSize);
|
||||
} catch (final IOException ioe) {
|
||||
logger.error("Failed to drop the FlowFiles from queue {} due to {}", getQueueIdentifier(), ioe.toString());
|
||||
logger.error("", ioe);
|
||||
|
||||
dropRequest.setState(DropFlowFileState.FAILURE, "Failed to drop FlowFiles due to " + ioe.toString());
|
||||
return;
|
||||
}
|
||||
|
||||
activeQueue.clear();
|
||||
incrementActiveQueueSize(-droppedSize.getObjectCount(), -droppedSize.getByteCount());
|
||||
dropRequest.setCurrentSize(size());
|
||||
dropRequest.setDroppedSize(dropRequest.getDroppedSize().add(droppedSize));
|
||||
|
||||
final QueueSize swapSize = getFlowFileQueueSize().swapQueueSize();
|
||||
logger.debug("For DropFlowFileRequest {}, Swap Queue has {} elements, Swapped Record Count = {}, Swapped Content Size = {}",
|
||||
requestIdentifier, swapQueue.size(), swapSize.getObjectCount(), swapSize.getByteCount());
|
||||
if (dropRequest.getState() == DropFlowFileState.CANCELED) {
|
||||
logger.info("Cancel requested for DropFlowFileRequest {}", requestIdentifier);
|
||||
return;
|
||||
}
|
||||
|
||||
try {
|
||||
droppedSize = dropAction.drop(swapQueue, requestor);
|
||||
} catch (final IOException ioe) {
|
||||
logger.error("Failed to drop the FlowFiles from queue {} due to {}", getQueueIdentifier(), ioe.toString());
|
||||
logger.error("", ioe);
|
||||
|
||||
dropRequest.setState(DropFlowFileState.FAILURE, "Failed to drop FlowFiles due to " + ioe.toString());
|
||||
return;
|
||||
}
|
||||
|
||||
swapQueue.clear();
|
||||
dropRequest.setCurrentSize(size());
|
||||
dropRequest.setDroppedSize(dropRequest.getDroppedSize().add(droppedSize));
|
||||
swapMode = false;
|
||||
incrementSwapQueueSize(-droppedSize.getObjectCount(), -droppedSize.getByteCount(), 0);
|
||||
logger.debug("For DropFlowFileRequest {}, dropped {} from Swap Queue", requestIdentifier, droppedSize);
|
||||
|
||||
final int swapFileCount = swapLocations.size();
|
||||
final Iterator<String> swapLocationItr = swapLocations.iterator();
|
||||
while (swapLocationItr.hasNext()) {
|
||||
final String swapLocation = swapLocationItr.next();
|
||||
|
||||
SwapContents swapContents = null;
|
||||
try {
|
||||
if (dropRequest.getState() == DropFlowFileState.CANCELED) {
|
||||
logger.info("Cancel requested for DropFlowFileRequest {}", requestIdentifier);
|
||||
return;
|
||||
}
|
||||
|
||||
swapContents = swapManager.swapIn(swapLocation, flowFileQueue);
|
||||
droppedSize = dropAction.drop(swapContents.getFlowFiles(), requestor);
|
||||
} catch (final IncompleteSwapFileException isfe) {
|
||||
swapContents = isfe.getPartialContents();
|
||||
final String warnMsg = "Failed to swap in FlowFiles from Swap File " + swapLocation + " because the file was corrupt. "
|
||||
+ "Some FlowFiles may not be dropped from the queue until NiFi is restarted.";
|
||||
|
||||
logger.warn(warnMsg);
|
||||
if (eventReporter != null) {
|
||||
eventReporter.reportEvent(Severity.WARNING, "Drop FlowFiles", warnMsg);
|
||||
}
|
||||
} catch (final IOException ioe) {
|
||||
logger.error("Failed to swap in FlowFiles from Swap File {} in order to drop the FlowFiles for Connection {} due to {}",
|
||||
swapLocation, getQueueIdentifier(), ioe.toString());
|
||||
logger.error("", ioe);
|
||||
if (eventReporter != null) {
|
||||
eventReporter.reportEvent(Severity.ERROR, "Drop FlowFiles", "Failed to swap in FlowFiles from Swap File " + swapLocation
|
||||
+ ". The FlowFiles contained in this Swap File will not be dropped from the queue");
|
||||
}
|
||||
|
||||
dropRequest.setState(DropFlowFileState.FAILURE, "Failed to swap in FlowFiles from Swap File " + swapLocation + " due to " + ioe.toString());
|
||||
if (swapContents != null) {
|
||||
activeQueue.addAll(swapContents.getFlowFiles()); // ensure that we don't lose the FlowFiles from our queue.
|
||||
}
|
||||
|
||||
return;
|
||||
}
|
||||
|
||||
dropRequest.setDroppedSize(dropRequest.getDroppedSize().add(droppedSize));
|
||||
incrementSwapQueueSize(-droppedSize.getObjectCount(), -droppedSize.getByteCount(), -1);
|
||||
|
||||
dropRequest.setCurrentSize(size());
|
||||
swapLocationItr.remove();
|
||||
logger.debug("For DropFlowFileRequest {}, dropped {} for Swap File {}", requestIdentifier, droppedSize, swapLocation);
|
||||
}
|
||||
|
||||
logger.debug("Dropped FlowFiles from {} Swap Files", swapFileCount);
|
||||
logger.info("Successfully dropped {} FlowFiles ({} bytes) from Connection with ID {} on behalf of {}",
|
||||
dropRequest.getDroppedSize().getObjectCount(), dropRequest.getDroppedSize().getByteCount(), getQueueIdentifier(), requestor);
|
||||
dropRequest.setState(DropFlowFileState.COMPLETE);
|
||||
} catch (final Exception e) {
|
||||
logger.error("Failed to drop FlowFiles from Connection with ID {} due to {}", getQueueIdentifier(), e.toString());
|
||||
logger.error("", e);
|
||||
dropRequest.setState(DropFlowFileState.FAILURE, "Failed to drop FlowFiles due to " + e.toString());
|
||||
}
|
||||
} finally {
|
||||
writeLock.unlock("Drop FlowFiles");
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
||||
public SwapSummary recoverSwappedFlowFiles() {
|
||||
int swapFlowFileCount = 0;
|
||||
long swapByteCount = 0L;
|
||||
Long maxId = null;
|
||||
List<ResourceClaim> resourceClaims = new ArrayList<>();
|
||||
final long startNanos = System.nanoTime();
|
||||
|
||||
writeLock.lock();
|
||||
try {
|
||||
final List<String> swapLocations;
|
||||
try {
|
||||
swapLocations = swapManager.recoverSwapLocations(flowFileQueue, swapPartitionName);
|
||||
} catch (final IOException ioe) {
|
||||
logger.error("Failed to determine whether or not any Swap Files exist for FlowFile Queue {}", getQueueIdentifier());
|
||||
logger.error("", ioe);
|
||||
if (eventReporter != null) {
|
||||
eventReporter.reportEvent(Severity.ERROR, "FlowFile Swapping", "Failed to determine whether or not any Swap Files exist for FlowFile Queue " +
|
||||
getQueueIdentifier() + "; see logs for more detials");
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
logger.debug("Recovered {} Swap Files for {}: {}", swapLocations.size(), flowFileQueue, swapLocations);
|
||||
for (final String swapLocation : swapLocations) {
|
||||
try {
|
||||
final SwapSummary summary = swapManager.getSwapSummary(swapLocation);
|
||||
final QueueSize queueSize = summary.getQueueSize();
|
||||
final Long maxSwapRecordId = summary.getMaxFlowFileId();
|
||||
if (maxSwapRecordId != null) {
|
||||
if (maxId == null || maxSwapRecordId > maxId) {
|
||||
maxId = maxSwapRecordId;
|
||||
}
|
||||
}
|
||||
|
||||
swapFlowFileCount += queueSize.getObjectCount();
|
||||
swapByteCount += queueSize.getByteCount();
|
||||
resourceClaims.addAll(summary.getResourceClaims());
|
||||
} catch (final IOException ioe) {
|
||||
logger.error("Failed to recover FlowFiles from Swap File {}; the file appears to be corrupt", swapLocation, ioe.toString());
|
||||
logger.error("", ioe);
|
||||
if (eventReporter != null) {
|
||||
eventReporter.reportEvent(Severity.ERROR, "FlowFile Swapping", "Failed to recover FlowFiles from Swap File " + swapLocation +
|
||||
"; the file appears to be corrupt. See logs for more details");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
incrementSwapQueueSize(swapFlowFileCount, swapByteCount, swapLocations.size());
|
||||
this.swapLocations.addAll(swapLocations);
|
||||
} finally {
|
||||
writeLock.unlock("Recover Swap Files");
|
||||
}
|
||||
|
||||
if (!swapLocations.isEmpty()) {
|
||||
final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNanos);
|
||||
logger.info("Recovered {} swap files for {} in {} millis", swapLocations.size(), this, millis);
|
||||
}
|
||||
|
||||
return new StandardSwapSummary(new QueueSize(swapFlowFileCount, swapByteCount), maxId, resourceClaims);
|
||||
}
|
||||
|
||||
|
||||
|
||||
protected void incrementActiveQueueSize(final int count, final long bytes) {
|
||||
boolean updated = false;
|
||||
while (!updated) {
|
||||
final FlowFileQueueSize original = size.get();
|
||||
final FlowFileQueueSize newSize = new FlowFileQueueSize(
|
||||
original.getActiveCount() + count, original.getActiveBytes() + bytes,
|
||||
original.getSwappedCount(), original.getSwappedBytes(), original.getSwapFileCount(),
|
||||
original.getUnacknowledgedCount(), original.getUnacknowledgedBytes());
|
||||
|
||||
updated = size.compareAndSet(original, newSize);
|
||||
|
||||
if (updated) {
|
||||
logIfNegative(original, newSize, "active");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void incrementSwapQueueSize(final int count, final long bytes, final int fileCount) {
|
||||
boolean updated = false;
|
||||
while (!updated) {
|
||||
final FlowFileQueueSize original = getFlowFileQueueSize();
|
||||
final FlowFileQueueSize newSize = new FlowFileQueueSize(original.getActiveCount(), original.getActiveBytes(),
|
||||
original.getSwappedCount() + count, original.getSwappedBytes() + bytes, original.getSwapFileCount() + fileCount,
|
||||
original.getUnacknowledgedCount(), original.getUnacknowledgedBytes());
|
||||
|
||||
updated = updateSize(original, newSize);
|
||||
if (updated) {
|
||||
logIfNegative(original, newSize, "swap");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void incrementUnacknowledgedQueueSize(final int count, final long bytes) {
|
||||
boolean updated = false;
|
||||
while (!updated) {
|
||||
final FlowFileQueueSize original = size.get();
|
||||
final FlowFileQueueSize newSize = new FlowFileQueueSize(original.getActiveCount(), original.getActiveBytes(),
|
||||
original.getSwappedCount(), original.getSwappedBytes(), original.getSwapFileCount(),
|
||||
original.getUnacknowledgedCount() + count, original.getUnacknowledgedBytes() + bytes);
|
||||
updated = size.compareAndSet(original, newSize);
|
||||
|
||||
if (updated) {
|
||||
logIfNegative(original, newSize, "Unacknowledged");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void logIfNegative(final FlowFileQueueSize original, final FlowFileQueueSize newSize, final String counterName) {
|
||||
if (newSize.getActiveBytes() < 0 || newSize.getActiveCount() < 0
|
||||
|| newSize.getSwappedBytes() < 0 || newSize.getSwappedCount() < 0
|
||||
|| newSize.getUnacknowledgedBytes() < 0 || newSize.getUnacknowledgedCount() < 0) {
|
||||
|
||||
logger.error("Updated Size of Queue " + counterName + " from " + original + " to " + newSize, new RuntimeException("Cannot create negative queue size"));
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
protected boolean updateSize(final FlowFileQueueSize expected, final FlowFileQueueSize updated) {
|
||||
return size.compareAndSet(expected, updated);
|
||||
}
|
||||
|
||||
public FlowFileQueueSize getFlowFileQueueSize() {
|
||||
return size.get();
|
||||
}
|
||||
|
||||
public void inheritQueueContents(final FlowFileQueueContents queueContents) {
|
||||
writeLock.lock();
|
||||
try {
|
||||
putAll(queueContents.getActiveFlowFiles());
|
||||
swapLocations.addAll(queueContents.getSwapLocations());
|
||||
incrementSwapQueueSize(queueContents.getSwapSize().getObjectCount(), queueContents.getSwapSize().getByteCount(), queueContents.getSwapLocations().size());
|
||||
} finally {
|
||||
writeLock.unlock("inheritQueueContents");
|
||||
}
|
||||
}
|
||||
|
||||
public FlowFileQueueContents packageForRebalance(final String newPartitionName) {
|
||||
writeLock.lock();
|
||||
try {
|
||||
final List<FlowFileRecord> activeRecords = new ArrayList<>(this.activeQueue);
|
||||
activeRecords.addAll(this.swapQueue);
|
||||
|
||||
final List<String> updatedSwapLocations = new ArrayList<>(swapLocations.size());
|
||||
for (final String swapLocation : swapLocations) {
|
||||
try {
|
||||
final String updatedSwapLocation = swapManager.changePartitionName(swapLocation, newPartitionName);
|
||||
updatedSwapLocations.add(updatedSwapLocation);
|
||||
} catch (final IOException ioe) {
|
||||
logger.error("Failed to update Swap File {} to reflect that the contents are now owned by Partition '{}'", swapLocation, newPartitionName, ioe);
|
||||
}
|
||||
}
|
||||
|
||||
this.swapLocations.clear();
|
||||
this.activeQueue.clear();
|
||||
this.swapQueue.clear();
|
||||
|
||||
this.swapMode = false;
|
||||
|
||||
QueueSize swapSize = new QueueSize(0, 0L);
|
||||
boolean updated = false;
|
||||
while (!updated) {
|
||||
final FlowFileQueueSize currentSize = getFlowFileQueueSize();
|
||||
swapSize = new QueueSize(currentSize.getSwappedCount(), currentSize.getSwappedBytes());
|
||||
|
||||
final FlowFileQueueSize updatedSize = new FlowFileQueueSize(0, 0, 0, 0, 0, currentSize.getUnacknowledgedCount(), currentSize.getUnacknowledgedBytes());
|
||||
updated = updateSize(currentSize, updatedSize);
|
||||
}
|
||||
|
||||
return new FlowFileQueueContents(activeRecords, updatedSwapLocations, swapSize);
|
||||
} finally {
|
||||
writeLock.unlock("transfer(SwappablePriorityQueue)");
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "SwappablePriorityQueue[queueId=" + flowFileQueue.getIdentifier() + "]";
|
||||
}
|
||||
}
|
|
@ -0,0 +1,41 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.nifi.controller.queue;
|
||||
|
||||
public class TimePeriod {
|
||||
private final String period;
|
||||
private final long millis;
|
||||
|
||||
public TimePeriod(final String period, final long millis) {
|
||||
this.period = period;
|
||||
this.millis = millis;
|
||||
}
|
||||
|
||||
public String getPeriod() {
|
||||
return period;
|
||||
}
|
||||
|
||||
public long getMillis() {
|
||||
return millis;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return period;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,91 @@
|
|||
/*
|
||||
* 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.controller.queue.clustered;
|
||||
|
||||
import org.apache.nifi.controller.repository.ContentNotFoundException;
|
||||
import org.apache.nifi.controller.repository.ContentRepository;
|
||||
import org.apache.nifi.controller.repository.FlowFileRecord;
|
||||
import org.apache.nifi.controller.repository.io.LimitedInputStream;
|
||||
import org.apache.nifi.stream.io.StreamUtils;
|
||||
|
||||
import java.io.EOFException;
|
||||
import java.io.FilterInputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
|
||||
public class ContentRepositoryFlowFileAccess implements FlowFileContentAccess {
|
||||
private final ContentRepository contentRepository;
|
||||
|
||||
public ContentRepositoryFlowFileAccess(final ContentRepository contentRepository) {
|
||||
this.contentRepository = contentRepository;
|
||||
}
|
||||
|
||||
@Override
|
||||
public InputStream read(final FlowFileRecord flowFile) throws IOException {
|
||||
final InputStream rawIn;
|
||||
try {
|
||||
rawIn = contentRepository.read(flowFile.getContentClaim());
|
||||
} catch (final ContentNotFoundException cnfe) {
|
||||
throw new ContentNotFoundException(flowFile, flowFile.getContentClaim(), cnfe.getMessage());
|
||||
}
|
||||
|
||||
if (flowFile.getContentClaimOffset() > 0) {
|
||||
try {
|
||||
StreamUtils.skip(rawIn, flowFile.getContentClaimOffset());
|
||||
} catch (final EOFException eof) {
|
||||
throw new ContentNotFoundException(flowFile, flowFile.getContentClaim(), "FlowFile has a Content Claim Offset of "
|
||||
+ flowFile.getContentClaimOffset() + " bytes but the Content Claim does not have that many bytes");
|
||||
}
|
||||
}
|
||||
|
||||
final InputStream limitedIn = new LimitedInputStream(rawIn, flowFile.getSize());
|
||||
// Wrap the Content Repository's InputStream with one that ensures that we are able to consume all of the FlowFile's content or else throws EOFException
|
||||
return new FilterInputStream(limitedIn) {
|
||||
private long bytesRead = 0;
|
||||
|
||||
@Override
|
||||
public int read(final byte[] b, final int off, final int len) throws IOException {
|
||||
return ensureNotTruncated(limitedIn.read(b, off, len));
|
||||
}
|
||||
|
||||
@Override
|
||||
public int read(final byte[] b) throws IOException {
|
||||
return ensureNotTruncated(limitedIn.read(b));
|
||||
}
|
||||
|
||||
@Override
|
||||
public int read() throws IOException {
|
||||
return ensureNotTruncated(limitedIn.read());
|
||||
}
|
||||
|
||||
private int ensureNotTruncated(final int length) throws EOFException {
|
||||
if (length > -1) {
|
||||
bytesRead += length;
|
||||
return length;
|
||||
}
|
||||
|
||||
if (bytesRead < flowFile.getSize()) {
|
||||
throw new EOFException("Expected " + flowFile + " to contain " + flowFile.getSize() + " bytes but the content repository only had " + bytesRead + " bytes for it");
|
||||
}
|
||||
|
||||
return length;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,29 @@
|
|||
/*
|
||||
* 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.controller.queue.clustered;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
|
||||
import org.apache.nifi.controller.repository.FlowFileRecord;
|
||||
|
||||
public interface FlowFileContentAccess {
|
||||
|
||||
InputStream read(FlowFileRecord flowFile) throws IOException;
|
||||
|
||||
}
|
|
@ -0,0 +1,42 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.nifi.controller.queue.clustered;
|
||||
|
||||
public class SimpleLimitThreshold implements TransactionThreshold {
|
||||
private final int countLimit;
|
||||
private final long byteLimit;
|
||||
|
||||
private int count = 0;
|
||||
private long bytes = 0L;
|
||||
|
||||
public SimpleLimitThreshold(final int count, final long bytes) {
|
||||
this.countLimit = count;
|
||||
this.byteLimit = bytes;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void adjust(final int flowFileCount, final long flowFileSize) {
|
||||
count += flowFileCount;
|
||||
bytes += flowFileSize;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isThresholdMet() {
|
||||
return count >= countLimit || bytes >= byteLimit;
|
||||
}
|
||||
}
|
File diff suppressed because it is too large
Load Diff
|
@ -0,0 +1,26 @@
|
|||
/*
|
||||
* 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.controller.queue.clustered;
|
||||
|
||||
public interface TransactionThreshold {
|
||||
|
||||
void adjust(int flowFileCount, long flowFileSize);
|
||||
|
||||
boolean isThresholdMet();
|
||||
|
||||
}
|
|
@ -0,0 +1,51 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.nifi.controller.queue.clustered;
|
||||
|
||||
import org.apache.nifi.controller.queue.FlowFileQueueContents;
|
||||
import org.apache.nifi.controller.queue.clustered.partition.FlowFilePartitioner;
|
||||
import org.apache.nifi.controller.repository.FlowFileRecord;
|
||||
|
||||
import java.util.Collection;
|
||||
import java.util.function.Function;
|
||||
|
||||
public interface TransferFailureDestination {
|
||||
/**
|
||||
* Puts all of the given FlowFiles to the appropriate destination queue
|
||||
*
|
||||
* @param flowFiles the FlowFiles to transfer
|
||||
* @param partitionerUsed the partitioner that was used to determine that the given FlowFiles should be grouped together in the first place
|
||||
*/
|
||||
void putAll(Collection<FlowFileRecord> flowFiles, FlowFilePartitioner partitionerUsed);
|
||||
|
||||
/**
|
||||
* Puts all of the given FlowFile Queue Contents to the appropriate destination queue
|
||||
*
|
||||
* @param queueContents a function that returns the FlowFileQueueContents, given a Partition Name
|
||||
* @param partitionerUsed the partitioner that was used to determine that the given FlowFiles should be grouped together in the first place
|
||||
*/
|
||||
void putAll(Function<String, FlowFileQueueContents> queueContents, FlowFilePartitioner partitionerUsed);
|
||||
|
||||
/**
|
||||
* Indicates whether or not FlowFiles will need to be rebalanced when transferred to the destination.
|
||||
*
|
||||
* @param partitionerUsed the partitioner that was used to determine that FlowFiles should be grouped together in the first place
|
||||
* @return <code>true</code> if FlowFiles will be rebalanced when transferred, <code>false</code> otherwise
|
||||
*/
|
||||
boolean isRebalanceOnFailure(FlowFilePartitioner partitionerUsed);
|
||||
}
|
|
@ -0,0 +1,27 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.nifi.controller.queue.clustered.client;
|
||||
|
||||
import org.apache.nifi.controller.repository.FlowFileRecord;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
|
||||
public interface LoadBalanceFlowFileCodec {
|
||||
void encode(FlowFileRecord flowFile, OutputStream out) throws IOException;
|
||||
}
|
|
@ -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.controller.queue.clustered.client;
|
||||
|
||||
import org.apache.nifi.controller.repository.FlowFileRecord;
|
||||
|
||||
import java.io.DataOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.Map;
|
||||
|
||||
public class StandardLoadBalanceFlowFileCodec implements LoadBalanceFlowFileCodec {
|
||||
|
||||
@Override
|
||||
public void encode(final FlowFileRecord flowFile, final OutputStream destination) throws IOException {
|
||||
final DataOutputStream out = new DataOutputStream(destination);
|
||||
|
||||
out.writeInt(flowFile.getAttributes().size());
|
||||
for (final Map.Entry<String, String> entry : flowFile.getAttributes().entrySet()) {
|
||||
writeString(entry.getKey(), out);
|
||||
writeString(entry.getValue(), out);
|
||||
}
|
||||
|
||||
out.writeLong(flowFile.getLineageStartDate());
|
||||
out.writeLong(flowFile.getEntryDate());
|
||||
}
|
||||
|
||||
private void writeString(final String value, final DataOutputStream out) throws IOException {
|
||||
final byte[] bytes = value.getBytes(StandardCharsets.UTF_8);
|
||||
out.writeInt(bytes.length);
|
||||
out.write(bytes);
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,49 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.nifi.controller.queue.clustered.client.async;
|
||||
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.controller.queue.LoadBalanceCompression;
|
||||
import org.apache.nifi.controller.repository.FlowFileRecord;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.function.BooleanSupplier;
|
||||
import java.util.function.Supplier;
|
||||
|
||||
public interface AsyncLoadBalanceClient {
|
||||
|
||||
NodeIdentifier getNodeIdentifier();
|
||||
|
||||
void start();
|
||||
|
||||
void stop();
|
||||
|
||||
void register(String connectionId, BooleanSupplier emptySupplier, Supplier<FlowFileRecord> flowFileSupplier,
|
||||
TransactionFailureCallback failureCallback, TransactionCompleteCallback successCallback,
|
||||
Supplier<LoadBalanceCompression> compressionSupplier, BooleanSupplier honorBackpressureSupplier);
|
||||
|
||||
void unregister(String connectionId);
|
||||
|
||||
boolean isRunning();
|
||||
|
||||
boolean isPenalized();
|
||||
|
||||
void nodeDisconnected();
|
||||
|
||||
boolean communicate() throws IOException;
|
||||
}
|
|
@ -0,0 +1,24 @@
|
|||
/*
|
||||
* 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.controller.queue.clustered.client.async;
|
||||
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
|
||||
public interface AsyncLoadBalanceClientFactory {
|
||||
AsyncLoadBalanceClient createClient(NodeIdentifier nodeIdentifier);
|
||||
}
|
|
@ -0,0 +1,32 @@
|
|||
/*
|
||||
* 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.controller.queue.clustered.client.async;
|
||||
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.controller.queue.LoadBalanceCompression;
|
||||
import org.apache.nifi.controller.repository.FlowFileRecord;
|
||||
|
||||
import java.util.function.BooleanSupplier;
|
||||
import java.util.function.Supplier;
|
||||
|
||||
public interface AsyncLoadBalanceClientRegistry {
|
||||
void register(String connectionId, NodeIdentifier nodeId, BooleanSupplier emptySupplier, Supplier<FlowFileRecord> flowFileSupplier, TransactionFailureCallback failureCallback,
|
||||
TransactionCompleteCallback successCallback, Supplier<LoadBalanceCompression> compressionSupplier, BooleanSupplier honorBackpressureSupplier);
|
||||
|
||||
void unregister(String connectionId, NodeIdentifier nodeId);
|
||||
}
|
|
@ -0,0 +1,26 @@
|
|||
/*
|
||||
* 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.controller.queue.clustered.client.async;
|
||||
|
||||
import org.apache.nifi.controller.repository.FlowFileRecord;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
public interface TransactionCompleteCallback {
|
||||
void onTransactionComplete(List<FlowFileRecord> flowFilesSent);
|
||||
}
|
|
@ -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.controller.queue.clustered.client.async;
|
||||
|
||||
import org.apache.nifi.controller.repository.FlowFileRecord;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
public interface TransactionFailureCallback {
|
||||
default void onTransactionFailed(final List<FlowFileRecord> flowFiles, final TransactionPhase transactionPhase) {
|
||||
onTransactionFailed(flowFiles, null, transactionPhase);
|
||||
}
|
||||
|
||||
void onTransactionFailed(List<FlowFileRecord> flowFiles, Exception cause, TransactionPhase transactionPhase);
|
||||
|
||||
boolean isRebalanceOnFailure();
|
||||
|
||||
enum TransactionPhase {
|
||||
/**
|
||||
* Failure occurred when connecting to the node
|
||||
*/
|
||||
CONNECTING,
|
||||
|
||||
/**
|
||||
* Failure occurred when sending data to the node
|
||||
*/
|
||||
SENDING;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,641 @@
|
|||
/*
|
||||
* 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.controller.queue.clustered.client.async.nio;
|
||||
|
||||
import org.apache.nifi.controller.queue.LoadBalanceCompression;
|
||||
import org.apache.nifi.controller.queue.clustered.FlowFileContentAccess;
|
||||
import org.apache.nifi.controller.queue.clustered.TransactionThreshold;
|
||||
import org.apache.nifi.controller.queue.clustered.client.LoadBalanceFlowFileCodec;
|
||||
import org.apache.nifi.controller.queue.clustered.protocol.LoadBalanceProtocolConstants;
|
||||
import org.apache.nifi.controller.queue.clustered.server.TransactionAbortedException;
|
||||
import org.apache.nifi.controller.repository.ContentNotFoundException;
|
||||
import org.apache.nifi.controller.repository.FlowFileRecord;
|
||||
import org.apache.nifi.remote.StandardVersionNegotiator;
|
||||
import org.apache.nifi.remote.VersionNegotiator;
|
||||
import org.apache.nifi.stream.io.ByteCountingOutputStream;
|
||||
import org.apache.nifi.stream.io.GZIPOutputStream;
|
||||
import org.apache.nifi.stream.io.StreamUtils;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.EOFException;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
import java.net.SocketTimeoutException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.channels.SelectionKey;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.OptionalInt;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.function.Supplier;
|
||||
import java.util.zip.CRC32;
|
||||
import java.util.zip.Checksum;
|
||||
|
||||
import static org.apache.nifi.controller.queue.clustered.protocol.LoadBalanceProtocolConstants.ABORT_PROTOCOL_NEGOTIATION;
|
||||
import static org.apache.nifi.controller.queue.clustered.protocol.LoadBalanceProtocolConstants.ABORT_TRANSACTION;
|
||||
import static org.apache.nifi.controller.queue.clustered.protocol.LoadBalanceProtocolConstants.CONFIRM_CHECKSUM;
|
||||
import static org.apache.nifi.controller.queue.clustered.protocol.LoadBalanceProtocolConstants.CONFIRM_COMPLETE_TRANSACTION;
|
||||
import static org.apache.nifi.controller.queue.clustered.protocol.LoadBalanceProtocolConstants.QUEUE_FULL;
|
||||
import static org.apache.nifi.controller.queue.clustered.protocol.LoadBalanceProtocolConstants.REJECT_CHECKSUM;
|
||||
import static org.apache.nifi.controller.queue.clustered.protocol.LoadBalanceProtocolConstants.REQEUST_DIFFERENT_VERSION;
|
||||
import static org.apache.nifi.controller.queue.clustered.protocol.LoadBalanceProtocolConstants.SPACE_AVAILABLE;
|
||||
import static org.apache.nifi.controller.queue.clustered.protocol.LoadBalanceProtocolConstants.VERSION_ACCEPTED;
|
||||
|
||||
|
||||
public class LoadBalanceSession {
|
||||
private static final Logger logger = LoggerFactory.getLogger(LoadBalanceSession.class);
|
||||
static final int MAX_DATA_FRAME_SIZE = 65535;
|
||||
private static final long PENALTY_MILLIS = TimeUnit.SECONDS.toMillis(2L);
|
||||
|
||||
private final RegisteredPartition partition;
|
||||
private final Supplier<FlowFileRecord> flowFileSupplier;
|
||||
private final FlowFileContentAccess flowFileContentAccess;
|
||||
private final LoadBalanceFlowFileCodec flowFileCodec;
|
||||
private final PeerChannel channel;
|
||||
private final int timeoutMillis;
|
||||
private final String peerDescription;
|
||||
private final String connectionId;
|
||||
private final TransactionThreshold transactionThreshold;
|
||||
|
||||
final VersionNegotiator negotiator = new StandardVersionNegotiator(1);
|
||||
private int protocolVersion = 1;
|
||||
|
||||
private final Checksum checksum = new CRC32();
|
||||
|
||||
// guarded by synchronizing on 'this'
|
||||
private ByteBuffer preparedFrame;
|
||||
private FlowFileRecord currentFlowFile;
|
||||
private List<FlowFileRecord> flowFilesSent = new ArrayList<>();
|
||||
private TransactionPhase phase = TransactionPhase.RECOMMEND_PROTOCOL_VERSION;
|
||||
private InputStream flowFileInputStream;
|
||||
private byte[] byteBuffer = new byte[MAX_DATA_FRAME_SIZE];
|
||||
private boolean complete = false;
|
||||
private long readTimeout;
|
||||
private long penaltyExpiration = -1L;
|
||||
|
||||
public LoadBalanceSession(final RegisteredPartition partition, final FlowFileContentAccess contentAccess, final LoadBalanceFlowFileCodec flowFileCodec, final PeerChannel peerChannel,
|
||||
final int timeoutMillis, final TransactionThreshold transactionThreshold) {
|
||||
this.partition = partition;
|
||||
this.flowFileSupplier = partition.getFlowFileRecordSupplier();
|
||||
this.connectionId = partition.getConnectionId();
|
||||
this.flowFileContentAccess = contentAccess;
|
||||
this.flowFileCodec = flowFileCodec;
|
||||
this.channel = peerChannel;
|
||||
this.peerDescription = peerChannel.getPeerDescription();
|
||||
|
||||
if (timeoutMillis < 1) {
|
||||
throw new IllegalArgumentException();
|
||||
}
|
||||
this.timeoutMillis = timeoutMillis;
|
||||
this.transactionThreshold = transactionThreshold;
|
||||
}
|
||||
|
||||
public RegisteredPartition getPartition() {
|
||||
return partition;
|
||||
}
|
||||
|
||||
public synchronized int getDesiredReadinessFlag() {
|
||||
return phase.getRequiredSelectionKey();
|
||||
}
|
||||
|
||||
public synchronized List<FlowFileRecord> getFlowFilesSent() {
|
||||
return Collections.unmodifiableList(flowFilesSent);
|
||||
}
|
||||
|
||||
public synchronized boolean isComplete() {
|
||||
return complete;
|
||||
}
|
||||
|
||||
public synchronized boolean communicate() throws IOException {
|
||||
if (isComplete()) {
|
||||
return false;
|
||||
}
|
||||
|
||||
if (isPenalized()) {
|
||||
logger.debug("Will not communicate with Peer {} for Connection {} because session is penalized", peerDescription, connectionId);
|
||||
return false;
|
||||
}
|
||||
|
||||
// If there's already a data frame prepared for writing, just write to the channel.
|
||||
if (preparedFrame != null && preparedFrame.hasRemaining()) {
|
||||
logger.trace("Current Frame is already available. Will continue writing current frame to channel");
|
||||
final int bytesWritten = channel.write(preparedFrame);
|
||||
return bytesWritten > 0;
|
||||
}
|
||||
|
||||
try {
|
||||
// Check if the phase is one that needs to receive data and if so, call the appropriate method.
|
||||
switch (phase) {
|
||||
case RECEIVE_SPACE_RESPONSE:
|
||||
return receiveSpaceAvailableResponse();
|
||||
case VERIFY_CHECKSUM:
|
||||
return verifyChecksum();
|
||||
case CONFIRM_TRANSACTION_COMPLETE:
|
||||
return confirmTransactionComplete();
|
||||
case RECEIVE_PROTOCOL_VERSION_ACKNOWLEDGMENT:
|
||||
return receiveProtocolVersionAcknowledgment();
|
||||
case RECEIVE_RECOMMENDED_PROTOCOL_VERSION:
|
||||
return receiveRecommendedProtocolVersion();
|
||||
}
|
||||
|
||||
// Otherwise, we need to send something so get the data frame that should be sent and write it to the channel
|
||||
final ByteBuffer byteBuffer = getDataFrame();
|
||||
preparedFrame = channel.prepareForWrite(byteBuffer); // Prepare data frame for writing. E.g., encrypt the data, etc.
|
||||
|
||||
final int bytesWritten = channel.write(preparedFrame);
|
||||
return bytesWritten > 0;
|
||||
} catch (final Exception e) {
|
||||
complete = true;
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
private boolean confirmTransactionComplete() throws IOException {
|
||||
logger.debug("Confirming Transaction Complete for Peer {}", peerDescription);
|
||||
|
||||
final OptionalInt transactionResponse = channel.read();
|
||||
if (!transactionResponse.isPresent()) {
|
||||
if (System.currentTimeMillis() > readTimeout) {
|
||||
throw new SocketTimeoutException("Timed out waiting for Peer " + peerDescription + " to confirm the transaction is complete");
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
final int response = transactionResponse.getAsInt();
|
||||
if (response < 0) {
|
||||
throw new EOFException("Confirmed checksum when writing data to Peer " + peerDescription + " but encountered End-of-File when expecting a Transaction Complete confirmation");
|
||||
}
|
||||
|
||||
if (response == ABORT_TRANSACTION) {
|
||||
throw new TransactionAbortedException("Confirmed checksum when writing data to Peer " + peerDescription + " but Peer aborted transaction instead of completing it");
|
||||
}
|
||||
if (response != CONFIRM_COMPLETE_TRANSACTION) {
|
||||
throw new IOException("Expected a CONFIRM_COMPLETE_TRANSACTION response from Peer " + peerDescription + " but received a value of " + response);
|
||||
}
|
||||
|
||||
complete = true;
|
||||
logger.debug("Successfully completed Transaction to send {} FlowFiles to Peer {} for Connection {}", flowFilesSent.size(), peerDescription, connectionId);
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
private boolean verifyChecksum() throws IOException {
|
||||
logger.debug("Verifying Checksum for Peer {}", peerDescription);
|
||||
|
||||
final OptionalInt checksumResponse = channel.read();
|
||||
if (!checksumResponse.isPresent()) {
|
||||
if (System.currentTimeMillis() > readTimeout) {
|
||||
throw new SocketTimeoutException("Timed out waiting for Peer " + peerDescription + " to verify the checksum");
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
final int response = checksumResponse.getAsInt();
|
||||
if (response < 0) {
|
||||
throw new EOFException("Encountered End-of-File when trying to verify Checksum with Peer " + peerDescription);
|
||||
}
|
||||
|
||||
if (response == REJECT_CHECKSUM) {
|
||||
throw new TransactionAbortedException("After transferring FlowFiles to Peer " + peerDescription + " received a REJECT_CHECKSUM response. Aborting transaction.");
|
||||
}
|
||||
if (response != CONFIRM_CHECKSUM) {
|
||||
throw new TransactionAbortedException("After transferring FlowFiles to Peer " + peerDescription + " received an unexpected response code " + response
|
||||
+ ". Aborting transaction.");
|
||||
}
|
||||
|
||||
logger.debug("Checksum confirmed. Writing COMPLETE_TRANSACTION flag");
|
||||
phase = TransactionPhase.SEND_TRANSACTION_COMPLETE;
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
|
||||
private ByteBuffer getDataFrame() throws IOException {
|
||||
switch (phase) {
|
||||
case RECOMMEND_PROTOCOL_VERSION:
|
||||
return recommendProtocolVersion();
|
||||
case ABORT_PROTOCOL_NEGOTIATION:
|
||||
return abortProtocolNegotiation();
|
||||
case SEND_CONNECTION_ID:
|
||||
return getConnectionId();
|
||||
case CHECK_SPACE:
|
||||
return checkSpace();
|
||||
case GET_NEXT_FLOWFILE:
|
||||
return getNextFlowFile();
|
||||
case SEND_FLOWFILE_DEFINITION:
|
||||
case SEND_FLOWFILE_CONTENTS:
|
||||
return getFlowFileContent();
|
||||
case SEND_CHECKSUM:
|
||||
return getChecksum();
|
||||
case SEND_TRANSACTION_COMPLETE:
|
||||
return getTransactionComplete();
|
||||
default:
|
||||
logger.debug("Phase of {}, returning null ByteBuffer", phase);
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
private ByteBuffer getTransactionComplete() {
|
||||
logger.debug("Sending Transaction Complete Indicator to Peer {}", peerDescription);
|
||||
|
||||
final ByteBuffer buffer = ByteBuffer.allocate(1);
|
||||
buffer.put((byte) LoadBalanceProtocolConstants.COMPLETE_TRANSACTION);
|
||||
buffer.rewind();
|
||||
|
||||
readTimeout = System.currentTimeMillis() + timeoutMillis;
|
||||
phase = TransactionPhase.CONFIRM_TRANSACTION_COMPLETE;
|
||||
return buffer;
|
||||
}
|
||||
|
||||
private ByteBuffer getChecksum() {
|
||||
logger.debug("Sending Checksum of {} to Peer {}", checksum.getValue(), peerDescription);
|
||||
|
||||
// No more FlowFiles.
|
||||
final ByteBuffer buffer = ByteBuffer.allocate(8);
|
||||
buffer.putLong(checksum.getValue());
|
||||
|
||||
readTimeout = System.currentTimeMillis() + timeoutMillis;
|
||||
phase = TransactionPhase.VERIFY_CHECKSUM;
|
||||
buffer.rewind();
|
||||
return buffer;
|
||||
}
|
||||
|
||||
private ByteBuffer getFlowFileContent() throws IOException {
|
||||
// This method is fairly inefficient, copying lots of byte[]. Can do better. But keeping it simple for
|
||||
// now to get this working. Revisit with optimizations later.
|
||||
try {
|
||||
if (flowFileInputStream == null) {
|
||||
flowFileInputStream = flowFileContentAccess.read(currentFlowFile);
|
||||
}
|
||||
|
||||
final int bytesRead = StreamUtils.fillBuffer(flowFileInputStream, byteBuffer, false);
|
||||
if (bytesRead < 1) {
|
||||
// If no data available, close the stream and move on to the next phase, returning a NO_DATA_FRAME buffer.
|
||||
flowFileInputStream.close();
|
||||
flowFileInputStream = null;
|
||||
phase = TransactionPhase.GET_NEXT_FLOWFILE;
|
||||
|
||||
final ByteBuffer buffer = ByteBuffer.allocate(1);
|
||||
buffer.put((byte) LoadBalanceProtocolConstants.NO_DATA_FRAME);
|
||||
buffer.rewind();
|
||||
|
||||
checksum.update(LoadBalanceProtocolConstants.NO_DATA_FRAME);
|
||||
|
||||
logger.debug("Sending NO_DATA_FRAME indicator to Peer {}", peerDescription);
|
||||
|
||||
return buffer;
|
||||
}
|
||||
|
||||
logger.trace("Sending Data Frame that is {} bytes long to Peer {}", bytesRead, peerDescription);
|
||||
final ByteBuffer buffer;
|
||||
|
||||
if (partition.getCompression() == LoadBalanceCompression.COMPRESS_ATTRIBUTES_AND_CONTENT) {
|
||||
final byte[] compressed = compressDataFrame(byteBuffer, bytesRead);
|
||||
final int compressedMaxLen = compressed.length;
|
||||
|
||||
buffer = ByteBuffer.allocate(3 + compressedMaxLen);
|
||||
buffer.put((byte) LoadBalanceProtocolConstants.DATA_FRAME_FOLLOWS);
|
||||
buffer.putShort((short) compressedMaxLen);
|
||||
|
||||
buffer.put(compressed, 0, compressedMaxLen);
|
||||
|
||||
} else {
|
||||
buffer = ByteBuffer.allocate(3 + bytesRead);
|
||||
buffer.put((byte) LoadBalanceProtocolConstants.DATA_FRAME_FOLLOWS);
|
||||
buffer.putShort((short) bytesRead);
|
||||
|
||||
buffer.put(byteBuffer, 0, bytesRead);
|
||||
}
|
||||
|
||||
final byte[] frameArray = buffer.array();
|
||||
checksum.update(frameArray, 0, frameArray.length);
|
||||
|
||||
phase = TransactionPhase.SEND_FLOWFILE_CONTENTS;
|
||||
buffer.rewind();
|
||||
return buffer;
|
||||
} catch (final ContentNotFoundException cnfe) {
|
||||
throw new ContentNotFoundException(currentFlowFile, cnfe.getMissingClaim(), cnfe.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
private byte[] compressDataFrame(final byte[] uncompressed, final int byteCount) throws IOException {
|
||||
try (final ByteArrayOutputStream baos = new ByteArrayOutputStream();
|
||||
final OutputStream gzipOut = new GZIPOutputStream(baos, 1)) {
|
||||
|
||||
gzipOut.write(uncompressed, 0, byteCount);
|
||||
gzipOut.close();
|
||||
|
||||
return baos.toByteArray();
|
||||
}
|
||||
}
|
||||
|
||||
private ByteBuffer getNextFlowFile() throws IOException {
|
||||
if (transactionThreshold.isThresholdMet()) {
|
||||
currentFlowFile = null;
|
||||
logger.debug("Transaction Threshold reached sending to Peer {}; Transitioning phase to SEND_CHECKSUM", peerDescription);
|
||||
} else {
|
||||
currentFlowFile = flowFileSupplier.get();
|
||||
|
||||
if (currentFlowFile == null) {
|
||||
logger.debug("No more FlowFiles to send to Peer {}; Transitioning phase to SEND_CHECKSUM", peerDescription);
|
||||
}
|
||||
}
|
||||
|
||||
if (currentFlowFile == null) {
|
||||
phase = TransactionPhase.SEND_CHECKSUM;
|
||||
return noMoreFlowFiles();
|
||||
}
|
||||
|
||||
transactionThreshold.adjust(1, currentFlowFile.getSize());
|
||||
logger.debug("Next FlowFile to send to Peer {} is {}", peerDescription, currentFlowFile);
|
||||
flowFilesSent.add(currentFlowFile);
|
||||
|
||||
final LoadBalanceCompression compression = partition.getCompression();
|
||||
final boolean compressAttributes = compression != LoadBalanceCompression.DO_NOT_COMPRESS;
|
||||
logger.debug("Compression to use for sending to Peer {} is {}", peerDescription, compression);
|
||||
|
||||
final byte[] flowFileEncoded;
|
||||
try (final ByteArrayOutputStream baos = new ByteArrayOutputStream()) {
|
||||
if (compressAttributes) {
|
||||
try (final OutputStream gzipOut = new GZIPOutputStream(baos, 1);
|
||||
final ByteCountingOutputStream out = new ByteCountingOutputStream(gzipOut)) {
|
||||
|
||||
flowFileCodec.encode(currentFlowFile, out);
|
||||
}
|
||||
} else {
|
||||
flowFileCodec.encode(currentFlowFile, baos);
|
||||
}
|
||||
|
||||
flowFileEncoded = baos.toByteArray();
|
||||
}
|
||||
|
||||
final int metadataLength = flowFileEncoded.length;
|
||||
final ByteBuffer buffer = ByteBuffer.allocate(flowFileEncoded.length + 5);
|
||||
buffer.put((byte) LoadBalanceProtocolConstants.MORE_FLOWFILES);
|
||||
checksum.update(LoadBalanceProtocolConstants.MORE_FLOWFILES);
|
||||
|
||||
buffer.putInt(metadataLength);
|
||||
checksum.update((metadataLength >> 24) & 0xFF);
|
||||
checksum.update((metadataLength >> 16) & 0xFF);
|
||||
checksum.update((metadataLength >> 8) & 0xFF);
|
||||
checksum.update(metadataLength & 0xFF);
|
||||
|
||||
buffer.put(flowFileEncoded);
|
||||
checksum.update(flowFileEncoded, 0, flowFileEncoded.length);
|
||||
|
||||
phase = TransactionPhase.SEND_FLOWFILE_DEFINITION;
|
||||
buffer.rewind();
|
||||
return buffer;
|
||||
}
|
||||
|
||||
|
||||
private ByteBuffer recommendProtocolVersion() {
|
||||
logger.debug("Recommending to Peer {} that Protocol Version {} be used", peerDescription, protocolVersion);
|
||||
|
||||
final ByteBuffer buffer = ByteBuffer.allocate(1);
|
||||
buffer.put((byte) protocolVersion);
|
||||
buffer.rewind();
|
||||
|
||||
readTimeout = System.currentTimeMillis() + timeoutMillis;
|
||||
phase = TransactionPhase.RECEIVE_PROTOCOL_VERSION_ACKNOWLEDGMENT;
|
||||
return buffer;
|
||||
}
|
||||
|
||||
private boolean receiveProtocolVersionAcknowledgment() throws IOException {
|
||||
logger.debug("Confirming Transaction Complete for Peer {}", peerDescription);
|
||||
|
||||
final OptionalInt ackResponse = channel.read();
|
||||
if (!ackResponse.isPresent()) {
|
||||
if (System.currentTimeMillis() > readTimeout) {
|
||||
throw new SocketTimeoutException("Timed out waiting for Peer " + peerDescription + " to acknowledge Protocol Version");
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
final int response = ackResponse.getAsInt();
|
||||
if (response < 0) {
|
||||
throw new EOFException("Encounter End-of-File with Peer " + peerDescription + " when expecting a Protocol Version Acknowledgment");
|
||||
}
|
||||
|
||||
if (response == VERSION_ACCEPTED) {
|
||||
logger.debug("Peer {} accepted Protocol Version {}", peerDescription, protocolVersion);
|
||||
phase = TransactionPhase.SEND_CONNECTION_ID;
|
||||
return true;
|
||||
}
|
||||
|
||||
if (response == REQEUST_DIFFERENT_VERSION) {
|
||||
logger.debug("Recommended using Protocol Version of {} with Peer {} but received REQUEST_DIFFERENT_VERSION response", protocolVersion, peerDescription);
|
||||
readTimeout = System.currentTimeMillis() + timeoutMillis;
|
||||
phase = TransactionPhase.RECEIVE_RECOMMENDED_PROTOCOL_VERSION;
|
||||
return true;
|
||||
}
|
||||
|
||||
throw new IOException("Failed to negotiate Protocol Version with Peer " + peerDescription + ". Recommended version " + protocolVersion + " but instead of an ACCEPT or REJECT " +
|
||||
"response got back a response of " + response);
|
||||
}
|
||||
|
||||
private boolean receiveRecommendedProtocolVersion() throws IOException {
|
||||
logger.debug("Receiving Protocol Version from Peer {}", peerDescription);
|
||||
|
||||
final OptionalInt recommendationResponse = channel.read();
|
||||
if (!recommendationResponse.isPresent()) {
|
||||
if (System.currentTimeMillis() > readTimeout) {
|
||||
throw new SocketTimeoutException("Timed out waiting for Peer " + peerDescription + " to recommend Protocol Version");
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
final int requestedVersion = recommendationResponse.getAsInt();
|
||||
if (requestedVersion < 0) {
|
||||
throw new EOFException("Encounter End-of-File with Peer " + peerDescription + " when expecting a Protocol Version Recommendation");
|
||||
}
|
||||
|
||||
if (negotiator.isVersionSupported(requestedVersion)) {
|
||||
protocolVersion = requestedVersion;
|
||||
phase = TransactionPhase.SEND_CONNECTION_ID;
|
||||
logger.debug("Peer {} recommended Protocol Version of {}. Accepting version.", peerDescription, requestedVersion);
|
||||
|
||||
return true;
|
||||
} else {
|
||||
final Integer preferred = negotiator.getPreferredVersion(requestedVersion);
|
||||
if (preferred == null) {
|
||||
logger.debug("Peer {} requested version {} of the Load Balance Protocol. This version is not acceptable. Aborting communications.", peerDescription, requestedVersion);
|
||||
phase = TransactionPhase.ABORT_PROTOCOL_NEGOTIATION;
|
||||
return true;
|
||||
} else {
|
||||
logger.debug("Peer {} requested version {} of the Protocol. Recommending version {} instead", peerDescription, requestedVersion, preferred);
|
||||
protocolVersion = preferred;
|
||||
phase = TransactionPhase.RECOMMEND_PROTOCOL_VERSION;
|
||||
return true;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private ByteBuffer noMoreFlowFiles() {
|
||||
final ByteBuffer buffer = ByteBuffer.allocate(1);
|
||||
buffer.put((byte) LoadBalanceProtocolConstants.NO_MORE_FLOWFILES);
|
||||
buffer.rewind();
|
||||
|
||||
checksum.update(LoadBalanceProtocolConstants.NO_MORE_FLOWFILES);
|
||||
return buffer;
|
||||
}
|
||||
|
||||
private ByteBuffer abortProtocolNegotiation() {
|
||||
final ByteBuffer buffer = ByteBuffer.allocate(1);
|
||||
buffer.put((byte) ABORT_PROTOCOL_NEGOTIATION);
|
||||
buffer.rewind();
|
||||
|
||||
return buffer;
|
||||
}
|
||||
|
||||
private ByteBuffer getConnectionId() {
|
||||
logger.debug("Sending Connection ID {} to Peer {}", connectionId, peerDescription);
|
||||
|
||||
final ByteBuffer buffer = ByteBuffer.allocate(connectionId.length() + 2);
|
||||
buffer.putShort((short) connectionId.length());
|
||||
buffer.put(connectionId.getBytes(StandardCharsets.UTF_8));
|
||||
buffer.rewind();
|
||||
|
||||
final byte[] frameBytes = buffer.array();
|
||||
checksum.update(frameBytes, 0, frameBytes.length);
|
||||
|
||||
phase = TransactionPhase.CHECK_SPACE;
|
||||
return buffer;
|
||||
}
|
||||
|
||||
private ByteBuffer checkSpace() {
|
||||
logger.debug("Sending a 'Check Space' request to Peer {} to determine if there is space in the queue for more FlowFiles", peerDescription);
|
||||
|
||||
final ByteBuffer buffer = ByteBuffer.allocate(1);
|
||||
|
||||
if (partition.isHonorBackpressure()) {
|
||||
buffer.put((byte) LoadBalanceProtocolConstants.CHECK_SPACE);
|
||||
checksum.update(LoadBalanceProtocolConstants.CHECK_SPACE);
|
||||
|
||||
readTimeout = System.currentTimeMillis() + timeoutMillis;
|
||||
phase = TransactionPhase.RECEIVE_SPACE_RESPONSE;
|
||||
} else {
|
||||
buffer.put((byte) LoadBalanceProtocolConstants.SKIP_SPACE_CHECK);
|
||||
checksum.update(LoadBalanceProtocolConstants.SKIP_SPACE_CHECK);
|
||||
|
||||
phase = TransactionPhase.GET_NEXT_FLOWFILE;
|
||||
}
|
||||
|
||||
buffer.rewind();
|
||||
return buffer;
|
||||
}
|
||||
|
||||
|
||||
private boolean receiveSpaceAvailableResponse() throws IOException {
|
||||
logger.debug("Receiving response from Peer {} to determine whether or not space is available in queue {}", peerDescription, connectionId);
|
||||
|
||||
final OptionalInt spaceAvailableResponse = channel.read();
|
||||
if (!spaceAvailableResponse.isPresent()) {
|
||||
if (System.currentTimeMillis() > readTimeout) {
|
||||
throw new SocketTimeoutException("Timed out waiting for Peer " + peerDescription + " to verify whether or not space is available for Connection " + connectionId);
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
final int response = spaceAvailableResponse.getAsInt();
|
||||
if (response < 0) {
|
||||
throw new EOFException("Encountered End-of-File when trying to verify with Peer " + peerDescription + " whether or not space is available in Connection " + connectionId);
|
||||
}
|
||||
|
||||
if (response == SPACE_AVAILABLE) {
|
||||
logger.debug("Peer {} has confirmed that space is available in Connection {}", peerDescription, connectionId);
|
||||
phase = TransactionPhase.GET_NEXT_FLOWFILE;
|
||||
} else if (response == QUEUE_FULL) {
|
||||
logger.debug("Peer {} has confirmed that the queue is full for Connection {}", peerDescription, connectionId);
|
||||
phase = TransactionPhase.RECOMMEND_PROTOCOL_VERSION;
|
||||
checksum.reset(); // We are restarting the session entirely so we need to reset our checksum
|
||||
penalize();
|
||||
} else {
|
||||
throw new TransactionAbortedException("After requesting to know whether or not Peer " + peerDescription + " has space available in Connection " + connectionId
|
||||
+ ", received unexpected response of " + response + ". Aborting transaction.");
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
private void penalize() {
|
||||
penaltyExpiration = System.currentTimeMillis() + PENALTY_MILLIS;
|
||||
}
|
||||
|
||||
private boolean isPenalized() {
|
||||
// check for penaltyExpiration > -1L is not strictly necessary as it's implied by the second check but is still
|
||||
// here because it's more efficient to check this than to make the system call to System.currentTimeMillis().
|
||||
return penaltyExpiration > -1L && System.currentTimeMillis() < penaltyExpiration;
|
||||
}
|
||||
|
||||
|
||||
private enum TransactionPhase {
|
||||
RECOMMEND_PROTOCOL_VERSION(SelectionKey.OP_WRITE),
|
||||
|
||||
RECEIVE_PROTOCOL_VERSION_ACKNOWLEDGMENT(SelectionKey.OP_READ),
|
||||
|
||||
RECEIVE_RECOMMENDED_PROTOCOL_VERSION(SelectionKey.OP_READ),
|
||||
|
||||
ABORT_PROTOCOL_NEGOTIATION(SelectionKey.OP_WRITE),
|
||||
|
||||
SEND_CONNECTION_ID(SelectionKey.OP_WRITE),
|
||||
|
||||
CHECK_SPACE(SelectionKey.OP_WRITE),
|
||||
|
||||
RECEIVE_SPACE_RESPONSE(SelectionKey.OP_READ),
|
||||
|
||||
SEND_FLOWFILE_DEFINITION(SelectionKey.OP_WRITE),
|
||||
|
||||
SEND_FLOWFILE_CONTENTS(SelectionKey.OP_WRITE),
|
||||
|
||||
GET_NEXT_FLOWFILE(SelectionKey.OP_WRITE),
|
||||
|
||||
SEND_CHECKSUM(SelectionKey.OP_WRITE),
|
||||
|
||||
VERIFY_CHECKSUM(SelectionKey.OP_READ),
|
||||
|
||||
SEND_TRANSACTION_COMPLETE(SelectionKey.OP_WRITE),
|
||||
|
||||
CONFIRM_TRANSACTION_COMPLETE(SelectionKey.OP_READ);
|
||||
|
||||
|
||||
private final int requiredSelectionKey;
|
||||
|
||||
TransactionPhase(final int requiredSelectionKey) {
|
||||
this.requiredSelectionKey = requiredSelectionKey;
|
||||
}
|
||||
|
||||
public int getRequiredSelectionKey() {
|
||||
return requiredSelectionKey;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,473 @@
|
|||
/*
|
||||
* 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.controller.queue.clustered.client.async.nio;
|
||||
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.controller.queue.LoadBalanceCompression;
|
||||
import org.apache.nifi.controller.queue.clustered.FlowFileContentAccess;
|
||||
import org.apache.nifi.controller.queue.clustered.SimpleLimitThreshold;
|
||||
import org.apache.nifi.controller.queue.clustered.TransactionThreshold;
|
||||
import org.apache.nifi.controller.queue.clustered.client.LoadBalanceFlowFileCodec;
|
||||
import org.apache.nifi.controller.queue.clustered.client.async.AsyncLoadBalanceClient;
|
||||
import org.apache.nifi.controller.queue.clustered.client.async.TransactionCompleteCallback;
|
||||
import org.apache.nifi.controller.queue.clustered.client.async.TransactionFailureCallback;
|
||||
import org.apache.nifi.controller.repository.FlowFileRecord;
|
||||
import org.apache.nifi.events.EventReporter;
|
||||
import org.apache.nifi.reporting.Severity;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import javax.net.ssl.SSLContext;
|
||||
import javax.net.ssl.SSLEngine;
|
||||
import java.io.IOException;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.net.Socket;
|
||||
import java.nio.channels.SelectionKey;
|
||||
import java.nio.channels.Selector;
|
||||
import java.nio.channels.SocketChannel;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Queue;
|
||||
import java.util.concurrent.LinkedBlockingQueue;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.concurrent.locks.Lock;
|
||||
import java.util.concurrent.locks.ReentrantLock;
|
||||
import java.util.function.BooleanSupplier;
|
||||
import java.util.function.Predicate;
|
||||
import java.util.function.Supplier;
|
||||
|
||||
|
||||
public class NioAsyncLoadBalanceClient implements AsyncLoadBalanceClient {
|
||||
private static final Logger logger = LoggerFactory.getLogger(NioAsyncLoadBalanceClient.class);
|
||||
private static final long PENALIZATION_MILLIS = TimeUnit.SECONDS.toMillis(1L);
|
||||
|
||||
private final NodeIdentifier nodeIdentifier;
|
||||
private final SSLContext sslContext;
|
||||
private final int timeoutMillis;
|
||||
private final FlowFileContentAccess flowFileContentAccess;
|
||||
private final LoadBalanceFlowFileCodec flowFileCodec;
|
||||
private final EventReporter eventReporter;
|
||||
|
||||
private volatile boolean running = false;
|
||||
private final AtomicLong penalizationEnd = new AtomicLong(0L);
|
||||
|
||||
private final Map<String, RegisteredPartition> registeredPartitions = new HashMap<>();
|
||||
private final Queue<RegisteredPartition> partitionQueue = new LinkedBlockingQueue<>();
|
||||
|
||||
// guarded by synchronizing on this
|
||||
private PeerChannel channel;
|
||||
private Selector selector;
|
||||
private SelectionKey selectionKey;
|
||||
|
||||
// While we use synchronization to guard most of the Class's state, we use a separate lock for the LoadBalanceSession.
|
||||
// We do this because we need to atomically decide whether or not we are able to communicate over the socket with another node and if so, continue on and do so.
|
||||
// However, we cannot do this within a synchronized block because if we did, then if Thread 1 were communicating with the remote node, and Thread 2 wanted to attempt
|
||||
// to do so, it would have to wait until Thread 1 released the synchronization. Instead, we want Thread 2 to determine that the resource is not free and move on.
|
||||
// I.e., we need to use the capability of Lock#tryLock, and the synchronized keyword does not offer this sort of functionality.
|
||||
private final Lock loadBalanceSessionLock = new ReentrantLock();
|
||||
private LoadBalanceSession loadBalanceSession = null;
|
||||
|
||||
|
||||
public NioAsyncLoadBalanceClient(final NodeIdentifier nodeIdentifier, final SSLContext sslContext, final int timeoutMillis, final FlowFileContentAccess flowFileContentAccess,
|
||||
final LoadBalanceFlowFileCodec flowFileCodec, final EventReporter eventReporter) {
|
||||
this.nodeIdentifier = nodeIdentifier;
|
||||
this.sslContext = sslContext;
|
||||
this.timeoutMillis = timeoutMillis;
|
||||
this.flowFileContentAccess = flowFileContentAccess;
|
||||
this.flowFileCodec = flowFileCodec;
|
||||
this.eventReporter = eventReporter;
|
||||
}
|
||||
|
||||
@Override
|
||||
public NodeIdentifier getNodeIdentifier() {
|
||||
return nodeIdentifier;
|
||||
}
|
||||
|
||||
public synchronized void register(final String connectionId, final BooleanSupplier emptySupplier, final Supplier<FlowFileRecord> flowFileSupplier,
|
||||
final TransactionFailureCallback failureCallback, final TransactionCompleteCallback successCallback,
|
||||
final Supplier<LoadBalanceCompression> compressionSupplier, final BooleanSupplier honorBackpressureSupplier) {
|
||||
|
||||
if (registeredPartitions.containsKey(connectionId)) {
|
||||
throw new IllegalStateException("Connection with ID " + connectionId + " is already registered");
|
||||
}
|
||||
|
||||
final RegisteredPartition partition = new RegisteredPartition(connectionId, emptySupplier, flowFileSupplier, failureCallback, successCallback, compressionSupplier, honorBackpressureSupplier);
|
||||
registeredPartitions.put(connectionId, partition);
|
||||
partitionQueue.add(partition);
|
||||
}
|
||||
|
||||
public synchronized void unregister(final String connectionId) {
|
||||
registeredPartitions.remove(connectionId);
|
||||
}
|
||||
|
||||
private synchronized Map<String, RegisteredPartition> getRegisteredPartitions() {
|
||||
return new HashMap<>(registeredPartitions);
|
||||
}
|
||||
|
||||
public void start() {
|
||||
running = true;
|
||||
logger.debug("{} started", this);
|
||||
}
|
||||
|
||||
public void stop() {
|
||||
running = false;
|
||||
logger.debug("{} stopped", this);
|
||||
close();
|
||||
}
|
||||
|
||||
private synchronized void close() {
|
||||
if (selector != null && selector.isOpen()) {
|
||||
try {
|
||||
selector.close();
|
||||
} catch (final Exception e) {
|
||||
logger.warn("Failed to close NIO Selector", e);
|
||||
}
|
||||
}
|
||||
|
||||
if (channel != null && channel.isOpen()) {
|
||||
try {
|
||||
channel.close();
|
||||
} catch (final Exception e) {
|
||||
logger.warn("Failed to close Socket Channel to {} for Load Balancing", nodeIdentifier, e);
|
||||
}
|
||||
}
|
||||
|
||||
channel = null;
|
||||
selector = null;
|
||||
}
|
||||
|
||||
public boolean isRunning() {
|
||||
return running;
|
||||
}
|
||||
|
||||
public boolean isPenalized() {
|
||||
final long endTimestamp = penalizationEnd.get();
|
||||
if (endTimestamp == 0) {
|
||||
return false;
|
||||
}
|
||||
|
||||
if (endTimestamp < System.currentTimeMillis()) {
|
||||
// set penalization end to 0 so that next time we don't need to check System.currentTimeMillis() because
|
||||
// systems calls are expensive enough that we'd like to avoid them when we can.
|
||||
penalizationEnd.compareAndSet(endTimestamp, 0L);
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
private void penalize() {
|
||||
logger.debug("Penalizing {}", this);
|
||||
this.penalizationEnd.set(System.currentTimeMillis() + PENALIZATION_MILLIS);
|
||||
}
|
||||
|
||||
|
||||
public boolean communicate() throws IOException {
|
||||
if (!running) {
|
||||
return false;
|
||||
}
|
||||
|
||||
// Use #tryLock here so that if another thread is already communicating with this Client, this thread
|
||||
// will not block and wait but instead will just return so that the Thread Pool can proceed to the next Client.
|
||||
if (!loadBalanceSessionLock.tryLock()) {
|
||||
return false;
|
||||
}
|
||||
|
||||
try {
|
||||
RegisteredPartition readyPartition = null;
|
||||
|
||||
if (!isConnectionEstablished()) {
|
||||
readyPartition = getReadyPartition();
|
||||
if (readyPartition == null) {
|
||||
logger.debug("{} has no connection with data ready to be transmitted so will penalize Client without communicating", this);
|
||||
penalize();
|
||||
return false;
|
||||
}
|
||||
|
||||
try {
|
||||
establishConnection();
|
||||
} catch (IOException e) {
|
||||
penalize();
|
||||
|
||||
partitionQueue.offer(readyPartition);
|
||||
|
||||
for (final RegisteredPartition partition : getRegisteredPartitions().values()) {
|
||||
logger.debug("Triggering Transaction Failure Callback for {} with Transaction Phase of CONNECTING", partition);
|
||||
partition.getFailureCallback().onTransactionFailed(Collections.emptyList(), e, TransactionFailureCallback.TransactionPhase.CONNECTING);
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
final LoadBalanceSession loadBalanceSession = getActiveTransaction(readyPartition);
|
||||
if (loadBalanceSession == null) {
|
||||
penalize();
|
||||
return false;
|
||||
}
|
||||
|
||||
selector.selectNow();
|
||||
final boolean ready = (loadBalanceSession.getDesiredReadinessFlag() & selectionKey.readyOps()) != 0;
|
||||
if (!ready) {
|
||||
return false;
|
||||
}
|
||||
|
||||
boolean anySuccess = false;
|
||||
boolean success;
|
||||
do {
|
||||
try {
|
||||
success = loadBalanceSession.communicate();
|
||||
} catch (final Exception e) {
|
||||
logger.error("Failed to communicate with Peer {}", nodeIdentifier.toString(), e);
|
||||
eventReporter.reportEvent(Severity.ERROR, "Load Balanced Connection", "Failed to communicate with Peer " + nodeIdentifier + " when load balancing data for Connection with ID " +
|
||||
loadBalanceSession.getPartition().getConnectionId() + " due to " + e);
|
||||
|
||||
penalize();
|
||||
loadBalanceSession.getPartition().getFailureCallback().onTransactionFailed(loadBalanceSession.getFlowFilesSent(), e, TransactionFailureCallback.TransactionPhase.SENDING);
|
||||
close();
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
anySuccess = anySuccess || success;
|
||||
} while (success);
|
||||
|
||||
if (loadBalanceSession.isComplete()) {
|
||||
loadBalanceSession.getPartition().getSuccessCallback().onTransactionComplete(loadBalanceSession.getFlowFilesSent());
|
||||
}
|
||||
|
||||
return anySuccess;
|
||||
} catch (final Exception e) {
|
||||
close();
|
||||
loadBalanceSession = null;
|
||||
throw e;
|
||||
} finally {
|
||||
loadBalanceSessionLock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* If any FlowFiles have been transferred in an active session, fail the transaction. Otherwise, gather up to the Transaction Threshold's limits
|
||||
* worth of FlowFiles and treat them as a failed transaction. In either case, terminate the session. This allows us to transfer FlowFiles from
|
||||
* queue partitions where the partitioner indicates that the data should be rebalanced, but does so in a way that we don't immediately rebalance
|
||||
* all FlowFiles. This is desirable in a case such as when we have a lot of data queued up in a connection and then a node temporarily disconnects.
|
||||
* We don't want to then just push all data to other nodes. We'd rather push the data out to other nodes slowly while waiting for the disconnected
|
||||
* node to reconnect. And if the node reconnects, we want to keep sending it data.
|
||||
*/
|
||||
public void nodeDisconnected() {
|
||||
if (!loadBalanceSessionLock.tryLock()) {
|
||||
// If we are not able to obtain the loadBalanceSessionLock, we cannot access the load balance session.
|
||||
return;
|
||||
}
|
||||
|
||||
try {
|
||||
final LoadBalanceSession session = getFailoverSession();
|
||||
if (session != null) {
|
||||
loadBalanceSession = null;
|
||||
|
||||
logger.debug("Node {} disconnected so will terminate the Load Balancing Session", nodeIdentifier);
|
||||
final List<FlowFileRecord> flowFilesSent = session.getFlowFilesSent();
|
||||
|
||||
if (!flowFilesSent.isEmpty()) {
|
||||
session.getPartition().getFailureCallback().onTransactionFailed(session.getFlowFilesSent(), TransactionFailureCallback.TransactionPhase.SENDING);
|
||||
}
|
||||
|
||||
close();
|
||||
penalize();
|
||||
return;
|
||||
}
|
||||
|
||||
// Obtain a partition that needs to be rebalanced on failure
|
||||
final RegisteredPartition readyPartition = getReadyPartition(partition -> partition.getFailureCallback().isRebalanceOnFailure());
|
||||
if (readyPartition == null) {
|
||||
return;
|
||||
}
|
||||
|
||||
partitionQueue.offer(readyPartition); // allow partition to be obtained again
|
||||
final TransactionThreshold threshold = newTransactionThreshold();
|
||||
|
||||
final List<FlowFileRecord> flowFiles = new ArrayList<>();
|
||||
while (!threshold.isThresholdMet()) {
|
||||
final FlowFileRecord flowFile = readyPartition.getFlowFileRecordSupplier().get();
|
||||
if (flowFile == null) {
|
||||
break;
|
||||
}
|
||||
|
||||
flowFiles.add(flowFile);
|
||||
threshold.adjust(1, flowFile.getSize());
|
||||
}
|
||||
|
||||
logger.debug("Node {} not connected so failing {} FlowFiles for Load Balancing", nodeIdentifier, flowFiles.size());
|
||||
readyPartition.getFailureCallback().onTransactionFailed(flowFiles, TransactionFailureCallback.TransactionPhase.SENDING);
|
||||
penalize(); // Don't just transfer FlowFiles out of queue's partition as fast as possible, because the node may only be disconnected for a short time.
|
||||
} finally {
|
||||
loadBalanceSessionLock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
private synchronized LoadBalanceSession getFailoverSession() {
|
||||
if (loadBalanceSession != null && !loadBalanceSession.isComplete()) {
|
||||
return loadBalanceSession;
|
||||
}
|
||||
|
||||
return null;
|
||||
}
|
||||
|
||||
|
||||
private RegisteredPartition getReadyPartition() {
|
||||
return getReadyPartition(partition -> true);
|
||||
}
|
||||
|
||||
private synchronized RegisteredPartition getReadyPartition(final Predicate<RegisteredPartition> filter) {
|
||||
final List<RegisteredPartition> polledPartitions = new ArrayList<>();
|
||||
|
||||
try {
|
||||
RegisteredPartition partition;
|
||||
while ((partition = partitionQueue.poll()) != null) {
|
||||
if (partition.isEmpty() || !filter.test(partition)) {
|
||||
polledPartitions.add(partition);
|
||||
continue;
|
||||
}
|
||||
|
||||
return partition;
|
||||
}
|
||||
|
||||
return null;
|
||||
} finally {
|
||||
polledPartitions.forEach(partitionQueue::offer);
|
||||
}
|
||||
}
|
||||
|
||||
private synchronized LoadBalanceSession getActiveTransaction(final RegisteredPartition proposedPartition) {
|
||||
if (loadBalanceSession != null && !loadBalanceSession.isComplete()) {
|
||||
return loadBalanceSession;
|
||||
}
|
||||
|
||||
final RegisteredPartition readyPartition = proposedPartition == null ? getReadyPartition() : proposedPartition;
|
||||
if (readyPartition == null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
loadBalanceSession = new LoadBalanceSession(readyPartition, flowFileContentAccess, flowFileCodec, channel, timeoutMillis, newTransactionThreshold());
|
||||
partitionQueue.offer(readyPartition);
|
||||
|
||||
return loadBalanceSession;
|
||||
}
|
||||
|
||||
private TransactionThreshold newTransactionThreshold() {
|
||||
return new SimpleLimitThreshold(1000, 10_000_000L);
|
||||
}
|
||||
|
||||
private synchronized boolean isConnectionEstablished() {
|
||||
return selector != null && channel != null && channel.isConnected();
|
||||
}
|
||||
|
||||
private synchronized void establishConnection() throws IOException {
|
||||
SocketChannel socketChannel = null;
|
||||
|
||||
try {
|
||||
selector = Selector.open();
|
||||
socketChannel = createChannel();
|
||||
|
||||
socketChannel.configureBlocking(true);
|
||||
|
||||
channel = createPeerChannel(socketChannel, nodeIdentifier.toString());
|
||||
channel.performHandshake();
|
||||
|
||||
socketChannel.configureBlocking(false);
|
||||
selectionKey = socketChannel.register(selector, SelectionKey.OP_WRITE | SelectionKey.OP_READ);
|
||||
} catch (Exception e) {
|
||||
logger.error("Unable to connect to {} for load balancing", nodeIdentifier, e);
|
||||
|
||||
if (selector != null) {
|
||||
try {
|
||||
selector.close();
|
||||
} catch (final Exception e1) {
|
||||
e.addSuppressed(e1);
|
||||
}
|
||||
}
|
||||
|
||||
if (channel != null) {
|
||||
try {
|
||||
channel.close();
|
||||
} catch (final Exception e1) {
|
||||
e.addSuppressed(e1);
|
||||
}
|
||||
}
|
||||
|
||||
if (socketChannel != null) {
|
||||
try {
|
||||
socketChannel.close();
|
||||
} catch (final Exception e1) {
|
||||
e.addSuppressed(e1);
|
||||
}
|
||||
}
|
||||
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
private PeerChannel createPeerChannel(final SocketChannel channel, final String peerDescription) {
|
||||
if (sslContext == null) {
|
||||
logger.debug("No SSL Context is available so will not perform SSL Handshake with Peer {}", peerDescription);
|
||||
return new PeerChannel(channel, null, peerDescription);
|
||||
}
|
||||
|
||||
logger.debug("Performing SSL Handshake with Peer {}", peerDescription);
|
||||
|
||||
final SSLEngine sslEngine = sslContext.createSSLEngine();
|
||||
sslEngine.setUseClientMode(true);
|
||||
sslEngine.setNeedClientAuth(true);
|
||||
|
||||
return new PeerChannel(channel, sslEngine, peerDescription);
|
||||
}
|
||||
|
||||
|
||||
private SocketChannel createChannel() throws IOException {
|
||||
final SocketChannel socketChannel = SocketChannel.open();
|
||||
try {
|
||||
socketChannel.configureBlocking(true);
|
||||
final Socket socket = socketChannel.socket();
|
||||
socket.setSoTimeout(timeoutMillis);
|
||||
|
||||
socket.connect(new InetSocketAddress(nodeIdentifier.getLoadBalanceAddress(), nodeIdentifier.getLoadBalancePort()));
|
||||
socket.setSoTimeout(timeoutMillis);
|
||||
|
||||
return socketChannel;
|
||||
} catch (final Exception e) {
|
||||
try {
|
||||
socketChannel.close();
|
||||
} catch (final Exception closeException) {
|
||||
e.addSuppressed(closeException);
|
||||
}
|
||||
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "NioAsyncLoadBalanceClient[nodeId=" + nodeIdentifier + "]";
|
||||
}
|
||||
}
|
|
@ -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.controller.queue.clustered.client.async.nio;
|
||||
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.controller.queue.clustered.FlowFileContentAccess;
|
||||
import org.apache.nifi.controller.queue.clustered.client.LoadBalanceFlowFileCodec;
|
||||
import org.apache.nifi.controller.queue.clustered.client.StandardLoadBalanceFlowFileCodec;
|
||||
import org.apache.nifi.controller.queue.clustered.client.async.AsyncLoadBalanceClientFactory;
|
||||
import org.apache.nifi.events.EventReporter;
|
||||
|
||||
import javax.net.ssl.SSLContext;
|
||||
|
||||
public class NioAsyncLoadBalanceClientFactory implements AsyncLoadBalanceClientFactory {
|
||||
private final SSLContext sslContext;
|
||||
private final int timeoutMillis;
|
||||
private final FlowFileContentAccess flowFileContentAccess;
|
||||
private final EventReporter eventReporter;
|
||||
private final LoadBalanceFlowFileCodec flowFileCodec;
|
||||
|
||||
public NioAsyncLoadBalanceClientFactory(final SSLContext sslContext, final int timeoutMillis, final FlowFileContentAccess flowFileContentAccess, final EventReporter eventReporter,
|
||||
final LoadBalanceFlowFileCodec loadBalanceFlowFileCodec) {
|
||||
this.sslContext = sslContext;
|
||||
this.timeoutMillis = timeoutMillis;
|
||||
this.flowFileContentAccess = flowFileContentAccess;
|
||||
this.eventReporter = eventReporter;
|
||||
this.flowFileCodec = loadBalanceFlowFileCodec;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public NioAsyncLoadBalanceClient createClient(final NodeIdentifier nodeIdentifier) {
|
||||
return new NioAsyncLoadBalanceClient(nodeIdentifier, sslContext, timeoutMillis, flowFileContentAccess, new StandardLoadBalanceFlowFileCodec(), eventReporter);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,122 @@
|
|||
/*
|
||||
* 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.controller.queue.clustered.client.async.nio;
|
||||
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.controller.queue.LoadBalanceCompression;
|
||||
import org.apache.nifi.controller.queue.clustered.client.async.AsyncLoadBalanceClient;
|
||||
import org.apache.nifi.controller.queue.clustered.client.async.AsyncLoadBalanceClientRegistry;
|
||||
import org.apache.nifi.controller.queue.clustered.client.async.TransactionCompleteCallback;
|
||||
import org.apache.nifi.controller.queue.clustered.client.async.TransactionFailureCallback;
|
||||
import org.apache.nifi.controller.repository.FlowFileRecord;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.CopyOnWriteArraySet;
|
||||
import java.util.function.BooleanSupplier;
|
||||
import java.util.function.Supplier;
|
||||
|
||||
public class NioAsyncLoadBalanceClientRegistry implements AsyncLoadBalanceClientRegistry {
|
||||
private static final Logger logger = LoggerFactory.getLogger(NioAsyncLoadBalanceClientRegistry.class);
|
||||
|
||||
private final NioAsyncLoadBalanceClientFactory clientFactory;
|
||||
private final int clientsPerNode;
|
||||
|
||||
private Map<NodeIdentifier, Set<AsyncLoadBalanceClient>> clientMap = new HashMap<>();
|
||||
private Set<AsyncLoadBalanceClient> allClients = new CopyOnWriteArraySet<>();
|
||||
private boolean running = false;
|
||||
|
||||
public NioAsyncLoadBalanceClientRegistry(final NioAsyncLoadBalanceClientFactory clientFactory, final int clientsPerNode) {
|
||||
this.clientFactory = clientFactory;
|
||||
this.clientsPerNode = clientsPerNode;
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized void register(final String connectionId, final NodeIdentifier nodeId, final BooleanSupplier emptySupplier, final Supplier<FlowFileRecord> flowFileSupplier,
|
||||
final TransactionFailureCallback failureCallback, final TransactionCompleteCallback successCallback,
|
||||
final Supplier<LoadBalanceCompression> compressionSupplier, final BooleanSupplier honorBackpressureSupplier) {
|
||||
|
||||
Set<AsyncLoadBalanceClient> clients = clientMap.get(nodeId);
|
||||
if (clients == null) {
|
||||
clients = registerClients(nodeId);
|
||||
}
|
||||
|
||||
clients.forEach(client -> client.register(connectionId, emptySupplier, flowFileSupplier, failureCallback, successCallback, compressionSupplier, honorBackpressureSupplier));
|
||||
logger.debug("Registered Connection with ID {} to send to Node {}", connectionId, nodeId);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public synchronized void unregister(final String connectionId, final NodeIdentifier nodeId) {
|
||||
final Set<AsyncLoadBalanceClient> clients = clientMap.remove(nodeId);
|
||||
if (clients == null) {
|
||||
return;
|
||||
}
|
||||
|
||||
clients.forEach(client -> client.unregister(connectionId));
|
||||
|
||||
allClients.removeAll(clients);
|
||||
logger.debug("Un-registered Connection with ID {} so that it will no longer send data to Node {}", connectionId, nodeId);
|
||||
}
|
||||
|
||||
private Set<AsyncLoadBalanceClient> registerClients(final NodeIdentifier nodeId) {
|
||||
final Set<AsyncLoadBalanceClient> clients = new HashSet<>();
|
||||
|
||||
for (int i=0; i < clientsPerNode; i++) {
|
||||
final AsyncLoadBalanceClient client = clientFactory.createClient(nodeId);
|
||||
clients.add(client);
|
||||
|
||||
logger.debug("Added client {} for communicating with Node {}", client, nodeId);
|
||||
}
|
||||
|
||||
clientMap.put(nodeId, clients);
|
||||
allClients.addAll(clients);
|
||||
|
||||
if (running) {
|
||||
clients.forEach(AsyncLoadBalanceClient::start);
|
||||
}
|
||||
|
||||
return clients;
|
||||
}
|
||||
|
||||
public synchronized Set<AsyncLoadBalanceClient> getAllClients() {
|
||||
return allClients;
|
||||
}
|
||||
|
||||
public synchronized void start() {
|
||||
if (running) {
|
||||
return;
|
||||
}
|
||||
|
||||
running = true;
|
||||
allClients.forEach(AsyncLoadBalanceClient::start);
|
||||
}
|
||||
|
||||
public synchronized void stop() {
|
||||
if (!running) {
|
||||
return;
|
||||
}
|
||||
|
||||
running = false;
|
||||
allClients.forEach(AsyncLoadBalanceClient::stop);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,107 @@
|
|||
/*
|
||||
* 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.controller.queue.clustered.client.async.nio;
|
||||
|
||||
import org.apache.nifi.cluster.coordination.ClusterCoordinator;
|
||||
import org.apache.nifi.cluster.coordination.node.NodeConnectionState;
|
||||
import org.apache.nifi.cluster.coordination.node.NodeConnectionStatus;
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.controller.queue.clustered.client.async.AsyncLoadBalanceClient;
|
||||
import org.apache.nifi.events.EventReporter;
|
||||
import org.apache.nifi.reporting.Severity;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
public class NioAsyncLoadBalanceClientTask implements Runnable {
|
||||
private static final Logger logger = LoggerFactory.getLogger(NioAsyncLoadBalanceClientTask.class);
|
||||
private static final String EVENT_CATEGORY = "Load-Balanced Connection";
|
||||
|
||||
private final NioAsyncLoadBalanceClientRegistry clientRegistry;
|
||||
private final ClusterCoordinator clusterCoordinator;
|
||||
private final EventReporter eventReporter;
|
||||
private volatile boolean running = true;
|
||||
|
||||
public NioAsyncLoadBalanceClientTask(final NioAsyncLoadBalanceClientRegistry clientRegistry, final ClusterCoordinator clusterCoordinator, final EventReporter eventReporter) {
|
||||
this.clientRegistry = clientRegistry;
|
||||
this.clusterCoordinator = clusterCoordinator;
|
||||
this.eventReporter = eventReporter;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
while (running) {
|
||||
try {
|
||||
boolean success = false;
|
||||
for (final AsyncLoadBalanceClient client : clientRegistry.getAllClients()) {
|
||||
if (!client.isRunning()) {
|
||||
logger.trace("Client {} is not running so will not communicate with it", client);
|
||||
continue;
|
||||
}
|
||||
|
||||
if (client.isPenalized()) {
|
||||
logger.trace("Client {} is penalized so will not communicate with it", client);
|
||||
continue;
|
||||
}
|
||||
|
||||
final NodeIdentifier clientNodeId = client.getNodeIdentifier();
|
||||
final NodeConnectionStatus connectionStatus = clusterCoordinator.getConnectionStatus(clientNodeId);
|
||||
if (connectionStatus == null) {
|
||||
logger.debug("Could not determine Connection Status for Node with ID {}; will not communicate with it", clientNodeId);
|
||||
continue;
|
||||
}
|
||||
|
||||
final NodeConnectionState connectionState = connectionStatus.getState();
|
||||
if (connectionState == NodeConnectionState.DISCONNECTED || connectionState == NodeConnectionState.DISCONNECTING) {
|
||||
client.nodeDisconnected();
|
||||
continue;
|
||||
}
|
||||
|
||||
if (connectionState != NodeConnectionState.CONNECTED) {
|
||||
logger.debug("Client {} is for node that is not currently connected (state = {}) so will not communicate with node", client, connectionState);
|
||||
continue;
|
||||
}
|
||||
|
||||
try {
|
||||
while (client.communicate()) {
|
||||
success = true;
|
||||
logger.trace("Client {} was able to make progress communicating with peer. Will continue to communicate with peer.", client);
|
||||
}
|
||||
} catch (final Exception e) {
|
||||
eventReporter.reportEvent(Severity.ERROR, EVENT_CATEGORY, "Failed to communicate with Peer "
|
||||
+ client.getNodeIdentifier() + " while trying to load balance data across the cluster due to " + e.toString());
|
||||
logger.error("Failed to communicate with Peer {} while trying to load balance data across the cluster.", client.getNodeIdentifier(), e);
|
||||
}
|
||||
|
||||
logger.trace("Client {} was no longer able to make progress communicating with peer. Will move on to the next client", client);
|
||||
}
|
||||
|
||||
if (!success) {
|
||||
logger.trace("Was unable to communicate with any client. Will sleep for 10 milliseconds.");
|
||||
Thread.sleep(10L);
|
||||
}
|
||||
} catch (final Exception e) {
|
||||
logger.error("Failed to communicate with peer while trying to load balance data across the cluster", e);
|
||||
eventReporter.reportEvent(Severity.ERROR, EVENT_CATEGORY, "Failed to comunicate with Peer while trying to load balance data across the cluster due to " + e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void stop() {
|
||||
running = false;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,358 @@
|
|||
/*
|
||||
* 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.controller.queue.clustered.client.async.nio;
|
||||
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import javax.net.ssl.SSLEngine;
|
||||
import javax.net.ssl.SSLEngineResult;
|
||||
import javax.net.ssl.SSLException;
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.channels.SocketChannel;
|
||||
import java.util.OptionalInt;
|
||||
|
||||
public class PeerChannel implements Closeable {
|
||||
private static final Logger logger = LoggerFactory.getLogger(PeerChannel.class);
|
||||
|
||||
private final SocketChannel socketChannel;
|
||||
private final SSLEngine sslEngine;
|
||||
private final String peerDescription;
|
||||
|
||||
private final ByteBuffer singleByteBuffer = ByteBuffer.allocate(1);
|
||||
private ByteBuffer destinationBuffer = ByteBuffer.allocate(16 * 1024); // buffer that SSLEngine is to write into
|
||||
private ByteBuffer streamBuffer = ByteBuffer.allocate(16 * 1024); // buffer for data that is read from SocketChannel
|
||||
private ByteBuffer applicationBuffer = ByteBuffer.allocate(0); // buffer for application-level data that is ready to be served up (i.e., already decrypted if necessary)
|
||||
|
||||
public PeerChannel(final SocketChannel socketChannel, final SSLEngine sslEngine, final String peerDescription) {
|
||||
this.socketChannel = socketChannel;
|
||||
this.sslEngine = sslEngine;
|
||||
this.peerDescription = peerDescription;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
socketChannel.close();
|
||||
}
|
||||
|
||||
public boolean isConnected() {
|
||||
return socketChannel.isConnected();
|
||||
}
|
||||
|
||||
public boolean isOpen() {
|
||||
return socketChannel.isOpen();
|
||||
}
|
||||
|
||||
public String getPeerDescription() {
|
||||
return peerDescription;
|
||||
}
|
||||
|
||||
public boolean write(final byte b) throws IOException {
|
||||
singleByteBuffer.clear();
|
||||
singleByteBuffer.put(b);
|
||||
singleByteBuffer.rewind();
|
||||
|
||||
final ByteBuffer prepared = prepareForWrite(singleByteBuffer);
|
||||
final int bytesWritten = write(prepared);
|
||||
return bytesWritten > 0;
|
||||
}
|
||||
|
||||
public OptionalInt read() throws IOException {
|
||||
singleByteBuffer.clear();
|
||||
final int bytesRead = read(singleByteBuffer);
|
||||
if (bytesRead < 0) {
|
||||
return OptionalInt.of(-1);
|
||||
}
|
||||
if (bytesRead == 0) {
|
||||
return OptionalInt.empty();
|
||||
}
|
||||
|
||||
singleByteBuffer.flip();
|
||||
|
||||
final byte read = singleByteBuffer.get();
|
||||
return OptionalInt.of(read & 0xFF);
|
||||
}
|
||||
|
||||
|
||||
|
||||
|
||||
/**
|
||||
* Reads the given ByteBuffer of data and returns a new ByteBuffer (which is "flipped" / ready to be read). The newly returned
|
||||
* ByteBuffer will be written to be written via the {@link #write(ByteBuffer)} method. I.e., it will have already been encrypted, if
|
||||
* necessary, and any other decorations that need to be applied before sending will already have been applied.
|
||||
*
|
||||
* @param plaintext the data to be prepped
|
||||
* @return a ByteBuffer containing the prepared data
|
||||
* @throws IOException if a failure occurs while encrypting the data
|
||||
*/
|
||||
public ByteBuffer prepareForWrite(final ByteBuffer plaintext) throws IOException {
|
||||
if (sslEngine == null) {
|
||||
return plaintext;
|
||||
}
|
||||
|
||||
|
||||
ByteBuffer prepared = ByteBuffer.allocate(Math.min(85, plaintext.capacity() - plaintext.position()));
|
||||
while (plaintext.hasRemaining()) {
|
||||
encrypt(plaintext);
|
||||
|
||||
final int bytesRemaining = prepared.capacity() - prepared.position();
|
||||
if (bytesRemaining < destinationBuffer.remaining()) {
|
||||
final ByteBuffer temp = ByteBuffer.allocate(prepared.capacity() + sslEngine.getSession().getApplicationBufferSize());
|
||||
prepared.flip();
|
||||
temp.put(prepared);
|
||||
prepared = temp;
|
||||
}
|
||||
|
||||
prepared.put(destinationBuffer);
|
||||
}
|
||||
|
||||
prepared.flip();
|
||||
return prepared;
|
||||
}
|
||||
|
||||
public int write(final ByteBuffer preparedBuffer) throws IOException {
|
||||
return socketChannel.write(preparedBuffer);
|
||||
}
|
||||
|
||||
|
||||
public int read(final ByteBuffer dst) throws IOException {
|
||||
// If we have data ready to go, then go ahead and copy it.
|
||||
final int bytesCopied = copy(applicationBuffer, dst);
|
||||
if (bytesCopied != 0) {
|
||||
return bytesCopied;
|
||||
}
|
||||
|
||||
final int bytesRead = socketChannel.read(streamBuffer);
|
||||
if (bytesRead < 1) {
|
||||
return bytesRead;
|
||||
}
|
||||
|
||||
if (bytesRead > 0) {
|
||||
logger.trace("Read {} bytes from SocketChannel", bytesRead);
|
||||
}
|
||||
|
||||
streamBuffer.flip();
|
||||
|
||||
try {
|
||||
if (sslEngine == null) {
|
||||
cloneToApplicationBuffer(streamBuffer);
|
||||
return copy(applicationBuffer, dst);
|
||||
} else {
|
||||
final boolean decrypted = decrypt(streamBuffer);
|
||||
logger.trace("Decryption after reading those bytes successful = {}", decrypted);
|
||||
|
||||
if (decrypted) {
|
||||
cloneToApplicationBuffer(destinationBuffer);
|
||||
logger.trace("Cloned destination buffer to application buffer");
|
||||
|
||||
return copy(applicationBuffer, dst);
|
||||
} else {
|
||||
// Not enough data to decrypt. Compact the buffer so that we keep the data we have
|
||||
// but prepare the buffer to be written to again.
|
||||
logger.debug("Not enough data to decrypt. Will need to consume more data before decrypting");
|
||||
streamBuffer.compact();
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
streamBuffer.compact();
|
||||
}
|
||||
}
|
||||
|
||||
private void cloneToApplicationBuffer(final ByteBuffer buffer) {
|
||||
if (applicationBuffer.capacity() < buffer.remaining()) {
|
||||
applicationBuffer = ByteBuffer.allocate(buffer.remaining());
|
||||
} else {
|
||||
applicationBuffer.clear();
|
||||
}
|
||||
|
||||
applicationBuffer.put(buffer);
|
||||
applicationBuffer.flip();
|
||||
}
|
||||
|
||||
private int copy(final ByteBuffer src, final ByteBuffer dst) {
|
||||
if (src != null && src.hasRemaining()) {
|
||||
final int bytesToCopy = Math.min(dst.remaining(), src.remaining());
|
||||
if (bytesToCopy < 1) {
|
||||
return bytesToCopy;
|
||||
}
|
||||
|
||||
final byte[] buff = new byte[bytesToCopy];
|
||||
src.get(buff);
|
||||
dst.put(buff);
|
||||
return bytesToCopy;
|
||||
}
|
||||
|
||||
return 0;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Encrypts the given buffer of data, writing the result into {@link #destinationBuffer}.
|
||||
* @param plaintext the data to encrypt
|
||||
* @throws IOException if the Peer closes the connection abruptly or if unable to perform the encryption
|
||||
*/
|
||||
private void encrypt(final ByteBuffer plaintext) throws IOException {
|
||||
if (sslEngine == null) {
|
||||
throw new SSLException("Unable to encrypt message because no SSLEngine has been configured");
|
||||
}
|
||||
|
||||
destinationBuffer.clear();
|
||||
|
||||
while (true) {
|
||||
final SSLEngineResult result = sslEngine.wrap(plaintext, destinationBuffer);
|
||||
|
||||
switch (result.getStatus()) {
|
||||
case OK:
|
||||
destinationBuffer.flip();
|
||||
return;
|
||||
case CLOSED:
|
||||
throw new IOException("Failed to encrypt data to write to Peer " + peerDescription + " because Peer unexpectedly closed connection");
|
||||
case BUFFER_OVERFLOW:
|
||||
// destinationBuffer is not large enough. Need to increase the size.
|
||||
final ByteBuffer tempBuffer = ByteBuffer.allocate(destinationBuffer.capacity() + sslEngine.getSession().getApplicationBufferSize());
|
||||
destinationBuffer.flip();
|
||||
tempBuffer.put(destinationBuffer);
|
||||
destinationBuffer = tempBuffer;
|
||||
break;
|
||||
case BUFFER_UNDERFLOW:
|
||||
// We should never get this result on a call to SSLEngine.wrap(), only on a call to unwrap().
|
||||
throw new IOException("Received unexpected Buffer Underflow result when encrypting data to write to Peer " + peerDescription);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
||||
|
||||
/**
|
||||
* Attempts to decrypt the given buffer of data, writing the result into {@link #destinationBuffer}. If successful, will return <code>true</code>.
|
||||
* If more data is needed in order to perform the decryption, will return <code>false</code>.
|
||||
*
|
||||
* @param encrypted the ByteBuffer containing the data to decrypt
|
||||
* @return <code>true</code> if decryption was successful, <code>false</code> otherwise
|
||||
* @throws IOException if the Peer closed the connection or if unable to decrypt the message
|
||||
*/
|
||||
private boolean decrypt(final ByteBuffer encrypted) throws IOException {
|
||||
if (sslEngine == null) {
|
||||
throw new SSLException("Unable to decrypt message because no SSLEngine has been configured");
|
||||
}
|
||||
|
||||
destinationBuffer.clear();
|
||||
|
||||
while (true) {
|
||||
final SSLEngineResult result = sslEngine.unwrap(encrypted, destinationBuffer);
|
||||
|
||||
switch (result.getStatus()) {
|
||||
case OK:
|
||||
destinationBuffer.flip();
|
||||
return true;
|
||||
case CLOSED:
|
||||
throw new IOException("Failed to decrypt data from Peer " + peerDescription + " because Peer unexpectedly closed connection");
|
||||
case BUFFER_OVERFLOW:
|
||||
// ecnryptedBuffer is not large enough. Need to increase the size.
|
||||
final ByteBuffer tempBuffer = ByteBuffer.allocate(encrypted.position() + sslEngine.getSession().getApplicationBufferSize());
|
||||
destinationBuffer.flip();
|
||||
tempBuffer.put(destinationBuffer);
|
||||
destinationBuffer = tempBuffer;
|
||||
|
||||
break;
|
||||
case BUFFER_UNDERFLOW:
|
||||
// Not enough data to decrypt. Must read more from the channel.
|
||||
return false;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
public void performHandshake() throws IOException {
|
||||
if (sslEngine == null) {
|
||||
return;
|
||||
}
|
||||
|
||||
sslEngine.beginHandshake();
|
||||
|
||||
final ByteBuffer emptyMessage = ByteBuffer.allocate(0);
|
||||
ByteBuffer unwrapBuffer = ByteBuffer.allocate(0);
|
||||
|
||||
while (true) {
|
||||
final SSLEngineResult.HandshakeStatus handshakeStatus = sslEngine.getHandshakeStatus();
|
||||
|
||||
switch (handshakeStatus) {
|
||||
case FINISHED:
|
||||
case NOT_HANDSHAKING:
|
||||
streamBuffer.clear();
|
||||
destinationBuffer.clear();
|
||||
logger.debug("Completed SSL Handshake with Peer {}", peerDescription);
|
||||
return;
|
||||
|
||||
case NEED_TASK:
|
||||
logger.debug("SSL Handshake with Peer {} Needs Task", peerDescription);
|
||||
|
||||
Runnable runnable;
|
||||
while ((runnable = sslEngine.getDelegatedTask()) != null) {
|
||||
runnable.run();
|
||||
}
|
||||
break;
|
||||
|
||||
case NEED_WRAP:
|
||||
logger.trace("SSL Handshake with Peer {} Needs Wrap", peerDescription);
|
||||
|
||||
encrypt(emptyMessage);
|
||||
final int bytesWritten = write(destinationBuffer);
|
||||
logger.debug("Wrote {} bytes for NEED_WRAP portion of Handshake", bytesWritten);
|
||||
break;
|
||||
|
||||
case NEED_UNWRAP:
|
||||
logger.trace("SSL Handshake with Peer {} Needs Unwrap", peerDescription);
|
||||
|
||||
while (sslEngine.getHandshakeStatus() == SSLEngineResult.HandshakeStatus.NEED_UNWRAP) {
|
||||
final boolean decrypted = decrypt(unwrapBuffer);
|
||||
if (decrypted) {
|
||||
logger.trace("Decryption was successful for NEED_UNWRAP portion of Handshake");
|
||||
break;
|
||||
}
|
||||
|
||||
if (unwrapBuffer.capacity() - unwrapBuffer.position() < 1) {
|
||||
logger.trace("Enlarging size of Buffer for NEED_UNWRAP portion of Handshake");
|
||||
|
||||
// destinationBuffer is not large enough. Need to increase the size.
|
||||
final ByteBuffer tempBuffer = ByteBuffer.allocate(unwrapBuffer.capacity() + sslEngine.getSession().getApplicationBufferSize());
|
||||
tempBuffer.put(unwrapBuffer);
|
||||
unwrapBuffer = tempBuffer;
|
||||
unwrapBuffer.flip();
|
||||
continue;
|
||||
}
|
||||
|
||||
logger.trace("Need to read more bytes for NEED_UNWRAP portion of Handshake");
|
||||
|
||||
// Need to read more data.
|
||||
unwrapBuffer.compact();
|
||||
final int bytesRead = socketChannel.read(unwrapBuffer);
|
||||
unwrapBuffer.flip();
|
||||
logger.debug("Read {} bytes for NEED_UNWRAP portion of Handshake", bytesRead);
|
||||
}
|
||||
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,75 @@
|
|||
/*
|
||||
* 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.controller.queue.clustered.client.async.nio;
|
||||
|
||||
import org.apache.nifi.controller.queue.LoadBalanceCompression;
|
||||
import org.apache.nifi.controller.queue.clustered.client.async.TransactionCompleteCallback;
|
||||
import org.apache.nifi.controller.queue.clustered.client.async.TransactionFailureCallback;
|
||||
import org.apache.nifi.controller.repository.FlowFileRecord;
|
||||
|
||||
import java.util.function.BooleanSupplier;
|
||||
import java.util.function.Supplier;
|
||||
|
||||
public class RegisteredPartition {
|
||||
private final String connectionId;
|
||||
private final Supplier<FlowFileRecord> flowFileRecordSupplier;
|
||||
private final TransactionFailureCallback failureCallback;
|
||||
private final BooleanSupplier emptySupplier;
|
||||
private final TransactionCompleteCallback successCallback;
|
||||
private final Supplier<LoadBalanceCompression> compressionSupplier;
|
||||
private final BooleanSupplier honorBackpressureSupplier;
|
||||
|
||||
public RegisteredPartition(final String connectionId, final BooleanSupplier emptySupplier, final Supplier<FlowFileRecord> flowFileSupplier, final TransactionFailureCallback failureCallback,
|
||||
final TransactionCompleteCallback successCallback, final Supplier<LoadBalanceCompression> compressionSupplier, final BooleanSupplier honorBackpressureSupplier) {
|
||||
this.connectionId = connectionId;
|
||||
this.emptySupplier = emptySupplier;
|
||||
this.flowFileRecordSupplier = flowFileSupplier;
|
||||
this.failureCallback = failureCallback;
|
||||
this.successCallback = successCallback;
|
||||
this.compressionSupplier = compressionSupplier;
|
||||
this.honorBackpressureSupplier = honorBackpressureSupplier;
|
||||
}
|
||||
|
||||
public boolean isEmpty() {
|
||||
return emptySupplier.getAsBoolean();
|
||||
}
|
||||
|
||||
public String getConnectionId() {
|
||||
return connectionId;
|
||||
}
|
||||
|
||||
public Supplier<FlowFileRecord> getFlowFileRecordSupplier() {
|
||||
return flowFileRecordSupplier;
|
||||
}
|
||||
|
||||
public TransactionFailureCallback getFailureCallback() {
|
||||
return failureCallback;
|
||||
}
|
||||
|
||||
public TransactionCompleteCallback getSuccessCallback() {
|
||||
return successCallback;
|
||||
}
|
||||
|
||||
public LoadBalanceCompression getCompression() {
|
||||
return compressionSupplier.get();
|
||||
}
|
||||
|
||||
public boolean isHonorBackpressure() {
|
||||
return honorBackpressureSupplier.getAsBoolean();
|
||||
}
|
||||
}
|
|
@ -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.controller.queue.clustered.partition;
|
||||
|
||||
import com.google.common.hash.Hashing;
|
||||
import org.apache.nifi.controller.repository.FlowFileRecord;
|
||||
|
||||
public class CorrelationAttributePartitioner implements FlowFilePartitioner {
|
||||
private final String partitioningAttribute;
|
||||
|
||||
public CorrelationAttributePartitioner(final String partitioningAttribute) {
|
||||
this.partitioningAttribute = partitioningAttribute;
|
||||
}
|
||||
|
||||
@Override
|
||||
public QueuePartition getPartition(final FlowFileRecord flowFile, final QueuePartition[] partitions, final QueuePartition localPartition) {
|
||||
final int hash = hash(flowFile);
|
||||
|
||||
// The consistentHash method appears to always return a bucket of '1' if there are 2 possible buckets,
|
||||
// so in this case we will just use modulo division to avoid this. I suspect this is a bug with the Guava
|
||||
// implementation, but it's not clear at this point.
|
||||
final int index;
|
||||
if (partitions.length < 3) {
|
||||
index = hash % partitions.length;
|
||||
} else {
|
||||
index = Hashing.consistentHash(hash, partitions.length);
|
||||
}
|
||||
|
||||
return partitions[index];
|
||||
}
|
||||
|
||||
protected int hash(final FlowFileRecord flowFile) {
|
||||
final String partitionAttributeValue = flowFile.getAttribute(partitioningAttribute);
|
||||
return (partitionAttributeValue == null) ? 0 : partitionAttributeValue.hashCode();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isRebalanceOnClusterResize() {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isRebalanceOnFailure() {
|
||||
return false;
|
||||
}
|
||||
}
|
|
@ -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.controller.queue.clustered.partition;
|
||||
|
||||
import org.apache.nifi.controller.repository.FlowFileRecord;
|
||||
|
||||
public class FirstNodePartitioner implements FlowFilePartitioner {
|
||||
|
||||
@Override
|
||||
public QueuePartition getPartition(final FlowFileRecord flowFile, final QueuePartition[] partitions, final QueuePartition localPartition) {
|
||||
return partitions[0];
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isRebalanceOnClusterResize() {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isRebalanceOnFailure() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isPartitionStatic() {
|
||||
return true;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,53 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.nifi.controller.queue.clustered.partition;
|
||||
|
||||
import org.apache.nifi.controller.repository.FlowFileRecord;
|
||||
|
||||
public interface FlowFilePartitioner {
|
||||
|
||||
/**
|
||||
* Determines which partition the given FlowFile should go to
|
||||
*
|
||||
* @param flowFile the FlowFile to partition
|
||||
* @param partitions the partitions to choose from
|
||||
* @param localPartition the local partition, which is also included in the given array of partitions
|
||||
* @return the partition for the FlowFile
|
||||
*/
|
||||
QueuePartition getPartition(FlowFileRecord flowFile, QueuePartition[] partitions, QueuePartition localPartition);
|
||||
|
||||
/**
|
||||
* @return <code>true</code> if a change in the size of a cluster should result in re-balancing all FlowFiles in queue,
|
||||
* <code>false</code> if a change in the size of a cluster does not require re-balancing.
|
||||
*/
|
||||
boolean isRebalanceOnClusterResize();
|
||||
|
||||
/**
|
||||
* @return <code>true</code> if FlowFiles should be rebalanced to another partition if they cannot be sent to the designated peer,
|
||||
* <code>false</code> if a failure should result in the FlowFiles remaining in same partition.
|
||||
*/
|
||||
boolean isRebalanceOnFailure();
|
||||
|
||||
/**
|
||||
* @return <code>true</code> if the return value of {@link #getPartition(FlowFileRecord, QueuePartition[], QueuePartition)} will be the same
|
||||
* regardless of how many times it is called or which FlowFiles are passed.
|
||||
*/
|
||||
default boolean isPartitionStatic() {
|
||||
return false;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,42 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.nifi.controller.queue.clustered.partition;
|
||||
|
||||
import org.apache.nifi.controller.repository.FlowFileRecord;
|
||||
|
||||
public class LocalPartitionPartitioner implements FlowFilePartitioner {
|
||||
@Override
|
||||
public QueuePartition getPartition(final FlowFileRecord flowFile, final QueuePartition[] partitions, final QueuePartition localPartition) {
|
||||
return localPartition;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isRebalanceOnClusterResize() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isRebalanceOnFailure() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isPartitionStatic() {
|
||||
return true;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,108 @@
|
|||
/*
|
||||
* 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.controller.queue.clustered.partition;
|
||||
|
||||
import org.apache.nifi.controller.queue.FlowFileQueueContents;
|
||||
import org.apache.nifi.controller.queue.LocalQueuePartitionDiagnostics;
|
||||
import org.apache.nifi.controller.repository.FlowFileRecord;
|
||||
import org.apache.nifi.processor.FlowFileFilter;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* An extension of a Queue Partition that contains the methods necessary for Processors, Funnels, and Ports to interact with the Partition
|
||||
* as if it were an entire FlowFile Queue itself.
|
||||
*/
|
||||
public interface LocalQueuePartition extends QueuePartition {
|
||||
/**
|
||||
* @return <code>true</code> if the active queue is empty, <code>false</code> otherwise
|
||||
*/
|
||||
boolean isActiveQueueEmpty();
|
||||
|
||||
/**
|
||||
* @return <code>true</code> if there is at least one FlowFile that has not yet been acknowledged, <code>false</code> if all FlowFiles have been acknowledged.
|
||||
*/
|
||||
boolean isUnacknowledgedFlowFile();
|
||||
|
||||
/**
|
||||
* Returns a single FlowFile with the highest priority that is available in the partition, or <code>null</code> if no FlowFile is available
|
||||
*
|
||||
* @param expiredRecords a Set of FlowFileRecord's to which any expired records that are encountered should be added
|
||||
* @return a single FlowFile with the highest priority that is available in the partition, or <code>null</code> if no FlowFile is available
|
||||
*/
|
||||
FlowFileRecord poll(Set<FlowFileRecord> expiredRecords);
|
||||
|
||||
/**
|
||||
* Returns up to <code>maxResults</code> FlowFiles from the queue
|
||||
*
|
||||
* @param maxResults the maximum number of FlowFiles to return
|
||||
* @param expiredRecords a Set of FlowFileRecord's to which any expired records that are encountered should be added
|
||||
* @return a List of FlowFiles (possibly empty) with the highest priority FlowFiles that are available in the partition
|
||||
*/
|
||||
List<FlowFileRecord> poll(int maxResults, Set<FlowFileRecord> expiredRecords);
|
||||
|
||||
/**
|
||||
* Returns a List of FlowFiles that match the given filter
|
||||
*
|
||||
* @param filter the filter to determine whether or not a given FlowFile should be returned
|
||||
* @param expiredRecords a Set of FlowFileRecord's to which any expired records that are encountered should be added
|
||||
* @return a List of FlowFiles (possibly empty) with FlowFiles that matched the given filter
|
||||
*/
|
||||
List<FlowFileRecord> poll(FlowFileFilter filter, Set<FlowFileRecord> expiredRecords);
|
||||
|
||||
/**
|
||||
* Acknowledges that the given FlowFile has been accounted for and is no longer the responsibility of this partition
|
||||
* @param flowFile the FlowFile that has been accounted for
|
||||
*/
|
||||
void acknowledge(FlowFileRecord flowFile);
|
||||
|
||||
/**
|
||||
* Acknowledges that the given FlowFiles have been accounted for and is no longer the responsibility of this partition
|
||||
* @param flowFiles the FlowFiles that have been accounted for
|
||||
*/
|
||||
void acknowledge(Collection<FlowFileRecord> flowFiles);
|
||||
|
||||
/**
|
||||
* Returns the FlowFile with the given UUID, or <code>null</code> if the FlowFile with that UUID is not found in the partition
|
||||
*
|
||||
* @param flowFileUuid the UUID of the FlowFile
|
||||
* @return the FlowFile with the given UUID or <code>null</code> if the FlowFile cannot be found
|
||||
* @throws IOException if unable to read swapped data from a swap file
|
||||
*/
|
||||
FlowFileRecord getFlowFile(final String flowFileUuid) throws IOException;
|
||||
|
||||
/**
|
||||
* Returns the FlowFiles that can be provided as the result of as "List FlowFiles" action
|
||||
* @return a List of FlowFiles
|
||||
*/
|
||||
List<FlowFileRecord> getListableFlowFiles();
|
||||
|
||||
/**
|
||||
* Inherits the contents of another queue/partition
|
||||
* @param queueContents the contents to inherit
|
||||
*/
|
||||
void inheritQueueContents(FlowFileQueueContents queueContents);
|
||||
|
||||
/**
|
||||
* @return diagnostics information about the queue partition
|
||||
*/
|
||||
LocalQueuePartitionDiagnostics getQueueDiagnostics();
|
||||
}
|
|
@ -0,0 +1,102 @@
|
|||
/*
|
||||
* 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.controller.queue.clustered.partition;
|
||||
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.controller.queue.DropFlowFileRequest;
|
||||
import org.apache.nifi.controller.queue.FlowFileQueueContents;
|
||||
import org.apache.nifi.controller.queue.QueueSize;
|
||||
import org.apache.nifi.controller.repository.FlowFileRecord;
|
||||
import org.apache.nifi.controller.repository.SwapSummary;
|
||||
import org.apache.nifi.flowfile.FlowFilePrioritizer;
|
||||
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
|
||||
/**
|
||||
* Represents a portion of a FlowFile Queue such that a FlowFile Queue can be broken into
|
||||
* a local queue partition and 0 or more Remote Queue Partitions.
|
||||
*/
|
||||
public interface QueuePartition {
|
||||
/**
|
||||
* Discovers any FlowFiles that have been swapped out, returning a summary of the swap files' contents
|
||||
* @return a summary of the swap files' contents
|
||||
*/
|
||||
SwapSummary recoverSwappedFlowFiles();
|
||||
|
||||
/**
|
||||
* @return the Node Identifier that this Queue Partition corresponds to, or and empty Optional if the Node Identifier is not yet known.
|
||||
*/
|
||||
Optional<NodeIdentifier> getNodeIdentifier();
|
||||
|
||||
/**
|
||||
* @return the name of the Partition that is used when serializing swap flowfiles in order to denote that a swap file belongs to this partition
|
||||
*/
|
||||
String getSwapPartitionName();
|
||||
|
||||
/**
|
||||
* Adds the given FlowFile to this partition
|
||||
* @param flowFile the FlowFile to add
|
||||
*/
|
||||
void put(FlowFileRecord flowFile);
|
||||
|
||||
/**
|
||||
* Adds the given FlowFiles to this partition
|
||||
* @param flowFiles the FlowFiles to add
|
||||
*/
|
||||
void putAll(Collection<FlowFileRecord> flowFiles);
|
||||
|
||||
/**
|
||||
* Drops the FlowFiles in this partition
|
||||
* @param dropRequest the FlowFile Drop Request
|
||||
* @param requestor the user making the request
|
||||
*/
|
||||
void dropFlowFiles(DropFlowFileRequest dropRequest, String requestor);
|
||||
|
||||
/**
|
||||
* Updates the prioritizers to use when queueing data
|
||||
* @param newPriorities the new priorities
|
||||
*/
|
||||
void setPriorities(List<FlowFilePrioritizer> newPriorities);
|
||||
|
||||
/**
|
||||
* Starts distributing FlowFiles to their desired destinations
|
||||
*
|
||||
* @param flowFilePartitioner the Partitioner that is being used to determine which FlowFiles should belong to this Partition
|
||||
*/
|
||||
void start(FlowFilePartitioner flowFilePartitioner);
|
||||
|
||||
/**
|
||||
* Stop distributing FlowFiles to other nodes in the cluster. This does not interrupt any active transactions but will cause the
|
||||
* partition to not create any more transactions until it is started again.
|
||||
*/
|
||||
void stop();
|
||||
|
||||
/**
|
||||
* Provides a {@link FlowFileQueueContents} that can be transferred to another partition
|
||||
* @param newPartitionName the name of the partition to which the data is being transferred (see {@link #getSwapPartitionName()}.
|
||||
* @return the contents of the queue
|
||||
*/
|
||||
FlowFileQueueContents packageForRebalance(String newPartitionName);
|
||||
|
||||
/**
|
||||
* @return the current size of the partition's queue
|
||||
*/
|
||||
QueueSize size();
|
||||
}
|
|
@ -0,0 +1,45 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.nifi.controller.queue.clustered.partition;
|
||||
|
||||
import org.apache.nifi.controller.queue.FlowFileQueueContents;
|
||||
import org.apache.nifi.controller.repository.FlowFileRecord;
|
||||
|
||||
import java.util.Collection;
|
||||
|
||||
/**
|
||||
* A partition whose sole job it is to redistribute FlowFiles to the appropriate partitions.
|
||||
*/
|
||||
public interface RebalancingPartition extends QueuePartition {
|
||||
|
||||
/**
|
||||
* Inherits all of the FlowFiles, including FlowFiles that have been swaped out, in order to
|
||||
* redistribute them across the cluster
|
||||
*
|
||||
* @param queueContents the contents of a FlowFileQueue (or partition)
|
||||
*/
|
||||
void rebalance(FlowFileQueueContents queueContents);
|
||||
|
||||
/**
|
||||
* Inherits all of the givne FlowFiles in order to redistribute them across the cluster
|
||||
*
|
||||
* @param flowFiles the FlowFiles to redistribute
|
||||
*/
|
||||
void rebalance(Collection<FlowFileRecord> flowFiles);
|
||||
|
||||
}
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue