From 60202a63da00dcfbbc954f31f343eb4ce14ef917 Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Mon, 23 Sep 2024 22:32:58 +0800 Subject: [PATCH 01/12] Pipe: add IoTDBConnector sender port configuration --- .../tool/tsfile/ImportTsFileRemotely.java | 6 +- .../iotdb/rpc/TNonblockingSocketWrapper.java | 20 +- .../IoTDBConfigNodeSyncClientManager.java | 12 +- .../protocol/IoTDBConfigRegionConnector.java | 12 +- .../IoTDBDataNodeAsyncClientManager.java | 20 +- .../IoTDBDataNodeSyncClientManager.java | 12 +- .../legacy/IoTDBLegacyPipeConnector.java | 6 +- .../async/IoTDBDataRegionAsyncConnector.java | 6 +- .../sync/IoTDBDataNodeSyncConnector.java | 12 +- .../sender/TwoStageAggregateSender.java | 4 + .../commons/client/ClientPoolFactory.java | 27 +- .../AsyncPipeDataTransferServiceClient.java | 40 ++- .../client/util/PortUtilizationManager.java | 290 ++++++++++++++++++ .../constant/PipeConnectorConstant.java | 15 + .../connector/client/IoTDBClientManager.java | 20 +- .../connector/client/IoTDBSyncClient.java | 40 ++- .../client/IoTDBSyncClientManager.java | 20 +- .../protocol/IoTDBAirGapConnector.java | 19 +- .../connector/protocol/IoTDBConnector.java | 111 +++++++ .../protocol/IoTDBSslSyncConnector.java | 12 +- 20 files changed, 675 insertions(+), 29 deletions(-) create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/util/PortUtilizationManager.java diff --git a/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/tsfile/ImportTsFileRemotely.java b/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/tsfile/ImportTsFileRemotely.java index 8f3c22f9e0b9..01c7f29c6e1a 100644 --- a/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/tsfile/ImportTsFileRemotely.java +++ b/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/tsfile/ImportTsFileRemotely.java @@ -301,7 +301,11 @@ private void initClient() { getEndPoint().getPort(), false, "", - ""); + "", + false, + 0, + 0, + null); } catch (final TTransportException e) { throw new PipeException("Sync client init error because " + e.getMessage()); } diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TNonblockingSocketWrapper.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TNonblockingSocketWrapper.java index 5f2b6cde2f4f..4a25306cdc01 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TNonblockingSocketWrapper.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TNonblockingSocketWrapper.java @@ -23,6 +23,7 @@ import org.apache.thrift.transport.TTransportException; import java.io.IOException; +import java.net.InetSocketAddress; import java.nio.channels.SocketChannel; /** @@ -42,7 +43,24 @@ public static TNonblockingSocket wrap(String host, int port) throws IOException public static TNonblockingSocket wrap(String host, int port, int timeout) throws IOException { try { - return new TNonblockingSocket(host, port, timeout); + TNonblockingSocket nonblockingSocket = new TNonblockingSocket(host, port, timeout); + nonblockingSocket.getSocketChannel().bind(new InetSocketAddress(0)); + return nonblockingSocket; + } catch (TTransportException e) { + // never happen + return null; + } + } + + public static TNonblockingSocket wrap( + String host, int port, int timeout, boolean isCustomSendPortDefined, int sendPort) + throws IOException { + try { + TNonblockingSocket nonblockingSocket = new TNonblockingSocket(host, port, timeout); + if (isCustomSendPortDefined) { + nonblockingSocket.getSocketChannel().bind(new InetSocketAddress(sendPort)); + } + return nonblockingSocket; } catch (TTransportException e) { // never happen return null; diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/connector/client/IoTDBConfigNodeSyncClientManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/connector/client/IoTDBConfigNodeSyncClientManager.java index b2967e3bdf86..a305e2912a23 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/connector/client/IoTDBConfigNodeSyncClientManager.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/connector/client/IoTDBConfigNodeSyncClientManager.java @@ -40,7 +40,11 @@ public IoTDBConfigNodeSyncClientManager( String trustStorePwd, String loadBalanceStrategy, boolean shouldReceiverConvertOnTypeMismatch, - String loadTsFileStrategy) { + String loadTsFileStrategy, + boolean isCustomSendPortDefined, + int minSendPortRange, + int maxSendPortRange, + List candidatePorts) { super( endPoints, useSSL, @@ -49,7 +53,11 @@ public IoTDBConfigNodeSyncClientManager( false, loadBalanceStrategy, shouldReceiverConvertOnTypeMismatch, - loadTsFileStrategy); + loadTsFileStrategy, + isCustomSendPortDefined, + minSendPortRange, + maxSendPortRange, + candidatePorts); } @Override diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/connector/protocol/IoTDBConfigRegionConnector.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/connector/protocol/IoTDBConfigRegionConnector.java index 3e4ca8ac9c50..476eb59b882c 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/connector/protocol/IoTDBConfigRegionConnector.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/connector/protocol/IoTDBConfigRegionConnector.java @@ -64,7 +64,11 @@ protected IoTDBSyncClientManager constructClient( final boolean useLeaderCache, final String loadBalanceStrategy, final boolean shouldReceiverConvertOnTypeMismatch, - final String loadTsFileStrategy) { + final String loadTsFileStrategy, + boolean isCustomSendPortDefined, + int minSendPortRange, + int maxSendPortRange, + List candidatePorts) { return new IoTDBConfigNodeSyncClientManager( nodeUrls, useSSL, @@ -72,7 +76,11 @@ protected IoTDBSyncClientManager constructClient( trustStorePwd, loadBalanceStrategy, shouldReceiverConvertOnTypeMismatch, - loadTsFileStrategy); + loadTsFileStrategy, + isCustomSendPortDefined, + minSendPortRange, + maxSendPortRange, + candidatePorts); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/client/IoTDBDataNodeAsyncClientManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/client/IoTDBDataNodeAsyncClientManager.java index 549923882d5a..e74bc51e621e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/client/IoTDBDataNodeAsyncClientManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/client/IoTDBDataNodeAsyncClientManager.java @@ -81,8 +81,18 @@ public IoTDBDataNodeAsyncClientManager( boolean useLeaderCache, String loadBalanceStrategy, boolean shouldReceiverConvertOnTypeMismatch, - String loadTsFileStrategy) { - super(endPoints, useLeaderCache); + String loadTsFileStrategy, + boolean isCustomSendPortDefined, + int minSendPortRange, + int maxSendPortRange, + List candidatePorts) { + super( + endPoints, + useLeaderCache, + isCustomSendPortDefined, + minSendPortRange, + maxSendPortRange, + candidatePorts); endPointSet = new HashSet<>(endPoints); @@ -94,7 +104,11 @@ public IoTDBDataNodeAsyncClientManager( receiverAttributes, new IClientManager.Factory() .createClientManager( - new ClientPoolFactory.AsyncPipeDataTransferServiceClientPoolFactory())); + new ClientPoolFactory.AsyncPipeDataTransferServiceClientPoolFactory( + isCustomSendPortDefined, + minSendPortRange, + maxSendPortRange, + candidatePorts))); } endPoint2Client = ASYNC_PIPE_DATA_TRANSFER_CLIENT_MANAGER_HOLDER.get(receiverAttributes); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/client/IoTDBDataNodeSyncClientManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/client/IoTDBDataNodeSyncClientManager.java index 5a341dc75745..e4d3b4358b71 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/client/IoTDBDataNodeSyncClientManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/client/IoTDBDataNodeSyncClientManager.java @@ -50,7 +50,11 @@ public IoTDBDataNodeSyncClientManager( boolean useLeaderCache, String loadBalanceStrategy, boolean shouldReceiverConvertOnTypeMismatch, - String loadTsFileStrategy) { + String loadTsFileStrategy, + boolean isCustomSendPortDefined, + int minSendPortRange, + int maxSendPortRange, + List candidatePorts) { super( endPoints, useSSL, @@ -59,7 +63,11 @@ public IoTDBDataNodeSyncClientManager( useLeaderCache, loadBalanceStrategy, shouldReceiverConvertOnTypeMismatch, - loadTsFileStrategy); + loadTsFileStrategy, + isCustomSendPortDefined, + minSendPortRange, + maxSendPortRange, + candidatePorts); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/legacy/IoTDBLegacyPipeConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/legacy/IoTDBLegacyPipeConnector.java index b647f149b727..780ad4d13229 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/legacy/IoTDBLegacyPipeConnector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/legacy/IoTDBLegacyPipeConnector.java @@ -236,7 +236,11 @@ public void handshake() throws Exception { port, useSSL, trustStore, - trustStorePwd); + trustStorePwd, + false, + 0, + 0, + null); final TSyncIdentityInfo identityInfo = new TSyncIdentityInfo( pipeName, System.currentTimeMillis(), syncConnectorVersion, databaseName); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/IoTDBDataRegionAsyncConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/IoTDBDataRegionAsyncConnector.java index 5aa4324a2f39..91ec9e2b9a22 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/IoTDBDataRegionAsyncConnector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/IoTDBDataRegionAsyncConnector.java @@ -126,7 +126,11 @@ public void customize( CONNECTOR_LEADER_CACHE_ENABLE_DEFAULT_VALUE), loadBalanceStrategy, shouldReceiverConvertOnTypeMismatch, - loadTsFileStrategy); + loadTsFileStrategy, + isCustomSendPortDefined, + minSendPortRange, + maxSendPortRange, + candidatePorts); if (isTabletBatchModeEnabled) { tabletBatchBuilder = new PipeTransferBatchReqBuilder(parameters); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBDataNodeSyncConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBDataNodeSyncConnector.java index 294e69a769ab..c420e15f9bfb 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBDataNodeSyncConnector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBDataNodeSyncConnector.java @@ -88,7 +88,11 @@ protected IoTDBSyncClientManager constructClient( final boolean useLeaderCache, final String loadBalanceStrategy, final boolean shouldReceiverConvertOnTypeMismatch, - final String loadTsFileStrategy) { + final String loadTsFileStrategy, + boolean isCustomSendPortDefined, + int minSendPortRange, + int maxSendPortRange, + List candidatePorts) { clientManager = new IoTDBDataNodeSyncClientManager( nodeUrls, @@ -98,7 +102,11 @@ protected IoTDBSyncClientManager constructClient( useLeaderCache, loadBalanceStrategy, shouldReceiverConvertOnTypeMismatch, - loadTsFileStrategy); + loadTsFileStrategy, + isCustomSendPortDefined, + minSendPortRange, + maxSendPortRange, + candidatePorts); return clientManager; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/twostage/exchange/sender/TwoStageAggregateSender.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/twostage/exchange/sender/TwoStageAggregateSender.java index a24b6c4fc391..d321eaf6d32a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/twostage/exchange/sender/TwoStageAggregateSender.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/twostage/exchange/sender/TwoStageAggregateSender.java @@ -216,6 +216,10 @@ private IoTDBSyncClient constructIoTDBSyncClient(TEndPoint endPoint) throws TTra endPoint.getPort(), false, null, + null, + false, + 0, + 0, null); } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/ClientPoolFactory.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/ClientPoolFactory.java index c59416d1eaa0..844c9e85a631 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/ClientPoolFactory.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/ClientPoolFactory.java @@ -43,6 +43,8 @@ import org.apache.commons.pool2.KeyedObjectPool; import org.apache.commons.pool2.impl.GenericKeyedObjectPool; +import java.util.List; + public class ClientPoolFactory { private static final CommonConfig conf = CommonDescriptor.getInstance().getConfig(); @@ -271,6 +273,25 @@ public KeyedObjectPool cre public static class AsyncPipeDataTransferServiceClientPoolFactory implements IClientPoolFactory { + private int minSendPortRange; + + private int maxSendPortRange; + + private List candidatePorts; + + private boolean isCustomSendPortDefined; + + public AsyncPipeDataTransferServiceClientPoolFactory( + boolean isCustomSendPortDefined, + int minSendPortRange, + int maxSendPortRange, + List candidatePorts) { + this.minSendPortRange = minSendPortRange; + this.maxSendPortRange = maxSendPortRange; + this.candidatePorts = candidatePorts; + this.isCustomSendPortDefined = isCustomSendPortDefined; + } + @Override public KeyedObjectPool createClientPool( ClientManager manager) { @@ -285,7 +306,11 @@ public KeyedObjectPool createClie .setSelectorNumOfAsyncClientManager( conf.getPipeAsyncConnectorSelectorNumber()) .build(), - ThreadName.PIPE_ASYNC_CONNECTOR_CLIENT_POOL.getName()), + ThreadName.PIPE_ASYNC_CONNECTOR_CLIENT_POOL.getName(), + isCustomSendPortDefined, + minSendPortRange, + maxSendPortRange, + candidatePorts), new ClientPoolProperty.Builder() .setMaxClientNumForEachNode(conf.getPipeAsyncConnectorMaxClientNumber()) .build() diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/async/AsyncPipeDataTransferServiceClient.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/async/AsyncPipeDataTransferServiceClient.java index 91fa97fc82c7..56ae671299f1 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/async/AsyncPipeDataTransferServiceClient.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/async/AsyncPipeDataTransferServiceClient.java @@ -24,6 +24,7 @@ import org.apache.iotdb.commons.client.ThriftClient; import org.apache.iotdb.commons.client.factory.AsyncThriftClientFactory; import org.apache.iotdb.commons.client.property.ThriftClientProperty; +import org.apache.iotdb.commons.client.util.PortUtilizationManager; import org.apache.iotdb.rpc.TNonblockingSocketWrapper; import org.apache.iotdb.service.rpc.thrift.IClientRPCService; @@ -36,6 +37,7 @@ import org.slf4j.LoggerFactory; import java.io.IOException; +import java.util.List; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; @@ -61,13 +63,19 @@ public AsyncPipeDataTransferServiceClient( ThriftClientProperty property, TEndPoint endpoint, TAsyncClientManager tClientManager, - ClientManager clientManager) + ClientManager clientManager, + boolean isCustomSendPortDefined, + Integer sendPort) throws IOException { super( property.getProtocolFactory(), tClientManager, TNonblockingSocketWrapper.wrap( - endpoint.getIp(), endpoint.getPort(), property.getConnectionTimeoutMs())); + endpoint.getIp(), + endpoint.getPort(), + property.getConnectionTimeoutMs(), + isCustomSendPortDefined, + sendPort)); setTimeout(property.getConnectionTimeoutMs()); this.printLogWhenEncounterException = property.isPrintLogWhenEncounterException(); this.endpoint = endpoint; @@ -177,11 +185,27 @@ public String toString() { public static class Factory extends AsyncThriftClientFactory { + private int minSendPortRange; + + private int maxSendPortRange; + + private List candidatePorts; + + private boolean isCustomSendPortDefined; + public Factory( ClientManager clientManager, ThriftClientProperty thriftClientProperty, - String threadName) { + String threadName, + boolean isCustomSendPortDefined, + int minSendPortRange, + int maxSendPortRange, + List candidatePorts) { super(clientManager, thriftClientProperty, threadName); + this.isCustomSendPortDefined = isCustomSendPortDefined; + this.minSendPortRange = minSendPortRange; + this.maxSendPortRange = maxSendPortRange; + this.candidatePorts = candidatePorts; } @Override @@ -193,12 +217,20 @@ public void destroyObject( @Override public PooledObject makeObject(TEndPoint endPoint) throws Exception { + Integer sendPort = 0; + if (isCustomSendPortDefined) { + sendPort = + PortUtilizationManager.INSTANCE.findAvailablePort( + minSendPortRange, maxSendPortRange, candidatePorts); + } return new DefaultPooledObject<>( new AsyncPipeDataTransferServiceClient( thriftClientProperty, endPoint, tManagers[clientCnt.incrementAndGet() % tManagers.length], - clientManager)); + clientManager, + isCustomSendPortDefined, + sendPort)); } @Override diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/util/PortUtilizationManager.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/util/PortUtilizationManager.java new file mode 100644 index 000000000000..36251450ac6f --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/util/PortUtilizationManager.java @@ -0,0 +1,290 @@ +/* + * 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.iotdb.commons.client.util; + +import org.apache.tsfile.utils.Pair; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; +import java.util.ListIterator; + +public class PortUtilizationManager { + + private static final Logger LOGGER = LoggerFactory.getLogger(PortUtilizationManager.class); + + private final List> occupiedPorts = new LinkedList<>(); + + private PortUtilizationManager() { + occupiedPorts.add(new Pair<>(1023, 1023)); + occupiedPorts.add(new Pair<>(65536, 65536)); + } + + public static final PortUtilizationManager INSTANCE = new PortUtilizationManager(); + + public Integer addIfPortAvailable(int candidatePort) { + synchronized (occupiedPorts) { + if (occupiedPorts.size() == 1 || occupiedPorts.isEmpty()) { + return null; + } + ListIterator> occupiedPortsIterator = occupiedPorts.listIterator(); + Pair previousRange = null; + Pair currentRange = occupiedPortsIterator.next(); + while (occupiedPortsIterator.hasNext()) { + previousRange = currentRange; + currentRange = occupiedPortsIterator.next(); + if (currentRange.getRight() <= candidatePort) { + continue; + } + if (previousRange.getLeft() >= candidatePort) { + break; + } + if (previousRange.getRight() >= candidatePort && currentRange.getLeft() <= candidatePort) { + return null; + } + Integer availablePort = checkPortAvailability(candidatePort); + if (availablePort == null) { + return null; + } + + if (availablePort == previousRange.getRight() + 1 + && currentRange.getLeft() - 1 == availablePort) { + previousRange.setRight(availablePort); + occupiedPortsIterator.remove(); + return availablePort; + } + + if (availablePort == previousRange.getRight() + 1) { + previousRange.setRight(availablePort); + return availablePort; + } + + if (availablePort == currentRange.getLeft() - 1) { + currentRange.setLeft(availablePort); + return availablePort; + } + + occupiedPortsIterator.add(new Pair<>(availablePort, availablePort)); + return availablePort; + } + } + return null; + } + + private Integer addIfPortAvailable(List candidatePorts) { + synchronized (occupiedPorts) { + if (occupiedPorts.size() == 1 || occupiedPorts.isEmpty() || candidatePorts.isEmpty()) { + return null; + } + + ListIterator> occupiedPortsIterator = occupiedPorts.listIterator(); + Iterator integerIterator = candidatePorts.iterator(); + Pair previousRange = null; + Pair currentRange = occupiedPortsIterator.next(); + Integer candidatePort = integerIterator.next(); + out: + while (occupiedPortsIterator.hasNext()) { + previousRange = currentRange; + currentRange = occupiedPortsIterator.next(); + + while (true) { + if (currentRange.getRight() <= candidatePort) { + continue out; + } + while (previousRange.getLeft() >= candidatePort) { + if (!integerIterator.hasNext()) { + return null; + } + candidatePort = integerIterator.next(); + } + if (candidatePort < currentRange.getLeft() + && candidatePort > previousRange.getRight() + && checkPortAvailability(candidatePort) != null) { + break; + } + if (!integerIterator.hasNext()) { + return null; + } + candidatePort = integerIterator.next(); + } + + if (candidatePort == previousRange.getRight() + 1 + && currentRange.getLeft() - 1 == candidatePort) { + previousRange.setRight(candidatePort); + occupiedPortsIterator.remove(); + return candidatePort; + } + + if (candidatePort == previousRange.getRight() + 1) { + previousRange.setRight(candidatePort); + return candidatePort; + } + + if (candidatePort == currentRange.getLeft() - 1) { + currentRange.setLeft(candidatePort); + return candidatePort; + } + + occupiedPortsIterator.add(new Pair<>(candidatePort, candidatePort)); + return candidatePort; + } + } + return null; + } + + private Integer addIfPortAvailable(int minSendPortRange, int maxSendPortRange) { + synchronized (occupiedPorts) { + if (occupiedPorts.size() == 1 || occupiedPorts.isEmpty()) { + return null; + } + + ListIterator> occupiedPortsIterator = occupiedPorts.listIterator(); + Pair previousRange = null; + Pair currentRange = occupiedPortsIterator.next(); + while (occupiedPortsIterator.hasNext()) { + previousRange = currentRange; + currentRange = occupiedPortsIterator.next(); + if (currentRange.getRight() <= minSendPortRange) { + continue; + } + if (previousRange.getLeft() >= maxSendPortRange) { + break; + } + + int max = Math.min(maxSendPortRange, currentRange.getLeft() - 1); + int min = Math.max(minSendPortRange, previousRange.getRight() + 1); + if (max < min) { + return null; + } + Integer availablePort = findAvailablePort(min, max); + if (availablePort == null) { + continue; + } + + if (availablePort == previousRange.getRight() + 1 + && currentRange.getLeft() - 1 == availablePort) { + previousRange.setRight(availablePort); + occupiedPortsIterator.remove(); + return availablePort; + } + + if (availablePort == previousRange.getRight() + 1) { + previousRange.setRight(availablePort); + return availablePort; + } + + if (availablePort == currentRange.getLeft() - 1) { + currentRange.setLeft(availablePort); + return availablePort; + } + + occupiedPortsIterator.add(new Pair<>(availablePort, availablePort)); + return availablePort; + } + } + return null; + } + + private Integer findAvailablePort(int min, int max) { + for (int i = min; i <= max; i++) { + try { + SecurityManager sm = System.getSecurityManager(); + if (sm != null) { + sm.checkListen(i); + } + return i; + } catch (Exception ignored) { + } + } + return null; + } + + private Integer checkPortAvailability(int port) { + try { + SecurityManager sm = System.getSecurityManager(); + if (sm != null) { + sm.checkListen(port); + } + return port; + } catch (Exception ignored) { + } + return null; + } + + public void releasePortIfUsed(int port) { + synchronized (occupiedPorts) { + if (occupiedPorts.isEmpty()) { + return; + } + + ListIterator> iterator = occupiedPorts.listIterator(); + Pair cur = null; + while (iterator.hasNext()) { + cur = iterator.next(); + if (port > cur.getRight()) { + continue; + } + if (port < cur.getLeft()) { + break; + } + if (cur.getLeft() == port) { + if (cur.getLeft().equals(cur.getRight())) { + iterator.remove(); + break; + } + cur.setLeft(cur.getLeft() + 1); + } + + if (cur.getRight() == port) { + if (cur.getLeft().equals(cur.getRight())) { + iterator.remove(); + break; + } + cur.setRight(cur.getRight() - 1); + break; + } + cur.setRight(port - 1); + iterator.add(new Pair<>(port + 1, cur.getRight())); + break; + } + } + } + + public Integer findAvailablePort( + int minSendPortRange, int maxSendPortRange, List candidatePorts) { + Integer port = addIfPortAvailable(candidatePorts); + if (port != null) { + return port; + } else { + LOGGER.debug("No available port found in the candidate list, trying range."); + } + port = addIfPortAvailable(minSendPortRange, maxSendPortRange); + if (port == null) { + String exceptionMessage = + String.format( + "Failed to find an available port within the range %d to %d.", + minSendPortRange, maxSendPortRange); + LOGGER.warn(exceptionMessage); + } + return port; + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/constant/PipeConnectorConstant.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/constant/PipeConnectorConstant.java index 77eccfb4069d..217802ff9a43 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/constant/PipeConnectorConstant.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/constant/PipeConnectorConstant.java @@ -46,6 +46,21 @@ public class PipeConnectorConstant { public static final String CONNECTOR_IOTDB_NODE_URLS_KEY = "connector.node-urls"; public static final String SINK_IOTDB_NODE_URLS_KEY = "sink.node-urls"; + public static final String CONNECTOR_IOTDB_SEND_PORT_MIN_KEY = "connector.send-port.range.min"; + public static final String SINK_IOTDB_SEND_PORT_MIN_KEY = "sink.send-port.range.min"; + public static final String CONNECTOR_IOTDB_SEND_PORT_MAX_KEY = "connector.send-port.range.max"; + public static final String SINK_IOTDB_SEND_PORT_MAX_KEY = "sink.send-port.range.max"; + public static final String CONNECTOR_IOTDB_SEND_PORT_CANDIDATE_KEY = + "connector.send-port.range.candidate"; + public static final String SINK_IOTDB_SEND_PORT_CANDIDATE_KEY = "sink.send-port.range.candidate"; + public static final String CONNECTOR_IOTDB_SEND_PORT_ENABLE_KEY = + "connector.send-port.range.enable"; + public static final String SINK_IOTDB_SEND_PORT_ENABLE_KEY = "sink.send-port.range.enable"; + public static final int CONNECTOR_IOTDB_SEND_PORT_MIN_VALUE = 0; + public static final int CONNECTOR_IOTDB_SEND_PORT_MAX_VALUE = 65535; + public static final String CONNECTOR_IOTDB_SEND_PORT_CANDIDATE_VALUE = ""; + public static final boolean CONNECTOR_IOTDB_SEND_PORT_ENABLE_VALUE = true; + public static final String SINK_IOTDB_SSL_ENABLE_KEY = "sink.ssl.enable"; public static final String SINK_IOTDB_SSL_TRUST_STORE_PATH_KEY = "sink.ssl.trust-store-path"; public static final String SINK_IOTDB_SSL_TRUST_STORE_PWD_KEY = "sink.ssl.trust-store-pwd"; diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/client/IoTDBClientManager.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/client/IoTDBClientManager.java index 73e0543fe675..22f4b14edc6c 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/client/IoTDBClientManager.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/client/IoTDBClientManager.java @@ -43,14 +43,32 @@ public abstract class IoTDBClientManager { // it is a DataNode receiver. The flag is useless for configNode receiver. protected boolean supportModsIfIsDataNodeReceiver = true; + protected int minSendPortRange; + + protected int maxSendPortRange; + + protected List candidatePorts; + + protected boolean isCustomSendPortDefined; + private static final int MAX_CONNECTION_TIMEOUT_MS = 24 * 60 * 60 * 1000; // 1 day private static final int FIRST_ADJUSTMENT_TIMEOUT_MS = 6 * 60 * 60 * 1000; // 6 hours protected static final AtomicInteger CONNECTION_TIMEOUT_MS = new AtomicInteger(PipeConfig.getInstance().getPipeConnectorTransferTimeoutMs()); - protected IoTDBClientManager(List endPointList, boolean useLeaderCache) { + protected IoTDBClientManager( + List endPointList, + boolean useLeaderCache, + boolean isCustomSendPortDefined, + int minSendPortRange, + int maxSendPortRange, + List candidatePorts) { this.endPointList = endPointList; this.useLeaderCache = useLeaderCache; + this.isCustomSendPortDefined = isCustomSendPortDefined; + this.minSendPortRange = minSendPortRange; + this.maxSendPortRange = maxSendPortRange; + this.candidatePorts = candidatePorts; } public boolean supportModsIfIsDataNodeReceiver() { diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/client/IoTDBSyncClient.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/client/IoTDBSyncClient.java index d79c430474c3..aa3438431f4a 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/client/IoTDBSyncClient.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/client/IoTDBSyncClient.java @@ -22,23 +22,28 @@ import org.apache.iotdb.common.rpc.thrift.TEndPoint; import org.apache.iotdb.commons.client.ThriftClient; import org.apache.iotdb.commons.client.property.ThriftClientProperty; +import org.apache.iotdb.commons.client.util.PortUtilizationManager; import org.apache.iotdb.commons.pipe.config.PipeConfig; import org.apache.iotdb.commons.pipe.connector.payload.thrift.request.IoTDBConnectorRequestVersion; import org.apache.iotdb.commons.pipe.connector.payload.thrift.request.PipeTransferSliceReq; import org.apache.iotdb.pipe.api.exception.PipeConnectionException; import org.apache.iotdb.rpc.DeepCopyRpcTransportFactory; import org.apache.iotdb.rpc.TSStatusCode; +import org.apache.iotdb.rpc.TimeoutChangeableTFastFramedTransport; import org.apache.iotdb.rpc.TimeoutChangeableTransport; import org.apache.iotdb.service.rpc.thrift.IClientRPCService; import org.apache.iotdb.service.rpc.thrift.TPipeTransferReq; import org.apache.iotdb.service.rpc.thrift.TPipeTransferResp; import org.apache.thrift.TException; +import org.apache.thrift.transport.TSocket; import org.apache.thrift.transport.TTransport; import org.apache.thrift.transport.TTransportException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.net.InetSocketAddress; +import java.util.List; import java.util.concurrent.atomic.AtomicInteger; public class IoTDBSyncClient extends IClientRPCService.Client @@ -51,6 +56,7 @@ public class IoTDBSyncClient extends IClientRPCService.Client private final String ipAddress; private final int port; private final TEndPoint endPoint; + private final boolean isCustomSendPortDefined; public IoTDBSyncClient( ThriftClientProperty property, @@ -58,7 +64,11 @@ public IoTDBSyncClient( int port, boolean useSSL, String trustStore, - String trustStorePwd) + String trustStorePwd, + boolean isCustomSendPortDefined, + int minSendPortRange, + int maxSendPortRange, + List candidatePorts) throws TTransportException { super( property @@ -76,8 +86,31 @@ public IoTDBSyncClient( this.ipAddress = ipAddress; this.port = port; this.endPoint = new TEndPoint(ipAddress, port); + this.isCustomSendPortDefined = isCustomSendPortDefined; final TTransport transport = getInputProtocol().getTransport(); if (!transport.isOpen()) { + if (isCustomSendPortDefined) { + PortUtilizationManager portUtilizationManager = PortUtilizationManager.INSTANCE; + Integer sendPort = + portUtilizationManager.findAvailablePort( + minSendPortRange, maxSendPortRange, candidatePorts); + if (sendPort == null) { + throw new PipeConnectionException( + String.format( + "Failed to find an available send port. Custom send port is defined." + + " No ports are available in the candidate list [%s] or within the range %d to %d.", + candidatePorts, minSendPortRange, maxSendPortRange)); + } + try { + final InetSocketAddress isa = new InetSocketAddress(sendPort); + ((TSocket) ((TimeoutChangeableTFastFramedTransport) transport).getSocket()) + .getSocket() + .bind(isa); + } catch (Exception e) { + String bindErrorMessage = "Failed to bind to the port: " + sendPort; + throw new PipeConnectionException(bindErrorMessage, e); + } + } transport.open(); } } @@ -166,6 +199,11 @@ public void close() throws Exception { @Override public void invalidate() { if (getInputProtocol().getTransport().isOpen()) { + if (isCustomSendPortDefined) { + PortUtilizationManager portUtilizationManager = PortUtilizationManager.INSTANCE; + portUtilizationManager.releasePortIfUsed( + ((TSocket) getInputProtocol().getTransport()).getSocket().getPort()); + } getInputProtocol().getTransport().close(); } } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/client/IoTDBSyncClientManager.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/client/IoTDBSyncClientManager.java index 81b5194621ce..365802b0377e 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/client/IoTDBSyncClientManager.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/client/IoTDBSyncClientManager.java @@ -72,8 +72,18 @@ protected IoTDBSyncClientManager( boolean useLeaderCache, String loadBalanceStrategy, boolean shouldReceiverConvertOnTypeMismatch, - String loadTsFileStrategy) { - super(endPoints, useLeaderCache); + String loadTsFileStrategy, + boolean isCustomSendPortDefined, + int minSendPortRange, + int maxSendPortRange, + List candidatePorts) { + super( + endPoints, + useLeaderCache, + isCustomSendPortDefined, + minSendPortRange, + maxSendPortRange, + candidatePorts); this.useSSL = useSSL; this.trustStorePath = trustStorePath; @@ -159,7 +169,11 @@ private void initClientAndStatus( endPoint.getPort(), useSSL, trustStorePath, - trustStorePwd)); + trustStorePwd, + isCustomSendPortDefined, + minSendPortRange, + maxSendPortRange, + candidatePorts)); } catch (Exception e) { throw new PipeConnectionException( String.format( diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/protocol/IoTDBAirGapConnector.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/protocol/IoTDBAirGapConnector.java index 8972fb424b3a..da44d2cbe5b2 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/protocol/IoTDBAirGapConnector.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/protocol/IoTDBAirGapConnector.java @@ -21,6 +21,7 @@ import org.apache.iotdb.common.rpc.thrift.TEndPoint; import org.apache.iotdb.common.rpc.thrift.TSStatus; +import org.apache.iotdb.commons.client.util.PortUtilizationManager; import org.apache.iotdb.commons.pipe.config.PipeConfig; import org.apache.iotdb.commons.pipe.connector.payload.airgap.AirGapELanguageConstant; import org.apache.iotdb.commons.pipe.connector.payload.airgap.AirGapOneByteResponse; @@ -158,7 +159,11 @@ public void handshake() throws Exception { // Close the socket if necessary if (sockets.get(i) != null) { try { - sockets.set(i, null).close(); + final Socket socket = sockets.set(i, null); + if (isCustomSendPortDefined) { + PortUtilizationManager.INSTANCE.releasePortIfUsed(socket.getPort()); + } + socket.close(); } catch (final Exception e) { LOGGER.warn( "Failed to close socket with target server ip: {}, port: {}, because: {}. Ignore it.", @@ -171,6 +176,12 @@ public void handshake() throws Exception { final AirGapSocket socket = new AirGapSocket(ip, port); try { + if (isCustomSendPortDefined) { + socket.bind( + new InetSocketAddress( + PortUtilizationManager.INSTANCE.findAvailablePort( + minSendPortRange, maxSendPortRange, candidatePorts))); + } socket.connect(new InetSocketAddress(ip, port), handshakeTimeoutMs); socket.setKeepAlive(true); sockets.set(i, socket); @@ -338,7 +349,11 @@ public void close() { for (int i = 0; i < sockets.size(); ++i) { try { if (sockets.get(i) != null) { - sockets.set(i, null).close(); + final Socket socket = sockets.set(i, null); + if (isCustomSendPortDefined) { + PortUtilizationManager.INSTANCE.releasePortIfUsed(socket.getPort()); + } + socket.close(); } } catch (final Exception e) { LOGGER.warn("Failed to close client {}.", i, e); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/protocol/IoTDBConnector.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/protocol/IoTDBConnector.java index db58cec22ca9..be2d37fa1030 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/protocol/IoTDBConnector.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/protocol/IoTDBConnector.java @@ -20,6 +20,8 @@ package org.apache.iotdb.commons.pipe.connector.protocol; import org.apache.iotdb.common.rpc.thrift.TEndPoint; +import org.apache.iotdb.commons.conf.CommonDescriptor; +import org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant; import org.apache.iotdb.commons.pipe.connector.compressor.PipeCompressor; import org.apache.iotdb.commons.pipe.connector.compressor.PipeCompressorConfig; import org.apache.iotdb.commons.pipe.connector.compressor.PipeCompressorFactory; @@ -42,12 +44,14 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.LinkedHashSet; import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; +import java.util.stream.Collectors; import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.CONNECTOR_COMPRESSOR_DEFAULT_VALUE; import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.CONNECTOR_COMPRESSOR_KEY; @@ -80,6 +84,14 @@ import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.CONNECTOR_IOTDB_NODE_URLS_KEY; import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.CONNECTOR_IOTDB_PLAIN_BATCH_SIZE_DEFAULT_VALUE; import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.CONNECTOR_IOTDB_PORT_KEY; +import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.CONNECTOR_IOTDB_SEND_PORT_CANDIDATE_KEY; +import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.CONNECTOR_IOTDB_SEND_PORT_CANDIDATE_VALUE; +import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.CONNECTOR_IOTDB_SEND_PORT_ENABLE_KEY; +import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.CONNECTOR_IOTDB_SEND_PORT_ENABLE_VALUE; +import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.CONNECTOR_IOTDB_SEND_PORT_MAX_KEY; +import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.CONNECTOR_IOTDB_SEND_PORT_MAX_VALUE; +import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.CONNECTOR_IOTDB_SEND_PORT_MIN_KEY; +import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.CONNECTOR_IOTDB_SEND_PORT_MIN_VALUE; import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.CONNECTOR_LOAD_BALANCE_ROUND_ROBIN_STRATEGY; import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.CONNECTOR_LOAD_BALANCE_STRATEGY_KEY; import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.CONNECTOR_LOAD_BALANCE_STRATEGY_SET; @@ -103,6 +115,10 @@ import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.SINK_IOTDB_IP_KEY; import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.SINK_IOTDB_NODE_URLS_KEY; import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.SINK_IOTDB_PORT_KEY; +import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.SINK_IOTDB_SEND_PORT_CANDIDATE_KEY; +import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.SINK_IOTDB_SEND_PORT_ENABLE_KEY; +import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.SINK_IOTDB_SEND_PORT_MAX_KEY; +import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.SINK_IOTDB_SEND_PORT_MIN_KEY; import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.SINK_LOAD_BALANCE_STRATEGY_KEY; import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.SINK_LOAD_TSFILE_STRATEGY_KEY; import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.SINK_RATE_LIMIT_KEY; @@ -113,11 +129,22 @@ public abstract class IoTDBConnector implements PipeConnector { "Exception occurred while parsing node urls from target servers: {}"; private static final String PARSE_URL_ERROR_MESSAGE = "Error occurred while parsing node urls from target servers, please check the specified 'host':'port' or 'node-urls'"; + public static final int MIN_PORT = + 0; // The minimum port number allocated to user processes by the operating system + public static final int MAX_PORT = 65535; // The maximum value for port numbers private static final Logger LOGGER = LoggerFactory.getLogger(IoTDBConnector.class); protected final List nodeUrls = new ArrayList<>(); + protected int minSendPortRange; + + protected int maxSendPortRange; + + protected List candidatePorts; + + protected boolean isCustomSendPortDefined; + protected String loadBalanceStrategy; protected String loadTsFileStrategy; @@ -175,6 +202,69 @@ public void validate(final PipeParameterValidator validator) throws Exception { Arrays.asList(CONNECTOR_IOTDB_BATCH_SIZE_KEY, SINK_IOTDB_BATCH_SIZE_KEY), CONNECTOR_IOTDB_PLAIN_BATCH_SIZE_DEFAULT_VALUE)); + minSendPortRange = + parameters.getIntOrDefault( + Arrays.asList(CONNECTOR_IOTDB_SEND_PORT_MIN_KEY, SINK_IOTDB_SEND_PORT_MIN_KEY), + CONNECTOR_IOTDB_SEND_PORT_MIN_VALUE); + maxSendPortRange = + parameters.getIntOrDefault( + Arrays.asList(CONNECTOR_IOTDB_SEND_PORT_MAX_KEY, SINK_IOTDB_SEND_PORT_MAX_KEY), + CONNECTOR_IOTDB_SEND_PORT_MAX_VALUE); + validator.validate( + args -> (boolean) args[0] && (boolean) args[1] && (boolean) args[2], + String.format( + "%s must be <= %s, but got %d > %d", + SINK_IOTDB_SEND_PORT_MIN_KEY, + SINK_IOTDB_SEND_PORT_MAX_KEY, + minSendPortRange, + maxSendPortRange), + minSendPortRange <= maxSendPortRange, + minSendPortRange >= MIN_PORT, + maxSendPortRange <= MAX_PORT); + + this.candidatePorts = + parseCandidatePorts( + parameters.getStringOrDefault( + Arrays.asList( + CONNECTOR_IOTDB_SEND_PORT_CANDIDATE_KEY, SINK_IOTDB_SEND_PORT_CANDIDATE_KEY), + CONNECTOR_IOTDB_SEND_PORT_CANDIDATE_VALUE)); + + if (!candidatePorts.isEmpty()) { + validator.validate( + arg -> (int) arg[0] >= MIN_PORT && (int) arg[1] <= MAX_PORT, + String.format( + "Candidate port range is invalid: Ports must be between 0 and 65535, but got minimum port: %d and maximum port: %d", + candidatePorts.get(0), candidatePorts.get(candidatePorts.size() - 1)), + candidatePorts.get(0), + candidatePorts.get(candidatePorts.size() - 1)); + } + this.isCustomSendPortDefined = + parameters.getBooleanOrDefault( + Arrays.asList(CONNECTOR_IOTDB_SEND_PORT_ENABLE_KEY, SINK_IOTDB_SEND_PORT_ENABLE_KEY), + CONNECTOR_IOTDB_SEND_PORT_ENABLE_VALUE); + + if (isCustomSendPortDefined) { + int maxUsablePortsNum = + calculateUsablePorts(minSendPortRange, maxSendPortRange, candidatePorts); + int parallelTaskNum = + parameters.getIntOrDefault( + Arrays.asList( + PipeConnectorConstant.CONNECTOR_IOTDB_PARALLEL_TASKS_KEY, + PipeConnectorConstant.SINK_IOTDB_PARALLEL_TASKS_KEY), + PipeConnectorConstant.CONNECTOR_IOTDB_PARALLEL_TASKS_DEFAULT_VALUE); + int requiredPortsNum = + parallelTaskNum + * CommonDescriptor.getInstance().getConfig().getPipeAsyncConnectorMaxClientNumber(); + + validator.validate( + arg -> (int) arg[0] > (int) arg[1], + String.format( + "Not enough available ports: There are %d available ports but require %d.", + maxUsablePortsNum, requiredPortsNum), + maxUsablePortsNum, + requiredPortsNum); + } + loadBalanceStrategy = parameters .getStringOrDefault( @@ -428,6 +518,27 @@ private void checkNodeUrls(final Set nodeUrls) throws PipeParameterNo } } + private static List parseCandidatePorts(String candidate) { + if (candidate == null || candidate.isEmpty()) { + return Collections.emptyList(); + } + return Arrays.stream(candidate.split(",")) + .map(String::trim) + .map(Integer::parseInt) + .sorted() + .collect(Collectors.toList()); + } + + private int calculateUsablePorts(int sendPortMin, int sendPortMax, List candidatePorts) { + int usablePortCount = sendPortMax - sendPortMin + 1; + for (int port : candidatePorts) { + if (sendPortMax < port && port < sendPortMin) { + usablePortCount++; + } + } + return usablePortCount; + } + @Override public void close() { // TODO: Not all the limiters should be closed here, but it's fine for now. diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/protocol/IoTDBSslSyncConnector.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/protocol/IoTDBSslSyncConnector.java index a14cfe3e214f..c55baab2820e 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/protocol/IoTDBSslSyncConnector.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/protocol/IoTDBSslSyncConnector.java @@ -123,7 +123,11 @@ public void customize(PipeParameters parameters, PipeConnectorRuntimeConfigurati useLeaderCache, loadBalanceStrategy, shouldReceiverConvertOnTypeMismatch, - loadTsFileStrategy); + loadTsFileStrategy, + isCustomSendPortDefined, + minSendPortRange, + maxSendPortRange, + candidatePorts); } protected abstract IoTDBSyncClientManager constructClient( @@ -134,7 +138,11 @@ protected abstract IoTDBSyncClientManager constructClient( boolean useLeaderCache, String loadBalanceStrategy, boolean shouldReceiverConvertOnTypeMismatch, - String loadTsFileStrategy); + String loadTsFileStrategy, + boolean isCustomSendPortDefined, + int minSendPortRange, + int maxSendPortRange, + List candidatePorts); @Override public void handshake() throws Exception { From 4dd9794ebe5b9bd6513875bacf50dbb1b79eb9e1 Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Wed, 25 Sep 2024 00:38:08 +0800 Subject: [PATCH 02/12] Modify code logic --- .../iotdb/rpc/TNonblockingSocketWrapper.java | 20 +- .../AsyncPipeDataTransferServiceClient.java | 53 +++- .../util/IoTDBConnectorPortManager.java | 291 ++++++++++++++++++ .../client/util/PortUtilizationManager.java | 290 ----------------- .../constant/PipeConnectorConstant.java | 4 +- .../connector/client/IoTDBSyncClient.java | 42 ++- .../protocol/IoTDBAirGapConnector.java | 19 +- .../commons/utils/function/Consumer.java | 24 ++ .../client/IoTDBConnectorPortManagerTest.java | 105 +++++++ 9 files changed, 490 insertions(+), 358 deletions(-) create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/util/IoTDBConnectorPortManager.java delete mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/util/PortUtilizationManager.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/utils/function/Consumer.java create mode 100644 iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/client/IoTDBConnectorPortManagerTest.java diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TNonblockingSocketWrapper.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TNonblockingSocketWrapper.java index 4a25306cdc01..5f2b6cde2f4f 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TNonblockingSocketWrapper.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TNonblockingSocketWrapper.java @@ -23,7 +23,6 @@ import org.apache.thrift.transport.TTransportException; import java.io.IOException; -import java.net.InetSocketAddress; import java.nio.channels.SocketChannel; /** @@ -43,24 +42,7 @@ public static TNonblockingSocket wrap(String host, int port) throws IOException public static TNonblockingSocket wrap(String host, int port, int timeout) throws IOException { try { - TNonblockingSocket nonblockingSocket = new TNonblockingSocket(host, port, timeout); - nonblockingSocket.getSocketChannel().bind(new InetSocketAddress(0)); - return nonblockingSocket; - } catch (TTransportException e) { - // never happen - return null; - } - } - - public static TNonblockingSocket wrap( - String host, int port, int timeout, boolean isCustomSendPortDefined, int sendPort) - throws IOException { - try { - TNonblockingSocket nonblockingSocket = new TNonblockingSocket(host, port, timeout); - if (isCustomSendPortDefined) { - nonblockingSocket.getSocketChannel().bind(new InetSocketAddress(sendPort)); - } - return nonblockingSocket; + return new TNonblockingSocket(host, port, timeout); } catch (TTransportException e) { // never happen return null; diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/async/AsyncPipeDataTransferServiceClient.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/async/AsyncPipeDataTransferServiceClient.java index 56ae671299f1..628f9b716c9c 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/async/AsyncPipeDataTransferServiceClient.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/async/AsyncPipeDataTransferServiceClient.java @@ -24,7 +24,7 @@ import org.apache.iotdb.commons.client.ThriftClient; import org.apache.iotdb.commons.client.factory.AsyncThriftClientFactory; import org.apache.iotdb.commons.client.property.ThriftClientProperty; -import org.apache.iotdb.commons.client.util.PortUtilizationManager; +import org.apache.iotdb.commons.client.util.IoTDBConnectorPortManager; import org.apache.iotdb.rpc.TNonblockingSocketWrapper; import org.apache.iotdb.service.rpc.thrift.IClientRPCService; @@ -37,6 +37,8 @@ import org.slf4j.LoggerFactory; import java.io.IOException; +import java.net.InetSocketAddress; +import java.nio.channels.SocketChannel; import java.util.List; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; @@ -65,17 +67,36 @@ public AsyncPipeDataTransferServiceClient( TAsyncClientManager tClientManager, ClientManager clientManager, boolean isCustomSendPortDefined, - Integer sendPort) + int minSendPortRange, + int maxSendPortRange, + List candidatePorts) throws IOException { super( property.getProtocolFactory(), tClientManager, TNonblockingSocketWrapper.wrap( - endpoint.getIp(), - endpoint.getPort(), - property.getConnectionTimeoutMs(), - isCustomSendPortDefined, - sendPort)); + endpoint.getIp(), endpoint.getPort(), property.getConnectionTimeoutMs())); + SocketChannel socketChannel = ((TNonblockingSocket) ___transport).getSocketChannel(); + if (isCustomSendPortDefined) { + IoTDBConnectorPortManager.INSTANCE.bingPort( + minSendPortRange, + maxSendPortRange, + candidatePorts, + (sendPort) -> { + socketChannel.bind(new InetSocketAddress(sendPort)); + }); + } else { + try { + socketChannel.bind(new InetSocketAddress(0)); + IoTDBConnectorPortManager.INSTANCE.addPortIfAvailable( + ((InetSocketAddress) + (((TNonblockingSocket) ___transport).getSocketChannel().getLocalAddress())) + .getPort()); + } catch (Exception e) { + LOGGER.warn( + "Failed to add port to PortUtilizationManager due to exception: {}", e.getMessage()); + } + } setTimeout(property.getConnectionTimeoutMs()); this.printLogWhenEncounterException = property.isPrintLogWhenEncounterException(); this.endpoint = endpoint; @@ -136,6 +157,14 @@ public void setTimeoutDynamically(int timeout) { } private void close() { + try { + IoTDBConnectorPortManager.INSTANCE.releaseUsedPort( + ((InetSocketAddress) + (((TNonblockingSocket) ___transport).getSocketChannel().getLocalAddress())) + .getPort()); + } catch (Exception e) { + LOGGER.warn("Failed to release port due to exception: ", e); + } ___transport.close(); ___currentMethod = null; } @@ -217,12 +246,6 @@ public void destroyObject( @Override public PooledObject makeObject(TEndPoint endPoint) throws Exception { - Integer sendPort = 0; - if (isCustomSendPortDefined) { - sendPort = - PortUtilizationManager.INSTANCE.findAvailablePort( - minSendPortRange, maxSendPortRange, candidatePorts); - } return new DefaultPooledObject<>( new AsyncPipeDataTransferServiceClient( thriftClientProperty, @@ -230,7 +253,9 @@ public PooledObject makeObject(TEndPoint end tManagers[clientCnt.incrementAndGet() % tManagers.length], clientManager, isCustomSendPortDefined, - sendPort)); + minSendPortRange, + maxSendPortRange, + candidatePorts)); } @Override diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/util/IoTDBConnectorPortManager.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/util/IoTDBConnectorPortManager.java new file mode 100644 index 000000000000..f64caaf254f6 --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/util/IoTDBConnectorPortManager.java @@ -0,0 +1,291 @@ +/* + * 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.iotdb.commons.client.util; + +import org.apache.iotdb.commons.utils.TestOnly; +import org.apache.iotdb.commons.utils.function.Consumer; + +import org.apache.tsfile.utils.Pair; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; +import java.util.ListIterator; + +public class IoTDBConnectorPortManager { + + private static final Logger LOGGER = LoggerFactory.getLogger(IoTDBConnectorPortManager.class); + + public List> getOccupiedPorts() { + return occupiedPorts; + } + + private final List> occupiedPorts = new LinkedList<>(); + + private IoTDBConnectorPortManager() { + occupiedPorts.add(new Pair<>(1023, 1023)); + occupiedPorts.add(new Pair<>(65536, 65536)); + } + + public static final IoTDBConnectorPortManager INSTANCE = new IoTDBConnectorPortManager(); + + // ===========================Iterator================================ + + public static class AvailablePortIterator implements Iterator { + private final ListIterator> occupiedPortsIterator; + private final Iterator> availableRangesIterator; + private ListIterator> preOccupiedPortsIterator; + private Pair availableRange; + private Pair previousRange; + private Pair currentRange; + private boolean hasNext = true; + private int availablePort = 0; + private int maxAvailablePort = -1; + + AvailablePortIterator( + final List> occupiedPorts, + final List> availableRanges) { + if (occupiedPorts.size() <= 1 || availableRanges.isEmpty()) { + hasNext = false; + } + this.occupiedPortsIterator = occupiedPorts.listIterator(); + this.availableRangesIterator = availableRanges.iterator(); + this.availableRange = availableRangesIterator.next(); + this.currentRange = occupiedPortsIterator.next(); + } + + @Override + public boolean hasNext() { + if (!hasNext) { + return false; + } + if (availablePort <= maxAvailablePort) { + return true; + } + while (occupiedPortsIterator.hasNext()) { + previousRange = currentRange; + currentRange = occupiedPortsIterator.next(); + preOccupiedPortsIterator.next(); + if (currentRange.getRight() <= availableRange.getLeft()) { + continue; + } + while (previousRange.getLeft() >= availableRange.getRight()) { + if (availableRangesIterator.hasNext()) { + availableRange = availableRangesIterator.next(); + } else { + hasNext = false; + break; + } + } + + final int max = Math.min(availableRange.getRight(), currentRange.getLeft() - 1); + final int min = Math.max(availableRange.getLeft(), previousRange.getRight() + 1); + if (max < min) { + continue; + } + availablePort = min; + maxAvailablePort = max; + break; + } + return hasNext; + } + + @Override + public Integer next() { + final int value = this.availablePort; + this.availablePort++; + return value; + } + + public void updateOccupiedRanges() { + final int value = availablePort - 1; + if (value == previousRange.getRight() + 1 && currentRange.getLeft() - 1 == value) { + previousRange.setRight(currentRange.getRight()); + occupiedPortsIterator.remove(); + } + if (value == previousRange.getRight() + 1) { + previousRange.setRight(value); + } + if (value == currentRange.getLeft() - 1) { + currentRange.setLeft(value); + } + preOccupiedPortsIterator.add(new Pair<>(value, value)); + } + } + + // ===========================add and release================================ + + public Integer addPortIfAvailable(final int candidatePort) { + synchronized (occupiedPorts) { + if (occupiedPorts.size() == 1 || occupiedPorts.isEmpty()) { + return null; + } + + ListIterator> occupiedPortsIterator = occupiedPorts.listIterator(); + ListIterator> previousIterator = occupiedPorts.listIterator(); + Pair previousRange = null; + Pair currentRange = occupiedPortsIterator.next(); + while (occupiedPortsIterator.hasNext()) { + previousRange = previousIterator.next(); + currentRange = occupiedPortsIterator.next(); + if (currentRange.getRight() <= candidatePort) { + continue; + } + if (previousRange.getLeft() >= candidatePort) { + break; + } + if (previousRange.getRight() >= candidatePort || currentRange.getLeft() <= candidatePort) { + return null; + } + if (candidatePort == previousRange.getRight() + 1 + && currentRange.getLeft() - 1 == candidatePort) { + previousRange.setRight(currentRange.getRight()); + occupiedPortsIterator.remove(); + return candidatePort; + } + + if (candidatePort == previousRange.getRight() + 1) { + previousRange.setRight(candidatePort); + return candidatePort; + } + + if (candidatePort == currentRange.getLeft() - 1) { + currentRange.setLeft(candidatePort); + return candidatePort; + } + + previousIterator.add(new Pair<>(candidatePort, candidatePort)); + return candidatePort; + } + } + return null; + } + + public void releaseUsedPort(final int port) { + synchronized (occupiedPorts) { + if (occupiedPorts.isEmpty()) { + return; + } + ListIterator> preIterator = occupiedPorts.listIterator(); + ListIterator> iterator = occupiedPorts.listIterator(); + Pair cur = null; + while (iterator.hasNext()) { + cur = iterator.next(); + if (port > cur.getRight()) { + continue; + } + if (port < cur.getLeft()) { + break; + } + if (cur.getLeft().equals(cur.getRight())) { + iterator.remove(); + break; + } + if (cur.getLeft() == port) { + cur.setLeft(port + 1); + break; + } + if (cur.getRight() == port) { + cur.setRight(port - 1); + break; + } + iterator.add(new Pair<>(port + 1, cur.getRight())); + cur.setRight(port - 1); + break; + } + } + } + + // ===========================bing================================ + + public void bingPort( + final int minSendPortRange, + final int maxSendPortRange, + final List candidatePorts, + final Consumer consumer) { + synchronized (occupiedPorts) { + AvailablePortIterator portIterator = + createAvailablePortIterator(minSendPortRange, maxSendPortRange, candidatePorts); + boolean portFound = false; + Exception lastException = null; + while (portIterator.hasNext()) { + try { + consumer.accept(portIterator.next()); + portIterator.updateOccupiedRanges(); + portFound = true; + break; + } catch (Exception e) { + lastException = e; + } + } + if (!portFound) { + LOGGER.warn( + String.format( + "Failed to find an available send port. Custom send port is defined. " + + "No ports are available in the candidate list [%s] or within the range %d to %d.", + candidatePorts, minSendPortRange, maxSendPortRange), + lastException); + } + } + } + + @TestOnly + public void resetPortManager() { + synchronized (occupiedPorts) { + occupiedPorts.clear(); + occupiedPorts.add(new Pair<>(1023, 1023)); + occupiedPorts.add(new Pair<>(65536, 65536)); + } + } + + public AvailablePortIterator createAvailablePortIterator( + final int minSendPortRange, final int maxSendPortRange, final List candidatePorts) { + return new AvailablePortIterator( + occupiedPorts, createPortRanges(candidatePorts, minSendPortRange, maxSendPortRange)); + } + + private List> createPortRanges( + final List candidatePorts, final int minSendPortRange, final int maxSendPortRange) { + List> range = new LinkedList<>(); + if (candidatePorts.isEmpty()) { + range.add(new Pair<>(minSendPortRange, maxSendPortRange)); + return range; + } + Iterator candidatePortIterator = candidatePorts.iterator(); + while (candidatePortIterator.hasNext()) { + int value = candidatePortIterator.next(); + if (value >= minSendPortRange) { + break; + } + range.add(new Pair<>(value, value)); + } + range.add(new Pair<>(minSendPortRange, maxSendPortRange)); + while (candidatePortIterator.hasNext()) { + int value = candidatePortIterator.next(); + if (value <= maxSendPortRange) { + continue; + } + range.add(new Pair<>(value, value)); + } + return range; + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/util/PortUtilizationManager.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/util/PortUtilizationManager.java deleted file mode 100644 index 36251450ac6f..000000000000 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/util/PortUtilizationManager.java +++ /dev/null @@ -1,290 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iotdb.commons.client.util; - -import org.apache.tsfile.utils.Pair; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.Iterator; -import java.util.LinkedList; -import java.util.List; -import java.util.ListIterator; - -public class PortUtilizationManager { - - private static final Logger LOGGER = LoggerFactory.getLogger(PortUtilizationManager.class); - - private final List> occupiedPorts = new LinkedList<>(); - - private PortUtilizationManager() { - occupiedPorts.add(new Pair<>(1023, 1023)); - occupiedPorts.add(new Pair<>(65536, 65536)); - } - - public static final PortUtilizationManager INSTANCE = new PortUtilizationManager(); - - public Integer addIfPortAvailable(int candidatePort) { - synchronized (occupiedPorts) { - if (occupiedPorts.size() == 1 || occupiedPorts.isEmpty()) { - return null; - } - ListIterator> occupiedPortsIterator = occupiedPorts.listIterator(); - Pair previousRange = null; - Pair currentRange = occupiedPortsIterator.next(); - while (occupiedPortsIterator.hasNext()) { - previousRange = currentRange; - currentRange = occupiedPortsIterator.next(); - if (currentRange.getRight() <= candidatePort) { - continue; - } - if (previousRange.getLeft() >= candidatePort) { - break; - } - if (previousRange.getRight() >= candidatePort && currentRange.getLeft() <= candidatePort) { - return null; - } - Integer availablePort = checkPortAvailability(candidatePort); - if (availablePort == null) { - return null; - } - - if (availablePort == previousRange.getRight() + 1 - && currentRange.getLeft() - 1 == availablePort) { - previousRange.setRight(availablePort); - occupiedPortsIterator.remove(); - return availablePort; - } - - if (availablePort == previousRange.getRight() + 1) { - previousRange.setRight(availablePort); - return availablePort; - } - - if (availablePort == currentRange.getLeft() - 1) { - currentRange.setLeft(availablePort); - return availablePort; - } - - occupiedPortsIterator.add(new Pair<>(availablePort, availablePort)); - return availablePort; - } - } - return null; - } - - private Integer addIfPortAvailable(List candidatePorts) { - synchronized (occupiedPorts) { - if (occupiedPorts.size() == 1 || occupiedPorts.isEmpty() || candidatePorts.isEmpty()) { - return null; - } - - ListIterator> occupiedPortsIterator = occupiedPorts.listIterator(); - Iterator integerIterator = candidatePorts.iterator(); - Pair previousRange = null; - Pair currentRange = occupiedPortsIterator.next(); - Integer candidatePort = integerIterator.next(); - out: - while (occupiedPortsIterator.hasNext()) { - previousRange = currentRange; - currentRange = occupiedPortsIterator.next(); - - while (true) { - if (currentRange.getRight() <= candidatePort) { - continue out; - } - while (previousRange.getLeft() >= candidatePort) { - if (!integerIterator.hasNext()) { - return null; - } - candidatePort = integerIterator.next(); - } - if (candidatePort < currentRange.getLeft() - && candidatePort > previousRange.getRight() - && checkPortAvailability(candidatePort) != null) { - break; - } - if (!integerIterator.hasNext()) { - return null; - } - candidatePort = integerIterator.next(); - } - - if (candidatePort == previousRange.getRight() + 1 - && currentRange.getLeft() - 1 == candidatePort) { - previousRange.setRight(candidatePort); - occupiedPortsIterator.remove(); - return candidatePort; - } - - if (candidatePort == previousRange.getRight() + 1) { - previousRange.setRight(candidatePort); - return candidatePort; - } - - if (candidatePort == currentRange.getLeft() - 1) { - currentRange.setLeft(candidatePort); - return candidatePort; - } - - occupiedPortsIterator.add(new Pair<>(candidatePort, candidatePort)); - return candidatePort; - } - } - return null; - } - - private Integer addIfPortAvailable(int minSendPortRange, int maxSendPortRange) { - synchronized (occupiedPorts) { - if (occupiedPorts.size() == 1 || occupiedPorts.isEmpty()) { - return null; - } - - ListIterator> occupiedPortsIterator = occupiedPorts.listIterator(); - Pair previousRange = null; - Pair currentRange = occupiedPortsIterator.next(); - while (occupiedPortsIterator.hasNext()) { - previousRange = currentRange; - currentRange = occupiedPortsIterator.next(); - if (currentRange.getRight() <= minSendPortRange) { - continue; - } - if (previousRange.getLeft() >= maxSendPortRange) { - break; - } - - int max = Math.min(maxSendPortRange, currentRange.getLeft() - 1); - int min = Math.max(minSendPortRange, previousRange.getRight() + 1); - if (max < min) { - return null; - } - Integer availablePort = findAvailablePort(min, max); - if (availablePort == null) { - continue; - } - - if (availablePort == previousRange.getRight() + 1 - && currentRange.getLeft() - 1 == availablePort) { - previousRange.setRight(availablePort); - occupiedPortsIterator.remove(); - return availablePort; - } - - if (availablePort == previousRange.getRight() + 1) { - previousRange.setRight(availablePort); - return availablePort; - } - - if (availablePort == currentRange.getLeft() - 1) { - currentRange.setLeft(availablePort); - return availablePort; - } - - occupiedPortsIterator.add(new Pair<>(availablePort, availablePort)); - return availablePort; - } - } - return null; - } - - private Integer findAvailablePort(int min, int max) { - for (int i = min; i <= max; i++) { - try { - SecurityManager sm = System.getSecurityManager(); - if (sm != null) { - sm.checkListen(i); - } - return i; - } catch (Exception ignored) { - } - } - return null; - } - - private Integer checkPortAvailability(int port) { - try { - SecurityManager sm = System.getSecurityManager(); - if (sm != null) { - sm.checkListen(port); - } - return port; - } catch (Exception ignored) { - } - return null; - } - - public void releasePortIfUsed(int port) { - synchronized (occupiedPorts) { - if (occupiedPorts.isEmpty()) { - return; - } - - ListIterator> iterator = occupiedPorts.listIterator(); - Pair cur = null; - while (iterator.hasNext()) { - cur = iterator.next(); - if (port > cur.getRight()) { - continue; - } - if (port < cur.getLeft()) { - break; - } - if (cur.getLeft() == port) { - if (cur.getLeft().equals(cur.getRight())) { - iterator.remove(); - break; - } - cur.setLeft(cur.getLeft() + 1); - } - - if (cur.getRight() == port) { - if (cur.getLeft().equals(cur.getRight())) { - iterator.remove(); - break; - } - cur.setRight(cur.getRight() - 1); - break; - } - cur.setRight(port - 1); - iterator.add(new Pair<>(port + 1, cur.getRight())); - break; - } - } - } - - public Integer findAvailablePort( - int minSendPortRange, int maxSendPortRange, List candidatePorts) { - Integer port = addIfPortAvailable(candidatePorts); - if (port != null) { - return port; - } else { - LOGGER.debug("No available port found in the candidate list, trying range."); - } - port = addIfPortAvailable(minSendPortRange, maxSendPortRange); - if (port == null) { - String exceptionMessage = - String.format( - "Failed to find an available port within the range %d to %d.", - minSendPortRange, maxSendPortRange); - LOGGER.warn(exceptionMessage); - } - return port; - } -} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/constant/PipeConnectorConstant.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/constant/PipeConnectorConstant.java index 217802ff9a43..8f79a222a092 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/constant/PipeConnectorConstant.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/constant/PipeConnectorConstant.java @@ -55,11 +55,11 @@ public class PipeConnectorConstant { public static final String SINK_IOTDB_SEND_PORT_CANDIDATE_KEY = "sink.send-port.range.candidate"; public static final String CONNECTOR_IOTDB_SEND_PORT_ENABLE_KEY = "connector.send-port.range.enable"; - public static final String SINK_IOTDB_SEND_PORT_ENABLE_KEY = "sink.send-port.range.enable"; + public static final String SINK_IOTDB_SEND_PORT_ENABLE_KEY = "sink.send-port.enable"; public static final int CONNECTOR_IOTDB_SEND_PORT_MIN_VALUE = 0; public static final int CONNECTOR_IOTDB_SEND_PORT_MAX_VALUE = 65535; public static final String CONNECTOR_IOTDB_SEND_PORT_CANDIDATE_VALUE = ""; - public static final boolean CONNECTOR_IOTDB_SEND_PORT_ENABLE_VALUE = true; + public static final boolean CONNECTOR_IOTDB_SEND_PORT_ENABLE_VALUE = false; public static final String SINK_IOTDB_SSL_ENABLE_KEY = "sink.ssl.enable"; public static final String SINK_IOTDB_SSL_TRUST_STORE_PATH_KEY = "sink.ssl.trust-store-path"; diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/client/IoTDBSyncClient.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/client/IoTDBSyncClient.java index aa3438431f4a..9a3be7af5d5c 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/client/IoTDBSyncClient.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/client/IoTDBSyncClient.java @@ -22,7 +22,7 @@ import org.apache.iotdb.common.rpc.thrift.TEndPoint; import org.apache.iotdb.commons.client.ThriftClient; import org.apache.iotdb.commons.client.property.ThriftClientProperty; -import org.apache.iotdb.commons.client.util.PortUtilizationManager; +import org.apache.iotdb.commons.client.util.IoTDBConnectorPortManager; import org.apache.iotdb.commons.pipe.config.PipeConfig; import org.apache.iotdb.commons.pipe.connector.payload.thrift.request.IoTDBConnectorRequestVersion; import org.apache.iotdb.commons.pipe.connector.payload.thrift.request.PipeTransferSliceReq; @@ -90,26 +90,16 @@ public IoTDBSyncClient( final TTransport transport = getInputProtocol().getTransport(); if (!transport.isOpen()) { if (isCustomSendPortDefined) { - PortUtilizationManager portUtilizationManager = PortUtilizationManager.INSTANCE; - Integer sendPort = - portUtilizationManager.findAvailablePort( - minSendPortRange, maxSendPortRange, candidatePorts); - if (sendPort == null) { - throw new PipeConnectionException( - String.format( - "Failed to find an available send port. Custom send port is defined." - + " No ports are available in the candidate list [%s] or within the range %d to %d.", - candidatePorts, minSendPortRange, maxSendPortRange)); - } - try { - final InetSocketAddress isa = new InetSocketAddress(sendPort); - ((TSocket) ((TimeoutChangeableTFastFramedTransport) transport).getSocket()) - .getSocket() - .bind(isa); - } catch (Exception e) { - String bindErrorMessage = "Failed to bind to the port: " + sendPort; - throw new PipeConnectionException(bindErrorMessage, e); - } + IoTDBConnectorPortManager.INSTANCE.bingPort( + minSendPortRange, + maxSendPortRange, + candidatePorts, + (sendPort) -> { + final InetSocketAddress isa = new InetSocketAddress(sendPort); + ((TSocket) ((TimeoutChangeableTFastFramedTransport) transport).getSocket()) + .getSocket() + .bind(isa); + }); } transport.open(); } @@ -200,9 +190,13 @@ public void close() throws Exception { public void invalidate() { if (getInputProtocol().getTransport().isOpen()) { if (isCustomSendPortDefined) { - PortUtilizationManager portUtilizationManager = PortUtilizationManager.INSTANCE; - portUtilizationManager.releasePortIfUsed( - ((TSocket) getInputProtocol().getTransport()).getSocket().getPort()); + IoTDBConnectorPortManager portUtilizationManager = IoTDBConnectorPortManager.INSTANCE; + portUtilizationManager.releaseUsedPort( + ((TSocket) + ((TimeoutChangeableTFastFramedTransport) getInputProtocol().getTransport()) + .getSocket()) + .getSocket() + .getPort()); } getInputProtocol().getTransport().close(); } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/protocol/IoTDBAirGapConnector.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/protocol/IoTDBAirGapConnector.java index da44d2cbe5b2..c053e971179f 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/protocol/IoTDBAirGapConnector.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/protocol/IoTDBAirGapConnector.java @@ -21,7 +21,7 @@ import org.apache.iotdb.common.rpc.thrift.TEndPoint; import org.apache.iotdb.common.rpc.thrift.TSStatus; -import org.apache.iotdb.commons.client.util.PortUtilizationManager; +import org.apache.iotdb.commons.client.util.IoTDBConnectorPortManager; import org.apache.iotdb.commons.pipe.config.PipeConfig; import org.apache.iotdb.commons.pipe.connector.payload.airgap.AirGapELanguageConstant; import org.apache.iotdb.commons.pipe.connector.payload.airgap.AirGapOneByteResponse; @@ -161,7 +161,7 @@ public void handshake() throws Exception { try { final Socket socket = sockets.set(i, null); if (isCustomSendPortDefined) { - PortUtilizationManager.INSTANCE.releasePortIfUsed(socket.getPort()); + IoTDBConnectorPortManager.INSTANCE.releaseUsedPort(socket.getPort()); } socket.close(); } catch (final Exception e) { @@ -175,13 +175,14 @@ public void handshake() throws Exception { final AirGapSocket socket = new AirGapSocket(ip, port); + if (isCustomSendPortDefined) { + IoTDBConnectorPortManager.INSTANCE.bingPort( + minSendPortRange, + maxSendPortRange, + candidatePorts, + (sendPort) -> socket.bind(new InetSocketAddress(sendPort))); + } try { - if (isCustomSendPortDefined) { - socket.bind( - new InetSocketAddress( - PortUtilizationManager.INSTANCE.findAvailablePort( - minSendPortRange, maxSendPortRange, candidatePorts))); - } socket.connect(new InetSocketAddress(ip, port), handshakeTimeoutMs); socket.setKeepAlive(true); sockets.set(i, socket); @@ -351,7 +352,7 @@ public void close() { if (sockets.get(i) != null) { final Socket socket = sockets.set(i, null); if (isCustomSendPortDefined) { - PortUtilizationManager.INSTANCE.releasePortIfUsed(socket.getPort()); + IoTDBConnectorPortManager.INSTANCE.releaseUsedPort(socket.getPort()); } socket.close(); } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/utils/function/Consumer.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/utils/function/Consumer.java new file mode 100644 index 000000000000..e6c84451d790 --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/utils/function/Consumer.java @@ -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.iotdb.commons.utils.function; + +public interface Consumer { + void accept(INPUT1 var1) throws THROWABLE; +} diff --git a/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/client/IoTDBConnectorPortManagerTest.java b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/client/IoTDBConnectorPortManagerTest.java new file mode 100644 index 000000000000..ca8855a04da1 --- /dev/null +++ b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/client/IoTDBConnectorPortManagerTest.java @@ -0,0 +1,105 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.commons.client; + +import org.apache.iotdb.commons.client.util.IoTDBConnectorPortManager; + +import org.apache.tsfile.utils.Pair; +import org.junit.Assert; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.LinkedList; +import java.util.List; +import java.util.Random; +import java.util.Set; + +public class IoTDBConnectorPortManagerTest { + + IoTDBConnectorPortManager portManager = IoTDBConnectorPortManager.INSTANCE; + + @Test + public void testAddIfPortAvailable() { + portManager.resetPortManager(); + Set ports = new HashSet<>(); + portManager.resetPortManager(); + Set set = new HashSet<>(); + addPort(set); + Assert.assertEquals(greEX(set), portManager.getOccupiedPorts()); + } + + @Test + public void releasePortIfUsed() { + portManager.resetPortManager(); + Set set = new HashSet<>(); + addPort(set); + Random random = new Random(); + while (set.size() > 200) { + int port = 1024 + random.nextInt(65535 - 1024); + set.remove(port); + portManager.releaseUsedPort(port); + } + Assert.assertEquals(greEX(set), portManager.getOccupiedPorts()); + } + + private void addPort(Set set) { + set.add(1023); + set.add(65536); + Random random = new Random(); + + while (set.size() < 300) { + int port = 1024 + random.nextInt(65535 - 1024); + set.add(port); + portManager.addPortIfAvailable(port); + } + for (int i = 0; i <= 100; i++) { + int port = i + 1024; + set.add(port); + portManager.addPortIfAvailable(port); + port = 65535 - i; + if (!set.contains(port)) { + set.add(port); + set.add(port); + } + portManager.addPortIfAvailable(port); + } + } + + private List> greEX(Set set) { + List ports = new ArrayList<>(set); + ports.sort(Integer::compare); + + int start = ports.get(0); + int end = start; + List> data = new LinkedList<>(); + for (int i = 1; i < ports.size(); i++) { + if (ports.get(i) == end + 1) { + end = ports.get(i); + } else { + data.add(new Pair<>(start, end)); + start = ports.get(i); + end = start; + } + } + data.add(new Pair<>(start, end)); + return data; + } +} From 46f4703e062a4d1b6140229296a545fc074e4dad Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Wed, 25 Sep 2024 15:52:44 +0800 Subject: [PATCH 03/12] Add unit test --- .../util/IoTDBConnectorPortManager.java | 153 +++++++++++++----- .../client/IoTDBConnectorPortManagerTest.java | 70 ++++++-- 2 files changed, 175 insertions(+), 48 deletions(-) diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/util/IoTDBConnectorPortManager.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/util/IoTDBConnectorPortManager.java index f64caaf254f6..e88e83c1adc4 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/util/IoTDBConnectorPortManager.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/util/IoTDBConnectorPortManager.java @@ -21,6 +21,7 @@ import org.apache.iotdb.commons.utils.TestOnly; import org.apache.iotdb.commons.utils.function.Consumer; +import org.apache.iotdb.pipe.api.exception.PipeConnectionException; import org.apache.tsfile.utils.Pair; import org.slf4j.Logger; @@ -31,6 +32,9 @@ import java.util.List; import java.util.ListIterator; +// By recording the port range that has been used, the problem of brute force iterative query of +// available ports can be reduced. For example, if ports 1, 2, 3, 4, 5, 6, 7, 8, 9, 11, 12, and 14 +// are used, they are recorded as 1-9, 11-12, 14, and the order is guaranteed. public class IoTDBConnectorPortManager { private static final Logger LOGGER = LoggerFactory.getLogger(IoTDBConnectorPortManager.class); @@ -41,8 +45,9 @@ public List> getOccupiedPorts() { private final List> occupiedPorts = new LinkedList<>(); + // Set the port boundaries private IoTDBConnectorPortManager() { - occupiedPorts.add(new Pair<>(1023, 1023)); + occupiedPorts.add(new Pair<>(-1, -1)); occupiedPorts.add(new Pair<>(65536, 65536)); } @@ -50,27 +55,33 @@ private IoTDBConnectorPortManager() { // ===========================Iterator================================ + // Iterator can be used to iterate over undocumented ports public static class AvailablePortIterator implements Iterator { - private final ListIterator> occupiedPortsIterator; - private final Iterator> availableRangesIterator; + private ListIterator> occupiedPortsIterator; private ListIterator> preOccupiedPortsIterator; + private Iterator> availableRangesIterator; private Pair availableRange; private Pair previousRange; private Pair currentRange; private boolean hasNext = true; - private int availablePort = 0; - private int maxAvailablePort = -1; + private int availablePort = -1; + private int maxAvailablePort = -2; AvailablePortIterator( final List> occupiedPorts, final List> availableRanges) { if (occupiedPorts.size() <= 1 || availableRanges.isEmpty()) { hasNext = false; + return; } this.occupiedPortsIterator = occupiedPorts.listIterator(); + this.preOccupiedPortsIterator = occupiedPorts.listIterator(); this.availableRangesIterator = availableRanges.iterator(); this.availableRange = availableRangesIterator.next(); this.currentRange = occupiedPortsIterator.next(); + previousRange = currentRange; + currentRange = occupiedPortsIterator.next(); + preOccupiedPortsIterator.next(); } @Override @@ -81,52 +92,98 @@ public boolean hasNext() { if (availablePort <= maxAvailablePort) { return true; } - while (occupiedPortsIterator.hasNext()) { - previousRange = currentRange; - currentRange = occupiedPortsIterator.next(); - preOccupiedPortsIterator.next(); + if (availablePort != -1) { + if (availableRange.getRight() <= currentRange.right) { + if (!updateAvailablePort()) { + return hasNext = false; + } + } else { + if (!updateCurrentRanges()) { + return hasNext = false; + } + } + } + out: + while (true) { if (currentRange.getRight() <= availableRange.getLeft()) { + if (!updateCurrentRanges()) { + return hasNext = false; + } continue; } - while (previousRange.getLeft() >= availableRange.getRight()) { - if (availableRangesIterator.hasNext()) { - availableRange = availableRangesIterator.next(); - } else { - hasNext = false; - break; + if (previousRange.getLeft() >= availableRange.getRight()) { + if (!updateAvailablePort()) { + return hasNext = false; } + continue; } - final int max = Math.min(availableRange.getRight(), currentRange.getLeft() - 1); final int min = Math.max(availableRange.getLeft(), previousRange.getRight() + 1); if (max < min) { + if (availableRange.getRight() <= currentRange.right) { + if (!updateAvailablePort()) { + return hasNext = false; + } + continue; + } + if (!updateCurrentRanges()) { + return hasNext = false; + } continue; } availablePort = min; maxAvailablePort = max; - break; + return hasNext = true; + } + } + + private boolean updateAvailablePort() { + if (!availableRangesIterator.hasNext()) { + return false; + } + availableRange = availableRangesIterator.next(); + return true; + } + + private boolean updateCurrentRanges() { + if (!occupiedPortsIterator.hasNext()) { + return false; } - return hasNext; + previousRange = currentRange; + currentRange = occupiedPortsIterator.next(); + preOccupiedPortsIterator.next(); + return true; } @Override public Integer next() { + if (availablePort > maxAvailablePort) { + if (!hasNext()) { + throw new PipeConnectionException("No more available ports to iterate."); + } + } final int value = this.availablePort; this.availablePort++; return value; } - public void updateOccupiedRanges() { + public void updateOccupiedPort() { + if (availablePort == -1) { + throw new IllegalStateException("Available port not initialized."); + } final int value = availablePort - 1; if (value == previousRange.getRight() + 1 && currentRange.getLeft() - 1 == value) { previousRange.setRight(currentRange.getRight()); occupiedPortsIterator.remove(); + return; } if (value == previousRange.getRight() + 1) { previousRange.setRight(value); + return; } if (value == currentRange.getLeft() - 1) { currentRange.setLeft(value); + return; } preOccupiedPortsIterator.add(new Pair<>(value, value)); } @@ -134,10 +191,10 @@ public void updateOccupiedRanges() { // ===========================add and release================================ - public Integer addPortIfAvailable(final int candidatePort) { + public boolean addPortIfAvailable(final int candidatePort) { synchronized (occupiedPorts) { if (occupiedPorts.size() == 1 || occupiedPorts.isEmpty()) { - return null; + return false; } ListIterator> occupiedPortsIterator = occupiedPorts.listIterator(); @@ -154,30 +211,30 @@ public Integer addPortIfAvailable(final int candidatePort) { break; } if (previousRange.getRight() >= candidatePort || currentRange.getLeft() <= candidatePort) { - return null; + return false; } if (candidatePort == previousRange.getRight() + 1 && currentRange.getLeft() - 1 == candidatePort) { previousRange.setRight(currentRange.getRight()); occupiedPortsIterator.remove(); - return candidatePort; + return true; } if (candidatePort == previousRange.getRight() + 1) { previousRange.setRight(candidatePort); - return candidatePort; + return true; } if (candidatePort == currentRange.getLeft() - 1) { currentRange.setLeft(candidatePort); - return candidatePort; + return true; } previousIterator.add(new Pair<>(candidatePort, candidatePort)); - return candidatePort; + return true; } } - return null; + return false; } public void releaseUsedPort(final int port) { @@ -185,7 +242,6 @@ public void releaseUsedPort(final int port) { if (occupiedPorts.isEmpty()) { return; } - ListIterator> preIterator = occupiedPorts.listIterator(); ListIterator> iterator = occupiedPorts.listIterator(); Pair cur = null; while (iterator.hasNext()) { @@ -230,7 +286,7 @@ public void bingPort( while (portIterator.hasNext()) { try { consumer.accept(portIterator.next()); - portIterator.updateOccupiedRanges(); + portIterator.updateOccupiedPort(); portFound = true; break; } catch (Exception e) { @@ -238,12 +294,13 @@ public void bingPort( } } if (!portFound) { - LOGGER.warn( + String exceptionMessage = String.format( "Failed to find an available send port. Custom send port is defined. " + "No ports are available in the candidate list [%s] or within the range %d to %d.", - candidatePorts, minSendPortRange, maxSendPortRange), - lastException); + candidatePorts, minSendPortRange, maxSendPortRange); + LOGGER.warn(exceptionMessage, lastException); + throw new PipeConnectionException(exceptionMessage); } } } @@ -252,7 +309,7 @@ public void bingPort( public void resetPortManager() { synchronized (occupiedPorts) { occupiedPorts.clear(); - occupiedPorts.add(new Pair<>(1023, 1023)); + occupiedPorts.add(new Pair<>(-1, -1)); occupiedPorts.add(new Pair<>(65536, 65536)); } } @@ -271,21 +328,45 @@ private List> createPortRanges( return range; } Iterator candidatePortIterator = candidatePorts.iterator(); + int tempValue = -1; while (candidatePortIterator.hasNext()) { int value = candidatePortIterator.next(); if (value >= minSendPortRange) { + tempValue = value; break; } - range.add(new Pair<>(value, value)); + addPair(range, value); + } + addPair(range, minSendPortRange, maxSendPortRange); + if (tempValue != -1) { + if (tempValue > maxSendPortRange) { + addPair(range, tempValue); + } } - range.add(new Pair<>(minSendPortRange, maxSendPortRange)); + while (candidatePortIterator.hasNext()) { int value = candidatePortIterator.next(); if (value <= maxSendPortRange) { continue; } - range.add(new Pair<>(value, value)); + addPair(range, value); } return range; } + + private void addPair(List> range, int value) { + if (!range.isEmpty() && range.get(range.size() - 1).getRight() == value - 1) { + range.get(range.size() - 1).setRight(value); + } else { + range.add(new Pair<>(value, value)); + } + } + + private void addPair(List> range, int minValue, int maxValue) { + if (!range.isEmpty() && range.get(range.size() - 1).getRight() == minValue - 1) { + range.get(range.size() - 1).setRight(maxValue); + } else { + range.add(new Pair<>(minValue, maxValue)); + } + } } diff --git a/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/client/IoTDBConnectorPortManagerTest.java b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/client/IoTDBConnectorPortManagerTest.java index ca8855a04da1..004343655c96 100644 --- a/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/client/IoTDBConnectorPortManagerTest.java +++ b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/client/IoTDBConnectorPortManagerTest.java @@ -34,16 +34,14 @@ public class IoTDBConnectorPortManagerTest { - IoTDBConnectorPortManager portManager = IoTDBConnectorPortManager.INSTANCE; + private final IoTDBConnectorPortManager portManager = IoTDBConnectorPortManager.INSTANCE; @Test public void testAddIfPortAvailable() { - portManager.resetPortManager(); - Set ports = new HashSet<>(); portManager.resetPortManager(); Set set = new HashSet<>(); addPort(set); - Assert.assertEquals(greEX(set), portManager.getOccupiedPorts()); + Assert.assertEquals(generateExpectedRanges(set), portManager.getOccupiedPorts()); } @Test @@ -57,16 +55,65 @@ public void releasePortIfUsed() { set.remove(port); portManager.releaseUsedPort(port); } - Assert.assertEquals(greEX(set), portManager.getOccupiedPorts()); + Assert.assertEquals(generateExpectedRanges(set), portManager.getOccupiedPorts()); } - private void addPort(Set set) { - set.add(1023); - set.add(65536); - Random random = new Random(); + @Test + public void testAvailablePortIterator() { + portManager.resetPortManager(); + Set set = new HashSet<>(); + addPort(set); + testPortRange(1024, 65535, new HashSet<>(), set); + Random r = new Random(); + Set cp = new HashSet<>(); + for (int i = 0; i < 1; i++) { + IoTDBConnectorPortManager.AvailablePortIterator iterator = + portManager.createAvailablePortIterator(1024, 65535, new ArrayList<>()); + while (iterator.hasNext()) { + int port = iterator.next(); + if (r.nextBoolean()) { + iterator.updateOccupiedPort(); + set.add(port); + break; + } + } + testPortRange(1024, 6553, cp, set); + int randomPort = 1024 + r.nextInt(65535 - 1024 - 200); + cp.clear(); + for (int j = 0; j < 40; j++) { + cp.add(1024 + r.nextInt(65535 - 1024)); + } + testPortRange(randomPort, randomPort + 200, cp, set); + } + } + private void testPortRange(int port, int maxPort, Set cp, Set set) { + int start = port; + int end = maxPort; + List candidatePorts = new ArrayList<>(cp); + if (!candidatePorts.isEmpty()) { + candidatePorts.sort(Integer::compare); + start = Math.min(port, candidatePorts.get(0)); + end = Math.max(maxPort, candidatePorts.get(candidatePorts.size() - 1)); + } + IoTDBConnectorPortManager.AvailablePortIterator iterator = + portManager.createAvailablePortIterator(port, maxPort, candidatePorts); + for (int i = start; i <= end; i++) { + if (set.contains(i) || ((i < port || i > maxPort) && !cp.contains(i))) { + continue; + } + Assert.assertTrue(iterator.hasNext()); + Assert.assertEquals(i, iterator.next().intValue()); + } + Assert.assertFalse(iterator.hasNext()); + } + + private void addPort(final Set set) { + set.add(-1); + set.add(65536); + final Random random = new Random(); while (set.size() < 300) { - int port = 1024 + random.nextInt(65535 - 1024); + final int port = 1024 + random.nextInt(65535 - 1024); set.add(port); portManager.addPortIfAvailable(port); } @@ -83,10 +130,9 @@ private void addPort(Set set) { } } - private List> greEX(Set set) { + private List> generateExpectedRanges(final Set set) { List ports = new ArrayList<>(set); ports.sort(Integer::compare); - int start = ports.get(0); int end = start; List> data = new LinkedList<>(); From f50b29561946d8c41cd8fe9ea41f77c440941dff Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Wed, 25 Sep 2024 16:23:29 +0800 Subject: [PATCH 04/12] modify code --- .../commons/pipe/connector/protocol/IoTDBSslSyncConnector.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/protocol/IoTDBSslSyncConnector.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/protocol/IoTDBSslSyncConnector.java index 708109872926..c4b76932d5d7 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/protocol/IoTDBSslSyncConnector.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/protocol/IoTDBSslSyncConnector.java @@ -143,8 +143,7 @@ protected abstract IoTDBSyncClientManager constructClient( final boolean isCustomSendPortDefined, final int minSendPortRange, final int maxSendPortRange, - fianl List candidatePorts); - + final List candidatePorts); @Override public void handshake() throws Exception { From 8e370a7879d0133e16a216a4a756e749fd5336cf Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Wed, 25 Sep 2024 23:46:59 +0800 Subject: [PATCH 05/12] modify code --- .../IoTDBPipeConnectorCustomPortIT.java | 252 ++++++++++++ .../AsyncPipeDataTransferServiceClient.java | 23 +- .../client/util/IoTDBConnectorPortBinder.java | 64 +++ .../util/IoTDBConnectorPortManager.java | 372 ------------------ .../constant/PipeConnectorConstant.java | 9 +- .../connector/client/IoTDBSyncClient.java | 35 +- .../protocol/IoTDBAirGapConnector.java | 13 +- .../connector/protocol/IoTDBConnector.java | 2 +- .../client/IoTDBConnectorPortManagerTest.java | 151 ------- 9 files changed, 341 insertions(+), 580 deletions(-) create mode 100644 integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeConnectorCustomPortIT.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/util/IoTDBConnectorPortBinder.java delete mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/util/IoTDBConnectorPortManager.java delete mode 100644 iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/client/IoTDBConnectorPortManagerTest.java diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeConnectorCustomPortIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeConnectorCustomPortIT.java new file mode 100644 index 000000000000..3809d27408fd --- /dev/null +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeConnectorCustomPortIT.java @@ -0,0 +1,252 @@ +/* + * 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.iotdb.pipe.it.autocreate; + +import org.apache.iotdb.common.rpc.thrift.TSStatus; +import org.apache.iotdb.commons.client.sync.SyncConfigNodeIServiceClient; +import org.apache.iotdb.confignode.rpc.thrift.TCreatePipeReq; +import org.apache.iotdb.consensus.ConsensusFactory; +import org.apache.iotdb.db.it.utils.TestUtils; +import org.apache.iotdb.it.env.MultiEnvFactory; +import org.apache.iotdb.it.framework.IoTDBTestRunner; +import org.apache.iotdb.itbase.category.MultiClusterIT2AutoCreateSchema; +import org.apache.iotdb.rpc.TSStatusCode; + +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; + +@RunWith(IoTDBTestRunner.class) +@Category({MultiClusterIT2AutoCreateSchema.class}) +public class IoTDBPipeConnectorCustomPortIT extends AbstractPipeDualAutoIT { + + @Override + @Before + public void setUp() { + // Override to enable air-gap + MultiEnvFactory.createEnv(2); + senderEnv = MultiEnvFactory.getEnv(0); + receiverEnv = MultiEnvFactory.getEnv(1); + + senderEnv + .getConfig() + .getCommonConfig() + .setAutoCreateSchemaEnabled(true) + .setConfigNodeConsensusProtocolClass(ConsensusFactory.RATIS_CONSENSUS) + .setSchemaRegionConsensusProtocolClass(ConsensusFactory.RATIS_CONSENSUS); + receiverEnv + .getConfig() + .getCommonConfig() + .setAutoCreateSchemaEnabled(true) + .setPipeAirGapReceiverEnabled(true) + .setConfigNodeConsensusProtocolClass(ConsensusFactory.RATIS_CONSENSUS) + .setSchemaRegionConsensusProtocolClass(ConsensusFactory.RATIS_CONSENSUS); + + // 10 min, assert that the operations will not time out + senderEnv.getConfig().getCommonConfig().setCnConnectionTimeoutMs(600000); + receiverEnv.getConfig().getCommonConfig().setCnConnectionTimeoutMs(600000); + + senderEnv.initClusterEnvironment(); + receiverEnv.initClusterEnvironment(); + } + + @Test + public void testPortsAvailable() { + try (final SyncConfigNodeIServiceClient client = + (SyncConfigNodeIServiceClient) senderEnv.getLeaderConfigNodeConnection()) { + if (!TestUtils.tryExecuteNonQueriesWithRetry( + senderEnv, + Arrays.asList( + "insert into root.db.d1(time, s1) values (2010-01-01T10:00:00+08:00, 1)", + "insert into root.db.d1(time, s1) values (2010-01-02T10:00:00+08:00, 2)", + "flush"))) { + return; + } + final Map extractorAttributes = new HashMap<>(); + final Map processorAttributes = new HashMap<>(); + final Map connectorAttributes = new HashMap<>(); + extractorAttributes.put("realtime.mode", "forced-log"); + connectorAttributes.put("connector", "iotdb-thrift-async-connector"); + connectorAttributes.put("batch.enable", "false"); + connectorAttributes.put("node-urls", receiverEnv.getIP() + ":" + receiverEnv.getPort()); + connectorAttributes.put("send-port.enable", "true"); + connectorAttributes.put("send-port.range.min", "1024"); + connectorAttributes.put("send-port.range.max", "1035"); + connectorAttributes.put("parallel.tasks", "16"); + + final TSStatus status = + client.createPipe( + new TCreatePipeReq("p1", connectorAttributes) + .setExtractorAttributes(extractorAttributes) + .setProcessorAttributes(processorAttributes)); + + Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode()); + Thread.sleep(5000); + TestUtils.tryExecuteNonQueriesWithRetry( + senderEnv, + Arrays.asList( + "insert into root.db.d1(time, s1) values (2010-01-01T10:00:00+08:00, 1)", + "insert into root.db.d1(time, s1) values (2010-01-02T10:00:00+08:00, 2)", + "flush")); + TestUtils.assertDataEventuallyOnEnv( + receiverEnv, "select count(*) from root.**", "Time,", new HashSet<>()); + } catch (Exception e) { + Assert.fail(); + } + } + + @Test + public void testAsyncConnector() { + try (final SyncConfigNodeIServiceClient client = + (SyncConfigNodeIServiceClient) senderEnv.getLeaderConfigNodeConnection()) { + final Map extractorAttributes = new HashMap<>(); + final Map processorAttributes = new HashMap<>(); + final Map connectorAttributes = new HashMap<>(); + extractorAttributes.put("realtime.mode", "forced-log"); + connectorAttributes.put("connector", "iotdb-thrift-async-connector"); + connectorAttributes.put("batch.enable", "false"); + connectorAttributes.put("node-urls", receiverEnv.getIP() + ":" + receiverEnv.getPort()); + connectorAttributes.put("send-port.enable", "true"); + connectorAttributes.put("send-port.range.min", "1024"); + connectorAttributes.put("send-port.range.max", "1124"); + connectorAttributes.put("parallel.tasks", "3"); + + final TSStatus status = + client.createPipe( + new TCreatePipeReq("p1", connectorAttributes) + .setExtractorAttributes(extractorAttributes) + .setProcessorAttributes(processorAttributes)); + + Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode()); + + Thread.sleep(10000); + TestUtils.tryExecuteNonQueriesWithRetry( + senderEnv, + Arrays.asList( + "insert into root.db.d1(time, s1) values (2010-01-01T10:00:00+08:00, 1)", + "insert into root.db.d1(time, s1) values (2010-01-02T10:00:00+08:00, 2)", + "flush")); + TestUtils.assertDataEventuallyOnEnv( + receiverEnv, + "select count(*) from root.**", + "count(root.db.d1.s1),", + Collections.singleton("2,")); + + } catch (Exception e) { + Assert.fail(); + } + } + + @Test + public void testSyncThriftConnector() { + try (final SyncConfigNodeIServiceClient client = + (SyncConfigNodeIServiceClient) senderEnv.getLeaderConfigNodeConnection()) { + final Map extractorAttributes = new HashMap<>(); + final Map processorAttributes = new HashMap<>(); + final Map connectorAttributes = new HashMap<>(); + extractorAttributes.put("realtime.mode", "forced-log"); + connectorAttributes.put("connector", "iotdb-thrift-sync-connector"); + connectorAttributes.put("batch.enable", "false"); + connectorAttributes.put("node-urls", receiverEnv.getIP() + ":" + receiverEnv.getPort()); + connectorAttributes.put("send-port.enable", "true"); + connectorAttributes.put("send-port.range.min", "1024"); + connectorAttributes.put("send-port.range.max", "1124"); + connectorAttributes.put("parallel.tasks", "3"); + + final TSStatus status = + client.createPipe( + new TCreatePipeReq("p1", connectorAttributes) + .setExtractorAttributes(extractorAttributes) + .setProcessorAttributes(processorAttributes)); + + Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode()); + + Thread.sleep(2000); + TestUtils.tryExecuteNonQueriesWithRetry( + senderEnv, + Arrays.asList( + "insert into root.db.d1(time, s1) values (2010-01-01T10:00:00+08:00, 1)", + "insert into root.db.d1(time, s1) values (2010-01-02T10:00:00+08:00, 2)", + "flush")); + TestUtils.assertDataEventuallyOnEnv( + receiverEnv, + "select count(*) from root.**", + "count(root.db.d1.s1),", + Collections.singleton("2,")); + + } catch (Exception e) { + Assert.fail(); + } + } + + @Test + public void TestAirGap() { + try (final SyncConfigNodeIServiceClient client = + (SyncConfigNodeIServiceClient) senderEnv.getLeaderConfigNodeConnection()) { + final Map extractorAttributes = new HashMap<>(); + final Map processorAttributes = new HashMap<>(); + final Map connectorAttributes = new HashMap<>(); + extractorAttributes.put("realtime.mode", "forced-log"); + connectorAttributes.put("connector", "iotdb-air-gap-connector"); + connectorAttributes.put("batch.enable", "false"); + connectorAttributes.put( + "node-urls", + receiverEnv.getIP() + + ":" + + receiverEnv.getDataNodeWrapper(0).getPipeAirGapReceiverPort()); + connectorAttributes.put("send-port.enable", "true"); + connectorAttributes.put("send-port.range.min", "1024"); + connectorAttributes.put("send-port.range.max", "1124"); + connectorAttributes.put("parallel.tasks", "3"); + + final TSStatus status = + client.createPipe( + new TCreatePipeReq("p1", connectorAttributes) + .setExtractorAttributes(extractorAttributes) + .setProcessorAttributes(processorAttributes)); + + Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode()); + + Thread.sleep(2000); + TestUtils.tryExecuteNonQueriesWithRetry( + senderEnv, + Arrays.asList( + "insert into root.db.d1(time, s1) values (2010-01-01T10:00:00+08:00, 1)", + "insert into root.db.d1(time, s1) values (2010-01-02T10:00:00+08:00, 2)", + "flush")); + TestUtils.assertDataEventuallyOnEnv( + receiverEnv, + "select count(*) from root.**", + "count(root.db.d1.s1),", + Collections.singleton("2,")); + + } catch (Exception e) { + Assert.fail(); + } + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/async/AsyncPipeDataTransferServiceClient.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/async/AsyncPipeDataTransferServiceClient.java index 628f9b716c9c..4b3220a5c03f 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/async/AsyncPipeDataTransferServiceClient.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/async/AsyncPipeDataTransferServiceClient.java @@ -24,7 +24,7 @@ import org.apache.iotdb.commons.client.ThriftClient; import org.apache.iotdb.commons.client.factory.AsyncThriftClientFactory; import org.apache.iotdb.commons.client.property.ThriftClientProperty; -import org.apache.iotdb.commons.client.util.IoTDBConnectorPortManager; +import org.apache.iotdb.commons.client.util.IoTDBConnectorPortBinder; import org.apache.iotdb.rpc.TNonblockingSocketWrapper; import org.apache.iotdb.service.rpc.thrift.IClientRPCService; @@ -78,24 +78,13 @@ public AsyncPipeDataTransferServiceClient( endpoint.getIp(), endpoint.getPort(), property.getConnectionTimeoutMs())); SocketChannel socketChannel = ((TNonblockingSocket) ___transport).getSocketChannel(); if (isCustomSendPortDefined) { - IoTDBConnectorPortManager.INSTANCE.bingPort( + IoTDBConnectorPortBinder.bindPort( minSendPortRange, maxSendPortRange, candidatePorts, (sendPort) -> { socketChannel.bind(new InetSocketAddress(sendPort)); }); - } else { - try { - socketChannel.bind(new InetSocketAddress(0)); - IoTDBConnectorPortManager.INSTANCE.addPortIfAvailable( - ((InetSocketAddress) - (((TNonblockingSocket) ___transport).getSocketChannel().getLocalAddress())) - .getPort()); - } catch (Exception e) { - LOGGER.warn( - "Failed to add port to PortUtilizationManager due to exception: {}", e.getMessage()); - } } setTimeout(property.getConnectionTimeoutMs()); this.printLogWhenEncounterException = property.isPrintLogWhenEncounterException(); @@ -157,14 +146,6 @@ public void setTimeoutDynamically(int timeout) { } private void close() { - try { - IoTDBConnectorPortManager.INSTANCE.releaseUsedPort( - ((InetSocketAddress) - (((TNonblockingSocket) ___transport).getSocketChannel().getLocalAddress())) - .getPort()); - } catch (Exception e) { - LOGGER.warn("Failed to release port due to exception: ", e); - } ___transport.close(); ___currentMethod = null; } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/util/IoTDBConnectorPortBinder.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/util/IoTDBConnectorPortBinder.java new file mode 100644 index 000000000000..049e926baa87 --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/util/IoTDBConnectorPortBinder.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.commons.client.util; + +import org.apache.iotdb.commons.utils.function.Consumer; +import org.apache.iotdb.pipe.api.exception.PipeConnectionException; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Iterator; +import java.util.List; + +public class IoTDBConnectorPortBinder { + + private static final Logger LOGGER = LoggerFactory.getLogger(IoTDBConnectorPortBinder.class); + + // ===========================bind================================ + + public static void bindPort( + final int minSendPortRange, + final int maxSendPortRange, + final List candidatePorts, + final Consumer consumer) { + boolean portFound = false; + Iterator iterator = candidatePorts.iterator(); + int port = minSendPortRange; + while (iterator.hasNext() || port <= maxSendPortRange) { + try { + int bindPort = iterator.hasNext() ? iterator.next() : port++; + consumer.accept(bindPort); + portFound = true; + break; + } catch (Exception ignored) { + } + } + if (!portFound) { + String exceptionMessage = + String.format( + "Failed to find an available send port. Custom send port is defined. " + + "No ports are available in the candidate list [%s] or within the range %d to %d.", + candidatePorts, minSendPortRange, maxSendPortRange); + LOGGER.warn(exceptionMessage); + throw new PipeConnectionException(exceptionMessage); + } + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/util/IoTDBConnectorPortManager.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/util/IoTDBConnectorPortManager.java deleted file mode 100644 index e88e83c1adc4..000000000000 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/util/IoTDBConnectorPortManager.java +++ /dev/null @@ -1,372 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iotdb.commons.client.util; - -import org.apache.iotdb.commons.utils.TestOnly; -import org.apache.iotdb.commons.utils.function.Consumer; -import org.apache.iotdb.pipe.api.exception.PipeConnectionException; - -import org.apache.tsfile.utils.Pair; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.Iterator; -import java.util.LinkedList; -import java.util.List; -import java.util.ListIterator; - -// By recording the port range that has been used, the problem of brute force iterative query of -// available ports can be reduced. For example, if ports 1, 2, 3, 4, 5, 6, 7, 8, 9, 11, 12, and 14 -// are used, they are recorded as 1-9, 11-12, 14, and the order is guaranteed. -public class IoTDBConnectorPortManager { - - private static final Logger LOGGER = LoggerFactory.getLogger(IoTDBConnectorPortManager.class); - - public List> getOccupiedPorts() { - return occupiedPorts; - } - - private final List> occupiedPorts = new LinkedList<>(); - - // Set the port boundaries - private IoTDBConnectorPortManager() { - occupiedPorts.add(new Pair<>(-1, -1)); - occupiedPorts.add(new Pair<>(65536, 65536)); - } - - public static final IoTDBConnectorPortManager INSTANCE = new IoTDBConnectorPortManager(); - - // ===========================Iterator================================ - - // Iterator can be used to iterate over undocumented ports - public static class AvailablePortIterator implements Iterator { - private ListIterator> occupiedPortsIterator; - private ListIterator> preOccupiedPortsIterator; - private Iterator> availableRangesIterator; - private Pair availableRange; - private Pair previousRange; - private Pair currentRange; - private boolean hasNext = true; - private int availablePort = -1; - private int maxAvailablePort = -2; - - AvailablePortIterator( - final List> occupiedPorts, - final List> availableRanges) { - if (occupiedPorts.size() <= 1 || availableRanges.isEmpty()) { - hasNext = false; - return; - } - this.occupiedPortsIterator = occupiedPorts.listIterator(); - this.preOccupiedPortsIterator = occupiedPorts.listIterator(); - this.availableRangesIterator = availableRanges.iterator(); - this.availableRange = availableRangesIterator.next(); - this.currentRange = occupiedPortsIterator.next(); - previousRange = currentRange; - currentRange = occupiedPortsIterator.next(); - preOccupiedPortsIterator.next(); - } - - @Override - public boolean hasNext() { - if (!hasNext) { - return false; - } - if (availablePort <= maxAvailablePort) { - return true; - } - if (availablePort != -1) { - if (availableRange.getRight() <= currentRange.right) { - if (!updateAvailablePort()) { - return hasNext = false; - } - } else { - if (!updateCurrentRanges()) { - return hasNext = false; - } - } - } - out: - while (true) { - if (currentRange.getRight() <= availableRange.getLeft()) { - if (!updateCurrentRanges()) { - return hasNext = false; - } - continue; - } - if (previousRange.getLeft() >= availableRange.getRight()) { - if (!updateAvailablePort()) { - return hasNext = false; - } - continue; - } - final int max = Math.min(availableRange.getRight(), currentRange.getLeft() - 1); - final int min = Math.max(availableRange.getLeft(), previousRange.getRight() + 1); - if (max < min) { - if (availableRange.getRight() <= currentRange.right) { - if (!updateAvailablePort()) { - return hasNext = false; - } - continue; - } - if (!updateCurrentRanges()) { - return hasNext = false; - } - continue; - } - availablePort = min; - maxAvailablePort = max; - return hasNext = true; - } - } - - private boolean updateAvailablePort() { - if (!availableRangesIterator.hasNext()) { - return false; - } - availableRange = availableRangesIterator.next(); - return true; - } - - private boolean updateCurrentRanges() { - if (!occupiedPortsIterator.hasNext()) { - return false; - } - previousRange = currentRange; - currentRange = occupiedPortsIterator.next(); - preOccupiedPortsIterator.next(); - return true; - } - - @Override - public Integer next() { - if (availablePort > maxAvailablePort) { - if (!hasNext()) { - throw new PipeConnectionException("No more available ports to iterate."); - } - } - final int value = this.availablePort; - this.availablePort++; - return value; - } - - public void updateOccupiedPort() { - if (availablePort == -1) { - throw new IllegalStateException("Available port not initialized."); - } - final int value = availablePort - 1; - if (value == previousRange.getRight() + 1 && currentRange.getLeft() - 1 == value) { - previousRange.setRight(currentRange.getRight()); - occupiedPortsIterator.remove(); - return; - } - if (value == previousRange.getRight() + 1) { - previousRange.setRight(value); - return; - } - if (value == currentRange.getLeft() - 1) { - currentRange.setLeft(value); - return; - } - preOccupiedPortsIterator.add(new Pair<>(value, value)); - } - } - - // ===========================add and release================================ - - public boolean addPortIfAvailable(final int candidatePort) { - synchronized (occupiedPorts) { - if (occupiedPorts.size() == 1 || occupiedPorts.isEmpty()) { - return false; - } - - ListIterator> occupiedPortsIterator = occupiedPorts.listIterator(); - ListIterator> previousIterator = occupiedPorts.listIterator(); - Pair previousRange = null; - Pair currentRange = occupiedPortsIterator.next(); - while (occupiedPortsIterator.hasNext()) { - previousRange = previousIterator.next(); - currentRange = occupiedPortsIterator.next(); - if (currentRange.getRight() <= candidatePort) { - continue; - } - if (previousRange.getLeft() >= candidatePort) { - break; - } - if (previousRange.getRight() >= candidatePort || currentRange.getLeft() <= candidatePort) { - return false; - } - if (candidatePort == previousRange.getRight() + 1 - && currentRange.getLeft() - 1 == candidatePort) { - previousRange.setRight(currentRange.getRight()); - occupiedPortsIterator.remove(); - return true; - } - - if (candidatePort == previousRange.getRight() + 1) { - previousRange.setRight(candidatePort); - return true; - } - - if (candidatePort == currentRange.getLeft() - 1) { - currentRange.setLeft(candidatePort); - return true; - } - - previousIterator.add(new Pair<>(candidatePort, candidatePort)); - return true; - } - } - return false; - } - - public void releaseUsedPort(final int port) { - synchronized (occupiedPorts) { - if (occupiedPorts.isEmpty()) { - return; - } - ListIterator> iterator = occupiedPorts.listIterator(); - Pair cur = null; - while (iterator.hasNext()) { - cur = iterator.next(); - if (port > cur.getRight()) { - continue; - } - if (port < cur.getLeft()) { - break; - } - if (cur.getLeft().equals(cur.getRight())) { - iterator.remove(); - break; - } - if (cur.getLeft() == port) { - cur.setLeft(port + 1); - break; - } - if (cur.getRight() == port) { - cur.setRight(port - 1); - break; - } - iterator.add(new Pair<>(port + 1, cur.getRight())); - cur.setRight(port - 1); - break; - } - } - } - - // ===========================bing================================ - - public void bingPort( - final int minSendPortRange, - final int maxSendPortRange, - final List candidatePorts, - final Consumer consumer) { - synchronized (occupiedPorts) { - AvailablePortIterator portIterator = - createAvailablePortIterator(minSendPortRange, maxSendPortRange, candidatePorts); - boolean portFound = false; - Exception lastException = null; - while (portIterator.hasNext()) { - try { - consumer.accept(portIterator.next()); - portIterator.updateOccupiedPort(); - portFound = true; - break; - } catch (Exception e) { - lastException = e; - } - } - if (!portFound) { - String exceptionMessage = - String.format( - "Failed to find an available send port. Custom send port is defined. " - + "No ports are available in the candidate list [%s] or within the range %d to %d.", - candidatePorts, minSendPortRange, maxSendPortRange); - LOGGER.warn(exceptionMessage, lastException); - throw new PipeConnectionException(exceptionMessage); - } - } - } - - @TestOnly - public void resetPortManager() { - synchronized (occupiedPorts) { - occupiedPorts.clear(); - occupiedPorts.add(new Pair<>(-1, -1)); - occupiedPorts.add(new Pair<>(65536, 65536)); - } - } - - public AvailablePortIterator createAvailablePortIterator( - final int minSendPortRange, final int maxSendPortRange, final List candidatePorts) { - return new AvailablePortIterator( - occupiedPorts, createPortRanges(candidatePorts, minSendPortRange, maxSendPortRange)); - } - - private List> createPortRanges( - final List candidatePorts, final int minSendPortRange, final int maxSendPortRange) { - List> range = new LinkedList<>(); - if (candidatePorts.isEmpty()) { - range.add(new Pair<>(minSendPortRange, maxSendPortRange)); - return range; - } - Iterator candidatePortIterator = candidatePorts.iterator(); - int tempValue = -1; - while (candidatePortIterator.hasNext()) { - int value = candidatePortIterator.next(); - if (value >= minSendPortRange) { - tempValue = value; - break; - } - addPair(range, value); - } - addPair(range, minSendPortRange, maxSendPortRange); - if (tempValue != -1) { - if (tempValue > maxSendPortRange) { - addPair(range, tempValue); - } - } - - while (candidatePortIterator.hasNext()) { - int value = candidatePortIterator.next(); - if (value <= maxSendPortRange) { - continue; - } - addPair(range, value); - } - return range; - } - - private void addPair(List> range, int value) { - if (!range.isEmpty() && range.get(range.size() - 1).getRight() == value - 1) { - range.get(range.size() - 1).setRight(value); - } else { - range.add(new Pair<>(value, value)); - } - } - - private void addPair(List> range, int minValue, int maxValue) { - if (!range.isEmpty() && range.get(range.size() - 1).getRight() == minValue - 1) { - range.get(range.size() - 1).setRight(maxValue); - } else { - range.add(new Pair<>(minValue, maxValue)); - } - } -} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/constant/PipeConnectorConstant.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/constant/PipeConnectorConstant.java index 8f79a222a092..7a23d8a85266 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/constant/PipeConnectorConstant.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/constant/PipeConnectorConstant.java @@ -51,12 +51,11 @@ public class PipeConnectorConstant { public static final String CONNECTOR_IOTDB_SEND_PORT_MAX_KEY = "connector.send-port.range.max"; public static final String SINK_IOTDB_SEND_PORT_MAX_KEY = "sink.send-port.range.max"; public static final String CONNECTOR_IOTDB_SEND_PORT_CANDIDATE_KEY = - "connector.send-port.range.candidate"; - public static final String SINK_IOTDB_SEND_PORT_CANDIDATE_KEY = "sink.send-port.range.candidate"; - public static final String CONNECTOR_IOTDB_SEND_PORT_ENABLE_KEY = - "connector.send-port.range.enable"; + "connector.send-port.candidate"; + public static final String SINK_IOTDB_SEND_PORT_CANDIDATE_KEY = "sink.send-port.candidate"; + public static final String CONNECTOR_IOTDB_SEND_PORT_ENABLE_KEY = "connector.send-port.enable"; public static final String SINK_IOTDB_SEND_PORT_ENABLE_KEY = "sink.send-port.enable"; - public static final int CONNECTOR_IOTDB_SEND_PORT_MIN_VALUE = 0; + public static final int CONNECTOR_IOTDB_SEND_PORT_MIN_VALUE = 1024; public static final int CONNECTOR_IOTDB_SEND_PORT_MAX_VALUE = 65535; public static final String CONNECTOR_IOTDB_SEND_PORT_CANDIDATE_VALUE = ""; public static final boolean CONNECTOR_IOTDB_SEND_PORT_ENABLE_VALUE = false; diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/client/IoTDBSyncClient.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/client/IoTDBSyncClient.java index 9a3be7af5d5c..db8465a3fd56 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/client/IoTDBSyncClient.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/client/IoTDBSyncClient.java @@ -22,7 +22,7 @@ import org.apache.iotdb.common.rpc.thrift.TEndPoint; import org.apache.iotdb.commons.client.ThriftClient; import org.apache.iotdb.commons.client.property.ThriftClientProperty; -import org.apache.iotdb.commons.client.util.IoTDBConnectorPortManager; +import org.apache.iotdb.commons.client.util.IoTDBConnectorPortBinder; import org.apache.iotdb.commons.pipe.config.PipeConfig; import org.apache.iotdb.commons.pipe.connector.payload.thrift.request.IoTDBConnectorRequestVersion; import org.apache.iotdb.commons.pipe.connector.payload.thrift.request.PipeTransferSliceReq; @@ -90,16 +90,20 @@ public IoTDBSyncClient( final TTransport transport = getInputProtocol().getTransport(); if (!transport.isOpen()) { if (isCustomSendPortDefined) { - IoTDBConnectorPortManager.INSTANCE.bingPort( - minSendPortRange, - maxSendPortRange, - candidatePorts, - (sendPort) -> { - final InetSocketAddress isa = new InetSocketAddress(sendPort); - ((TSocket) ((TimeoutChangeableTFastFramedTransport) transport).getSocket()) - .getSocket() - .bind(isa); - }); + try { + IoTDBConnectorPortBinder.bindPort( + minSendPortRange, + maxSendPortRange, + candidatePorts, + (sendPort) -> { + final InetSocketAddress isa = new InetSocketAddress(sendPort); + ((TSocket) ((TimeoutChangeableTFastFramedTransport) transport).getSocket()) + .getSocket() + .bind(isa); + }); + } catch (Exception e) { + LOGGER.warn("Failed to bind port:", e); + } } transport.open(); } @@ -189,15 +193,6 @@ public void close() throws Exception { @Override public void invalidate() { if (getInputProtocol().getTransport().isOpen()) { - if (isCustomSendPortDefined) { - IoTDBConnectorPortManager portUtilizationManager = IoTDBConnectorPortManager.INSTANCE; - portUtilizationManager.releaseUsedPort( - ((TSocket) - ((TimeoutChangeableTFastFramedTransport) getInputProtocol().getTransport()) - .getSocket()) - .getSocket() - .getPort()); - } getInputProtocol().getTransport().close(); } } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/protocol/IoTDBAirGapConnector.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/protocol/IoTDBAirGapConnector.java index c053e971179f..abf7dcd0f737 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/protocol/IoTDBAirGapConnector.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/protocol/IoTDBAirGapConnector.java @@ -21,7 +21,7 @@ import org.apache.iotdb.common.rpc.thrift.TEndPoint; import org.apache.iotdb.common.rpc.thrift.TSStatus; -import org.apache.iotdb.commons.client.util.IoTDBConnectorPortManager; +import org.apache.iotdb.commons.client.util.IoTDBConnectorPortBinder; import org.apache.iotdb.commons.pipe.config.PipeConfig; import org.apache.iotdb.commons.pipe.connector.payload.airgap.AirGapELanguageConstant; import org.apache.iotdb.commons.pipe.connector.payload.airgap.AirGapOneByteResponse; @@ -160,9 +160,6 @@ public void handshake() throws Exception { if (sockets.get(i) != null) { try { final Socket socket = sockets.set(i, null); - if (isCustomSendPortDefined) { - IoTDBConnectorPortManager.INSTANCE.releaseUsedPort(socket.getPort()); - } socket.close(); } catch (final Exception e) { LOGGER.warn( @@ -176,7 +173,7 @@ public void handshake() throws Exception { final AirGapSocket socket = new AirGapSocket(ip, port); if (isCustomSendPortDefined) { - IoTDBConnectorPortManager.INSTANCE.bingPort( + IoTDBConnectorPortBinder.bindPort( minSendPortRange, maxSendPortRange, candidatePorts, @@ -350,11 +347,7 @@ public void close() { for (int i = 0; i < sockets.size(); ++i) { try { if (sockets.get(i) != null) { - final Socket socket = sockets.set(i, null); - if (isCustomSendPortDefined) { - IoTDBConnectorPortManager.INSTANCE.releaseUsedPort(socket.getPort()); - } - socket.close(); + sockets.set(i, null).close(); } } catch (final Exception e) { LOGGER.warn("Failed to close client {}.", i, e); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/protocol/IoTDBConnector.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/protocol/IoTDBConnector.java index be2d37fa1030..b87c422fdf6e 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/protocol/IoTDBConnector.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/protocol/IoTDBConnector.java @@ -532,7 +532,7 @@ private static List parseCandidatePorts(String candidate) { private int calculateUsablePorts(int sendPortMin, int sendPortMax, List candidatePorts) { int usablePortCount = sendPortMax - sendPortMin + 1; for (int port : candidatePorts) { - if (sendPortMax < port && port < sendPortMin) { + if (sendPortMax < port || port < sendPortMin) { usablePortCount++; } } diff --git a/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/client/IoTDBConnectorPortManagerTest.java b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/client/IoTDBConnectorPortManagerTest.java deleted file mode 100644 index 004343655c96..000000000000 --- a/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/client/IoTDBConnectorPortManagerTest.java +++ /dev/null @@ -1,151 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iotdb.commons.client; - -import org.apache.iotdb.commons.client.util.IoTDBConnectorPortManager; - -import org.apache.tsfile.utils.Pair; -import org.junit.Assert; -import org.junit.Test; - -import java.util.ArrayList; -import java.util.HashSet; -import java.util.LinkedList; -import java.util.List; -import java.util.Random; -import java.util.Set; - -public class IoTDBConnectorPortManagerTest { - - private final IoTDBConnectorPortManager portManager = IoTDBConnectorPortManager.INSTANCE; - - @Test - public void testAddIfPortAvailable() { - portManager.resetPortManager(); - Set set = new HashSet<>(); - addPort(set); - Assert.assertEquals(generateExpectedRanges(set), portManager.getOccupiedPorts()); - } - - @Test - public void releasePortIfUsed() { - portManager.resetPortManager(); - Set set = new HashSet<>(); - addPort(set); - Random random = new Random(); - while (set.size() > 200) { - int port = 1024 + random.nextInt(65535 - 1024); - set.remove(port); - portManager.releaseUsedPort(port); - } - Assert.assertEquals(generateExpectedRanges(set), portManager.getOccupiedPorts()); - } - - @Test - public void testAvailablePortIterator() { - portManager.resetPortManager(); - Set set = new HashSet<>(); - addPort(set); - testPortRange(1024, 65535, new HashSet<>(), set); - Random r = new Random(); - Set cp = new HashSet<>(); - for (int i = 0; i < 1; i++) { - IoTDBConnectorPortManager.AvailablePortIterator iterator = - portManager.createAvailablePortIterator(1024, 65535, new ArrayList<>()); - while (iterator.hasNext()) { - int port = iterator.next(); - if (r.nextBoolean()) { - iterator.updateOccupiedPort(); - set.add(port); - break; - } - } - testPortRange(1024, 6553, cp, set); - int randomPort = 1024 + r.nextInt(65535 - 1024 - 200); - cp.clear(); - for (int j = 0; j < 40; j++) { - cp.add(1024 + r.nextInt(65535 - 1024)); - } - testPortRange(randomPort, randomPort + 200, cp, set); - } - } - - private void testPortRange(int port, int maxPort, Set cp, Set set) { - int start = port; - int end = maxPort; - List candidatePorts = new ArrayList<>(cp); - if (!candidatePorts.isEmpty()) { - candidatePorts.sort(Integer::compare); - start = Math.min(port, candidatePorts.get(0)); - end = Math.max(maxPort, candidatePorts.get(candidatePorts.size() - 1)); - } - IoTDBConnectorPortManager.AvailablePortIterator iterator = - portManager.createAvailablePortIterator(port, maxPort, candidatePorts); - for (int i = start; i <= end; i++) { - if (set.contains(i) || ((i < port || i > maxPort) && !cp.contains(i))) { - continue; - } - Assert.assertTrue(iterator.hasNext()); - Assert.assertEquals(i, iterator.next().intValue()); - } - Assert.assertFalse(iterator.hasNext()); - } - - private void addPort(final Set set) { - set.add(-1); - set.add(65536); - final Random random = new Random(); - while (set.size() < 300) { - final int port = 1024 + random.nextInt(65535 - 1024); - set.add(port); - portManager.addPortIfAvailable(port); - } - for (int i = 0; i <= 100; i++) { - int port = i + 1024; - set.add(port); - portManager.addPortIfAvailable(port); - port = 65535 - i; - if (!set.contains(port)) { - set.add(port); - set.add(port); - } - portManager.addPortIfAvailable(port); - } - } - - private List> generateExpectedRanges(final Set set) { - List ports = new ArrayList<>(set); - ports.sort(Integer::compare); - int start = ports.get(0); - int end = start; - List> data = new LinkedList<>(); - for (int i = 1; i < ports.size(); i++) { - if (ports.get(i) == end + 1) { - end = ports.get(i); - } else { - data.add(new Pair<>(start, end)); - start = ports.get(i); - end = start; - } - } - data.add(new Pair<>(start, end)); - return data; - } -} From a7b3666f2b60b1e444382ffd3db8dde8356249e4 Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Thu, 26 Sep 2024 20:26:03 +0800 Subject: [PATCH 06/12] Modify function definition --- .../IoTDBPipeConnectorCustomPortIT.java | 256 +++++++++--------- .../tool/tsfile/ImportTsFileRemotely.java | 6 +- .../IoTDBConfigNodeSyncClientManager.java | 4 +- .../protocol/IoTDBConfigRegionConnector.java | 4 +- .../IoTDBDataNodeAsyncClientManager.java | 6 +- .../IoTDBDataNodeSyncClientManager.java | 4 +- .../legacy/IoTDBLegacyPipeConnector.java | 6 +- .../async/IoTDBDataRegionAsyncConnector.java | 2 +- .../sync/IoTDBDataNodeSyncConnector.java | 8 +- .../sender/TwoStageAggregateSender.java | 4 - .../commons/client/ClientPoolFactory.java | 8 +- .../AsyncPipeDataTransferServiceClient.java | 33 ++- .../client/util/IoTDBConnectorPortBinder.java | 29 +- .../constant/PipeConnectorConstant.java | 17 +- .../connector/client/IoTDBClientManager.java | 6 +- .../connector/client/IoTDBSyncClient.java | 61 +++-- .../client/IoTDBSyncClientManager.java | 6 +- .../protocol/IoTDBAirGapConnector.java | 13 +- .../connector/protocol/IoTDBConnector.java | 149 +++++----- .../protocol/IoTDBSslSyncConnector.java | 4 +- 20 files changed, 336 insertions(+), 290 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeConnectorCustomPortIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeConnectorCustomPortIT.java index 3809d27408fd..b93c700534eb 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeConnectorCustomPortIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeConnectorCustomPortIT.java @@ -76,153 +76,149 @@ public void setUp() { } @Test - public void testPortsAvailable() { - try (final SyncConfigNodeIServiceClient client = - (SyncConfigNodeIServiceClient) senderEnv.getLeaderConfigNodeConnection()) { - if (!TestUtils.tryExecuteNonQueriesWithRetry( - senderEnv, - Arrays.asList( - "insert into root.db.d1(time, s1) values (2010-01-01T10:00:00+08:00, 1)", - "insert into root.db.d1(time, s1) values (2010-01-02T10:00:00+08:00, 2)", - "flush"))) { - return; - } - final Map extractorAttributes = new HashMap<>(); - final Map processorAttributes = new HashMap<>(); - final Map connectorAttributes = new HashMap<>(); - extractorAttributes.put("realtime.mode", "forced-log"); - connectorAttributes.put("connector", "iotdb-thrift-async-connector"); - connectorAttributes.put("batch.enable", "false"); - connectorAttributes.put("node-urls", receiverEnv.getIP() + ":" + receiverEnv.getPort()); - connectorAttributes.put("send-port.enable", "true"); - connectorAttributes.put("send-port.range.min", "1024"); - connectorAttributes.put("send-port.range.max", "1035"); - connectorAttributes.put("parallel.tasks", "16"); - - final TSStatus status = - client.createPipe( - new TCreatePipeReq("p1", connectorAttributes) - .setExtractorAttributes(extractorAttributes) - .setProcessorAttributes(processorAttributes)); - - Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode()); - Thread.sleep(5000); - TestUtils.tryExecuteNonQueriesWithRetry( - senderEnv, - Arrays.asList( - "insert into root.db.d1(time, s1) values (2010-01-01T10:00:00+08:00, 1)", - "insert into root.db.d1(time, s1) values (2010-01-02T10:00:00+08:00, 2)", - "flush")); - TestUtils.assertDataEventuallyOnEnv( - receiverEnv, "select count(*) from root.**", "Time,", new HashSet<>()); - } catch (Exception e) { - Assert.fail(); - } + public void testRangePortsAvailable() { + doTest( + "iotdb-thrift-async-connector", + receiverEnv.getIP() + ":" + receiverEnv.getPort(), + "range", + "1024", + "1025", + 0, + "16", + 10000, + 0); } @Test - public void testAsyncConnector() { - try (final SyncConfigNodeIServiceClient client = - (SyncConfigNodeIServiceClient) senderEnv.getLeaderConfigNodeConnection()) { - final Map extractorAttributes = new HashMap<>(); - final Map processorAttributes = new HashMap<>(); - final Map connectorAttributes = new HashMap<>(); - extractorAttributes.put("realtime.mode", "forced-log"); - connectorAttributes.put("connector", "iotdb-thrift-async-connector"); - connectorAttributes.put("batch.enable", "false"); - connectorAttributes.put("node-urls", receiverEnv.getIP() + ":" + receiverEnv.getPort()); - connectorAttributes.put("send-port.enable", "true"); - connectorAttributes.put("send-port.range.min", "1024"); - connectorAttributes.put("send-port.range.max", "1124"); - connectorAttributes.put("parallel.tasks", "3"); - - final TSStatus status = - client.createPipe( - new TCreatePipeReq("p1", connectorAttributes) - .setExtractorAttributes(extractorAttributes) - .setProcessorAttributes(processorAttributes)); - - Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode()); - - Thread.sleep(10000); - TestUtils.tryExecuteNonQueriesWithRetry( - senderEnv, - Arrays.asList( - "insert into root.db.d1(time, s1) values (2010-01-01T10:00:00+08:00, 1)", - "insert into root.db.d1(time, s1) values (2010-01-02T10:00:00+08:00, 2)", - "flush")); - TestUtils.assertDataEventuallyOnEnv( - receiverEnv, - "select count(*) from root.**", - "count(root.db.d1.s1),", - Collections.singleton("2,")); - - } catch (Exception e) { - Assert.fail(); - } + public void testCandidatePortsAvailable() { + doTest( + "iotdb-thrift-async-connector", + receiverEnv.getIP() + ":" + receiverEnv.getPort(), + "candidate", + null, + null, + 4, + "16", + 10000, + 0); } @Test - public void testSyncThriftConnector() { - try (final SyncConfigNodeIServiceClient client = - (SyncConfigNodeIServiceClient) senderEnv.getLeaderConfigNodeConnection()) { - final Map extractorAttributes = new HashMap<>(); - final Map processorAttributes = new HashMap<>(); - final Map connectorAttributes = new HashMap<>(); - extractorAttributes.put("realtime.mode", "forced-log"); - connectorAttributes.put("connector", "iotdb-thrift-sync-connector"); - connectorAttributes.put("batch.enable", "false"); - connectorAttributes.put("node-urls", receiverEnv.getIP() + ":" + receiverEnv.getPort()); - connectorAttributes.put("send-port.enable", "true"); - connectorAttributes.put("send-port.range.min", "1024"); - connectorAttributes.put("send-port.range.max", "1124"); - connectorAttributes.put("parallel.tasks", "3"); + public void testAsyncPortRange() { + doTest( + "iotdb-thrift-async-connector", + receiverEnv.getIP() + ":" + receiverEnv.getPort(), + "range", + "1024", + "1224", + 0, + "3", + 2000, + 2); + } - final TSStatus status = - client.createPipe( - new TCreatePipeReq("p1", connectorAttributes) - .setExtractorAttributes(extractorAttributes) - .setProcessorAttributes(processorAttributes)); + @Test + public void testAsyncPortCandidate() { + doTest( + "iotdb-thrift-async-connector", + receiverEnv.getIP() + ":" + receiverEnv.getPort(), + "candidate", + "1024", + "1224", + 30, + "1", + 2000, + 2); + } - Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode()); + @Test + public void testSyncThriftPortRange() { + doTest( + "iotdb-thrift-sync-connector", + receiverEnv.getIP() + ":" + receiverEnv.getPort(), + "range", + "1024", + "1224", + 30, + "1", + 2000, + 2); + } - Thread.sleep(2000); - TestUtils.tryExecuteNonQueriesWithRetry( - senderEnv, - Arrays.asList( - "insert into root.db.d1(time, s1) values (2010-01-01T10:00:00+08:00, 1)", - "insert into root.db.d1(time, s1) values (2010-01-02T10:00:00+08:00, 2)", - "flush")); - TestUtils.assertDataEventuallyOnEnv( - receiverEnv, - "select count(*) from root.**", - "count(root.db.d1.s1),", - Collections.singleton("2,")); + @Test + public void testSyncThriftPortCandidate() { + doTest( + "iotdb-thrift-sync-connector", + receiverEnv.getIP() + ":" + receiverEnv.getPort(), + "candidate", + "1024", + "1224", + 30, + "1", + 2000, + 2); + } - } catch (Exception e) { - Assert.fail(); - } + @Test + public void testAirGapPortRange() { + doTest( + "iotdb-air-gap-connector", + receiverEnv.getIP() + ":" + receiverEnv.getDataNodeWrapper(0).getPipeAirGapReceiverPort(), + "range", + "1024", + "1224", + 30, + "1", + 2000, + 2); } @Test - public void TestAirGap() { + public void testAirGapPortCandidate() { + doTest( + "iotdb-air-gap-connector", + receiverEnv.getIP() + ":" + receiverEnv.getDataNodeWrapper(0).getPipeAirGapReceiverPort(), + "candidate", + "1024", + "1224", + 30, + "1", + 2000, + 2); + } + + private void doTest( + final String connector, + final String urls, + final String strategy, + final String min, + final String max, + final int candidateNum, + final String tasks, + final long sleepTime, + final int resultNum) { try (final SyncConfigNodeIServiceClient client = (SyncConfigNodeIServiceClient) senderEnv.getLeaderConfigNodeConnection()) { final Map extractorAttributes = new HashMap<>(); final Map processorAttributes = new HashMap<>(); final Map connectorAttributes = new HashMap<>(); extractorAttributes.put("realtime.mode", "forced-log"); - connectorAttributes.put("connector", "iotdb-air-gap-connector"); + connectorAttributes.put("connector", connector); connectorAttributes.put("batch.enable", "false"); - connectorAttributes.put( - "node-urls", - receiverEnv.getIP() - + ":" - + receiverEnv.getDataNodeWrapper(0).getPipeAirGapReceiverPort()); - connectorAttributes.put("send-port.enable", "true"); - connectorAttributes.put("send-port.range.min", "1024"); - connectorAttributes.put("send-port.range.max", "1124"); - connectorAttributes.put("parallel.tasks", "3"); + connectorAttributes.put("node-urls", urls); + connectorAttributes.put("send-port.restriction-strategy", strategy); + if (strategy.equals("range")) { + connectorAttributes.put("send-port.range.min", min); + connectorAttributes.put("send-port.range.max", max); + } else { + StringBuilder candidateBuilder = new StringBuilder(); + for (int i = 0; i < candidateNum; i++) { + candidateBuilder.append(1024 + i).append(","); + } + candidateBuilder.append(1024 + candidateNum); + connectorAttributes.put("send-port.candidate", candidateBuilder.toString()); + } + connectorAttributes.put("parallel.tasks", tasks); final TSStatus status = client.createPipe( @@ -232,7 +228,7 @@ public void TestAirGap() { Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode()); - Thread.sleep(2000); + Thread.sleep(sleepTime); TestUtils.tryExecuteNonQueriesWithRetry( senderEnv, Arrays.asList( @@ -242,8 +238,8 @@ public void TestAirGap() { TestUtils.assertDataEventuallyOnEnv( receiverEnv, "select count(*) from root.**", - "count(root.db.d1.s1),", - Collections.singleton("2,")); + resultNum == 0 ? "Time," : "count(root.db.d1.s1),", + resultNum == 0 ? new HashSet<>() : Collections.singleton(resultNum + ",")); } catch (Exception e) { Assert.fail(); diff --git a/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/tsfile/ImportTsFileRemotely.java b/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/tsfile/ImportTsFileRemotely.java index 01c7f29c6e1a..8f3c22f9e0b9 100644 --- a/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/tsfile/ImportTsFileRemotely.java +++ b/iotdb-client/cli/src/main/java/org/apache/iotdb/tool/tsfile/ImportTsFileRemotely.java @@ -301,11 +301,7 @@ private void initClient() { getEndPoint().getPort(), false, "", - "", - false, - 0, - 0, - null); + ""); } catch (final TTransportException e) { throw new PipeException("Sync client init error because " + e.getMessage()); } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/connector/client/IoTDBConfigNodeSyncClientManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/connector/client/IoTDBConfigNodeSyncClientManager.java index a305e2912a23..ba5c0c911b0b 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/connector/client/IoTDBConfigNodeSyncClientManager.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/connector/client/IoTDBConfigNodeSyncClientManager.java @@ -41,7 +41,7 @@ public IoTDBConfigNodeSyncClientManager( String loadBalanceStrategy, boolean shouldReceiverConvertOnTypeMismatch, String loadTsFileStrategy, - boolean isCustomSendPortDefined, + String customSendPortStrategy, int minSendPortRange, int maxSendPortRange, List candidatePorts) { @@ -54,7 +54,7 @@ public IoTDBConfigNodeSyncClientManager( loadBalanceStrategy, shouldReceiverConvertOnTypeMismatch, loadTsFileStrategy, - isCustomSendPortDefined, + customSendPortStrategy, minSendPortRange, maxSendPortRange, candidatePorts); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/connector/protocol/IoTDBConfigRegionConnector.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/connector/protocol/IoTDBConfigRegionConnector.java index a49b26640411..2e4c3dd3c737 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/connector/protocol/IoTDBConfigRegionConnector.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/connector/protocol/IoTDBConfigRegionConnector.java @@ -66,7 +66,7 @@ protected IoTDBSyncClientManager constructClient( final String loadBalanceStrategy, final boolean shouldReceiverConvertOnTypeMismatch, final String loadTsFileStrategy, - boolean isCustomSendPortDefined, + String customSendPortStrategy, int minSendPortRange, int maxSendPortRange, List candidatePorts) { @@ -78,7 +78,7 @@ protected IoTDBSyncClientManager constructClient( loadBalanceStrategy, shouldReceiverConvertOnTypeMismatch, loadTsFileStrategy, - isCustomSendPortDefined, + customSendPortStrategy, minSendPortRange, maxSendPortRange, candidatePorts); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/client/IoTDBDataNodeAsyncClientManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/client/IoTDBDataNodeAsyncClientManager.java index e74bc51e621e..81bc6485d90c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/client/IoTDBDataNodeAsyncClientManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/client/IoTDBDataNodeAsyncClientManager.java @@ -82,14 +82,14 @@ public IoTDBDataNodeAsyncClientManager( String loadBalanceStrategy, boolean shouldReceiverConvertOnTypeMismatch, String loadTsFileStrategy, - boolean isCustomSendPortDefined, + String customSendPortStrategy, int minSendPortRange, int maxSendPortRange, List candidatePorts) { super( endPoints, useLeaderCache, - isCustomSendPortDefined, + customSendPortStrategy, minSendPortRange, maxSendPortRange, candidatePorts); @@ -105,7 +105,7 @@ public IoTDBDataNodeAsyncClientManager( new IClientManager.Factory() .createClientManager( new ClientPoolFactory.AsyncPipeDataTransferServiceClientPoolFactory( - isCustomSendPortDefined, + this.customSendPortStrategy, minSendPortRange, maxSendPortRange, candidatePorts))); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/client/IoTDBDataNodeSyncClientManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/client/IoTDBDataNodeSyncClientManager.java index d4465c85a8b2..e46ae18715ce 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/client/IoTDBDataNodeSyncClientManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/client/IoTDBDataNodeSyncClientManager.java @@ -51,7 +51,7 @@ public IoTDBDataNodeSyncClientManager( final String loadBalanceStrategy, final boolean shouldReceiverConvertOnTypeMismatch, final String loadTsFileStrategy, - final boolean isCustomSendPortDefined, + final String customSendPortStrategy, final int minSendPortRange, final int maxSendPortRange, final List candidatePorts) { @@ -65,7 +65,7 @@ public IoTDBDataNodeSyncClientManager( loadBalanceStrategy, shouldReceiverConvertOnTypeMismatch, loadTsFileStrategy, - isCustomSendPortDefined, + customSendPortStrategy, minSendPortRange, maxSendPortRange, candidatePorts); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/legacy/IoTDBLegacyPipeConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/legacy/IoTDBLegacyPipeConnector.java index 780ad4d13229..b647f149b727 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/legacy/IoTDBLegacyPipeConnector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/legacy/IoTDBLegacyPipeConnector.java @@ -236,11 +236,7 @@ public void handshake() throws Exception { port, useSSL, trustStore, - trustStorePwd, - false, - 0, - 0, - null); + trustStorePwd); final TSyncIdentityInfo identityInfo = new TSyncIdentityInfo( pipeName, System.currentTimeMillis(), syncConnectorVersion, databaseName); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/IoTDBDataRegionAsyncConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/IoTDBDataRegionAsyncConnector.java index 91ec9e2b9a22..9d4f9270d6b8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/IoTDBDataRegionAsyncConnector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/IoTDBDataRegionAsyncConnector.java @@ -127,7 +127,7 @@ public void customize( loadBalanceStrategy, shouldReceiverConvertOnTypeMismatch, loadTsFileStrategy, - isCustomSendPortDefined, + customSendPortStrategy, minSendPortRange, maxSendPortRange, candidatePorts); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBDataNodeSyncConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBDataNodeSyncConnector.java index ff74f608d4ed..5434a2100c19 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBDataNodeSyncConnector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBDataNodeSyncConnector.java @@ -90,9 +90,9 @@ protected IoTDBSyncClientManager constructClient( final String loadBalanceStrategy, final boolean shouldReceiverConvertOnTypeMismatch, final String loadTsFileStrategy, - boolean isCustomSendPortDefined, - int minSendPortRange, - int maxSendPortRange, + final String customSendPortStrategy, + final int minSendPortRange, + final int maxSendPortRange, List candidatePorts) { clientManager = new IoTDBDataNodeSyncClientManager( @@ -104,7 +104,7 @@ protected IoTDBSyncClientManager constructClient( loadBalanceStrategy, shouldReceiverConvertOnTypeMismatch, loadTsFileStrategy, - isCustomSendPortDefined, + customSendPortStrategy, minSendPortRange, maxSendPortRange, candidatePorts); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/twostage/exchange/sender/TwoStageAggregateSender.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/twostage/exchange/sender/TwoStageAggregateSender.java index d321eaf6d32a..a24b6c4fc391 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/twostage/exchange/sender/TwoStageAggregateSender.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/twostage/exchange/sender/TwoStageAggregateSender.java @@ -216,10 +216,6 @@ private IoTDBSyncClient constructIoTDBSyncClient(TEndPoint endPoint) throws TTra endPoint.getPort(), false, null, - null, - false, - 0, - 0, null); } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/ClientPoolFactory.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/ClientPoolFactory.java index 844c9e85a631..9f28baa0d208 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/ClientPoolFactory.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/ClientPoolFactory.java @@ -279,17 +279,17 @@ public static class AsyncPipeDataTransferServiceClientPoolFactory private List candidatePorts; - private boolean isCustomSendPortDefined; + private String customSendPortStrategy; public AsyncPipeDataTransferServiceClientPoolFactory( - boolean isCustomSendPortDefined, + String customSendPortStrategy, int minSendPortRange, int maxSendPortRange, List candidatePorts) { this.minSendPortRange = minSendPortRange; this.maxSendPortRange = maxSendPortRange; this.candidatePorts = candidatePorts; - this.isCustomSendPortDefined = isCustomSendPortDefined; + this.customSendPortStrategy = customSendPortStrategy; } @Override @@ -307,7 +307,7 @@ public KeyedObjectPool createClie conf.getPipeAsyncConnectorSelectorNumber()) .build(), ThreadName.PIPE_ASYNC_CONNECTOR_CLIENT_POOL.getName(), - isCustomSendPortDefined, + customSendPortStrategy, minSendPortRange, maxSendPortRange, candidatePorts), diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/async/AsyncPipeDataTransferServiceClient.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/async/AsyncPipeDataTransferServiceClient.java index 4b3220a5c03f..e671cad6befc 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/async/AsyncPipeDataTransferServiceClient.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/async/AsyncPipeDataTransferServiceClient.java @@ -66,7 +66,7 @@ public AsyncPipeDataTransferServiceClient( TEndPoint endpoint, TAsyncClientManager tClientManager, ClientManager clientManager, - boolean isCustomSendPortDefined, + String customSendPortStrategy, int minSendPortRange, int maxSendPortRange, List candidatePorts) @@ -77,15 +77,14 @@ public AsyncPipeDataTransferServiceClient( TNonblockingSocketWrapper.wrap( endpoint.getIp(), endpoint.getPort(), property.getConnectionTimeoutMs())); SocketChannel socketChannel = ((TNonblockingSocket) ___transport).getSocketChannel(); - if (isCustomSendPortDefined) { - IoTDBConnectorPortBinder.bindPort( - minSendPortRange, - maxSendPortRange, - candidatePorts, - (sendPort) -> { - socketChannel.bind(new InetSocketAddress(sendPort)); - }); - } + IoTDBConnectorPortBinder.bindPort( + customSendPortStrategy, + minSendPortRange, + maxSendPortRange, + candidatePorts, + (sendPort) -> { + socketChannel.bind(new InetSocketAddress(sendPort)); + }); setTimeout(property.getConnectionTimeoutMs()); this.printLogWhenEncounterException = property.isPrintLogWhenEncounterException(); this.endpoint = endpoint; @@ -195,24 +194,24 @@ public String toString() { public static class Factory extends AsyncThriftClientFactory { - private int minSendPortRange; + private final int minSendPortRange; - private int maxSendPortRange; + private final int maxSendPortRange; - private List candidatePorts; + private final List candidatePorts; - private boolean isCustomSendPortDefined; + private final String customSendPortStrategy; public Factory( ClientManager clientManager, ThriftClientProperty thriftClientProperty, String threadName, - boolean isCustomSendPortDefined, + String customSendPortStrategy, int minSendPortRange, int maxSendPortRange, List candidatePorts) { super(clientManager, thriftClientProperty, threadName); - this.isCustomSendPortDefined = isCustomSendPortDefined; + this.customSendPortStrategy = customSendPortStrategy; this.minSendPortRange = minSendPortRange; this.maxSendPortRange = maxSendPortRange; this.candidatePorts = candidatePorts; @@ -233,7 +232,7 @@ public PooledObject makeObject(TEndPoint end endPoint, tManagers[clientCnt.incrementAndGet() % tManagers.length], clientManager, - isCustomSendPortDefined, + customSendPortStrategy, minSendPortRange, maxSendPortRange, candidatePorts)); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/util/IoTDBConnectorPortBinder.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/util/IoTDBConnectorPortBinder.java index 049e926baa87..6f12b84fb38b 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/util/IoTDBConnectorPortBinder.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/util/IoTDBConnectorPortBinder.java @@ -19,13 +19,13 @@ package org.apache.iotdb.commons.client.util; +import org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant; import org.apache.iotdb.commons.utils.function.Consumer; import org.apache.iotdb.pipe.api.exception.PipeConnectionException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.Iterator; import java.util.List; public class IoTDBConnectorPortBinder { @@ -35,28 +35,37 @@ public class IoTDBConnectorPortBinder { // ===========================bind================================ public static void bindPort( + final String customSendPortStrategy, final int minSendPortRange, final int maxSendPortRange, final List candidatePorts, final Consumer consumer) { + final boolean isRange = + PipeConnectorConstant.CONNECTOR_IOTDB_SEND_PORT_RESTRICTION_RANGE_STRATEGY.equals( + customSendPortStrategy); boolean portFound = false; - Iterator iterator = candidatePorts.iterator(); - int port = minSendPortRange; - while (iterator.hasNext() || port <= maxSendPortRange) { + int index = 0; + boolean isNotEnd = true; + while (isNotEnd) { + int port = isRange ? minSendPortRange + index : candidatePorts.get(index); try { - int bindPort = iterator.hasNext() ? iterator.next() : port++; - consumer.accept(bindPort); + consumer.accept(port); portFound = true; break; } catch (Exception ignored) { } + index++; + isNotEnd = isRange ? port <= maxSendPortRange : candidatePorts.size() > index; } if (!portFound) { String exceptionMessage = - String.format( - "Failed to find an available send port. Custom send port is defined. " - + "No ports are available in the candidate list [%s] or within the range %d to %d.", - candidatePorts, minSendPortRange, maxSendPortRange); + isRange + ? String.format( + "Failed to find an available send port within the range %d to %d.", + minSendPortRange, maxSendPortRange) + : String.format( + "Failed to find an available send port in the candidate list [%s].", + candidatePorts); LOGGER.warn(exceptionMessage); throw new PipeConnectionException(exceptionMessage); } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/constant/PipeConnectorConstant.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/constant/PipeConnectorConstant.java index 7a23d8a85266..6966027185ac 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/constant/PipeConnectorConstant.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/constant/PipeConnectorConstant.java @@ -53,12 +53,21 @@ public class PipeConnectorConstant { public static final String CONNECTOR_IOTDB_SEND_PORT_CANDIDATE_KEY = "connector.send-port.candidate"; public static final String SINK_IOTDB_SEND_PORT_CANDIDATE_KEY = "sink.send-port.candidate"; - public static final String CONNECTOR_IOTDB_SEND_PORT_ENABLE_KEY = "connector.send-port.enable"; - public static final String SINK_IOTDB_SEND_PORT_ENABLE_KEY = "sink.send-port.enable"; - public static final int CONNECTOR_IOTDB_SEND_PORT_MIN_VALUE = 1024; + public static final String CONNECTOR_IOTDB_SEND_PORT_RESTRICTION_STRATEGY_KEY = + "connector.send-port.restriction-strategy"; + public static final String SINK_IOTDB_SEND_PORT_RESTRICTION_STRATEGY_KEY = + "sink.send-port.restriction-strategy"; + public static final int CONNECTOR_IOTDB_SEND_PORT_MIN_VALUE = 0; public static final int CONNECTOR_IOTDB_SEND_PORT_MAX_VALUE = 65535; public static final String CONNECTOR_IOTDB_SEND_PORT_CANDIDATE_VALUE = ""; - public static final boolean CONNECTOR_IOTDB_SEND_PORT_ENABLE_VALUE = false; + public static final String CONNECTOR_IOTDB_SEND_PORT_RESTRICTION_RANGE_STRATEGY = "range"; + public static final String CONNECTOR_IOTDB_SEND_PORT_RESTRICTION_CANDIDATE_STRATEGY = "candidate"; + public static final Set CONNECTOR_IOTDB_SEND_PORT_RESTRICTION_STRATEGY_SET = + Collections.unmodifiableSet( + new HashSet<>( + Arrays.asList( + CONNECTOR_IOTDB_SEND_PORT_RESTRICTION_CANDIDATE_STRATEGY, + CONNECTOR_IOTDB_SEND_PORT_RESTRICTION_RANGE_STRATEGY))); public static final String SINK_IOTDB_SSL_ENABLE_KEY = "sink.ssl.enable"; public static final String SINK_IOTDB_SSL_TRUST_STORE_PATH_KEY = "sink.ssl.trust-store-path"; diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/client/IoTDBClientManager.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/client/IoTDBClientManager.java index 22f4b14edc6c..2cfd7964aa27 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/client/IoTDBClientManager.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/client/IoTDBClientManager.java @@ -49,7 +49,7 @@ public abstract class IoTDBClientManager { protected List candidatePorts; - protected boolean isCustomSendPortDefined; + protected String customSendPortStrategy; private static final int MAX_CONNECTION_TIMEOUT_MS = 24 * 60 * 60 * 1000; // 1 day private static final int FIRST_ADJUSTMENT_TIMEOUT_MS = 6 * 60 * 60 * 1000; // 6 hours @@ -59,13 +59,13 @@ public abstract class IoTDBClientManager { protected IoTDBClientManager( List endPointList, boolean useLeaderCache, - boolean isCustomSendPortDefined, + String customSendPortStrategy, int minSendPortRange, int maxSendPortRange, List candidatePorts) { this.endPointList = endPointList; this.useLeaderCache = useLeaderCache; - this.isCustomSendPortDefined = isCustomSendPortDefined; + this.customSendPortStrategy = customSendPortStrategy; this.minSendPortRange = minSendPortRange; this.maxSendPortRange = maxSendPortRange; this.candidatePorts = candidatePorts; diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/client/IoTDBSyncClient.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/client/IoTDBSyncClient.java index db8465a3fd56..c35fef804743 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/client/IoTDBSyncClient.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/client/IoTDBSyncClient.java @@ -56,7 +56,6 @@ public class IoTDBSyncClient extends IClientRPCService.Client private final String ipAddress; private final int port; private final TEndPoint endPoint; - private final boolean isCustomSendPortDefined; public IoTDBSyncClient( ThriftClientProperty property, @@ -65,7 +64,7 @@ public IoTDBSyncClient( boolean useSSL, String trustStore, String trustStorePwd, - boolean isCustomSendPortDefined, + String customSendPortStrategy, int minSendPortRange, int maxSendPortRange, List candidatePorts) @@ -86,25 +85,49 @@ public IoTDBSyncClient( this.ipAddress = ipAddress; this.port = port; this.endPoint = new TEndPoint(ipAddress, port); - this.isCustomSendPortDefined = isCustomSendPortDefined; final TTransport transport = getInputProtocol().getTransport(); if (!transport.isOpen()) { - if (isCustomSendPortDefined) { - try { - IoTDBConnectorPortBinder.bindPort( - minSendPortRange, - maxSendPortRange, - candidatePorts, - (sendPort) -> { - final InetSocketAddress isa = new InetSocketAddress(sendPort); - ((TSocket) ((TimeoutChangeableTFastFramedTransport) transport).getSocket()) - .getSocket() - .bind(isa); - }); - } catch (Exception e) { - LOGGER.warn("Failed to bind port:", e); - } - } + IoTDBConnectorPortBinder.bindPort( + customSendPortStrategy, + minSendPortRange, + maxSendPortRange, + candidatePorts, + (sendPort) -> { + final InetSocketAddress isa = new InetSocketAddress(sendPort); + ((TSocket) ((TimeoutChangeableTFastFramedTransport) transport).getSocket()) + .getSocket() + .bind(isa); + }); + transport.open(); + } + } + + public IoTDBSyncClient( + ThriftClientProperty property, + String ipAddress, + int port, + boolean useSSL, + String trustStore, + String trustStorePwd) + throws TTransportException { + super( + property + .getProtocolFactory() + .getProtocol( + useSSL + ? DeepCopyRpcTransportFactory.INSTANCE.getTransport( + ipAddress, + port, + property.getConnectionTimeoutMs(), + trustStore, + trustStorePwd) + : DeepCopyRpcTransportFactory.INSTANCE.getTransport( + ipAddress, port, property.getConnectionTimeoutMs()))); + this.ipAddress = ipAddress; + this.port = port; + this.endPoint = new TEndPoint(ipAddress, port); + final TTransport transport = getInputProtocol().getTransport(); + if (!transport.isOpen()) { transport.open(); } } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/client/IoTDBSyncClientManager.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/client/IoTDBSyncClientManager.java index 365802b0377e..9022e20a1951 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/client/IoTDBSyncClientManager.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/client/IoTDBSyncClientManager.java @@ -73,14 +73,14 @@ protected IoTDBSyncClientManager( String loadBalanceStrategy, boolean shouldReceiverConvertOnTypeMismatch, String loadTsFileStrategy, - boolean isCustomSendPortDefined, + String customSendPortStrategy, int minSendPortRange, int maxSendPortRange, List candidatePorts) { super( endPoints, useLeaderCache, - isCustomSendPortDefined, + customSendPortStrategy, minSendPortRange, maxSendPortRange, candidatePorts); @@ -170,7 +170,7 @@ private void initClientAndStatus( useSSL, trustStorePath, trustStorePwd, - isCustomSendPortDefined, + customSendPortStrategy, minSendPortRange, maxSendPortRange, candidatePorts)); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/protocol/IoTDBAirGapConnector.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/protocol/IoTDBAirGapConnector.java index abf7dcd0f737..3952b260259b 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/protocol/IoTDBAirGapConnector.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/protocol/IoTDBAirGapConnector.java @@ -171,14 +171,13 @@ public void handshake() throws Exception { } final AirGapSocket socket = new AirGapSocket(ip, port); + IoTDBConnectorPortBinder.bindPort( + customSendPortStrategy, + minSendPortRange, + maxSendPortRange, + candidatePorts, + (sendPort) -> socket.bind(new InetSocketAddress(sendPort))); - if (isCustomSendPortDefined) { - IoTDBConnectorPortBinder.bindPort( - minSendPortRange, - maxSendPortRange, - candidatePorts, - (sendPort) -> socket.bind(new InetSocketAddress(sendPort))); - } try { socket.connect(new InetSocketAddress(ip, port), handshakeTimeoutMs); socket.setKeepAlive(true); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/protocol/IoTDBConnector.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/protocol/IoTDBConnector.java index b87c422fdf6e..ed9d939faa43 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/protocol/IoTDBConnector.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/protocol/IoTDBConnector.java @@ -86,12 +86,13 @@ import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.CONNECTOR_IOTDB_PORT_KEY; import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.CONNECTOR_IOTDB_SEND_PORT_CANDIDATE_KEY; import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.CONNECTOR_IOTDB_SEND_PORT_CANDIDATE_VALUE; -import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.CONNECTOR_IOTDB_SEND_PORT_ENABLE_KEY; -import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.CONNECTOR_IOTDB_SEND_PORT_ENABLE_VALUE; import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.CONNECTOR_IOTDB_SEND_PORT_MAX_KEY; import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.CONNECTOR_IOTDB_SEND_PORT_MAX_VALUE; import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.CONNECTOR_IOTDB_SEND_PORT_MIN_KEY; import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.CONNECTOR_IOTDB_SEND_PORT_MIN_VALUE; +import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.CONNECTOR_IOTDB_SEND_PORT_RESTRICTION_RANGE_STRATEGY; +import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.CONNECTOR_IOTDB_SEND_PORT_RESTRICTION_STRATEGY_KEY; +import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.CONNECTOR_IOTDB_SEND_PORT_RESTRICTION_STRATEGY_SET; import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.CONNECTOR_LOAD_BALANCE_ROUND_ROBIN_STRATEGY; import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.CONNECTOR_LOAD_BALANCE_STRATEGY_KEY; import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.CONNECTOR_LOAD_BALANCE_STRATEGY_SET; @@ -116,9 +117,9 @@ import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.SINK_IOTDB_NODE_URLS_KEY; import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.SINK_IOTDB_PORT_KEY; import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.SINK_IOTDB_SEND_PORT_CANDIDATE_KEY; -import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.SINK_IOTDB_SEND_PORT_ENABLE_KEY; import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.SINK_IOTDB_SEND_PORT_MAX_KEY; import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.SINK_IOTDB_SEND_PORT_MIN_KEY; +import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.SINK_IOTDB_SEND_PORT_RESTRICTION_STRATEGY_KEY; import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.SINK_LOAD_BALANCE_STRATEGY_KEY; import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.SINK_LOAD_TSFILE_STRATEGY_KEY; import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.SINK_RATE_LIMIT_KEY; @@ -143,7 +144,7 @@ public abstract class IoTDBConnector implements PipeConnector { protected List candidatePorts; - protected boolean isCustomSendPortDefined; + protected String customSendPortStrategy; protected String loadBalanceStrategy; @@ -202,60 +203,68 @@ public void validate(final PipeParameterValidator validator) throws Exception { Arrays.asList(CONNECTOR_IOTDB_BATCH_SIZE_KEY, SINK_IOTDB_BATCH_SIZE_KEY), CONNECTOR_IOTDB_PLAIN_BATCH_SIZE_DEFAULT_VALUE)); - minSendPortRange = - parameters.getIntOrDefault( - Arrays.asList(CONNECTOR_IOTDB_SEND_PORT_MIN_KEY, SINK_IOTDB_SEND_PORT_MIN_KEY), - CONNECTOR_IOTDB_SEND_PORT_MIN_VALUE); - maxSendPortRange = - parameters.getIntOrDefault( - Arrays.asList(CONNECTOR_IOTDB_SEND_PORT_MAX_KEY, SINK_IOTDB_SEND_PORT_MAX_KEY), - CONNECTOR_IOTDB_SEND_PORT_MAX_VALUE); + this.customSendPortStrategy = + parameters + .getStringOrDefault( + Arrays.asList( + CONNECTOR_IOTDB_SEND_PORT_RESTRICTION_STRATEGY_KEY, + SINK_IOTDB_SEND_PORT_RESTRICTION_STRATEGY_KEY), + CONNECTOR_IOTDB_SEND_PORT_RESTRICTION_RANGE_STRATEGY) + .trim() + .toLowerCase(); + validator.validate( - args -> (boolean) args[0] && (boolean) args[1] && (boolean) args[2], + arg -> CONNECTOR_IOTDB_SEND_PORT_RESTRICTION_STRATEGY_SET.contains(customSendPortStrategy), String.format( - "%s must be <= %s, but got %d > %d", - SINK_IOTDB_SEND_PORT_MIN_KEY, - SINK_IOTDB_SEND_PORT_MAX_KEY, - minSendPortRange, - maxSendPortRange), - minSendPortRange <= maxSendPortRange, - minSendPortRange >= MIN_PORT, - maxSendPortRange <= MAX_PORT); - - this.candidatePorts = - parseCandidatePorts( - parameters.getStringOrDefault( - Arrays.asList( - CONNECTOR_IOTDB_SEND_PORT_CANDIDATE_KEY, SINK_IOTDB_SEND_PORT_CANDIDATE_KEY), - CONNECTOR_IOTDB_SEND_PORT_CANDIDATE_VALUE)); + "send port restriction strategy should be one of %s, but got %s.", + CONNECTOR_IOTDB_SEND_PORT_RESTRICTION_STRATEGY_SET, customSendPortStrategy), + customSendPortStrategy); - if (!candidatePorts.isEmpty()) { + final int parallelTaskNum = + parameters.getIntOrDefault( + Arrays.asList( + PipeConnectorConstant.CONNECTOR_IOTDB_PARALLEL_TASKS_KEY, + PipeConnectorConstant.SINK_IOTDB_PARALLEL_TASKS_KEY), + PipeConnectorConstant.CONNECTOR_IOTDB_PARALLEL_TASKS_DEFAULT_VALUE); + final int requiredPortsNum = + parallelTaskNum + * CommonDescriptor.getInstance().getConfig().getPipeAsyncConnectorMaxClientNumber(); + + if (CONNECTOR_IOTDB_SEND_PORT_RESTRICTION_RANGE_STRATEGY.equals(customSendPortStrategy)) { + minSendPortRange = + parameters.getIntOrDefault( + Arrays.asList(CONNECTOR_IOTDB_SEND_PORT_MIN_KEY, SINK_IOTDB_SEND_PORT_MIN_KEY), + CONNECTOR_IOTDB_SEND_PORT_MIN_VALUE); + maxSendPortRange = + parameters.getIntOrDefault( + Arrays.asList(CONNECTOR_IOTDB_SEND_PORT_MAX_KEY, SINK_IOTDB_SEND_PORT_MAX_KEY), + CONNECTOR_IOTDB_SEND_PORT_MAX_VALUE); validator.validate( - arg -> (int) arg[0] >= MIN_PORT && (int) arg[1] <= MAX_PORT, + args -> (int) args[0] < (int) args[1], String.format( - "Candidate port range is invalid: Ports must be between 0 and 65535, but got minimum port: %d and maximum port: %d", - candidatePorts.get(0), candidatePorts.get(candidatePorts.size() - 1)), - candidatePorts.get(0), - candidatePorts.get(candidatePorts.size() - 1)); - } - this.isCustomSendPortDefined = - parameters.getBooleanOrDefault( - Arrays.asList(CONNECTOR_IOTDB_SEND_PORT_ENABLE_KEY, SINK_IOTDB_SEND_PORT_ENABLE_KEY), - CONNECTOR_IOTDB_SEND_PORT_ENABLE_VALUE); - - if (isCustomSendPortDefined) { - int maxUsablePortsNum = - calculateUsablePorts(minSendPortRange, maxSendPortRange, candidatePorts); - int parallelTaskNum = - parameters.getIntOrDefault( - Arrays.asList( - PipeConnectorConstant.CONNECTOR_IOTDB_PARALLEL_TASKS_KEY, - PipeConnectorConstant.SINK_IOTDB_PARALLEL_TASKS_KEY), - PipeConnectorConstant.CONNECTOR_IOTDB_PARALLEL_TASKS_DEFAULT_VALUE); - int requiredPortsNum = - parallelTaskNum - * CommonDescriptor.getInstance().getConfig().getPipeAsyncConnectorMaxClientNumber(); - + "%s must be less than or equal to %s, but got %d > %d.", + SINK_IOTDB_SEND_PORT_MIN_KEY, + SINK_IOTDB_SEND_PORT_MAX_KEY, + minSendPortRange, + maxSendPortRange), + minSendPortRange, + maxSendPortRange); + validator.validate( + args -> (int) args[0] <= (int) args[1] || (int) args[2] >= (int) args[3], + String.format( + "Port range is invalid: %s must be >= %d and %s must be <= %d. Current values are %d and %d respectively.", + SINK_IOTDB_SEND_PORT_MIN_KEY, + MIN_PORT, + SINK_IOTDB_SEND_PORT_MAX_KEY, + MAX_PORT, + minSendPortRange, + maxSendPortRange), + MIN_PORT, + minSendPortRange, + MAX_PORT, + maxSendPortRange); + + final int maxUsablePortsNum = maxSendPortRange - minSendPortRange + 1; validator.validate( arg -> (int) arg[0] > (int) arg[1], String.format( @@ -263,6 +272,29 @@ public void validate(final PipeParameterValidator validator) throws Exception { maxUsablePortsNum, requiredPortsNum), maxUsablePortsNum, requiredPortsNum); + } else { + this.candidatePorts = + parseCandidatePorts( + parameters.getStringOrDefault( + Arrays.asList( + CONNECTOR_IOTDB_SEND_PORT_CANDIDATE_KEY, SINK_IOTDB_SEND_PORT_CANDIDATE_KEY), + CONNECTOR_IOTDB_SEND_PORT_CANDIDATE_VALUE)); + final int maxUsablePortsNum = candidatePorts.size(); + validator.validate( + arg -> (int) arg[0] > (int) arg[1], + String.format( + "Not enough available ports: There are %d available ports but require %d.", + maxUsablePortsNum, requiredPortsNum), + maxUsablePortsNum, + requiredPortsNum); + + validator.validate( + arg -> (int) arg[0] >= MIN_PORT && (int) arg[1] <= MAX_PORT, + String.format( + "Candidate port range is invalid: Ports must be between 0 and 65535, but got minimum port: %d and maximum port: %d", + candidatePorts.get(0), candidatePorts.get(candidatePorts.size() - 1)), + candidatePorts.get(0), + candidatePorts.get(candidatePorts.size() - 1)); } loadBalanceStrategy = @@ -522,6 +554,7 @@ private static List parseCandidatePorts(String candidate) { if (candidate == null || candidate.isEmpty()) { return Collections.emptyList(); } + System.out.println(candidate); return Arrays.stream(candidate.split(",")) .map(String::trim) .map(Integer::parseInt) @@ -529,16 +562,6 @@ private static List parseCandidatePorts(String candidate) { .collect(Collectors.toList()); } - private int calculateUsablePorts(int sendPortMin, int sendPortMax, List candidatePorts) { - int usablePortCount = sendPortMax - sendPortMin + 1; - for (int port : candidatePorts) { - if (sendPortMax < port || port < sendPortMin) { - usablePortCount++; - } - } - return usablePortCount; - } - @Override public void close() { // TODO: Not all the limiters should be closed here, but it's fine for now. diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/protocol/IoTDBSslSyncConnector.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/protocol/IoTDBSslSyncConnector.java index c4b76932d5d7..f41cccf2c217 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/protocol/IoTDBSslSyncConnector.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/protocol/IoTDBSslSyncConnector.java @@ -125,7 +125,7 @@ public void customize( loadBalanceStrategy, shouldReceiverConvertOnTypeMismatch, loadTsFileStrategy, - isCustomSendPortDefined, + customSendPortStrategy, minSendPortRange, maxSendPortRange, candidatePorts); @@ -140,7 +140,7 @@ protected abstract IoTDBSyncClientManager constructClient( final String loadBalanceStrategy, final boolean shouldReceiverConvertOnTypeMismatch, final String loadTsFileStrategy, - final boolean isCustomSendPortDefined, + final String customSendPortStrategy, final int minSendPortRange, final int maxSendPortRange, final List candidatePorts); From 3db93494c4257f6bd28d9dccd36907d9d2b03f70 Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Thu, 26 Sep 2024 23:54:43 +0800 Subject: [PATCH 07/12] fix --- .../pipe/connector/protocol/IoTDBConnector.java | 14 +++----------- 1 file changed, 3 insertions(+), 11 deletions(-) diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/protocol/IoTDBConnector.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/protocol/IoTDBConnector.java index ed9d939faa43..a78340307f37 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/protocol/IoTDBConnector.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/protocol/IoTDBConnector.java @@ -21,7 +21,6 @@ import org.apache.iotdb.common.rpc.thrift.TEndPoint; import org.apache.iotdb.commons.conf.CommonDescriptor; -import org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant; import org.apache.iotdb.commons.pipe.connector.compressor.PipeCompressor; import org.apache.iotdb.commons.pipe.connector.compressor.PipeCompressorConfig; import org.apache.iotdb.commons.pipe.connector.compressor.PipeCompressorFactory; @@ -220,15 +219,8 @@ public void validate(final PipeParameterValidator validator) throws Exception { CONNECTOR_IOTDB_SEND_PORT_RESTRICTION_STRATEGY_SET, customSendPortStrategy), customSendPortStrategy); - final int parallelTaskNum = - parameters.getIntOrDefault( - Arrays.asList( - PipeConnectorConstant.CONNECTOR_IOTDB_PARALLEL_TASKS_KEY, - PipeConnectorConstant.SINK_IOTDB_PARALLEL_TASKS_KEY), - PipeConnectorConstant.CONNECTOR_IOTDB_PARALLEL_TASKS_DEFAULT_VALUE); final int requiredPortsNum = - parallelTaskNum - * CommonDescriptor.getInstance().getConfig().getPipeAsyncConnectorMaxClientNumber(); + CommonDescriptor.getInstance().getConfig().getPipeAsyncConnectorMaxClientNumber(); if (CONNECTOR_IOTDB_SEND_PORT_RESTRICTION_RANGE_STRATEGY.equals(customSendPortStrategy)) { minSendPortRange = @@ -250,7 +242,7 @@ public void validate(final PipeParameterValidator validator) throws Exception { minSendPortRange, maxSendPortRange); validator.validate( - args -> (int) args[0] <= (int) args[1] || (int) args[2] >= (int) args[3], + args -> (int) args[0] <= (int) args[1] && (int) args[2] >= (int) args[3], String.format( "Port range is invalid: %s must be >= %d and %s must be <= %d. Current values are %d and %d respectively.", SINK_IOTDB_SEND_PORT_MIN_KEY, @@ -266,7 +258,7 @@ public void validate(final PipeParameterValidator validator) throws Exception { final int maxUsablePortsNum = maxSendPortRange - minSendPortRange + 1; validator.validate( - arg -> (int) arg[0] > (int) arg[1], + arg -> (int) arg[0] >= (int) arg[1], String.format( "Not enough available ports: There are %d available ports but require %d.", maxUsablePortsNum, requiredPortsNum), From 5f1d94fc715bc3c66ff15cc2f9b6792630529be8 Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Fri, 27 Sep 2024 10:33:12 +0800 Subject: [PATCH 08/12] Add test case --- .../IoTDBPipeConnectorCustomPortIT.java | 25 +++++++++++-------- .../client/util/IoTDBConnectorPortBinder.java | 6 ++--- .../protocol/IoTDBAirGapConnector.java | 3 +-- .../connector/protocol/IoTDBConnector.java | 3 +-- 4 files changed, 20 insertions(+), 17 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeConnectorCustomPortIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeConnectorCustomPortIT.java index b93c700534eb..8f7a800237b9 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeConnectorCustomPortIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeConnectorCustomPortIT.java @@ -75,6 +75,20 @@ public void setUp() { receiverEnv.initClusterEnvironment(); } + @Test + public void testRangePortsUnavailable() { + doTest( + "iotdb-thrift-async-connector", + receiverEnv.getIP() + ":" + receiverEnv.getPort(), + "range", + "1026", + "1025", + 0, + 10000, + 0); + } + + // Test Port Insufficient @Test public void testRangePortsAvailable() { doTest( @@ -84,11 +98,11 @@ public void testRangePortsAvailable() { "1024", "1025", 0, - "16", 10000, 0); } + // Test Port Insufficient @Test public void testCandidatePortsAvailable() { doTest( @@ -98,7 +112,6 @@ public void testCandidatePortsAvailable() { null, null, 4, - "16", 10000, 0); } @@ -112,7 +125,6 @@ public void testAsyncPortRange() { "1024", "1224", 0, - "3", 2000, 2); } @@ -126,7 +138,6 @@ public void testAsyncPortCandidate() { "1024", "1224", 30, - "1", 2000, 2); } @@ -140,7 +151,6 @@ public void testSyncThriftPortRange() { "1024", "1224", 30, - "1", 2000, 2); } @@ -154,7 +164,6 @@ public void testSyncThriftPortCandidate() { "1024", "1224", 30, - "1", 2000, 2); } @@ -168,7 +177,6 @@ public void testAirGapPortRange() { "1024", "1224", 30, - "1", 2000, 2); } @@ -182,7 +190,6 @@ public void testAirGapPortCandidate() { "1024", "1224", 30, - "1", 2000, 2); } @@ -194,7 +201,6 @@ private void doTest( final String min, final String max, final int candidateNum, - final String tasks, final long sleepTime, final int resultNum) { try (final SyncConfigNodeIServiceClient client = @@ -218,7 +224,6 @@ private void doTest( candidateBuilder.append(1024 + candidateNum); connectorAttributes.put("send-port.candidate", candidateBuilder.toString()); } - connectorAttributes.put("parallel.tasks", tasks); final TSStatus status = client.createPipe( diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/util/IoTDBConnectorPortBinder.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/util/IoTDBConnectorPortBinder.java index 6f12b84fb38b..2761800a0afd 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/util/IoTDBConnectorPortBinder.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/util/IoTDBConnectorPortBinder.java @@ -45,8 +45,8 @@ public static void bindPort( customSendPortStrategy); boolean portFound = false; int index = 0; - boolean isNotEnd = true; - while (isNotEnd) { + boolean searching = isRange || !candidatePorts.isEmpty(); + while (searching) { int port = isRange ? minSendPortRange + index : candidatePorts.get(index); try { consumer.accept(port); @@ -55,7 +55,7 @@ public static void bindPort( } catch (Exception ignored) { } index++; - isNotEnd = isRange ? port <= maxSendPortRange : candidatePorts.size() > index; + searching = isRange ? port <= maxSendPortRange : candidatePorts.size() > index; } if (!portFound) { String exceptionMessage = diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/protocol/IoTDBAirGapConnector.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/protocol/IoTDBAirGapConnector.java index 3952b260259b..4c5776030701 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/protocol/IoTDBAirGapConnector.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/protocol/IoTDBAirGapConnector.java @@ -159,8 +159,7 @@ public void handshake() throws Exception { // Close the socket if necessary if (sockets.get(i) != null) { try { - final Socket socket = sockets.set(i, null); - socket.close(); + sockets.set(i, null).close(); } catch (final Exception e) { LOGGER.warn( "Failed to close socket with target server ip: {}, port: {}, because: {}. Ignore it.", diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/protocol/IoTDBConnector.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/protocol/IoTDBConnector.java index a78340307f37..3d352c07ad17 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/protocol/IoTDBConnector.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/protocol/IoTDBConnector.java @@ -232,7 +232,7 @@ public void validate(final PipeParameterValidator validator) throws Exception { Arrays.asList(CONNECTOR_IOTDB_SEND_PORT_MAX_KEY, SINK_IOTDB_SEND_PORT_MAX_KEY), CONNECTOR_IOTDB_SEND_PORT_MAX_VALUE); validator.validate( - args -> (int) args[0] < (int) args[1], + args -> (int) args[0] <= (int) args[1], String.format( "%s must be less than or equal to %s, but got %d > %d.", SINK_IOTDB_SEND_PORT_MIN_KEY, @@ -546,7 +546,6 @@ private static List parseCandidatePorts(String candidate) { if (candidate == null || candidate.isEmpty()) { return Collections.emptyList(); } - System.out.println(candidate); return Arrays.stream(candidate.split(",")) .map(String::trim) .map(Integer::parseInt) From 0d0442f10751bd2b2bac215c6fd2d27a55f218c8 Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Fri, 27 Sep 2024 14:46:59 +0800 Subject: [PATCH 09/12] Remove the logic check of comparing available ports with actually used ports --- .../connector/protocol/IoTDBConnector.java | 24 +++---------------- 1 file changed, 3 insertions(+), 21 deletions(-) diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/protocol/IoTDBConnector.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/protocol/IoTDBConnector.java index 3d352c07ad17..19b2b6078b16 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/protocol/IoTDBConnector.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/protocol/IoTDBConnector.java @@ -20,7 +20,6 @@ package org.apache.iotdb.commons.pipe.connector.protocol; import org.apache.iotdb.common.rpc.thrift.TEndPoint; -import org.apache.iotdb.commons.conf.CommonDescriptor; import org.apache.iotdb.commons.pipe.connector.compressor.PipeCompressor; import org.apache.iotdb.commons.pipe.connector.compressor.PipeCompressorConfig; import org.apache.iotdb.commons.pipe.connector.compressor.PipeCompressorFactory; @@ -219,9 +218,6 @@ public void validate(final PipeParameterValidator validator) throws Exception { CONNECTOR_IOTDB_SEND_PORT_RESTRICTION_STRATEGY_SET, customSendPortStrategy), customSendPortStrategy); - final int requiredPortsNum = - CommonDescriptor.getInstance().getConfig().getPipeAsyncConnectorMaxClientNumber(); - if (CONNECTOR_IOTDB_SEND_PORT_RESTRICTION_RANGE_STRATEGY.equals(customSendPortStrategy)) { minSendPortRange = parameters.getIntOrDefault( @@ -255,15 +251,6 @@ public void validate(final PipeParameterValidator validator) throws Exception { minSendPortRange, MAX_PORT, maxSendPortRange); - - final int maxUsablePortsNum = maxSendPortRange - minSendPortRange + 1; - validator.validate( - arg -> (int) arg[0] >= (int) arg[1], - String.format( - "Not enough available ports: There are %d available ports but require %d.", - maxUsablePortsNum, requiredPortsNum), - maxUsablePortsNum, - requiredPortsNum); } else { this.candidatePorts = parseCandidatePorts( @@ -271,15 +258,10 @@ public void validate(final PipeParameterValidator validator) throws Exception { Arrays.asList( CONNECTOR_IOTDB_SEND_PORT_CANDIDATE_KEY, SINK_IOTDB_SEND_PORT_CANDIDATE_KEY), CONNECTOR_IOTDB_SEND_PORT_CANDIDATE_VALUE)); - final int maxUsablePortsNum = candidatePorts.size(); validator.validate( - arg -> (int) arg[0] > (int) arg[1], - String.format( - "Not enough available ports: There are %d available ports but require %d.", - maxUsablePortsNum, requiredPortsNum), - maxUsablePortsNum, - requiredPortsNum); - + arg -> (int) arg > 0, + "The number of candidate ports must be greater than 0.", + candidatePorts.size()); validator.validate( arg -> (int) arg[0] >= MIN_PORT && (int) arg[1] <= MAX_PORT, String.format( From f904b6d3c2240562d2a2e732e056046733574a81 Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Fri, 27 Sep 2024 14:48:40 +0800 Subject: [PATCH 10/12] Delete test case --- .../IoTDBPipeConnectorCustomPortIT.java | 41 ------------------- 1 file changed, 41 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeConnectorCustomPortIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeConnectorCustomPortIT.java index 8f7a800237b9..217d9fe87355 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeConnectorCustomPortIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeConnectorCustomPortIT.java @@ -75,47 +75,6 @@ public void setUp() { receiverEnv.initClusterEnvironment(); } - @Test - public void testRangePortsUnavailable() { - doTest( - "iotdb-thrift-async-connector", - receiverEnv.getIP() + ":" + receiverEnv.getPort(), - "range", - "1026", - "1025", - 0, - 10000, - 0); - } - - // Test Port Insufficient - @Test - public void testRangePortsAvailable() { - doTest( - "iotdb-thrift-async-connector", - receiverEnv.getIP() + ":" + receiverEnv.getPort(), - "range", - "1024", - "1025", - 0, - 10000, - 0); - } - - // Test Port Insufficient - @Test - public void testCandidatePortsAvailable() { - doTest( - "iotdb-thrift-async-connector", - receiverEnv.getIP() + ":" + receiverEnv.getPort(), - "candidate", - null, - null, - 4, - 10000, - 0); - } - @Test public void testAsyncPortRange() { doTest( From 347bbbf773dfa186b65a14501b7b7fd48d8fe287 Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Fri, 27 Sep 2024 15:15:46 +0800 Subject: [PATCH 11/12] Modify code --- .../IoTDBPipeConnectorCustomPortIT.java | 71 ++++--------------- 1 file changed, 14 insertions(+), 57 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeConnectorCustomPortIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeConnectorCustomPortIT.java index 217d9fe87355..9e5f4a15cf22 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeConnectorCustomPortIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeConnectorCustomPortIT.java @@ -38,7 +38,6 @@ import java.util.Arrays; import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; import java.util.Map; @RunWith(IoTDBTestRunner.class) @@ -78,14 +77,7 @@ public void setUp() { @Test public void testAsyncPortRange() { doTest( - "iotdb-thrift-async-connector", - receiverEnv.getIP() + ":" + receiverEnv.getPort(), - "range", - "1024", - "1224", - 0, - 2000, - 2); + "iotdb-thrift-async-connector", receiverEnv.getIP() + ":" + receiverEnv.getPort(), "range"); } @Test @@ -93,25 +85,13 @@ public void testAsyncPortCandidate() { doTest( "iotdb-thrift-async-connector", receiverEnv.getIP() + ":" + receiverEnv.getPort(), - "candidate", - "1024", - "1224", - 30, - 2000, - 2); + "candidate"); } @Test public void testSyncThriftPortRange() { doTest( - "iotdb-thrift-sync-connector", - receiverEnv.getIP() + ":" + receiverEnv.getPort(), - "range", - "1024", - "1224", - 30, - 2000, - 2); + "iotdb-thrift-sync-connector", receiverEnv.getIP() + ":" + receiverEnv.getPort(), "range"); } @Test @@ -119,12 +99,7 @@ public void testSyncThriftPortCandidate() { doTest( "iotdb-thrift-sync-connector", receiverEnv.getIP() + ":" + receiverEnv.getPort(), - "candidate", - "1024", - "1224", - 30, - 2000, - 2); + "candidate"); } @Test @@ -132,12 +107,7 @@ public void testAirGapPortRange() { doTest( "iotdb-air-gap-connector", receiverEnv.getIP() + ":" + receiverEnv.getDataNodeWrapper(0).getPipeAirGapReceiverPort(), - "range", - "1024", - "1224", - 30, - 2000, - 2); + "range"); } @Test @@ -145,23 +115,10 @@ public void testAirGapPortCandidate() { doTest( "iotdb-air-gap-connector", receiverEnv.getIP() + ":" + receiverEnv.getDataNodeWrapper(0).getPipeAirGapReceiverPort(), - "candidate", - "1024", - "1224", - 30, - 2000, - 2); + "candidate"); } - private void doTest( - final String connector, - final String urls, - final String strategy, - final String min, - final String max, - final int candidateNum, - final long sleepTime, - final int resultNum) { + private void doTest(final String connector, final String urls, final String strategy) { try (final SyncConfigNodeIServiceClient client = (SyncConfigNodeIServiceClient) senderEnv.getLeaderConfigNodeConnection()) { final Map extractorAttributes = new HashMap<>(); @@ -173,14 +130,14 @@ private void doTest( connectorAttributes.put("node-urls", urls); connectorAttributes.put("send-port.restriction-strategy", strategy); if (strategy.equals("range")) { - connectorAttributes.put("send-port.range.min", min); - connectorAttributes.put("send-port.range.max", max); + connectorAttributes.put("send-port.range.min", "1024"); + connectorAttributes.put("send-port.range.max", "1524"); } else { StringBuilder candidateBuilder = new StringBuilder(); - for (int i = 0; i < candidateNum; i++) { + for (int i = 0; i < 30; i++) { candidateBuilder.append(1024 + i).append(","); } - candidateBuilder.append(1024 + candidateNum); + candidateBuilder.append(1024 + 30); connectorAttributes.put("send-port.candidate", candidateBuilder.toString()); } @@ -192,7 +149,7 @@ private void doTest( Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode()); - Thread.sleep(sleepTime); + Thread.sleep(2000); TestUtils.tryExecuteNonQueriesWithRetry( senderEnv, Arrays.asList( @@ -202,8 +159,8 @@ private void doTest( TestUtils.assertDataEventuallyOnEnv( receiverEnv, "select count(*) from root.**", - resultNum == 0 ? "Time," : "count(root.db.d1.s1),", - resultNum == 0 ? new HashSet<>() : Collections.singleton(resultNum + ",")); + "count(root.db.d1.s1),", + Collections.singleton("2,")); } catch (Exception e) { Assert.fail(); From b8414db4df962e0fdc3b3119a7a5c4f3b45b4675 Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Sun, 29 Sep 2024 16:44:56 +0800 Subject: [PATCH 12/12] Fix the problem of Async creating Client --- .../IoTDBDataNodeAsyncClientManager.java | 61 +++++------ .../async/IoTDBDataRegionAsyncConnector.java | 32 ++++-- .../commons/client/ClientPoolFactory.java | 36 ++----- .../AsyncPipeDataTransferServiceClient.java | 44 +++----- .../commons/client/async/AsyncTEndPoint.java | 101 ++++++++++++++++++ .../connector/client/IoTDBClientManager.java | 5 + 6 files changed, 177 insertions(+), 102 deletions(-) create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/async/AsyncTEndPoint.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/client/IoTDBDataNodeAsyncClientManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/client/IoTDBDataNodeAsyncClientManager.java index 81bc6485d90c..99ba8658ee7b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/client/IoTDBDataNodeAsyncClientManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/client/IoTDBDataNodeAsyncClientManager.java @@ -23,6 +23,7 @@ import org.apache.iotdb.commons.client.ClientPoolFactory; import org.apache.iotdb.commons.client.IClientManager; import org.apache.iotdb.commons.client.async.AsyncPipeDataTransferServiceClient; +import org.apache.iotdb.commons.client.async.AsyncTEndPoint; import org.apache.iotdb.commons.conf.CommonDescriptor; import org.apache.iotdb.commons.pipe.config.PipeConfig; import org.apache.iotdb.commons.pipe.connector.client.IoTDBClientManager; @@ -59,16 +60,18 @@ public class IoTDBDataNodeAsyncClientManager extends IoTDBClientManager private static final Logger LOGGER = LoggerFactory.getLogger(IoTDBDataNodeAsyncClientManager.class); - private final Set endPointSet; + private final Set endPointSet; + private final List tEndPointList; private static final Map RECEIVER_ATTRIBUTES_REF_COUNT = new ConcurrentHashMap<>(); private final String receiverAttributes; - // receiverAttributes -> IClientManager - private static final Map> + // receiverAttributes -> IClientManager + private static final Map< + String, IClientManager> ASYNC_PIPE_DATA_TRANSFER_CLIENT_MANAGER_HOLDER = new ConcurrentHashMap<>(); - private final IClientManager endPoint2Client; + private final IClientManager endPoint2Client; private final LoadBalancer loadBalancer; @@ -77,23 +80,13 @@ public class IoTDBDataNodeAsyncClientManager extends IoTDBClientManager private final String loadTsFileStrategy; public IoTDBDataNodeAsyncClientManager( - List endPoints, + List endPoints, boolean useLeaderCache, String loadBalanceStrategy, boolean shouldReceiverConvertOnTypeMismatch, - String loadTsFileStrategy, - String customSendPortStrategy, - int minSendPortRange, - int maxSendPortRange, - List candidatePorts) { - super( - endPoints, - useLeaderCache, - customSendPortStrategy, - minSendPortRange, - maxSendPortRange, - candidatePorts); - + String loadTsFileStrategy) { + super(null, useLeaderCache); + this.tEndPointList = endPoints; endPointSet = new HashSet<>(endPoints); receiverAttributes = @@ -102,13 +95,9 @@ public IoTDBDataNodeAsyncClientManager( if (!ASYNC_PIPE_DATA_TRANSFER_CLIENT_MANAGER_HOLDER.containsKey(receiverAttributes)) { ASYNC_PIPE_DATA_TRANSFER_CLIENT_MANAGER_HOLDER.putIfAbsent( receiverAttributes, - new IClientManager.Factory() + new IClientManager.Factory() .createClientManager( - new ClientPoolFactory.AsyncPipeDataTransferServiceClientPoolFactory( - this.customSendPortStrategy, - minSendPortRange, - maxSendPortRange, - candidatePorts))); + new ClientPoolFactory.AsyncPipeDataTransferServiceClientPoolFactory())); } endPoint2Client = ASYNC_PIPE_DATA_TRANSFER_CLIENT_MANAGER_HOLDER.get(receiverAttributes); @@ -146,10 +135,10 @@ public AsyncPipeDataTransferServiceClient borrowClient(String deviceId) throws E return borrowClient(); } - return borrowClient(LEADER_CACHE_MANAGER.getLeaderEndPoint(deviceId)); + return borrowClient((AsyncTEndPoint) LEADER_CACHE_MANAGER.getLeaderEndPoint(deviceId)); } - public AsyncPipeDataTransferServiceClient borrowClient(TEndPoint endPoint) throws Exception { + public AsyncPipeDataTransferServiceClient borrowClient(AsyncTEndPoint endPoint) throws Exception { if (!useLeaderCache || Objects.isNull(endPoint)) { return borrowClient(); } @@ -296,13 +285,13 @@ private void waitHandshakeFinished(AtomicBoolean isHandshakeFinished) { } } - public void updateLeaderCache(String deviceId, TEndPoint endPoint) { + public void updateLeaderCache(String deviceId, AsyncTEndPoint endPoint) { if (!useLeaderCache || deviceId == null || endPoint == null) { return; } if (!endPointSet.contains(endPoint)) { - endPointList.add(endPoint); + tEndPointList.add(endPoint); endPointSet.add(endPoint); } @@ -315,8 +304,9 @@ public void close() { receiverAttributes, (attributes, refCount) -> { if (refCount <= 1) { - final IClientManager clientManager = - ASYNC_PIPE_DATA_TRANSFER_CLIENT_MANAGER_HOLDER.remove(receiverAttributes); + final IClientManager + clientManager = + ASYNC_PIPE_DATA_TRANSFER_CLIENT_MANAGER_HOLDER.remove(receiverAttributes); if (clientManager != null) { try { clientManager.close(); @@ -340,9 +330,10 @@ private interface LoadBalancer { private class RoundRobinLoadBalancer implements LoadBalancer { @Override public AsyncPipeDataTransferServiceClient borrowClient() throws Exception { - final int clientSize = endPointList.size(); + final int clientSize = tEndPointList.size(); while (true) { - final TEndPoint targetNodeUrl = endPointList.get((int) (currentClientIndex++ % clientSize)); + final AsyncTEndPoint targetNodeUrl = + tEndPointList.get((int) (currentClientIndex++ % clientSize)); final AsyncPipeDataTransferServiceClient client = endPoint2Client.borrowClient(targetNodeUrl); if (handshakeIfNecessary(targetNodeUrl, client)) { @@ -355,9 +346,9 @@ public AsyncPipeDataTransferServiceClient borrowClient() throws Exception { private class RandomLoadBalancer implements LoadBalancer { @Override public AsyncPipeDataTransferServiceClient borrowClient() throws Exception { - final int clientSize = endPointList.size(); + final int clientSize = tEndPointList.size(); while (true) { - final TEndPoint targetNodeUrl = endPointList.get((int) (Math.random() * clientSize)); + final AsyncTEndPoint targetNodeUrl = tEndPointList.get((int) (Math.random() * clientSize)); final AsyncPipeDataTransferServiceClient client = endPoint2Client.borrowClient(targetNodeUrl); if (handshakeIfNecessary(targetNodeUrl, client)) { @@ -371,7 +362,7 @@ private class PriorityLoadBalancer implements LoadBalancer { @Override public AsyncPipeDataTransferServiceClient borrowClient() throws Exception { while (true) { - for (final TEndPoint targetNodeUrl : endPointList) { + for (final AsyncTEndPoint targetNodeUrl : tEndPointList) { final AsyncPipeDataTransferServiceClient client = endPoint2Client.borrowClient(targetNodeUrl); if (handshakeIfNecessary(targetNodeUrl, client)) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/IoTDBDataRegionAsyncConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/IoTDBDataRegionAsyncConnector.java index 9d4f9270d6b8..cea1daf1f647 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/IoTDBDataRegionAsyncConnector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/IoTDBDataRegionAsyncConnector.java @@ -21,6 +21,7 @@ import org.apache.iotdb.common.rpc.thrift.TEndPoint; import org.apache.iotdb.commons.client.async.AsyncPipeDataTransferServiceClient; +import org.apache.iotdb.commons.client.async.AsyncTEndPoint; import org.apache.iotdb.commons.pipe.connector.protocol.IoTDBConnector; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.db.pipe.agent.task.subtask.connector.PipeConnectorSubtask; @@ -70,6 +71,7 @@ import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.CONNECTOR_LEADER_CACHE_ENABLE_DEFAULT_VALUE; import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.CONNECTOR_LEADER_CACHE_ENABLE_KEY; @@ -120,17 +122,23 @@ public void customize( clientManager = new IoTDBDataNodeAsyncClientManager( - nodeUrls, + nodeUrls.stream() + .map( + tEndPoint -> + new AsyncTEndPoint( + tEndPoint.getIp(), + tEndPoint.getPort(), + minSendPortRange, + maxSendPortRange, + candidatePorts, + customSendPortStrategy)) + .collect(Collectors.toList()), parameters.getBooleanOrDefault( Arrays.asList(SINK_LEADER_CACHE_ENABLE_KEY, CONNECTOR_LEADER_CACHE_ENABLE_KEY), CONNECTOR_LEADER_CACHE_ENABLE_DEFAULT_VALUE), loadBalanceStrategy, shouldReceiverConvertOnTypeMismatch, - loadTsFileStrategy, - customSendPortStrategy, - minSendPortRange, - maxSendPortRange, - candidatePorts); + loadTsFileStrategy); if (isTabletBatchModeEnabled) { tabletBatchBuilder = new PipeTransferBatchReqBuilder(parameters); @@ -263,7 +271,7 @@ private void transfer( final PipeTransferTabletBatchEventHandler pipeTransferTabletBatchEventHandler) { AsyncPipeDataTransferServiceClient client = null; try { - client = clientManager.borrowClient(endPoint); + client = clientManager.borrowClient((AsyncTEndPoint) endPoint); pipeTransferTabletBatchEventHandler.transfer(client); } catch (final Exception ex) { logOnClientException(client, ex); @@ -383,7 +391,15 @@ public void transfer(final Event event) throws Exception { //////////////////////////// Leader cache update //////////////////////////// public void updateLeaderCache(final String deviceId, final TEndPoint endPoint) { - clientManager.updateLeaderCache(deviceId, endPoint); + clientManager.updateLeaderCache( + deviceId, + new AsyncTEndPoint( + endPoint.getIp(), + endPoint.getPort(), + minSendPortRange, + maxSendPortRange, + candidatePorts, + customSendPortStrategy)); } //////////////////////////// Exception handlers //////////////////////////// diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/ClientPoolFactory.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/ClientPoolFactory.java index 9f28baa0d208..968413d54abf 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/ClientPoolFactory.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/ClientPoolFactory.java @@ -28,6 +28,7 @@ import org.apache.iotdb.commons.client.async.AsyncDataNodeMPPDataExchangeServiceClient; import org.apache.iotdb.commons.client.async.AsyncPipeConsensusServiceClient; import org.apache.iotdb.commons.client.async.AsyncPipeDataTransferServiceClient; +import org.apache.iotdb.commons.client.async.AsyncTEndPoint; import org.apache.iotdb.commons.client.property.ClientPoolProperty; import org.apache.iotdb.commons.client.property.PipeConsensusClientProperty; import org.apache.iotdb.commons.client.property.ThriftClientProperty; @@ -43,8 +44,6 @@ import org.apache.commons.pool2.KeyedObjectPool; import org.apache.commons.pool2.impl.GenericKeyedObjectPool; -import java.util.List; - public class ClientPoolFactory { private static final CommonConfig conf = CommonDescriptor.getInstance().getConfig(); @@ -271,31 +270,14 @@ public KeyedObjectPool cre } public static class AsyncPipeDataTransferServiceClientPoolFactory - implements IClientPoolFactory { - - private int minSendPortRange; - - private int maxSendPortRange; + implements IClientPoolFactory { - private List candidatePorts; - - private String customSendPortStrategy; - - public AsyncPipeDataTransferServiceClientPoolFactory( - String customSendPortStrategy, - int minSendPortRange, - int maxSendPortRange, - List candidatePorts) { - this.minSendPortRange = minSendPortRange; - this.maxSendPortRange = maxSendPortRange; - this.candidatePorts = candidatePorts; - this.customSendPortStrategy = customSendPortStrategy; - } + public AsyncPipeDataTransferServiceClientPoolFactory() {} @Override - public KeyedObjectPool createClientPool( - ClientManager manager) { - final GenericKeyedObjectPool clientPool = + public KeyedObjectPool createClientPool( + ClientManager manager) { + final GenericKeyedObjectPool clientPool = new GenericKeyedObjectPool<>( new AsyncPipeDataTransferServiceClient.Factory( manager, @@ -306,11 +288,7 @@ public KeyedObjectPool createClie .setSelectorNumOfAsyncClientManager( conf.getPipeAsyncConnectorSelectorNumber()) .build(), - ThreadName.PIPE_ASYNC_CONNECTOR_CLIENT_POOL.getName(), - customSendPortStrategy, - minSendPortRange, - maxSendPortRange, - candidatePorts), + ThreadName.PIPE_ASYNC_CONNECTOR_CLIENT_POOL.getName()), new ClientPoolProperty.Builder() .setMaxClientNumForEachNode(conf.getPipeAsyncConnectorMaxClientNumber()) .build() diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/async/AsyncPipeDataTransferServiceClient.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/async/AsyncPipeDataTransferServiceClient.java index e671cad6befc..28e91efe0f85 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/async/AsyncPipeDataTransferServiceClient.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/async/AsyncPipeDataTransferServiceClient.java @@ -54,8 +54,8 @@ public class AsyncPipeDataTransferServiceClient extends IClientRPCService.AsyncC private final boolean printLogWhenEncounterException; - private final TEndPoint endpoint; - private final ClientManager clientManager; + private final AsyncTEndPoint endpoint; + private final ClientManager clientManager; private final AtomicBoolean shouldReturnSelf = new AtomicBoolean(true); @@ -63,9 +63,9 @@ public class AsyncPipeDataTransferServiceClient extends IClientRPCService.AsyncC public AsyncPipeDataTransferServiceClient( ThriftClientProperty property, - TEndPoint endpoint, + AsyncTEndPoint endpoint, TAsyncClientManager tClientManager, - ClientManager clientManager, + ClientManager clientManager, String customSendPortStrategy, int minSendPortRange, int maxSendPortRange, @@ -192,39 +192,23 @@ public String toString() { } public static class Factory - extends AsyncThriftClientFactory { - - private final int minSendPortRange; - - private final int maxSendPortRange; - - private final List candidatePorts; - - private final String customSendPortStrategy; + extends AsyncThriftClientFactory { public Factory( - ClientManager clientManager, + ClientManager clientManager, ThriftClientProperty thriftClientProperty, - String threadName, - String customSendPortStrategy, - int minSendPortRange, - int maxSendPortRange, - List candidatePorts) { + String threadName) { super(clientManager, thriftClientProperty, threadName); - this.customSendPortStrategy = customSendPortStrategy; - this.minSendPortRange = minSendPortRange; - this.maxSendPortRange = maxSendPortRange; - this.candidatePorts = candidatePorts; } @Override public void destroyObject( - TEndPoint endPoint, PooledObject pooledObject) { + AsyncTEndPoint endPoint, PooledObject pooledObject) { pooledObject.getObject().close(); } @Override - public PooledObject makeObject(TEndPoint endPoint) + public PooledObject makeObject(AsyncTEndPoint endPoint) throws Exception { return new DefaultPooledObject<>( new AsyncPipeDataTransferServiceClient( @@ -232,15 +216,15 @@ public PooledObject makeObject(TEndPoint end endPoint, tManagers[clientCnt.incrementAndGet() % tManagers.length], clientManager, - customSendPortStrategy, - minSendPortRange, - maxSendPortRange, - candidatePorts)); + endPoint.getCustomSendPortStrategy(), + endPoint.getMaxSendPortRange(), + endPoint.getMaxSendPortRange(), + endPoint.getCandidatePorts())); } @Override public boolean validateObject( - TEndPoint endPoint, PooledObject pooledObject) { + AsyncTEndPoint endPoint, PooledObject pooledObject) { return pooledObject.getObject().isReady(); } } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/async/AsyncTEndPoint.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/async/AsyncTEndPoint.java new file mode 100644 index 000000000000..8981c6f38fcd --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/async/AsyncTEndPoint.java @@ -0,0 +1,101 @@ +/* + * 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.iotdb.commons.client.async; + +import org.apache.iotdb.common.rpc.thrift.TEndPoint; + +import java.util.List; +import java.util.Objects; + +public class AsyncTEndPoint extends TEndPoint { + + private int minSendPortRange; + private int maxSendPortRange; + private List candidatePorts; + private String customSendPortStrategy; + + public AsyncTEndPoint( + String ip, + int port, + int minSendPortRange, + int maxSendPortRange, + List candidatePorts, + String customSendPortStrategy) { + super(ip, port); + this.minSendPortRange = minSendPortRange; + this.maxSendPortRange = maxSendPortRange; + this.candidatePorts = candidatePorts; + this.customSendPortStrategy = customSendPortStrategy; + } + + public int getMinSendPortRange() { + return minSendPortRange; + } + + public void setMinSendPortRange(int minSendPortRange) { + this.minSendPortRange = minSendPortRange; + } + + public int getMaxSendPortRange() { + return maxSendPortRange; + } + + public void setMaxSendPortRange(int maxSendPortRange) { + this.maxSendPortRange = maxSendPortRange; + } + + public List getCandidatePorts() { + return candidatePorts; + } + + public void setCandidatePorts(List candidatePorts) { + this.candidatePorts = candidatePorts; + } + + public String getCustomSendPortStrategy() { + return customSendPortStrategy; + } + + public void setCustomSendPortStrategy(String customSendPortStrategy) { + this.customSendPortStrategy = customSendPortStrategy; + } + + @Override + public int hashCode() { + return Objects.hash( + super.hashCode(), + minSendPortRange, + maxSendPortRange, + candidatePorts, + customSendPortStrategy); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) return true; + if (!(obj instanceof AsyncTEndPoint)) return false; + AsyncTEndPoint that = (AsyncTEndPoint) obj; + return minSendPortRange == that.minSendPortRange + && maxSendPortRange == that.maxSendPortRange + && Objects.equals(candidatePorts, that.candidatePorts) + && Objects.equals(customSendPortStrategy, that.customSendPortStrategy) + && super.equals(obj); + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/client/IoTDBClientManager.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/client/IoTDBClientManager.java index 2cfd7964aa27..5cb338eb7656 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/client/IoTDBClientManager.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/client/IoTDBClientManager.java @@ -71,6 +71,11 @@ protected IoTDBClientManager( this.candidatePorts = candidatePorts; } + protected IoTDBClientManager(List endPointList, boolean useLeaderCache) { + this.endPointList = endPointList; + this.useLeaderCache = useLeaderCache; + } + public boolean supportModsIfIsDataNodeReceiver() { return supportModsIfIsDataNodeReceiver; }