Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Pipe: add IoTDBConnector sender port configuration #13593

Draft
wants to merge 14 commits into
base: master
Choose a base branch
from
Original file line number Diff line number Diff line change
@@ -0,0 +1,169 @@
/*
* 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.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 testAsyncPortRange() {
doTest(
"iotdb-thrift-async-connector", receiverEnv.getIP() + ":" + receiverEnv.getPort(), "range");
}

@Test
public void testAsyncPortCandidate() {
doTest(
"iotdb-thrift-async-connector",
receiverEnv.getIP() + ":" + receiverEnv.getPort(),
"candidate");
}

@Test
public void testSyncThriftPortRange() {
doTest(
"iotdb-thrift-sync-connector", receiverEnv.getIP() + ":" + receiverEnv.getPort(), "range");
}

@Test
public void testSyncThriftPortCandidate() {
doTest(
"iotdb-thrift-sync-connector",
receiverEnv.getIP() + ":" + receiverEnv.getPort(),
"candidate");
}

@Test
public void testAirGapPortRange() {
doTest(
"iotdb-air-gap-connector",
receiverEnv.getIP() + ":" + receiverEnv.getDataNodeWrapper(0).getPipeAirGapReceiverPort(),
"range");
}

@Test
public void testAirGapPortCandidate() {
doTest(
"iotdb-air-gap-connector",
receiverEnv.getIP() + ":" + receiverEnv.getDataNodeWrapper(0).getPipeAirGapReceiverPort(),
"candidate");
}

private void doTest(final String connector, final String urls, final String strategy) {
try (final SyncConfigNodeIServiceClient client =
(SyncConfigNodeIServiceClient) senderEnv.getLeaderConfigNodeConnection()) {
final Map<String, String> extractorAttributes = new HashMap<>();
final Map<String, String> processorAttributes = new HashMap<>();
final Map<String, String> connectorAttributes = new HashMap<>();
extractorAttributes.put("realtime.mode", "forced-log");
connectorAttributes.put("connector", connector);
connectorAttributes.put("batch.enable", "false");
connectorAttributes.put("node-urls", urls);
connectorAttributes.put("send-port.restriction-strategy", strategy);
if (strategy.equals("range")) {
connectorAttributes.put("send-port.range.min", "1024");
connectorAttributes.put("send-port.range.max", "1524");
} else {
StringBuilder candidateBuilder = new StringBuilder();
for (int i = 0; i < 30; i++) {
candidateBuilder.append(1024 + i).append(",");
}
candidateBuilder.append(1024 + 30);
connectorAttributes.put("send-port.candidate", candidateBuilder.toString());
}

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();
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,11 @@ public IoTDBConfigNodeSyncClientManager(
String trustStorePwd,
String loadBalanceStrategy,
boolean shouldReceiverConvertOnTypeMismatch,
String loadTsFileStrategy) {
String loadTsFileStrategy,
String customSendPortStrategy,
int minSendPortRange,
int maxSendPortRange,
List<Integer> candidatePorts) {
super(
endPoints,
useSSL,
Expand All @@ -49,7 +53,11 @@ public IoTDBConfigNodeSyncClientManager(
false,
loadBalanceStrategy,
shouldReceiverConvertOnTypeMismatch,
loadTsFileStrategy);
loadTsFileStrategy,
customSendPortStrategy,
minSendPortRange,
maxSendPortRange,
candidatePorts);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -65,15 +65,23 @@ protected IoTDBSyncClientManager constructClient(
final boolean useLeaderCache,
final String loadBalanceStrategy,
final boolean shouldReceiverConvertOnTypeMismatch,
final String loadTsFileStrategy) {
final String loadTsFileStrategy,
String customSendPortStrategy,
int minSendPortRange,
int maxSendPortRange,
List<Integer> candidatePorts) {
return new IoTDBConfigNodeSyncClientManager(
nodeUrls,
useSSL,
Objects.nonNull(trustStorePath) ? ConfigNodeConfig.addHomeDir(trustStorePath) : null,
trustStorePwd,
loadBalanceStrategy,
shouldReceiverConvertOnTypeMismatch,
loadTsFileStrategy);
loadTsFileStrategy,
customSendPortStrategy,
minSendPortRange,
maxSendPortRange,
candidatePorts);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -59,16 +60,18 @@ public class IoTDBDataNodeAsyncClientManager extends IoTDBClientManager
private static final Logger LOGGER =
LoggerFactory.getLogger(IoTDBDataNodeAsyncClientManager.class);

private final Set<TEndPoint> endPointSet;
private final Set<AsyncTEndPoint> endPointSet;
private final List<AsyncTEndPoint> tEndPointList;

private static final Map<String, Integer> RECEIVER_ATTRIBUTES_REF_COUNT =
new ConcurrentHashMap<>();
private final String receiverAttributes;

// receiverAttributes -> IClientManager<TEndPoint, AsyncPipeDataTransferServiceClient>
private static final Map<String, IClientManager<TEndPoint, AsyncPipeDataTransferServiceClient>>
// receiverAttributes -> IClientManager<AsyncTEndPoint, AsyncPipeDataTransferServiceClient>
private static final Map<
String, IClientManager<AsyncTEndPoint, AsyncPipeDataTransferServiceClient>>
ASYNC_PIPE_DATA_TRANSFER_CLIENT_MANAGER_HOLDER = new ConcurrentHashMap<>();
private final IClientManager<TEndPoint, AsyncPipeDataTransferServiceClient> endPoint2Client;
private final IClientManager<AsyncTEndPoint, AsyncPipeDataTransferServiceClient> endPoint2Client;

private final LoadBalancer loadBalancer;

Expand All @@ -77,13 +80,13 @@ public class IoTDBDataNodeAsyncClientManager extends IoTDBClientManager
private final String loadTsFileStrategy;

public IoTDBDataNodeAsyncClientManager(
List<TEndPoint> endPoints,
List<AsyncTEndPoint> endPoints,
boolean useLeaderCache,
String loadBalanceStrategy,
boolean shouldReceiverConvertOnTypeMismatch,
String loadTsFileStrategy) {
super(endPoints, useLeaderCache);

super(null, useLeaderCache);
this.tEndPointList = endPoints;
endPointSet = new HashSet<>(endPoints);

receiverAttributes =
Expand All @@ -92,7 +95,7 @@ public IoTDBDataNodeAsyncClientManager(
if (!ASYNC_PIPE_DATA_TRANSFER_CLIENT_MANAGER_HOLDER.containsKey(receiverAttributes)) {
ASYNC_PIPE_DATA_TRANSFER_CLIENT_MANAGER_HOLDER.putIfAbsent(
receiverAttributes,
new IClientManager.Factory<TEndPoint, AsyncPipeDataTransferServiceClient>()
new IClientManager.Factory<AsyncTEndPoint, AsyncPipeDataTransferServiceClient>()
.createClientManager(
new ClientPoolFactory.AsyncPipeDataTransferServiceClientPoolFactory()));
}
Expand Down Expand Up @@ -132,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();
}
Expand Down Expand Up @@ -282,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);
}

Expand All @@ -301,8 +304,9 @@ public void close() {
receiverAttributes,
(attributes, refCount) -> {
if (refCount <= 1) {
final IClientManager<TEndPoint, AsyncPipeDataTransferServiceClient> clientManager =
ASYNC_PIPE_DATA_TRANSFER_CLIENT_MANAGER_HOLDER.remove(receiverAttributes);
final IClientManager<AsyncTEndPoint, AsyncPipeDataTransferServiceClient>
clientManager =
ASYNC_PIPE_DATA_TRANSFER_CLIENT_MANAGER_HOLDER.remove(receiverAttributes);
if (clientManager != null) {
try {
clientManager.close();
Expand All @@ -326,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)) {
Expand All @@ -341,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)) {
Expand All @@ -357,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)) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,7 +50,12 @@ public IoTDBDataNodeSyncClientManager(
final boolean useLeaderCache,
final String loadBalanceStrategy,
final boolean shouldReceiverConvertOnTypeMismatch,
final String loadTsFileStrategy) {
final String loadTsFileStrategy,
final String customSendPortStrategy,
final int minSendPortRange,
final int maxSendPortRange,
final List<Integer> candidatePorts) {

super(
endPoints,
useSSL,
Expand All @@ -59,7 +64,11 @@ public IoTDBDataNodeSyncClientManager(
useLeaderCache,
loadBalanceStrategy,
shouldReceiverConvertOnTypeMismatch,
loadTsFileStrategy);
loadTsFileStrategy,
customSendPortStrategy,
minSendPortRange,
maxSendPortRange,
candidatePorts);
}

@Override
Expand Down
Loading
Loading