Skip to content
Draft
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -0,0 +1,200 @@
/*
* 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.single;

import org.apache.iotdb.commons.client.sync.SyncConfigNodeIServiceClient;
import org.apache.iotdb.commons.cluster.NodeStatus;
import org.apache.iotdb.confignode.rpc.thrift.TDataNodeInfo;
import org.apache.iotdb.confignode.rpc.thrift.TShowDataNodesResp;
import org.apache.iotdb.db.it.utils.TestUtils;
import org.apache.iotdb.isession.SessionConfig;
import org.apache.iotdb.it.env.cluster.node.DataNodeWrapper;
import org.apache.iotdb.it.framework.IoTDBTestRunner;
import org.apache.iotdb.itbase.category.MultiClusterIT1;
import org.apache.iotdb.itbase.env.BaseEnv;
import org.apache.iotdb.rpc.TSStatusCode;

import org.awaitility.Awaitility;
import org.junit.Assert;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.runner.RunWith;

import java.sql.Connection;
import java.sql.ResultSet;
import java.sql.SQLException;
import java.sql.Statement;
import java.util.Arrays;
import java.util.Collections;
import java.util.concurrent.TimeUnit;

@RunWith(IoTDBTestRunner.class)
@Category({MultiClusterIT1.class})
public class IoTDBShowReceiversIT extends AbstractPipeSingleIT {

@Test
public void testShowReceiversInTreeAndTableModel() {
createWriteBackPipe("root.show_receivers", "show_receivers_pipe");

assertShowReceivers("show receivers", BaseEnv.TREE_SQL_DIALECT);
assertShowReceivers("select * from information_schema.receivers", BaseEnv.TABLE_SQL_DIALECT);
}

@Test
public void testShowReceiversWithStoppedDataNode() throws Exception {
Assert.assertTrue(env.getDataNodeWrapperList().size() >= 3);
createWriteBackPipe("root.show_receivers_ha", "show_receivers_ha_pipe");

assertShowReceivers("show receivers", BaseEnv.TREE_SQL_DIALECT, "show_receivers_ha_pipe");
assertShowReceivers(
"select * from information_schema.receivers",
BaseEnv.TABLE_SQL_DIALECT,
"show_receivers_ha_pipe");

final int stoppedDataNodeIndex = 0;
final DataNodeWrapper stoppedDataNode = env.getDataNodeWrapper(stoppedDataNodeIndex);
final int stoppedDataNodeId = getDataNodeId(stoppedDataNode);
final DataNodeWrapper queryDataNode = env.getDataNodeWrapper(1);

env.shutdownDataNode(stoppedDataNodeIndex);
env.ensureNodeStatus(
Collections.singletonList(stoppedDataNode), Collections.singletonList(NodeStatus.Unknown));

assertShowReceiversWithoutDataNode(
"show receivers", BaseEnv.TREE_SQL_DIALECT, queryDataNode, stoppedDataNodeId);
assertShowReceiversWithoutDataNode(
"select * from information_schema.receivers",
BaseEnv.TABLE_SQL_DIALECT,
queryDataNode,
stoppedDataNodeId);
}

private void createWriteBackPipe(final String database, final String pipeName) {
TestUtils.executeNonQueries(
env,
Arrays.asList(
"create database " + database,
"create timeseries " + database + ".d1.s1 with datatype=INT32, encoding=PLAIN",
"create pipe "
+ pipeName
+ " with source ('pattern'='"
+ database
+ "') with sink ('sink'='write-back-sink')",
"insert into " + database + ".d1(time, s1) values (1, 1)",
"flush"),
null);
}

private void assertShowReceivers(final String sql, final String sqlDialect) {
assertShowReceivers(sql, sqlDialect, "show_receivers_pipe");
}

private void assertShowReceivers(
final String sql, final String sqlDialect, final String pipeName) {
Awaitility.await()
.pollInSameThread()
.pollDelay(1L, TimeUnit.SECONDS)
.pollInterval(1L, TimeUnit.SECONDS)
.atMost(60L, TimeUnit.SECONDS)
.untilAsserted(() -> Assert.assertTrue(hasExpectedReceiver(sql, sqlDialect, pipeName)));
}

private boolean hasExpectedReceiver(
final String sql, final String sqlDialect, final String pipeName) throws SQLException {
try (final Connection connection = env.getConnection(sqlDialect);
final Statement statement = connection.createStatement();
final ResultSet resultSet = statement.executeQuery(sql)) {
while (resultSet.next()) {
if ("DataNode".equals(resultSet.getString(1))
&& "thrift".equals(resultSet.getString(3))
&& resultSet.getString(4) != null
&& !resultSet.getString(4).isEmpty()
&& resultSet.getString(5) != null
&& !resultSet.getString(5).isEmpty()
&& resultSet.getInt(6) >= 1
&& resultSet.getInt(7) >= 1
&& resultSet.getString(8).contains(pipeName + "@")
&& "root".equals(resultSet.getString(9))
&& resultSet.getString(10) != null
&& !resultSet.getString(10).isEmpty()
&& resultSet.getString(11) != null
&& resultSet.getString(12) != null) {
return true;
}
}
return false;
}
}

private int getDataNodeId(final DataNodeWrapper targetDataNode) throws Exception {
try (final SyncConfigNodeIServiceClient client =
(SyncConfigNodeIServiceClient) env.getLeaderConfigNodeConnection()) {
final TShowDataNodesResp response = client.showDataNodes();
Assert.assertEquals(
TSStatusCode.SUCCESS_STATUS.getStatusCode(), response.getStatus().getCode());
for (final TDataNodeInfo dataNodeInfo : response.getDataNodesInfoList()) {
if (targetDataNode.getIp().equals(dataNodeInfo.getRpcAddresss())
&& targetDataNode.getPort() == dataNodeInfo.getRpcPort()) {
return dataNodeInfo.getDataNodeId();
}
}
}
throw new AssertionError("Cannot find DataNodeId for " + targetDataNode.getIpAndPortString());
}

private void assertShowReceiversWithoutDataNode(
final String sql,
final String sqlDialect,
final DataNodeWrapper queryDataNode,
final int excludedDataNodeId) {
Awaitility.await()
.pollInSameThread()
.pollDelay(1L, TimeUnit.SECONDS)
.pollInterval(1L, TimeUnit.SECONDS)
.atMost(60L, TimeUnit.SECONDS)
.untilAsserted(
() ->
assertQueryResultDoesNotContainDataNode(
sql, sqlDialect, queryDataNode, excludedDataNodeId));
}

private void assertQueryResultDoesNotContainDataNode(
final String sql,
final String sqlDialect,
final DataNodeWrapper queryDataNode,
final int excludedDataNodeId)
throws SQLException {
try (final Connection connection =
env.getConnection(
queryDataNode,
SessionConfig.DEFAULT_USER,
SessionConfig.DEFAULT_PASSWORD,
sqlDialect);
final Statement statement = connection.createStatement();
final ResultSet resultSet = statement.executeQuery(sql)) {
while (resultSet.next()) {
final int receiverDataNodeId = resultSet.getInt(2);
if (!resultSet.wasNull()) {
Assert.assertNotEquals(excludedDataNodeId, receiverDataNodeId);
}
}
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -196,6 +196,7 @@ keyWords
| QUERY
| QUERYID
| QUOTA
| RECEIVERS
| RANGE
| READONLY
| READ
Expand Down Expand Up @@ -298,4 +299,4 @@ keyWords
| OPTION
| INF
| CURRENT_TIMESTAMP
;
;
Original file line number Diff line number Diff line change
Expand Up @@ -55,7 +55,7 @@ ddlStatement
// ExternalService
| createService | startService | stopService | dropService | showService
// Pipe Task
| createPipe | alterPipe | dropPipe | startPipe | stopPipe | showPipes
| createPipe | alterPipe | dropPipe | startPipe | stopPipe | showPipes | showReceivers
// Pipe Plugin
| createPipePlugin | dropPipePlugin | showPipePlugins
// Subscription
Expand Down Expand Up @@ -701,6 +701,10 @@ showPipes
: SHOW ((PIPE pipeName=identifier) | PIPES (WHERE (CONNECTOR | SINK) USED BY pipeName=identifier)?)
;

showReceivers
: SHOW RECEIVERS
;

// Pipe Plugin =========================================================================================
createPipePlugin
: CREATE PIPEPLUGIN (IF NOT EXISTS)? pluginName=identifier AS className=STRING_LITERAL uriClause
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -723,6 +723,10 @@ QUOTA
: Q U O T A
;

RECEIVERS
: R E C E I V E R S
;

RANGE
: R A N G E
;
Expand Down Expand Up @@ -1411,4 +1415,4 @@ fragment V: [vV];
fragment W: [wW];
fragment X: [xX];
fragment Y: [yY];
fragment Z: [zZ];
fragment Z: [zZ];
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@
import org.apache.iotdb.commons.pipe.datastructure.pattern.TreePattern;
import org.apache.iotdb.commons.pipe.receiver.IoTDBFileReceiver;
import org.apache.iotdb.commons.pipe.receiver.PipeReceiverStatusHandler;
import org.apache.iotdb.commons.pipe.receiver.runtime.PipeReceiverRuntimeRegistry;
import org.apache.iotdb.commons.pipe.sink.payload.airgap.AirGapPseudoTPipeTransferRequest;
import org.apache.iotdb.commons.pipe.sink.payload.thrift.request.PipeRequestType;
import org.apache.iotdb.commons.pipe.sink.payload.thrift.request.PipeTransferCompressedReq;
Expand Down Expand Up @@ -194,20 +195,22 @@ public TPipeTransferResp receive(final TPipeTransferReq req) {
PipeTransferConfigNodeHandshakeV1Req.fromTPipeTransferReq(req));
PipeConfigNodeReceiverMetrics.getInstance()
.recordHandshakeConfigNodeV1Timer(System.nanoTime() - startTime);
return resp;
return recordConfigNodeHandshakeIfSuccess(resp, req);
case HANDSHAKE_CONFIGNODE_V2:
resp =
handleTransferHandshakeV2(
PipeTransferConfigNodeHandshakeV2Req.fromTPipeTransferReq(req));
userEntity.setAuditLogOperation(AuditLogOperation.DDL);
if (Objects.nonNull(userEntity)) {
userEntity.setAuditLogOperation(AuditLogOperation.DDL);
}
PipeConfigNodeReceiverMetrics.getInstance()
.recordHandshakeConfigNodeV2Timer(System.nanoTime() - startTime);
return resp;
return recordConfigNodeHandshakeIfSuccess(resp, req);
case TRANSFER_CONFIG_PLAN:
resp = handleTransferConfigPlan(PipeTransferConfigPlanReq.fromTPipeTransferReq(req));
PipeConfigNodeReceiverMetrics.getInstance()
.recordTransferConfigPlanTimer(System.nanoTime() - startTime);
return resp;
return recordConfigNodeTransferIfSuccess(resp);
case TRANSFER_CONFIG_SNAPSHOT_PIECE:
resp =
handleTransferFilePiece(
Expand All @@ -216,14 +219,14 @@ public TPipeTransferResp receive(final TPipeTransferReq req) {
false);
PipeConfigNodeReceiverMetrics.getInstance()
.recordTransferConfigSnapshotPieceTimer(System.nanoTime() - startTime);
return resp;
return recordConfigNodeTransferIfSuccess(resp);
case TRANSFER_CONFIG_SNAPSHOT_SEAL:
resp =
handleTransferFileSealV2(
PipeTransferConfigSnapshotSealReq.fromTPipeTransferReq(req));
PipeConfigNodeReceiverMetrics.getInstance()
.recordTransferConfigSnapshotSealTimer(System.nanoTime() - startTime);
return resp;
return recordConfigNodeTransferIfSuccess(resp);
case TRANSFER_COMPRESSED:
return receive(PipeTransferCompressedReq.fromTPipeTransferReq(req));
default:
Expand Down Expand Up @@ -262,6 +265,30 @@ private boolean needHandshake(final PipeRequestType type) {
&& type != PipeRequestType.HANDSHAKE_CONFIGNODE_V2;
}

private TPipeTransferResp recordConfigNodeHandshakeIfSuccess(
final TPipeTransferResp resp, final TPipeTransferReq req) {
if (isSuccess(resp)) {
recordPipeReceiverHandshake(
PipeReceiverRuntimeRegistry.NODE_TYPE_CONFIG_NODE,
ConfigNodeDescriptor.getInstance().getConf().getConfigNodeId(),
getProtocol(req));
}
return resp;
}

private TPipeTransferResp recordConfigNodeTransferIfSuccess(final TPipeTransferResp resp) {
if (isSuccess(resp)) {
recordPipeReceiverTransfer();
}
return resp;
}

private static String getProtocol(final TPipeTransferReq req) {
return req instanceof AirGapPseudoTPipeTransferRequest
? PipeReceiverRuntimeRegistry.PROTOCOL_AIR_GAP
: PipeReceiverRuntimeRegistry.PROTOCOL_THRIFT;
}

private TPipeTransferResp handleTransferConfigPlan(final PipeTransferConfigPlanReq req)
throws IOException {
return new TPipeTransferResp(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -89,6 +89,7 @@ protected byte[] generateHandShakeV2Payload() throws IOException {
Boolean.toString(shouldMarkAsPipeRequest));
params.put(
PipeTransferHandshakeConstant.HANDSHAKE_KEY_SKIP_IF, Boolean.toString(skipIfNoPrivileges));
appendPipeInfoToHandshakeParams(params);

return PipeTransferConfigNodeHandshakeV2Req.toTPipeTransferBytes(params);
}
Expand Down
Loading