Github user apsaltis commented on a diff in the pull request:
https://github.com/apache/nifi/pull/1177#discussion_r87223668
--- Diff:
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetTCP.java
---
@@ -0,0 +1,668 @@
+/*
+ * 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.processors.standard;
+
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.components.Validator;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.net.SocketAddress;
+import java.net.SocketTimeoutException;
+import java.net.StandardSocketOptions;
+import java.nio.ByteBuffer;
+import java.nio.CharBuffer;
+import java.nio.channels.AlreadyConnectedException;
+import java.nio.channels.ConnectionPendingException;
+import java.nio.channels.SocketChannel;
+import java.nio.channels.UnresolvedAddressException;
+import java.nio.channels.UnsupportedAddressTypeException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetDecoder;
+import java.nio.charset.StandardCharsets;
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+@SupportsBatching
+@SideEffectFree
+@Tags({"get", "fetch", "poll", "tcp", "ingest", "source", "input"})
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@CapabilityDescription("Connects over TCP to the provided server. When
receiving data this will writes either the" +
+ " full receive buffer or messages based on demarcator to the
content of a FlowFile. ")
+public class GetTCP extends AbstractProcessor {
+
+ private static final Validator ENDPOINT_VALIDATOR = new Validator() {
+ @Override
+ public ValidationResult validate(final String subject, final
String value, final ValidationContext context) {
+ if (null == value || value.isEmpty()) {
+ return new
ValidationResult.Builder().subject(subject).input(value).valid(false).explanation(subject
+ " cannot be empty").build();
+ }
+ //The format should be <host>:<port>{,<host>:<port>}
+ //first split on ,
+ final String[] hostPortPairs = value.split(",");
+ boolean validHostPortPairs = true;
+ String reason = "";
+ String offendingSubject = subject;
+
+ if(0 == hostPortPairs.length){
+ return new
ValidationResult.Builder().subject(subject).input(value).valid(false).explanation(offendingSubject
+ " cannot be empty").build();
+ }
+ for (final String hostPortPair : hostPortPairs) {
+ offendingSubject = hostPortPair;
+ //split pair
+ if (hostPortPair.isEmpty()) {
+ validHostPortPairs = false;
+ reason = "endpoint is empty";
+ break;
+ }
+ if (!hostPortPair.contains(":")) {
+ validHostPortPairs = false;
+ reason = "endpoint pair does not contain valid
delimiter";
+ break;
+ }
+ final String[] parts = hostPortPair.split(":");
+
+ if (1 == parts.length) {
+ validHostPortPairs = false;
+ reason = "could not determine the port";
+ break;
+ } else {
+ try {
+ final int intVal = Integer.parseInt(parts[1]);
+
+ if (intVal <= 0) {
+ reason = "not a positive value";
+ validHostPortPairs = false;
+ break;
+ }
+ } catch (final NumberFormatException e) {
+ reason = "not a valid integer";
+ validHostPortPairs = false;
+ break;
+ }
+ }
+ //if we already have a bad pair then exit now.
+ if (!validHostPortPairs) {
+ break;
+ }
+ }
+ return new
ValidationResult.Builder().subject(offendingSubject).input(value).explanation(reason).valid(validHostPortPairs).build();
+ }
+ };
+
+ public static final PropertyDescriptor ENDPOINT_LIST = new
PropertyDescriptor
+ .Builder().name("Endpoint List")
+ .description("A comma delimited list of the servers to connect
to. The format should be " +
+ "<server_address>:<port>. Only one server will be
connected to at a time, the others " +
+ "will be used as fail overs.")
+ .required(true)
+ .addValidator(ENDPOINT_VALIDATOR)
+ .build();
+
+ public static final PropertyDescriptor FAILOVER_ENDPOINT = new
PropertyDescriptor
+ .Builder().name("Failover Endpoint")
+ .description("A failover server to connect to if one of the
main ones is unreachable after the connection " +
+ "attempt count. The format should be
<server_address>:<port>.")
--- End diff --
There are actually two different use case in play. That first is that there
is a single server you want to connect to and get data from and if that fails
then you want to connect to a backup server. I can see how the endpoint list
maybe used to satisfy this. However, there is a second use case which may not
be apparent. In the second case the endpoint list forms a logical grouping of
servers to connect to simultaneously and receive information. Thus, they are
all active and do not serve as a backup. In this case the failover server is
used when connectivity to one of the endpoint servers fails.
I am totally open to other ways of handling the two use cases if this is
confusing/misleading/or not as clean as it could be.
---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---