浏览代码

Remove parent-task bans on channels disconnect (#66066)

Like #56620, this change relies on channel disconnect instead of node 
leave events to remove parent-task ban markers.

Relates #65443
Relates #56620
Nhat Nguyen 4 年之前
父节点
当前提交
0a574589af

+ 54 - 0
server/src/internalClusterTest/java/org/elasticsearch/action/admin/cluster/node/tasks/CancellableTasksIT.java

@@ -55,6 +55,7 @@ import org.elasticsearch.tasks.TaskManager;
 import org.elasticsearch.test.ESIntegTestCase;
 import org.elasticsearch.test.InternalTestCluster;
 import org.elasticsearch.threadpool.ThreadPool;
+import org.elasticsearch.transport.Transport;
 import org.elasticsearch.transport.TransportException;
 import org.elasticsearch.transport.TransportResponseHandler;
 import org.elasticsearch.transport.TransportService;
@@ -307,6 +308,58 @@ public class CancellableTasksIT extends ESIntegTestCase {
         }
     }
 
+    public void testRemoveBanParentsOnDisconnect() throws Exception {
+        Set<DiscoveryNode> nodes = StreamSupport.stream(clusterService().state().nodes().spliterator(), false).collect(Collectors.toSet());
+        final TestRequest rootRequest = generateTestRequest(nodes, 0, between(1, 4));
+        client().execute(TransportTestAction.ACTION, rootRequest);
+        Set<TestRequest> pendingRequests = allowPartialRequest(rootRequest);
+        TaskId rootTaskId = getRootTaskId(rootRequest);
+        ActionFuture<CancelTasksResponse> cancelFuture = client().admin().cluster().prepareCancelTasks()
+            .setTaskId(rootTaskId).waitForCompletion(true).execute();
+        try {
+            assertBusy(() -> {
+                for (DiscoveryNode node : nodes) {
+                    TaskManager taskManager = internalCluster().getInstance(TransportService.class, node.getName()).getTaskManager();
+                    Set<TaskId> expectedBans = new HashSet<>();
+                    for (TestRequest req : pendingRequests) {
+                        if (req.node.equals(node)) {
+                            List<Task> childTasks = taskManager.getTasks().values().stream()
+                                .filter(t -> t.getParentTaskId() != null && t.getDescription().equals(req.taskDescription()))
+                                .collect(Collectors.toList());
+                            assertThat(childTasks, hasSize(1));
+                            CancellableTask childTask = (CancellableTask) childTasks.get(0);
+                            assertTrue(childTask.isCancelled());
+                            expectedBans.add(childTask.getParentTaskId());
+                        }
+                    }
+                    assertThat(taskManager.getBannedTaskIds(), equalTo(expectedBans));
+                }
+            }, 30, TimeUnit.SECONDS);
+
+            final Set<TaskId> bannedParents = new HashSet<>();
+            for (DiscoveryNode node : nodes) {
+                TaskManager taskManager = internalCluster().getInstance(TransportService.class, node.getName()).getTaskManager();
+                bannedParents.addAll(taskManager.getBannedTaskIds());
+            }
+            // Disconnect some outstanding child connections
+            for (DiscoveryNode node : nodes) {
+                TaskManager taskManager = internalCluster().getInstance(TransportService.class, node.getName()).getTaskManager();
+                for (TaskId bannedParent : bannedParents) {
+                    if (bannedParent.getNodeId().equals(node.getId()) && randomBoolean()) {
+                        Collection<Transport.Connection> childConns = taskManager.startBanOnChildTasks(bannedParent.getId(), () -> {});
+                        for (Transport.Connection connection : randomSubsetOf(childConns)) {
+                            connection.close();
+                        }
+                    }
+                }
+            }
+        } finally {
+            allowEntireRequest(rootRequest);
+            cancelFuture.actionGet();
+            ensureAllBansRemoved();
+        }
+    }
+
     static TaskId getRootTaskId(TestRequest request) throws Exception {
         SetOnce<TaskId> taskId = new SetOnce<>();
         assertBusy(() -> {
@@ -326,6 +379,7 @@ public class CancellableTasksIT extends ESIntegTestCase {
             rootTask.actionGet();
         } catch (Exception e) {
             final Throwable cause = ExceptionsHelper.unwrap(e, TaskCancelledException.class);
+            assertNotNull(cause);
             assertThat(cause.getMessage(), anyOf(
                 equalTo("The parent task was cancelled, shouldn't start any child tasks"),
                 containsString("Task cancelled before it started:"),

+ 1 - 1
server/src/main/java/org/elasticsearch/tasks/TaskCancellationService.java

@@ -242,7 +242,7 @@ public class TaskCancellationService {
             if (request.ban) {
                 logger.debug("Received ban for the parent [{}] on the node [{}], reason: [{}]", request.parentTaskId,
                     localNodeId(), request.reason);
-                final List<CancellableTask> childTasks = taskManager.setBan(request.parentTaskId, request.reason);
+                final List<CancellableTask> childTasks = taskManager.setBan(request.parentTaskId, request.reason,  channel);
                 final GroupedActionListener<Void> listener = new GroupedActionListener<>(
                     new ChannelActionListener<>(channel, BAN_PARENT_ACTION_NAME, request).map(r -> TransportResponse.Empty.INSTANCE),
                     childTasks.size() + 1);

+ 108 - 27
server/src/main/java/org/elasticsearch/tasks/TaskManager.java

@@ -29,6 +29,7 @@ import org.elasticsearch.Assertions;
 import org.elasticsearch.ElasticsearchException;
 import org.elasticsearch.ElasticsearchTimeoutException;
 import org.elasticsearch.ExceptionsHelper;
+import org.elasticsearch.Version;
 import org.elasticsearch.action.ActionListener;
 import org.elasticsearch.action.ActionRequest;
 import org.elasticsearch.action.ActionResponse;
@@ -47,8 +48,11 @@ import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
 import org.elasticsearch.common.util.concurrent.ConcurrentMapLong;
 import org.elasticsearch.common.util.concurrent.ThreadContext;
 import org.elasticsearch.threadpool.ThreadPool;
+import org.elasticsearch.transport.TaskTransportChannel;
 import org.elasticsearch.transport.TcpChannel;
+import org.elasticsearch.transport.TcpTransportChannel;
 import org.elasticsearch.transport.Transport;
+import org.elasticsearch.transport.TransportChannel;
 import org.elasticsearch.transport.TransportService;
 
 import java.io.IOException;
@@ -56,6 +60,7 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
@@ -66,6 +71,7 @@ import java.util.concurrent.Semaphore;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.function.BiConsumer;
+import java.util.function.Consumer;
 import java.util.stream.Collectors;
 import java.util.stream.StreamSupport;
 
@@ -92,7 +98,7 @@ public class TaskManager implements ClusterStateApplier {
 
     private final AtomicLong taskIdGenerator = new AtomicLong();
 
-    private final Map<TaskId, String> banedParents = new ConcurrentHashMap<>();
+    private final Map<TaskId, Ban> bannedParents = new ConcurrentHashMap<>();
 
     private TaskResultsService taskResultsService;
 
@@ -196,13 +202,13 @@ public class TaskManager implements ClusterStateApplier {
         CancellableTaskHolder oldHolder = cancellableTasks.put(task.getId(), holder);
         assert oldHolder == null;
         // Check if this task was banned before we start it. The empty check is used to avoid
-        // computing the hash code of the parent taskId as most of the time banedParents is empty.
-        if (task.getParentTaskId().isSet() && banedParents.isEmpty() == false) {
-            String reason = banedParents.get(task.getParentTaskId());
-            if (reason != null) {
+        // computing the hash code of the parent taskId as most of the time bannedParents is empty.
+        if (task.getParentTaskId().isSet() && bannedParents.isEmpty() == false) {
+            final Ban ban = bannedParents.get(task.getParentTaskId());
+            if (ban != null) {
                 try {
-                    holder.cancel(reason);
-                    throw new TaskCancelledException("Task cancelled before it started: " + reason);
+                    holder.cancel(ban.reason);
+                    throw new TaskCancelledException("Task cancelled before it started: " + ban.reason);
                 } finally {
                     // let's clean up the registration
                     unregister(task);
@@ -383,7 +389,7 @@ public class TaskManager implements ClusterStateApplier {
      * Will be used in task manager stats and for debugging.
      */
     public int getBanCount() {
-        return banedParents.size();
+        return bannedParents.size();
     }
 
     /**
@@ -392,14 +398,27 @@ public class TaskManager implements ClusterStateApplier {
      * This method is called when a parent task that has children is cancelled.
      * @return a list of pending cancellable child tasks
      */
-    public List<CancellableTask> setBan(TaskId parentTaskId, String reason) {
+    public List<CancellableTask> setBan(TaskId parentTaskId, String reason, TransportChannel channel) {
         logger.trace("setting ban for the parent task {} {}", parentTaskId, reason);
-
-        // Set the ban first, so the newly created tasks cannot be registered
-        synchronized (banedParents) {
-            if (lastDiscoveryNodes.nodeExists(parentTaskId.getNodeId())) {
-                // Only set the ban if the node is the part of the cluster
-                banedParents.put(parentTaskId, reason);
+        synchronized (bannedParents) {
+            if (channel.getVersion().onOrAfter(Version.V_8_0_0)) {
+                final Ban ban = bannedParents.computeIfAbsent(parentTaskId, k -> new Ban(reason, true));
+                assert ban.perChannel : "not a ban per channel";
+                while (channel instanceof TaskTransportChannel) {
+                    channel = ((TaskTransportChannel) channel).getChannel();
+                }
+                if (channel instanceof TcpTransportChannel) {
+                    startTrackingChannel(((TcpTransportChannel) channel).getChannel(), ban::registerChannel);
+                } else {
+                    assert channel.getChannelType().equals("direct") : "expect direct channel; got [" + channel + "]";
+                    ban.registerChannel(DIRECT_CHANNEL_TRACKER);
+                }
+            } else {
+                if (lastDiscoveryNodes.nodeExists(parentTaskId.getNodeId())) {
+                    // Only set the ban if the node is the part of the cluster
+                    final Ban existing = bannedParents.put(parentTaskId, new Ban(reason, false));
+                    assert existing == null || existing.perChannel == false : "not a ban per node";
+                }
             }
         }
         return cancellableTasks.values().stream()
@@ -415,12 +434,52 @@ public class TaskManager implements ClusterStateApplier {
      */
     public void removeBan(TaskId parentTaskId) {
         logger.trace("removing ban for the parent task {}", parentTaskId);
-        banedParents.remove(parentTaskId);
+        bannedParents.remove(parentTaskId);
     }
 
     // for testing
     public Set<TaskId> getBannedTaskIds() {
-        return Collections.unmodifiableSet(banedParents.keySet());
+        return Collections.unmodifiableSet(bannedParents.keySet());
+    }
+
+    private class Ban {
+        final String reason;
+        final boolean perChannel; // TODO: Remove this in 8.0
+        final Set<ChannelPendingTaskTracker> channels;
+
+        Ban(String reason, boolean perChannel) {
+            assert Thread.holdsLock(bannedParents);
+            this.reason = reason;
+            this.perChannel = perChannel;
+            if (perChannel) {
+                this.channels = new HashSet<>();
+            } else {
+                this.channels = Set.of();
+            }
+        }
+
+        void registerChannel(ChannelPendingTaskTracker channel) {
+            assert Thread.holdsLock(bannedParents);
+            assert perChannel : "not a ban per channel";
+            channels.add(channel);
+        }
+
+        boolean unregisterChannel(ChannelPendingTaskTracker channel) {
+            assert Thread.holdsLock(bannedParents);
+            assert perChannel : "not a ban per channel";
+            return channels.remove(channel);
+        }
+
+        int registeredChannels() {
+            assert Thread.holdsLock(bannedParents);
+            assert perChannel : "not a ban per channel";
+            return channels.size();
+        }
+
+        @Override
+        public String toString() {
+            return "Ban{" + "reason='" + reason + '\'' + ", perChannel=" + perChannel + ", channels=" + channels + '}';
+        }
     }
 
     /**
@@ -444,15 +503,15 @@ public class TaskManager implements ClusterStateApplier {
     public void applyClusterState(ClusterChangedEvent event) {
         lastDiscoveryNodes = event.state().getNodes();
         if (event.nodesRemoved()) {
-            synchronized (banedParents) {
+            synchronized (bannedParents) {
                 lastDiscoveryNodes = event.state().getNodes();
                 // Remove all bans that were registered by nodes that are no longer in the cluster state
-                Iterator<TaskId> banIterator = banedParents.keySet().iterator();
+                final Iterator<Map.Entry<TaskId, Ban>> banIterator = bannedParents.entrySet().iterator();
                 while (banIterator.hasNext()) {
-                    TaskId taskId = banIterator.next();
-                    if (lastDiscoveryNodes.nodeExists(taskId.getNodeId()) == false) {
-                        logger.debug("Removing ban for the parent [{}] on the node [{}], reason: the parent node is gone", taskId,
-                            event.state().getNodes().getLocalNode());
+                    final Map.Entry<TaskId, Ban> ban = banIterator.next();
+                    if (ban.getValue().perChannel == false && lastDiscoveryNodes.nodeExists(ban.getKey().getNodeId()) == false) {
+                        logger.debug("Removing ban for the parent [{}] on the node [{}], reason: the parent node is gone",
+                            ban.getKey(), event.state().getNodes().getLocalNode());
                         banIterator.remove();
                     }
                 }
@@ -619,11 +678,16 @@ public class TaskManager implements ClusterStateApplier {
      */
     public Releasable startTrackingCancellableChannelTask(TcpChannel channel, CancellableTask task) {
         assert cancellableTasks.containsKey(task.getId()) : "task [" + task.getId() + "] is not registered yet";
+        final ChannelPendingTaskTracker tracker = startTrackingChannel(channel, trackerChannel -> trackerChannel.addTask(task));
+        return () -> tracker.removeTask(task);
+    }
+
+    private ChannelPendingTaskTracker startTrackingChannel(TcpChannel channel, Consumer<ChannelPendingTaskTracker> onRegister) {
         final ChannelPendingTaskTracker tracker = channelPendingTaskTrackers.compute(channel, (k, curr) -> {
             if (curr == null) {
                 curr = new ChannelPendingTaskTracker();
             }
-            curr.addTask(task);
+            onRegister.accept(curr);
             return curr;
         });
         if (tracker.registered.compareAndSet(false, true)) {
@@ -631,13 +695,13 @@ public class TaskManager implements ClusterStateApplier {
                 r -> {
                     final ChannelPendingTaskTracker removedTracker = channelPendingTaskTrackers.remove(channel);
                     assert removedTracker == tracker;
-                    cancelTasksOnChannelClosed(tracker.drainTasks());
+                    onChannelClosed(tracker);
                 },
                 e -> {
                     assert false : new AssertionError("must not be here", e);
                 }));
         }
-        return () -> tracker.removeTask(task);
+        return tracker;
     }
 
     // for testing
@@ -645,6 +709,8 @@ public class TaskManager implements ClusterStateApplier {
         return channelPendingTaskTrackers.size();
     }
 
+    private static final ChannelPendingTaskTracker DIRECT_CHANNEL_TRACKER = new ChannelPendingTaskTracker();
+
     private static class ChannelPendingTaskTracker {
         final AtomicBoolean registered = new AtomicBoolean();
         final Semaphore permits = Assertions.ENABLED ? new Semaphore(Integer.MAX_VALUE) : null;
@@ -678,7 +744,8 @@ public class TaskManager implements ClusterStateApplier {
         }
     }
 
-    private void cancelTasksOnChannelClosed(Set<CancellableTask> tasks) {
+    private void onChannelClosed(ChannelPendingTaskTracker channel) {
+        final Set<CancellableTask> tasks = channel.drainTasks();
         if (tasks.isEmpty() == false) {
             threadPool.generic().execute(new AbstractRunnable() {
                 @Override
@@ -694,6 +761,20 @@ public class TaskManager implements ClusterStateApplier {
                 }
             });
         }
+
+        // Unregister the closing channel and remove bans whose has no registered channels
+        synchronized (bannedParents) {
+            final Iterator<Map.Entry<TaskId, Ban>> iterator = bannedParents.entrySet().iterator();
+            while (iterator.hasNext()) {
+                final Map.Entry<TaskId, Ban> entry = iterator.next();
+                final Ban ban = entry.getValue();
+                if (ban.perChannel) {
+                    if (ban.unregisterChannel(channel) && entry.getValue().registeredChannels() == 0) {
+                        iterator.remove();
+                    }
+                }
+            }
+        }
     }
 
     public void cancelTaskAndDescendants(CancellableTask task, String reason, boolean waitForCompletion, ActionListener<Void> listener) {

+ 7 - 1
server/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/CancellableTasksTests.java

@@ -20,6 +20,7 @@ package org.elasticsearch.action.admin.cluster.node.tasks;
 
 import com.carrotsearch.randomizedtesting.RandomizedContext;
 import com.carrotsearch.randomizedtesting.generators.RandomNumbers;
+import org.elasticsearch.Version;
 import org.elasticsearch.action.ActionListener;
 import org.elasticsearch.action.admin.cluster.node.tasks.cancel.CancelTasksAction;
 import org.elasticsearch.action.admin.cluster.node.tasks.cancel.CancelTasksRequest;
@@ -41,6 +42,8 @@ import org.elasticsearch.tasks.TaskId;
 import org.elasticsearch.tasks.TaskInfo;
 import org.elasticsearch.tasks.TaskManager;
 import org.elasticsearch.threadpool.ThreadPool;
+import org.elasticsearch.transport.FakeTcpChannel;
+import org.elasticsearch.transport.TestTransportChannels;
 import org.elasticsearch.transport.TransportRequest;
 import org.elasticsearch.transport.TransportService;
 
@@ -356,7 +359,10 @@ public class CancellableTasksTests extends TaskManagerTestCase {
         CancellableNodesRequest parentRequest = new CancellableNodesRequest("parent");
         final Task parentTask = taskManager.register("test", "test", parentRequest);
         final TaskId parentTaskId = parentTask.taskInfo(testNodes[0].getNodeId(), false).getTaskId();
-        taskManager.setBan(new TaskId(testNodes[0].getNodeId(), parentTask.getId()), "test");
+        taskManager.setBan(new TaskId(testNodes[0].getNodeId(), parentTask.getId()), "test",
+            TestTransportChannels.newFakeTcpTransportChannel(
+                testNodes[0].getNodeId(), new FakeTcpChannel(), threadPool,
+                "test", randomNonNegativeLong(), Version.CURRENT));
         CancellableNodesRequest childRequest = new CancellableNodesRequest("child");
         childRequest.setParentTask(parentTaskId);
         CancellableTestNodesAction testAction = new CancellableTestNodesAction("internal:testAction", threadPool, testNodes[1]

+ 54 - 5
server/src/test/java/org/elasticsearch/tasks/TaskManagerTests.java

@@ -19,10 +19,12 @@
 
 package org.elasticsearch.tasks;
 
+import org.elasticsearch.Version;
 import org.elasticsearch.action.ActionListener;
 import org.elasticsearch.action.admin.cluster.node.tasks.TransportTasksActionTests;
 import org.elasticsearch.common.lease.Releasable;
 import org.elasticsearch.common.lease.Releasables;
+import org.elasticsearch.common.network.CloseableChannel;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.unit.TimeValue;
 import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
@@ -31,6 +33,8 @@ import org.elasticsearch.threadpool.TestThreadPool;
 import org.elasticsearch.threadpool.ThreadPool;
 import org.elasticsearch.transport.FakeTcpChannel;
 import org.elasticsearch.transport.TcpChannel;
+import org.elasticsearch.transport.TcpTransportChannel;
+import org.elasticsearch.transport.TestTransportChannels;
 import org.elasticsearch.transport.TransportRequest;
 import org.elasticsearch.transport.TransportService;
 import org.junit.After;
@@ -46,7 +50,9 @@ import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.Phaser;
 import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
 
+import static org.hamcrest.Matchers.empty;
 import static org.hamcrest.Matchers.equalTo;
 import static org.hamcrest.Matchers.everyItem;
 import static org.hamcrest.Matchers.in;
@@ -149,11 +155,18 @@ public class TaskManagerTests extends ESTestCase {
                 int iterations = randomIntBetween(50, 500);
                 for (int i = 0; i < iterations; i++) {
                     final FakeTcpChannel channel = randomFrom(channels);
-                    final Task task = taskManager.register("transport", "test", new CancellableRequest(threadName + ":" + i));
-                    expectedCancelledTasks.add(task);
-                    taskManager.startTrackingCancellableChannelTask(channel, (CancellableTask) task);
-                    if (randomInt(100) < 5) {
-                        randomFrom(channels).close();
+                    if (randomBoolean()) {
+                        final Task task = taskManager.register("transport", "test", new CancellableRequest(threadName + ":" + i));
+                        expectedCancelledTasks.add(task);
+                        taskManager.startTrackingCancellableChannelTask(channel, (CancellableTask) task);
+                        if (randomInt(100) < 5) {
+                            randomFrom(channels).close();
+                        }
+                    } else {
+                        final TaskId taskId = new TaskId("node", between(1, 100));
+                        final TcpTransportChannel tcpTransportChannel = TestTransportChannels.newFakeTcpTransportChannel(
+                            "node-" + i, channel, threadPool, "action-" + i, randomIntBetween(0, 1000), Version.CURRENT);
+                        taskManager.setBan(taskId, "test", tcpTransportChannel);
                     }
                 }
             });
@@ -166,6 +179,42 @@ public class TaskManagerTests extends ESTestCase {
             thread.join();
         }
         assertBusy(() -> assertThat(cancelledTasks, equalTo(expectedCancelledTasks)), 1, TimeUnit.MINUTES);
+        assertBusy(() -> assertThat(taskManager.getBannedTaskIds(), empty()), 1, TimeUnit.MINUTES);
+        assertThat(taskManager.numberOfChannelPendingTaskTrackers(), equalTo(0));
+    }
+
+    public void testRemoveBansOnChannelDisconnects() throws Exception {
+        final TaskManager taskManager = new TaskManager(Settings.EMPTY, threadPool, Set.of());
+        taskManager.setTaskCancellationService(new TaskCancellationService(mock(TransportService.class)) {
+            @Override
+            void cancelTaskAndDescendants(CancellableTask task, String reason, boolean waitForCompletion, ActionListener<Void> listener) {
+            }
+        });
+        Map<TaskId, Set<TcpChannel>> installedBans = new HashMap<>();
+        FakeTcpChannel[] channels = new FakeTcpChannel[randomIntBetween(1, 10)];
+        for (int i = 0; i < channels.length; i++) {
+            channels[i] = new SingleThreadedTcpChannel();
+        }
+        int iterations = randomIntBetween(1, 200);
+        for (int i = 0; i < iterations; i++) {
+            final FakeTcpChannel channel = randomFrom(channels);
+            if (channel.isOpen() && randomBoolean()) {
+                channel.close();
+            }
+            TaskId taskId = new TaskId("node-" + randomIntBetween(1, 3), randomIntBetween(1, 100));
+            installedBans.computeIfAbsent(taskId, t -> new HashSet<>()).add(channel);
+            taskManager.setBan(taskId, "test", TestTransportChannels.newFakeTcpTransportChannel(
+                "node", channel, threadPool, "action", randomIntBetween(1, 10000), Version.CURRENT));
+        }
+        final Set<TaskId> expectedBannedTasks = installedBans.entrySet().stream()
+            .filter(e -> e.getValue().stream().anyMatch(CloseableChannel::isOpen))
+            .map(Map.Entry::getKey)
+            .collect(Collectors.toSet());
+        assertBusy(() -> assertThat(taskManager.getBannedTaskIds(), equalTo(expectedBannedTasks)), 30, TimeUnit.SECONDS);
+        for (FakeTcpChannel channel : channels) {
+            channel.close();
+        }
+        assertBusy(() -> assertThat(taskManager.getBannedTaskIds(), empty()));
         assertThat(taskManager.numberOfChannelPendingTaskTrackers(), equalTo(0));
     }
 

+ 34 - 0
test/framework/src/main/java/org/elasticsearch/transport/TestTransportChannels.java

@@ -0,0 +1,34 @@
+/*
+ * Licensed to Elasticsearch under one or more contributor
+ * license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright
+ * ownership. Elasticsearch 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.elasticsearch.transport;
+
+import org.elasticsearch.Version;
+import org.elasticsearch.common.util.BigArrays;
+import org.elasticsearch.threadpool.ThreadPool;
+
+public class TestTransportChannels {
+
+    public static TcpTransportChannel newFakeTcpTransportChannel(String nodeName, TcpChannel channel, ThreadPool threadPool,
+                                                                 String action, long requestId, Version version) {
+        return new TcpTransportChannel(
+            new OutboundHandler(nodeName, version, new StatsTracker(), threadPool, BigArrays.NON_RECYCLING_INSTANCE),
+            channel, action, requestId, version, false, false, () -> {});
+    }
+}