From cfe19176edbee1712dd9a8c839e81916adce7e72 Mon Sep 17 00:00:00 2001 From: shuwenwei <55970239+shuwenwei@users.noreply.github.com> Date: Wed, 15 May 2024 16:36:06 +0800 Subject: [PATCH 01/42] Fix compaction write rate can not hot load --- .../src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java index 27bf490e4664..82e2e09fbfa0 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java @@ -1171,7 +1171,7 @@ private void loadCompactionHotModifiedProps(Properties properties) throws Interr conf.setCompactionWriteThroughputMbPerSec( Integer.parseInt( properties.getProperty( - "merge_write_throughput_mb_per_sec", + "compaction_write_throughput_mb_per_sec", Integer.toString(conf.getCompactionWriteThroughputMbPerSec())))); // update compaction_read_operation_per_sec From e3aa4d8954c75ce231d887d41f57bbd942b40bb3 Mon Sep 17 00:00:00 2001 From: Zikun Ma <55695098+DanielWang2035@users.noreply.github.com> Date: Wed, 15 May 2024 18:28:07 +0800 Subject: [PATCH 02/42] Pipe: RPC compression for data sync connectors (SNAPPY, GZIP, LZ4, ZSTD and LZMA2) (#12519) Co-authored-by: Steve Yurong Su --- .../IoTDBPipeConnectorCompressionIT.java | 182 ++++++++++++++++++ .../parameter/PipeParameterValidator.java | 4 +- .../IoTDBConfigRegionAirGapConnector.java | 27 +-- .../protocol/IoTDBConfigRegionConnector.java | 20 +- .../protocol/IoTDBConfigNodeReceiver.java | 3 + .../request/PipeTransferPlanNodeReq.java | 1 + .../airgap/IoTDBDataNodeAirGapConnector.java | 12 +- .../IoTDBDataRegionAirGapConnector.java | 24 ++- .../IoTDBSchemaRegionAirGapConnector.java | 15 +- .../async/IoTDBDataRegionAsyncConnector.java | 18 +- .../PipeTransferTabletBatchEventHandler.java | 7 +- ...peTransferTsFileInsertionEventHandler.java | 19 +- .../sync/IoTDBDataNodeSyncConnector.java | 5 +- .../sync/IoTDBDataRegionSyncConnector.java | 34 ++-- .../sync/IoTDBSchemaRegionConnector.java | 15 +- .../thrift/IoTDBDataNodeReceiver.java | 3 + .../constant/PipeConnectorConstant.java | 18 ++ .../connector/compressor/PipeCompressor.java | 57 ++++++ .../compressor/PipeCompressorFactory.java | 80 ++++++++ .../compressor/PipeGZIPCompressor.java | 47 +++++ .../compressor/PipeLZ4Compressor.java | 47 +++++ .../compressor/PipeLZMA2Compressor.java | 47 +++++ .../compressor/PipeSnappyCompressor.java | 47 +++++ .../compressor/PipeZSTDCompressor.java | 47 +++++ .../thrift/request/PipeRequestType.java | 3 + .../request/PipeTransferCompressedReq.java | 146 ++++++++++++++ .../protocol/IoTDBAirGapConnector.java | 7 +- .../connector/protocol/IoTDBConnector.java | 62 ++++++ .../protocol/IoTDBSslSyncConnector.java | 8 +- 29 files changed, 921 insertions(+), 84 deletions(-) create mode 100644 integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeConnectorCompressionIT.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/compressor/PipeCompressor.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/compressor/PipeCompressorFactory.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/compressor/PipeGZIPCompressor.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/compressor/PipeLZ4Compressor.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/compressor/PipeLZMA2Compressor.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/compressor/PipeSnappyCompressor.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/compressor/PipeZSTDCompressor.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/payload/thrift/request/PipeTransferCompressedReq.java diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeConnectorCompressionIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeConnectorCompressionIT.java new file mode 100644 index 000000000000..e5f76a00eb00 --- /dev/null +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeConnectorCompressionIT.java @@ -0,0 +1,182 @@ +/* + * 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.env.cluster.node.DataNodeWrapper; +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 IoTDBPipeConnectorCompressionIT 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().getConfigNodeConfig().setConnectionTimeoutMs(600000); + receiverEnv.getConfig().getConfigNodeConfig().setConnectionTimeoutMs(600000); + + senderEnv.initClusterEnvironment(); + receiverEnv.initClusterEnvironment(); + } + + @Test + public void testCompression1() throws Exception { + doTest("iotdb-thrift-connector", "stream", true, "snappy"); + } + + @Test + public void testCompression2() throws Exception { + doTest("iotdb-thrift-connector", "batch", true, "snappy, lzma2"); + } + + @Test + public void testCompression3() throws Exception { + doTest("iotdb-thrift-sync-connector", "stream", false, "snappy, snappy"); + } + + @Test + public void testCompression4() throws Exception { + doTest("iotdb-thrift-sync-connector", "batch", true, "gzip, zstd"); + } + + @Test + public void testCompression5() throws Exception { + doTest("iotdb-air-gap-connector", "stream", false, "lzma2, lz4"); + } + + @Test + public void testCompression6() throws Exception { + doTest("iotdb-air-gap-connector", "batch", true, "lzma2"); + } + + private void doTest( + String connectorType, String realtimeMode, boolean useBatchMode, String compressionTypes) + throws Exception { + final DataNodeWrapper receiverDataNode = receiverEnv.getDataNodeWrapper(0); + + final String receiverIp = receiverDataNode.getIp(); + final int receiverPort = + connectorType.contains("air-gap") + ? receiverDataNode.getPipeAirGapReceiverPort() + : receiverDataNode.getPort(); + + 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("extractor", "iotdb-extractor"); + extractorAttributes.put("extractor.realtime.mode", realtimeMode); + + processorAttributes.put("processor", "do-nothing-processor"); + + connectorAttributes.put("connector", connectorType); + connectorAttributes.put("connector.batch.enable", useBatchMode ? "true" : "false"); + connectorAttributes.put("connector.ip", receiverIp); + connectorAttributes.put("connector.port", Integer.toString(receiverPort)); + connectorAttributes.put("connector.user", "root"); + connectorAttributes.put("connector.password", "root"); + connectorAttributes.put("connector.compressor", compressionTypes); + + final TSStatus status = + client.createPipe( + new TCreatePipeReq("p1", connectorAttributes) + .setExtractorAttributes(extractorAttributes) + .setProcessorAttributes(processorAttributes)); + + Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode()); + Assert.assertEquals( + TSStatusCode.SUCCESS_STATUS.getStatusCode(), client.startPipe("p1").getCode()); + + TestUtils.assertDataEventuallyOnEnv( + receiverEnv, + "select count(*) from root.**", + "count(root.db.d1.s1),", + Collections.singleton("2,")); + + if (!TestUtils.tryExecuteNonQueriesWithRetry( + senderEnv, + Arrays.asList( + "insert into root.db.d1(time, s1) values (now(), 3)", + "insert into root.db.d1(time, s1) values (now(), 4)", + "insert into root.db.d1(time, s1) values (now(), 5)", + "insert into root.db.d1(time, s1) values (now(), 6)", + "insert into root.db.d1(time, s1) values (now(), 7)", + "insert into root.db.d1(time, s1) values (now(), 8)", + "flush"))) { + return; + } + + TestUtils.assertDataEventuallyOnEnv( + receiverEnv, + "select count(*) from root.**", + "count(root.db.d1.s1),", + Collections.singleton("8,")); + } + } +} diff --git a/iotdb-api/pipe-api/src/main/java/org/apache/iotdb/pipe/api/customizer/parameter/PipeParameterValidator.java b/iotdb-api/pipe-api/src/main/java/org/apache/iotdb/pipe/api/customizer/parameter/PipeParameterValidator.java index 01e9507f7fe5..1fa0046ccdab 100644 --- a/iotdb-api/pipe-api/src/main/java/org/apache/iotdb/pipe/api/customizer/parameter/PipeParameterValidator.java +++ b/iotdb-api/pipe-api/src/main/java/org/apache/iotdb/pipe/api/customizer/parameter/PipeParameterValidator.java @@ -69,7 +69,9 @@ public PipeParameterValidator validateAttributeValueRange( } throw new PipeParameterNotValidException( - String.format("The value of %s should be one of %s", key, Arrays.toString(optionalValues))); + String.format( + "Invalid value %s of %s. The value should be one of %s", + actualValue, key, Arrays.toString(optionalValues))); } /** diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/connector/protocol/IoTDBConfigRegionAirGapConnector.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/connector/protocol/IoTDBConfigRegionAirGapConnector.java index 3235043fb45b..3ca8031d2681 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/connector/protocol/IoTDBConfigRegionAirGapConnector.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/connector/protocol/IoTDBConfigRegionAirGapConnector.java @@ -56,8 +56,9 @@ public class IoTDBConfigRegionAirGapConnector extends IoTDBAirGapConnector { @Override protected byte[] generateHandShakeV1Payload() throws IOException { - return PipeTransferConfigNodeHandshakeV1Req.toTPipeTransferBytes( - CommonDescriptor.getInstance().getConfig().getTimestampPrecision()); + return compressIfNeeded( + PipeTransferConfigNodeHandshakeV1Req.toTPipeTransferBytes( + CommonDescriptor.getInstance().getConfig().getTimestampPrecision())); } @Override @@ -70,7 +71,7 @@ protected byte[] generateHandShakeV2Payload() throws IOException { PipeTransferHandshakeConstant.HANDSHAKE_KEY_TIME_PRECISION, CommonDescriptor.getInstance().getConfig().getTimestampPrecision()); - return PipeTransferConfigNodeHandshakeV2Req.toTPipeTransferBytes(params); + return compressIfNeeded(PipeTransferConfigNodeHandshakeV2Req.toTPipeTransferBytes(params)); } @Override @@ -150,8 +151,9 @@ private void doTransfer( throws PipeException, IOException { if (!send( socket, - PipeTransferConfigPlanReq.toTPipeTransferBytes( - pipeConfigRegionWritePlanEvent.getConfigPhysicalPlan()))) { + compressIfNeeded( + PipeTransferConfigPlanReq.toTPipeTransferBytes( + pipeConfigRegionWritePlanEvent.getConfigPhysicalPlan())))) { final String errorMessage = String.format( "Transfer config region write plan %s error. Socket: %s.", @@ -197,13 +199,14 @@ private void doTransfer( // 2. Transfer file seal signal, which means the snapshots are transferred completely if (!send( socket, - PipeTransferConfigSnapshotSealReq.toTPipeTransferBytes( - snapshot.getName(), - snapshot.length(), - Objects.nonNull(templateFile) ? templateFile.getName() : null, - Objects.nonNull(templateFile) ? templateFile.length() : 0, - pipeConfigRegionSnapshotEvent.getFileType(), - pipeConfigRegionSnapshotEvent.toSealTypeString()))) { + compressIfNeeded( + PipeTransferConfigSnapshotSealReq.toTPipeTransferBytes( + snapshot.getName(), + snapshot.length(), + Objects.nonNull(templateFile) ? templateFile.getName() : null, + Objects.nonNull(templateFile) ? templateFile.length() : 0, + pipeConfigRegionSnapshotEvent.getFileType(), + pipeConfigRegionSnapshotEvent.toSealTypeString())))) { final String errorMessage = String.format("Seal config region snapshot %s error. Socket %s.", snapshot, socket); // Send handshake because we don't know whether the receiver side configNode 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 c48e29cd72de..a2962c194f33 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 @@ -127,8 +127,9 @@ private void doTransfer(final PipeConfigRegionWritePlanEvent pipeConfigRegionWri clientAndStatus .getLeft() .pipeTransfer( - PipeTransferConfigPlanReq.toTPipeTransferReq( - pipeConfigRegionWritePlanEvent.getConfigPhysicalPlan())); + compressIfNeeded( + PipeTransferConfigPlanReq.toTPipeTransferReq( + pipeConfigRegionWritePlanEvent.getConfigPhysicalPlan()))); } catch (final Exception e) { clientAndStatus.setRight(false); throw new PipeConnectionException( @@ -192,13 +193,14 @@ private void doTransfer(final PipeConfigRegionSnapshotEvent snapshotEvent) clientAndStatus .getLeft() .pipeTransfer( - PipeTransferConfigSnapshotSealReq.toTPipeTransferReq( - snapshotFile.getName(), - snapshotFile.length(), - Objects.nonNull(templateFile) ? templateFile.getName() : null, - Objects.nonNull(templateFile) ? templateFile.length() : 0, - snapshotEvent.getFileType(), - snapshotEvent.toSealTypeString())); + compressIfNeeded( + PipeTransferConfigSnapshotSealReq.toTPipeTransferReq( + snapshotFile.getName(), + snapshotFile.length(), + Objects.nonNull(templateFile) ? templateFile.getName() : null, + Objects.nonNull(templateFile) ? templateFile.length() : 0, + snapshotEvent.getFileType(), + snapshotEvent.toSealTypeString()))); } catch (final Exception e) { clientAndStatus.setRight(false); throw new PipeConnectionException( diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/receiver/protocol/IoTDBConfigNodeReceiver.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/receiver/protocol/IoTDBConfigNodeReceiver.java index 502d4b50f732..0a78553516a9 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/receiver/protocol/IoTDBConfigNodeReceiver.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/receiver/protocol/IoTDBConfigNodeReceiver.java @@ -24,6 +24,7 @@ import org.apache.iotdb.commons.pipe.connector.PipeReceiverStatusHandler; import org.apache.iotdb.commons.pipe.connector.payload.airgap.AirGapPseudoTPipeTransferRequest; import org.apache.iotdb.commons.pipe.connector.payload.thrift.request.PipeRequestType; +import org.apache.iotdb.commons.pipe.connector.payload.thrift.request.PipeTransferCompressedReq; import org.apache.iotdb.commons.pipe.connector.payload.thrift.request.PipeTransferFileSealReqV1; import org.apache.iotdb.commons.pipe.connector.payload.thrift.request.PipeTransferFileSealReqV2; import org.apache.iotdb.commons.pipe.receiver.IoTDBFileReceiver; @@ -124,6 +125,8 @@ public TPipeTransferResp receive(final TPipeTransferReq req) { case TRANSFER_CONFIG_SNAPSHOT_SEAL: return handleTransferFileSealV2( PipeTransferConfigSnapshotSealReq.fromTPipeTransferReq(req)); + case TRANSFER_COMPRESSED: + return receive(PipeTransferCompressedReq.fromTPipeTransferReq(req)); default: break; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferPlanNodeReq.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferPlanNodeReq.java index 9209473c3a21..580dbe2ed4ca 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferPlanNodeReq.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferPlanNodeReq.java @@ -72,6 +72,7 @@ public static PipeTransferPlanNodeReq fromTPipeTransferReq(TPipeTransferReq tran } /////////////////////////////// Air Gap /////////////////////////////// + public static byte[] toTPipeTransferBytes(PlanNode planNode) throws IOException { try (final PublicBAOS byteArrayOutputStream = new PublicBAOS(); final DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream)) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/airgap/IoTDBDataNodeAirGapConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/airgap/IoTDBDataNodeAirGapConnector.java index 09ecfce07be8..01c7de76da23 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/airgap/IoTDBDataNodeAirGapConnector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/airgap/IoTDBDataNodeAirGapConnector.java @@ -88,8 +88,9 @@ protected boolean mayNeedHandshakeWhenFail() { @Override protected byte[] generateHandShakeV1Payload() throws IOException { - return PipeTransferDataNodeHandshakeV1Req.toTPipeTransferBytes( - CommonDescriptor.getInstance().getConfig().getTimestampPrecision()); + return compressIfNeeded( + PipeTransferDataNodeHandshakeV1Req.toTPipeTransferBytes( + CommonDescriptor.getInstance().getConfig().getTimestampPrecision())); } @Override @@ -102,7 +103,7 @@ protected byte[] generateHandShakeV2Payload() throws IOException { PipeTransferHandshakeConstant.HANDSHAKE_KEY_TIME_PRECISION, CommonDescriptor.getInstance().getConfig().getTimestampPrecision()); - return PipeTransferDataNodeHandshakeV2Req.toTPipeTransferBytes(params); + return compressIfNeeded(PipeTransferDataNodeHandshakeV2Req.toTPipeTransferBytes(params)); } protected void doTransferWrapper( @@ -126,8 +127,9 @@ private void doTransfer( throws PipeException, IOException { if (!send( socket, - PipeTransferPlanNodeReq.toTPipeTransferBytes( - pipeSchemaRegionWritePlanEvent.getPlanNode()))) { + compressIfNeeded( + PipeTransferPlanNodeReq.toTPipeTransferBytes( + pipeSchemaRegionWritePlanEvent.getPlanNode())))) { final String errorMessage = String.format( "Transfer data node write plan %s error. Socket: %s.", diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/airgap/IoTDBDataRegionAirGapConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/airgap/IoTDBDataRegionAirGapConnector.java index 0548e4da48f0..84ca0b0ccdb8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/airgap/IoTDBDataRegionAirGapConnector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/airgap/IoTDBDataRegionAirGapConnector.java @@ -170,10 +170,11 @@ private void doTransfer( final InsertNode insertNode = pipeInsertNodeTabletInsertionEvent.getInsertNodeViaCacheIfPossible(); final byte[] bytes = - Objects.isNull(insertNode) - ? PipeTransferTabletBinaryReq.toTPipeTransferBytes( - pipeInsertNodeTabletInsertionEvent.getByteBuffer()) - : PipeTransferTabletInsertNodeReq.toTPipeTransferBytes(insertNode); + compressIfNeeded( + Objects.isNull(insertNode) + ? PipeTransferTabletBinaryReq.toTPipeTransferBytes( + pipeInsertNodeTabletInsertionEvent.getByteBuffer()) + : PipeTransferTabletInsertNodeReq.toTPipeTransferBytes(insertNode)); if (!send(socket, bytes)) { final String errorMessage = @@ -209,9 +210,10 @@ private void doTransfer( throws PipeException, IOException { if (!send( socket, - PipeTransferTabletRawReq.toTPipeTransferBytes( - pipeRawTabletInsertionEvent.convertToTablet(), - pipeRawTabletInsertionEvent.isAligned()))) { + compressIfNeeded( + PipeTransferTabletRawReq.toTPipeTransferBytes( + pipeRawTabletInsertionEvent.convertToTablet(), + pipeRawTabletInsertionEvent.isAligned())))) { final String errorMessage = String.format( "Transfer PipeRawTabletInsertionEvent %s error. Socket: %s.", @@ -254,8 +256,9 @@ private void doTransfer( // 2. Transfer file seal signal with mod, which means the file is transferred completely if (!send( socket, - PipeTransferTsFileSealWithModReq.toTPipeTransferBytes( - modFile.getName(), modFile.length(), tsFile.getName(), tsFile.length()))) { + compressIfNeeded( + PipeTransferTsFileSealWithModReq.toTPipeTransferBytes( + modFile.getName(), modFile.length(), tsFile.getName(), tsFile.length())))) { receiverStatusHandler.handle( new TSStatus(TSStatusCode.PIPE_RECEIVER_USER_CONFLICT_EXCEPTION.getStatusCode()) .setMessage(errorMessage), @@ -269,7 +272,8 @@ private void doTransfer( // 2. Transfer file seal signal without mod, which means the file is transferred completely if (!send( socket, - PipeTransferTsFileSealReq.toTPipeTransferBytes(tsFile.getName(), tsFile.length()))) { + compressIfNeeded( + PipeTransferTsFileSealReq.toTPipeTransferBytes(tsFile.getName(), tsFile.length())))) { receiverStatusHandler.handle( new TSStatus(TSStatusCode.PIPE_RECEIVER_USER_CONFLICT_EXCEPTION.getStatusCode()) .setMessage(errorMessage), diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/airgap/IoTDBSchemaRegionAirGapConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/airgap/IoTDBSchemaRegionAirGapConnector.java index 74375cff1120..d4e6706a5a4e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/airgap/IoTDBSchemaRegionAirGapConnector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/airgap/IoTDBSchemaRegionAirGapConnector.java @@ -114,13 +114,14 @@ private void doTransfer( // 2. Transfer file seal signal, which means the snapshots is transferred completely if (!send( socket, - PipeTransferSchemaSnapshotSealReq.toTPipeTransferBytes( - mtreeSnapshotFile.getName(), - mtreeSnapshotFile.length(), - Objects.nonNull(tagLogSnapshotFile) ? tagLogSnapshotFile.getName() : null, - Objects.nonNull(tagLogSnapshotFile) ? tagLogSnapshotFile.length() : 0, - pipeSchemaRegionSnapshotEvent.getDatabaseName(), - pipeSchemaRegionSnapshotEvent.toSealTypeString()))) { + compressIfNeeded( + PipeTransferSchemaSnapshotSealReq.toTPipeTransferBytes( + mtreeSnapshotFile.getName(), + mtreeSnapshotFile.length(), + Objects.nonNull(tagLogSnapshotFile) ? tagLogSnapshotFile.getName() : null, + Objects.nonNull(tagLogSnapshotFile) ? tagLogSnapshotFile.length() : 0, + pipeSchemaRegionSnapshotEvent.getDatabaseName(), + pipeSchemaRegionSnapshotEvent.toSealTypeString())))) { final String errorMessage = String.format( "Seal schema region snapshot file %s and %s error. Socket %s.", 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 74cbfe2e39b6..c9f076b0f047 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 @@ -185,10 +185,11 @@ private void transferWithoutCheck(final TabletInsertionEvent tabletInsertionEven final InsertNode insertNode = pipeInsertNodeTabletInsertionEvent.getInsertNodeViaCacheIfPossible(); final TPipeTransferReq pipeTransferReq = - Objects.isNull(insertNode) - ? PipeTransferTabletBinaryReq.toTPipeTransferReq( - pipeInsertNodeTabletInsertionEvent.getByteBuffer()) - : PipeTransferTabletInsertNodeReq.toTPipeTransferReq(insertNode); + compressIfNeeded( + Objects.isNull(insertNode) + ? PipeTransferTabletBinaryReq.toTPipeTransferReq( + pipeInsertNodeTabletInsertionEvent.getByteBuffer()) + : PipeTransferTabletInsertNodeReq.toTPipeTransferReq(insertNode)); final PipeTransferTabletInsertNodeEventHandler pipeTransferInsertNodeReqHandler = new PipeTransferTabletInsertNodeEventHandler( pipeInsertNodeTabletInsertionEvent, pipeTransferReq, this); @@ -207,10 +208,11 @@ private void transferWithoutCheck(final TabletInsertionEvent tabletInsertionEven return; } - final PipeTransferTabletRawReq pipeTransferTabletRawReq = - PipeTransferTabletRawReq.toTPipeTransferReq( - pipeRawTabletInsertionEvent.convertToTablet(), - pipeRawTabletInsertionEvent.isAligned()); + final TPipeTransferReq pipeTransferTabletRawReq = + compressIfNeeded( + PipeTransferTabletRawReq.toTPipeTransferReq( + pipeRawTabletInsertionEvent.convertToTablet(), + pipeRawTabletInsertionEvent.isAligned())); final PipeTransferTabletRawEventHandler pipeTransferTabletReqHandler = new PipeTransferTabletRawEventHandler( pipeRawTabletInsertionEvent, pipeTransferTabletRawReq, this); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/handler/PipeTransferTabletBatchEventHandler.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/handler/PipeTransferTabletBatchEventHandler.java index 500ed4cab981..88e413439507 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/handler/PipeTransferTabletBatchEventHandler.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/handler/PipeTransferTabletBatchEventHandler.java @@ -22,6 +22,7 @@ import org.apache.iotdb.common.rpc.thrift.TEndPoint; import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.commons.client.async.AsyncPipeDataTransferServiceClient; +import org.apache.iotdb.commons.pipe.connector.payload.thrift.request.PipeTransferCompressedReq; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.db.pipe.connector.payload.evolvable.builder.PipeEventBatch; import org.apache.iotdb.db.pipe.connector.protocol.thrift.async.IoTDBDataRegionAsyncConnector; @@ -59,7 +60,11 @@ public PipeTransferTabletBatchEventHandler( // Deep copy to keep Ids' and events' reference requestCommitIds = batch.deepCopyRequestCommitIds(); events = batch.deepCopyEvents(); - req = batch.toTPipeTransferReq(); + req = + connector.isRpcCompressionEnabled() + ? batch.toTPipeTransferReq() + : PipeTransferCompressedReq.toTPipeTransferReq( + batch.toTPipeTransferReq(), connector.getCompressors()); this.connector = connector; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/handler/PipeTransferTsFileInsertionEventHandler.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/handler/PipeTransferTsFileInsertionEventHandler.java index aebff325920b..07669de76bfe 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/handler/PipeTransferTsFileInsertionEventHandler.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/handler/PipeTransferTsFileInsertionEventHandler.java @@ -22,6 +22,7 @@ import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.commons.client.async.AsyncPipeDataTransferServiceClient; import org.apache.iotdb.commons.pipe.config.PipeConfig; +import org.apache.iotdb.commons.pipe.connector.payload.thrift.request.PipeTransferCompressedReq; import org.apache.iotdb.commons.pipe.connector.payload.thrift.response.PipeTransferFilePieceResp; import org.apache.iotdb.db.pipe.connector.client.IoTDBDataNodeAsyncClientManager; import org.apache.iotdb.db.pipe.connector.payload.evolvable.request.PipeTransferTsFilePieceReq; @@ -31,6 +32,7 @@ import org.apache.iotdb.db.pipe.connector.protocol.thrift.async.IoTDBDataRegionAsyncConnector; import org.apache.iotdb.db.pipe.event.common.tsfile.PipeTsFileInsertionEvent; import org.apache.iotdb.rpc.TSStatusCode; +import org.apache.iotdb.service.rpc.thrift.TPipeTransferReq; import org.apache.iotdb.service.rpc.thrift.TPipeTransferResp; import org.apache.thrift.TException; @@ -134,12 +136,19 @@ public void transfer( readLength == readFileBufferSize ? readBuffer : Arrays.copyOfRange(readBuffer, 0, readLength); + final TPipeTransferReq uncompressedReq = + PipeTransferCompressedReq.toTPipeTransferReq( + transferMod + ? PipeTransferTsFilePieceWithModReq.toTPipeTransferReq( + currentFile.getName(), position, payload) + : PipeTransferTsFilePieceReq.toTPipeTransferReq( + currentFile.getName(), position, payload), + connector.getCompressors()); client.pipeTransfer( - transferMod - ? PipeTransferTsFilePieceWithModReq.toTPipeTransferReq( - currentFile.getName(), position, payload) - : PipeTransferTsFilePieceReq.toTPipeTransferReq( - currentFile.getName(), position, payload), + connector.isRpcCompressionEnabled() + ? PipeTransferCompressedReq.toTPipeTransferReq( + uncompressedReq, connector.getCompressors()) + : uncompressedReq, this); position += readLength; } 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 378e6f742818..33c97ce27f29 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 @@ -117,8 +117,9 @@ protected void doTransfer(final PipeSchemaRegionWritePlanEvent pipeSchemaRegionW clientAndStatus .getLeft() .pipeTransfer( - PipeTransferPlanNodeReq.toTPipeTransferReq( - pipeSchemaRegionWritePlanEvent.getPlanNode())); + compressIfNeeded( + PipeTransferPlanNodeReq.toTPipeTransferReq( + pipeSchemaRegionWritePlanEvent.getPlanNode()))); } catch (final Exception e) { clientAndStatus.setRight(false); throw new PipeConnectionException( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBDataRegionSyncConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBDataRegionSyncConnector.java index 568baa13144f..cb93bffd4111 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBDataRegionSyncConnector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBDataRegionSyncConnector.java @@ -176,7 +176,10 @@ private void doTransfer(Pair endPointAndBatch) { final TPipeTransferResp resp; try { - resp = clientAndStatus.getLeft().pipeTransfer(batchToTransfer.toTPipeTransferReq()); + resp = + clientAndStatus + .getLeft() + .pipeTransfer(compressIfNeeded(batchToTransfer.toTPipeTransferReq())); } catch (final Exception e) { clientAndStatus.setRight(false); throw new PipeConnectionException( @@ -239,15 +242,18 @@ private void doTransfer( resp = clientAndStatus .getLeft() - .pipeTransfer(PipeTransferTabletInsertNodeReq.toTPipeTransferReq(insertNode)); + .pipeTransfer( + compressIfNeeded( + PipeTransferTabletInsertNodeReq.toTPipeTransferReq(insertNode))); } else { clientAndStatus = clientManager.getClient(); resp = clientAndStatus .getLeft() .pipeTransfer( - PipeTransferTabletBinaryReq.toTPipeTransferReq( - pipeInsertNodeTabletInsertionEvent.getByteBuffer())); + compressIfNeeded( + PipeTransferTabletBinaryReq.toTPipeTransferReq( + pipeInsertNodeTabletInsertionEvent.getByteBuffer()))); } } catch (final Exception e) { if (clientAndStatus != null) { @@ -303,9 +309,10 @@ private void doTransfer(final PipeRawTabletInsertionEvent pipeRawTabletInsertion clientAndStatus .getLeft() .pipeTransfer( - PipeTransferTabletRawReq.toTPipeTransferReq( - pipeRawTabletInsertionEvent.convertToTablet(), - pipeRawTabletInsertionEvent.isAligned())); + compressIfNeeded( + PipeTransferTabletRawReq.toTPipeTransferReq( + pipeRawTabletInsertionEvent.convertToTablet(), + pipeRawTabletInsertionEvent.isAligned()))); } catch (final Exception e) { clientAndStatus.setRight(false); throw new PipeConnectionException( @@ -364,8 +371,12 @@ private void doTransfer(final PipeTsFileInsertionEvent pipeTsFileInsertionEvent) clientAndStatus .getLeft() .pipeTransfer( - PipeTransferTsFileSealWithModReq.toTPipeTransferReq( - modFile.getName(), modFile.length(), tsFile.getName(), tsFile.length())); + compressIfNeeded( + PipeTransferTsFileSealWithModReq.toTPipeTransferReq( + modFile.getName(), + modFile.length(), + tsFile.getName(), + tsFile.length()))); } catch (final Exception e) { clientAndStatus.setRight(false); clientManager.adjustTimeoutIfNecessary(e); @@ -381,8 +392,9 @@ private void doTransfer(final PipeTsFileInsertionEvent pipeTsFileInsertionEvent) clientAndStatus .getLeft() .pipeTransfer( - PipeTransferTsFileSealReq.toTPipeTransferReq( - tsFile.getName(), tsFile.length())); + compressIfNeeded( + PipeTransferTsFileSealReq.toTPipeTransferReq( + tsFile.getName(), tsFile.length()))); } catch (final Exception e) { clientAndStatus.setRight(false); clientManager.adjustTimeoutIfNecessary(e); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBSchemaRegionConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBSchemaRegionConnector.java index 00ae9861dcf8..667050320d07 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBSchemaRegionConnector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBSchemaRegionConnector.java @@ -103,13 +103,14 @@ private void doTransfer(final PipeSchemaRegionSnapshotEvent snapshotEvent) clientAndStatus .getLeft() .pipeTransfer( - PipeTransferSchemaSnapshotSealReq.toTPipeTransferReq( - mTreeSnapshotFile.getName(), - mTreeSnapshotFile.length(), - Objects.nonNull(tagLogSnapshotFile) ? tagLogSnapshotFile.getName() : null, - Objects.nonNull(tagLogSnapshotFile) ? tagLogSnapshotFile.length() : 0, - snapshotEvent.getDatabaseName(), - snapshotEvent.toSealTypeString())); + compressIfNeeded( + PipeTransferSchemaSnapshotSealReq.toTPipeTransferReq( + mTreeSnapshotFile.getName(), + mTreeSnapshotFile.length(), + Objects.nonNull(tagLogSnapshotFile) ? tagLogSnapshotFile.getName() : null, + Objects.nonNull(tagLogSnapshotFile) ? tagLogSnapshotFile.length() : 0, + snapshotEvent.getDatabaseName(), + snapshotEvent.toSealTypeString()))); } catch (final Exception e) { clientAndStatus.setRight(false); throw new PipeConnectionException( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/thrift/IoTDBDataNodeReceiver.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/thrift/IoTDBDataNodeReceiver.java index c5f1d5fa99d4..dfd8be444b64 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/thrift/IoTDBDataNodeReceiver.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/thrift/IoTDBDataNodeReceiver.java @@ -25,6 +25,7 @@ import org.apache.iotdb.commons.pipe.connector.PipeReceiverStatusHandler; import org.apache.iotdb.commons.pipe.connector.payload.airgap.AirGapPseudoTPipeTransferRequest; import org.apache.iotdb.commons.pipe.connector.payload.thrift.request.PipeRequestType; +import org.apache.iotdb.commons.pipe.connector.payload.thrift.request.PipeTransferCompressedReq; import org.apache.iotdb.commons.pipe.connector.payload.thrift.request.PipeTransferFileSealReqV1; import org.apache.iotdb.commons.pipe.connector.payload.thrift.request.PipeTransferFileSealReqV2; import org.apache.iotdb.commons.pipe.receiver.IoTDBFileReceiver; @@ -188,6 +189,8 @@ public synchronized TPipeTransferResp receive(final TPipeTransferReq req) { // Config requests will first be received by the DataNode receiver, // then transferred to ConfigNode receiver to execute. return handleTransferConfigPlan(req); + case TRANSFER_COMPRESSED: + return receive(PipeTransferCompressedReq.fromTPipeTransferReq(req)); default: break; } 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 bb9d075910df..331c4d79a2f8 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 @@ -155,6 +155,24 @@ public class PipeConnectorConstant { CONNECTOR_LOAD_BALANCE_RANDOM_STRATEGY, CONNECTOR_LOAD_BALANCE_PRIORITY_STRATEGY))); + public static final String CONNECTOR_COMPRESSOR_KEY = "connector.compressor"; + public static final String SINK_COMPRESSOR_KEY = "sink.compressor"; + public static final String CONNECTOR_COMPRESSOR_DEFAULT_VALUE = ""; + public static final String CONNECTOR_COMPRESSOR_SNAPPY = "snappy"; + public static final String CONNECTOR_COMPRESSOR_GZIP = "gzip"; + public static final String CONNECTOR_COMPRESSOR_LZ4 = "lz4"; + public static final String CONNECTOR_COMPRESSOR_ZSTD = "zstd"; + public static final String CONNECTOR_COMPRESSOR_LZMA2 = "lzma2"; + public static final Set CONNECTOR_COMPRESSOR_SET = + Collections.unmodifiableSet( + new HashSet<>( + Arrays.asList( + CONNECTOR_COMPRESSOR_SNAPPY, + CONNECTOR_COMPRESSOR_GZIP, + CONNECTOR_COMPRESSOR_LZ4, + CONNECTOR_COMPRESSOR_ZSTD, + CONNECTOR_COMPRESSOR_LZMA2))); + public static final String SINK_TOPIC_KEY = "sink.topic"; public static final String SINK_CONSUMER_GROUP_KEY = "sink.consumer-group"; diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/compressor/PipeCompressor.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/compressor/PipeCompressor.java new file mode 100644 index 000000000000..4d5113e4c20e --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/compressor/PipeCompressor.java @@ -0,0 +1,57 @@ +/* + * 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.pipe.connector.compressor; + +import java.io.IOException; + +public abstract class PipeCompressor { + + public enum PipeCompressionType { + SNAPPY((byte) 0), + GZIP((byte) 1), + LZ4((byte) 2), + ZSTD((byte) 3), + LZMA2((byte) 4); + + final byte index; + + PipeCompressionType(byte index) { + this.index = index; + } + + public byte getIndex() { + return index; + } + } + + private final PipeCompressionType compressionType; + + protected PipeCompressor(PipeCompressionType compressionType) { + this.compressionType = compressionType; + } + + public abstract byte[] compress(byte[] data) throws IOException; + + public abstract byte[] decompress(byte[] byteArray) throws IOException; + + public byte serialize() { + return compressionType.getIndex(); + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/compressor/PipeCompressorFactory.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/compressor/PipeCompressorFactory.java new file mode 100644 index 000000000000..14ae972fc4f0 --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/compressor/PipeCompressorFactory.java @@ -0,0 +1,80 @@ +/* + * 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.pipe.connector.compressor; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.CONNECTOR_COMPRESSOR_GZIP; +import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.CONNECTOR_COMPRESSOR_LZ4; +import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.CONNECTOR_COMPRESSOR_LZMA2; +import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.CONNECTOR_COMPRESSOR_SNAPPY; +import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.CONNECTOR_COMPRESSOR_ZSTD; + +public class PipeCompressorFactory { + + private static Map COMPRESSOR_NAME_TO_INSTANCE = new HashMap<>(); + + static { + COMPRESSOR_NAME_TO_INSTANCE.put(CONNECTOR_COMPRESSOR_SNAPPY, new PipeSnappyCompressor()); + COMPRESSOR_NAME_TO_INSTANCE.put(CONNECTOR_COMPRESSOR_GZIP, new PipeGZIPCompressor()); + COMPRESSOR_NAME_TO_INSTANCE.put(CONNECTOR_COMPRESSOR_LZ4, new PipeLZ4Compressor()); + COMPRESSOR_NAME_TO_INSTANCE.put(CONNECTOR_COMPRESSOR_ZSTD, new PipeZSTDCompressor()); + COMPRESSOR_NAME_TO_INSTANCE.put(CONNECTOR_COMPRESSOR_LZMA2, new PipeLZMA2Compressor()); + COMPRESSOR_NAME_TO_INSTANCE = Collections.unmodifiableMap(COMPRESSOR_NAME_TO_INSTANCE); + } + + public static PipeCompressor getCompressor(String name) { + final PipeCompressor compressor = COMPRESSOR_NAME_TO_INSTANCE.get(name); + if (compressor == null) { + throw new UnsupportedOperationException("PipeCompressor not found for name: " + name); + } + return compressor; + } + + private static Map COMPRESSOR_INDEX_TO_INSTANCE = new HashMap<>(); + + static { + COMPRESSOR_INDEX_TO_INSTANCE.put( + PipeCompressor.PipeCompressionType.SNAPPY.getIndex(), new PipeSnappyCompressor()); + COMPRESSOR_INDEX_TO_INSTANCE.put( + PipeCompressor.PipeCompressionType.GZIP.getIndex(), new PipeGZIPCompressor()); + COMPRESSOR_INDEX_TO_INSTANCE.put( + PipeCompressor.PipeCompressionType.LZ4.getIndex(), new PipeLZ4Compressor()); + COMPRESSOR_INDEX_TO_INSTANCE.put( + PipeCompressor.PipeCompressionType.ZSTD.getIndex(), new PipeZSTDCompressor()); + COMPRESSOR_INDEX_TO_INSTANCE.put( + PipeCompressor.PipeCompressionType.LZMA2.getIndex(), new PipeLZMA2Compressor()); + COMPRESSOR_INDEX_TO_INSTANCE = Collections.unmodifiableMap(COMPRESSOR_INDEX_TO_INSTANCE); + } + + public static PipeCompressor getCompressor(byte index) { + final PipeCompressor compressor = COMPRESSOR_INDEX_TO_INSTANCE.get(index); + if (compressor == null) { + throw new UnsupportedOperationException("PipeCompressor not found for index: " + index); + } + return compressor; + } + + private PipeCompressorFactory() { + // Empty constructor + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/compressor/PipeGZIPCompressor.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/compressor/PipeGZIPCompressor.java new file mode 100644 index 000000000000..2cc1887555cd --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/compressor/PipeGZIPCompressor.java @@ -0,0 +1,47 @@ +/* + * 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.pipe.connector.compressor; + +import org.apache.tsfile.compress.ICompressor; +import org.apache.tsfile.compress.IUnCompressor; +import org.apache.tsfile.file.metadata.enums.CompressionType; + +import java.io.IOException; + +public class PipeGZIPCompressor extends PipeCompressor { + + private static final ICompressor COMPRESSOR = ICompressor.getCompressor(CompressionType.GZIP); + private static final IUnCompressor DECOMPRESSOR = + IUnCompressor.getUnCompressor(CompressionType.GZIP); + + public PipeGZIPCompressor() { + super(PipeCompressionType.GZIP); + } + + @Override + public byte[] compress(byte[] data) throws IOException { + return COMPRESSOR.compress(data); + } + + @Override + public byte[] decompress(byte[] byteArray) throws IOException { + return DECOMPRESSOR.uncompress(byteArray); + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/compressor/PipeLZ4Compressor.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/compressor/PipeLZ4Compressor.java new file mode 100644 index 000000000000..5d6db90e6610 --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/compressor/PipeLZ4Compressor.java @@ -0,0 +1,47 @@ +/* + * 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.pipe.connector.compressor; + +import org.apache.tsfile.compress.ICompressor; +import org.apache.tsfile.compress.IUnCompressor; +import org.apache.tsfile.file.metadata.enums.CompressionType; + +import java.io.IOException; + +public class PipeLZ4Compressor extends PipeCompressor { + + private static final ICompressor COMPRESSOR = ICompressor.getCompressor(CompressionType.LZ4); + private static final IUnCompressor DECOMPRESSOR = + IUnCompressor.getUnCompressor(CompressionType.LZ4); + + public PipeLZ4Compressor() { + super(PipeCompressionType.LZ4); + } + + @Override + public byte[] compress(byte[] data) throws IOException { + return COMPRESSOR.compress(data); + } + + @Override + public byte[] decompress(byte[] byteArray) throws IOException { + return DECOMPRESSOR.uncompress(byteArray); + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/compressor/PipeLZMA2Compressor.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/compressor/PipeLZMA2Compressor.java new file mode 100644 index 000000000000..9e7c0a8b8d37 --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/compressor/PipeLZMA2Compressor.java @@ -0,0 +1,47 @@ +/* + * 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.pipe.connector.compressor; + +import org.apache.tsfile.compress.ICompressor; +import org.apache.tsfile.compress.IUnCompressor; +import org.apache.tsfile.file.metadata.enums.CompressionType; + +import java.io.IOException; + +public class PipeLZMA2Compressor extends PipeCompressor { + + private static final ICompressor COMPRESSOR = ICompressor.getCompressor(CompressionType.LZMA2); + private static final IUnCompressor DECOMPRESSOR = + IUnCompressor.getUnCompressor(CompressionType.LZMA2); + + public PipeLZMA2Compressor() { + super(PipeCompressionType.LZMA2); + } + + @Override + public byte[] compress(byte[] data) throws IOException { + return COMPRESSOR.compress(data); + } + + @Override + public byte[] decompress(byte[] byteArray) throws IOException { + return DECOMPRESSOR.uncompress(byteArray); + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/compressor/PipeSnappyCompressor.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/compressor/PipeSnappyCompressor.java new file mode 100644 index 000000000000..72333c1e929b --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/compressor/PipeSnappyCompressor.java @@ -0,0 +1,47 @@ +/* + * 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.pipe.connector.compressor; + +import org.apache.tsfile.compress.ICompressor; +import org.apache.tsfile.compress.IUnCompressor; +import org.apache.tsfile.file.metadata.enums.CompressionType; + +import java.io.IOException; + +public class PipeSnappyCompressor extends PipeCompressor { + + private static final ICompressor COMPRESSOR = ICompressor.getCompressor(CompressionType.SNAPPY); + private static final IUnCompressor DECOMPRESSOR = + IUnCompressor.getUnCompressor(CompressionType.SNAPPY); + + public PipeSnappyCompressor() { + super(PipeCompressionType.SNAPPY); + } + + @Override + public byte[] compress(byte[] data) throws IOException { + return COMPRESSOR.compress(data); + } + + @Override + public byte[] decompress(byte[] byteArray) throws IOException { + return DECOMPRESSOR.uncompress(byteArray); + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/compressor/PipeZSTDCompressor.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/compressor/PipeZSTDCompressor.java new file mode 100644 index 000000000000..72782353d689 --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/compressor/PipeZSTDCompressor.java @@ -0,0 +1,47 @@ +/* + * 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.pipe.connector.compressor; + +import org.apache.tsfile.compress.ICompressor; +import org.apache.tsfile.compress.IUnCompressor; +import org.apache.tsfile.file.metadata.enums.CompressionType; + +import java.io.IOException; + +public class PipeZSTDCompressor extends PipeCompressor { + + private static final ICompressor COMPRESSOR = ICompressor.getCompressor(CompressionType.ZSTD); + private static final IUnCompressor DECOMPRESSOR = + IUnCompressor.getUnCompressor(CompressionType.ZSTD); + + public PipeZSTDCompressor() { + super(PipeCompressionType.ZSTD); + } + + @Override + public byte[] compress(byte[] data) throws IOException { + return COMPRESSOR.compress(data); + } + + @Override + public byte[] decompress(byte[] byteArray) throws IOException { + return DECOMPRESSOR.uncompress(byteArray); + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/payload/thrift/request/PipeRequestType.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/payload/thrift/request/PipeRequestType.java index cbe7b395d1ef..003c8b9afb32 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/payload/thrift/request/PipeRequestType.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/payload/thrift/request/PipeRequestType.java @@ -50,6 +50,9 @@ public enum PipeRequestType { TRANSFER_CONFIG_PLAN((short) 200), TRANSFER_CONFIG_SNAPSHOT_PIECE((short) 201), TRANSFER_CONFIG_SNAPSHOT_SEAL((short) 202), + + // RPC Compression + TRANSFER_COMPRESSED((short) 300), ; private final short type; diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/payload/thrift/request/PipeTransferCompressedReq.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/payload/thrift/request/PipeTransferCompressedReq.java new file mode 100644 index 000000000000..f5e114949bd7 --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/payload/thrift/request/PipeTransferCompressedReq.java @@ -0,0 +1,146 @@ +/* + * 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.pipe.connector.payload.thrift.request; + +import org.apache.iotdb.commons.pipe.connector.compressor.PipeCompressor; +import org.apache.iotdb.commons.pipe.connector.compressor.PipeCompressorFactory; +import org.apache.iotdb.service.rpc.thrift.TPipeTransferReq; + +import org.apache.tsfile.utils.BytesUtils; +import org.apache.tsfile.utils.PublicBAOS; +import org.apache.tsfile.utils.ReadWriteIOUtils; + +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +public class PipeTransferCompressedReq extends TPipeTransferReq { + + /** Generate a compressed req with provided compressors. */ + public static TPipeTransferReq toTPipeTransferReq( + final TPipeTransferReq originalReq, final List compressors) + throws IOException { + // The generated PipeTransferCompressedReq consists of: + // version + // type: TRANSFER_COMPRESSED + // body: + // (byte) count of compressors + // (bytes) 1 byte for each compressor + // compressed req: + // (byte) version + // (2 bytes) type + // (bytes) body + final PipeTransferCompressedReq compressedReq = new PipeTransferCompressedReq(); + compressedReq.version = IoTDBConnectorRequestVersion.VERSION_1.getVersion(); + compressedReq.type = PipeRequestType.TRANSFER_COMPRESSED.getType(); + + try (final PublicBAOS byteArrayOutputStream = new PublicBAOS(); + final DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream)) { + ReadWriteIOUtils.write((byte) compressors.size(), outputStream); + for (final PipeCompressor compressor : compressors) { + ReadWriteIOUtils.write(compressor.serialize(), outputStream); + } + + byte[] body = + BytesUtils.concatByteArrayList( + Arrays.asList( + new byte[] {originalReq.version}, + BytesUtils.shortToBytes(originalReq.type), + originalReq.getBody())); + for (final PipeCompressor compressor : compressors) { + body = compressor.compress(body); + } + outputStream.write(body); + + compressedReq.body = + ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size()); + } + return compressedReq; + } + + /** Get the original req from a compressed req. */ + public static TPipeTransferReq fromTPipeTransferReq(final TPipeTransferReq transferReq) + throws IOException { + final ByteBuffer compressedBuffer = transferReq.body; + + final List compressors = new ArrayList<>(); + final int compressorsSize = ReadWriteIOUtils.readByte(compressedBuffer); + for (int i = 0; i < compressorsSize; ++i) { + compressors.add( + PipeCompressorFactory.getCompressor(ReadWriteIOUtils.readByte(compressedBuffer))); + } + + byte[] body = new byte[compressedBuffer.remaining()]; + compressedBuffer.get(body); + + for (int i = compressors.size() - 1; i >= 0; --i) { + body = compressors.get(i).decompress(body); + } + + final ByteBuffer decompressedBuffer = ByteBuffer.wrap(body); + + final TPipeTransferReq decompressedReq = new TPipeTransferReq(); + decompressedReq.version = ReadWriteIOUtils.readByte(decompressedBuffer); + decompressedReq.type = ReadWriteIOUtils.readShort(decompressedBuffer); + decompressedReq.body = decompressedBuffer.slice(); + + return decompressedReq; + } + + /** + * For air-gap connectors. Generate the bytes of a compressed req from the bytes of original req. + */ + public static byte[] toTPipeTransferReqBytes( + final byte[] rawReqInBytes, final List compressors) throws IOException { + // The generated bytes consists of: + // (byte) version + // (2 bytes) type: TRANSFER_COMPRESSED + // (byte) count of compressors + // (bytes) 1 byte for each compressor + // compressed req: + // (byte) version + // (2 bytes) type + // (bytes) body + try (final PublicBAOS byteArrayOutputStream = new PublicBAOS(); + final DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream)) { + ReadWriteIOUtils.write(IoTDBConnectorRequestVersion.VERSION_1.getVersion(), outputStream); + ReadWriteIOUtils.write(PipeRequestType.TRANSFER_COMPRESSED.getType(), outputStream); + ReadWriteIOUtils.write((byte) compressors.size(), outputStream); + for (final PipeCompressor compressor : compressors) { + ReadWriteIOUtils.write(compressor.serialize(), outputStream); + } + + byte[] compressedReq = rawReqInBytes; + for (final PipeCompressor compressor : compressors) { + compressedReq = compressor.compress(compressedReq); + } + outputStream.write(compressedReq); + + return byteArrayOutputStream.toByteArray(); + } + } + + private PipeTransferCompressedReq() { + // Empty constructor + } +} 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 3034b581d5d0..ec2d22ddfa64 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 @@ -226,9 +226,10 @@ protected void transferFilePieces(File file, Socket socket, boolean isMultiFile) : Arrays.copyOfRange(readBuffer, 0, readLength); if (!send( socket, - isMultiFile - ? getTransferMultiFilePieceBytes(file.getName(), position, payload) - : getTransferSingleFilePieceBytes(file.getName(), position, payload))) { + compressIfNeeded( + isMultiFile + ? getTransferMultiFilePieceBytes(file.getName(), position, payload) + : getTransferSingleFilePieceBytes(file.getName(), position, payload)))) { final String errorMessage = String.format("Transfer file %s error. Socket %s.", file, socket); if (mayNeedHandshakeWhenFail()) { 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 65b32a483969..751f0dec05f5 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,16 +21,21 @@ import org.apache.iotdb.common.rpc.thrift.TEndPoint; import org.apache.iotdb.commons.pipe.connector.PipeReceiverStatusHandler; +import org.apache.iotdb.commons.pipe.connector.compressor.PipeCompressor; +import org.apache.iotdb.commons.pipe.connector.compressor.PipeCompressorFactory; +import org.apache.iotdb.commons.pipe.connector.payload.thrift.request.PipeTransferCompressedReq; import org.apache.iotdb.commons.utils.NodeUrlUtils; import org.apache.iotdb.pipe.api.PipeConnector; import org.apache.iotdb.pipe.api.customizer.configuration.PipeConnectorRuntimeConfiguration; import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameterValidator; import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameters; import org.apache.iotdb.pipe.api.exception.PipeParameterNotValidException; +import org.apache.iotdb.service.rpc.thrift.TPipeTransferReq; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.LinkedHashSet; @@ -38,6 +43,9 @@ import java.util.Objects; import java.util.Set; +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; +import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.CONNECTOR_COMPRESSOR_SET; import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.CONNECTOR_EXCEPTION_CONFLICT_RECORD_IGNORED_DATA_DEFAULT_VALUE; import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.CONNECTOR_EXCEPTION_CONFLICT_RECORD_IGNORED_DATA_KEY; import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.CONNECTOR_EXCEPTION_CONFLICT_RESOLVE_STRATEGY_DEFAULT_VALUE; @@ -57,6 +65,7 @@ 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; +import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.SINK_COMPRESSOR_KEY; import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.SINK_EXCEPTION_CONFLICT_RECORD_IGNORED_DATA_KEY; import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.SINK_EXCEPTION_CONFLICT_RESOLVE_STRATEGY_KEY; import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.SINK_EXCEPTION_CONFLICT_RETRY_MAX_TIME_SECONDS_KEY; @@ -82,6 +91,9 @@ public abstract class IoTDBConnector implements PipeConnector { protected String loadBalanceStrategy; + protected boolean isRpcCompressionEnabled; + protected final List compressors = new ArrayList<>(); + protected boolean isTabletBatchModeEnabled = true; protected PipeReceiverStatusHandler receiverStatusHandler; @@ -127,6 +139,36 @@ public void validate(PipeParameterValidator validator) throws Exception { CONNECTOR_LOAD_BALANCE_STRATEGY_SET, loadBalanceStrategy), loadBalanceStrategy); + final String compressionTypes = + parameters + .getStringOrDefault( + Arrays.asList(CONNECTOR_COMPRESSOR_KEY, SINK_COMPRESSOR_KEY), + CONNECTOR_COMPRESSOR_DEFAULT_VALUE) + .toLowerCase(); + if (!compressionTypes.isEmpty()) { + for (final String compressionType : compressionTypes.split(",")) { + final String trimmedCompressionType = compressionType.trim(); + if (trimmedCompressionType.isEmpty()) { + continue; + } + + validator.validate( + arg -> CONNECTOR_COMPRESSOR_SET.contains(trimmedCompressionType), + String.format( + "Compressor should be one of %s, but got %s.", + CONNECTOR_COMPRESSOR_SET, trimmedCompressionType), + trimmedCompressionType); + compressors.add(PipeCompressorFactory.getCompressor(trimmedCompressionType)); + } + } + validator.validate( + arg -> compressors.size() <= Byte.MAX_VALUE, + String.format( + "The number of compressors should be less than or equal to %d, but got %d.", + Byte.MAX_VALUE, compressors.size()), + compressors.size()); + isRpcCompressionEnabled = !compressors.isEmpty(); + validator.validate( arg -> arg.equals("retry") || arg.equals("ignore"), String.format( @@ -260,6 +302,26 @@ private void checkNodeUrls(Set nodeUrls) throws PipeParameterNotValid } } + protected TPipeTransferReq compressIfNeeded(TPipeTransferReq req) throws IOException { + return isRpcCompressionEnabled + ? PipeTransferCompressedReq.toTPipeTransferReq(req, compressors) + : req; + } + + protected byte[] compressIfNeeded(byte[] reqInBytes) throws IOException { + return isRpcCompressionEnabled + ? PipeTransferCompressedReq.toTPipeTransferReqBytes(reqInBytes, compressors) + : reqInBytes; + } + + public boolean isRpcCompressionEnabled() { + return isRpcCompressionEnabled; + } + + public List getCompressors() { + return compressors; + } + public PipeReceiverStatusHandler statusHandler() { return receiverStatusHandler; } 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 e366584fe8f1..4e1ab4d47891 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 @@ -166,9 +166,11 @@ protected void transferFilePieces( clientAndStatus .getLeft() .pipeTransfer( - isMultiFile - ? getTransferMultiFilePieceReq(file.getName(), position, payLoad) - : getTransferSingleFilePieceReq(file.getName(), position, payLoad))); + compressIfNeeded( + isMultiFile + ? getTransferMultiFilePieceReq(file.getName(), position, payLoad) + : getTransferSingleFilePieceReq( + file.getName(), position, payLoad)))); } catch (Exception e) { clientAndStatus.setRight(false); throw new PipeConnectionException( From 44ed96111abe1d4f3bb0f7a8696ea9397a3466ae Mon Sep 17 00:00:00 2001 From: Chen YZ <43774645+Cpaulyz@users.noreply.github.com> Date: Thu, 16 May 2024 14:44:45 +0800 Subject: [PATCH 03/42] Add device number metric --- .../metric/SchemaEngineMemMetric.java | 15 ++++++++++++- .../metric/SchemaRegionMemMetric.java | 21 +++++++++++++++++++ .../rescon/ISchemaEngineStatistics.java | 2 ++ .../rescon/MemSchemaEngineStatistics.java | 14 +++++++++++++ .../rescon/MemSchemaRegionStatistics.java | 3 +++ 5 files changed, 54 insertions(+), 1 deletion(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/metric/SchemaEngineMemMetric.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/metric/SchemaEngineMemMetric.java index 88080782e164..b80f0f6b6efc 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/metric/SchemaEngineMemMetric.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/metric/SchemaEngineMemMetric.java @@ -37,7 +37,7 @@ public class SchemaEngineMemMetric implements ISchemaEngineMetric { private static final String TOTAL_MEM_USAGE = "schema_region_total_mem_usage"; private static final String MEM_CAPACITY = "schema_region_mem_capacity"; private static final String REGION_NUMBER = "schema_region_number"; - + private static final String DEVICE_NUMBER = "schema_region_total_device_cnt"; private static final String SCHEMA_CONSENSUS = "schema_region_consensus"; private static final String SCHEMA_ENGINE_MODE = "schema_engine_mode"; @@ -49,6 +49,13 @@ public SchemaEngineMemMetric(MemSchemaEngineStatistics engineStatistics) { @Override public void bindTo(AbstractMetricService metricService) { + metricService.createAutoGauge( + Metric.MEM.toString(), + MetricLevel.IMPORTANT, + engineStatistics, + ISchemaEngineStatistics::getTotalDevicesNumber, + Tag.NAME.toString(), + DEVICE_NUMBER); metricService.createAutoGauge( Metric.MEM.toString(), MetricLevel.IMPORTANT, @@ -94,6 +101,8 @@ public void bindTo(AbstractMetricService metricService) { @Override public void unbindFrom(AbstractMetricService metricService) { + metricService.remove( + MetricType.AUTO_GAUGE, Metric.MEM.toString(), Tag.NAME.toString(), DEVICE_NUMBER); metricService.remove( MetricType.AUTO_GAUGE, Metric.MEM.toString(), @@ -108,6 +117,10 @@ public void unbindFrom(AbstractMetricService metricService) { MetricType.AUTO_GAUGE, Metric.SCHEMA_ENGINE.toString(), Tag.NAME.toString(), MEM_CAPACITY); metricService.remove( MetricType.AUTO_GAUGE, Metric.SCHEMA_ENGINE.toString(), Tag.NAME.toString(), REGION_NUMBER); + metricService.remove( + MetricType.GAUGE, Metric.SCHEMA_ENGINE.toString(), Tag.NAME.toString(), SCHEMA_ENGINE_MODE); + metricService.remove( + MetricType.GAUGE, Metric.SCHEMA_ENGINE.toString(), Tag.NAME.toString(), SCHEMA_CONSENSUS); } /** diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/metric/SchemaRegionMemMetric.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/metric/SchemaRegionMemMetric.java index 1437d5d651e2..ae255cc0caa1 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/metric/SchemaRegionMemMetric.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/metric/SchemaRegionMemMetric.java @@ -34,6 +34,7 @@ public class SchemaRegionMemMetric implements ISchemaRegionMetric { private static final String MEM_USAGE = "schema_region_mem_usage"; private static final String SERIES_CNT = "schema_region_series_cnt"; + private static final String DEVICE_NUMBER = "schema_region_device_cnt"; private static final String TEMPLATE_CNT = "activated_template_cnt"; private static final String TEMPLATE_SERIES_CNT = "template_series_cnt"; private static final String TRAVERSER_TIMER = "schema_region_traverser_timer"; @@ -52,6 +53,17 @@ public SchemaRegionMemMetric(MemSchemaRegionStatistics regionStatistics, String @Override public void bindTo(AbstractMetricService metricService) { + metricService.createAutoGauge( + Metric.SCHEMA_REGION.toString(), + MetricLevel.IMPORTANT, + regionStatistics, + MemSchemaRegionStatistics::getDevicesNumber, + Tag.NAME.toString(), + DEVICE_NUMBER, + Tag.REGION.toString(), + regionTagValue, + Tag.DATABASE.toString(), + database); metricService.createAutoGauge( Metric.SCHEMA_REGION.toString(), MetricLevel.IMPORTANT, @@ -111,6 +123,15 @@ public void bindTo(AbstractMetricService metricService) { @Override public void unbindFrom(AbstractMetricService metricService) { traverserTimer = DoNothingMetricManager.DO_NOTHING_TIMER; + metricService.remove( + MetricType.AUTO_GAUGE, + Metric.SCHEMA_REGION.toString(), + Tag.NAME.toString(), + DEVICE_NUMBER, + Tag.REGION.toString(), + regionTagValue, + Tag.DATABASE.toString(), + database); metricService.remove( MetricType.AUTO_GAUGE, Metric.SCHEMA_REGION.toString(), diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/rescon/ISchemaEngineStatistics.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/rescon/ISchemaEngineStatistics.java index 1cdd6aa11b2e..cca3cecfc692 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/rescon/ISchemaEngineStatistics.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/rescon/ISchemaEngineStatistics.java @@ -31,6 +31,8 @@ public interface ISchemaEngineStatistics { long getTotalSeriesNumber(); + long getTotalDevicesNumber(); + int getSchemaRegionNumber(); long getTemplateSeriesNumber(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/rescon/MemSchemaEngineStatistics.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/rescon/MemSchemaEngineStatistics.java index b48d27fb7989..9bc847634dd6 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/rescon/MemSchemaEngineStatistics.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/rescon/MemSchemaEngineStatistics.java @@ -44,6 +44,7 @@ public class MemSchemaEngineStatistics implements ISchemaEngineStatistics { protected final AtomicLong memoryUsage = new AtomicLong(0); private final AtomicLong totalMeasurementNumber = new AtomicLong(0); private final AtomicLong totalViewNumber = new AtomicLong(0); + private final AtomicLong totalDeviceNumber = new AtomicLong(0); private final Map templateUsage = new ConcurrentHashMap<>(); private volatile boolean allowToCreateNewSeries = true; @@ -116,6 +117,11 @@ public long getTotalSeriesNumber() { return totalMeasurementNumber.get() + totalViewNumber.get() + getTemplateSeriesNumber(); } + @Override + public long getTotalDevicesNumber() { + return totalDeviceNumber.get(); + } + @Override public int getSchemaRegionNumber() { return SchemaEngine.getInstance().getSchemaRegionNumber(); @@ -165,6 +171,14 @@ public void deleteView(long deletedNum) { totalViewNumber.addAndGet(-deletedNum); } + public void addDevice() { + totalDeviceNumber.incrementAndGet(); + } + + public void deleteDevice(long cnt) { + totalDeviceNumber.addAndGet(-cnt); + } + @Override public MemSchemaEngineStatistics getAsMemSchemaEngineStatistics() { return this; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/rescon/MemSchemaRegionStatistics.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/rescon/MemSchemaRegionStatistics.java index c5fd13ad93fb..be960b3807b6 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/rescon/MemSchemaRegionStatistics.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/rescon/MemSchemaRegionStatistics.java @@ -96,10 +96,12 @@ public long getDevicesNumber() { public void addDevice() { devicesNumber.incrementAndGet(); + schemaEngineStatistics.addDevice(); } public void deleteDevice() { devicesNumber.decrementAndGet(); + schemaEngineStatistics.deleteDevice(1L); } @Override @@ -162,6 +164,7 @@ public void clear() { schemaEngineStatistics.releaseMemory(memoryUsage.get()); schemaEngineStatistics.deleteMeasurement(measurementNumber.get()); schemaEngineStatistics.deleteView(viewNumber.get()); + schemaEngineStatistics.deleteDevice(devicesNumber.get()); memoryUsage.getAndSet(0); measurementNumber.getAndSet(0); devicesNumber.getAndSet(0); From 58331a6c73dbd9fb0e58ffaa432de5d87e8517ea Mon Sep 17 00:00:00 2001 From: William Song <48054931+SzyWilliam@users.noreply.github.com> Date: Thu, 16 May 2024 21:13:39 +0800 Subject: [PATCH 04/42] bump ratis to 3.1.0-snapshot (#12533) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index c05397697cd1..e7ab8384d093 100644 --- a/pom.xml +++ b/pom.xml @@ -140,7 +140,7 @@ is for ensuring the SNAPSHOT will stay available. We should however have the Ratis folks do a new release soon, as releasing with this version is more than sub-ideal. --> - 3.1.0-611b80a-SNAPSHOT + 3.1.0-2fddd52-SNAPSHOT 1.0.4 1.1.13 3.5.10 From b50ec7d64a421cebb098340291483ea65ae46cda Mon Sep 17 00:00:00 2001 From: OneSizeFitQuorum Date: Thu, 16 May 2024 21:43:46 +0800 Subject: [PATCH 05/42] Revert "bump ratis to 3.1.0-snapshot (#12533)" This reverts commit 58331a6c73dbd9fb0e58ffaa432de5d87e8517ea. --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index e7ab8384d093..c05397697cd1 100644 --- a/pom.xml +++ b/pom.xml @@ -140,7 +140,7 @@ is for ensuring the SNAPSHOT will stay available. We should however have the Ratis folks do a new release soon, as releasing with this version is more than sub-ideal. --> - 3.1.0-2fddd52-SNAPSHOT + 3.1.0-611b80a-SNAPSHOT 1.0.4 1.1.13 3.5.10 From e0eea1d14bf15841b6b2437f0315b699a9b6f519 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Fri, 17 May 2024 09:53:59 +0800 Subject: [PATCH 06/42] JDBC: Fix maxRows cannot be set to 0 (#12540) --- .../src/main/java/org/apache/iotdb/jdbc/IoTDBStatement.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/iotdb-client/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBStatement.java b/iotdb-client/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBStatement.java index ab2e598d934b..b1784410915a 100644 --- a/iotdb-client/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBStatement.java +++ b/iotdb-client/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBStatement.java @@ -577,8 +577,8 @@ public int getMaxRows() throws SQLException { @Override public void setMaxRows(int num) throws SQLException { checkConnection("setMaxRows"); - if (num <= 0) { - throw new SQLException(String.format("maxRows %d must be > 0!", num)); + if (num < 0) { + throw new SQLException(String.format("maxRows %d must be >= 0!", num)); } this.maxRows = num; } From 2f1ca39f391a6539df98b1b0112761b701f923f4 Mon Sep 17 00:00:00 2001 From: Chen YZ <43774645+Cpaulyz@users.noreply.github.com> Date: Fri, 17 May 2024 10:59:37 +0800 Subject: [PATCH 07/42] Fix non-root user query data returns unexpected path --- .../apache/iotdb/db/it/auth/IoTDBAuthIT.java | 29 +++++++++++++++++++ .../common/schematree/ClusterSchemaTree.java | 5 +--- .../analyze/schema/ClusterSchemaFetcher.java | 4 ++- 3 files changed, 33 insertions(+), 5 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/auth/IoTDBAuthIT.java b/integration-test/src/test/java/org/apache/iotdb/db/it/auth/IoTDBAuthIT.java index 25e1b6833f9a..93165a16ff6a 100644 --- a/integration-test/src/test/java/org/apache/iotdb/db/it/auth/IoTDBAuthIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/db/it/auth/IoTDBAuthIT.java @@ -324,6 +324,35 @@ public void createDeleteTimeSeriesTest() throws SQLException { } } + @Test + public void templateQueryTest() throws SQLException { + try (Connection adminCon = EnvFactory.getEnv().getConnection(); + Statement adminStmt = adminCon.createStatement()) { + adminStmt.execute("CREATE USER tempuser 'temppw'"); + try (Connection userCon = EnvFactory.getEnv().getConnection("tempuser", "temppw"); + Statement userStmt = userCon.createStatement()) { + adminStmt.execute( + "GRANT READ_DATA ON root.sg.aligned_template.temperature TO USER tempuser"); + adminStmt.execute("CREATE DATABASE root.sg"); + adminStmt.execute( + "create device template t1 aligned (temperature FLOAT encoding=Gorilla, status BOOLEAN encoding=PLAIN);"); + adminStmt.execute("set device template t1 to root.sg.aligned_template;"); + adminStmt.execute("create timeseries using device template on root.sg.aligned_template;"); + adminStmt.execute( + "insert into root.sg.aligned_template(time,temperature,status) values(1,20,false),(2,22.1,true),(3,18,false);"); + + ResultSet set1 = adminStmt.executeQuery("SELECT * from root.sg.aligned_template"); + assertEquals(3, set1.getMetaData().getColumnCount()); + assertEquals("root.sg.aligned_template.temperature", set1.getMetaData().getColumnName(2)); + assertEquals("root.sg.aligned_template.status", set1.getMetaData().getColumnName(3)); + + ResultSet set2 = userStmt.executeQuery("SELECT * from root.sg.aligned_template"); + assertEquals(2, set2.getMetaData().getColumnCount()); + assertEquals("root.sg.aligned_template.temperature", set2.getMetaData().getColumnName(2)); + } + } + } + @Test public void insertQueryTest() throws SQLException { try (Connection adminCon = EnvFactory.getEnv().getConnection(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/schematree/ClusterSchemaTree.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/schematree/ClusterSchemaTree.java index 995c4542c82e..c461d0da83ff 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/schematree/ClusterSchemaTree.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/schematree/ClusterSchemaTree.java @@ -21,7 +21,6 @@ import org.apache.iotdb.commons.path.MeasurementPath; import org.apache.iotdb.commons.path.PartialPath; -import org.apache.iotdb.commons.path.PathPatternTree; import org.apache.iotdb.commons.schema.view.LogicalViewSchema; import org.apache.iotdb.commons.utils.PathUtils; import org.apache.iotdb.commons.utils.TestOnly; @@ -76,8 +75,6 @@ public class ClusterSchemaTree implements ISchemaTree { private Map templateMap = new HashMap<>(); - private PathPatternTree authorityScope; - public ClusterSchemaTree() { root = new SchemaInternalNode(PATH_ROOT); } @@ -102,7 +99,7 @@ public Pair, Integer> searchMeasurementPaths( PartialPath pathPattern, int slimit, int soffset, boolean isPrefixMatch) { try (SchemaTreeVisitorWithLimitOffsetWrapper visitor = SchemaTreeVisitorFactory.createSchemaTreeMeasurementVisitor( - root, pathPattern, isPrefixMatch, slimit, soffset, authorityScope)) { + root, pathPattern, isPrefixMatch, slimit, soffset)) { visitor.setTemplateMap(templateMap); return new Pair<>(visitor.getAllResult(), visitor.getNextOffset()); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/schema/ClusterSchemaFetcher.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/schema/ClusterSchemaFetcher.java index a0ea490b9901..bae7472ab8d0 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/schema/ClusterSchemaFetcher.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/schema/ClusterSchemaFetcher.java @@ -92,7 +92,9 @@ public ClusterSchemaTree fetchSchema( Set explicitDevicePatternList = new HashSet<>(); int explicitDevicePatternCount = 0; for (PartialPath pattern : pathPatternList) { - if (pattern.hasExplicitDevice() && templateManager.checkTemplateSetInfo(pattern) != null) { + if (withTemplate + && pattern.hasExplicitDevice() + && templateManager.checkTemplateSetInfo(pattern) != null) { explicitDevicePatternList.add(pattern.getDevicePath()); explicitDevicePatternCount++; } else if (!pattern.hasWildcard()) { From 29fab68548970d195a3cb18e0031ba115f216cb0 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Fri, 17 May 2024 13:26:00 +0800 Subject: [PATCH 08/42] Pipe IT: Added flush on IoTDBPipeExtractorIT#testExtractorPatternMatch() to avoid pipe on previous follower getting unclosed historical tsfile from the old leader (#12534) --- .../it/autocreate/IoTDBPipeExtractorIT.java | 36 +++++++++++-------- 1 file changed, 22 insertions(+), 14 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeExtractorIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeExtractorIT.java index ba2e80212675..541d628aa7eb 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeExtractorIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeExtractorIT.java @@ -78,7 +78,7 @@ public void testExtractorValidParameter() throws Exception { try (final Connection connection = senderEnv.getConnection(); final Statement statement = connection.createStatement()) { statement.execute(p1_1); - } catch (SQLException e) { + } catch (final SQLException e) { fail(e.getMessage()); } @@ -98,7 +98,7 @@ public void testExtractorValidParameter() throws Exception { try (final Connection connection = senderEnv.getConnection(); final Statement statement = connection.createStatement()) { statement.execute(p1_2); - } catch (SQLException e) { + } catch (final SQLException e) { fail(e.getMessage()); } @@ -118,7 +118,7 @@ public void testExtractorValidParameter() throws Exception { try (final Connection connection = senderEnv.getConnection(); final Statement statement = connection.createStatement()) { statement.execute(p1_3); - } catch (SQLException e) { + } catch (final SQLException e) { fail(e.getMessage()); } @@ -139,7 +139,7 @@ public void testExtractorValidParameter() throws Exception { try (final Connection connection = senderEnv.getConnection(); final Statement statement = connection.createStatement()) { statement.execute(p1_4); - } catch (SQLException e) { + } catch (final SQLException e) { fail(e.getMessage()); } @@ -160,7 +160,7 @@ public void testExtractorValidParameter() throws Exception { try (final Connection connection = senderEnv.getConnection(); final Statement statement = connection.createStatement()) { statement.execute(p1_5); - } catch (SQLException e) { + } catch (final SQLException e) { fail(e.getMessage()); } @@ -185,7 +185,7 @@ public void testExtractorValidParameter() throws Exception { try (final Connection connection = senderEnv.getConnection(); final Statement statement = connection.createStatement()) { statement.execute(p2_1); - } catch (SQLException e) { + } catch (final SQLException e) { fail(e.getMessage()); } @@ -204,7 +204,7 @@ public void testExtractorValidParameter() throws Exception { try (final Connection connection = senderEnv.getConnection(); final Statement statement = connection.createStatement()) { statement.execute(p2_2); - } catch (SQLException e) { + } catch (final SQLException e) { fail(e.getMessage()); } @@ -224,7 +224,7 @@ public void testExtractorValidParameter() throws Exception { try (final Connection connection = senderEnv.getConnection(); final Statement statement = connection.createStatement()) { statement.execute(p2_3); - } catch (SQLException e) { + } catch (final SQLException e) { fail(e.getMessage()); } @@ -244,7 +244,7 @@ public void testExtractorValidParameter() throws Exception { try (final Connection connection = senderEnv.getConnection(); final Statement statement = connection.createStatement()) { statement.execute(p2_4); - } catch (SQLException e) { + } catch (final SQLException e) { fail(e.getMessage()); } @@ -270,7 +270,7 @@ public void testExtractorValidParameter() throws Exception { try (final Connection connection = senderEnv.getConnection(); final Statement statement = connection.createStatement()) { statement.execute(p3_1); - } catch (SQLException e) { + } catch (final SQLException e) { fail(e.getMessage()); } @@ -305,7 +305,7 @@ public void testExtractorInvalidParameter() throws Exception { final Statement statement = connection.createStatement()) { statement.execute(String.format(formatString, invalidStartTime)); fail(); - } catch (SQLException ignored) { + } catch (final SQLException ignored) { } } assertPipeCount(0); @@ -328,7 +328,7 @@ public void testExtractorInvalidParameter() throws Exception { final Statement statement = connection.createStatement()) { statement.execute(p2); fail(); - } catch (SQLException ignored) { + } catch (final SQLException ignored) { } assertPipeCount(0); @@ -351,7 +351,7 @@ public void testExtractorInvalidParameter() throws Exception { final Statement statement = connection.createStatement()) { statement.execute(p3); fail(); - } catch (SQLException ignored) { + } catch (final SQLException ignored) { } assertPipeCount(0); @@ -372,7 +372,7 @@ public void testExtractorInvalidParameter() throws Exception { final Statement statement = connection.createStatement()) { statement.execute(p4); fail(); - } catch (SQLException ignored) { + } catch (final SQLException ignored) { } assertPipeCount(0); } @@ -447,6 +447,14 @@ public void testExtractorPatternMatch() throws Exception { Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode()); Assert.assertEquals( TSStatusCode.SUCCESS_STATUS.getStatusCode(), client.startPipe("p" + i).getCode()); + // We add flush here because the pipe may be created on the new IoT leader + // and the old leader's data may come as an unclosed historical tsfile + // and is skipped flush when the pipe starts. In this case, the "waitForTsFileClose()" + // may not return until a flush is executed, namely the data transfer relies + // on a flush operation. + if (!TestUtils.tryExecuteNonQueryWithRetry(senderEnv, "flush")) { + return; + } assertTimeseriesCountOnReceiver(receiverEnv, expectedTimeseriesCount.get(i)); } From 8562eead2bc54da50fc48fa4f49f75de93145506 Mon Sep 17 00:00:00 2001 From: Alan Choo <43991780+HeimingZ@users.noreply.github.com> Date: Fri, 17 May 2024 14:23:35 +0800 Subject: [PATCH 09/42] add remote storage block for tiered storage (#12535) --- .../storageengine/dataregion/tsfile/TsFileResourceBlockType.java | 1 + 1 file changed, 1 insertion(+) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResourceBlockType.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResourceBlockType.java index 0a599659500a..a45073a1cc89 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResourceBlockType.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResourceBlockType.java @@ -27,6 +27,7 @@ public enum TsFileResourceBlockType { EMPTY_BLOCK((byte) 0), PROGRESS_INDEX((byte) 1), + REMOTE_STORAGE_BLOCK((byte) 2), ; private final byte type; From 95708409a01591630307538618755723cc106ef4 Mon Sep 17 00:00:00 2001 From: William Song <48054931+SzyWilliam@users.noreply.github.com> Date: Sat, 18 May 2024 00:01:10 +0800 Subject: [PATCH 10/42] [RatisConsensus] Use empty map for create peer (#12463) * address review issues * use empty list for createPeer in data region * haha --- .../impl/DataNodeInternalRPCServiceImpl.java | 19 +++++++++++++++++-- 1 file changed, 17 insertions(+), 2 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/DataNodeInternalRPCServiceImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/DataNodeInternalRPCServiceImpl.java index a46158aafb0b..c575ca0781df 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/DataNodeInternalRPCServiceImpl.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/DataNodeInternalRPCServiceImpl.java @@ -53,6 +53,7 @@ import org.apache.iotdb.commons.trigger.TriggerInformation; import org.apache.iotdb.commons.udf.UDFInformation; import org.apache.iotdb.commons.udf.service.UDFManagementService; +import org.apache.iotdb.consensus.ConsensusFactory; import org.apache.iotdb.consensus.common.Peer; import org.apache.iotdb.consensus.exception.ConsensusException; import org.apache.iotdb.consensus.exception.ConsensusGroupAlreadyExistException; @@ -2155,9 +2156,23 @@ private TSStatus createNewRegionPeer(ConsensusGroupId regionId, List peers TSStatus status = new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode()); try { if (regionId instanceof DataRegionId) { - DataRegionConsensusImpl.getInstance().createLocalPeer(regionId, peers); + // RatisConsensus requires an empty peer list during region transition + final List createPeers = + ConsensusFactory.RATIS_CONSENSUS.equals( + IoTDBDescriptor.getInstance().getConfig().getDataRegionConsensusProtocolClass()) + ? Collections.emptyList() + : peers; + DataRegionConsensusImpl.getInstance().createLocalPeer(regionId, createPeers); } else { - SchemaRegionConsensusImpl.getInstance().createLocalPeer(regionId, peers); + // RatisConsensus requires an empty peer list during region transition + final List createPeers = + ConsensusFactory.RATIS_CONSENSUS.equals( + IoTDBDescriptor.getInstance() + .getConfig() + .getSchemaRegionConsensusProtocolClass()) + ? Collections.emptyList() + : peers; + SchemaRegionConsensusImpl.getInstance().createLocalPeer(regionId, createPeers); } } catch (ConsensusException e) { if (!(e instanceof ConsensusGroupAlreadyExistException)) { From 9f92b1ae8293af05759984f5bfa781412c931d9a Mon Sep 17 00:00:00 2001 From: William Song <48054931+SzyWilliam@users.noreply.github.com> Date: Sat, 18 May 2024 00:01:47 +0800 Subject: [PATCH 11/42] bump ratis 3.1.0-d09cbac-SNAPSHOT (#12547) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index c05397697cd1..2c2e15f406bc 100644 --- a/pom.xml +++ b/pom.xml @@ -140,7 +140,7 @@ is for ensuring the SNAPSHOT will stay available. We should however have the Ratis folks do a new release soon, as releasing with this version is more than sub-ideal. --> - 3.1.0-611b80a-SNAPSHOT + 3.1.0-d09cbac-SNAPSHOT 1.0.4 1.1.13 3.5.10 From 81382d71c0538de3e48fca4a27d895eaec08747a Mon Sep 17 00:00:00 2001 From: Haonan Date: Sat, 18 May 2024 11:13:00 +0800 Subject: [PATCH 12/42] Fix client cpp IT warning (#12548) --- .github/workflows/client-cpp.yml | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/.github/workflows/client-cpp.yml b/.github/workflows/client-cpp.yml index e110a90ff3e2..a213aeb5f5e0 100644 --- a/.github/workflows/client-cpp.yml +++ b/.github/workflows/client-cpp.yml @@ -45,15 +45,14 @@ jobs: if: matrix.os == 'ubuntu-latest' shell: bash run: sudo apt-get update && sudo apt-get install libboost-all-dev - - name: Install CPP Dependencies (Mac)` + - name: Install CPP Dependencies (Mac) if: matrix.os == 'macos-latest' shell: bash run: | brew install boost brew install bison - echo 'export PATH="/usr/local/opt/bison/bin:$PATH"' >> ~/.bash_profile - source ~/.bash_profile && export LDFLAGS="-L/usr/local/opt/bison/lib" - brew install openssl + echo 'export PATH="/opt/homebrew/opt/bison/bin:$PATH"' >> ~/.bash_profile + source ~/.bash_profile && export LDFLAGS="-L/opt/homebrew/opt/bison/lib" - name: Cache Maven packages uses: actions/cache@v4 with: From 9f61a4e2aca6988404e604cbf63da3c9aeb8cd59 Mon Sep 17 00:00:00 2001 From: Christofer Dutz Date: Mon, 20 May 2024 03:31:58 +0200 Subject: [PATCH 13/42] Feat/jenkins auto redeploy (#12542) --- Jenkinsfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/Jenkinsfile b/Jenkinsfile index f8ef4f77371d..db45da55e184 100644 --- a/Jenkinsfile +++ b/Jenkinsfile @@ -169,7 +169,7 @@ pipeline { steps { echo 'Deploying' // Deploy the artifacts using the wagon-maven-plugin. - sh 'mvn -f jenkins.pom -X -P deploy-snapshots wagon:upload -P get-jar-with-dependencies' + sh 'until mvn -f jenkins.pom -X -P deploy-snapshots wagon:upload || (( count++ >= 5 )); do echo "Retrying to deploy"; done' } } From e1ea758f173e7c6e6d896ff56e57e3b9c01dc17c Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Mon, 20 May 2024 12:35:32 +0800 Subject: [PATCH 14/42] ODBC: Add example to the example module (#12544) Co-authored-by: Steve Yurong Su --- example/odbc/.gitignore | 344 ++++++++++++++++++ example/odbc/ODBC-Example.sln | 35 ++ example/odbc/ODBC-Example/ODBC-Example.csproj | 40 ++ example/odbc/ODBC-Example/ODBC_Example.cs | 84 +++++ example/odbc/README.md | 81 +++++ 5 files changed, 584 insertions(+) create mode 100644 example/odbc/.gitignore create mode 100644 example/odbc/ODBC-Example.sln create mode 100644 example/odbc/ODBC-Example/ODBC-Example.csproj create mode 100644 example/odbc/ODBC-Example/ODBC_Example.cs create mode 100644 example/odbc/README.md diff --git a/example/odbc/.gitignore b/example/odbc/.gitignore new file mode 100644 index 000000000000..badd8dc039a1 --- /dev/null +++ b/example/odbc/.gitignore @@ -0,0 +1,344 @@ +## Ignore Visual Studio temporary files, build results, and +## files generated by popular Visual Studio add-ons. +## +## Get latest from https://github.com/github/gitignore/blob/master/VisualStudio.gitignore + +# User-specific files +*.rsuser +*.suo +*.user +*.userosscache +*.sln.docstates + +# User-specific files (MonoDevelop/Xamarin Studio) +*.userprefs + +# Mono auto generated files +mono_crash.* + +# Build results +[Dd]ebug/ +[Dd]ebugPublic/ +[Rr]elease/ +[Rr]eleases/ +x64/ +x86/ +[Aa][Rr][Mm]/ +[Aa][Rr][Mm]64/ +bld/ +[Bb]in/ +[Oo]bj/ +[Ll]og/ + +# Visual Studio 2015/2017 cache/options directory +.vs/ +# Uncomment if you have tasks that create the project's static files in wwwroot +#wwwroot/ + +# Visual Studio 2017 auto generated files +Generated\ Files/ + +# MSTest test Results +[Tt]est[Rr]esult*/ +[Bb]uild[Ll]og.* + +# NUNIT +*.VisualState.xml +TestResult.xml + +# Build Results of an ATL Project +[Dd]ebugPS/ +[Rr]eleasePS/ +dlldata.c + +# Benchmark Results +BenchmarkDotNet.Artifacts/ + +# .NET Core +project.lock.json +project.fragment.lock.json +artifacts/ + +# StyleCop +StyleCopReport.xml + +# Files built by Visual Studio +*_i.c +*_p.c +*_h.h +*.ilk +*.meta +*.obj +*.iobj +*.pch +*.pdb +*.ipdb +*.pgc +*.pgd +*.rsp +*.sbr +*.tlb +*.tli +*.tlh +*.tmp +*.tmp_proj +*_wpftmp.csproj +*.log +*.vspscc +*.vssscc +.builds +*.pidb +*.svclog +*.scc + +# Chutzpah Test files +_Chutzpah* + +# Visual C++ cache files +ipch/ +*.aps +*.ncb +*.opendb +*.opensdf +*.sdf +*.cachefile +*.VC.db +*.VC.VC.opendb + +# Visual Studio profiler +*.psess +*.vsp +*.vspx +*.sap + +# Visual Studio Trace Files +*.e2e + +# TFS 2012 Local Workspace +$tf/ + +# Guidance Automation Toolkit +*.gpState + +# ReSharper is a .NET coding add-in +_ReSharper*/ +*.[Rr]e[Ss]harper +*.DotSettings.user + +# JustCode is a .NET coding add-in +.JustCode + +# TeamCity is a build add-in +_TeamCity* + +# DotCover is a Code Coverage Tool +*.dotCover + +# AxoCover is a Code Coverage Tool +.axoCover/* +!.axoCover/settings.json + +# Visual Studio code coverage results +*.coverage +*.coveragexml + +# NCrunch +_NCrunch_* +.*crunch*.local.xml +nCrunchTemp_* + +# MightyMoose +*.mm.* +AutoTest.Net/ + +# Web workbench (sass) +.sass-cache/ + +# Installshield output folder +[Ee]xpress/ + +# DocProject is a documentation generator add-in +DocProject/buildhelp/ +DocProject/Help/*.HxT +DocProject/Help/*.HxC +DocProject/Help/*.hhc +DocProject/Help/*.hhk +DocProject/Help/*.hhp +DocProject/Help/Html2 +DocProject/Help/html + +# Click-Once directory +publish/ + +# Publish Web Output +*.[Pp]ublish.xml +*.azurePubxml +# Note: Comment the next line if you want to checkin your web deploy settings, +# but database connection strings (with potential passwords) will be unencrypted +*.pubxml +*.publishproj + +# Microsoft Azure Web App publish settings. Comment the next line if you want to +# checkin your Azure Web App publish settings, but sensitive information contained +# in these scripts will be unencrypted +PublishScripts/ + +# NuGet Packages +*.nupkg +# The packages folder can be ignored because of Package Restore +**/[Pp]ackages/* +# except build/, which is used as an MSBuild target. +!**/[Pp]ackages/build/ +# Uncomment if necessary however generally it will be regenerated when needed +#!**/[Pp]ackages/repositories.config +# NuGet v3's project.json files produces more ignorable files +*.nuget.props +*.nuget.targets + +# Microsoft Azure Build Output +csx/ +*.build.csdef + +# Microsoft Azure Emulator +ecf/ +rcf/ + +# Windows Store app package directories and files +AppPackages/ +BundleArtifacts/ +Package.StoreAssociation.xml +_pkginfo.txt +*.appx +*.appxbundle +*.appxupload + +# Visual Studio cache files +# files ending in .cache can be ignored +*.[Cc]ache +# but keep track of directories ending in .cache +!?*.[Cc]ache/ + +# Others +ClientBin/ +~$* +*~ +*.dbmdl +*.dbproj.schemaview +*.jfm +*.pfx +*.publishsettings +orleans.codegen.cs + +# Including strong name files can present a security risk +# (https://github.com/github/gitignore/pull/2483#issue-259490424) +#*.snk + +# Since there are multiple workflows, uncomment next line to ignore bower_components +# (https://github.com/github/gitignore/pull/1529#issuecomment-104372622) +#bower_components/ + +# RIA/Silverlight projects +Generated_Code/ + +# Backup & report files from converting an old project file +# to a newer Visual Studio version. Backup files are not needed, +# because we have git ;-) +_UpgradeReport_Files/ +Backup*/ +UpgradeLog*.XML +UpgradeLog*.htm +ServiceFabricBackup/ +*.rptproj.bak + +# SQL Server files +*.mdf +*.ldf +*.ndf + +# Business Intelligence projects +*.rdl.data +*.bim.layout +*.bim_*.settings +*.rptproj.rsuser +*- Backup*.rdl + +# Microsoft Fakes +FakesAssemblies/ + +# GhostDoc plugin setting file +*.GhostDoc.xml + +# Node.js Tools for Visual Studio +.ntvs_analysis.dat +node_modules/ + +# Visual Studio 6 build log +*.plg + +# Visual Studio 6 workspace options file +*.opt + +# Visual Studio 6 auto-generated workspace file (contains which files were open etc.) +*.vbw + +# Visual Studio LightSwitch build output +**/*.HTMLClient/GeneratedArtifacts +**/*.DesktopClient/GeneratedArtifacts +**/*.DesktopClient/ModelManifest.xml +**/*.Server/GeneratedArtifacts +**/*.Server/ModelManifest.xml +_Pvt_Extensions + +# Paket dependency manager +.paket/paket.exe +paket-files/ + +# FAKE - F# Make +.fake/ + +# CodeRush personal settings +.cr/personal + +# Python Tools for Visual Studio (PTVS) +__pycache__/ +*.pyc + +# Cake - Uncomment if you are using it +# tools/** +# !tools/packages.config + +# Tabs Studio +*.tss + +# Telerik's JustMock configuration file +*.jmconfig + +# BizTalk build output +*.btp.cs +*.btm.cs +*.odx.cs +*.xsd.cs + +# OpenCover UI analysis results +OpenCover/ + +# Azure Stream Analytics local run output +ASALocalRun/ + +# MSBuild Binary and Structured Log +*.binlog + +# NVidia Nsight GPU debugger configuration file +*.nvuser + +# MFractors (Xamarin productivity tool) working folder +.mfractor/ + +# Local History for Visual Studio +.localhistory/ + +# BeatPulse healthcheck temp database +healthchecksdb + +# Backup folder for Package Reference Convert tool in Visual Studio 2017 +MigrationBackup/ diff --git a/example/odbc/ODBC-Example.sln b/example/odbc/ODBC-Example.sln new file mode 100644 index 000000000000..cd65a9117c08 --- /dev/null +++ b/example/odbc/ODBC-Example.sln @@ -0,0 +1,35 @@ +Microsoft Visual Studio Solution File, Format Version 12.00 + +# +# 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 +# +# https://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. +# + +Project("{FAE04EC0-301F-11D3-BF4B-00C04F79EFBC}") = "ODBC-Example", "ODBC-Example\ODBC-Example.csproj", "{BFB210E1-8AB2-4E10-A8EB-41DD7016EE8D}" +EndProject +Global + GlobalSection(SolutionConfigurationPlatforms) = preSolution + Debug|Any CPU = Debug|Any CPU + Release|Any CPU = Release|Any CPU + EndGlobalSection + GlobalSection(ProjectConfigurationPlatforms) = postSolution + {BFB210E1-8AB2-4E10-A8EB-41DD7016EE8D}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {BFB210E1-8AB2-4E10-A8EB-41DD7016EE8D}.Debug|Any CPU.Build.0 = Debug|Any CPU + {BFB210E1-8AB2-4E10-A8EB-41DD7016EE8D}.Release|Any CPU.ActiveCfg = Release|Any CPU + {BFB210E1-8AB2-4E10-A8EB-41DD7016EE8D}.Release|Any CPU.Build.0 = Release|Any CPU + EndGlobalSection +EndGlobal diff --git a/example/odbc/ODBC-Example/ODBC-Example.csproj b/example/odbc/ODBC-Example/ODBC-Example.csproj new file mode 100644 index 000000000000..65a483e02c22 --- /dev/null +++ b/example/odbc/ODBC-Example/ODBC-Example.csproj @@ -0,0 +1,40 @@ + + + + + org.apache.iotdb.odbc + true + Apache IoTDB + The Apache Software Foundation + https://iotdb.apache.org + 1.3.2-SNAPSHOT + IoTDB: Example: ODBC + https://www.apache.org/licenses/LICENSE-2.0 + Exe + net8.0 + enable + enable + + + + + + + diff --git a/example/odbc/ODBC-Example/ODBC_Example.cs b/example/odbc/ODBC-Example/ODBC_Example.cs new file mode 100644 index 000000000000..5d7c76d41abd --- /dev/null +++ b/example/odbc/ODBC-Example/ODBC_Example.cs @@ -0,0 +1,84 @@ +// +// 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 +// +// https://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. +// + +using System.Data.Odbc; + +// Get a connection +var dbConnection = new OdbcConnection("DSN=ZappySys JDBC Bridge"); +dbConnection.Open(); + +// Execute the write commands to prepare data +var dbCommand = dbConnection.CreateCommand(); +dbCommand.CommandText = "insert into root.Keller.Flur.Energieversorgung(time, s1) values(1715670861634, 1)"; +dbCommand.ExecuteNonQuery(); +dbCommand.CommandText = "insert into root.Keller.Flur.Energieversorgung(time, s2) values(1715670861634, true)"; +dbCommand.ExecuteNonQuery(); +dbCommand.CommandText = "insert into root.Keller.Flur.Energieversorgung(time, s3) values(1715670861634, 3.1)"; +dbCommand.ExecuteNonQuery(); + +// Execute the read command +dbCommand.CommandText = "SELECT * FROM root.Keller.Flur.Energieversorgung"; +var dbReader = dbCommand.ExecuteReader(); + +// Write the output header +var fCount = dbReader.FieldCount; +Console.Write(":"); +for(var i = 0; i < fCount; i++) +{ + var fName = dbReader.GetName(i); + Console.Write(fName + ":"); +} +Console.WriteLine(); + +// Output the content +while (dbReader.Read()) +{ + Console.Write(":"); + for(var i = 0; i < fCount; i++) + { + var fieldType = dbReader.GetFieldType(i); + switch (fieldType.Name) + { + case "DateTime": + var dateTime = dbReader.GetInt64(i); + Console.Write(dateTime + ":"); + break; + case "Double": + if (dbReader.IsDBNull(i)) + { + Console.Write("null:"); + } + else + { + var fValue = dbReader.GetDouble(i); + Console.Write(fValue + ":"); + } + break; + default: + Console.Write(fieldType.Name + ":"); + break; + } + } + Console.WriteLine(); +} + +// Shut down gracefully +dbReader.Close(); +dbCommand.Dispose(); +dbConnection.Close(); diff --git a/example/odbc/README.md b/example/odbc/README.md new file mode 100644 index 000000000000..5f62ce6953cb --- /dev/null +++ b/example/odbc/README.md @@ -0,0 +1,81 @@ + + +# ODBC +With IoTDB JDBC, IoTDB can be accessed using the ODBC-JDBC bridge. The example program in "odbc" can write and read data through ODBC using the bridge. + +## Dependencies +* IoTDB-JDBC's jar-with-dependency package +* ODBC-JDBC bridge (e.g. ZappySys JDBC Bridge) + +## Deployment +### Preparing JDBC package +Download the source code of IoTDB, and execute the following command in root directory: +```shell +mvn clean package -pl iotdb-client/jdbc -am -DskipTests -P get-jar-with-dependencies +``` +Then, you can see the output `iotdb-jdbc-1.3.2-SNAPSHOT-jar-with-dependencies.jar` under `iotdb-client/jdbc/target` directory. + +### Preparing ODBC-JDBC Bridge +*Note: Here we only provide one kind of ODBC-JDBC bridge as the instance. Readers can use other ODBC-JDBC bridges to access IoTDB with the IOTDB-JDBC.* +1. **Download Zappy-Sys ODBC-JDBC Bridge**: + Enter the https://zappysys.com/products/odbc-powerpack/odbc-jdbc-bridge-driver/ website, and click "download". + + ![ZappySys_website.jpg](https://alioss.timecho.com/upload/ZappySys_website.jpg) + +2. **Prepare IoTDB**: Set up IoTDB cluster, and write a row of data arbitrarily. + ```sql + IoTDB > insert into root.ln.wf02.wt02(timestamp,status) values(1,true) + ``` + +3. **Deploy and Test the Bridge**: + 1. Open ODBC Data Sources(32/64 bit), depending on the bits of Windows. One possible position is `C:\ProgramData\Microsoft\Windows\Start Menu\Programs\Administrative Tools`. + + ![ODBC_ADD_EN.jpg](https://alioss.timecho.com/upload/ODBC_ADD_EN.jpg) + + 3. Click on "add" and select ZappySys JDBC Bridge. + + ![ODBC_CREATE_EN.jpg](https://alioss.timecho.com/upload/ODBC_CREATE_EN.jpg) + + 5. Fill in the following settings: + + | Property | Content | Example | + |---------------------|-----------------------------------------------------------|--------------------------------------------------------------------------------------------------------------------| + | Connection String | jdbc:iotdb://\:\/ | jdbc:iotdb://127.0.0.1:6667/ | + | Driver Class | org.apache.iotdb.jdbc.IoTDBDriver | org.apache.iotdb.jdbc.IoTDBDriver | + | JDBC driver file(s) | The path of IoTDB JDBC jar-with-dependencies | C:\Users\13361\Documents\GitHub\iotdb\iotdb-client\jdbc\target\iotdb-jdbc-1.3.2-SNAPSHOT-jar-with-dependencies.jar | + | User name | IoTDB's user name | root | + | User password | IoTDB's password | root | + + ![ODBC_CONNECTION.png](https://alioss.timecho.com/upload/ODBC_CONNECTION.png) + + 6. Click on "Test Connection" button, and a "Test Connection: SUCCESSFUL" should appear. + + ![ODBC_CONFIG_EN.jpg](https://alioss.timecho.com/upload/ODBC_CONFIG_EN.jpg) + + 7. Click the "Preview" button above, and replace the original query text with `select * from root.**`, then click "Preview Data", and the query result should correctly. + + ![ODBC_TEST.jpg](https://alioss.timecho.com/upload/ODBC_TEST.jpg) + +## Operate IoTDB's data with ODBC example +After correct deployment, you can use the example program to operate IoTDB's data. + +This program can write data into IoTDB, and query the data we have just written. You can directly open the "odbc-example" with JetBrains Rider and run it. From f9292455bcea284a7cefa462402b4c85b1cec496 Mon Sep 17 00:00:00 2001 From: Potato Date: Mon, 20 May 2024 14:03:34 +0800 Subject: [PATCH 15/42] Fixed deadlock problem caused by concurrent auto create schema and delete database (#12489) Signed-off-by: OneSizeFitQuorum --- .../java/org/apache/iotdb/confignode/manager/ConfigManager.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java index 1be83c2beba0..0c407afec2f4 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java @@ -606,7 +606,7 @@ public DataSet getMatchedDatabaseSchemas(GetDatabasePlan getDatabaseReq) { } @Override - public synchronized TSStatus setDatabase(DatabaseSchemaPlan databaseSchemaPlan) { + public TSStatus setDatabase(DatabaseSchemaPlan databaseSchemaPlan) { TSStatus status = confirmLeader(); if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { return clusterSchemaManager.setDatabase(databaseSchemaPlan, false); From 318718dad2805d5e66b6957a52907b15630507e2 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 20 May 2024 14:19:46 +0800 Subject: [PATCH 16/42] Bump org.jacoco:jacoco-maven-plugin from 0.8.10 to 0.8.12 (#12457) Bumps [org.jacoco:jacoco-maven-plugin](https://github.com/jacoco/jacoco) from 0.8.10 to 0.8.12. - [Release notes](https://github.com/jacoco/jacoco/releases) - [Commits](https://github.com/jacoco/jacoco/compare/v0.8.10...v0.8.12) --- updated-dependencies: - dependency-name: org.jacoco:jacoco-maven-plugin dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 2c2e15f406bc..ad35b59689f6 100644 --- a/pom.xml +++ b/pom.xml @@ -1907,7 +1907,7 @@ org.jacoco jacoco-maven-plugin - 0.8.10 + 0.8.12 com.coderplus.maven.plugins From d81972227d9722b3565b4afc6fa94fc9214e0f9c Mon Sep 17 00:00:00 2001 From: Haonan Date: Mon, 20 May 2024 16:18:46 +0800 Subject: [PATCH 17/42] Check duplicated measurements in one row for all insert APIs --- .../session/it/IoTDBSessionSimpleIT.java | 134 ++++++++++++++++++ .../plan/analyze/AnalyzeVisitor.java | 14 +- .../queryengine/plan/parser/ASTVisitor.java | 6 +- .../statement/crud/InsertBaseStatement.java | 17 ++- .../crud/InsertMultiTabletsStatement.java | 7 + .../statement/crud/InsertRowStatement.java | 11 ++ .../crud/InsertRowsOfOneDeviceStatement.java | 7 + .../statement/crud/InsertRowsStatement.java | 7 + .../plan/statement/crud/InsertStatement.java | 35 +---- .../statement/crud/InsertTabletStatement.java | 11 ++ 10 files changed, 206 insertions(+), 43 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/session/it/IoTDBSessionSimpleIT.java b/integration-test/src/test/java/org/apache/iotdb/session/it/IoTDBSessionSimpleIT.java index e9c1d05d3a61..0a89ee56ded1 100644 --- a/integration-test/src/test/java/org/apache/iotdb/session/it/IoTDBSessionSimpleIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/session/it/IoTDBSessionSimpleIT.java @@ -652,6 +652,34 @@ public void insertTabletWithWrongTimestampPrecisionTest() { } } + @Test + @Category({LocalStandaloneIT.class, ClusterIT.class}) + public void insertTabletWithDuplicatedMeasurementsTest() { + try (ISession session = EnvFactory.getEnv().getSessionConnection()) { + List schemaList = new ArrayList<>(); + schemaList.add(new MeasurementSchema("s0", TSDataType.DOUBLE, TSEncoding.RLE)); + schemaList.add(new MeasurementSchema("s0", TSDataType.DOUBLE, TSEncoding.RLE)); + schemaList.add(new MeasurementSchema("s0", TSDataType.DOUBLE, TSEncoding.RLE)); + + Tablet tablet = new Tablet("root.sg1.d1", schemaList); + for (long time = 0L; time < 10L; time++) { + int rowIndex = tablet.rowSize++; + tablet.addTimestamp(rowIndex, time); + + tablet.addValue(schemaList.get(0).getMeasurementId(), rowIndex, (double) time); + tablet.addValue(schemaList.get(1).getMeasurementId(), rowIndex, (double) time); + tablet.addValue(schemaList.get(2).getMeasurementId(), rowIndex, (double) time); + } + + if (tablet.rowSize != 0) { + session.insertTablet(tablet); + tablet.reset(); + } + } catch (Exception e) { + Assert.assertTrue(e.getMessage().contains("Insertion contains duplicated measurement: s0")); + } + } + @Test @Category({LocalStandaloneIT.class, ClusterIT.class}) public void createTimeSeriesWithDoubleTicksTest() { @@ -983,6 +1011,112 @@ public void insertOneDeviceRecordsWithIncorrectOrderTest() { } } + @Test + @Category({LocalStandaloneIT.class, ClusterIT.class}) + public void insertOneDeviceRecordsWithDuplicatedMeasurementsTest() { + try (ISession session = EnvFactory.getEnv().getSessionConnection()) { + List times = new ArrayList<>(); + List> measurements = new ArrayList<>(); + List> datatypes = new ArrayList<>(); + List> values = new ArrayList<>(); + + addLine( + times, + measurements, + datatypes, + values, + 3L, + "s1", + "s2", + TSDataType.INT32, + TSDataType.INT32, + 1, + 2); + addLine( + times, + measurements, + datatypes, + values, + 2L, + "s2", + "s2", + TSDataType.INT32, + TSDataType.INT32, + 3, + 4); + addLine( + times, + measurements, + datatypes, + values, + 1L, + "s4", + "s5", + TSDataType.FLOAT, + TSDataType.BOOLEAN, + 5.0f, + Boolean.TRUE); + session.insertRecordsOfOneDevice("root.sg.d1", times, measurements, datatypes, values); + } catch (Exception e) { + Assert.assertTrue(e.getMessage().contains("Insertion contains duplicated measurement: s2")); + } + } + + @Test + @Category({LocalStandaloneIT.class, ClusterIT.class}) + public void insertRecordsWithDuplicatedMeasurementsTest() { + try (ISession session = EnvFactory.getEnv().getSessionConnection()) { + List times = new ArrayList<>(); + List> measurements = new ArrayList<>(); + List> datatypes = new ArrayList<>(); + List> values = new ArrayList<>(); + List devices = new ArrayList<>(); + + devices.add("root.sg.d1"); + addLine( + times, + measurements, + datatypes, + values, + 3L, + "s1", + "s2", + TSDataType.INT32, + TSDataType.INT32, + 1, + 2); + devices.add("root.sg.d2"); + addLine( + times, + measurements, + datatypes, + values, + 2L, + "s2", + "s2", + TSDataType.INT32, + TSDataType.INT32, + 3, + 4); + devices.add("root.sg.d3"); + addLine( + times, + measurements, + datatypes, + values, + 1L, + "s4", + "s5", + TSDataType.FLOAT, + TSDataType.BOOLEAN, + 5.0f, + Boolean.TRUE); + session.insertRecords(devices, times, measurements, datatypes, values); + } catch (Exception e) { + Assert.assertTrue(e.getMessage().contains("Insertion contains duplicated measurement: s2")); + } + } + @Test @Category({LocalStandaloneIT.class, ClusterIT.class}) public void insertStringRecordsOfOneDeviceWithOrderTest() { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/AnalyzeVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/AnalyzeVisitor.java index a7448a935299..15d05e3430bc 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/AnalyzeVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/AnalyzeVisitor.java @@ -2212,7 +2212,6 @@ private void checkAliasUniqueness( @Override public Analysis visitInsert(InsertStatement insertStatement, MPPQueryContext context) { context.setQueryType(QueryType.WRITE); - insertStatement.semanticCheck(); long[] timeArray = insertStatement.getTimes(); PartialPath devicePath = insertStatement.getDevice(); String[] measurementList = insertStatement.getMeasurementList(); @@ -2223,9 +2222,7 @@ public Analysis visitInsert(InsertStatement insertStatement, MPPQueryContext con insertRowStatement.setTime(timeArray[0]); insertRowStatement.setMeasurements(measurementList); insertRowStatement.setDataTypes(new TSDataType[measurementList.length]); - Object[] values = new Object[measurementList.length]; - System.arraycopy(insertStatement.getValuesList().get(0), 0, values, 0, values.length); - insertRowStatement.setValues(values); + insertRowStatement.setValues(insertStatement.getValuesList().get(0)); insertRowStatement.setNeedInferType(true); insertRowStatement.setAligned(insertStatement.isAligned()); return insertRowStatement.accept(this, context); @@ -2257,9 +2254,7 @@ public Analysis visitInsert(InsertStatement insertStatement, MPPQueryContext con statement.setTime(timeArray[i]); TSDataType[] dataTypes = new TSDataType[measurementList.length]; statement.setDataTypes(dataTypes); - Object[] values = new Object[measurementList.length]; - System.arraycopy(insertStatement.getValuesList().get(i), 0, values, 0, values.length); - statement.setValues(values); + statement.setValues(insertStatement.getValuesList().get(i)); statement.setAligned(insertStatement.isAligned()); statement.setNeedInferType(true); insertRowStatementList.add(statement); @@ -2541,6 +2536,7 @@ public Analysis visitAlterTimeseries( public Analysis visitInsertTablet( InsertTabletStatement insertTabletStatement, MPPQueryContext context) { context.setQueryType(QueryType.WRITE); + insertTabletStatement.semanticCheck(); Analysis analysis = new Analysis(); validateSchema(analysis, insertTabletStatement, context); InsertBaseStatement realStatement = removeLogicalView(analysis, insertTabletStatement); @@ -2572,6 +2568,7 @@ public Analysis visitInsertTablet( @Override public Analysis visitInsertRow(InsertRowStatement insertRowStatement, MPPQueryContext context) { context.setQueryType(QueryType.WRITE); + insertRowStatement.semanticCheck(); Analysis analysis = new Analysis(); validateSchema(analysis, insertRowStatement, context); InsertBaseStatement realInsertStatement = removeLogicalView(analysis, insertRowStatement); @@ -2622,6 +2619,7 @@ private Analysis computeAnalysisForInsertRows( public Analysis visitInsertRows( InsertRowsStatement insertRowsStatement, MPPQueryContext context) { context.setQueryType(QueryType.WRITE); + insertRowsStatement.semanticCheck(); Analysis analysis = new Analysis(); validateSchema(analysis, insertRowsStatement, context); InsertRowsStatement realInsertRowsStatement = @@ -2661,6 +2659,7 @@ private Analysis computeAnalysisForMultiTablets( public Analysis visitInsertMultiTablets( InsertMultiTabletsStatement insertMultiTabletsStatement, MPPQueryContext context) { context.setQueryType(QueryType.WRITE); + insertMultiTabletsStatement.semanticCheck(); Analysis analysis = new Analysis(); validateSchema(analysis, insertMultiTabletsStatement, context); InsertMultiTabletsStatement realStatement = @@ -2678,6 +2677,7 @@ public Analysis visitInsertMultiTablets( public Analysis visitInsertRowsOfOneDevice( InsertRowsOfOneDeviceStatement insertRowsOfOneDeviceStatement, MPPQueryContext context) { context.setQueryType(QueryType.WRITE); + insertRowsOfOneDeviceStatement.semanticCheck(); Analysis analysis = new Analysis(); validateSchema(analysis, insertRowsOfOneDeviceStatement, context); InsertBaseStatement realInsertStatement = diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/parser/ASTVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/parser/ASTVisitor.java index 3d3efc1b8c3c..201b7e7302f9 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/parser/ASTVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/parser/ASTVisitor.java @@ -1885,13 +1885,13 @@ private void parseInsertValuesSpec( if (timeIndex == -1 && rows.size() != 1) { throw new SemanticException("need timestamps when insert multi rows"); } - List valuesList = new ArrayList<>(); + List valuesList = new ArrayList<>(); long[] timeArray = new long[rows.size()]; for (int i = 0, size = rows.size(); i < size; i++) { IoTDBSqlParser.RowContext row = rows.get(i); // parse timestamp long timestamp; - List valueList = new ArrayList<>(); + List valueList = new ArrayList<>(); // using now() instead if (timeIndex == -1) { timestamp = CommonDateTimeUtils.currentTime(); @@ -1913,7 +1913,7 @@ private void parseInsertValuesSpec( } } - valuesList.add(valueList.toArray(new String[0])); + valuesList.add(valueList.toArray(new Object[0])); } insertStatement.setTimes(timeArray); insertStatement.setValuesList(valuesList); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertBaseStatement.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertBaseStatement.java index 605f72fd831f..10f7b03dfc0e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertBaseStatement.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertBaseStatement.java @@ -199,6 +199,21 @@ protected void selfCheckDataTypes(int index) public abstract Object getFirstValueOfIndex(int index); + public void semanticCheck() { + Set deduplicatedMeasurements = new HashSet<>(); + for (String measurement : measurements) { + if (measurement == null || measurement.isEmpty()) { + throw new SemanticException( + "Measurement contains null or empty string: " + Arrays.toString(measurements)); + } + if (deduplicatedMeasurements.contains(measurement)) { + throw new SemanticException("Insertion contains duplicated measurement: " + measurement); + } else { + deduplicatedMeasurements.add(measurement); + } + } + } + // region partial insert /** * Mark failed measurement, measurements[index], dataTypes[index] and values/columns[index] would @@ -310,7 +325,7 @@ protected Map>> getMapFromDeviceToMeasur } } // construct map from device to measurements and record the index of its measurement - // schemaengine + // schema Map>> mapFromDeviceToMeasurementAndIndex = new HashMap<>(); for (int i = 0; i < this.measurements.length; i++) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertMultiTabletsStatement.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertMultiTabletsStatement.java index bbc00e1975bf..6601af629475 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertMultiTabletsStatement.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertMultiTabletsStatement.java @@ -110,6 +110,13 @@ protected boolean checkAndCastDataType(int columnIndex, TSDataType dataType) { return false; } + @Override + public void semanticCheck() { + for (InsertTabletStatement insertTabletStatement : insertTabletStatementList) { + insertTabletStatement.semanticCheck(); + } + } + @Override public long getMinTime() { throw new NotImplementedException(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertRowStatement.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertRowStatement.java index e6403c385770..d5a4cf765e6c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertRowStatement.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertRowStatement.java @@ -255,6 +255,17 @@ public void markFailedMeasurement(int index, Exception cause) { values[index] = null; } + @Override + public void semanticCheck() { + super.semanticCheck(); + if (measurements.length != values.length) { + throw new SemanticException( + String.format( + "the measurementList's size %d is not consistent with the valueList's size %d", + measurements.length, values.length)); + } + } + public boolean isNeedSplit() { return hasLogicalViewNeedProcess(); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertRowsOfOneDeviceStatement.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertRowsOfOneDeviceStatement.java index 9725e54c3816..e67ef7c65c2d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertRowsOfOneDeviceStatement.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertRowsOfOneDeviceStatement.java @@ -129,6 +129,13 @@ protected boolean checkAndCastDataType(int columnIndex, TSDataType dataType) { return false; } + @Override + public void semanticCheck() { + for (InsertRowStatement insertRowStatement : insertRowStatementList) { + insertRowStatement.semanticCheck(); + } + } + @Override public long getMinTime() { throw new NotImplementedException(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertRowsStatement.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertRowsStatement.java index c29610d25f4d..669165798bc2 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertRowsStatement.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertRowsStatement.java @@ -128,6 +128,13 @@ protected boolean checkAndCastDataType(int columnIndex, TSDataType dataType) { return false; } + @Override + public void semanticCheck() { + for (InsertRowStatement insertRowStatement : insertRowStatementList) { + insertRowStatement.semanticCheck(); + } + } + @Override public long getMinTime() { throw new NotImplementedException(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertStatement.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertStatement.java index 3e0a99140306..f74e8682344f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertStatement.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertStatement.java @@ -23,17 +23,13 @@ import org.apache.iotdb.commons.auth.entity.PrivilegeType; import org.apache.iotdb.commons.path.PartialPath; import org.apache.iotdb.db.auth.AuthorityChecker; -import org.apache.iotdb.db.exception.sql.SemanticException; import org.apache.iotdb.db.queryengine.plan.statement.Statement; import org.apache.iotdb.db.queryengine.plan.statement.StatementType; import org.apache.iotdb.db.queryengine.plan.statement.StatementVisitor; import org.apache.iotdb.rpc.TSStatusCode; import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashSet; import java.util.List; -import java.util.Set; /** this class extends {@code Statement} and process insert statement. */ public class InsertStatement extends Statement { @@ -43,7 +39,7 @@ public class InsertStatement extends Statement { private long[] times; private String[] measurementList; - private List valuesList; + private List valuesList; private boolean isAligned; @@ -90,11 +86,11 @@ public void setMeasurementList(String[] measurementList) { this.measurementList = measurementList; } - public List getValuesList() { + public List getValuesList() { return valuesList; } - public void setValuesList(List valuesList) { + public void setValuesList(List valuesList) { this.valuesList = valuesList; } @@ -118,29 +114,4 @@ public void setAligned(boolean aligned) { public R accept(StatementVisitor visitor, C context) { return visitor.visitInsert(this, context); } - - public void semanticCheck() { - Set deduplicatedMeasurements = new HashSet<>(); - for (String measurement : measurementList) { - if (measurement == null || measurement.isEmpty()) { - throw new SemanticException( - "Measurement contains null or empty string: " + Arrays.toString(measurementList)); - } - if (deduplicatedMeasurements.contains(measurement)) { - throw new SemanticException("Insertion contains duplicated measurement: " + measurement); - } else { - deduplicatedMeasurements.add(measurement); - } - } - - int measurementsNum = measurementList.length; - for (int i = 0; i < times.length; i++) { - if (measurementsNum != valuesList.get(i).length) { - throw new SemanticException( - String.format( - "the measurementList's size %d is not consistent with the valueList's size %d", - measurementsNum, valuesList.get(i).length)); - } - } - } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertTabletStatement.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertTabletStatement.java index 8d9bdf8d9713..a5dd5fbf511f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertTabletStatement.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertTabletStatement.java @@ -194,6 +194,17 @@ public void markFailedMeasurement(int index, Exception cause) { columns[index] = null; } + @Override + public void semanticCheck() { + super.semanticCheck(); + if (measurements.length != columns.length) { + throw new SemanticException( + String.format( + "the measurementList's size %d is not consistent with the columnList's size %d", + measurements.length, columns.length)); + } + } + public boolean isNeedSplit() { return hasLogicalViewNeedProcess(); } From 337bfe5b891c10c6c2c61b914817647b1609fec4 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Mon, 20 May 2024 16:32:06 +0800 Subject: [PATCH 18/42] Pipe IT: Fix IoTDBPipeClusterIT#testCreatePipesWithSameConnector() fails because the thread is interrupted or encountered non-pipe exceptions (#12553) --- .../it/autocreate/IoTDBPipeClusterIT.java | 55 ++++++++++--------- 1 file changed, 29 insertions(+), 26 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeClusterIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeClusterIT.java index 37e860906ad9..c7e0289adc51 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeClusterIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeClusterIT.java @@ -107,7 +107,7 @@ public void testWithAllParametersInHybridMode() throws Exception { testWithAllParameters("hybrid"); } - public void testWithAllParameters(String realtimeMode) throws Exception { + public void testWithAllParameters(final String realtimeMode) throws Exception { final DataNodeWrapper receiverDataNode = receiverEnv.getDataNodeWrapper(0); final String receiverIp = receiverDataNode.getIp(); @@ -226,18 +226,18 @@ public void testPipeAfterDataRegionLeaderStop() throws Exception { leaderIndex = i; try { senderEnv.shutdownDataNode(i); - } catch (Exception e) { + } catch (final Exception e) { e.printStackTrace(); return; } try { TimeUnit.SECONDS.sleep(1); - } catch (InterruptedException ignored) { + } catch (final InterruptedException ignored) { } try { senderEnv.startDataNode(i); ((AbstractEnv) senderEnv).checkClusterStatusWithoutUnknown(); - } catch (Exception e) { + } catch (final Exception e) { e.printStackTrace(); return; } @@ -338,7 +338,7 @@ public void testPipeAfterRegisterNewDataNode() throws Exception { try { senderEnv.registerNewDataNode(true); - } catch (Exception e) { + } catch (final Exception e) { e.printStackTrace(); return; } @@ -425,21 +425,21 @@ public void testCreatePipeWhenRegisteringNewDataNode() throws Exception { new TCreatePipeReq("p" + i, connectorAttributes) .setExtractorAttributes(extractorAttributes) .setProcessorAttributes(processorAttributes)); - } catch (TException e) { + } catch (final TException e) { // Not sure if the "createPipe" has succeeded e.printStackTrace(); return; } try { Thread.sleep(100); - } catch (Exception ignored) { + } catch (final Exception ignored) { } } }); t.start(); try { senderEnv.registerNewDataNode(true); - } catch (Exception e) { + } catch (final Exception e) { e.printStackTrace(); return; } @@ -494,13 +494,13 @@ public void testRegisteringNewDataNodeWhenTransferringData() throws Exception { } Thread.sleep(100); } - } catch (InterruptedException ignored) { + } catch (final InterruptedException ignored) { } }); t.start(); try { senderEnv.registerNewDataNode(true); - } catch (Exception e) { + } catch (final Exception e) { e.printStackTrace(); return; } @@ -518,7 +518,7 @@ public void testRegisteringNewDataNodeWhenTransferringData() throws Exception { try { senderEnv.shutdownDataNode(senderEnv.getDataNodeWrapperList().size() - 1); senderEnv.getDataNodeWrapperList().remove(senderEnv.getDataNodeWrapperList().size() - 1); - } catch (Exception e) { + } catch (final Exception e) { e.printStackTrace(); } } @@ -562,7 +562,7 @@ public void testRegisteringNewDataNodeAfterTransferringData() throws Exception { try { senderEnv.registerNewDataNode(true); - } catch (Exception e) { + } catch (final Exception e) { e.printStackTrace(); return; } @@ -580,7 +580,7 @@ public void testRegisteringNewDataNodeAfterTransferringData() throws Exception { try { senderEnv.shutdownDataNode(senderEnv.getDataNodeWrapperList().size() - 1); senderEnv.getDataNodeWrapperList().remove(senderEnv.getDataNodeWrapperList().size() - 1); - } catch (Exception e) { + } catch (final Exception e) { e.printStackTrace(); } } @@ -634,7 +634,7 @@ public void testNewDataNodeFailureParallelToTransferringData() throws Exception senderEnv.shutdownDataNode(senderEnv.getDataNodeWrapperList().size() - 1); senderEnv.getDataNodeWrapperList().remove(senderEnv.getDataNodeWrapperList().size() - 1); ((AbstractEnv) senderEnv).checkClusterStatusWithoutUnknown(); - } catch (Exception e) { + } catch (final Exception e) { e.printStackTrace(); return; } @@ -735,11 +735,11 @@ public void testConcurrentlyCreatePipeOfSameName() throws Exception { if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { successCount.incrementAndGet(); } - } catch (InterruptedException e) { + } catch (final InterruptedException e) { Thread.currentThread().interrupt(); - } catch (TException | ClientManagerException | IOException e) { + } catch (final TException | ClientManagerException | IOException e) { e.printStackTrace(); - } catch (Exception e) { + } catch (final Exception e) { // Fail iff pipe exception occurs e.printStackTrace(); fail(e.getMessage()); @@ -765,11 +765,11 @@ public void testConcurrentlyCreatePipeOfSameName() throws Exception { if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { successCount.incrementAndGet(); } - } catch (InterruptedException e) { + } catch (final InterruptedException e) { Thread.currentThread().interrupt(); - } catch (TException | ClientManagerException | IOException e) { + } catch (final TException | ClientManagerException | IOException e) { e.printStackTrace(); - } catch (Exception e) { + } catch (final Exception e) { // Fail iff pipe exception occurs e.printStackTrace(); fail(e.getMessage()); @@ -806,7 +806,7 @@ public void testCreate100PipesWithSameConnector() throws Exception { testCreatePipesWithSameConnector(100); } - private void testCreatePipesWithSameConnector(int pipeCount) throws Exception { + private void testCreatePipesWithSameConnector(final int pipeCount) throws Exception { final DataNodeWrapper receiverDataNode = receiverEnv.getDataNodeWrapper(0); final String receiverIp = receiverDataNode.getIp(); @@ -821,6 +821,7 @@ private void testCreatePipesWithSameConnector(int pipeCount) throws Exception { connectorAttributes.put("connector.ip", receiverIp); connectorAttributes.put("connector.port", Integer.toString(receiverPort)); + final AtomicInteger successCount = new AtomicInteger(0); final List threads = new ArrayList<>(); for (int i = 0; i < pipeCount; ++i) { final int finalI = i; @@ -836,11 +837,13 @@ private void testCreatePipesWithSameConnector(int pipeCount) throws Exception { .setProcessorAttributes(processorAttributes)); Assert.assertEquals( TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode()); - } catch (InterruptedException e) { + successCount.incrementAndGet(); + } catch (final InterruptedException e) { + e.printStackTrace(); Thread.currentThread().interrupt(); - } catch (TException | ClientManagerException | IOException e) { + } catch (final TException | ClientManagerException | IOException e) { e.printStackTrace(); - } catch (Exception e) { + } catch (final Exception e) { // Fail iff pipe exception occurs e.printStackTrace(); fail(e.getMessage()); @@ -856,10 +859,10 @@ private void testCreatePipesWithSameConnector(int pipeCount) throws Exception { try (final SyncConfigNodeIServiceClient client = (SyncConfigNodeIServiceClient) senderEnv.getLeaderConfigNodeConnection()) { List showPipeResult = client.showPipe(new TShowPipeReq()).pipeInfoList; - Assert.assertEquals(pipeCount, showPipeResult.size()); + Assert.assertEquals(successCount.get(), showPipeResult.size()); showPipeResult = client.showPipe(new TShowPipeReq().setPipeName("p1").setWhereClause(true)).pipeInfoList; - Assert.assertEquals(pipeCount, showPipeResult.size()); + Assert.assertEquals(successCount.get(), showPipeResult.size()); } } From 70f18ccc4e28083dd751453555c3b5bab90869d2 Mon Sep 17 00:00:00 2001 From: Zhijia Cao Date: Mon, 20 May 2024 17:00:25 +0800 Subject: [PATCH 19/42] Load Tsfile Tool: load does not delete the source tsfile by default (#12549) --- .../queryengine/plan/statement/crud/LoadTsFileStatement.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/LoadTsFileStatement.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/LoadTsFileStatement.java index ad015cdaccca..06fa870aee6d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/LoadTsFileStatement.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/LoadTsFileStatement.java @@ -53,7 +53,7 @@ public LoadTsFileStatement(String filePath) throws FileNotFoundException { this.file = new File(filePath); this.databaseLevel = IoTDBDescriptor.getInstance().getConfig().getDefaultStorageGroupLevel(); this.verifySchema = true; - this.deleteAfterLoad = true; + this.deleteAfterLoad = false; this.autoCreateDatabase = IoTDBDescriptor.getInstance().getConfig().isAutoCreateSchemaEnabled(); this.tsFiles = new ArrayList<>(); this.resources = new ArrayList<>(); @@ -78,7 +78,7 @@ protected LoadTsFileStatement() { this.file = null; this.databaseLevel = IoTDBDescriptor.getInstance().getConfig().getDefaultStorageGroupLevel(); this.verifySchema = true; - this.deleteAfterLoad = true; + this.deleteAfterLoad = false; this.autoCreateDatabase = IoTDBDescriptor.getInstance().getConfig().isAutoCreateSchemaEnabled(); this.tsFiles = new ArrayList<>(); this.resources = new ArrayList<>(); From 9d564e336183eac5bae5ef0f8281525430add690 Mon Sep 17 00:00:00 2001 From: ppppoooo <1214343862@qq.com> Date: Mon, 20 May 2024 19:38:07 +0800 Subject: [PATCH 20/42] Pipe: implemented ChangingValueSamplingProcessor (#12466) Co-authored-by: Steve Yurong Su --- .../PipeDataRegionProcessorConstructor.java | 4 + .../changing/ChangingValueFilter.java | 95 +++++++++ .../ChangingValueSamplingProcessor.java | 199 ++++++++++++++++++ .../sdt/SwingingDoorTrendingFilter.java | 4 +- .../constant/PipeProcessorConstant.java | 11 + .../plugin/builtin/BuiltinPipePlugin.java | 4 + .../ChangingValueSamplingProcessor.java | 30 +++ 7 files changed, 346 insertions(+), 1 deletion(-) create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/downsampling/changing/ChangingValueFilter.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/downsampling/changing/ChangingValueSamplingProcessor.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/plugin/builtin/processor/downsampling/ChangingValueSamplingProcessor.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/plugin/dataregion/PipeDataRegionProcessorConstructor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/plugin/dataregion/PipeDataRegionProcessorConstructor.java index 15578f2fc2a3..6d747d156a2b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/plugin/dataregion/PipeDataRegionProcessorConstructor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/plugin/dataregion/PipeDataRegionProcessorConstructor.java @@ -27,6 +27,7 @@ import org.apache.iotdb.db.pipe.processor.aggregate.AggregateProcessor; import org.apache.iotdb.db.pipe.processor.aggregate.operator.processor.StandardStatisticsOperatorProcessor; import org.apache.iotdb.db.pipe.processor.aggregate.window.processor.TumblingWindowingProcessor; +import org.apache.iotdb.db.pipe.processor.downsampling.changing.ChangingValueSamplingProcessor; import org.apache.iotdb.db.pipe.processor.downsampling.sdt.SwingingDoorTrendingSamplingProcessor; import org.apache.iotdb.db.pipe.processor.downsampling.tumbling.TumblingTimeSamplingProcessor; import org.apache.iotdb.db.pipe.processor.twostage.plugin.TwoStageCountProcessor; @@ -47,6 +48,9 @@ protected void initConstructors() { pluginConstructors.put( BuiltinPipePlugin.SDT_SAMPLING_PROCESSOR.getPipePluginName(), SwingingDoorTrendingSamplingProcessor::new); + pluginConstructors.put( + BuiltinPipePlugin.CHANGING_VALUE_SAMPLING_PROCESSOR.getPipePluginName(), + ChangingValueSamplingProcessor::new); pluginConstructors.put( BuiltinPipePlugin.THROWING_EXCEPTION_PROCESSOR.getPipePluginName(), ThrowingExceptionProcessor::new); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/downsampling/changing/ChangingValueFilter.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/downsampling/changing/ChangingValueFilter.java new file mode 100644 index 000000000000..cc83fbeb21ec --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/downsampling/changing/ChangingValueFilter.java @@ -0,0 +1,95 @@ +/* + * 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.db.pipe.processor.downsampling.changing; + +import org.apache.iotdb.pipe.api.type.Binary; + +import java.util.Objects; + +public class ChangingValueFilter { + + private final ChangingValueSamplingProcessor processor; + + /** + * The last stored time and value we compare current point against lastReadTimestamp and + * lastReadValue + */ + private long lastStoredTimestamp; + + private T lastStoredValue; + + public ChangingValueFilter( + ChangingValueSamplingProcessor processor, long firstTimestamp, T firstValue) { + this.processor = processor; + init(firstTimestamp, firstValue); + } + + private void init(long firstTimestamp, T firstValue) { + lastStoredTimestamp = firstTimestamp; + lastStoredValue = firstValue; + } + + public boolean filter(long timestamp, T value) { + try { + return tryFilter(timestamp, value); + } catch (Exception e) { + init(timestamp, value); + return true; + } + } + + private boolean tryFilter(long timestamp, T value) { + final long timeDiff = Math.abs(timestamp - lastStoredTimestamp); + + if (timeDiff <= processor.getCompressionMinTimeInterval()) { + return false; + } + + if (timeDiff >= processor.getCompressionMaxTimeInterval()) { + reset(timestamp, value); + return true; + } + + // For boolean and string type, we only compare the value + if (value instanceof Boolean || value instanceof String || value instanceof Binary) { + if (Objects.equals(lastStoredValue, value)) { + return false; + } + + reset(timestamp, value); + return true; + } + + // For other numerical types, we compare the value difference + if (Math.abs( + Double.parseDouble(lastStoredValue.toString()) - Double.parseDouble(value.toString())) + > processor.getCompressionDeviation()) { + reset(timestamp, value); + return true; + } + + return false; + } + + private void reset(long timestamp, T value) { + lastStoredTimestamp = timestamp; + lastStoredValue = value; + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/downsampling/changing/ChangingValueSamplingProcessor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/downsampling/changing/ChangingValueSamplingProcessor.java new file mode 100644 index 000000000000..6badb70755ba --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/downsampling/changing/ChangingValueSamplingProcessor.java @@ -0,0 +1,199 @@ +/* + * 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.db.pipe.processor.downsampling.changing; + +import org.apache.iotdb.commons.pipe.config.constant.PipeProcessorConstant; +import org.apache.iotdb.db.pipe.event.common.row.PipeRemarkableRow; +import org.apache.iotdb.db.pipe.event.common.row.PipeRow; +import org.apache.iotdb.db.pipe.processor.downsampling.DownSamplingProcessor; +import org.apache.iotdb.db.pipe.processor.downsampling.PartialPathLastObjectCache; +import org.apache.iotdb.pipe.api.access.Row; +import org.apache.iotdb.pipe.api.collector.RowCollector; +import org.apache.iotdb.pipe.api.customizer.configuration.PipeProcessorRuntimeConfiguration; +import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameterValidator; +import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameters; + +import org.apache.tsfile.common.constant.TsFileConstant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.concurrent.atomic.AtomicReference; + +public class ChangingValueSamplingProcessor extends DownSamplingProcessor { + + private static final Logger LOGGER = + LoggerFactory.getLogger(ChangingValueSamplingProcessor.class); + + /** + * The maximum absolute difference the user set if the data's value is within + * compressionDeviation, it will be compressed and discarded after compression + */ + private double compressionDeviation; + + /** + * The minimum time distance between two stored data points if current point time to the last + * stored point time distance <= compressionMinTimeInterval, current point will NOT be stored + * regardless of compression deviation + */ + private long compressionMinTimeInterval; + + /** + * The maximum time distance between two stored data points if current point time to the last + * stored point time distance >= compressionMaxTimeInterval, current point will be stored + * regardless of compression deviation + */ + private long compressionMaxTimeInterval; + + private PartialPathLastObjectCache> pathLastObjectCache; + + @Override + public void validate(PipeParameterValidator validator) throws Exception { + super.validate(validator); + + final PipeParameters parameters = validator.getParameters(); + compressionDeviation = + parameters.getDoubleOrDefault( + PipeProcessorConstant.PROCESSOR_CHANGING_VALUE_COMPRESSION_DEVIATION, + PipeProcessorConstant.PROCESSOR_CHANGING_VALUE_COMPRESSION_DEVIATION_DEFAULT_VALUE); + compressionMinTimeInterval = + parameters.getLongOrDefault( + PipeProcessorConstant.PROCESSOR_CHANGING_VALUE_MIN_TIME_INTERVAL_KEY, + PipeProcessorConstant.PROCESSOR_CHANGING_VALUE_MIN_TIME_INTERVAL_DEFAULT_VALUE); + compressionMaxTimeInterval = + parameters.getLongOrDefault( + PipeProcessorConstant.PROCESSOR_CHANGING_VALUE_MAX_TIME_INTERVAL_KEY, + PipeProcessorConstant.PROCESSOR_CHANGING_VALUE_MAX_TIME_INTERVAL_DEFAULT_VALUE); + + validator + .validate( + compressionDeviation -> (Double) compressionDeviation >= 0, + String.format( + "%s must be >= 0, but got %s", + PipeProcessorConstant.PROCESSOR_CHANGING_VALUE_COMPRESSION_DEVIATION, + compressionDeviation), + compressionDeviation) + .validate( + compressionMinTimeInterval -> (Long) compressionMinTimeInterval >= 0, + String.format( + "%s must be >= 0, but got %s", + PipeProcessorConstant.PROCESSOR_CHANGING_VALUE_MIN_TIME_INTERVAL_KEY, + compressionMinTimeInterval), + compressionMinTimeInterval) + .validate( + compressionMaxTimeInterval -> (Long) compressionMaxTimeInterval >= 0, + String.format( + "%s must be >= 0, but got %s", + PipeProcessorConstant.PROCESSOR_CHANGING_VALUE_MAX_TIME_INTERVAL_KEY, + compressionMaxTimeInterval), + compressionMaxTimeInterval) + .validate( + minMaxPair -> (Long) minMaxPair[0] <= (Long) minMaxPair[1], + String.format( + "%s must be <= %s, but got %s and %s", + PipeProcessorConstant.PROCESSOR_CHANGING_VALUE_MIN_TIME_INTERVAL_KEY, + PipeProcessorConstant.PROCESSOR_CHANGING_VALUE_MAX_TIME_INTERVAL_KEY, + compressionMinTimeInterval, + compressionMaxTimeInterval), + compressionMinTimeInterval, + compressionMaxTimeInterval); + } + + @Override + public void customize( + PipeParameters parameters, PipeProcessorRuntimeConfiguration configuration) { + super.customize(parameters, configuration); + + LOGGER.info( + "ChangingValueSamplingProcessor in {} is initialized with {}: {}, {}: {}, {}: {}.", + dataBaseNameWithPathSeparator, + PipeProcessorConstant.PROCESSOR_CHANGING_VALUE_COMPRESSION_DEVIATION, + compressionDeviation, + PipeProcessorConstant.PROCESSOR_CHANGING_VALUE_MIN_TIME_INTERVAL_KEY, + compressionMinTimeInterval, + PipeProcessorConstant.PROCESSOR_CHANGING_VALUE_MAX_TIME_INTERVAL_KEY, + compressionMaxTimeInterval); + } + + @Override + protected PartialPathLastObjectCache initPathLastObjectCache(long memoryLimitInBytes) { + pathLastObjectCache = + new PartialPathLastObjectCache>(memoryLimitInBytes) { + @Override + protected long calculateMemoryUsage(ChangingValueFilter object) { + return 64; // Long.BYTES * 8 + } + }; + return pathLastObjectCache; + } + + @Override + protected void processRow( + Row row, + RowCollector rowCollector, + String deviceSuffix, + AtomicReference exception) { + final PipeRemarkableRow remarkableRow = new PipeRemarkableRow((PipeRow) row); + + boolean hasNonNullMeasurements = false; + for (int i = 0, size = row.size(); i < size; i++) { + if (row.isNull(i)) { + continue; + } + + final String timeSeriesSuffix = + deviceSuffix + TsFileConstant.PATH_SEPARATOR + row.getColumnName(i); + final ChangingValueFilter filter = + pathLastObjectCache.getPartialPathLastObject(timeSeriesSuffix); + + if (filter != null) { + if (filter.filter(row.getTime(), row.getObject(i))) { + hasNonNullMeasurements = true; + } else { + remarkableRow.markNull(i); + } + } else { + hasNonNullMeasurements = true; + pathLastObjectCache.setPartialPathLastObject( + timeSeriesSuffix, new ChangingValueFilter<>(this, row.getTime(), row.getObject(i))); + } + } + + if (hasNonNullMeasurements) { + try { + rowCollector.collectRow(remarkableRow); + } catch (IOException e) { + exception.set(e); + } + } + } + + double getCompressionDeviation() { + return compressionDeviation; + } + + long getCompressionMinTimeInterval() { + return compressionMinTimeInterval; + } + + long getCompressionMaxTimeInterval() { + return compressionMaxTimeInterval; + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/downsampling/sdt/SwingingDoorTrendingFilter.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/downsampling/sdt/SwingingDoorTrendingFilter.java index cdbae8fef593..8d9e73deb070 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/downsampling/sdt/SwingingDoorTrendingFilter.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/downsampling/sdt/SwingingDoorTrendingFilter.java @@ -19,6 +19,8 @@ package org.apache.iotdb.db.pipe.processor.downsampling.sdt; +import org.apache.iotdb.pipe.api.type.Binary; + import java.util.Objects; public class SwingingDoorTrendingFilter { @@ -93,7 +95,7 @@ private boolean tryFilter(long timestamp, T value) { } // For boolean and string type, we only compare the value - if (value instanceof Boolean || value instanceof String) { + if (value instanceof Boolean || value instanceof String || value instanceof Binary) { if (Objects.equals(lastStoredValue, value)) { return false; } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/constant/PipeProcessorConstant.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/constant/PipeProcessorConstant.java index 49fa2e2cab87..22bc87b2917b 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/constant/PipeProcessorConstant.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/constant/PipeProcessorConstant.java @@ -66,6 +66,17 @@ public class PipeProcessorConstant { "processor.sdt.max-time-interval"; public static final long PROCESSOR_SDT_MAX_TIME_INTERVAL_DEFAULT_VALUE = Long.MAX_VALUE; + public static final String PROCESSOR_CHANGING_VALUE_COMPRESSION_DEVIATION = + "processor.changing-value.compression-deviation"; + public static final double PROCESSOR_CHANGING_VALUE_COMPRESSION_DEVIATION_DEFAULT_VALUE = 0; + public static final String PROCESSOR_CHANGING_VALUE_MIN_TIME_INTERVAL_KEY = + "processor.changing-value.min-time-interval"; + public static final long PROCESSOR_CHANGING_VALUE_MIN_TIME_INTERVAL_DEFAULT_VALUE = 0; + public static final String PROCESSOR_CHANGING_VALUE_MAX_TIME_INTERVAL_KEY = + "processor.changing-value.max-time-interval"; + public static final long PROCESSOR_CHANGING_VALUE_MAX_TIME_INTERVAL_DEFAULT_VALUE = + Long.MAX_VALUE; + public static final String PROCESSOR_OUTPUT_SERIES_KEY = "processor.output-series"; private PipeProcessorConstant() { diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/plugin/builtin/BuiltinPipePlugin.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/plugin/builtin/BuiltinPipePlugin.java index 511282ff3290..a6ae2e9fd382 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/plugin/builtin/BuiltinPipePlugin.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/plugin/builtin/BuiltinPipePlugin.java @@ -35,6 +35,7 @@ import org.apache.iotdb.commons.pipe.plugin.builtin.processor.aggregate.StandardStatisticsProcessor; import org.apache.iotdb.commons.pipe.plugin.builtin.processor.aggregate.TumblingWindowingProcessor; import org.apache.iotdb.commons.pipe.plugin.builtin.processor.donothing.DoNothingProcessor; +import org.apache.iotdb.commons.pipe.plugin.builtin.processor.downsampling.ChangingValueSamplingProcessor; import org.apache.iotdb.commons.pipe.plugin.builtin.processor.downsampling.SwingingDoorTrendingSamplingProcessor; import org.apache.iotdb.commons.pipe.plugin.builtin.processor.downsampling.TumblingTimeSamplingProcessor; import org.apache.iotdb.commons.pipe.plugin.builtin.processor.throwing.ThrowingExceptionProcessor; @@ -59,6 +60,8 @@ public enum BuiltinPipePlugin { TUMBLING_TIME_SAMPLING_PROCESSOR( "tumbling-time-sampling-processor", TumblingTimeSamplingProcessor.class), SDT_SAMPLING_PROCESSOR("sdt-sampling-processor", SwingingDoorTrendingSamplingProcessor.class), + CHANGING_VALUE_SAMPLING_PROCESSOR( + "changing-value-sampling-processor", ChangingValueSamplingProcessor.class), THROWING_EXCEPTION_PROCESSOR("throwing-exception-processor", ThrowingExceptionProcessor.class), AGGREGATE_PROCESSOR("aggregate-processor", AggregateProcessor.class), COUNT_POINT_PROCESSOR("count-point-processor", TwoStageCountProcessor.class), @@ -126,6 +129,7 @@ public String getClassName() { // Processors TUMBLING_TIME_SAMPLING_PROCESSOR.getPipePluginName().toUpperCase(), SDT_SAMPLING_PROCESSOR.getPipePluginName().toUpperCase(), + CHANGING_VALUE_SAMPLING_PROCESSOR.getPipePluginName().toUpperCase(), THROWING_EXCEPTION_PROCESSOR.getPipePluginName().toUpperCase(), AGGREGATE_PROCESSOR.getPipePluginName().toUpperCase(), STANDARD_STATISTICS_PROCESSOR.getPipePluginName().toUpperCase(), diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/plugin/builtin/processor/downsampling/ChangingValueSamplingProcessor.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/plugin/builtin/processor/downsampling/ChangingValueSamplingProcessor.java new file mode 100644 index 000000000000..a9b060ff6bae --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/plugin/builtin/processor/downsampling/ChangingValueSamplingProcessor.java @@ -0,0 +1,30 @@ +/* + * 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.pipe.plugin.builtin.processor.downsampling; + +import org.apache.iotdb.commons.pipe.plugin.builtin.processor.PlaceHolderProcessor; + +/** + * This class is a placeholder and should not be initialized. It represents the + * changing-value-sampling-processor. There is a real implementation in the server module but cannot + * be imported here. The pipe agent in the server module will replace this class with the real + * implementation when initializing the changing-value-sampling-processor. + */ +public class ChangingValueSamplingProcessor extends PlaceHolderProcessor {} From b0306420a1079f47a6743087773ba1ae4517d610 Mon Sep 17 00:00:00 2001 From: Steve Yurong Su Date: Mon, 20 May 2024 22:41:24 +0800 Subject: [PATCH 21/42] Pipe: Fix RPC payload compression in PipeTransferTsFileInsertionEventHandler (#12558) --- ...peTransferTsFileInsertionEventHandler.java | 23 ++++++++++--------- .../DataRegionWatermarkInjector.java | 2 +- 2 files changed, 13 insertions(+), 12 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/handler/PipeTransferTsFileInsertionEventHandler.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/handler/PipeTransferTsFileInsertionEventHandler.java index 07669de76bfe..8d02aeb8fa87 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/handler/PipeTransferTsFileInsertionEventHandler.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/handler/PipeTransferTsFileInsertionEventHandler.java @@ -123,10 +123,13 @@ public void transfer( } else if (currentFile == tsFile) { isSealSignalSent.set(true); client.pipeTransfer( - transferMod - ? PipeTransferTsFileSealWithModReq.toTPipeTransferReq( - modFile.getName(), modFile.length(), tsFile.getName(), tsFile.length()) - : PipeTransferTsFileSealReq.toTPipeTransferReq(tsFile.getName(), tsFile.length()), + PipeTransferCompressedReq.toTPipeTransferReq( + transferMod + ? PipeTransferTsFileSealWithModReq.toTPipeTransferReq( + modFile.getName(), modFile.length(), tsFile.getName(), tsFile.length()) + : PipeTransferTsFileSealReq.toTPipeTransferReq( + tsFile.getName(), tsFile.length()), + connector.getCompressors()), this); } return; @@ -137,13 +140,11 @@ public void transfer( ? readBuffer : Arrays.copyOfRange(readBuffer, 0, readLength); final TPipeTransferReq uncompressedReq = - PipeTransferCompressedReq.toTPipeTransferReq( - transferMod - ? PipeTransferTsFilePieceWithModReq.toTPipeTransferReq( - currentFile.getName(), position, payload) - : PipeTransferTsFilePieceReq.toTPipeTransferReq( - currentFile.getName(), position, payload), - connector.getCompressors()); + transferMod + ? PipeTransferTsFilePieceWithModReq.toTPipeTransferReq( + currentFile.getName(), position, payload) + : PipeTransferTsFilePieceReq.toTPipeTransferReq( + currentFile.getName(), position, payload); client.pipeTransfer( connector.isRpcCompressionEnabled() ? PipeTransferCompressedReq.toTPipeTransferReq( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/DataRegionWatermarkInjector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/DataRegionWatermarkInjector.java index 5d8bbf615aef..a8d95d448103 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/DataRegionWatermarkInjector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/DataRegionWatermarkInjector.java @@ -28,7 +28,7 @@ public class DataRegionWatermarkInjector { private static final Logger LOGGER = LoggerFactory.getLogger(DataRegionWatermarkInjector.class); - public static final long MIN_INJECTION_INTERVAL_IN_MS = 30 * 1000; // 30s + public static final long MIN_INJECTION_INTERVAL_IN_MS = 30 * 1000L; // 30s private final int regionId; From e4f4ed03113d3d4ac92a0bbb57736443713ea046 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Tue, 21 May 2024 14:27:58 +0800 Subject: [PATCH 22/42] Pipe: Added metrics to schema transfer / remaining event / remaining time (#12478) Co-authored-by: Steve Yurong Su --- .../pipe/it/manual/IoTDBPipeInclusionIT.java | 3 +- iotdb-core/confignode/pom.xml | 4 + .../pipe/execution/PipeConfigNodeSubtask.java | 52 ++-- .../extractor/IoTDBConfigRegionExtractor.java | 13 +- .../metric/PipeConfigNodeListenerMetrics.java | 67 +++++ .../pipe/metric/PipeConfigNodeMetrics.java | 14 +- .../PipeConfigNodeRemainingTimeMetrics.java | 169 +++++++++++++ .../PipeConfigNodeRemainingTimeOperator.java | 119 +++++++++ .../PipeConfigRegionConnectorMetrics.java | 157 ++++++++++++ .../PipeConfigRegionExtractorMetrics.java | 139 +++++++++++ iotdb-core/datanode/pom.xml | 4 + .../PipeSchemaRegionListenerManager.java | 44 ++-- .../agent/task/PipeDataNodeTaskAgent.java | 61 +++-- .../async/IoTDBDataRegionAsyncConnector.java | 22 ++ .../dataregion/IoTDBDataRegionExtractor.java | 36 ++- .../PipeRealtimeDataRegionExtractor.java | 28 ++- ...PipeRealtimeDataRegionHybridExtractor.java | 4 +- .../realtime/epoch/TsFileEpoch.java | 4 +- .../IoTDBSchemaRegionExtractor.java | 11 +- .../db/pipe/metric/PipeDataNodeMetrics.java | 20 +- ...eDataNodeRemainingEventAndTimeMetrics.java | 231 ++++++++++++++++++ ...DataNodeRemainingEventAndTimeOperator.java | 207 ++++++++++++++++ ...va => PipeDataRegionConnectorMetrics.java} | 91 +++---- ...va => PipeDataRegionExtractorMetrics.java} | 129 +++++----- .../PipeSchemaRegionConnectorMetrics.java | 159 ++++++++++++ .../PipeSchemaRegionExtractorMetrics.java | 144 +++++++++++ .../PipeSchemaRegionListenerMetrics.java | 135 ++++++++++ .../task/connection/PipeEventCollector.java | 7 +- .../connector/PipeConnectorSubtask.java | 84 +++++-- .../PipeConnectorSubtaskManager.java | 6 + .../processor/PipeProcessorSubtask.java | 14 +- .../core/reporter/IoTDBJmxReporter.java | 4 +- .../iotdb/metrics/core/type/IoTDBRate.java | 2 +- .../{uitls => utils}/IoTDBCachedGauge.java | 2 +- .../IoTDBCachedGaugeMBean.java | 2 +- .../IoTDBMetricObjNameFactory.java | 2 +- .../{uitls => utils}/IoTDBMovingAverage.java | 2 +- .../{uitls => utils}/ObjectNameFactory.java | 2 +- .../commons/client/ClientPoolFactory.java | 2 +- .../iotdb/commons/conf/CommonConfig.java | 16 +- .../iotdb/commons/conf/CommonDescriptor.java | 5 + .../pipe/agent/task/PipeTaskAgent.java | 75 +++--- .../iotdb/commons/pipe/config/PipeConfig.java | 7 + .../AbstractSerializableListeningQueue.java | 10 + .../pipe/event/PipeWritePlanEvent.java | 9 +- .../pipe/event/ProgressReportEvent.java | 83 +++++++ .../pipe/extractor/IoTDBExtractor.java | 23 +- .../IoTDBNonDataRegionExtractor.java | 49 ++-- .../pipe/metric/PipeEventCommitMetrics.java | 32 +-- .../pipe/progress/PipeEventCommitManager.java | 58 +++-- .../pipe/progress/PipeEventCommitter.java | 16 +- .../commons/pipe/task/meta/PipeTaskMeta.java | 2 +- .../task/subtask/PipeReportableSubtask.java | 2 +- .../commons/service/metric/enums/Metric.java | 9 + pom.xml | 6 +- 55 files changed, 2241 insertions(+), 357 deletions(-) create mode 100644 iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/metric/PipeConfigNodeListenerMetrics.java create mode 100644 iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/metric/PipeConfigNodeRemainingTimeMetrics.java create mode 100644 iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/metric/PipeConfigNodeRemainingTimeOperator.java create mode 100644 iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/metric/PipeConfigRegionConnectorMetrics.java create mode 100644 iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/metric/PipeConfigRegionExtractorMetrics.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeDataNodeRemainingEventAndTimeMetrics.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeDataNodeRemainingEventAndTimeOperator.java rename iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/{PipeConnectorMetrics.java => PipeDataRegionConnectorMetrics.java} (77%) rename iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/{PipeExtractorMetrics.java => PipeDataRegionExtractorMetrics.java} (72%) create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeSchemaRegionConnectorMetrics.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeSchemaRegionExtractorMetrics.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeSchemaRegionListenerMetrics.java rename iotdb-core/metrics/core/src/main/java/org/apache/iotdb/metrics/core/{uitls => utils}/IoTDBCachedGauge.java (98%) rename iotdb-core/metrics/core/src/main/java/org/apache/iotdb/metrics/core/{uitls => utils}/IoTDBCachedGaugeMBean.java (95%) rename iotdb-core/metrics/core/src/main/java/org/apache/iotdb/metrics/core/{uitls => utils}/IoTDBMetricObjNameFactory.java (98%) rename iotdb-core/metrics/core/src/main/java/org/apache/iotdb/metrics/core/{uitls => utils}/IoTDBMovingAverage.java (98%) rename iotdb-core/metrics/core/src/main/java/org/apache/iotdb/metrics/core/{uitls => utils}/ObjectNameFactory.java (96%) create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/event/ProgressReportEvent.java diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/IoTDBPipeInclusionIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/IoTDBPipeInclusionIT.java index 0b1f0f63cc2a..c27fa7f46c4f 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/IoTDBPipeInclusionIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/IoTDBPipeInclusionIT.java @@ -206,8 +206,7 @@ public void testPureDeleteInclusion() throws Exception { receiverEnv, "select * from root.**", "Time,root.ln.wf01.wt01.status1,", - Collections.emptySet(), - 10); + Collections.emptySet()); } } } diff --git a/iotdb-core/confignode/pom.xml b/iotdb-core/confignode/pom.xml index 1e007335c9e0..896799b3824f 100644 --- a/iotdb-core/confignode/pom.xml +++ b/iotdb-core/confignode/pom.xml @@ -99,6 +99,10 @@ tsfile ${tsfile.version} + + io.dropwizard.metrics + metrics-core + org.slf4j slf4j-api diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/execution/PipeConfigNodeSubtask.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/execution/PipeConfigNodeSubtask.java index fc7281410a29..220d0388be26 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/execution/PipeConfigNodeSubtask.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/execution/PipeConfigNodeSubtask.java @@ -31,6 +31,7 @@ import org.apache.iotdb.commons.pipe.task.subtask.PipeAbstractConnectorSubtask; import org.apache.iotdb.confignode.manager.pipe.agent.PipeConfigNodeAgent; import org.apache.iotdb.confignode.manager.pipe.extractor.IoTDBConfigRegionExtractor; +import org.apache.iotdb.confignode.manager.pipe.metric.PipeConfigRegionConnectorMetrics; import org.apache.iotdb.pipe.api.PipeExtractor; import org.apache.iotdb.pipe.api.PipeProcessor; import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameterValidator; @@ -60,15 +61,15 @@ public class PipeConfigNodeSubtask extends PipeAbstractConnectorSubtask { private PipeProcessor processor; public PipeConfigNodeSubtask( - String pipeName, - long creationTime, - Map extractorAttributes, - Map processorAttributes, - Map connectorAttributes, - PipeTaskMeta pipeTaskMeta) + final String pipeName, + final long creationTime, + final Map extractorAttributes, + final Map processorAttributes, + final Map connectorAttributes, + final PipeTaskMeta pipeTaskMeta) throws Exception { // We initialize outputPipeConnector by initConnector() - super(pipeName, creationTime, null); + super(pipeName + "_" + creationTime, creationTime, null); this.pipeName = pipeName; this.pipeTaskMeta = pipeTaskMeta; @@ -76,11 +77,12 @@ public PipeConfigNodeSubtask( initProcessor(processorAttributes); initConnector(connectorAttributes); + PipeConfigRegionConnectorMetrics.getInstance().register(this); PipeEventCommitManager.getInstance() .register(pipeName, creationTime, CONFIG_REGION_ID.getId(), pipeName + "_" + creationTime); } - private void initExtractor(Map extractorAttributes) throws Exception { + private void initExtractor(final Map extractorAttributes) throws Exception { final PipeParameters extractorParameters = new PipeParameters(extractorAttributes); // 1. Construct extractor @@ -94,7 +96,7 @@ private void initExtractor(Map extractorAttributes) throws Excep final PipeTaskRuntimeConfiguration runtimeConfiguration = new PipeTaskRuntimeConfiguration( new PipeTaskExtractorRuntimeEnvironment( - taskID, creationTime, CONFIG_REGION_ID.getId(), pipeTaskMeta)); + pipeName, creationTime, CONFIG_REGION_ID.getId(), pipeTaskMeta)); extractor.customize(extractorParameters, runtimeConfiguration); } catch (final Exception e) { try { @@ -109,12 +111,12 @@ private void initExtractor(Map extractorAttributes) throws Excep } } - private void initProcessor(Map processorAttributes) { + private void initProcessor(final Map processorAttributes) { final PipeParameters processorParameters = new PipeParameters(processorAttributes); final PipeTaskRuntimeConfiguration runtimeConfiguration = new PipeTaskRuntimeConfiguration( - new PipeTaskRuntimeEnvironment(taskID, creationTime, CONFIG_REGION_ID.getId())); + new PipeTaskRuntimeEnvironment(pipeName, creationTime, CONFIG_REGION_ID.getId())); processor = PipeConfigNodeAgent.plugin() @@ -126,7 +128,7 @@ private void initProcessor(Map processorAttributes) { runtimeConfiguration); } - private void initConnector(Map connectorAttributes) throws Exception { + private void initConnector(final Map connectorAttributes) throws Exception { final PipeParameters connectorParameters = new PipeParameters(connectorAttributes); // 1. Construct connector @@ -139,7 +141,7 @@ private void initConnector(Map connectorAttributes) throws Excep // 3. Customize connector final PipeTaskRuntimeConfiguration runtimeConfiguration = new PipeTaskRuntimeConfiguration( - new PipeTaskRuntimeEnvironment(taskID, creationTime, CONFIG_REGION_ID.getId())); + new PipeTaskRuntimeEnvironment(pipeName, creationTime, CONFIG_REGION_ID.getId())); outputPipeConnector.customize(connectorParameters, runtimeConfiguration); // 4. Handshake @@ -180,9 +182,10 @@ protected boolean executeOnce() throws Exception { } outputPipeConnector.transfer(event); - decreaseReferenceCountAndReleaseLastEvent(true); - } catch (PipeException e) { + + PipeConfigRegionConnectorMetrics.getInstance().markConfigEvent(taskID); + } catch (final PipeException e) { if (!isClosed.get()) { throw e; } else { @@ -192,7 +195,7 @@ protected boolean executeOnce() throws Exception { e); clearReferenceCountAndReleaseLastEvent(); } - } catch (Exception e) { + } catch (final Exception e) { if (!isClosed.get()) { throw new PipeException( String.format( @@ -213,22 +216,23 @@ public void close() { PipeEventCommitManager.getInstance() .deregister(pipeName, creationTime, CONFIG_REGION_ID.getId()); + PipeConfigRegionConnectorMetrics.getInstance().deregister(taskID); try { extractor.close(); - } catch (Exception e) { + } catch (final Exception e) { LOGGER.info("Error occurred during closing PipeExtractor.", e); } try { processor.close(); - } catch (Exception e) { + } catch (final Exception e) { LOGGER.info("Error occurred during closing PipeProcessor.", e); } try { outputPipeConnector.close(); - } catch (Exception e) { + } catch (final Exception e) { LOGGER.info("Error occurred during closing PipeConnector.", e); } finally { // Should be after connector.close() @@ -239,12 +243,18 @@ public void close() { //////////////////////////// Error report //////////////////////////// @Override - protected String getRootCause(Throwable throwable) { + protected String getRootCause(final Throwable throwable) { return throwable != null ? throwable.getMessage() : null; } @Override - protected void report(EnrichedEvent event, PipeRuntimeException exception) { + protected void report(final EnrichedEvent event, final PipeRuntimeException exception) { PipeConfigNodeAgent.runtime().report(event, exception); } + + //////////////////////////// APIs provided for metric framework //////////////////////////// + + public String getPipeName() { + return pipeName; + } } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/extractor/IoTDBConfigRegionExtractor.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/extractor/IoTDBConfigRegionExtractor.java index 3e996ca7dc21..a5eb544c95ae 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/extractor/IoTDBConfigRegionExtractor.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/extractor/IoTDBConfigRegionExtractor.java @@ -31,6 +31,8 @@ import org.apache.iotdb.confignode.manager.pipe.agent.PipeConfigNodeAgent; import org.apache.iotdb.confignode.manager.pipe.event.PipeConfigRegionSnapshotEvent; import org.apache.iotdb.confignode.manager.pipe.event.PipeConfigRegionWritePlanEvent; +import org.apache.iotdb.confignode.manager.pipe.metric.PipeConfigNodeRemainingTimeMetrics; +import org.apache.iotdb.confignode.manager.pipe.metric.PipeConfigRegionExtractorMetrics; import org.apache.iotdb.confignode.service.ConfigNode; import org.apache.iotdb.consensus.ConsensusFactory; import org.apache.iotdb.consensus.exception.ConsensusException; @@ -40,6 +42,7 @@ import org.apache.iotdb.pipe.api.exception.PipeException; import java.util.HashSet; +import java.util.Objects; import java.util.Set; public class IoTDBConfigRegionExtractor extends IoTDBNonDataRegionExtractor { @@ -61,6 +64,9 @@ public void customize( super.customize(parameters, configuration); listenedTypeSet = ConfigRegionListeningFilter.parseListeningPlanTypeSet(parameters); + + PipeConfigRegionExtractorMetrics.getInstance().register(this); + PipeConfigNodeRemainingTimeMetrics.getInstance().register(this); } @Override @@ -83,7 +89,7 @@ protected void triggerSnapshot() { .triggerSnapshot( new ConfigRegionId(ConfigNodeDescriptor.getInstance().getConf().getConfigRegionId()), true); - } catch (ConsensusException e) { + } catch (final ConsensusException e) { throw new PipeException("Exception encountered when triggering schema region snapshot.", e); } } @@ -125,5 +131,10 @@ public synchronized void close() throws Exception { return; } super.close(); + + if (Objects.nonNull(taskID)) { + PipeConfigRegionExtractorMetrics.getInstance().deregister(taskID); + PipeConfigNodeRemainingTimeMetrics.getInstance().deregister(pipeName + "_" + creationTime); + } } } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/metric/PipeConfigNodeListenerMetrics.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/metric/PipeConfigNodeListenerMetrics.java new file mode 100644 index 000000000000..554c89326ee9 --- /dev/null +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/metric/PipeConfigNodeListenerMetrics.java @@ -0,0 +1,67 @@ +/* + * 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.confignode.manager.pipe.metric; + +import org.apache.iotdb.commons.service.metric.enums.Metric; +import org.apache.iotdb.confignode.manager.pipe.agent.PipeConfigNodeAgent; +import org.apache.iotdb.confignode.manager.pipe.extractor.ConfigRegionListeningQueue; +import org.apache.iotdb.metrics.AbstractMetricService; +import org.apache.iotdb.metrics.metricsets.IMetricSet; +import org.apache.iotdb.metrics.utils.MetricLevel; +import org.apache.iotdb.metrics.utils.MetricType; + +public class PipeConfigNodeListenerMetrics implements IMetricSet { + + //////////////////////////// bindTo & unbindFrom (metric framework) //////////////////////////// + + @Override + public void bindTo(final AbstractMetricService metricService) { + metricService.createAutoGauge( + Metric.PIPE_CONFIG_LINKED_QUEUE_SIZE.toString(), + MetricLevel.IMPORTANT, + PipeConfigNodeAgent.runtime().listener(), + ConfigRegionListeningQueue::getSize); + } + + @Override + public void unbindFrom(final AbstractMetricService metricService) { + metricService.remove(MetricType.AUTO_GAUGE, Metric.PIPE_CONFIG_LINKED_QUEUE_SIZE.toString()); + } + + //////////////////////////// singleton //////////////////////////// + + private static class PipeConfigNodeListenerMetricsHolder { + + private static final PipeConfigNodeListenerMetrics INSTANCE = + new PipeConfigNodeListenerMetrics(); + + private PipeConfigNodeListenerMetricsHolder() { + // Empty constructor + } + } + + public static PipeConfigNodeListenerMetrics getInstance() { + return PipeConfigNodeListenerMetricsHolder.INSTANCE; + } + + private PipeConfigNodeListenerMetrics() { + // Empty constructor + } +} diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/metric/PipeConfigNodeMetrics.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/metric/PipeConfigNodeMetrics.java index a43a4033c0b9..d819ccc075e8 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/metric/PipeConfigNodeMetrics.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/metric/PipeConfigNodeMetrics.java @@ -27,21 +27,29 @@ public class PipeConfigNodeMetrics implements IMetricSet { private final PipeTaskInfoMetrics pipeTaskInfoMetrics; - public PipeConfigNodeMetrics(PipeManager pipeManager) { + public PipeConfigNodeMetrics(final PipeManager pipeManager) { this.pipeTaskInfoMetrics = new PipeTaskInfoMetrics(pipeManager); } //////////////////////////// bindTo & unbindFrom (metric framework) //////////////////////////// @Override - public void bindTo(AbstractMetricService metricService) { + public void bindTo(final AbstractMetricService metricService) { PipeProcedureMetrics.getInstance().bindTo(metricService); pipeTaskInfoMetrics.bindTo(metricService); + PipeConfigNodeListenerMetrics.getInstance().bindTo(metricService); + PipeConfigRegionExtractorMetrics.getInstance().bindTo(metricService); + PipeConfigRegionConnectorMetrics.getInstance().bindTo(metricService); + PipeConfigNodeRemainingTimeMetrics.getInstance().bindTo(metricService); } @Override - public void unbindFrom(AbstractMetricService metricService) { + public void unbindFrom(final AbstractMetricService metricService) { PipeProcedureMetrics.getInstance().unbindFrom(metricService); pipeTaskInfoMetrics.unbindFrom(metricService); + PipeConfigNodeListenerMetrics.getInstance().unbindFrom(metricService); + PipeConfigRegionExtractorMetrics.getInstance().unbindFrom(metricService); + PipeConfigRegionConnectorMetrics.getInstance().unbindFrom(metricService); + PipeConfigNodeRemainingTimeMetrics.getInstance().unbindFrom(metricService); } } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/metric/PipeConfigNodeRemainingTimeMetrics.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/metric/PipeConfigNodeRemainingTimeMetrics.java new file mode 100644 index 000000000000..0ef7b48d1c7e --- /dev/null +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/metric/PipeConfigNodeRemainingTimeMetrics.java @@ -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.confignode.manager.pipe.metric; + +import org.apache.iotdb.commons.pipe.config.plugin.env.PipeTaskRuntimeEnvironment; +import org.apache.iotdb.commons.pipe.progress.PipeEventCommitManager; +import org.apache.iotdb.commons.service.metric.enums.Metric; +import org.apache.iotdb.commons.service.metric.enums.Tag; +import org.apache.iotdb.confignode.manager.pipe.extractor.IoTDBConfigRegionExtractor; +import org.apache.iotdb.metrics.AbstractMetricService; +import org.apache.iotdb.metrics.metricsets.IMetricSet; +import org.apache.iotdb.metrics.utils.MetricLevel; +import org.apache.iotdb.metrics.utils.MetricType; + +import com.google.common.collect.ImmutableSet; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.ConcurrentHashMap; + +public class PipeConfigNodeRemainingTimeMetrics implements IMetricSet { + + private static final Logger LOGGER = + LoggerFactory.getLogger(PipeConfigNodeRemainingTimeMetrics.class); + + private volatile AbstractMetricService metricService; + + private final Map remainingTimeOperatorMap = + new ConcurrentHashMap<>(); + + //////////////////////////// bindTo & unbindFrom (metric framework) //////////////////////////// + + @Override + public void bindTo(final AbstractMetricService metricService) { + this.metricService = metricService; + ImmutableSet.copyOf(remainingTimeOperatorMap.keySet()).forEach(this::createMetrics); + } + + private void createMetrics(final String pipeID) { + createAutoGauge(pipeID); + } + + private void createAutoGauge(final String pipeID) { + final PipeConfigNodeRemainingTimeOperator operator = remainingTimeOperatorMap.get(pipeID); + metricService.createAutoGauge( + Metric.PIPE_CONFIGNODE_REMAINING_TIME.toString(), + MetricLevel.IMPORTANT, + operator, + PipeConfigNodeRemainingTimeOperator::getRemainingTime, + Tag.NAME.toString(), + operator.getPipeName(), + Tag.CREATION_TIME.toString(), + String.valueOf(operator.getCreationTime())); + } + + @Override + public void unbindFrom(final AbstractMetricService metricService) { + ImmutableSet.copyOf(remainingTimeOperatorMap.keySet()).forEach(this::deregister); + if (!remainingTimeOperatorMap.isEmpty()) { + LOGGER.warn( + "Failed to unbind from pipe remaining time metrics, RemainingTimeOperator map not empty"); + } + } + + private void removeMetrics(final String pipeID) { + removeAutoGauge(pipeID); + } + + private void removeAutoGauge(final String pipeID) { + final PipeConfigNodeRemainingTimeOperator operator = remainingTimeOperatorMap.get(pipeID); + metricService.remove( + MetricType.AUTO_GAUGE, + Metric.PIPE_CONFIGNODE_REMAINING_TIME.toString(), + Tag.NAME.toString(), + operator.getPipeName(), + Tag.CREATION_TIME.toString(), + String.valueOf(operator.getCreationTime())); + remainingTimeOperatorMap.remove(pipeID); + } + + //////////////////////////// register & deregister (pipe integration) //////////////////////////// + + public void register(final IoTDBConfigRegionExtractor extractor) { + // The metric is global thus the regionId is omitted + final String pipeID = extractor.getPipeName() + "_" + extractor.getCreationTime(); + remainingTimeOperatorMap + .computeIfAbsent(pipeID, k -> new PipeConfigNodeRemainingTimeOperator()) + .register(extractor); + if (Objects.nonNull(metricService)) { + createMetrics(pipeID); + } + } + + public void deregister(final String pipeID) { + if (!remainingTimeOperatorMap.containsKey(pipeID)) { + LOGGER.warn( + "Failed to deregister pipe remaining time metrics, RemainingTimeOperator({}) does not exist", + pipeID); + return; + } + if (Objects.nonNull(metricService)) { + removeMetrics(pipeID); + } + } + + public void markRegionCommit(final PipeTaskRuntimeEnvironment pipeTaskRuntimeEnvironment) { + // Filter commit attempt from assigner + final String pipeName = pipeTaskRuntimeEnvironment.getPipeName(); + final long creationTime = pipeTaskRuntimeEnvironment.getCreationTime(); + final String pipeID = pipeName + "_" + creationTime; + + if (Objects.isNull(metricService)) { + return; + } + final PipeConfigNodeRemainingTimeOperator operator = remainingTimeOperatorMap.get(pipeID); + if (Objects.isNull(operator)) { + LOGGER.warn( + "Failed to mark pipe region commit, RemainingTimeOperator({}) does not exist", pipeID); + return; + } + // Prevent not set pipeName / creation times & potential differences between pipeNames and + // creation times + if (!Objects.equals(pipeName, operator.getPipeName()) + || !Objects.equals(creationTime, operator.getCreationTime())) { + return; + } + + operator.markConfigRegionCommit(); + } + + //////////////////////////// singleton //////////////////////////// + + private static class PipeConfigNodeRemainingTimeMetricsHolder { + + private static final PipeConfigNodeRemainingTimeMetrics INSTANCE = + new PipeConfigNodeRemainingTimeMetrics(); + + private PipeConfigNodeRemainingTimeMetricsHolder() { + // Empty constructor + } + } + + public static PipeConfigNodeRemainingTimeMetrics getInstance() { + return PipeConfigNodeRemainingTimeMetricsHolder.INSTANCE; + } + + private PipeConfigNodeRemainingTimeMetrics() { + PipeEventCommitManager.getInstance().setCommitRateMarker(this::markRegionCommit); + } +} diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/metric/PipeConfigNodeRemainingTimeOperator.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/metric/PipeConfigNodeRemainingTimeOperator.java new file mode 100644 index 000000000000..cbc23f2e9be9 --- /dev/null +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/metric/PipeConfigNodeRemainingTimeOperator.java @@ -0,0 +1,119 @@ +/* + * 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.confignode.manager.pipe.metric; + +import org.apache.iotdb.commons.pipe.config.PipeConfig; +import org.apache.iotdb.confignode.manager.pipe.extractor.IoTDBConfigRegionExtractor; + +import com.codahale.metrics.Clock; +import com.codahale.metrics.ExponentialMovingAverages; +import com.codahale.metrics.Meter; + +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; + +class PipeConfigNodeRemainingTimeOperator { + + private static final long CONFIG_NODE_REMAINING_MAX_SECONDS = 365 * 24 * 60 * 60L; // 1 year + + private String pipeName; + private long creationTime = 0; + + private final ConcurrentMap + configRegionExtractors = new ConcurrentHashMap<>(); + private final Meter configRegionCommitMeter = + new Meter(new ExponentialMovingAverages(), Clock.defaultClock()); + + private double lastConfigRegionCommitSmoothingValue = Long.MIN_VALUE; + + //////////////////////////// Tags //////////////////////////// + + String getPipeName() { + return pipeName; + } + + long getCreationTime() { + return creationTime; + } + + //////////////////////////// Remaining time calculation //////////////////////////// + + /** + * This will calculate the estimated remaining time of pipe. + * + *

Notes: + * + *

1. The events in pipe assigner are omitted. + * + *

2. Other pipes' events sharing the same connectorSubtasks may be over-calculated. + * + * @return The estimated remaining time + */ + double getRemainingTime() { + final double pipeRemainingTimeCommitRateSmoothingFactor = + PipeConfig.getInstance().getPipeRemainingTimeCommitRateSmoothingFactor(); + + // Do not calculate heartbeat event + final long totalConfigRegionWriteEventCount = + configRegionExtractors.keySet().stream() + .map(IoTDBConfigRegionExtractor::getUnTransferredEventCount) + .reduce(Long::sum) + .orElse(0L); + + lastConfigRegionCommitSmoothingValue = + lastConfigRegionCommitSmoothingValue == Long.MIN_VALUE + ? configRegionCommitMeter.getOneMinuteRate() + : pipeRemainingTimeCommitRateSmoothingFactor + * configRegionCommitMeter.getOneMinuteRate() + + (1 - pipeRemainingTimeCommitRateSmoothingFactor) + * lastConfigRegionCommitSmoothingValue; + final double configRegionRemainingTime; + if (totalConfigRegionWriteEventCount <= 0) { + configRegionRemainingTime = 0; + } else { + configRegionRemainingTime = + lastConfigRegionCommitSmoothingValue <= 0 + ? Double.MAX_VALUE + : totalConfigRegionWriteEventCount / lastConfigRegionCommitSmoothingValue; + } + + return configRegionRemainingTime >= CONFIG_NODE_REMAINING_MAX_SECONDS + ? CONFIG_NODE_REMAINING_MAX_SECONDS + : configRegionRemainingTime; + } + + //////////////////////////// Register & deregister (pipe integration) //////////////////////////// + + void register(final IoTDBConfigRegionExtractor extractor) { + setNameAndCreationTime(extractor.getPipeName(), extractor.getCreationTime()); + configRegionExtractors.put(extractor, extractor); + } + + private void setNameAndCreationTime(final String pipeName, final long creationTime) { + this.pipeName = pipeName; + this.creationTime = creationTime; + } + + //////////////////////////// Rate //////////////////////////// + + void markConfigRegionCommit() { + configRegionCommitMeter.mark(); + } +} diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/metric/PipeConfigRegionConnectorMetrics.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/metric/PipeConfigRegionConnectorMetrics.java new file mode 100644 index 000000000000..01d9b4bb4d33 --- /dev/null +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/metric/PipeConfigRegionConnectorMetrics.java @@ -0,0 +1,157 @@ +/* + * 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.confignode.manager.pipe.metric; + +import org.apache.iotdb.commons.service.metric.enums.Metric; +import org.apache.iotdb.commons.service.metric.enums.Tag; +import org.apache.iotdb.confignode.manager.pipe.execution.PipeConfigNodeSubtask; +import org.apache.iotdb.metrics.AbstractMetricService; +import org.apache.iotdb.metrics.metricsets.IMetricSet; +import org.apache.iotdb.metrics.type.Rate; +import org.apache.iotdb.metrics.utils.MetricLevel; +import org.apache.iotdb.metrics.utils.MetricType; + +import com.google.common.collect.ImmutableSet; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Objects; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; + +public class PipeConfigRegionConnectorMetrics implements IMetricSet { + + private static final Logger LOGGER = + LoggerFactory.getLogger(PipeConfigRegionConnectorMetrics.class); + + private volatile AbstractMetricService metricService; + + private final ConcurrentMap subtaskMap = new ConcurrentHashMap<>(); + private final ConcurrentMap configRateMap = new ConcurrentHashMap<>(); + + //////////////////////////// bindTo & unbindFrom (metric framework) //////////////////////////// + + @Override + public void bindTo(final AbstractMetricService metricService) { + this.metricService = metricService; + ImmutableSet.copyOf(subtaskMap.keySet()).forEach(this::createMetrics); + } + + private void createMetrics(final String taskID) { + createRate(taskID); + } + + private void createRate(final String taskID) { + final PipeConfigNodeSubtask subtask = subtaskMap.get(taskID); + // Transfer event rate + configRateMap.put( + taskID, + metricService.getOrCreateRate( + Metric.PIPE_CONNECTOR_CONFIG_TRANSFER.toString(), + MetricLevel.IMPORTANT, + Tag.NAME.toString(), + subtask.getPipeName(), + Tag.CREATION_TIME.toString(), + String.valueOf(subtask.getCreationTime()))); + } + + @Override + public void unbindFrom(final AbstractMetricService metricService) { + ImmutableSet.copyOf(subtaskMap.keySet()).forEach(this::deregister); + if (!subtaskMap.isEmpty()) { + LOGGER.warn( + "Failed to unbind from pipe config region connector metrics, connector map not empty"); + } + } + + private void removeMetrics(final String taskID) { + removeRate(taskID); + } + + private void removeRate(final String taskID) { + final PipeConfigNodeSubtask subtask = subtaskMap.get(taskID); + // Transfer event rate + metricService.remove( + MetricType.RATE, + Metric.PIPE_CONNECTOR_CONFIG_TRANSFER.toString(), + Tag.NAME.toString(), + subtask.getPipeName(), + Tag.CREATION_TIME.toString(), + String.valueOf(subtask.getCreationTime())); + configRateMap.remove(taskID); + } + + //////////////////////////// register & deregister (pipe integration) //////////////////////////// + + public void register(final PipeConfigNodeSubtask pipeConfigNodeSubtask) { + final String taskID = pipeConfigNodeSubtask.getTaskID(); + subtaskMap.putIfAbsent(taskID, pipeConfigNodeSubtask); + if (Objects.nonNull(metricService)) { + createMetrics(taskID); + } + } + + public void deregister(final String taskID) { + if (!subtaskMap.containsKey(taskID)) { + LOGGER.warn( + "Failed to deregister pipe config region connector metrics, PipeConfigNodeSubtask({}) does not exist", + taskID); + return; + } + if (Objects.nonNull(metricService)) { + removeMetrics(taskID); + } + subtaskMap.remove(taskID); + } + + public void markConfigEvent(final String taskID) { + if (Objects.isNull(metricService)) { + return; + } + final Rate rate = configRateMap.get(taskID); + if (rate == null) { + LOGGER.warn( + "Failed to mark pipe config region write plan event, PipeConfigNodeSubtask({}) does not exist", + taskID); + return; + } + rate.mark(); + } + + //////////////////////////// singleton //////////////////////////// + + private static class PipeConfigNodeSubtaskMetricsHolder { + + private static final PipeConfigRegionConnectorMetrics INSTANCE = + new PipeConfigRegionConnectorMetrics(); + + private PipeConfigNodeSubtaskMetricsHolder() { + // Empty constructor + } + } + + public static PipeConfigRegionConnectorMetrics getInstance() { + return PipeConfigNodeSubtaskMetricsHolder.INSTANCE; + } + + private PipeConfigRegionConnectorMetrics() { + // Empty constructor + } +} diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/metric/PipeConfigRegionExtractorMetrics.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/metric/PipeConfigRegionExtractorMetrics.java new file mode 100644 index 000000000000..879c31999d19 --- /dev/null +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/metric/PipeConfigRegionExtractorMetrics.java @@ -0,0 +1,139 @@ +/* + * 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.confignode.manager.pipe.metric; + +import org.apache.iotdb.commons.service.metric.enums.Metric; +import org.apache.iotdb.commons.service.metric.enums.Tag; +import org.apache.iotdb.confignode.manager.pipe.extractor.IoTDBConfigRegionExtractor; +import org.apache.iotdb.metrics.AbstractMetricService; +import org.apache.iotdb.metrics.metricsets.IMetricSet; +import org.apache.iotdb.metrics.utils.MetricLevel; +import org.apache.iotdb.metrics.utils.MetricType; + +import com.google.common.collect.ImmutableSet; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.ConcurrentHashMap; + +public class PipeConfigRegionExtractorMetrics implements IMetricSet { + + private static final Logger LOGGER = + LoggerFactory.getLogger(PipeConfigRegionExtractorMetrics.class); + + private volatile AbstractMetricService metricService; + + private final Map extractorMap = new ConcurrentHashMap<>(); + + //////////////////////////// bindTo & unbindFrom (metric framework) //////////////////////////// + + @Override + public void bindTo(final AbstractMetricService metricService) { + this.metricService = metricService; + ImmutableSet.copyOf(extractorMap.keySet()).forEach(this::createMetrics); + } + + private void createMetrics(final String taskID) { + createAutoGauge(taskID); + } + + private void createAutoGauge(final String taskID) { + final IoTDBConfigRegionExtractor extractor = extractorMap.get(taskID); + metricService.createAutoGauge( + Metric.UNTRANSFERRED_CONFIG_COUNT.toString(), + MetricLevel.IMPORTANT, + extractorMap.get(taskID), + IoTDBConfigRegionExtractor::getUnTransferredEventCount, + Tag.NAME.toString(), + extractor.getPipeName(), + Tag.CREATION_TIME.toString(), + String.valueOf(extractor.getCreationTime())); + } + + @Override + public void unbindFrom(final AbstractMetricService metricService) { + ImmutableSet.copyOf(extractorMap.keySet()).forEach(this::deregister); + if (!extractorMap.isEmpty()) { + LOGGER.warn( + "Failed to unbind from pipe config region extractor metrics, extractor map not empty"); + } + } + + private void removeMetrics(final String taskID) { + removeAutoGauge(taskID); + } + + private void removeAutoGauge(final String taskID) { + final IoTDBConfigRegionExtractor extractor = extractorMap.get(taskID); + // Pending event count + metricService.remove( + MetricType.AUTO_GAUGE, + Metric.UNTRANSFERRED_CONFIG_COUNT.toString(), + Tag.NAME.toString(), + extractor.getPipeName(), + Tag.CREATION_TIME.toString(), + String.valueOf(extractor.getCreationTime())); + } + + //////////////////////////// pipe integration //////////////////////////// + + public void register(final IoTDBConfigRegionExtractor extractor) { + final String taskID = extractor.getTaskID(); + extractorMap.putIfAbsent(taskID, extractor); + if (Objects.nonNull(metricService)) { + createMetrics(taskID); + } + } + + public void deregister(final String taskID) { + if (!extractorMap.containsKey(taskID)) { + LOGGER.warn( + "Failed to deregister pipe config region extractor metrics, IoTDBConfigRegionExtractor({}) does not exist", + taskID); + return; + } + if (Objects.nonNull(metricService)) { + removeMetrics(taskID); + } + extractorMap.remove(taskID); + } + + //////////////////////////// singleton //////////////////////////// + + private static class PipeConfigRegionExtractorMetricsHolder { + + private static final PipeConfigRegionExtractorMetrics INSTANCE = + new PipeConfigRegionExtractorMetrics(); + + private PipeConfigRegionExtractorMetricsHolder() { + // Empty constructor + } + } + + public static PipeConfigRegionExtractorMetrics getInstance() { + return PipeConfigRegionExtractorMetricsHolder.INSTANCE; + } + + private PipeConfigRegionExtractorMetrics() { + // Empty constructor + } +} diff --git a/iotdb-core/datanode/pom.xml b/iotdb-core/datanode/pom.xml index b8ecee62a292..67ee544008f7 100644 --- a/iotdb-core/datanode/pom.xml +++ b/iotdb-core/datanode/pom.xml @@ -147,6 +147,10 @@ org.apache.ratis ratis-common + + io.dropwizard.metrics + metrics-core + org.slf4j slf4j-api diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/runtime/PipeSchemaRegionListenerManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/runtime/PipeSchemaRegionListenerManager.java index 85df884d7b03..bf7c6ed103a6 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/runtime/PipeSchemaRegionListenerManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/runtime/PipeSchemaRegionListenerManager.java @@ -22,6 +22,7 @@ import org.apache.iotdb.commons.consensus.SchemaRegionId; import org.apache.iotdb.commons.pipe.task.PipeTask; import org.apache.iotdb.db.pipe.extractor.schemaregion.SchemaRegionListeningQueue; +import org.apache.iotdb.db.pipe.metric.PipeSchemaRegionListenerMetrics; import java.util.Map; import java.util.Set; @@ -31,47 +32,47 @@ public class PipeSchemaRegionListenerManager { - private final Map id2StatusMap = + private final Map id2ListenerMap = new ConcurrentHashMap<>(); public synchronized Set regionIds() { - return id2StatusMap.keySet(); + return id2ListenerMap.keySet(); } - public synchronized SchemaRegionListeningQueue listener(SchemaRegionId schemaRegionId) { - return id2StatusMap.computeIfAbsent(schemaRegionId, k -> new PipeSchemaRegionListener()) + public synchronized SchemaRegionListeningQueue listener(final SchemaRegionId schemaRegionId) { + return id2ListenerMap.computeIfAbsent(schemaRegionId, PipeSchemaRegionListener::new) .listeningQueue; } - public synchronized int increaseAndGetReferenceCount(SchemaRegionId schemaRegionId) { - return id2StatusMap - .computeIfAbsent(schemaRegionId, k -> new PipeSchemaRegionListener()) + public synchronized int increaseAndGetReferenceCount(final SchemaRegionId schemaRegionId) { + return id2ListenerMap + .computeIfAbsent(schemaRegionId, PipeSchemaRegionListener::new) .listeningQueueReferenceCount .incrementAndGet(); } - public synchronized int decreaseAndGetReferenceCount(SchemaRegionId schemaRegionId) { - return id2StatusMap - .computeIfAbsent(schemaRegionId, k -> new PipeSchemaRegionListener()) + public synchronized int decreaseAndGetReferenceCount(final SchemaRegionId schemaRegionId) { + return id2ListenerMap + .computeIfAbsent(schemaRegionId, PipeSchemaRegionListener::new) .listeningQueueReferenceCount .updateAndGet(v -> v > 0 ? v - 1 : 0); } - public synchronized void notifyLeaderReady(SchemaRegionId schemaRegionId) { - id2StatusMap - .computeIfAbsent(schemaRegionId, k -> new PipeSchemaRegionListener()) + public synchronized void notifyLeaderReady(final SchemaRegionId schemaRegionId) { + id2ListenerMap + .computeIfAbsent(schemaRegionId, PipeSchemaRegionListener::new) .notifyLeaderReady(); } - public synchronized void notifyLeaderUnavailable(SchemaRegionId schemaRegionId) { - id2StatusMap - .computeIfAbsent(schemaRegionId, k -> new PipeSchemaRegionListener()) + public synchronized void notifyLeaderUnavailable(final SchemaRegionId schemaRegionId) { + id2ListenerMap + .computeIfAbsent(schemaRegionId, PipeSchemaRegionListener::new) .notifyLeaderUnavailable(); } - public synchronized boolean isLeaderReady(SchemaRegionId schemaRegionId) { - return id2StatusMap - .computeIfAbsent(schemaRegionId, k -> new PipeSchemaRegionListener()) + public synchronized boolean isLeaderReady(final SchemaRegionId schemaRegionId) { + return id2ListenerMap + .computeIfAbsent(schemaRegionId, PipeSchemaRegionListener::new) .isLeaderReady(); } @@ -82,6 +83,11 @@ private static class PipeSchemaRegionListener { private final AtomicBoolean isLeaderReady = new AtomicBoolean(false); + protected PipeSchemaRegionListener(final SchemaRegionId schemaRegionId) { + PipeSchemaRegionListenerMetrics.getInstance() + .register(listeningQueue, schemaRegionId.getId()); + } + /** * Get leader ready state, DO NOT use consensus layer's leader ready flag because * SimpleConsensus' ready flag is always {@code true}. Note that this flag has nothing to do diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java index 003a678e9ddc..2671de28f198 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java @@ -42,7 +42,8 @@ import org.apache.iotdb.db.pipe.extractor.dataregion.IoTDBDataRegionExtractor; import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.listener.PipeInsertionDataNodeListener; import org.apache.iotdb.db.pipe.extractor.schemaregion.SchemaRegionListeningFilter; -import org.apache.iotdb.db.pipe.metric.PipeExtractorMetrics; +import org.apache.iotdb.db.pipe.metric.PipeDataNodeRemainingEventAndTimeMetrics; +import org.apache.iotdb.db.pipe.metric.PipeDataRegionExtractorMetrics; import org.apache.iotdb.db.pipe.resource.PipeResourceManager; import org.apache.iotdb.db.pipe.task.PipeDataNodeTask; import org.apache.iotdb.db.pipe.task.builder.PipeDataNodeBuilder; @@ -92,7 +93,7 @@ protected boolean isShutdown() { } @Override - protected Map buildPipeTasks(PipeMeta pipeMetaFromConfigNode) + protected Map buildPipeTasks(final PipeMeta pipeMetaFromConfigNode) throws IllegalPathException { return new PipeDataNodeBuilder(pipeMetaFromConfigNode).build(); } @@ -101,7 +102,9 @@ protected Map buildPipeTasks(PipeMeta pipeMetaFromConfigNode) @Override protected void createPipeTask( - int consensusGroupId, PipeStaticMeta pipeStaticMeta, PipeTaskMeta pipeTaskMeta) + final int consensusGroupId, + final PipeStaticMeta pipeStaticMeta, + final PipeTaskMeta pipeTaskMeta) throws IllegalPathException { if (pipeTaskMeta.getLeaderNodeId() == CONFIG.getDataNodeId()) { final PipeParameters extractorParameters = pipeStaticMeta.getExtractorParameters(); @@ -135,7 +138,7 @@ protected void createPipeTask( @Override public List handlePipeMetaChangesInternal( - List pipeMetaListFromCoordinator) { + final List pipeMetaListFromCoordinator) { // Do nothing if the node is removing or removed if (isShutdown()) { return Collections.emptyList(); @@ -148,7 +151,7 @@ public List handlePipeMetaChangesInternal( final Set validSchemaRegionIds = clearSchemaRegionListeningQueueIfNecessary(pipeMetaListFromCoordinator); closeSchemaRegionListeningQueueIfNecessary(validSchemaRegionIds, exceptionMessages); - } catch (Exception e) { + } catch (final Exception e) { throw new PipeException("Failed to clear/close schema region listening queue.", e); } @@ -156,7 +159,7 @@ public List handlePipeMetaChangesInternal( } private Set clearSchemaRegionListeningQueueIfNecessary( - List pipeMetaListFromCoordinator) throws IllegalPathException { + final List pipeMetaListFromCoordinator) throws IllegalPathException { final Map schemaRegionId2ListeningQueueNewFirstIndex = new HashMap<>(); // Check each pipe @@ -202,8 +205,8 @@ private Set clearSchemaRegionListeningQueueIfNecessary( } private void closeSchemaRegionListeningQueueIfNecessary( - Set validSchemaRegionIds, - List exceptionMessages) { + final Set validSchemaRegionIds, + final List exceptionMessages) { if (!exceptionMessages.isEmpty()) { return; } @@ -219,7 +222,7 @@ private void closeSchemaRegionListeningQueueIfNecessary( .write( schemaRegionId, new PipeOperateSchemaQueueNode(new PlanNodeId(""), false)); - } catch (ConsensusException e) { + } catch (final ConsensusException e) { throw new PipeException( "Failed to close listening queue for SchemaRegion " + schemaRegionId, e); } @@ -227,13 +230,33 @@ private void closeSchemaRegionListeningQueueIfNecessary( }); } + @Override + protected void dropPipe(final String pipeName, final long creationTime) { + super.dropPipe(pipeName, creationTime); + + PipeDataNodeRemainingEventAndTimeMetrics.getInstance() + .deregister(pipeName + "_" + creationTime); + } + + @Override + protected void dropPipe(final String pipeName) { + super.dropPipe(pipeName); + + final PipeMeta pipeMeta = pipeMetaKeeper.getPipeMeta(pipeName); + if (Objects.nonNull(pipeMeta)) { + final long creationTime = pipeMeta.getStaticMeta().getCreationTime(); + PipeDataNodeRemainingEventAndTimeMetrics.getInstance() + .deregister(pipeName + "_" + creationTime); + } + } + public void stopAllPipesWithCriticalException() { super.stopAllPipesWithCriticalException(CONFIG.getDataNodeId()); } ///////////////////////// Heartbeat ///////////////////////// - public void collectPipeMetaList(TDataNodeHeartbeatResp resp) throws TException { + public void collectPipeMetaList(final TDataNodeHeartbeatResp resp) throws TException { // Try the lock instead of directly acquire it to prevent the block of the cluster heartbeat // 10s is the half of the HEARTBEAT_TIMEOUT_TIME defined in class BaseNodeCache in ConfigNode if (!tryReadLockWithTimeOut(10)) { @@ -246,7 +269,7 @@ public void collectPipeMetaList(TDataNodeHeartbeatResp resp) throws TException { } } - private void collectPipeMetaListInternal(TDataNodeHeartbeatResp resp) throws TException { + private void collectPipeMetaListInternal(final TDataNodeHeartbeatResp resp) throws TException { // Do nothing if data node is removing or removed, or request does not need pipe meta list if (PipeAgent.runtime().isShutdown()) { return; @@ -266,15 +289,15 @@ private void collectPipeMetaListInternal(TDataNodeHeartbeatResp resp) throws TEx logger.ifPresent(l -> l.info("Reporting pipe meta: {}", pipeMeta.coreReportMessage())); } LOGGER.info("Reported {} pipe metas.", pipeMetaBinaryList.size()); - } catch (IOException e) { + } catch (final IOException e) { throw new TException(e); } resp.setPipeMetaList(pipeMetaBinaryList); } @Override - protected void collectPipeMetaListInternal(TPipeHeartbeatReq req, TPipeHeartbeatResp resp) - throws TException { + protected void collectPipeMetaListInternal( + final TPipeHeartbeatReq req, final TPipeHeartbeatResp resp) throws TException { // Do nothing if data node is removing or removed, or request does not need pipe meta list if (PipeAgent.runtime().isShutdown()) { return; @@ -295,7 +318,7 @@ protected void collectPipeMetaListInternal(TPipeHeartbeatReq req, TPipeHeartbeat logger.ifPresent(l -> l.info("Reporting pipe meta: {}", pipeMeta.coreReportMessage())); } LOGGER.info("Reported {} pipe metas.", pipeMetaBinaryList.size()); - } catch (IOException e) { + } catch (final IOException e) { throw new TException(e); } resp.setPipeMetaList(pipeMetaBinaryList); @@ -318,7 +341,7 @@ public void restartAllStuckPipes() { private void restartAllStuckPipesInternal() { final Map taskId2ExtractorMap = - PipeExtractorMetrics.getInstance().getExtractorMap(); + PipeDataRegionExtractorMetrics.getInstance().getExtractorMap(); final Set stuckPipes = new HashSet<>(); for (final PipeMeta pipeMeta : pipeMetaKeeper.getPipeMetaList()) { @@ -394,7 +417,7 @@ private boolean mayWalSizeReachThrottleThreshold() { return 3 * WALManager.getInstance().getTotalDiskUsage() > 2 * CONFIG.getThrottleThreshold(); } - private void restartStuckPipe(PipeMeta pipeMeta) { + private void restartStuckPipe(final PipeMeta pipeMeta) { LOGGER.warn("Pipe {} will be restarted because of stuck.", pipeMeta.getStaticMeta()); final long startTime = System.currentTimeMillis(); changePipeStatusBeforeRestart(pipeMeta.getStaticMeta().getPipeName()); @@ -405,7 +428,7 @@ private void restartStuckPipe(PipeMeta pipeMeta) { System.currentTimeMillis() - startTime); } - private void changePipeStatusBeforeRestart(String pipeName) { + private void changePipeStatusBeforeRestart(final String pipeName) { final PipeMeta pipeMeta = pipeMetaKeeper.getPipeMeta(pipeName); final Map pipeTasks = pipeTaskManager.getPipeTasks(pipeMeta.getStaticMeta()); final Set taskRegionIds = new HashSet<>(pipeTasks.keySet()); @@ -453,7 +476,7 @@ private void changePipeStatusBeforeRestart(String pipeName) { ///////////////////////// Utils ///////////////////////// - public Set getPipeTaskRegionIdSet(String pipeName, long creationTime) { + public Set getPipeTaskRegionIdSet(final String pipeName, final long creationTime) { final PipeMeta pipeMeta = pipeMetaKeeper.getPipeMeta(pipeName); return pipeMeta == null || pipeMeta.getStaticMeta().getCreationTime() != creationTime ? Collections.emptySet() 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 c9f076b0f047..dcab0b69b3c5 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 @@ -61,6 +61,7 @@ import java.util.concurrent.BlockingQueue; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.CONNECTOR_IOTDB_BATCH_MODE_ENABLE_KEY; import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.CONNECTOR_LEADER_CACHE_ENABLE_DEFAULT_VALUE; @@ -503,4 +504,25 @@ public synchronized void close() throws Exception { public int getRetryEventQueueSize() { return retryEventQueue.size(); } + + // For performance, this will not acquire lock and does not guarantee the correct + // result. However, this shall not cause any exceptions when concurrently read & written. + public int getRetryEventCount(final String pipeName) { + final AtomicInteger count = new AtomicInteger(0); + try { + retryEventQueue.forEach( + event -> { + if (event instanceof EnrichedEvent + && pipeName.equals(((EnrichedEvent) event).getPipeName())) { + count.incrementAndGet(); + } + }); + return count.get(); + } catch (Exception e) { + if (LOGGER.isDebugEnabled()) { + LOGGER.debug("Failed to get retry event count for pipe {}.", pipeName, e); + } + return count.get(); + } + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/IoTDBDataRegionExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/IoTDBDataRegionExtractor.java index b253e1396b0b..b8ee7eb06202 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/IoTDBDataRegionExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/IoTDBDataRegionExtractor.java @@ -33,7 +33,8 @@ import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.PipeRealtimeDataRegionHybridExtractor; import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.PipeRealtimeDataRegionLogExtractor; import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.PipeRealtimeDataRegionTsFileExtractor; -import org.apache.iotdb.db.pipe.metric.PipeExtractorMetrics; +import org.apache.iotdb.db.pipe.metric.PipeDataNodeRemainingEventAndTimeMetrics; +import org.apache.iotdb.db.pipe.metric.PipeDataRegionExtractorMetrics; import org.apache.iotdb.db.storageengine.StorageEngine; import org.apache.iotdb.db.storageengine.dataregion.wal.utils.WALMode; import org.apache.iotdb.pipe.api.customizer.configuration.PipeExtractorRuntimeConfiguration; @@ -309,7 +310,8 @@ public void customize( } // register metric after generating taskID - PipeExtractorMetrics.getInstance().register(this); + PipeDataRegionExtractorMetrics.getInstance().register(this); + PipeDataNodeRemainingEventAndTimeMetrics.getInstance().register(this); } @Override @@ -415,11 +417,11 @@ public Event supply() throws Exception { if (Objects.nonNull(event)) { if (event instanceof TabletInsertionEvent) { - PipeExtractorMetrics.getInstance().markTabletEvent(taskID); + PipeDataRegionExtractorMetrics.getInstance().markTabletEvent(taskID); } else if (event instanceof TsFileInsertionEvent) { - PipeExtractorMetrics.getInstance().markTsFileEvent(taskID); + PipeDataRegionExtractorMetrics.getInstance().markTsFileEvent(taskID); } else if (event instanceof PipeHeartbeatEvent) { - PipeExtractorMetrics.getInstance().markPipeHeartbeatEvent(taskID); + PipeDataRegionExtractorMetrics.getInstance().markPipeHeartbeatEvent(taskID); } } @@ -435,7 +437,7 @@ public void close() throws Exception { historicalExtractor.close(); realtimeExtractor.close(); if (Objects.nonNull(taskID)) { - PipeExtractorMetrics.getInstance().deregister(taskID); + PipeDataRegionExtractorMetrics.getInstance().deregister(taskID); } } @@ -456,22 +458,6 @@ public boolean hasConsumedAllHistoricalTsFiles() { //////////////////////////// APIs provided for metric framework //////////////////////////// - public String getTaskID() { - return taskID; - } - - public String getPipeName() { - return pipeName; - } - - public int getDataRegionId() { - return regionId; - } - - public long getCreationTime() { - return creationTime; - } - public int getHistoricalTsFileInsertionEventCount() { return hasBeenStarted.get() ? historicalExtractor.getPendingQueueSize() : 0; } @@ -487,4 +473,10 @@ public int getRealtimeTsFileInsertionEventCount() { public int getPipeHeartbeatEventCount() { return hasBeenStarted.get() ? realtimeExtractor.getPipeHeartbeatEventCount() : 0; } + + public int getEventCount() { + return hasBeenStarted.get() + ? (historicalExtractor.getPendingQueueSize() + realtimeExtractor.getEventCount()) + : 0; + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionExtractor.java index b8c5a713ec24..22a24dc387a2 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionExtractor.java @@ -111,7 +111,7 @@ protected PipeRealtimeDataRegionExtractor() { } @Override - public void validate(PipeParameterValidator validator) throws Exception { + public void validate(final PipeParameterValidator validator) throws Exception { final PipeParameters parameters = validator.getParameters(); try { @@ -134,14 +134,15 @@ public void validate(PipeParameterValidator validator) throws Exception { SOURCE_END_TIME_KEY, EXTRACTOR_END_TIME_KEY)); } - } catch (Exception e) { + } catch (final Exception e) { // compatible with the current validation framework throw new PipeParameterNotValidException(e.getMessage()); } } @Override - public void customize(PipeParameters parameters, PipeExtractorRuntimeConfiguration configuration) + public void customize( + final PipeParameters parameters, final PipeExtractorRuntimeConfiguration configuration) throws Exception { final PipeTaskExtractorRuntimeEnvironment environment = (PipeTaskExtractorRuntimeEnvironment) configuration.getRuntimeEnvironment(); @@ -235,7 +236,7 @@ private void clearPendingQueue() { /** * @param event the {@link Event} from the {@link StorageEngine} */ - public final void extract(PipeRealtimeEvent event) { + public final void extract(final PipeRealtimeEvent event) { if (isDbNameCoveredByPattern) { event.skipParsingPattern(); } @@ -270,9 +271,9 @@ public final void extract(PipeRealtimeEvent event) { } } - protected abstract void doExtract(PipeRealtimeEvent event); + protected abstract void doExtract(final PipeRealtimeEvent event); - protected void extractHeartbeat(PipeRealtimeEvent event) { + protected void extractHeartbeat(final PipeRealtimeEvent event) { // Bind extractor so that the heartbeat event can later inform the extractor of queue size ((PipeHeartbeatEvent) event.getEvent()).bindExtractor(this); @@ -308,7 +309,7 @@ protected void extractHeartbeat(PipeRealtimeEvent event) { } } - protected void extractDeletion(PipeRealtimeEvent event) { + protected void extractDeletion(final PipeRealtimeEvent event) { if (!pendingQueue.waitedOffer(event)) { // This would not happen, but just in case. // Pending is unbounded, so it should never reach capacity. @@ -325,11 +326,11 @@ protected void extractDeletion(PipeRealtimeEvent event) { } } - protected Event supplyHeartbeat(PipeRealtimeEvent event) { + protected Event supplyHeartbeat(final PipeRealtimeEvent event) { if (event.increaseReferenceCount(PipeRealtimeDataRegionExtractor.class.getName())) { return event.getEvent(); } else { - // this would not happen, but just in case. + // This would not happen, but just in case. LOGGER.error( "Heartbeat Event {} can not be supplied because " + "the reference count can not be increased", @@ -342,7 +343,7 @@ protected Event supplyHeartbeat(PipeRealtimeEvent event) { } } - protected Event supplyDeletion(PipeRealtimeEvent event) { + protected Event supplyDeletion(final PipeRealtimeEvent event) { if (event.increaseReferenceCount(PipeRealtimeDataRegionExtractor.class.getName())) { return event.getEvent(); } else { @@ -393,7 +394,8 @@ public final long getRealtimeDataExtractionEndTime() { return realtimeDataExtractionEndTime; } - public void setDataRegionTimePartitionIdBound(@NonNull Pair timePartitionIdBound) { + public void setDataRegionTimePartitionIdBound( + @NonNull final Pair timePartitionIdBound) { LOGGER.info( "PipeRealtimeDataRegionExtractor({}) observed data region {} time partition growth, recording time partition id bound: {}.", taskID, @@ -446,6 +448,10 @@ public int getPipeHeartbeatEventCount() { return pendingQueue.getPipeHeartbeatEventCount(); } + public int getEventCount() { + return pendingQueue.size(); + } + public String getTaskID() { return taskID; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java index 2161cefc87c8..210c52581203 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java @@ -29,7 +29,7 @@ import org.apache.iotdb.db.pipe.event.realtime.PipeRealtimeEvent; import org.apache.iotdb.db.pipe.extractor.dataregion.IoTDBDataRegionExtractor; import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.epoch.TsFileEpoch; -import org.apache.iotdb.db.pipe.metric.PipeExtractorMetrics; +import org.apache.iotdb.db.pipe.metric.PipeDataRegionExtractorMetrics; import org.apache.iotdb.db.pipe.resource.PipeResourceManager; import org.apache.iotdb.db.storageengine.dataregion.wal.WALManager; import org.apache.iotdb.pipe.api.event.Event; @@ -232,7 +232,7 @@ private boolean mayMemTablePinnedCountReachDangerousThreshold() { private boolean isHistoricalTsFileEventCountExceededLimit() { final IoTDBDataRegionExtractor extractor = - PipeExtractorMetrics.getInstance().getExtractorMap().get(getTaskID()); + PipeDataRegionExtractorMetrics.getInstance().getExtractorMap().get(getTaskID()); return Objects.nonNull(extractor) && extractor.getHistoricalTsFileInsertionEventCount() >= PipeConfig.getInstance().getPipeMaxAllowedHistoricalTsFilePerDataRegion(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/epoch/TsFileEpoch.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/epoch/TsFileEpoch.java index 976c385188f2..abef1db549e6 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/epoch/TsFileEpoch.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/epoch/TsFileEpoch.java @@ -20,7 +20,7 @@ package org.apache.iotdb.db.pipe.extractor.dataregion.realtime.epoch; import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.PipeRealtimeDataRegionExtractor; -import org.apache.iotdb.db.pipe.metric.PipeExtractorMetrics; +import org.apache.iotdb.db.pipe.metric.PipeDataRegionExtractorMetrics; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; @@ -56,7 +56,7 @@ public void migrateState( public void setExtractorsRecentProcessedTsFileEpochState() { dataRegionExtractor2State.forEach( (extractor, state) -> - PipeExtractorMetrics.getInstance() + PipeDataRegionExtractorMetrics.getInstance() .setRecentProcessedTsFileEpochState(extractor.getTaskID(), state.get())); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/schemaregion/IoTDBSchemaRegionExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/schemaregion/IoTDBSchemaRegionExtractor.java index 47b8739147ac..a0798e3818fc 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/schemaregion/IoTDBSchemaRegionExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/schemaregion/IoTDBSchemaRegionExtractor.java @@ -31,6 +31,8 @@ import org.apache.iotdb.db.pipe.agent.PipeAgent; import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionSnapshotEvent; import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionWritePlanEvent; +import org.apache.iotdb.db.pipe.metric.PipeDataNodeRemainingEventAndTimeMetrics; +import org.apache.iotdb.db.pipe.metric.PipeSchemaRegionExtractorMetrics; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeType; @@ -42,6 +44,7 @@ import org.apache.iotdb.pipe.api.exception.PipeException; import java.util.HashSet; +import java.util.Objects; import java.util.Set; public class IoTDBSchemaRegionExtractor extends IoTDBNonDataRegionExtractor { @@ -67,6 +70,9 @@ public void customize( schemaRegionId = new SchemaRegionId(regionId); listenedTypeSet = SchemaRegionListeningFilter.parseListeningPlanTypeSet(parameters); + + PipeSchemaRegionExtractorMetrics.getInstance().register(this); + PipeDataNodeRemainingEventAndTimeMetrics.getInstance().register(this); } @Override @@ -96,7 +102,7 @@ protected boolean needTransferSnapshot() { protected void triggerSnapshot() { try { SchemaRegionConsensusImpl.getInstance().triggerSnapshot(schemaRegionId, true); - } catch (ConsensusException e) { + } catch (final ConsensusException e) { throw new PipeException("Exception encountered when triggering schema region snapshot.", e); } } @@ -151,5 +157,8 @@ public synchronized void close() throws Exception { // has no schema pipe after one successful sync PipeAgent.runtime().decreaseAndGetSchemaListenerReferenceCount(schemaRegionId); } + if (Objects.nonNull(taskID)) { + PipeSchemaRegionExtractorMetrics.getInstance().deregister(taskID); + } } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeDataNodeMetrics.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeDataNodeMetrics.java index bbbb268636cb..2f39cc75c252 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeDataNodeMetrics.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeDataNodeMetrics.java @@ -28,27 +28,35 @@ public class PipeDataNodeMetrics implements IMetricSet { //////////////////////////// bindTo & unbindFrom (metric framework) //////////////////////////// @Override - public void bindTo(AbstractMetricService metricService) { + public void bindTo(final AbstractMetricService metricService) { PipeAssignerMetrics.getInstance().bindTo(metricService); - PipeExtractorMetrics.getInstance().bindTo(metricService); + PipeDataRegionExtractorMetrics.getInstance().bindTo(metricService); PipeProcessorMetrics.getInstance().bindTo(metricService); - PipeConnectorMetrics.getInstance().bindTo(metricService); + PipeDataRegionConnectorMetrics.getInstance().bindTo(metricService); PipeHeartbeatEventMetrics.getInstance().bindTo(metricService); PipeWALInsertNodeCacheMetrics.getInstance().bindTo(metricService); PipeResourceMetrics.getInstance().bindTo(metricService); PipeEventCommitMetrics.getInstance().bindTo(metricService); + PipeSchemaRegionListenerMetrics.getInstance().bindTo(metricService); + PipeSchemaRegionExtractorMetrics.getInstance().bindTo(metricService); + PipeSchemaRegionConnectorMetrics.getInstance().bindTo(metricService); + PipeDataNodeRemainingEventAndTimeMetrics.getInstance().bindTo(metricService); } @Override - public void unbindFrom(AbstractMetricService metricService) { + public void unbindFrom(final AbstractMetricService metricService) { PipeAssignerMetrics.getInstance().unbindFrom(metricService); - PipeExtractorMetrics.getInstance().unbindFrom(metricService); + PipeDataRegionExtractorMetrics.getInstance().unbindFrom(metricService); PipeProcessorMetrics.getInstance().unbindFrom(metricService); - PipeConnectorMetrics.getInstance().unbindFrom(metricService); + PipeDataRegionConnectorMetrics.getInstance().unbindFrom(metricService); PipeHeartbeatEventMetrics.getInstance().unbindFrom(metricService); PipeWALInsertNodeCacheMetrics.getInstance().unbindFrom(metricService); PipeResourceMetrics.getInstance().unbindFrom(metricService); PipeEventCommitMetrics.getInstance().unbindFrom(metricService); + PipeSchemaRegionListenerMetrics.getInstance().unbindFrom(metricService); + PipeSchemaRegionExtractorMetrics.getInstance().unbindFrom(metricService); + PipeSchemaRegionConnectorMetrics.getInstance().unbindFrom(metricService); + PipeDataNodeRemainingEventAndTimeMetrics.getInstance().unbindFrom(metricService); } //////////////////////////// singleton //////////////////////////// diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeDataNodeRemainingEventAndTimeMetrics.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeDataNodeRemainingEventAndTimeMetrics.java new file mode 100644 index 000000000000..2bfe9106c01d --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeDataNodeRemainingEventAndTimeMetrics.java @@ -0,0 +1,231 @@ +/* + * 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.db.pipe.metric; + +import org.apache.iotdb.commons.consensus.DataRegionId; +import org.apache.iotdb.commons.consensus.SchemaRegionId; +import org.apache.iotdb.commons.pipe.config.plugin.env.PipeTaskRuntimeEnvironment; +import org.apache.iotdb.commons.pipe.progress.PipeEventCommitManager; +import org.apache.iotdb.commons.service.metric.enums.Metric; +import org.apache.iotdb.commons.service.metric.enums.Tag; +import org.apache.iotdb.db.pipe.extractor.dataregion.IoTDBDataRegionExtractor; +import org.apache.iotdb.db.pipe.extractor.schemaregion.IoTDBSchemaRegionExtractor; +import org.apache.iotdb.db.pipe.task.subtask.connector.PipeConnectorSubtask; +import org.apache.iotdb.db.pipe.task.subtask.processor.PipeProcessorSubtask; +import org.apache.iotdb.db.schemaengine.SchemaEngine; +import org.apache.iotdb.db.storageengine.StorageEngine; +import org.apache.iotdb.metrics.AbstractMetricService; +import org.apache.iotdb.metrics.metricsets.IMetricSet; +import org.apache.iotdb.metrics.utils.MetricLevel; +import org.apache.iotdb.metrics.utils.MetricType; + +import com.google.common.collect.ImmutableSet; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.ConcurrentHashMap; + +public class PipeDataNodeRemainingEventAndTimeMetrics implements IMetricSet { + + private static final Logger LOGGER = + LoggerFactory.getLogger(PipeDataNodeRemainingEventAndTimeMetrics.class); + + private volatile AbstractMetricService metricService; + + private final Map + remainingEventAndTimeOperatorMap = new ConcurrentHashMap<>(); + + //////////////////////////// bindTo & unbindFrom (metric framework) //////////////////////////// + + @Override + public void bindTo(final AbstractMetricService metricService) { + this.metricService = metricService; + ImmutableSet.copyOf(remainingEventAndTimeOperatorMap.keySet()).forEach(this::createMetrics); + } + + private void createMetrics(final String pipeID) { + createAutoGauge(pipeID); + } + + private void createAutoGauge(final String pipeID) { + final PipeDataNodeRemainingEventAndTimeOperator operator = + remainingEventAndTimeOperatorMap.get(pipeID); + metricService.createAutoGauge( + Metric.PIPE_DATANODE_REMAINING_EVENT_COUNT.toString(), + MetricLevel.IMPORTANT, + operator, + PipeDataNodeRemainingEventAndTimeOperator::getRemainingEvents, + Tag.NAME.toString(), + operator.getPipeName(), + Tag.CREATION_TIME.toString(), + String.valueOf(operator.getCreationTime())); + metricService.createAutoGauge( + Metric.PIPE_DATANODE_REMAINING_TIME.toString(), + MetricLevel.IMPORTANT, + operator, + PipeDataNodeRemainingEventAndTimeOperator::getRemainingTime, + Tag.NAME.toString(), + operator.getPipeName(), + Tag.CREATION_TIME.toString(), + String.valueOf(operator.getCreationTime())); + } + + @Override + public void unbindFrom(final AbstractMetricService metricService) { + ImmutableSet.copyOf(remainingEventAndTimeOperatorMap.keySet()).forEach(this::deregister); + if (!remainingEventAndTimeOperatorMap.isEmpty()) { + LOGGER.warn( + "Failed to unbind from pipe remaining event and time metrics, RemainingEventAndTimeOperator map not empty"); + } + } + + private void removeMetrics(final String pipeID) { + removeAutoGauge(pipeID); + } + + private void removeAutoGauge(final String pipeID) { + final PipeDataNodeRemainingEventAndTimeOperator operator = + remainingEventAndTimeOperatorMap.get(pipeID); + metricService.remove( + MetricType.AUTO_GAUGE, + Metric.PIPE_DATANODE_REMAINING_TIME.toString(), + Tag.NAME.toString(), + operator.getPipeName(), + Tag.CREATION_TIME.toString(), + String.valueOf(operator.getCreationTime())); + remainingEventAndTimeOperatorMap.remove(pipeID); + } + + //////////////////////////// register & deregister (pipe integration) //////////////////////////// + + public void register(final IoTDBDataRegionExtractor extractor) { + // The metric is global thus the regionId is omitted + final String pipeID = extractor.getPipeName() + "_" + extractor.getCreationTime(); + remainingEventAndTimeOperatorMap + .computeIfAbsent(pipeID, k -> new PipeDataNodeRemainingEventAndTimeOperator()) + .register(extractor); + if (Objects.nonNull(metricService)) { + createMetrics(pipeID); + } + } + + public void register(final PipeProcessorSubtask processorSubtask) { + // The metric is global thus the regionId is omitted + final String pipeID = processorSubtask.getPipeName() + "_" + processorSubtask.getCreationTime(); + remainingEventAndTimeOperatorMap + .computeIfAbsent(pipeID, k -> new PipeDataNodeRemainingEventAndTimeOperator()) + .register(processorSubtask); + if (Objects.nonNull(metricService)) { + createMetrics(pipeID); + } + } + + public void register( + final PipeConnectorSubtask connectorSubtask, final String pipeName, final long creationTime) { + // The metric is global thus the regionId is omitted + final String pipeID = pipeName + "_" + creationTime; + remainingEventAndTimeOperatorMap + .computeIfAbsent(pipeID, k -> new PipeDataNodeRemainingEventAndTimeOperator()) + .register(connectorSubtask, pipeName, creationTime); + if (Objects.nonNull(metricService)) { + createMetrics(pipeID); + } + } + + public void register(final IoTDBSchemaRegionExtractor extractor) { + // The metric is global thus the regionId is omitted + final String pipeID = extractor.getPipeName() + "_" + extractor.getCreationTime(); + remainingEventAndTimeOperatorMap + .computeIfAbsent(pipeID, k -> new PipeDataNodeRemainingEventAndTimeOperator()) + .register(extractor); + if (Objects.nonNull(metricService)) { + createMetrics(pipeID); + } + } + + public void deregister(final String pipeID) { + if (!remainingEventAndTimeOperatorMap.containsKey(pipeID)) { + LOGGER.warn( + "Failed to deregister pipe remaining event and time metrics, RemainingEventAndTimeOperator({}) does not exist", + pipeID); + return; + } + if (Objects.nonNull(metricService)) { + removeMetrics(pipeID); + } + } + + public void markRegionCommit(final PipeTaskRuntimeEnvironment pipeTaskRuntimeEnvironment) { + // Filter commit attempt from assigner + final String pipeName = pipeTaskRuntimeEnvironment.getPipeName(); + final int regionId = pipeTaskRuntimeEnvironment.getRegionId(); + final long creationTime = pipeTaskRuntimeEnvironment.getCreationTime(); + final String pipeID = pipeName + "_" + creationTime; + + if (Objects.isNull(metricService)) { + return; + } + final PipeDataNodeRemainingEventAndTimeOperator operator = + remainingEventAndTimeOperatorMap.get(pipeID); + if (Objects.isNull(operator)) { + LOGGER.warn( + "Failed to mark pipe region commit, RemainingEventAndTimeOperator({}) does not exist", + pipeID); + return; + } + // Prevent not set pipeName / creation times & potential differences between pipeNames and + // creation times + if (!Objects.equals(pipeName, operator.getPipeName()) + || !Objects.equals(creationTime, operator.getCreationTime())) { + return; + } + + // Prevent empty region-ids + if (StorageEngine.getInstance().getAllDataRegionIds().contains(new DataRegionId(regionId))) { + operator.markDataRegionCommit(); + } + + if (SchemaEngine.getInstance().getAllSchemaRegionIds().contains(new SchemaRegionId(regionId))) { + operator.markSchemaRegionCommit(); + } + } + + //////////////////////////// singleton //////////////////////////// + + private static class PipeDataNodeRemainingEventAndTimeMetricsHolder { + + private static final PipeDataNodeRemainingEventAndTimeMetrics INSTANCE = + new PipeDataNodeRemainingEventAndTimeMetrics(); + + private PipeDataNodeRemainingEventAndTimeMetricsHolder() { + // Empty constructor + } + } + + public static PipeDataNodeRemainingEventAndTimeMetrics getInstance() { + return PipeDataNodeRemainingEventAndTimeMetricsHolder.INSTANCE; + } + + private PipeDataNodeRemainingEventAndTimeMetrics() { + PipeEventCommitManager.getInstance().setCommitRateMarker(this::markRegionCommit); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeDataNodeRemainingEventAndTimeOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeDataNodeRemainingEventAndTimeOperator.java new file mode 100644 index 000000000000..ca311824a653 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeDataNodeRemainingEventAndTimeOperator.java @@ -0,0 +1,207 @@ +/* + * 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.db.pipe.metric; + +import org.apache.iotdb.commons.pipe.config.PipeConfig; +import org.apache.iotdb.db.pipe.extractor.dataregion.IoTDBDataRegionExtractor; +import org.apache.iotdb.db.pipe.extractor.schemaregion.IoTDBSchemaRegionExtractor; +import org.apache.iotdb.db.pipe.task.subtask.connector.PipeConnectorSubtask; +import org.apache.iotdb.db.pipe.task.subtask.processor.PipeProcessorSubtask; + +import com.codahale.metrics.Clock; +import com.codahale.metrics.ExponentialMovingAverages; +import com.codahale.metrics.Meter; + +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; + +class PipeDataNodeRemainingEventAndTimeOperator { + + private static final long DATA_NODE_REMAINING_MAX_SECONDS = 365 * 24 * 60 * 60L; // 1 year + + private String pipeName; + private long creationTime = 0; + + private final ConcurrentMap + dataRegionExtractors = new ConcurrentHashMap<>(); + private final ConcurrentMap dataRegionProcessors = + new ConcurrentHashMap<>(); + private final ConcurrentMap dataRegionConnectors = + new ConcurrentHashMap<>(); + private final ConcurrentMap + schemaRegionExtractors = new ConcurrentHashMap<>(); + private final Meter dataRegionCommitMeter = + new Meter(new ExponentialMovingAverages(), Clock.defaultClock()); + private final Meter schemaRegionCommitMeter = + new Meter(new ExponentialMovingAverages(), Clock.defaultClock()); + + private double lastDataRegionCommitSmoothingValue = Long.MIN_VALUE; + private double lastSchemaRegionCommitSmoothingValue = Long.MIN_VALUE; + + //////////////////////////// Tags //////////////////////////// + + String getPipeName() { + return pipeName; + } + + long getCreationTime() { + return creationTime; + } + + //////////////////////////// Remaining event & time calculation //////////////////////////// + + long getRemainingEvents() { + return dataRegionExtractors.keySet().stream() + .map(IoTDBDataRegionExtractor::getEventCount) + .reduce(Integer::sum) + .orElse(0) + + dataRegionProcessors.keySet().stream() + .map(PipeProcessorSubtask::getEventCount) + .reduce(Integer::sum) + .orElse(0) + + dataRegionConnectors.keySet().stream() + .map(connectorSubtask -> connectorSubtask.getEventCount(pipeName)) + .reduce(Integer::sum) + .orElse(0) + + schemaRegionExtractors.keySet().stream() + .map(IoTDBSchemaRegionExtractor::getUnTransferredEventCount) + .reduce(Long::sum) + .orElse(0L); + } + + /** + * This will calculate the estimated remaining time of pipe. + * + *

Note: The events in pipe assigner are omitted. + * + * @return The estimated remaining time + */ + double getRemainingTime() { + final double pipeRemainingTimeCommitRateSmoothingFactor = + PipeConfig.getInstance().getPipeRemainingTimeCommitRateSmoothingFactor(); + + // Do not take heartbeat event into account + final int totalDataRegionWriteEventCount = + dataRegionExtractors.keySet().stream() + .map(IoTDBDataRegionExtractor::getEventCount) + .reduce(Integer::sum) + .orElse(0) + + dataRegionProcessors.keySet().stream() + .map(PipeProcessorSubtask::getEventCount) + .reduce(Integer::sum) + .orElse(0) + + dataRegionConnectors.keySet().stream() + .map(connectorSubtask -> connectorSubtask.getEventCount(pipeName)) + .reduce(Integer::sum) + .orElse(0) + - dataRegionExtractors.keySet().stream() + .map(IoTDBDataRegionExtractor::getPipeHeartbeatEventCount) + .reduce(Integer::sum) + .orElse(0) + - dataRegionProcessors.keySet().stream() + .map(PipeProcessorSubtask::getPipeHeartbeatEventCount) + .reduce(Integer::sum) + .orElse(0) + - dataRegionConnectors.keySet().stream() + .map(PipeConnectorSubtask::getPipeHeartbeatEventCount) + .reduce(Integer::sum) + .orElse(0); + + lastDataRegionCommitSmoothingValue = + lastDataRegionCommitSmoothingValue == Long.MIN_VALUE + ? dataRegionCommitMeter.getOneMinuteRate() + : pipeRemainingTimeCommitRateSmoothingFactor * dataRegionCommitMeter.getOneMinuteRate() + + (1 - pipeRemainingTimeCommitRateSmoothingFactor) + * lastDataRegionCommitSmoothingValue; + final double dataRegionRemainingTime; + if (totalDataRegionWriteEventCount <= 0) { + dataRegionRemainingTime = 0; + } else { + dataRegionRemainingTime = + lastDataRegionCommitSmoothingValue <= 0 + ? Double.MAX_VALUE + : totalDataRegionWriteEventCount / lastDataRegionCommitSmoothingValue; + } + + final long totalSchemaRegionWriteEventCount = + schemaRegionExtractors.keySet().stream() + .map(IoTDBSchemaRegionExtractor::getUnTransferredEventCount) + .reduce(Long::sum) + .orElse(0L); + + lastSchemaRegionCommitSmoothingValue = + lastSchemaRegionCommitSmoothingValue == Long.MIN_VALUE + ? schemaRegionCommitMeter.getOneMinuteRate() + : pipeRemainingTimeCommitRateSmoothingFactor + * schemaRegionCommitMeter.getOneMinuteRate() + + (1 - pipeRemainingTimeCommitRateSmoothingFactor) + * lastSchemaRegionCommitSmoothingValue; + final double schemaRegionRemainingTime; + if (totalSchemaRegionWriteEventCount <= 0) { + schemaRegionRemainingTime = 0; + } else { + schemaRegionRemainingTime = + lastSchemaRegionCommitSmoothingValue <= 0 + ? Double.MAX_VALUE + : totalSchemaRegionWriteEventCount / lastSchemaRegionCommitSmoothingValue; + } + + final double result = Math.max(dataRegionRemainingTime, schemaRegionRemainingTime); + return result >= DATA_NODE_REMAINING_MAX_SECONDS ? DATA_NODE_REMAINING_MAX_SECONDS : result; + } + + //////////////////////////// Register & deregister (pipe integration) //////////////////////////// + + void register(final IoTDBDataRegionExtractor extractor) { + setNameAndCreationTime(extractor.getPipeName(), extractor.getCreationTime()); + dataRegionExtractors.put(extractor, extractor); + } + + void register(final PipeProcessorSubtask processorSubtask) { + setNameAndCreationTime(processorSubtask.getPipeName(), processorSubtask.getCreationTime()); + dataRegionProcessors.put(processorSubtask, processorSubtask); + } + + void register( + final PipeConnectorSubtask connectorSubtask, final String pipeName, final long creationTime) { + setNameAndCreationTime(pipeName, creationTime); + dataRegionConnectors.put(connectorSubtask, connectorSubtask); + } + + void register(final IoTDBSchemaRegionExtractor extractor) { + setNameAndCreationTime(extractor.getPipeName(), extractor.getCreationTime()); + schemaRegionExtractors.put(extractor, extractor); + } + + private void setNameAndCreationTime(final String pipeName, final long creationTime) { + this.pipeName = pipeName; + this.creationTime = creationTime; + } + + //////////////////////////// Rate //////////////////////////// + + void markDataRegionCommit() { + dataRegionCommitMeter.mark(); + } + + void markSchemaRegionCommit() { + schemaRegionCommitMeter.mark(); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeConnectorMetrics.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeDataRegionConnectorMetrics.java similarity index 77% rename from iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeConnectorMetrics.java rename to iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeDataRegionConnectorMetrics.java index caaeecf5e054..ef41dd993f71 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeConnectorMetrics.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeDataRegionConnectorMetrics.java @@ -38,9 +38,10 @@ import java.util.Objects; import java.util.concurrent.ConcurrentHashMap; -public class PipeConnectorMetrics implements IMetricSet { +public class PipeDataRegionConnectorMetrics implements IMetricSet { - private static final Logger LOGGER = LoggerFactory.getLogger(PipeConnectorMetrics.class); + private static final Logger LOGGER = + LoggerFactory.getLogger(PipeDataRegionConnectorMetrics.class); private volatile AbstractMetricService metricService; @@ -55,22 +56,22 @@ public class PipeConnectorMetrics implements IMetricSet { //////////////////////////// bindTo & unbindFrom (metric framework) //////////////////////////// @Override - public void bindTo(AbstractMetricService metricService) { + public void bindTo(final AbstractMetricService metricService) { this.metricService = metricService; - ImmutableSet taskIDs = ImmutableSet.copyOf(connectorMap.keySet()); + final ImmutableSet taskIDs = ImmutableSet.copyOf(connectorMap.keySet()); for (String taskID : taskIDs) { createMetrics(taskID); } } - private void createMetrics(String taskID) { + private void createMetrics(final String taskID) { createAutoGauge(taskID); createRate(taskID); } - private void createAutoGauge(String taskID) { - PipeConnectorSubtask connector = connectorMap.get(taskID); - // pending event count + private void createAutoGauge(final String taskID) { + final PipeConnectorSubtask connector = connectorMap.get(taskID); + // Pending event count metricService.createAutoGauge( Metric.UNTRANSFERRED_TABLET_COUNT.toString(), MetricLevel.IMPORTANT, @@ -104,7 +105,7 @@ private void createAutoGauge(String taskID) { String.valueOf(connector.getConnectorIndex()), Tag.CREATION_TIME.toString(), String.valueOf(connector.getCreationTime())); - // metrics related to IoTDBThriftAsyncConnector + // Metrics related to IoTDBThriftAsyncConnector metricService.createAutoGauge( Metric.PIPE_ASYNC_CONNECTOR_RETRY_EVENT_QUEUE_SIZE.toString(), MetricLevel.IMPORTANT, @@ -118,9 +119,9 @@ private void createAutoGauge(String taskID) { String.valueOf(connector.getCreationTime())); } - private void createRate(String taskID) { - PipeConnectorSubtask connector = connectorMap.get(taskID); - // transfer event rate + private void createRate(final String taskID) { + final PipeConnectorSubtask connector = connectorMap.get(taskID); + // Transfer event rate tabletRateMap.put( taskID, metricService.getOrCreateRate( @@ -157,24 +158,25 @@ private void createRate(String taskID) { } @Override - public void unbindFrom(AbstractMetricService metricService) { - ImmutableSet taskIDs = ImmutableSet.copyOf(connectorMap.keySet()); - for (String taskID : taskIDs) { + public void unbindFrom(final AbstractMetricService metricService) { + final ImmutableSet taskIDs = ImmutableSet.copyOf(connectorMap.keySet()); + for (final String taskID : taskIDs) { deregister(taskID); } if (!connectorMap.isEmpty()) { - LOGGER.warn("Failed to unbind from pipe connector metrics, connector map not empty"); + LOGGER.warn( + "Failed to unbind from pipe data region connector metrics, connector map not empty"); } } - private void removeMetrics(String taskID) { + private void removeMetrics(final String taskID) { removeAutoGauge(taskID); removeRate(taskID); } - private void removeAutoGauge(String taskID) { - PipeConnectorSubtask connector = connectorMap.get(taskID); - // pending event count + private void removeAutoGauge(final String taskID) { + final PipeConnectorSubtask connector = connectorMap.get(taskID); + // Pending event count metricService.remove( MetricType.AUTO_GAUGE, Metric.UNTRANSFERRED_TABLET_COUNT.toString(), @@ -202,7 +204,7 @@ private void removeAutoGauge(String taskID) { String.valueOf(connector.getConnectorIndex()), Tag.CREATION_TIME.toString(), String.valueOf(connector.getCreationTime())); - // metrics related to IoTDBThriftAsyncConnector + // Metrics related to IoTDBThriftAsyncConnector metricService.remove( MetricType.AUTO_GAUGE, Metric.PIPE_ASYNC_CONNECTOR_RETRY_EVENT_QUEUE_SIZE.toString(), @@ -214,9 +216,9 @@ private void removeAutoGauge(String taskID) { String.valueOf(connector.getCreationTime())); } - private void removeRate(String taskID) { - PipeConnectorSubtask connector = connectorMap.get(taskID); - // transfer event rate + private void removeRate(final String taskID) { + final PipeConnectorSubtask connector = connectorMap.get(taskID); + // Transfer event rate metricService.remove( MetricType.RATE, Metric.PIPE_CONNECTOR_TABLET_TRANSFER.toString(), @@ -251,18 +253,18 @@ private void removeRate(String taskID) { //////////////////////////// register & deregister (pipe integration) //////////////////////////// - public void register(@NonNull PipeConnectorSubtask pipeConnectorSubtask) { - String taskID = pipeConnectorSubtask.getTaskID(); + public void register(@NonNull final PipeConnectorSubtask pipeConnectorSubtask) { + final String taskID = pipeConnectorSubtask.getTaskID(); connectorMap.putIfAbsent(taskID, pipeConnectorSubtask); if (Objects.nonNull(metricService)) { createMetrics(taskID); } } - public void deregister(String taskID) { + public void deregister(final String taskID) { if (!connectorMap.containsKey(taskID)) { LOGGER.warn( - "Failed to deregister pipe connector metrics, PipeConnectorSubtask({}) does not exist", + "Failed to deregister pipe data region connector metrics, PipeConnectorSubtask({}) does not exist", taskID); return; } @@ -272,43 +274,41 @@ public void deregister(String taskID) { connectorMap.remove(taskID); } - public void markTabletEvent(String taskID) { + public void markTabletEvent(final String taskID) { if (Objects.isNull(metricService)) { return; } - Rate rate = tabletRateMap.get(taskID); + final Rate rate = tabletRateMap.get(taskID); if (rate == null) { LOGGER.warn( - "Failed to mark pipe connector tablet event, PipeConnectorSubtask({}) does not exist", + "Failed to mark pipe data region connector tablet event, PipeConnectorSubtask({}) does not exist", taskID); return; } rate.mark(); } - public void markTsFileEvent(String taskID) { + public void markTsFileEvent(final String taskID) { if (Objects.isNull(metricService)) { return; } - Rate rate = tsFileRateMap.get(taskID); + final Rate rate = tsFileRateMap.get(taskID); if (rate == null) { LOGGER.warn( - "Failed to mark pipe connector tsfile event, PipeConnectorSubtask({}) does not exist", + "Failed to mark pipe data region connector tsfile event, PipeConnectorSubtask({}) does not exist", taskID); return; } rate.mark(); } - public void markPipeHeartbeatEvent(String taskID) { + public void markPipeHeartbeatEvent(final String taskID) { if (Objects.isNull(metricService)) { return; } - Rate rate = pipeHeartbeatRateMap.get(taskID); + final Rate rate = pipeHeartbeatRateMap.get(taskID); if (rate == null) { - LOGGER.warn( - "Failed to mark pipe connector heartbeat event, PipeConnectorSubtask({}) does not exist", - taskID); + // Do not warn for the schema region events return; } rate.mark(); @@ -318,18 +318,19 @@ public void markPipeHeartbeatEvent(String taskID) { private static class PipeConnectorMetricsHolder { - private static final PipeConnectorMetrics INSTANCE = new PipeConnectorMetrics(); + private static final PipeDataRegionConnectorMetrics INSTANCE = + new PipeDataRegionConnectorMetrics(); private PipeConnectorMetricsHolder() { - // empty constructor + // Empty constructor } } - public static PipeConnectorMetrics getInstance() { - return PipeConnectorMetrics.PipeConnectorMetricsHolder.INSTANCE; + public static PipeDataRegionConnectorMetrics getInstance() { + return PipeDataRegionConnectorMetrics.PipeConnectorMetricsHolder.INSTANCE; } - private PipeConnectorMetrics() { - // empty constructor + private PipeDataRegionConnectorMetrics() { + // Empty constructor } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeExtractorMetrics.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeDataRegionExtractorMetrics.java similarity index 72% rename from iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeExtractorMetrics.java rename to iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeDataRegionExtractorMetrics.java index 310300453f7c..6382297d71fc 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeExtractorMetrics.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeDataRegionExtractorMetrics.java @@ -39,9 +39,10 @@ import java.util.Objects; import java.util.concurrent.ConcurrentHashMap; -public class PipeExtractorMetrics implements IMetricSet { +public class PipeDataRegionExtractorMetrics implements IMetricSet { - private static final Logger LOGGER = LoggerFactory.getLogger(PipeExtractorMetrics.class); + private static final Logger LOGGER = + LoggerFactory.getLogger(PipeDataRegionExtractorMetrics.class); private volatile AbstractMetricService metricService; @@ -62,23 +63,23 @@ public Map getExtractorMap() { //////////////////////////// bindTo & unbindFrom (metric framework) //////////////////////////// @Override - public void bindTo(AbstractMetricService metricService) { + public void bindTo(final AbstractMetricService metricService) { this.metricService = metricService; - ImmutableSet taskIDs = ImmutableSet.copyOf(extractorMap.keySet()); - for (String taskID : taskIDs) { + final ImmutableSet taskIDs = ImmutableSet.copyOf(extractorMap.keySet()); + for (final String taskID : taskIDs) { createMetrics(taskID); } } - private void createMetrics(String taskID) { + private void createMetrics(final String taskID) { createAutoGauge(taskID); createRate(taskID); createGauge(taskID); } - private void createAutoGauge(String taskID) { - IoTDBDataRegionExtractor extractor = extractorMap.get(taskID); - // pending event count + private void createAutoGauge(final String taskID) { + final IoTDBDataRegionExtractor extractor = extractorMap.get(taskID); + // Pending event count metricService.createAutoGauge( Metric.UNPROCESSED_HISTORICAL_TSFILE_COUNT.toString(), MetricLevel.IMPORTANT, @@ -87,7 +88,7 @@ private void createAutoGauge(String taskID) { Tag.NAME.toString(), extractor.getPipeName(), Tag.REGION.toString(), - String.valueOf(extractor.getDataRegionId()), + String.valueOf(extractor.getRegionId()), Tag.CREATION_TIME.toString(), String.valueOf(extractor.getCreationTime())); metricService.createAutoGauge( @@ -98,7 +99,7 @@ private void createAutoGauge(String taskID) { Tag.NAME.toString(), extractor.getPipeName(), Tag.REGION.toString(), - String.valueOf(extractor.getDataRegionId()), + String.valueOf(extractor.getRegionId()), Tag.CREATION_TIME.toString(), String.valueOf(extractor.getCreationTime())); metricService.createAutoGauge( @@ -109,7 +110,7 @@ private void createAutoGauge(String taskID) { Tag.NAME.toString(), extractor.getPipeName(), Tag.REGION.toString(), - String.valueOf(extractor.getDataRegionId()), + String.valueOf(extractor.getRegionId()), Tag.CREATION_TIME.toString(), String.valueOf(extractor.getCreationTime())); metricService.createAutoGauge( @@ -120,14 +121,14 @@ private void createAutoGauge(String taskID) { Tag.NAME.toString(), extractor.getPipeName(), Tag.REGION.toString(), - String.valueOf(extractor.getDataRegionId()), + String.valueOf(extractor.getRegionId()), Tag.CREATION_TIME.toString(), String.valueOf(extractor.getCreationTime())); } - private void createRate(String taskID) { - IoTDBDataRegionExtractor extractor = extractorMap.get(taskID); - // supply event rate + private void createRate(final String taskID) { + final IoTDBDataRegionExtractor extractor = extractorMap.get(taskID); + // Supply event rate tabletRateMap.put( taskID, metricService.getOrCreateRate( @@ -136,7 +137,7 @@ private void createRate(String taskID) { Tag.NAME.toString(), extractor.getPipeName(), Tag.REGION.toString(), - String.valueOf(extractor.getDataRegionId()), + String.valueOf(extractor.getRegionId()), Tag.CREATION_TIME.toString(), String.valueOf(extractor.getCreationTime()))); tsFileRateMap.put( @@ -147,7 +148,7 @@ private void createRate(String taskID) { Tag.NAME.toString(), extractor.getPipeName(), Tag.REGION.toString(), - String.valueOf(extractor.getDataRegionId()), + String.valueOf(extractor.getRegionId()), Tag.CREATION_TIME.toString(), String.valueOf(extractor.getCreationTime()))); pipeHeartbeatRateMap.put( @@ -158,14 +159,14 @@ private void createRate(String taskID) { Tag.NAME.toString(), extractor.getPipeName(), Tag.REGION.toString(), - String.valueOf(extractor.getDataRegionId()), + String.valueOf(extractor.getRegionId()), Tag.CREATION_TIME.toString(), String.valueOf(extractor.getCreationTime()))); } - private void createGauge(String taskID) { - IoTDBDataRegionExtractor extractor = extractorMap.get(taskID); - // tsfile epoch state + private void createGauge(final String taskID) { + final IoTDBDataRegionExtractor extractor = extractorMap.get(taskID); + // Tsfile epoch state recentProcessedTsFileEpochStateMap.put( taskID, metricService.getOrCreateGauge( @@ -174,15 +175,15 @@ private void createGauge(String taskID) { Tag.NAME.toString(), extractor.getPipeName(), Tag.REGION.toString(), - String.valueOf(extractor.getDataRegionId()), + String.valueOf(extractor.getRegionId()), Tag.CREATION_TIME.toString(), String.valueOf(extractor.getCreationTime()))); } @Override - public void unbindFrom(AbstractMetricService metricService) { - ImmutableSet taskIDs = ImmutableSet.copyOf(extractorMap.keySet()); - for (String taskID : taskIDs) { + public void unbindFrom(final AbstractMetricService metricService) { + final ImmutableSet taskIDs = ImmutableSet.copyOf(extractorMap.keySet()); + for (final String taskID : taskIDs) { deregister(taskID); } if (!extractorMap.isEmpty()) { @@ -190,14 +191,14 @@ public void unbindFrom(AbstractMetricService metricService) { } } - private void removeMetrics(String taskID) { + private void removeMetrics(final String taskID) { removeAutoGauge(taskID); removeRate(taskID); removeGauge(taskID); } - private void removeAutoGauge(String taskID) { - IoTDBDataRegionExtractor extractor = extractorMap.get(taskID); + private void removeAutoGauge(final String taskID) { + final IoTDBDataRegionExtractor extractor = extractorMap.get(taskID); // pending event count metricService.remove( MetricType.AUTO_GAUGE, @@ -205,7 +206,7 @@ private void removeAutoGauge(String taskID) { Tag.NAME.toString(), extractor.getPipeName(), Tag.REGION.toString(), - String.valueOf(extractor.getDataRegionId()), + String.valueOf(extractor.getRegionId()), Tag.CREATION_TIME.toString(), String.valueOf(extractor.getCreationTime())); metricService.remove( @@ -214,7 +215,7 @@ private void removeAutoGauge(String taskID) { Tag.NAME.toString(), extractor.getPipeName(), Tag.REGION.toString(), - String.valueOf(extractor.getDataRegionId()), + String.valueOf(extractor.getRegionId()), Tag.CREATION_TIME.toString(), String.valueOf(extractor.getCreationTime())); metricService.remove( @@ -223,7 +224,7 @@ private void removeAutoGauge(String taskID) { Tag.NAME.toString(), extractor.getPipeName(), Tag.REGION.toString(), - String.valueOf(extractor.getDataRegionId()), + String.valueOf(extractor.getRegionId()), Tag.CREATION_TIME.toString(), String.valueOf(extractor.getCreationTime())); metricService.remove( @@ -232,13 +233,13 @@ private void removeAutoGauge(String taskID) { Tag.NAME.toString(), extractor.getPipeName(), Tag.REGION.toString(), - String.valueOf(extractor.getDataRegionId()), + String.valueOf(extractor.getRegionId()), Tag.CREATION_TIME.toString(), String.valueOf(extractor.getCreationTime())); } - private void removeRate(String taskID) { - IoTDBDataRegionExtractor extractor = extractorMap.get(taskID); + private void removeRate(final String taskID) { + final IoTDBDataRegionExtractor extractor = extractorMap.get(taskID); // supply event rate metricService.remove( MetricType.RATE, @@ -246,7 +247,7 @@ private void removeRate(String taskID) { Tag.NAME.toString(), extractor.getPipeName(), Tag.REGION.toString(), - String.valueOf(extractor.getDataRegionId()), + String.valueOf(extractor.getRegionId()), Tag.CREATION_TIME.toString(), String.valueOf(extractor.getCreationTime())); metricService.remove( @@ -255,7 +256,7 @@ private void removeRate(String taskID) { Tag.NAME.toString(), extractor.getPipeName(), Tag.REGION.toString(), - String.valueOf(extractor.getDataRegionId()), + String.valueOf(extractor.getRegionId()), Tag.CREATION_TIME.toString(), String.valueOf(extractor.getCreationTime())); metricService.remove( @@ -264,7 +265,7 @@ private void removeRate(String taskID) { Tag.NAME.toString(), extractor.getPipeName(), Tag.REGION.toString(), - String.valueOf(extractor.getDataRegionId()), + String.valueOf(extractor.getRegionId()), Tag.CREATION_TIME.toString(), String.valueOf(extractor.getCreationTime())); tabletRateMap.remove(taskID); @@ -272,34 +273,34 @@ private void removeRate(String taskID) { pipeHeartbeatRateMap.remove(taskID); } - private void removeGauge(String taskID) { - IoTDBDataRegionExtractor extractor = extractorMap.get(taskID); - // tsfile epoch state + private void removeGauge(final String taskID) { + final IoTDBDataRegionExtractor extractor = extractorMap.get(taskID); + // Tsfile epoch state metricService.remove( MetricType.GAUGE, Metric.PIPE_EXTRACTOR_TSFILE_EPOCH_STATE.toString(), Tag.NAME.toString(), extractor.getPipeName(), Tag.REGION.toString(), - String.valueOf(extractor.getDataRegionId()), + String.valueOf(extractor.getRegionId()), Tag.CREATION_TIME.toString(), String.valueOf(extractor.getCreationTime())); } //////////////////////////// register & deregister (pipe integration) //////////////////////////// - public void register(@NonNull IoTDBDataRegionExtractor extractor) { - String taskID = extractor.getTaskID(); + public void register(@NonNull final IoTDBDataRegionExtractor extractor) { + final String taskID = extractor.getTaskID(); extractorMap.putIfAbsent(taskID, extractor); if (Objects.nonNull(metricService)) { createMetrics(taskID); } } - public void deregister(String taskID) { + public void deregister(final String taskID) { if (!extractorMap.containsKey(taskID)) { LOGGER.warn( - "Failed to deregister pipe extractor metrics, IoTDBDataRegionExtractor({}) does not exist", + "Failed to deregister pipe data region extractor metrics, IoTDBDataRegionExtractor({}) does not exist", taskID); return; } @@ -309,53 +310,54 @@ public void deregister(String taskID) { extractorMap.remove(taskID); } - public void markTabletEvent(String taskID) { + public void markTabletEvent(final String taskID) { if (Objects.isNull(metricService)) { return; } - Rate rate = tabletRateMap.get(taskID); + final Rate rate = tabletRateMap.get(taskID); if (rate == null) { LOGGER.warn( - "Failed to mark pipe extractor tablet event, IoTDBDataRegionExtractor({}) does not exist", + "Failed to mark pipe data region extractor tablet event, IoTDBDataRegionExtractor({}) does not exist", taskID); return; } rate.mark(); } - public void markTsFileEvent(String taskID) { + public void markTsFileEvent(final String taskID) { if (Objects.isNull(metricService)) { return; } - Rate rate = tsFileRateMap.get(taskID); + final Rate rate = tsFileRateMap.get(taskID); if (rate == null) { LOGGER.warn( - "Failed to mark pipe extractor tsfile event, IoTDBDataRegionExtractor({}) does not exist", + "Failed to mark pipe data region extractor tsfile event, IoTDBDataRegionExtractor({}) does not exist", taskID); return; } rate.mark(); } - public void markPipeHeartbeatEvent(String taskID) { + public void markPipeHeartbeatEvent(final String taskID) { if (Objects.isNull(metricService)) { return; } - Rate rate = pipeHeartbeatRateMap.get(taskID); + final Rate rate = pipeHeartbeatRateMap.get(taskID); if (rate == null) { LOGGER.warn( - "Failed to mark pipe extractor heartbeat event, IoTDBDataRegionExtractor({}) does not exist", + "Failed to mark pipe data region extractor heartbeat event, IoTDBDataRegionExtractor({}) does not exist", taskID); return; } rate.mark(); } - public void setRecentProcessedTsFileEpochState(String taskID, TsFileEpoch.State state) { + public void setRecentProcessedTsFileEpochState( + final String taskID, final TsFileEpoch.State state) { if (Objects.isNull(metricService)) { return; } - Gauge gauge = recentProcessedTsFileEpochStateMap.get(taskID); + final Gauge gauge = recentProcessedTsFileEpochStateMap.get(taskID); if (gauge == null) { LOGGER.warn( "Failed to set recent processed tsfile epoch state, PipeRealtimeDataRegionExtractor({}) does not exist", @@ -369,18 +371,19 @@ public void setRecentProcessedTsFileEpochState(String taskID, TsFileEpoch.State private static class PipeExtractorMetricsHolder { - private static final PipeExtractorMetrics INSTANCE = new PipeExtractorMetrics(); + private static final PipeDataRegionExtractorMetrics INSTANCE = + new PipeDataRegionExtractorMetrics(); private PipeExtractorMetricsHolder() { - // empty constructor + // Empty constructor } } - public static PipeExtractorMetrics getInstance() { - return PipeExtractorMetrics.PipeExtractorMetricsHolder.INSTANCE; + public static PipeDataRegionExtractorMetrics getInstance() { + return PipeDataRegionExtractorMetrics.PipeExtractorMetricsHolder.INSTANCE; } - private PipeExtractorMetrics() { - // empty constructor + private PipeDataRegionExtractorMetrics() { + // Empty constructor } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeSchemaRegionConnectorMetrics.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeSchemaRegionConnectorMetrics.java new file mode 100644 index 000000000000..52553c4ebb22 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeSchemaRegionConnectorMetrics.java @@ -0,0 +1,159 @@ +/* + * 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.db.pipe.metric; + +import org.apache.iotdb.commons.service.metric.enums.Metric; +import org.apache.iotdb.commons.service.metric.enums.Tag; +import org.apache.iotdb.db.pipe.task.subtask.connector.PipeConnectorSubtask; +import org.apache.iotdb.metrics.AbstractMetricService; +import org.apache.iotdb.metrics.metricsets.IMetricSet; +import org.apache.iotdb.metrics.type.Rate; +import org.apache.iotdb.metrics.utils.MetricLevel; +import org.apache.iotdb.metrics.utils.MetricType; + +import com.google.common.collect.ImmutableSet; +import org.checkerframework.checker.nullness.qual.NonNull; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Objects; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; + +public class PipeSchemaRegionConnectorMetrics implements IMetricSet { + + private static final Logger LOGGER = + LoggerFactory.getLogger(PipeSchemaRegionConnectorMetrics.class); + + private volatile AbstractMetricService metricService; + + private final ConcurrentMap connectorMap = + new ConcurrentHashMap<>(); + private final ConcurrentMap schemaRateMap = new ConcurrentHashMap<>(); + + //////////////////////////// bindTo & unbindFrom (metric framework) //////////////////////////// + + @Override + public void bindTo(final AbstractMetricService metricService) { + this.metricService = metricService; + ImmutableSet.copyOf(connectorMap.keySet()).forEach(this::createMetrics); + } + + private void createMetrics(final String taskID) { + createRate(taskID); + } + + private void createRate(final String taskID) { + final PipeConnectorSubtask connector = connectorMap.get(taskID); + // Transfer event rate + schemaRateMap.put( + taskID, + metricService.getOrCreateRate( + Metric.PIPE_CONNECTOR_SCHEMA_TRANSFER.toString(), + MetricLevel.IMPORTANT, + Tag.NAME.toString(), + connector.getAttributeSortedString(), + Tag.CREATION_TIME.toString(), + String.valueOf(connector.getCreationTime()))); + } + + @Override + public void unbindFrom(final AbstractMetricService metricService) { + ImmutableSet.copyOf(connectorMap.keySet()).forEach(this::deregister); + if (!connectorMap.isEmpty()) { + LOGGER.warn( + "Failed to unbind from pipe schema region connector metrics, connector map not empty"); + } + } + + private void removeMetrics(final String taskID) { + removeRate(taskID); + } + + private void removeRate(final String taskID) { + final PipeConnectorSubtask connector = connectorMap.get(taskID); + // Transfer event rate + metricService.remove( + MetricType.RATE, + Metric.PIPE_CONNECTOR_SCHEMA_TRANSFER.toString(), + Tag.NAME.toString(), + connector.getAttributeSortedString(), + Tag.CREATION_TIME.toString(), + String.valueOf(connector.getCreationTime())); + schemaRateMap.remove(taskID); + } + + //////////////////////////// Register & deregister (pipe integration) //////////////////////////// + + public void register(@NonNull final PipeConnectorSubtask pipeConnectorSubtask) { + final String taskID = pipeConnectorSubtask.getTaskID(); + connectorMap.putIfAbsent(taskID, pipeConnectorSubtask); + if (Objects.nonNull(metricService)) { + createMetrics(taskID); + } + } + + public void deregister(final String taskID) { + if (!connectorMap.containsKey(taskID)) { + LOGGER.warn( + "Failed to deregister pipe schema region connector metrics, PipeConnectorSubtask({}) does not exist", + taskID); + return; + } + if (Objects.nonNull(metricService)) { + removeMetrics(taskID); + } + connectorMap.remove(taskID); + } + + public void markSchemaEvent(final String taskID) { + if (Objects.isNull(metricService)) { + return; + } + final Rate rate = schemaRateMap.get(taskID); + if (rate == null) { + LOGGER.warn( + "Failed to mark pipe schema region write plan event, PipeConnectorSubtask({}) does not exist", + taskID); + return; + } + rate.mark(); + } + + //////////////////////////// singleton //////////////////////////// + + private static class PipeSchemaRegionConnectorMetricsHolder { + + private static final PipeSchemaRegionConnectorMetrics INSTANCE = + new PipeSchemaRegionConnectorMetrics(); + + private PipeSchemaRegionConnectorMetricsHolder() { + // Empty constructor + } + } + + public static PipeSchemaRegionConnectorMetrics getInstance() { + return PipeSchemaRegionConnectorMetricsHolder.INSTANCE; + } + + private PipeSchemaRegionConnectorMetrics() { + // Empty constructor + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeSchemaRegionExtractorMetrics.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeSchemaRegionExtractorMetrics.java new file mode 100644 index 000000000000..44480ecac0fa --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeSchemaRegionExtractorMetrics.java @@ -0,0 +1,144 @@ +/* + * 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.db.pipe.metric; + +import org.apache.iotdb.commons.service.metric.enums.Metric; +import org.apache.iotdb.commons.service.metric.enums.Tag; +import org.apache.iotdb.db.pipe.extractor.schemaregion.IoTDBSchemaRegionExtractor; +import org.apache.iotdb.metrics.AbstractMetricService; +import org.apache.iotdb.metrics.metricsets.IMetricSet; +import org.apache.iotdb.metrics.utils.MetricLevel; +import org.apache.iotdb.metrics.utils.MetricType; + +import com.google.common.collect.ImmutableSet; +import org.checkerframework.checker.nullness.qual.NonNull; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.ConcurrentHashMap; + +public class PipeSchemaRegionExtractorMetrics implements IMetricSet { + + private static final Logger LOGGER = + LoggerFactory.getLogger(PipeSchemaRegionExtractorMetrics.class); + + private volatile AbstractMetricService metricService; + + private final Map extractorMap = new ConcurrentHashMap<>(); + + //////////////////////////// bindTo & unbindFrom (metric framework) //////////////////////////// + + @Override + public void bindTo(final AbstractMetricService metricService) { + this.metricService = metricService; + ImmutableSet.copyOf(extractorMap.keySet()).forEach(this::createMetrics); + } + + private void createMetrics(final String taskID) { + createAutoGauge(taskID); + } + + private void createAutoGauge(final String taskID) { + final IoTDBSchemaRegionExtractor extractor = extractorMap.get(taskID); + metricService.createAutoGauge( + Metric.UNTRANSFERRED_SCHEMA_COUNT.toString(), + MetricLevel.IMPORTANT, + extractorMap.get(taskID), + IoTDBSchemaRegionExtractor::getUnTransferredEventCount, + Tag.NAME.toString(), + extractor.getPipeName(), + Tag.REGION.toString(), + String.valueOf(extractor.getRegionId()), + Tag.CREATION_TIME.toString(), + String.valueOf(extractor.getCreationTime())); + } + + @Override + public void unbindFrom(final AbstractMetricService metricService) { + ImmutableSet.copyOf(extractorMap.keySet()).forEach(this::deregister); + if (!extractorMap.isEmpty()) { + LOGGER.warn( + "Failed to unbind from pipe schema region extractor metrics, extractor map not empty"); + } + } + + private void removeMetrics(final String taskID) { + removeAutoGauge(taskID); + } + + private void removeAutoGauge(final String taskID) { + final IoTDBSchemaRegionExtractor extractor = extractorMap.get(taskID); + // pending event count + metricService.remove( + MetricType.AUTO_GAUGE, + Metric.UNTRANSFERRED_SCHEMA_COUNT.toString(), + Tag.NAME.toString(), + extractor.getPipeName(), + Tag.REGION.toString(), + String.valueOf(extractor.getRegionId()), + Tag.CREATION_TIME.toString(), + String.valueOf(extractor.getCreationTime())); + } + + //////////////////////////// register & deregister (pipe integration) //////////////////////////// + + public void register(@NonNull final IoTDBSchemaRegionExtractor extractor) { + final String taskID = extractor.getTaskID(); + extractorMap.putIfAbsent(taskID, extractor); + if (Objects.nonNull(metricService)) { + createMetrics(taskID); + } + } + + public void deregister(final String taskID) { + if (!extractorMap.containsKey(taskID)) { + LOGGER.warn( + "Failed to deregister pipe schema region extractor metrics, IoTDBSchemaRegionExtractor({}) does not exist", + taskID); + return; + } + if (Objects.nonNull(metricService)) { + removeMetrics(taskID); + } + extractorMap.remove(taskID); + } + + //////////////////////////// singleton //////////////////////////// + + private static class PipeSchemaRegionExtractorMetricsHolder { + + private static final PipeSchemaRegionExtractorMetrics INSTANCE = + new PipeSchemaRegionExtractorMetrics(); + + private PipeSchemaRegionExtractorMetricsHolder() { + // Empty constructor + } + } + + public static PipeSchemaRegionExtractorMetrics getInstance() { + return PipeSchemaRegionExtractorMetricsHolder.INSTANCE; + } + + private PipeSchemaRegionExtractorMetrics() { + // Empty constructor + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeSchemaRegionListenerMetrics.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeSchemaRegionListenerMetrics.java new file mode 100644 index 000000000000..3ee091c21478 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeSchemaRegionListenerMetrics.java @@ -0,0 +1,135 @@ +/* + * 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.db.pipe.metric; + +import org.apache.iotdb.commons.service.metric.enums.Metric; +import org.apache.iotdb.commons.service.metric.enums.Tag; +import org.apache.iotdb.db.pipe.extractor.schemaregion.SchemaRegionListeningQueue; +import org.apache.iotdb.metrics.AbstractMetricService; +import org.apache.iotdb.metrics.metricsets.IMetricSet; +import org.apache.iotdb.metrics.utils.MetricLevel; +import org.apache.iotdb.metrics.utils.MetricType; + +import com.google.common.collect.ImmutableSet; +import org.checkerframework.checker.nullness.qual.NonNull; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Objects; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; + +public class PipeSchemaRegionListenerMetrics implements IMetricSet { + + private static final Logger LOGGER = + LoggerFactory.getLogger(PipeSchemaRegionListenerMetrics.class); + + private volatile AbstractMetricService metricService; + + private final ConcurrentMap listeningQueueMap = + new ConcurrentHashMap<>(); + + //////////////////////////// bindTo & unbindFrom (metric framework) //////////////////////////// + + @Override + public void bindTo(final AbstractMetricService metricService) { + this.metricService = metricService; + ImmutableSet.copyOf(listeningQueueMap.keySet()).forEach(this::createMetrics); + } + + private void createMetrics(final Integer schemaRegionId) { + createAutoGauge(schemaRegionId); + } + + private void createAutoGauge(final Integer schemaRegionId) { + metricService.createAutoGauge( + Metric.PIPE_SCHEMA_LINKED_QUEUE_SIZE.toString(), + MetricLevel.IMPORTANT, + listeningQueueMap.get(schemaRegionId), + SchemaRegionListeningQueue::getSize, + Tag.REGION.toString(), + String.valueOf(schemaRegionId)); + } + + @Override + public void unbindFrom(final AbstractMetricService metricService) { + ImmutableSet.copyOf(listeningQueueMap.keySet()).forEach(this::deregister); + if (!listeningQueueMap.isEmpty()) { + LOGGER.warn( + "Failed to unbind from pipe schema region listener metrics, listening queue map not empty"); + } + } + + private void removeMetrics(final Integer schemaRegionId) { + removeAutoGauge(schemaRegionId); + } + + private void removeAutoGauge(final Integer schemaRegionId) { + metricService.remove( + MetricType.AUTO_GAUGE, + Metric.PIPE_SCHEMA_LINKED_QUEUE_SIZE.toString(), + Tag.REGION.toString(), + String.valueOf(schemaRegionId)); + } + + //////////////////////////// register & deregister (pipe integration) //////////////////////////// + + public void register( + @NonNull final SchemaRegionListeningQueue schemaRegionListeningQueue, + final Integer schemaRegionId) { + listeningQueueMap.putIfAbsent(schemaRegionId, schemaRegionListeningQueue); + if (Objects.nonNull(metricService)) { + createMetrics(schemaRegionId); + } + } + + public void deregister(final Integer schemaRegionId) { + if (!listeningQueueMap.containsKey(schemaRegionId)) { + LOGGER.warn( + "Failed to deregister schema region listener metrics, SchemaRegionListeningQueue({}) does not exist", + schemaRegionId); + return; + } + if (Objects.nonNull(metricService)) { + removeMetrics(schemaRegionId); + } + listeningQueueMap.remove(schemaRegionId); + } + + //////////////////////////// singleton //////////////////////////// + + private static class PipeSchemaRegionListenerMetricsHolder { + + private static final PipeSchemaRegionListenerMetrics INSTANCE = + new PipeSchemaRegionListenerMetrics(); + + private PipeSchemaRegionListenerMetricsHolder() { + // Empty constructor + } + } + + public static PipeSchemaRegionListenerMetrics getInstance() { + return PipeSchemaRegionListenerMetricsHolder.INSTANCE; + } + + private PipeSchemaRegionListenerMetrics() { + // Empty constructor + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/connection/PipeEventCollector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/connection/PipeEventCollector.java index 38779d44bb0a..30a16f3c2a71 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/connection/PipeEventCollector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/connection/PipeEventCollector.java @@ -20,6 +20,7 @@ package org.apache.iotdb.db.pipe.task.connection; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; +import org.apache.iotdb.commons.pipe.event.ProgressReportEvent; import org.apache.iotdb.commons.pipe.progress.PipeEventCommitManager; import org.apache.iotdb.commons.pipe.task.connection.BoundedBlockingPendingQueue; import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent; @@ -73,7 +74,7 @@ public void collect(final Event event) { parseAndCollectEvent((PipeRawTabletInsertionEvent) event); } else if (event instanceof PipeTsFileInsertionEvent) { parseAndCollectEvent((PipeTsFileInsertionEvent) event); - } else { + } else if (!(event instanceof ProgressReportEvent)) { collectEvent(event); } } catch (final PipeException e) { @@ -223,4 +224,8 @@ public int getTsFileInsertionEventCount() { public int getPipeHeartbeatEventCount() { return bufferQueue.getPipeHeartbeatEventCount(); } + + public int getEventCount() { + return bufferQueue.size(); + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/subtask/connector/PipeConnectorSubtask.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/subtask/connector/PipeConnectorSubtask.java index c45f2b6aedeb..300369ae9ed9 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/subtask/connector/PipeConnectorSubtask.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/subtask/connector/PipeConnectorSubtask.java @@ -28,7 +28,9 @@ import org.apache.iotdb.db.pipe.connector.protocol.thrift.async.IoTDBDataRegionAsyncConnector; import org.apache.iotdb.db.pipe.event.UserDefinedEnrichedEvent; import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent; -import org.apache.iotdb.db.pipe.metric.PipeConnectorMetrics; +import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionWritePlanEvent; +import org.apache.iotdb.db.pipe.metric.PipeDataRegionConnectorMetrics; +import org.apache.iotdb.db.pipe.metric.PipeSchemaRegionConnectorMetrics; import org.apache.iotdb.db.pipe.task.connection.PipeEventCollector; import org.apache.iotdb.db.utils.ErrorHandlingUtils; import org.apache.iotdb.pipe.api.PipeConnector; @@ -41,6 +43,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.concurrent.atomic.AtomicInteger; + public class PipeConnectorSubtask extends PipeAbstractConnectorSubtask { private static final Logger LOGGER = LoggerFactory.getLogger(PipeConnectorSubtask.class); @@ -63,17 +67,22 @@ public class PipeConnectorSubtask extends PipeAbstractConnectorSubtask { private long lastHeartbeatEventInjectTime = System.currentTimeMillis(); public PipeConnectorSubtask( - String taskID, - long creationTime, - String attributeSortedString, - int connectorIndex, - BoundedBlockingPendingQueue inputPendingQueue, - PipeConnector outputPipeConnector) { + final String taskID, + final long creationTime, + final String attributeSortedString, + final int connectorIndex, + final BoundedBlockingPendingQueue inputPendingQueue, + final PipeConnector outputPipeConnector) { super(taskID, creationTime, outputPipeConnector); this.attributeSortedString = attributeSortedString; this.connectorIndex = connectorIndex; this.inputPendingQueue = inputPendingQueue; - PipeConnectorMetrics.getInstance().register(this); + + if (!attributeSortedString.startsWith("schema_")) { + PipeDataRegionConnectorMetrics.getInstance().register(this); + } else { + PipeSchemaRegionConnectorMetrics.getInstance().register(this); + } } @Override @@ -100,10 +109,13 @@ protected boolean executeOnce() { if (event instanceof TabletInsertionEvent) { outputPipeConnector.transfer((TabletInsertionEvent) event); - PipeConnectorMetrics.getInstance().markTabletEvent(taskID); + PipeDataRegionConnectorMetrics.getInstance().markTabletEvent(taskID); } else if (event instanceof TsFileInsertionEvent) { outputPipeConnector.transfer((TsFileInsertionEvent) event); - PipeConnectorMetrics.getInstance().markTsFileEvent(taskID); + PipeDataRegionConnectorMetrics.getInstance().markTsFileEvent(taskID); + } else if (event instanceof PipeSchemaRegionWritePlanEvent) { + outputPipeConnector.transfer(event); + PipeSchemaRegionConnectorMetrics.getInstance().markSchemaEvent(taskID); } else if (event instanceof PipeHeartbeatEvent) { transferHeartbeatEvent((PipeHeartbeatEvent) event); } else { @@ -114,7 +126,7 @@ protected boolean executeOnce() { } decreaseReferenceCountAndReleaseLastEvent(true); - } catch (PipeException e) { + } catch (final PipeException e) { if (!isClosed.get()) { throw e; } else { @@ -124,7 +136,7 @@ protected boolean executeOnce() { e); clearReferenceCountAndReleaseLastEvent(); } - } catch (Exception e) { + } catch (final Exception e) { if (!isClosed.get()) { throw new PipeException( String.format( @@ -144,11 +156,11 @@ protected boolean executeOnce() { return true; } - private void transferHeartbeatEvent(PipeHeartbeatEvent event) { + private void transferHeartbeatEvent(final PipeHeartbeatEvent event) { try { outputPipeConnector.heartbeat(); outputPipeConnector.transfer(event); - } catch (Exception e) { + } catch (final Exception e) { throw new PipeConnectionException( "PipeConnector: " + outputPipeConnector.getClass().getName() @@ -160,16 +172,21 @@ private void transferHeartbeatEvent(PipeHeartbeatEvent event) { lastHeartbeatEventInjectTime = System.currentTimeMillis(); event.onTransferred(); - PipeConnectorMetrics.getInstance().markPipeHeartbeatEvent(taskID); + PipeDataRegionConnectorMetrics.getInstance().markPipeHeartbeatEvent(taskID); } @Override public void close() { - PipeConnectorMetrics.getInstance().deregister(taskID); + if (!attributeSortedString.startsWith("schema_")) { + PipeDataRegionConnectorMetrics.getInstance().deregister(taskID); + } else { + PipeSchemaRegionConnectorMetrics.getInstance().deregister(taskID); + } + isClosed.set(true); try { outputPipeConnector.close(); - } catch (Exception e) { + } catch (final Exception e) { LOGGER.info( "Exception occurred when closing pipe connector subtask {}, root cause: {}", taskID, @@ -193,7 +210,7 @@ public void close() { * When a pipe is dropped, the connector maybe reused and will not be closed. So we just discard * its queued events in the output pipe connector. */ - public void discardEventsOfPipe(String pipeNameToDrop) { + public void discardEventsOfPipe(final String pipeNameToDrop) { if (outputPipeConnector instanceof IoTDBDataRegionAsyncConnector) { ((IoTDBDataRegionAsyncConnector) outputPipeConnector).discardEventsOfPipe(pipeNameToDrop); } @@ -227,15 +244,42 @@ public int getAsyncConnectorRetryEventQueueSize() { : 0; } + // For performance, this will not acquire lock and does not guarantee the correct + // result. However, this shall not cause any exceptions when concurrently read & written. + public int getEventCount(final String pipeName) { + final AtomicInteger count = new AtomicInteger(0); + try { + inputPendingQueue.forEach( + event -> { + if (event instanceof EnrichedEvent + && pipeName.equals(((EnrichedEvent) event).getPipeName())) { + count.incrementAndGet(); + } + }); + } catch (Exception e) { + if (LOGGER.isDebugEnabled()) { + LOGGER.debug( + "Exception occurred when counting event of pipe {}, root cause: {}", + pipeName, + ErrorHandlingUtils.getRootCause(e).getMessage(), + e); + } + } + return count.get() + + (outputPipeConnector instanceof IoTDBDataRegionAsyncConnector + ? ((IoTDBDataRegionAsyncConnector) outputPipeConnector).getRetryEventCount(pipeName) + : 0); + } + //////////////////////////// Error report //////////////////////////// @Override - protected String getRootCause(Throwable throwable) { + protected String getRootCause(final Throwable throwable) { return ErrorHandlingUtils.getRootCause(throwable).getMessage(); } @Override - protected void report(EnrichedEvent event, PipeRuntimeException exception) { + protected void report(final EnrichedEvent event, final PipeRuntimeException exception) { PipeAgent.runtime().report(event, exception); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/subtask/connector/PipeConnectorSubtaskManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/subtask/connector/PipeConnectorSubtaskManager.java index 19fddd0c70d5..b28206bf306c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/subtask/connector/PipeConnectorSubtaskManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/subtask/connector/PipeConnectorSubtaskManager.java @@ -30,6 +30,7 @@ import org.apache.iotdb.commons.pipe.task.connection.BoundedBlockingPendingQueue; import org.apache.iotdb.db.pipe.agent.PipeAgent; import org.apache.iotdb.db.pipe.execution.PipeConnectorSubtaskExecutor; +import org.apache.iotdb.db.pipe.metric.PipeDataNodeRemainingEventAndTimeMetrics; import org.apache.iotdb.db.pipe.metric.PipeDataRegionEventCounter; import org.apache.iotdb.db.storageengine.StorageEngine; import org.apache.iotdb.pipe.api.PipeConnector; @@ -157,6 +158,11 @@ public synchronized String register( for (final PipeConnectorSubtaskLifeCycle lifeCycle : attributeSortedString2SubtaskLifeCycleMap.get(attributeSortedString)) { lifeCycle.register(); + if (isDataRegionConnector) { + PipeDataNodeRemainingEventAndTimeMetrics.getInstance() + .register( + lifeCycle.getSubtask(), environment.getPipeName(), environment.getCreationTime()); + } } return attributeSortedString; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/subtask/processor/PipeProcessorSubtask.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/subtask/processor/PipeProcessorSubtask.java index 49d50f7380c2..6839dc1c8280 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/subtask/processor/PipeProcessorSubtask.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/subtask/processor/PipeProcessorSubtask.java @@ -19,6 +19,7 @@ package org.apache.iotdb.db.pipe.task.subtask.processor; +import org.apache.iotdb.commons.consensus.DataRegionId; import org.apache.iotdb.commons.exception.pipe.PipeRuntimeException; import org.apache.iotdb.commons.exception.pipe.PipeRuntimeOutOfMemoryCriticalException; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; @@ -29,8 +30,10 @@ import org.apache.iotdb.db.pipe.agent.PipeAgent; import org.apache.iotdb.db.pipe.event.UserDefinedEnrichedEvent; import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent; +import org.apache.iotdb.db.pipe.metric.PipeDataNodeRemainingEventAndTimeMetrics; import org.apache.iotdb.db.pipe.metric.PipeProcessorMetrics; import org.apache.iotdb.db.pipe.task.connection.PipeEventCollector; +import org.apache.iotdb.db.storageengine.StorageEngine; import org.apache.iotdb.db.utils.ErrorHandlingUtils; import org.apache.iotdb.pipe.api.PipeProcessor; import org.apache.iotdb.pipe.api.event.Event; @@ -80,7 +83,12 @@ public PipeProcessorSubtask( this.inputEventSupplier = inputEventSupplier; this.pipeProcessor = pipeProcessor; this.outputEventCollector = outputEventCollector; - PipeProcessorMetrics.getInstance().register(this); + + // Only register dataRegions + if (StorageEngine.getInstance().getAllDataRegionIds().contains(new DataRegionId(regionId))) { + PipeProcessorMetrics.getInstance().register(this); + } + PipeDataNodeRemainingEventAndTimeMetrics.getInstance().register(this); } @Override @@ -259,6 +267,10 @@ public int getPipeHeartbeatEventCount() { return outputEventCollector.getPipeHeartbeatEventCount(); } + public int getEventCount() { + return outputEventCollector.getEventCount(); + } + //////////////////////////// Error report //////////////////////////// @Override diff --git a/iotdb-core/metrics/core/src/main/java/org/apache/iotdb/metrics/core/reporter/IoTDBJmxReporter.java b/iotdb-core/metrics/core/src/main/java/org/apache/iotdb/metrics/core/reporter/IoTDBJmxReporter.java index 9a00273e89a5..914b8cc9efc2 100644 --- a/iotdb-core/metrics/core/src/main/java/org/apache/iotdb/metrics/core/reporter/IoTDBJmxReporter.java +++ b/iotdb-core/metrics/core/src/main/java/org/apache/iotdb/metrics/core/reporter/IoTDBJmxReporter.java @@ -21,8 +21,8 @@ import org.apache.iotdb.metrics.AbstractMetricManager; import org.apache.iotdb.metrics.core.IoTDBMetricManager; -import org.apache.iotdb.metrics.core.uitls.IoTDBMetricObjNameFactory; -import org.apache.iotdb.metrics.core.uitls.ObjectNameFactory; +import org.apache.iotdb.metrics.core.utils.IoTDBMetricObjNameFactory; +import org.apache.iotdb.metrics.core.utils.ObjectNameFactory; import org.apache.iotdb.metrics.reporter.JmxReporter; import org.apache.iotdb.metrics.type.IMetric; import org.apache.iotdb.metrics.utils.MetricInfo; diff --git a/iotdb-core/metrics/core/src/main/java/org/apache/iotdb/metrics/core/type/IoTDBRate.java b/iotdb-core/metrics/core/src/main/java/org/apache/iotdb/metrics/core/type/IoTDBRate.java index 71b733b867e8..519aa5052d54 100644 --- a/iotdb-core/metrics/core/src/main/java/org/apache/iotdb/metrics/core/type/IoTDBRate.java +++ b/iotdb-core/metrics/core/src/main/java/org/apache/iotdb/metrics/core/type/IoTDBRate.java @@ -19,7 +19,7 @@ package org.apache.iotdb.metrics.core.type; -import org.apache.iotdb.metrics.core.uitls.IoTDBMovingAverage; +import org.apache.iotdb.metrics.core.utils.IoTDBMovingAverage; import org.apache.iotdb.metrics.type.Rate; import org.apache.iotdb.metrics.utils.AbstractMetricMBean; diff --git a/iotdb-core/metrics/core/src/main/java/org/apache/iotdb/metrics/core/uitls/IoTDBCachedGauge.java b/iotdb-core/metrics/core/src/main/java/org/apache/iotdb/metrics/core/utils/IoTDBCachedGauge.java similarity index 98% rename from iotdb-core/metrics/core/src/main/java/org/apache/iotdb/metrics/core/uitls/IoTDBCachedGauge.java rename to iotdb-core/metrics/core/src/main/java/org/apache/iotdb/metrics/core/utils/IoTDBCachedGauge.java index 9c1dd7194a26..6fcdcbba17dd 100644 --- a/iotdb-core/metrics/core/src/main/java/org/apache/iotdb/metrics/core/uitls/IoTDBCachedGauge.java +++ b/iotdb-core/metrics/core/src/main/java/org/apache/iotdb/metrics/core/utils/IoTDBCachedGauge.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.iotdb.metrics.core.uitls; +package org.apache.iotdb.metrics.core.utils; import org.apache.iotdb.metrics.type.AutoGauge; import org.apache.iotdb.metrics.utils.AbstractMetricMBean; diff --git a/iotdb-core/metrics/core/src/main/java/org/apache/iotdb/metrics/core/uitls/IoTDBCachedGaugeMBean.java b/iotdb-core/metrics/core/src/main/java/org/apache/iotdb/metrics/core/utils/IoTDBCachedGaugeMBean.java similarity index 95% rename from iotdb-core/metrics/core/src/main/java/org/apache/iotdb/metrics/core/uitls/IoTDBCachedGaugeMBean.java rename to iotdb-core/metrics/core/src/main/java/org/apache/iotdb/metrics/core/utils/IoTDBCachedGaugeMBean.java index 165aa7670ef0..8986eb26926c 100644 --- a/iotdb-core/metrics/core/src/main/java/org/apache/iotdb/metrics/core/uitls/IoTDBCachedGaugeMBean.java +++ b/iotdb-core/metrics/core/src/main/java/org/apache/iotdb/metrics/core/utils/IoTDBCachedGaugeMBean.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.iotdb.metrics.core.uitls; +package org.apache.iotdb.metrics.core.utils; import javax.management.ObjectName; diff --git a/iotdb-core/metrics/core/src/main/java/org/apache/iotdb/metrics/core/uitls/IoTDBMetricObjNameFactory.java b/iotdb-core/metrics/core/src/main/java/org/apache/iotdb/metrics/core/utils/IoTDBMetricObjNameFactory.java similarity index 98% rename from iotdb-core/metrics/core/src/main/java/org/apache/iotdb/metrics/core/uitls/IoTDBMetricObjNameFactory.java rename to iotdb-core/metrics/core/src/main/java/org/apache/iotdb/metrics/core/utils/IoTDBMetricObjNameFactory.java index 7f2b0aa733fc..ee76cf3805e7 100644 --- a/iotdb-core/metrics/core/src/main/java/org/apache/iotdb/metrics/core/uitls/IoTDBMetricObjNameFactory.java +++ b/iotdb-core/metrics/core/src/main/java/org/apache/iotdb/metrics/core/utils/IoTDBMetricObjNameFactory.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.iotdb.metrics.core.uitls; +package org.apache.iotdb.metrics.core.utils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/iotdb-core/metrics/core/src/main/java/org/apache/iotdb/metrics/core/uitls/IoTDBMovingAverage.java b/iotdb-core/metrics/core/src/main/java/org/apache/iotdb/metrics/core/utils/IoTDBMovingAverage.java similarity index 98% rename from iotdb-core/metrics/core/src/main/java/org/apache/iotdb/metrics/core/uitls/IoTDBMovingAverage.java rename to iotdb-core/metrics/core/src/main/java/org/apache/iotdb/metrics/core/utils/IoTDBMovingAverage.java index 7edf7b06b596..2b4d982bb364 100644 --- a/iotdb-core/metrics/core/src/main/java/org/apache/iotdb/metrics/core/uitls/IoTDBMovingAverage.java +++ b/iotdb-core/metrics/core/src/main/java/org/apache/iotdb/metrics/core/utils/IoTDBMovingAverage.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.iotdb.metrics.core.uitls; +package org.apache.iotdb.metrics.core.utils; import com.codahale.metrics.Clock; import com.codahale.metrics.EWMA; diff --git a/iotdb-core/metrics/core/src/main/java/org/apache/iotdb/metrics/core/uitls/ObjectNameFactory.java b/iotdb-core/metrics/core/src/main/java/org/apache/iotdb/metrics/core/utils/ObjectNameFactory.java similarity index 96% rename from iotdb-core/metrics/core/src/main/java/org/apache/iotdb/metrics/core/uitls/ObjectNameFactory.java rename to iotdb-core/metrics/core/src/main/java/org/apache/iotdb/metrics/core/utils/ObjectNameFactory.java index 3b732329a142..1c019781300f 100644 --- a/iotdb-core/metrics/core/src/main/java/org/apache/iotdb/metrics/core/uitls/ObjectNameFactory.java +++ b/iotdb-core/metrics/core/src/main/java/org/apache/iotdb/metrics/core/utils/ObjectNameFactory.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.iotdb.metrics.core.uitls; +package org.apache.iotdb.metrics.core.utils; import javax.management.ObjectName; 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 44ef38949891..0277c84cb9ab 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 @@ -243,7 +243,7 @@ public KeyedObjectPool createClie new AsyncPipeDataTransferServiceClient.Factory( manager, new ThriftClientProperty.Builder() - .setConnectionTimeoutMs((int) conf.getPipeConnectorTransferTimeoutMs()) + .setConnectionTimeoutMs(conf.getPipeConnectorTransferTimeoutMs()) .setRpcThriftCompressionEnabled( conf.isPipeConnectorRPCThriftCompressionEnabled()) .setSelectorNumOfAsyncClientManager( diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java index 8cc7ff7eefa3..12ef440b81b9 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java @@ -230,10 +230,11 @@ public class CommonConfig { private float pipeLeaderCacheMemoryUsagePercentage = 0.1F; private long pipeListeningQueueTransferSnapshotThreshold = 1000; private int pipeSnapshotExecutionMaxBatchSize = 1000; + private double pipeRemainingTimeCommitRateSmoothingFactor = 0.5; - private long twoStageAggregateMaxCombinerLiveTimeInMs = 8 * 60 * 1000; // 8 minutes - private long twoStageAggregateDataRegionInfoCacheTimeInMs = 3 * 60 * 1000; // 3 minutes - private long twoStageAggregateSenderEndPointsCacheInMs = 3 * 60 * 1000; // 3 minutes + private long twoStageAggregateMaxCombinerLiveTimeInMs = 8 * 60 * 1000L; // 8 minutes + private long twoStageAggregateDataRegionInfoCacheTimeInMs = 3 * 60 * 1000L; // 3 minutes + private long twoStageAggregateSenderEndPointsCacheInMs = 3 * 60 * 1000L; // 3 minutes private int subscriptionSubtaskExecutorMaxThreadNum = Math.min(5, Math.max(1, Runtime.getRuntime().availableProcessors() / 2)); @@ -1008,6 +1009,15 @@ public void setPipeSnapshotExecutionMaxBatchSize(int pipeSnapshotExecutionMaxBat this.pipeSnapshotExecutionMaxBatchSize = pipeSnapshotExecutionMaxBatchSize; } + public double getPipeRemainingTimeCommitRateSmoothingFactor() { + return pipeRemainingTimeCommitRateSmoothingFactor; + } + + public void setPipeRemainingTimeCommitRateSmoothingFactor( + double pipeRemainingTimeCommitRateSmoothingFactor) { + this.pipeRemainingTimeCommitRateSmoothingFactor = pipeRemainingTimeCommitRateSmoothingFactor; + } + public long getTwoStageAggregateMaxCombinerLiveTimeInMs() { return twoStageAggregateMaxCombinerLiveTimeInMs; } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonDescriptor.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonDescriptor.java index 9d0bd3cc2c01..e1ec6c03dbbf 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonDescriptor.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonDescriptor.java @@ -533,6 +533,11 @@ private void loadPipeProps(Properties properties) { properties.getProperty( "pipe_snapshot_execution_max_batch_size", String.valueOf(config.getPipeSnapshotExecutionMaxBatchSize())))); + config.setPipeRemainingTimeCommitRateSmoothingFactor( + Double.parseDouble( + properties.getProperty( + "pipe_remaining_time_commit_rate_smoothing_factor", + String.valueOf(config.getPipeRemainingTimeCommitRateSmoothingFactor())))); config.setTwoStageAggregateMaxCombinerLiveTimeInMs( Long.parseLong( diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/PipeTaskAgent.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/PipeTaskAgent.java index 152df8973970..a1844bef5722 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/PipeTaskAgent.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/PipeTaskAgent.java @@ -85,10 +85,10 @@ protected void acquireReadLock() { pipeMetaKeeper.acquireReadLock(); } - protected boolean tryReadLockWithTimeOut(long timeOutInSeconds) { + protected boolean tryReadLockWithTimeOut(final long timeOutInSeconds) { try { return pipeMetaKeeper.tryReadLock(timeOutInSeconds); - } catch (InterruptedException e) { + } catch (final InterruptedException e) { Thread.currentThread().interrupt(); LOGGER.warn("Interruption during requiring pipeMetaKeeper read lock.", e); return false; @@ -106,7 +106,7 @@ protected void acquireWriteLock() { protected boolean tryWriteLockWithTimeOut(long timeOutInSeconds) { try { return pipeMetaKeeper.tryWriteLock(timeOutInSeconds); - } catch (InterruptedException e) { + } catch (final InterruptedException e) { Thread.currentThread().interrupt(); LOGGER.warn("Interruption during requiring pipeMetaKeeper write lock.", e); return false; @@ -120,7 +120,7 @@ protected void releaseWriteLock() { ////////////////////////// Pipe Task Management Entry ////////////////////////// public TPushPipeMetaRespExceptionMessage handleSinglePipeMetaChanges( - PipeMeta pipeMetaFromCoordinator) { + final PipeMeta pipeMetaFromCoordinator) { acquireWriteLock(); try { return handleSinglePipeMetaChangesInternal(pipeMetaFromCoordinator); @@ -130,7 +130,7 @@ public TPushPipeMetaRespExceptionMessage handleSinglePipeMetaChanges( } protected TPushPipeMetaRespExceptionMessage handleSinglePipeMetaChangesInternal( - PipeMeta pipeMetaFromCoordinator) { + final PipeMeta pipeMetaFromCoordinator) { // Do nothing if node is removing or removed if (isShutdown()) { return null; @@ -139,7 +139,7 @@ protected TPushPipeMetaRespExceptionMessage handleSinglePipeMetaChangesInternal( try { executeSinglePipeMetaChanges(pipeMetaFromCoordinator); return null; - } catch (Exception e) { + } catch (final Exception e) { final String pipeName = pipeMetaFromCoordinator.getStaticMeta().getPipeName(); final String errorMessage = String.format( @@ -190,9 +190,9 @@ private void executeSinglePipeMetaChanges(final PipeMeta metaFromCoordinator) } private void executeSinglePipeRuntimeMetaChanges( - /* @NotNull */ PipeStaticMeta pipeStaticMeta, - /* @NotNull */ PipeRuntimeMeta runtimeMetaFromCoordinator, - /* @NotNull */ PipeRuntimeMeta runtimeMetaInAgent) + /* @NotNull */ final PipeStaticMeta pipeStaticMeta, + /* @NotNull */ final PipeRuntimeMeta runtimeMetaFromCoordinator, + /* @NotNull */ final PipeRuntimeMeta runtimeMetaInAgent) throws IllegalPathException { // 1. Handle region group leader changed first final Map consensusGroupIdToTaskMetaMapFromCoordinator = @@ -290,7 +290,7 @@ private void executeSinglePipeRuntimeMetaChanges( } } - public TPushPipeMetaRespExceptionMessage handleDropPipe(String pipeName) { + public TPushPipeMetaRespExceptionMessage handleDropPipe(final String pipeName) { acquireWriteLock(); try { return handleDropPipeInternal(pipeName); @@ -299,7 +299,7 @@ public TPushPipeMetaRespExceptionMessage handleDropPipe(String pipeName) { } } - protected TPushPipeMetaRespExceptionMessage handleDropPipeInternal(String pipeName) { + protected TPushPipeMetaRespExceptionMessage handleDropPipeInternal(final String pipeName) { // Do nothing if node is removing or removed if (isShutdown()) { return null; @@ -308,7 +308,7 @@ protected TPushPipeMetaRespExceptionMessage handleDropPipeInternal(String pipeNa try { dropPipe(pipeName); return null; - } catch (Exception e) { + } catch (final Exception e) { final String errorMessage = String.format("Failed to drop pipe %s, because %s", pipeName, e.getMessage()); LOGGER.warn("Failed to drop pipe {}", pipeName, e); @@ -318,7 +318,7 @@ protected TPushPipeMetaRespExceptionMessage handleDropPipeInternal(String pipeNa } public List handlePipeMetaChanges( - List pipeMetaListFromCoordinator) { + final List pipeMetaListFromCoordinator) { acquireWriteLock(); try { return handlePipeMetaChangesInternal(pipeMetaListFromCoordinator); @@ -328,7 +328,7 @@ public List handlePipeMetaChanges( } protected List handlePipeMetaChangesInternal( - List pipeMetaListFromCoordinator) { + final List pipeMetaListFromCoordinator) { // Do nothing if the node is removing or removed if (isShutdown()) { return Collections.emptyList(); @@ -341,7 +341,7 @@ protected List handlePipeMetaChangesInternal( for (final PipeMeta metaFromCoordinator : pipeMetaListFromCoordinator) { try { executeSinglePipeMetaChanges(metaFromCoordinator); - } catch (Exception e) { + } catch (final Exception e) { final String pipeName = metaFromCoordinator.getStaticMeta().getPipeName(); final String errorMessage = String.format( @@ -365,7 +365,7 @@ protected List handlePipeMetaChangesInternal( if (!pipeNamesFromCoordinator.contains(pipeName)) { dropPipe(metaInAgent.getStaticMeta().getPipeName()); } - } catch (Exception e) { + } catch (final Exception e) { // Report the exception message for CN to sense the failure of meta sync final String errorMessage = String.format( @@ -415,7 +415,7 @@ private void dropAllPipeTasksInternal() { * if the pipe already exists or is created but should not be started * @throws IllegalStateException if the status is illegal */ - private boolean createPipe(PipeMeta pipeMetaFromCoordinator) throws IllegalPathException { + private boolean createPipe(final PipeMeta pipeMetaFromCoordinator) throws IllegalPathException { final String pipeName = pipeMetaFromCoordinator.getStaticMeta().getPipeName(); final long creationTime = pipeMetaFromCoordinator.getStaticMeta().getCreationTime(); @@ -459,10 +459,10 @@ private boolean createPipe(PipeMeta pipeMetaFromCoordinator) throws IllegalPathE return needToStartPipe; } - protected abstract Map buildPipeTasks(PipeMeta pipeMetaFromCoordinator) + protected abstract Map buildPipeTasks(final PipeMeta pipeMetaFromCoordinator) throws IllegalPathException; - private void dropPipe(String pipeName, long creationTime) { + protected void dropPipe(final String pipeName, final long creationTime) { final PipeMeta existedPipeMeta = pipeMetaKeeper.getPipeMeta(pipeName); if (!checkBeforeDropPipe(existedPipeMeta, pipeName, creationTime)) { @@ -498,7 +498,7 @@ private void dropPipe(String pipeName, long creationTime) { pipeMetaKeeper.removePipeMeta(pipeName); } - private void dropPipe(String pipeName) { + protected void dropPipe(final String pipeName) { final PipeMeta existedPipeMeta = pipeMetaKeeper.getPipeMeta(pipeName); if (!checkBeforeDropPipe(existedPipeMeta, pipeName)) { @@ -531,7 +531,7 @@ private void dropPipe(String pipeName) { pipeMetaKeeper.removePipeMeta(pipeName); } - private void startPipe(String pipeName, long creationTime) { + private void startPipe(final String pipeName, final long creationTime) { final PipeMeta existedPipeMeta = pipeMetaKeeper.getPipeMeta(pipeName); if (!checkBeforeStartPipe(existedPipeMeta, pipeName, creationTime)) { @@ -568,7 +568,7 @@ private void startPipe(String pipeName, long creationTime) { .forEach(PipeTaskMeta::clearExceptionMessages); } - protected void stopPipe(String pipeName, long creationTime) { + protected void stopPipe(final String pipeName, final long creationTime) { final PipeMeta existedPipeMeta = pipeMetaKeeper.getPipeMeta(pipeName); if (!checkBeforeStopPipe(existedPipeMeta, pipeName, creationTime)) { @@ -608,7 +608,8 @@ protected void stopPipe(String pipeName, long creationTime) { * @throws IllegalStateException if current {@link PipeStatus} is illegal. */ protected boolean checkBeforeCreatePipe( - PipeMeta existedPipeMeta, String pipeName, long creationTime) throws IllegalStateException { + final PipeMeta existedPipeMeta, final String pipeName, final long creationTime) + throws IllegalStateException { if (existedPipeMeta.getStaticMeta().getCreationTime() == creationTime) { final PipeStatus status = existedPipeMeta.getRuntimeMeta().getStatus().get(); switch (status) { @@ -652,7 +653,8 @@ protected boolean checkBeforeCreatePipe( * @throws IllegalStateException if current {@link PipeStatus} is illegal. */ protected boolean checkBeforeStartPipe( - PipeMeta existedPipeMeta, String pipeName, long creationTime) throws IllegalStateException { + final PipeMeta existedPipeMeta, final String pipeName, final long creationTime) + throws IllegalStateException { if (existedPipeMeta == null) { LOGGER.info( "Pipe {} (creation time = {}) has already been dropped or has not been created. " @@ -718,7 +720,8 @@ protected boolean checkBeforeStartPipe( * @throws IllegalStateException if current {@link PipeStatus} is illegal. */ protected boolean checkBeforeStopPipe( - PipeMeta existedPipeMeta, String pipeName, long creationTime) throws IllegalStateException { + final PipeMeta existedPipeMeta, final String pipeName, final long creationTime) + throws IllegalStateException { if (existedPipeMeta == null) { LOGGER.info( "Pipe {} (creation time = {}) has already been dropped or has not been created. " @@ -781,7 +784,7 @@ protected boolean checkBeforeStopPipe( * @return {@code true} if need to drop {@link PipeTask}s, {@code false} if no need to drop. */ protected boolean checkBeforeDropPipe( - PipeMeta existedPipeMeta, String pipeName, long creationTime) { + final PipeMeta existedPipeMeta, final String pipeName, final long creationTime) { if (existedPipeMeta == null) { LOGGER.info( "Pipe {} (creation time = {}) has already been dropped or has not been created. " @@ -809,7 +812,7 @@ protected boolean checkBeforeDropPipe( * * @return {@code true} if need to drop {@link PipeTask}s, {@code false} if no need to drop. */ - protected boolean checkBeforeDropPipe(PipeMeta existedPipeMeta, String pipeName) { + protected boolean checkBeforeDropPipe(final PipeMeta existedPipeMeta, final String pipeName) { if (existedPipeMeta == null) { LOGGER.info( "Pipe {} has already been dropped or has not been created. Skip dropping.", pipeName); @@ -822,10 +825,12 @@ protected boolean checkBeforeDropPipe(PipeMeta existedPipeMeta, String pipeName) ///////////////////////// Manage by consensusGroupId ///////////////////////// protected abstract void createPipeTask( - int consensusGroupId, PipeStaticMeta pipeStaticMeta, PipeTaskMeta pipeTaskMeta) + final int consensusGroupId, + final PipeStaticMeta pipeStaticMeta, + final PipeTaskMeta pipeTaskMeta) throws IllegalPathException; - private void dropPipeTask(int consensusGroupId, PipeStaticMeta pipeStaticMeta) { + private void dropPipeTask(final int consensusGroupId, final PipeStaticMeta pipeStaticMeta) { pipeMetaKeeper .getPipeMeta(pipeStaticMeta.getPipeName()) .getRuntimeMeta() @@ -837,7 +842,7 @@ private void dropPipeTask(int consensusGroupId, PipeStaticMeta pipeStaticMeta) { } } - private void startPipeTask(int consensusGroupId, PipeStaticMeta pipeStaticMeta) { + private void startPipeTask(final int consensusGroupId, final PipeStaticMeta pipeStaticMeta) { final PipeTask pipeTask = pipeTaskManager.getPipeTask(pipeStaticMeta, consensusGroupId); if (pipeTask != null) { pipeTask.start(); @@ -870,13 +875,13 @@ protected void stopAllPipesWithCriticalException(final int currentNodeId) { ++retryCount); } } - } catch (InterruptedException e) { + } catch (final InterruptedException e) { LOGGER.error( "Interrupted when trying to stop all pipes with critical exception, exception message: {}", e.getMessage(), e); Thread.currentThread().interrupt(); - } catch (Exception e) { + } catch (final Exception e) { LOGGER.error( "Failed to stop all pipes with critical exception, exception message: {}", e.getMessage(), @@ -885,7 +890,7 @@ protected void stopAllPipesWithCriticalException(final int currentNodeId) { }); } - private void stopAllPipesWithCriticalExceptionInternal(int currentNodeId) { + private void stopAllPipesWithCriticalExceptionInternal(final int currentNodeId) { // 1. track exception in all pipe tasks that share the same connector that have critical // exceptions. final Map @@ -980,7 +985,7 @@ private void stopAllPipesWithCriticalExceptionInternal(int currentNodeId) { }); } - public void collectPipeMetaList(TPipeHeartbeatReq req, TPipeHeartbeatResp resp) + public void collectPipeMetaList(final TPipeHeartbeatReq req, final TPipeHeartbeatResp resp) throws TException { acquireReadLock(); try { @@ -991,5 +996,5 @@ public void collectPipeMetaList(TPipeHeartbeatReq req, TPipeHeartbeatResp resp) } protected abstract void collectPipeMetaListInternal( - TPipeHeartbeatReq req, TPipeHeartbeatResp resp) throws TException; + final TPipeHeartbeatReq req, final TPipeHeartbeatResp resp) throws TException; } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java index 351f4da40ade..1821fe74a7c8 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java @@ -139,6 +139,10 @@ public int getPipeSnapshotExecutionMaxBatchSize() { return COMMON_CONFIG.getPipeSnapshotExecutionMaxBatchSize(); } + public double getPipeRemainingTimeCommitRateSmoothingFactor() { + return COMMON_CONFIG.getPipeRemainingTimeCommitRateSmoothingFactor(); + } + /////////////////////////////// Meta Consistency /////////////////////////////// public boolean isSeperatedPipeHeartbeatEnabled() { @@ -318,6 +322,9 @@ public void printAllConfigs() { "PipeListeningQueueTransferSnapshotThreshold: {}", getPipeListeningQueueTransferSnapshotThreshold()); LOGGER.info("PipeSnapshotExecutionMaxBatchSize: {}", getPipeSnapshotExecutionMaxBatchSize()); + LOGGER.info( + "PipeRemainingTimeCommitRateSmoothingFactor: {}", + getPipeRemainingTimeCommitRateSmoothingFactor()); LOGGER.info("PipeAsyncConnectorSelectorNumber: {}", getPipeAsyncConnectorSelectorNumber()); LOGGER.info("PipeAsyncConnectorMaxClientNumber: {}", getPipeAsyncConnectorMaxClientNumber()); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/datastructure/queue/listening/AbstractSerializableListeningQueue.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/datastructure/queue/listening/AbstractSerializableListeningQueue.java index 21ef5d10d325..10482fd40192 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/datastructure/queue/listening/AbstractSerializableListeningQueue.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/datastructure/queue/listening/AbstractSerializableListeningQueue.java @@ -193,4 +193,14 @@ public synchronized void close() throws IOException { public synchronized boolean isOpened() { return !isClosed.get(); } + + /////////////////////////////// APIs provided for metric framework /////////////////////////////// + + public long getSize() { + return queue.size(); + } + + public long getTailIndex() { + return queue.getTailIndex(); + } } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/event/PipeWritePlanEvent.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/event/PipeWritePlanEvent.java index 8cf1a7d3154b..d37aee82aaad 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/event/PipeWritePlanEvent.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/event/PipeWritePlanEvent.java @@ -23,13 +23,8 @@ import org.apache.iotdb.commons.pipe.pattern.PipePattern; import org.apache.iotdb.commons.pipe.task.meta.PipeTaskMeta; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - public abstract class PipeWritePlanEvent extends EnrichedEvent implements SerializableEvent { - private static final Logger LOGGER = LoggerFactory.getLogger(PipeWritePlanEvent.class); - protected boolean isGeneratedByPipe; protected ProgressIndex progressIndex; @@ -43,13 +38,13 @@ protected PipeWritePlanEvent( this.isGeneratedByPipe = isGeneratedByPipe; } - /** This event doesn't share resources with other events. */ + /** {@link PipeWritePlanEvent} does not share resources with other events. */ @Override public boolean internallyIncreaseResourceReferenceCount(final String holderMessage) { return true; } - /** This event doesn't share resources with other events. */ + /** This {@link PipeWritePlanEvent} does not share resources with other events. */ @Override public boolean internallyDecreaseResourceReferenceCount(final String holderMessage) { return true; diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/event/ProgressReportEvent.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/event/ProgressReportEvent.java new file mode 100644 index 000000000000..2851b03593c1 --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/event/ProgressReportEvent.java @@ -0,0 +1,83 @@ +/* + * 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.pipe.event; + +import org.apache.iotdb.commons.consensus.index.ProgressIndex; +import org.apache.iotdb.commons.pipe.pattern.PipePattern; +import org.apache.iotdb.commons.pipe.task.meta.PipeTaskMeta; + +/** + * {@link ProgressReportEvent} is an {@link EnrichedEvent} that is used only for progress report. It + * is bind to a {@link ProgressIndex} and will be committed after all its preceding {@link + * EnrichedEvent}s. + */ +public class ProgressReportEvent extends EnrichedEvent { + + private ProgressIndex progressIndex; + + public ProgressReportEvent( + final String pipeName, + final PipeTaskMeta pipeTaskMeta, + final PipePattern pipePattern, + final long startTime, + final long endTime) { + super(pipeName, pipeTaskMeta, pipePattern, startTime, endTime); + } + + @Override + public boolean internallyIncreaseResourceReferenceCount(final String holderMessage) { + return true; + } + + @Override + public boolean internallyDecreaseResourceReferenceCount(final String holderMessage) { + return true; + } + + @Override + public void bindProgressIndex(final ProgressIndex progressIndex) { + this.progressIndex = progressIndex; + } + + @Override + public ProgressIndex getProgressIndex() { + return progressIndex; + } + + @Override + public EnrichedEvent shallowCopySelfAndBindPipeTaskMetaForProgressReport( + final String pipeName, + final PipeTaskMeta pipeTaskMeta, + final PipePattern pattern, + final long startTime, + final long endTime) { + return new ProgressReportEvent(pipeName, pipeTaskMeta, pattern, startTime, endTime); + } + + @Override + public boolean isGeneratedByPipe() { + return false; + } + + @Override + public boolean mayEventTimeOverlappedWithTimeRange() { + return true; + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/extractor/IoTDBExtractor.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/extractor/IoTDBExtractor.java index 573d319eef56..77eeb3aa91e9 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/extractor/IoTDBExtractor.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/extractor/IoTDBExtractor.java @@ -54,7 +54,7 @@ public abstract class IoTDBExtractor implements PipeExtractor { protected final AtomicBoolean hasBeenStarted = new AtomicBoolean(false); @Override - public void validate(PipeParameterValidator validator) throws Exception { + public void validate(final PipeParameterValidator validator) throws Exception { validator .validate( args -> optionsAreAllLegal((String) args), @@ -88,7 +88,8 @@ public void validate(PipeParameterValidator validator) throws Exception { } @Override - public void customize(PipeParameters parameters, PipeExtractorRuntimeConfiguration configuration) + public void customize( + final PipeParameters parameters, final PipeExtractorRuntimeConfiguration configuration) throws Exception { final PipeTaskExtractorRuntimeEnvironment environment = ((PipeTaskExtractorRuntimeEnvironment) configuration.getRuntimeEnvironment()); @@ -113,4 +114,22 @@ public void start() throws Exception { } hasBeenStarted.set(true); } + + //////////////////////////// APIs provided for metric framework //////////////////////////// + + public String getTaskID() { + return taskID; + } + + public String getPipeName() { + return pipeName; + } + + public int getRegionId() { + return regionId; + } + + public long getCreationTime() { + return creationTime; + } } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/extractor/IoTDBNonDataRegionExtractor.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/extractor/IoTDBNonDataRegionExtractor.java index 21670b29466d..e41957458950 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/extractor/IoTDBNonDataRegionExtractor.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/extractor/IoTDBNonDataRegionExtractor.java @@ -26,6 +26,7 @@ import org.apache.iotdb.commons.pipe.datastructure.queue.listening.AbstractPipeListeningQueue; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.commons.pipe.event.PipeSnapshotEvent; +import org.apache.iotdb.commons.pipe.event.ProgressReportEvent; import org.apache.iotdb.pipe.api.event.Event; import org.apache.iotdb.pipe.api.exception.PipeException; @@ -39,6 +40,9 @@ public abstract class IoTDBNonDataRegionExtractor extends IoTDBExtractor { private List historicalEvents = new LinkedList<>(); + // A fixed size initialized only when the historicalEvents are first + // filled. Used only for metric framework. + private int historicalEventsCount = 0; private ConcurrentIterableLinkedQueue.DynamicIterator iterator; @@ -96,6 +100,7 @@ private long findSnapshot() { ? queueTailIndex2Snapshots.getLeft() : Long.MIN_VALUE; historicalEvents = new LinkedList<>(queueTailIndex2Snapshots.getRight()); + historicalEventsCount = historicalEvents.size(); return nextIndex; } @@ -138,14 +143,19 @@ public EnrichedEvent supply() throws Exception { } // Realtime - EnrichedEvent realtimeEvent; - do { - realtimeEvent = (EnrichedEvent) iterator.next(getMaxBlockingTimeMs()); - if (Objects.isNull(realtimeEvent)) { - return null; - } - } while (!isTypeListened(realtimeEvent) - || (!isForwardingPipeRequests && realtimeEvent.isGeneratedByPipe())); + EnrichedEvent realtimeEvent = (EnrichedEvent) iterator.next(getMaxBlockingTimeMs()); + if (Objects.isNull(realtimeEvent)) { + return null; + } + + if (!isTypeListened(realtimeEvent) + || (!isForwardingPipeRequests && realtimeEvent.isGeneratedByPipe())) { + final ProgressReportEvent event = + new ProgressReportEvent(pipeName, pipeTaskMeta, null, Long.MIN_VALUE, Long.MAX_VALUE); + event.bindProgressIndex(new MetaProgressIndex(iterator.getNextIndex() - 1)); + event.increaseReferenceCount(IoTDBNonDataRegionExtractor.class.getName()); + return event; + } realtimeEvent = realtimeEvent.shallowCopySelfAndBindPipeTaskMetaForProgressReport( @@ -157,22 +167,23 @@ public EnrichedEvent supply() throws Exception { protected abstract long getMaxBlockingTimeMs(); - protected abstract boolean isTypeListened(Event event); + protected abstract boolean isTypeListened(final Event event); - protected abstract void confineHistoricalEventTransferTypes(PipeSnapshotEvent event); + protected abstract void confineHistoricalEventTransferTypes(final PipeSnapshotEvent event); @Override public void close() throws Exception { - if (hasBeenClosed.get()) { - return; - } - hasBeenClosed.set(true); - - if (!hasBeenStarted.get()) { - return; - } - getListeningQueue().returnIterator(iterator); historicalEvents.clear(); } + + //////////////////////////// APIs provided for metric framework //////////////////////////// + + public long getUnTransferredEventCount() { + return !(pipeTaskMeta.getProgressIndex() instanceof MinimumProgressIndex) + ? getListeningQueue().getTailIndex() + - ((MetaProgressIndex) pipeTaskMeta.getProgressIndex()).getIndex() + - 1 + : getListeningQueue().getSize() + historicalEventsCount; + } } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/metric/PipeEventCommitMetrics.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/metric/PipeEventCommitMetrics.java index e7ab3ef42483..1cf32c4a71ac 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/metric/PipeEventCommitMetrics.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/metric/PipeEventCommitMetrics.java @@ -46,19 +46,19 @@ public class PipeEventCommitMetrics implements IMetricSet { //////////////////////////// bindTo & unbindFrom (metric framework) //////////////////////////// @Override - public void bindTo(AbstractMetricService metricService) { + public void bindTo(final AbstractMetricService metricService) { this.metricService = metricService; - ImmutableSet committerKeys = ImmutableSet.copyOf(eventCommitterMap.keySet()); + final ImmutableSet committerKeys = ImmutableSet.copyOf(eventCommitterMap.keySet()); for (String committerKey : committerKeys) { createMetrics(committerKey); } } - private void createMetrics(String committerKey) { + private void createMetrics(final String committerKey) { createAutoGauge(committerKey); } - private void createAutoGauge(String committerKey) { + private void createAutoGauge(final String committerKey) { PipeEventCommitter eventCommitter = eventCommitterMap.get(committerKey); metricService.createAutoGauge( Metric.PIPE_EVENT_COMMIT_QUEUE_SIZE.toString(), @@ -68,12 +68,12 @@ private void createAutoGauge(String committerKey) { Tag.NAME.toString(), String.valueOf(eventCommitter.getPipeName()), Tag.REGION.toString(), - String.valueOf(eventCommitter.getDataRegionId())); + String.valueOf(eventCommitter.getRegionId())); } @Override - public void unbindFrom(AbstractMetricService metricService) { - ImmutableSet committerKeys = ImmutableSet.copyOf(eventCommitterMap.keySet()); + public void unbindFrom(final AbstractMetricService metricService) { + final ImmutableSet committerKeys = ImmutableSet.copyOf(eventCommitterMap.keySet()); for (String committerKey : committerKeys) { deregister(committerKey); } @@ -82,24 +82,24 @@ public void unbindFrom(AbstractMetricService metricService) { } } - private void removeMetrics(String committerKey) { + private void removeMetrics(final String committerKey) { removeAutoGauge(committerKey); } - private void removeAutoGauge(String committerKey) { - PipeEventCommitter eventCommitter = eventCommitterMap.get(committerKey); + private void removeAutoGauge(final String committerKey) { + final PipeEventCommitter eventCommitter = eventCommitterMap.get(committerKey); metricService.remove( MetricType.AUTO_GAUGE, Metric.PIPE_EVENT_COMMIT_QUEUE_SIZE.toString(), Tag.NAME.toString(), String.valueOf(eventCommitter.getPipeName()), Tag.REGION.toString(), - String.valueOf(eventCommitter.getDataRegionId())); + String.valueOf(eventCommitter.getRegionId())); } //////////////////////////// register & deregister (pipe integration) //////////////////////////// - public void register(PipeEventCommitter eventCommitter, String committerKey) { + public void register(final PipeEventCommitter eventCommitter, final String committerKey) { if (Objects.isNull(eventCommitter)) { return; } @@ -110,14 +110,14 @@ public void register(PipeEventCommitter eventCommitter, String committerKey) { } } - public void deregister(String committerKey) { + public void deregister(final String committerKey) { if (!eventCommitterMap.containsKey(committerKey)) { LOGGER.warn( "Failed to deregister pipe event commit metrics, PipeEventCommitter({}) does not exist", committerKey); return; } - if (Objects.nonNull(committerKey)) { + if (Objects.nonNull(metricService) && Objects.nonNull(committerKey)) { removeMetrics(committerKey); } eventCommitterMap.remove(committerKey); @@ -130,7 +130,7 @@ private static class PipeEventCommitMetricsHolder { private static final PipeEventCommitMetrics INSTANCE = new PipeEventCommitMetrics(); private PipeEventCommitMetricsHolder() { - // empty constructor + // Empty constructor } } @@ -139,6 +139,6 @@ public static PipeEventCommitMetrics getInstance() { } private PipeEventCommitMetrics() { - // empty constructor + // Empty constructor } } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/progress/PipeEventCommitManager.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/progress/PipeEventCommitManager.java index bb985248854f..9f9369ae56b6 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/progress/PipeEventCommitManager.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/progress/PipeEventCommitManager.java @@ -19,6 +19,7 @@ package org.apache.iotdb.commons.pipe.progress; +import org.apache.iotdb.commons.pipe.config.plugin.env.PipeTaskRuntimeEnvironment; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.commons.pipe.metric.PipeEventCommitMetrics; @@ -26,16 +27,24 @@ import org.slf4j.LoggerFactory; import java.util.Map; +import java.util.Objects; import java.util.concurrent.ConcurrentHashMap; +import java.util.function.Consumer; public class PipeEventCommitManager { private static final Logger LOGGER = LoggerFactory.getLogger(PipeEventCommitManager.class); - // key: pipeName_dataRegionId + // key: pipeName_regionId private final Map eventCommitterMap = new ConcurrentHashMap<>(); - public void register(String pipeName, long creationTime, int regionId, String pipePluginName) { + private Consumer commitRateMarker; + + public void register( + final String pipeName, + final long creationTime, + final int regionId, + final String pipePluginName) { if (pipeName == null || pipePluginName == null) { return; } @@ -43,20 +52,21 @@ public void register(String pipeName, long creationTime, int regionId, String pi final String committerKey = generateCommitterKey(pipeName, creationTime, regionId); if (eventCommitterMap.containsKey(committerKey)) { LOGGER.warn( - "Pipe with same name is already registered on this data region, overwriting: {}", + "Pipe with same name is already registered on this region, overwriting: {}", committerKey); } - PipeEventCommitter eventCommitter = new PipeEventCommitter(pipeName, creationTime, regionId); + final PipeEventCommitter eventCommitter = + new PipeEventCommitter(pipeName, creationTime, regionId); eventCommitterMap.put(committerKey, eventCommitter); PipeEventCommitMetrics.getInstance().register(eventCommitter, committerKey); - LOGGER.info("Pipe committer registered for pipe on data region: {}", committerKey); + LOGGER.info("Pipe committer registered for pipe on region: {}", committerKey); } - public void deregister(String pipeName, long creationTime, int regionId) { + public void deregister(final String pipeName, final long creationTime, final int regionId) { final String committerKey = generateCommitterKey(pipeName, creationTime, regionId); eventCommitterMap.remove(committerKey); PipeEventCommitMetrics.getInstance().deregister(committerKey); - LOGGER.info("Pipe committer deregistered for pipe on data region: {}", committerKey); + LOGGER.info("Pipe committer deregistered for pipe on region: {}", committerKey); } /** @@ -64,7 +74,7 @@ public void deregister(String pipeName, long creationTime, int regionId) { * calling this. */ public void enrichWithCommitterKeyAndCommitId( - EnrichedEvent event, long creationTime, int regionId) { + final EnrichedEvent event, final long creationTime, final int regionId) { if (event == null || event.getPipeName() == null || !event.needToCommit()) { return; } @@ -77,15 +87,15 @@ public void enrichWithCommitterKeyAndCommitId( event.setCommitterKeyAndCommitId(committerKey, committer.generateCommitId()); } - public void commit(EnrichedEvent event, String committerKey) { - if (event == null + public void commit(final EnrichedEvent event, final String committerKey) { + if (committerKey == null + || event == null || !event.needToCommit() - || event.getCommitId() <= EnrichedEvent.NO_COMMIT_ID - || committerKey == null) { + || event.getCommitId() <= EnrichedEvent.NO_COMMIT_ID) { return; } - final PipeEventCommitter committer = eventCommitterMap.get(committerKey); + if (committer == null) { if (LOGGER.isDebugEnabled()) { LOGGER.debug( @@ -96,13 +106,33 @@ public void commit(EnrichedEvent event, String committerKey) { } return; } + committer.commit(event); + if (Objects.nonNull(commitRateMarker)) { + try { + commitRateMarker.accept( + new PipeTaskRuntimeEnvironment( + committer.getPipeName(), committer.getCreationTime(), committer.getRegionId())); + } catch (Exception e) { + if (LOGGER.isDebugEnabled()) { + LOGGER.debug( + "Failed to mark commit rate for pipe: {}, stack trace: {}", + committerKey, + Thread.currentThread().getStackTrace()); + } + } + } } - private static String generateCommitterKey(String pipeName, long creationTime, int regionId) { + private static String generateCommitterKey( + final String pipeName, final long creationTime, final int regionId) { return String.format("%s_%s_%s", pipeName, regionId, creationTime); } + public void setCommitRateMarker(final Consumer commitRateMarker) { + this.commitRateMarker = commitRateMarker; + } + private PipeEventCommitManager() { // Do nothing but make it private. } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/progress/PipeEventCommitter.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/progress/PipeEventCommitter.java index fb0830dd6810..2dba84a38775 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/progress/PipeEventCommitter.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/progress/PipeEventCommitter.java @@ -37,7 +37,7 @@ public class PipeEventCommitter { private final String pipeName; private final long creationTime; - private final int dataRegionId; + private final int regionId; private final AtomicLong commitIdGenerator = new AtomicLong(0); private final AtomicLong lastCommitId = new AtomicLong(0); @@ -49,25 +49,25 @@ public class PipeEventCommitter { event -> Objects.requireNonNull(event, "committable event cannot be null").getCommitId())); - PipeEventCommitter(String pipeName, long creationTime, int dataRegionId) { + PipeEventCommitter(final String pipeName, final long creationTime, final int regionId) { // make it package-private this.pipeName = pipeName; this.creationTime = creationTime; - this.dataRegionId = dataRegionId; + this.regionId = regionId; } public synchronized long generateCommitId() { return commitIdGenerator.incrementAndGet(); } - public synchronized void commit(EnrichedEvent event) { + public synchronized void commit(final EnrichedEvent event) { commitQueue.offer(event); LOGGER.info( "COMMIT QUEUE OFFER: pipe name {}, creation time {}, region id {}, event commit id {}, last commit id {}, commit queue size {}", pipeName, creationTime, - dataRegionId, + regionId, event.getCommitId(), lastCommitId.get(), commitQueue.size()); @@ -98,7 +98,7 @@ public synchronized void commit(EnrichedEvent event) { "COMMIT QUEUE POLL: pipe name {}, creation time {}, region id {}, last commit id {}, commit queue size after commit {}", pipeName, creationTime, - dataRegionId, + regionId, lastCommitId.get(), commitQueue.size()); } @@ -115,8 +115,8 @@ public long getCreationTime() { return creationTime; } - public int getDataRegionId() { - return dataRegionId; + public int getRegionId() { + return regionId; } public long commitQueueSize() { diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeTaskMeta.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeTaskMeta.java index 2a22ab9a7a69..23b4c645beb3 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeTaskMeta.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeTaskMeta.java @@ -172,7 +172,7 @@ public int hashCode() { @Override public String toString() { - return "PipeTask{" + return "PipeTaskMeta{" + "progressIndex='" + progressIndex.get() + "', leaderNodeId=" diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/subtask/PipeReportableSubtask.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/subtask/PipeReportableSubtask.java index 63d807ba347e..327152dd0c37 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/subtask/PipeReportableSubtask.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/subtask/PipeReportableSubtask.java @@ -102,7 +102,7 @@ private void onEnrichedEventFailure(Throwable throwable) { taskID, creationTime, this.getClass().getSimpleName(), - retryCount.get(), + retryCount.get() - 1, throwable.getMessage(), getRootCause(throwable)); LOGGER.warn(errorMessage, throwable); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/service/metric/enums/Metric.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/service/metric/enums/Metric.java index 4a60286d4c10..e19f7dcacccc 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/service/metric/enums/Metric.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/service/metric/enums/Metric.java @@ -154,6 +154,15 @@ public enum Metric { PIPE_EVENT_COMMIT_QUEUE_SIZE("pipe_event_commit_queue_size"), PIPE_PROCEDURE("pipe_procedure"), PIPE_TASK_STATUS("pipe_task_status"), + PIPE_SCHEMA_LINKED_QUEUE_SIZE("pipe_schema_linked_queue_size"), + UNTRANSFERRED_SCHEMA_COUNT("untransferred_schema_count"), + PIPE_CONNECTOR_SCHEMA_TRANSFER("pipe_connector_schema_transfer"), + PIPE_DATANODE_REMAINING_EVENT_COUNT("pipe_datanode_remaining_event_count"), + PIPE_DATANODE_REMAINING_TIME("pipe_datanode_remaining_time"), + PIPE_CONFIG_LINKED_QUEUE_SIZE("pipe_config_linked_queue_size"), + UNTRANSFERRED_CONFIG_COUNT("untransferred_config_count"), + PIPE_CONNECTOR_CONFIG_TRANSFER("pipe_connector_config_transfer"), + PIPE_CONFIGNODE_REMAINING_TIME("pipe_confignode_remaining_time"), // load related LOAD_MEM("load_mem"); diff --git a/pom.xml b/pom.xml index ad35b59689f6..f3054f579aa4 100644 --- a/pom.xml +++ b/pom.xml @@ -70,7 +70,7 @@ false 3.4.4 1.21.1 - 4.2.19 + 4.2.19 11.1.0 true @@ -1053,12 +1053,12 @@ io.dropwizard.metrics metrics-core - ${dropqizard.metrics.version} + ${dropwizard.metrics.version} io.dropwizard.metrics metrics-jmx - ${dropqizard.metrics.version} + ${dropwizard.metrics.version} org.eclipse.jetty From 77d61207f6cd1816a042b961b6bca6ac83b58131 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Tue, 21 May 2024 16:26:19 +0800 Subject: [PATCH 23/42] Pipe IT: Fix CN connection timeout in IT is not functional (#12563) --- .../iotdb/it/env/cluster/config/MppCommonConfig.java | 6 ++++++ .../it/env/cluster/config/MppConfigNodeConfig.java | 6 ------ .../it/env/cluster/config/MppSharedCommonConfig.java | 7 +++++++ .../it/env/remote/config/RemoteCommonConfig.java | 5 +++++ .../it/env/remote/config/RemoteConfigNodeConfig.java | 5 ----- .../org/apache/iotdb/itbase/env/CommonConfig.java | 2 ++ .../apache/iotdb/itbase/env/ConfigNodeConfig.java | 2 -- .../pipe/it/autocreate/AbstractPipeDualAutoIT.java | 4 ++-- .../iotdb/pipe/it/autocreate/IoTDBPipeClusterIT.java | 4 ++-- .../autocreate/IoTDBPipeConnectorCompressionIT.java | 4 ++-- .../pipe/it/autocreate/IoTDBPipeIdempotentIT.java | 4 ++-- .../pipe/it/autocreate/IoTDBPipeProcessorIT.java | 4 ++-- .../pipe/it/autocreate/IoTDBPipeProtocolIT.java | 12 ++++++------ .../pipe/it/manual/AbstractPipeDualManualIT.java | 4 ++-- .../pipe/it/manual/IoTDBPipeMetaHistoricalIT.java | 4 ++-- .../pipe/it/manual/IoTDBPipeMetaLeaderChangeIT.java | 4 ++-- .../iotdb/pipe/it/single/AbstractPipeSingleIT.java | 2 +- .../it/dual/AbstractSubscriptionDualIT.java | 4 ++-- 18 files changed, 45 insertions(+), 38 deletions(-) diff --git a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppCommonConfig.java b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppCommonConfig.java index 595bc4ddaa6e..95fed66d08b2 100644 --- a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppCommonConfig.java +++ b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppCommonConfig.java @@ -432,6 +432,12 @@ public CommonConfig setTagAttributeTotalSize(int tagAttributeTotalSize) { return this; } + @Override + public CommonConfig setCnConnectionTimeoutMs(int connectionTimeoutMs) { + setProperty("cn_connection_timeout_ms", String.valueOf(connectionTimeoutMs)); + return this; + } + // For part of the log directory public String getClusterConfigStr() { return fromConsensusFullNameToAbbr(properties.getProperty(CONFIG_NODE_CONSENSUS_PROTOCOL_CLASS)) diff --git a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppConfigNodeConfig.java b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppConfigNodeConfig.java index 7fc6494a212b..62ccbb0aa4ff 100644 --- a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppConfigNodeConfig.java +++ b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppConfigNodeConfig.java @@ -55,10 +55,4 @@ public ConfigNodeConfig setMetricReporterType(List metricReporterTypes) properties.setProperty("cn_metric_reporter_list", String.join(",", metricReporterTypes)); return this; } - - @Override - public ConfigNodeConfig setConnectionTimeoutMs(long connectionTimeoutMs) { - properties.setProperty("cn_connection_timeout_ms", String.valueOf(connectionTimeoutMs)); - return this; - } } diff --git a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppSharedCommonConfig.java b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppSharedCommonConfig.java index 96ab48ab58fb..1851590dd411 100644 --- a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppSharedCommonConfig.java +++ b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppSharedCommonConfig.java @@ -438,4 +438,11 @@ public CommonConfig setTagAttributeTotalSize(int tagAttributeTotalSize) { cnConfig.setTagAttributeTotalSize(tagAttributeTotalSize); return this; } + + @Override + public CommonConfig setCnConnectionTimeoutMs(int connectionTimeoutMs) { + dnConfig.setCnConnectionTimeoutMs(connectionTimeoutMs); + cnConfig.setCnConnectionTimeoutMs(connectionTimeoutMs); + return this; + } } diff --git a/integration-test/src/main/java/org/apache/iotdb/it/env/remote/config/RemoteCommonConfig.java b/integration-test/src/main/java/org/apache/iotdb/it/env/remote/config/RemoteCommonConfig.java index 2a97a53f5893..0ac82fc87137 100644 --- a/integration-test/src/main/java/org/apache/iotdb/it/env/remote/config/RemoteCommonConfig.java +++ b/integration-test/src/main/java/org/apache/iotdb/it/env/remote/config/RemoteCommonConfig.java @@ -308,4 +308,9 @@ public CommonConfig setWalMode(String walMode) { public CommonConfig setTagAttributeTotalSize(int tagAttributeTotalSize) { return this; } + + @Override + public CommonConfig setCnConnectionTimeoutMs(int connectionTimeoutMs) { + return this; + } } diff --git a/integration-test/src/main/java/org/apache/iotdb/it/env/remote/config/RemoteConfigNodeConfig.java b/integration-test/src/main/java/org/apache/iotdb/it/env/remote/config/RemoteConfigNodeConfig.java index b9349b9023ae..33a6bc48afd7 100644 --- a/integration-test/src/main/java/org/apache/iotdb/it/env/remote/config/RemoteConfigNodeConfig.java +++ b/integration-test/src/main/java/org/apache/iotdb/it/env/remote/config/RemoteConfigNodeConfig.java @@ -28,9 +28,4 @@ public class RemoteConfigNodeConfig implements ConfigNodeConfig { public ConfigNodeConfig setMetricReporterType(List metricReporterTypes) { return this; } - - @Override - public ConfigNodeConfig setConnectionTimeoutMs(long connectionTimeoutMs) { - return this; - } } diff --git a/integration-test/src/main/java/org/apache/iotdb/itbase/env/CommonConfig.java b/integration-test/src/main/java/org/apache/iotdb/itbase/env/CommonConfig.java index 5bd7805aa491..af47c8254e53 100644 --- a/integration-test/src/main/java/org/apache/iotdb/itbase/env/CommonConfig.java +++ b/integration-test/src/main/java/org/apache/iotdb/itbase/env/CommonConfig.java @@ -137,4 +137,6 @@ CommonConfig setEnableAutoLeaderBalanceForIoTConsensus( CommonConfig setWalMode(String walMode); CommonConfig setTagAttributeTotalSize(int tagAttributeTotalSize); + + CommonConfig setCnConnectionTimeoutMs(int connectionTimeoutMs); } diff --git a/integration-test/src/main/java/org/apache/iotdb/itbase/env/ConfigNodeConfig.java b/integration-test/src/main/java/org/apache/iotdb/itbase/env/ConfigNodeConfig.java index 31a955417d27..bf7179ef7028 100644 --- a/integration-test/src/main/java/org/apache/iotdb/itbase/env/ConfigNodeConfig.java +++ b/integration-test/src/main/java/org/apache/iotdb/itbase/env/ConfigNodeConfig.java @@ -24,6 +24,4 @@ /** This interface is used to handle properties in iotdb-confignode.properties. */ public interface ConfigNodeConfig { ConfigNodeConfig setMetricReporterType(List metricReporterTypes); - - ConfigNodeConfig setConnectionTimeoutMs(long connectionTimeoutMs); } diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/AbstractPipeDualAutoIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/AbstractPipeDualAutoIT.java index d733d5523a6d..a0a27791a318 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/AbstractPipeDualAutoIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/AbstractPipeDualAutoIT.java @@ -52,8 +52,8 @@ public void setUp() { .setSchemaRegionConsensusProtocolClass(ConsensusFactory.RATIS_CONSENSUS); // 10 min, assert that the operations will not time out - senderEnv.getConfig().getConfigNodeConfig().setConnectionTimeoutMs(600000); - receiverEnv.getConfig().getConfigNodeConfig().setConnectionTimeoutMs(600000); + senderEnv.getConfig().getCommonConfig().setCnConnectionTimeoutMs(600000); + receiverEnv.getConfig().getCommonConfig().setCnConnectionTimeoutMs(600000); senderEnv.initClusterEnvironment(); receiverEnv.initClusterEnvironment(); diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeClusterIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeClusterIT.java index c7e0289adc51..e90c7ecf8ade 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeClusterIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeClusterIT.java @@ -85,8 +85,8 @@ public void setUp() { .setDataRegionConsensusProtocolClass(ConsensusFactory.IOT_CONSENSUS); // 10 min, assert that the operations will not time out - senderEnv.getConfig().getConfigNodeConfig().setConnectionTimeoutMs(600000); - receiverEnv.getConfig().getConfigNodeConfig().setConnectionTimeoutMs(600000); + senderEnv.getConfig().getCommonConfig().setCnConnectionTimeoutMs(600000); + receiverEnv.getConfig().getCommonConfig().setCnConnectionTimeoutMs(600000); senderEnv.initClusterEnvironment(3, 3, 180); receiverEnv.initClusterEnvironment(3, 3, 180); diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeConnectorCompressionIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeConnectorCompressionIT.java index e5f76a00eb00..be4aa458d9fb 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeConnectorCompressionIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeConnectorCompressionIT.java @@ -68,8 +68,8 @@ public void setUp() { .setSchemaRegionConsensusProtocolClass(ConsensusFactory.RATIS_CONSENSUS); // 10 min, assert that the operations will not time out - senderEnv.getConfig().getConfigNodeConfig().setConnectionTimeoutMs(600000); - receiverEnv.getConfig().getConfigNodeConfig().setConnectionTimeoutMs(600000); + senderEnv.getConfig().getCommonConfig().setCnConnectionTimeoutMs(600000); + receiverEnv.getConfig().getCommonConfig().setCnConnectionTimeoutMs(600000); senderEnv.initClusterEnvironment(); receiverEnv.initClusterEnvironment(); diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeIdempotentIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeIdempotentIT.java index 0f8d1f824483..2cd7e287796e 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeIdempotentIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeIdempotentIT.java @@ -74,8 +74,8 @@ public void setUp() { .setSchemaRegionConsensusProtocolClass(ConsensusFactory.RATIS_CONSENSUS); // 10 min, assert that the operations will not time out - senderEnv.getConfig().getConfigNodeConfig().setConnectionTimeoutMs(600000); - receiverEnv.getConfig().getConfigNodeConfig().setConnectionTimeoutMs(600000); + senderEnv.getConfig().getCommonConfig().setCnConnectionTimeoutMs(600000); + receiverEnv.getConfig().getCommonConfig().setCnConnectionTimeoutMs(600000); senderEnv.initClusterEnvironment(); receiverEnv.initClusterEnvironment(); diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeProcessorIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeProcessorIT.java index 81d4dbefcbf8..4556416f5fbe 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeProcessorIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeProcessorIT.java @@ -67,8 +67,8 @@ public void setUp() { .setSchemaRegionConsensusProtocolClass(ConsensusFactory.RATIS_CONSENSUS); // 10 min, assert that the operations will not time out - senderEnv.getConfig().getConfigNodeConfig().setConnectionTimeoutMs(600000); - receiverEnv.getConfig().getConfigNodeConfig().setConnectionTimeoutMs(600000); + senderEnv.getConfig().getCommonConfig().setCnConnectionTimeoutMs(600000); + receiverEnv.getConfig().getCommonConfig().setCnConnectionTimeoutMs(600000); senderEnv.initClusterEnvironment(); receiverEnv.initClusterEnvironment(); diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeProtocolIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeProtocolIT.java index e8e11e6fec91..373e9d4dbea7 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeProtocolIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeProtocolIT.java @@ -86,8 +86,8 @@ private void innerSetUp( .setDataReplicationFactor(dataRegionReplicationFactor); // 10 min, assert that the operations will not time out - senderEnv.getConfig().getConfigNodeConfig().setConnectionTimeoutMs(600000); - receiverEnv.getConfig().getConfigNodeConfig().setConnectionTimeoutMs(600000); + senderEnv.getConfig().getCommonConfig().setCnConnectionTimeoutMs(600000); + receiverEnv.getConfig().getCommonConfig().setCnConnectionTimeoutMs(600000); senderEnv.initClusterEnvironment(configNodesNum, dataNodesNum); receiverEnv.initClusterEnvironment(configNodesNum, dataNodesNum); @@ -180,8 +180,8 @@ public void testPipeOnBothSenderAndReceiver() throws Exception { .setDataReplicationFactor(1); // 10 min, assert that the operations will not time out - senderEnv.getConfig().getConfigNodeConfig().setConnectionTimeoutMs(600000); - receiverEnv.getConfig().getConfigNodeConfig().setConnectionTimeoutMs(600000); + senderEnv.getConfig().getCommonConfig().setCnConnectionTimeoutMs(600000); + receiverEnv.getConfig().getCommonConfig().setCnConnectionTimeoutMs(600000); senderEnv.initClusterEnvironment(3, 3); receiverEnv.initClusterEnvironment(1, 1); @@ -373,8 +373,8 @@ private void doTestUseNodeUrls(String connectorName) throws Exception { .setDataReplicationFactor(2); // 10 min, assert that the operations will not time out - senderEnv.getConfig().getConfigNodeConfig().setConnectionTimeoutMs(600000); - receiverEnv.getConfig().getConfigNodeConfig().setConnectionTimeoutMs(600000); + senderEnv.getConfig().getCommonConfig().setCnConnectionTimeoutMs(600000); + receiverEnv.getConfig().getCommonConfig().setCnConnectionTimeoutMs(600000); senderEnv.initClusterEnvironment(1, 1); receiverEnv.initClusterEnvironment(1, 3); diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/AbstractPipeDualManualIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/AbstractPipeDualManualIT.java index 61f42482dc43..ae81f1ffb20a 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/AbstractPipeDualManualIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/AbstractPipeDualManualIT.java @@ -52,8 +52,8 @@ public void setUp() { .setSchemaRegionConsensusProtocolClass(ConsensusFactory.RATIS_CONSENSUS); // 10 min, assert that the operations will not time out - senderEnv.getConfig().getConfigNodeConfig().setConnectionTimeoutMs(600000); - receiverEnv.getConfig().getConfigNodeConfig().setConnectionTimeoutMs(600000); + senderEnv.getConfig().getCommonConfig().setCnConnectionTimeoutMs(600000); + receiverEnv.getConfig().getCommonConfig().setCnConnectionTimeoutMs(600000); senderEnv.initClusterEnvironment(); receiverEnv.initClusterEnvironment(); diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/IoTDBPipeMetaHistoricalIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/IoTDBPipeMetaHistoricalIT.java index c917acbde1ab..89662c19f633 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/IoTDBPipeMetaHistoricalIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/IoTDBPipeMetaHistoricalIT.java @@ -74,8 +74,8 @@ public void setUp() { .setDataReplicationFactor(2); // 10 min, assert that the operations will not time out - senderEnv.getConfig().getConfigNodeConfig().setConnectionTimeoutMs(600000); - receiverEnv.getConfig().getConfigNodeConfig().setConnectionTimeoutMs(600000); + senderEnv.getConfig().getCommonConfig().setCnConnectionTimeoutMs(600000); + receiverEnv.getConfig().getCommonConfig().setCnConnectionTimeoutMs(600000); senderEnv.initClusterEnvironment(); receiverEnv.initClusterEnvironment(3, 3); diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/IoTDBPipeMetaLeaderChangeIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/IoTDBPipeMetaLeaderChangeIT.java index 5928e6e33c74..fb70e8e37838 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/IoTDBPipeMetaLeaderChangeIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/IoTDBPipeMetaLeaderChangeIT.java @@ -59,8 +59,8 @@ public void setUp() { .setSchemaReplicationFactor(3); // 10 min, assert that the operations will not time out - senderEnv.getConfig().getConfigNodeConfig().setConnectionTimeoutMs(600000); - receiverEnv.getConfig().getConfigNodeConfig().setConnectionTimeoutMs(600000); + senderEnv.getConfig().getCommonConfig().setCnConnectionTimeoutMs(600000); + receiverEnv.getConfig().getCommonConfig().setCnConnectionTimeoutMs(600000); senderEnv.initClusterEnvironment(3, 3, 180); receiverEnv.initClusterEnvironment(); diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/single/AbstractPipeSingleIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/single/AbstractPipeSingleIT.java index c03c0b499b5d..70bb4df4a93e 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/single/AbstractPipeSingleIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/single/AbstractPipeSingleIT.java @@ -35,7 +35,7 @@ public void setUp() { env = MultiEnvFactory.getEnv(0); env.getConfig().getCommonConfig().setAutoCreateSchemaEnabled(true); // 10 min, assert that the operations will not time out - env.getConfig().getConfigNodeConfig().setConnectionTimeoutMs(600000); + env.getConfig().getCommonConfig().setCnConnectionTimeoutMs(600000); env.initClusterEnvironment(); } diff --git a/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/AbstractSubscriptionDualIT.java b/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/AbstractSubscriptionDualIT.java index ecad24e2b4bd..00e393bdb50d 100644 --- a/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/AbstractSubscriptionDualIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/AbstractSubscriptionDualIT.java @@ -41,8 +41,8 @@ public void setUp() { receiverEnv.getConfig().getCommonConfig().setPipeAirGapReceiverEnabled(true); // 10 min, assert that the operations will not time out - senderEnv.getConfig().getConfigNodeConfig().setConnectionTimeoutMs(600000); - receiverEnv.getConfig().getConfigNodeConfig().setConnectionTimeoutMs(600000); + senderEnv.getConfig().getCommonConfig().setCnConnectionTimeoutMs(600000); + receiverEnv.getConfig().getCommonConfig().setCnConnectionTimeoutMs(600000); senderEnv.initClusterEnvironment(); receiverEnv.initClusterEnvironment(); From 3b0dfcd17365dac3fb244bf4e10a9664d204636a Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Tue, 21 May 2024 17:22:38 +0800 Subject: [PATCH 24/42] Pipe: Enable realtime first strategy in data transfer (#12559) Co-authored-by: Steve Yurong Su --- .../it/autocreate/IoTDBPipeDataSinkIT.java | 17 ++- .../common/heartbeat/PipeHeartbeatEvent.java | 34 +---- ...PipeRealtimeDataRegionHybridExtractor.java | 9 +- ...eDataNodeRemainingEventAndTimeMetrics.java | 12 -- ...DataNodeRemainingEventAndTimeOperator.java | 17 --- .../db/pipe/metric/PipeProcessorMetrics.java | 39 ----- .../task/connection/PipeEventCollector.java | 96 +------------ .../task/stage/PipeTaskConnectorStage.java | 4 +- .../task/stage/PipeTaskProcessorStage.java | 4 +- .../connector/PipeConnectorSubtask.java | 6 +- .../PipeConnectorSubtaskLifeCycle.java | 8 +- .../PipeConnectorSubtaskManager.java | 20 ++- .../PipeRealtimePriorityBlockingQueue.java | 136 ++++++++++++++++++ .../processor/PipeProcessorSubtask.java | 31 +--- .../broker/SubscriptionBroker.java | 4 +- .../broker/SubscriptionPrefetchingQueue.java | 6 +- .../stage/SubscriptionTaskConnectorStage.java | 4 +- .../subtask/SubscriptionConnectorSubtask.java | 6 +- ...SubscriptionConnectorSubtaskLifeCycle.java | 4 +- .../SubscriptionConnectorSubtaskManager.java | 20 ++- .../PipeConnectorSubtaskExecutorTest.java | 4 +- .../iotdb/commons/conf/CommonConfig.java | 11 -- .../iotdb/commons/conf/CommonDescriptor.java | 7 - .../iotdb/commons/pipe/config/PipeConfig.java | 5 - .../constant/PipeConnectorConstant.java | 4 + 25 files changed, 216 insertions(+), 292 deletions(-) create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/subtask/connector/PipeRealtimePriorityBlockingQueue.java diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeDataSinkIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeDataSinkIT.java index 7e18ad9b8769..d3bfe710aca7 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeDataSinkIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeDataSinkIT.java @@ -36,13 +36,14 @@ 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 IoTDBPipeDataSinkIT extends AbstractPipeDualAutoIT { @Test - public void testThriftConnector() throws Exception { + public void testThriftConnectorWithRealtimeFirst() throws Exception { final DataNodeWrapper receiverDataNode = receiverEnv.getDataNodeWrapper(0); final String receiverIp = receiverDataNode.getIp(); @@ -50,6 +51,15 @@ public void testThriftConnector() throws Exception { try (final SyncConfigNodeIServiceClient client = (SyncConfigNodeIServiceClient) senderEnv.getLeaderConfigNodeConnection()) { + + // Do not fail if the failure has nothing to do with pipe + // Because the failures will randomly generate due to resource limitation + if (!TestUtils.tryExecuteNonQueriesWithRetry( + senderEnv, + Arrays.asList("insert into root.vehicle.d0(time, s1) values (0, 1)", "flush"))) { + return; + } + final Map extractorAttributes = new HashMap<>(); final Map processorAttributes = new HashMap<>(); final Map connectorAttributes = new HashMap<>(); @@ -60,6 +70,7 @@ public void testThriftConnector() throws Exception { connectorAttributes.put("connector.batch.enable", "false"); connectorAttributes.put("connector.ip", receiverIp); connectorAttributes.put("connector.port", Integer.toString(receiverPort)); + connectorAttributes.put("connector.realtime-first", "true"); final TSStatus status = client.createPipe( @@ -76,7 +87,7 @@ public void testThriftConnector() throws Exception { // Because the failures will randomly generate due to resource limitation if (!TestUtils.tryExecuteNonQueriesWithRetry( senderEnv, - Arrays.asList("insert into root.vehicle.d0(time, s1) values (0, 1)", "flush"))) { + Arrays.asList("insert into root.vehicle.d0(time, s1) values (1, 1)", "flush"))) { return; } @@ -84,7 +95,7 @@ public void testThriftConnector() throws Exception { receiverEnv, "select * from root.**", "Time,root.vehicle.d0.s1,", - Collections.singleton("0,1.0,")); + Collections.unmodifiableSet(new HashSet<>(Arrays.asList("0,1.0,", "1,1.0,")))); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/heartbeat/PipeHeartbeatEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/heartbeat/PipeHeartbeatEvent.java index c675ff5542cb..c3d0798c7e9b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/heartbeat/PipeHeartbeatEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/heartbeat/PipeHeartbeatEvent.java @@ -23,13 +23,11 @@ import org.apache.iotdb.commons.consensus.index.impl.MinimumProgressIndex; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.commons.pipe.pattern.PipePattern; -import org.apache.iotdb.commons.pipe.task.connection.BoundedBlockingPendingQueue; import org.apache.iotdb.commons.pipe.task.connection.UnboundedBlockingPendingQueue; import org.apache.iotdb.commons.pipe.task.meta.PipeTaskMeta; import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.PipeRealtimeDataRegionExtractor; import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.PipeRealtimeDataRegionHybridExtractor; import org.apache.iotdb.db.pipe.metric.PipeHeartbeatEventMetrics; -import org.apache.iotdb.db.pipe.task.connection.EnrichedDeque; import org.apache.iotdb.db.utils.DateTimeUtils; import org.apache.iotdb.pipe.api.event.Event; @@ -58,10 +56,6 @@ public class PipeHeartbeatEvent extends EnrichedEvent { private int extractorQueueTsFileSize; private int extractorQueueSize; - private int bufferQueueTabletSize; - private int bufferQueueTsFileSize; - private int bufferQueueSize; - private int connectorQueueTabletSize; private int connectorQueueTsFileSize; private int connectorQueueSize; @@ -195,20 +189,7 @@ public void recordExtractorQueueSize(UnboundedBlockingPendingQueue pendin } } - public void recordBufferQueueSize(EnrichedDeque bufferQueue) { - if (shouldPrintMessage) { - bufferQueueTabletSize = bufferQueue.getTabletInsertionEventCount(); - bufferQueueTsFileSize = bufferQueue.getTsFileInsertionEventCount(); - bufferQueueSize = bufferQueue.size(); - } - - if (extractor instanceof PipeRealtimeDataRegionHybridExtractor) { - ((PipeRealtimeDataRegionHybridExtractor) extractor) - .informProcessorEventCollectorQueueTsFileSize(bufferQueue.getTsFileInsertionEventCount()); - } - } - - public void recordConnectorQueueSize(BoundedBlockingPendingQueue pendingQueue) { + public void recordConnectorQueueSize(UnboundedBlockingPendingQueue pendingQueue) { if (shouldPrintMessage) { connectorQueueTabletSize = pendingQueue.getTabletInsertionEventCount(); connectorQueueTsFileSize = pendingQueue.getTsFileInsertionEventCount(); @@ -271,13 +252,6 @@ public String toString() { final String extractorQueueSizeMessage = timeAssigned != 0 ? Integer.toString(extractorQueueSize) : unknownMessage; - final String bufferQueueTabletSizeMessage = - timeProcessed != 0 ? Integer.toString(bufferQueueTabletSize) : unknownMessage; - final String bufferQueueTsFileSizeMessage = - timeProcessed != 0 ? Integer.toString(bufferQueueTsFileSize) : unknownMessage; - final String bufferQueueSizeMessage = - timeProcessed != 0 ? Integer.toString(bufferQueueSize) : unknownMessage; - final String connectorQueueTabletSizeMessage = timeProcessed != 0 ? Integer.toString(connectorQueueTabletSize) : unknownMessage; final String connectorQueueTsFileSizeMessage = @@ -308,12 +282,6 @@ public String toString() { + extractorQueueTsFileSizeMessage + ", extractorQueueSize=" + extractorQueueSizeMessage - + ", bufferQueueTabletSize=" - + bufferQueueTabletSizeMessage - + ", bufferQueueTsFileSize=" - + bufferQueueTsFileSizeMessage - + ", bufferQueueSize=" - + bufferQueueSizeMessage + ", connectorQueueTabletSize=" + connectorQueueTabletSizeMessage + ", connectorQueueTsFileSize=" diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java index 210c52581203..381b8d738585 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java @@ -47,7 +47,6 @@ public class PipeRealtimeDataRegionHybridExtractor extends PipeRealtimeDataRegio private static final Logger LOGGER = LoggerFactory.getLogger(PipeRealtimeDataRegionHybridExtractor.class); - private final AtomicInteger processorEventCollectorQueueTsFileSize = new AtomicInteger(0); private final AtomicInteger connectorInputPendingQueueTsFileSize = new AtomicInteger(0); @Override @@ -239,9 +238,7 @@ private boolean isHistoricalTsFileEventCountExceededLimit() { } private boolean isRealtimeTsFileEventCountExceededLimit() { - return pendingQueue.getTsFileInsertionEventCount() - + processorEventCollectorQueueTsFileSize.get() - + connectorInputPendingQueueTsFileSize.get() + return pendingQueue.getTsFileInsertionEventCount() + connectorInputPendingQueueTsFileSize.get() >= PipeConfig.getInstance().getPipeMaxAllowedPendingTsFileEpochPerDataRegion(); } @@ -250,10 +247,6 @@ private boolean mayTsFileLinkedCountReachDangerousThreshold() { >= PipeConfig.getInstance().getPipeMaxAllowedLinkedTsFileCount(); } - public void informProcessorEventCollectorQueueTsFileSize(final int queueSize) { - processorEventCollectorQueueTsFileSize.set(queueSize); - } - public void informConnectorInputPendingQueueTsFileSize(final int queueSize) { connectorInputPendingQueueTsFileSize.set(queueSize); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeDataNodeRemainingEventAndTimeMetrics.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeDataNodeRemainingEventAndTimeMetrics.java index 2bfe9106c01d..5173d64e04fd 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeDataNodeRemainingEventAndTimeMetrics.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeDataNodeRemainingEventAndTimeMetrics.java @@ -28,7 +28,6 @@ import org.apache.iotdb.db.pipe.extractor.dataregion.IoTDBDataRegionExtractor; import org.apache.iotdb.db.pipe.extractor.schemaregion.IoTDBSchemaRegionExtractor; import org.apache.iotdb.db.pipe.task.subtask.connector.PipeConnectorSubtask; -import org.apache.iotdb.db.pipe.task.subtask.processor.PipeProcessorSubtask; import org.apache.iotdb.db.schemaengine.SchemaEngine; import org.apache.iotdb.db.storageengine.StorageEngine; import org.apache.iotdb.metrics.AbstractMetricService; @@ -128,17 +127,6 @@ public void register(final IoTDBDataRegionExtractor extractor) { } } - public void register(final PipeProcessorSubtask processorSubtask) { - // The metric is global thus the regionId is omitted - final String pipeID = processorSubtask.getPipeName() + "_" + processorSubtask.getCreationTime(); - remainingEventAndTimeOperatorMap - .computeIfAbsent(pipeID, k -> new PipeDataNodeRemainingEventAndTimeOperator()) - .register(processorSubtask); - if (Objects.nonNull(metricService)) { - createMetrics(pipeID); - } - } - public void register( final PipeConnectorSubtask connectorSubtask, final String pipeName, final long creationTime) { // The metric is global thus the regionId is omitted diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeDataNodeRemainingEventAndTimeOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeDataNodeRemainingEventAndTimeOperator.java index ca311824a653..a4b2fd3e7b79 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeDataNodeRemainingEventAndTimeOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeDataNodeRemainingEventAndTimeOperator.java @@ -72,10 +72,6 @@ long getRemainingEvents() { .map(IoTDBDataRegionExtractor::getEventCount) .reduce(Integer::sum) .orElse(0) - + dataRegionProcessors.keySet().stream() - .map(PipeProcessorSubtask::getEventCount) - .reduce(Integer::sum) - .orElse(0) + dataRegionConnectors.keySet().stream() .map(connectorSubtask -> connectorSubtask.getEventCount(pipeName)) .reduce(Integer::sum) @@ -103,10 +99,6 @@ long getRemainingEvents() { .map(IoTDBDataRegionExtractor::getEventCount) .reduce(Integer::sum) .orElse(0) - + dataRegionProcessors.keySet().stream() - .map(PipeProcessorSubtask::getEventCount) - .reduce(Integer::sum) - .orElse(0) + dataRegionConnectors.keySet().stream() .map(connectorSubtask -> connectorSubtask.getEventCount(pipeName)) .reduce(Integer::sum) @@ -115,10 +107,6 @@ long getRemainingEvents() { .map(IoTDBDataRegionExtractor::getPipeHeartbeatEventCount) .reduce(Integer::sum) .orElse(0) - - dataRegionProcessors.keySet().stream() - .map(PipeProcessorSubtask::getPipeHeartbeatEventCount) - .reduce(Integer::sum) - .orElse(0) - dataRegionConnectors.keySet().stream() .map(PipeConnectorSubtask::getPipeHeartbeatEventCount) .reduce(Integer::sum) @@ -174,11 +162,6 @@ void register(final IoTDBDataRegionExtractor extractor) { dataRegionExtractors.put(extractor, extractor); } - void register(final PipeProcessorSubtask processorSubtask) { - setNameAndCreationTime(processorSubtask.getPipeName(), processorSubtask.getCreationTime()); - dataRegionProcessors.put(processorSubtask, processorSubtask); - } - void register( final PipeConnectorSubtask connectorSubtask, final String pipeName, final long creationTime) { setNameAndCreationTime(pipeName, creationTime); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeProcessorMetrics.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeProcessorMetrics.java index 05a267c6bae0..8cc2383c847f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeProcessorMetrics.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeProcessorMetrics.java @@ -64,48 +64,9 @@ public void bindTo(AbstractMetricService metricService) { } private void createMetrics(String taskID) { - createAutoGauge(taskID); createRate(taskID); } - private void createAutoGauge(String taskID) { - PipeProcessorSubtask processor = processorMap.get(taskID); - // pending event count - metricService.createAutoGauge( - Metric.BUFFERED_TABLET_COUNT.toString(), - MetricLevel.IMPORTANT, - processor, - PipeProcessorSubtask::getTabletInsertionEventCount, - Tag.NAME.toString(), - processor.getPipeName(), - Tag.REGION.toString(), - String.valueOf(processor.getRegionId()), - Tag.CREATION_TIME.toString(), - String.valueOf(processor.getCreationTime())); - metricService.createAutoGauge( - Metric.BUFFERED_TSFILE_COUNT.toString(), - MetricLevel.IMPORTANT, - processor, - PipeProcessorSubtask::getTsFileInsertionEventCount, - Tag.NAME.toString(), - processor.getPipeName(), - Tag.REGION.toString(), - String.valueOf(processor.getRegionId()), - Tag.CREATION_TIME.toString(), - String.valueOf(processor.getCreationTime())); - metricService.createAutoGauge( - Metric.BUFFERED_HEARTBEAT_COUNT.toString(), - MetricLevel.IMPORTANT, - processor, - PipeProcessorSubtask::getPipeHeartbeatEventCount, - Tag.NAME.toString(), - processor.getPipeName(), - Tag.REGION.toString(), - String.valueOf(processor.getRegionId()), - Tag.CREATION_TIME.toString(), - String.valueOf(processor.getCreationTime())); - } - private void createRate(String taskID) { PipeProcessorSubtask processor = processorMap.get(taskID); // process event rate diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/connection/PipeEventCollector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/connection/PipeEventCollector.java index 30a16f3c2a71..cdd735985719 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/connection/PipeEventCollector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/connection/PipeEventCollector.java @@ -22,7 +22,7 @@ import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.commons.pipe.event.ProgressReportEvent; import org.apache.iotdb.commons.pipe.progress.PipeEventCommitManager; -import org.apache.iotdb.commons.pipe.task.connection.BoundedBlockingPendingQueue; +import org.apache.iotdb.commons.pipe.task.connection.UnboundedBlockingPendingQueue; import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent; import org.apache.iotdb.db.pipe.event.common.tablet.PipeInsertNodeTabletInsertionEvent; import org.apache.iotdb.db.pipe.event.common.tablet.PipeRawTabletInsertionEvent; @@ -35,34 +35,27 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.LinkedList; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; -public class PipeEventCollector implements EventCollector, AutoCloseable { +public class PipeEventCollector implements EventCollector { private static final Logger LOGGER = LoggerFactory.getLogger(PipeEventCollector.class); - private final BoundedBlockingPendingQueue pendingQueue; - - private final EnrichedDeque bufferQueue; + private final UnboundedBlockingPendingQueue pendingQueue; private final long creationTime; private final int regionId; - private final AtomicBoolean isClosed = new AtomicBoolean(false); - private final AtomicInteger collectInvocationCount = new AtomicInteger(0); public PipeEventCollector( - final BoundedBlockingPendingQueue pendingQueue, + final UnboundedBlockingPendingQueue pendingQueue, final long creationTime, final int regionId) { this.pendingQueue = pendingQueue; this.creationTime = creationTime; this.regionId = regionId; - bufferQueue = new EnrichedDeque<>(new LinkedList<>()); } @Override @@ -128,7 +121,7 @@ private void parseAndCollectEvent(final PipeTsFileInsertionEvent sourceEvent) th } } - private synchronized void collectEvent(final Event event) { + private void collectEvent(final Event event) { collectInvocationCount.incrementAndGet(); if (event instanceof EnrichedEvent) { @@ -138,32 +131,12 @@ private synchronized void collectEvent(final Event event) { PipeEventCommitManager.getInstance() .enrichWithCommitterKeyAndCommitId((EnrichedEvent) event, creationTime, regionId); } + if (event instanceof PipeHeartbeatEvent) { - ((PipeHeartbeatEvent) event).recordBufferQueueSize(bufferQueue); ((PipeHeartbeatEvent) event).recordConnectorQueueSize(pendingQueue); } - while (!isClosed.get() && !bufferQueue.isEmpty()) { - final Event bufferedEvent = bufferQueue.peek(); - // Try to put already buffered events into pending queue, if pending queue is full, wait for - // pending queue to be available with timeout. - if (pendingQueue.waitedOffer(bufferedEvent)) { - bufferQueue.poll(); - } else { - // We can NOT keep too many PipeHeartbeatEvent in bufferQueue because they may cause OOM. - if (event instanceof PipeHeartbeatEvent - && bufferQueue.peekLast() instanceof PipeHeartbeatEvent) { - ((EnrichedEvent) event).decreaseReferenceCount(PipeEventCollector.class.getName(), false); - } else { - bufferQueue.offer(event); - } - return; - } - } - - if (!pendingQueue.waitedOffer(event)) { - bufferQueue.offer(event); - } + pendingQueue.directOffer(event); } public void resetCollectInvocationCount() { @@ -173,59 +146,4 @@ public void resetCollectInvocationCount() { public boolean hasNoCollectInvocationAfterReset() { return collectInvocationCount.get() == 0; } - - public boolean isBufferQueueEmpty() { - return bufferQueue.isEmpty(); - } - - /** - * Try to collect buffered events into {@link PipeEventCollector#pendingQueue}. - * - * @return {@code true} if there are still buffered events after this operation, {@code false} - * otherwise. - */ - public synchronized boolean tryCollectBufferedEvents() { - while (!isClosed.get() && !bufferQueue.isEmpty()) { - final Event bufferedEvent = bufferQueue.peek(); - if (pendingQueue.waitedOffer(bufferedEvent)) { - bufferQueue.poll(); - } else { - return true; - } - } - return false; - } - - public void close() { - isClosed.set(true); - doClose(); - } - - private synchronized void doClose() { - bufferQueue.forEach( - event -> { - if (event instanceof EnrichedEvent) { - ((EnrichedEvent) event).clearReferenceCount(PipeEventCollector.class.getName()); - } - }); - bufferQueue.clear(); - } - - //////////////////////////// APIs provided for metric framework //////////////////////////// - - public int getTabletInsertionEventCount() { - return bufferQueue.getTabletInsertionEventCount(); - } - - public int getTsFileInsertionEventCount() { - return bufferQueue.getTsFileInsertionEventCount(); - } - - public int getPipeHeartbeatEventCount() { - return bufferQueue.getPipeHeartbeatEventCount(); - } - - public int getEventCount() { - return bufferQueue.size(); - } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/stage/PipeTaskConnectorStage.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/stage/PipeTaskConnectorStage.java index 001978ebe90d..a5ec4fed78bb 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/stage/PipeTaskConnectorStage.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/stage/PipeTaskConnectorStage.java @@ -20,7 +20,7 @@ package org.apache.iotdb.db.pipe.task.stage; import org.apache.iotdb.commons.pipe.config.plugin.env.PipeTaskConnectorRuntimeEnvironment; -import org.apache.iotdb.commons.pipe.task.connection.BoundedBlockingPendingQueue; +import org.apache.iotdb.commons.pipe.task.connection.UnboundedBlockingPendingQueue; import org.apache.iotdb.commons.pipe.task.stage.PipeTaskStage; import org.apache.iotdb.db.pipe.execution.PipeConnectorSubtaskExecutor; import org.apache.iotdb.db.pipe.task.subtask.connector.PipeConnectorSubtaskManager; @@ -83,7 +83,7 @@ public void dropSubtask() throws PipeException { .deregister(pipeName, creationTime, regionId, connectorSubtaskId); } - public BoundedBlockingPendingQueue getPipeConnectorPendingQueue() { + public UnboundedBlockingPendingQueue getPipeConnectorPendingQueue() { return PipeConnectorSubtaskManager.instance().getPipeConnectorPendingQueue(connectorSubtaskId); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/stage/PipeTaskProcessorStage.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/stage/PipeTaskProcessorStage.java index d88bfb0530c4..c7ceaf1f9784 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/stage/PipeTaskProcessorStage.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/stage/PipeTaskProcessorStage.java @@ -25,7 +25,7 @@ import org.apache.iotdb.commons.pipe.config.plugin.env.PipeTaskProcessorRuntimeEnvironment; import org.apache.iotdb.commons.pipe.plugin.builtin.BuiltinPipePlugin; import org.apache.iotdb.commons.pipe.task.EventSupplier; -import org.apache.iotdb.commons.pipe.task.connection.BoundedBlockingPendingQueue; +import org.apache.iotdb.commons.pipe.task.connection.UnboundedBlockingPendingQueue; import org.apache.iotdb.commons.pipe.task.meta.PipeTaskMeta; import org.apache.iotdb.commons.pipe.task.stage.PipeTaskStage; import org.apache.iotdb.db.pipe.agent.PipeAgent; @@ -65,7 +65,7 @@ public PipeTaskProcessorStage( PipeParameters pipeProcessorParameters, int regionId, EventSupplier pipeExtractorInputEventSupplier, - BoundedBlockingPendingQueue pipeConnectorOutputPendingQueue, + UnboundedBlockingPendingQueue pipeConnectorOutputPendingQueue, PipeProcessorSubtaskExecutor executor, PipeTaskMeta pipeTaskMeta) { final PipeProcessorRuntimeConfiguration runtimeConfiguration = diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/subtask/connector/PipeConnectorSubtask.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/subtask/connector/PipeConnectorSubtask.java index 300369ae9ed9..22e87c7e5c8e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/subtask/connector/PipeConnectorSubtask.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/subtask/connector/PipeConnectorSubtask.java @@ -22,7 +22,7 @@ import org.apache.iotdb.commons.exception.pipe.PipeRuntimeException; import org.apache.iotdb.commons.pipe.config.PipeConfig; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; -import org.apache.iotdb.commons.pipe.task.connection.BoundedBlockingPendingQueue; +import org.apache.iotdb.commons.pipe.task.connection.UnboundedBlockingPendingQueue; import org.apache.iotdb.commons.pipe.task.subtask.PipeAbstractConnectorSubtask; import org.apache.iotdb.db.pipe.agent.PipeAgent; import org.apache.iotdb.db.pipe.connector.protocol.thrift.async.IoTDBDataRegionAsyncConnector; @@ -50,7 +50,7 @@ public class PipeConnectorSubtask extends PipeAbstractConnectorSubtask { private static final Logger LOGGER = LoggerFactory.getLogger(PipeConnectorSubtask.class); // For input - protected final BoundedBlockingPendingQueue inputPendingQueue; + protected final UnboundedBlockingPendingQueue inputPendingQueue; // Record these variables to provide corresponding value to tag key of monitoring metrics private final String attributeSortedString; @@ -71,7 +71,7 @@ public PipeConnectorSubtask( final long creationTime, final String attributeSortedString, final int connectorIndex, - final BoundedBlockingPendingQueue inputPendingQueue, + final UnboundedBlockingPendingQueue inputPendingQueue, final PipeConnector outputPipeConnector) { super(taskID, creationTime, outputPipeConnector); this.attributeSortedString = attributeSortedString; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/subtask/connector/PipeConnectorSubtaskLifeCycle.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/subtask/connector/PipeConnectorSubtaskLifeCycle.java index 8391f4c7c810..a28a0289b08d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/subtask/connector/PipeConnectorSubtaskLifeCycle.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/subtask/connector/PipeConnectorSubtaskLifeCycle.java @@ -19,7 +19,7 @@ package org.apache.iotdb.db.pipe.task.subtask.connector; -import org.apache.iotdb.commons.pipe.task.connection.BoundedBlockingPendingQueue; +import org.apache.iotdb.commons.pipe.task.connection.UnboundedBlockingPendingQueue; import org.apache.iotdb.db.pipe.execution.PipeConnectorSubtaskExecutor; import org.apache.iotdb.pipe.api.event.Event; @@ -32,7 +32,7 @@ public class PipeConnectorSubtaskLifeCycle implements AutoCloseable { protected final PipeConnectorSubtaskExecutor executor; protected final PipeConnectorSubtask subtask; - private final BoundedBlockingPendingQueue pendingQueue; + private final UnboundedBlockingPendingQueue pendingQueue; private int runningTaskCount; private int registeredTaskCount; @@ -40,7 +40,7 @@ public class PipeConnectorSubtaskLifeCycle implements AutoCloseable { public PipeConnectorSubtaskLifeCycle( PipeConnectorSubtaskExecutor executor, PipeConnectorSubtask subtask, - BoundedBlockingPendingQueue pendingQueue) { + UnboundedBlockingPendingQueue pendingQueue) { this.executor = executor; this.subtask = subtask; this.pendingQueue = pendingQueue; @@ -53,7 +53,7 @@ public PipeConnectorSubtask getSubtask() { return subtask; } - public BoundedBlockingPendingQueue getPendingQueue() { + public UnboundedBlockingPendingQueue getPendingQueue() { return pendingQueue; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/subtask/connector/PipeConnectorSubtaskManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/subtask/connector/PipeConnectorSubtaskManager.java index b28206bf306c..d85f04b68eaf 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/subtask/connector/PipeConnectorSubtaskManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/subtask/connector/PipeConnectorSubtaskManager.java @@ -20,14 +20,13 @@ package org.apache.iotdb.db.pipe.task.subtask.connector; import org.apache.iotdb.commons.consensus.DataRegionId; -import org.apache.iotdb.commons.pipe.config.PipeConfig; import org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant; import org.apache.iotdb.commons.pipe.config.constant.SystemConstant; import org.apache.iotdb.commons.pipe.config.plugin.configuraion.PipeTaskRuntimeConfiguration; import org.apache.iotdb.commons.pipe.config.plugin.env.PipeTaskConnectorRuntimeEnvironment; import org.apache.iotdb.commons.pipe.plugin.builtin.BuiltinPipePlugin; import org.apache.iotdb.commons.pipe.progress.PipeEventCommitManager; -import org.apache.iotdb.commons.pipe.task.connection.BoundedBlockingPendingQueue; +import org.apache.iotdb.commons.pipe.task.connection.UnboundedBlockingPendingQueue; import org.apache.iotdb.db.pipe.agent.PipeAgent; import org.apache.iotdb.db.pipe.execution.PipeConnectorSubtaskExecutor; import org.apache.iotdb.db.pipe.metric.PipeDataNodeRemainingEventAndTimeMetrics; @@ -84,6 +83,7 @@ public synchronized String register( .contains(new DataRegionId(environment.getRegionId())); final int connectorNum; + boolean realTimeFirst = false; String attributeSortedString = generateAttributeSortedString(pipeConnectorParameters); if (isDataRegionConnector) { connectorNum = @@ -92,6 +92,12 @@ public synchronized String register( PipeConnectorConstant.CONNECTOR_IOTDB_PARALLEL_TASKS_KEY, PipeConnectorConstant.SINK_IOTDB_PARALLEL_TASKS_KEY), PipeConnectorConstant.CONNECTOR_IOTDB_PARALLEL_TASKS_DEFAULT_VALUE); + realTimeFirst = + pipeConnectorParameters.getBooleanOrDefault( + Arrays.asList( + PipeConnectorConstant.CONNECTOR_REALTIME_FIRST_KEY, + PipeConnectorConstant.SINK_REALTIME_FIRST_KEY), + PipeConnectorConstant.CONNECTOR_REALTIME_FIRST_DEFAULT_VALUE); attributeSortedString = "data_" + attributeSortedString; } else { // Do not allow parallel tasks for schema region connectors @@ -105,10 +111,10 @@ public synchronized String register( new ArrayList<>(connectorNum); // Shared pending queue for all subtasks - final BoundedBlockingPendingQueue pendingQueue = - new BoundedBlockingPendingQueue<>( - PipeConfig.getInstance().getPipeConnectorPendingQueueSize(), - new PipeDataRegionEventCounter()); + final UnboundedBlockingPendingQueue pendingQueue = + realTimeFirst + ? new PipeRealtimePriorityBlockingQueue() + : new UnboundedBlockingPendingQueue<>(new PipeDataRegionEventCounter()); for (int connectorIndex = 0; connectorIndex < connectorNum; connectorIndex++) { final PipeConnector pipeConnector = @@ -210,7 +216,7 @@ public synchronized void stop(final String attributeSortedString) { } } - public BoundedBlockingPendingQueue getPipeConnectorPendingQueue( + public UnboundedBlockingPendingQueue getPipeConnectorPendingQueue( final String attributeSortedString) { if (!attributeSortedString2SubtaskLifeCycleMap.containsKey(attributeSortedString)) { throw new PipeException( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/subtask/connector/PipeRealtimePriorityBlockingQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/subtask/connector/PipeRealtimePriorityBlockingQueue.java new file mode 100644 index 000000000000..82b09346322d --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/subtask/connector/PipeRealtimePriorityBlockingQueue.java @@ -0,0 +1,136 @@ +/* + * 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.db.pipe.task.subtask.connector; + +import org.apache.iotdb.commons.pipe.event.EnrichedEvent; +import org.apache.iotdb.commons.pipe.task.connection.UnboundedBlockingPendingQueue; +import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent; +import org.apache.iotdb.db.pipe.metric.PipeDataRegionEventCounter; +import org.apache.iotdb.db.pipe.task.connection.PipeEventCollector; +import org.apache.iotdb.pipe.api.event.Event; +import org.apache.iotdb.pipe.api.event.dml.insertion.TsFileInsertionEvent; + +import java.util.Objects; +import java.util.concurrent.BlockingDeque; +import java.util.concurrent.LinkedBlockingDeque; +import java.util.function.Consumer; + +public class PipeRealtimePriorityBlockingQueue extends UnboundedBlockingPendingQueue { + + private final BlockingDeque tsfileInsertEventDeque = + new LinkedBlockingDeque<>(); + + public PipeRealtimePriorityBlockingQueue() { + super(new PipeDataRegionEventCounter()); + } + + @Override + public boolean directOffer(final Event event) { + if (event instanceof TsFileInsertionEvent) { + tsfileInsertEventDeque.add((TsFileInsertionEvent) event); + return true; + } + + if (event instanceof PipeHeartbeatEvent && super.peekLast() instanceof PipeHeartbeatEvent) { + // We can NOT keep too many PipeHeartbeatEvent in bufferQueue because they may cause OOM. + ((EnrichedEvent) event).decreaseReferenceCount(PipeEventCollector.class.getName(), false); + } else { + super.directOffer(event); + } + return true; + } + + @Override + public boolean waitedOffer(final Event event) { + return directOffer(event); + } + + @Override + public boolean put(final Event event) { + directOffer(event); + return true; + } + + @Override + public Event directPoll() { + Event event = super.directPoll(); + if (Objects.isNull(event)) { + event = tsfileInsertEventDeque.pollLast(); + } + return event; + } + + /** + * Try to poll the freshest insertion event from the queue. First, try to poll the first offered + * non-TsFileInsertionEvent. If no such event is available, poll the last offered + * TsFileInsertionEvent. If no event is available, block until an event is available. + * + * @return the freshest insertion event. can be null if no event is available. + */ + @Override + public Event waitedPoll() { + Event event = null; + + if (!super.isEmpty()) { + // Sequentially poll the first offered non-TsFileInsertionEvent + event = super.directPoll(); + } else if (!tsfileInsertEventDeque.isEmpty()) { + // Always poll the last offered event + event = tsfileInsertEventDeque.pollLast(); + } + + // If no event is available, block until an event is available + if (Objects.isNull(event)) { + event = super.waitedPoll(); + if (Objects.isNull(event)) { + event = tsfileInsertEventDeque.pollLast(); + } + } + + return event; + } + + @Override + public void clear() { + super.clear(); + tsfileInsertEventDeque.clear(); + } + + @Override + public void forEach(final Consumer action) { + super.forEach(action); + tsfileInsertEventDeque.forEach(action); + } + + @Override + public boolean isEmpty() { + return super.isEmpty() && tsfileInsertEventDeque.isEmpty(); + } + + @Override + public int size() { + return super.size() + tsfileInsertEventDeque.size(); + } + + @Override + public int getTsFileInsertionEventCount() { + return tsfileInsertEventDeque.size(); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/subtask/processor/PipeProcessorSubtask.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/subtask/processor/PipeProcessorSubtask.java index 6839dc1c8280..2b48becaed2a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/subtask/processor/PipeProcessorSubtask.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/subtask/processor/PipeProcessorSubtask.java @@ -30,7 +30,6 @@ import org.apache.iotdb.db.pipe.agent.PipeAgent; import org.apache.iotdb.db.pipe.event.UserDefinedEnrichedEvent; import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent; -import org.apache.iotdb.db.pipe.metric.PipeDataNodeRemainingEventAndTimeMetrics; import org.apache.iotdb.db.pipe.metric.PipeProcessorMetrics; import org.apache.iotdb.db.pipe.task.connection.PipeEventCollector; import org.apache.iotdb.db.storageengine.StorageEngine; @@ -88,7 +87,6 @@ public PipeProcessorSubtask( if (StorageEngine.getInstance().getAllDataRegionIds().contains(new DataRegionId(regionId))) { PipeProcessorMetrics.getInstance().register(this); } - PipeDataNodeRemainingEventAndTimeMetrics.getInstance().register(this); } @Override @@ -124,15 +122,8 @@ protected boolean executeOnce() throws Exception { // Record the last event for retry when exception occurs setLastEvent(event); - if ( - // Though there is no event to process, there may still be some buffered events - // in the outputEventCollector. Return true if there are still buffered events, - // false otherwise. - event == null - // If there are still buffered events, process them first, the newly supplied - // event will be processed in the next round. - || !outputEventCollector.isBufferQueueEmpty()) { - return outputEventCollector.tryCollectBufferedEvents(); + if (Objects.isNull(event)) { + return false; } outputEventCollector.resetCollectInvocationCount(); @@ -216,8 +207,6 @@ public void close() { ErrorHandlingUtils.getRootCause(e).getMessage(), e); } finally { - outputEventCollector.close(); - // should be called after pipeProcessor.close() super.close(); } @@ -255,22 +244,6 @@ public int getRegionId() { return regionId; } - public int getTabletInsertionEventCount() { - return outputEventCollector.getTabletInsertionEventCount(); - } - - public int getTsFileInsertionEventCount() { - return outputEventCollector.getTsFileInsertionEventCount(); - } - - public int getPipeHeartbeatEventCount() { - return outputEventCollector.getPipeHeartbeatEventCount(); - } - - public int getEventCount() { - return outputEventCollector.getEventCount(); - } - //////////////////////////// Error report //////////////////////////// @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionBroker.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionBroker.java index d8da569051ac..d0e900e0a5aa 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionBroker.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionBroker.java @@ -19,7 +19,7 @@ package org.apache.iotdb.db.subscription.broker; -import org.apache.iotdb.commons.pipe.task.connection.BoundedBlockingPendingQueue; +import org.apache.iotdb.commons.pipe.task.connection.UnboundedBlockingPendingQueue; import org.apache.iotdb.db.subscription.timer.SubscriptionPollTimer; import org.apache.iotdb.pipe.api.event.Event; @@ -89,7 +89,7 @@ public void commit(Map> topicNameToSubscriptionCommitIds) { /////////////////////////////// prefetching queue /////////////////////////////// public void bindPrefetchingQueue( - String topicName, BoundedBlockingPendingQueue inputPendingQueue) { + String topicName, UnboundedBlockingPendingQueue inputPendingQueue) { final SubscriptionPrefetchingQueue prefetchingQueue = topicNameToPrefetchingQueue.get(topicName); if (Objects.nonNull(prefetchingQueue)) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingQueue.java index 9a75e464475c..c9a6a4fbab20 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingQueue.java @@ -20,7 +20,7 @@ package org.apache.iotdb.db.subscription.broker; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; -import org.apache.iotdb.commons.pipe.task.connection.BoundedBlockingPendingQueue; +import org.apache.iotdb.commons.pipe.task.connection.UnboundedBlockingPendingQueue; import org.apache.iotdb.commons.subscription.config.SubscriptionConfig; import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.pipe.agent.PipeAgent; @@ -53,7 +53,7 @@ public class SubscriptionPrefetchingQueue { private final String brokerId; // consumer group id private final String topicName; - private final BoundedBlockingPendingQueue inputPendingQueue; + private final UnboundedBlockingPendingQueue inputPendingQueue; private final Map uncommittedEvents; private final LinkedBlockingQueue prefetchingQueue; @@ -61,7 +61,7 @@ public class SubscriptionPrefetchingQueue { private final AtomicLong subscriptionCommitIdGenerator = new AtomicLong(0); public SubscriptionPrefetchingQueue( - String brokerId, String topicName, BoundedBlockingPendingQueue inputPendingQueue) { + String brokerId, String topicName, UnboundedBlockingPendingQueue inputPendingQueue) { this.brokerId = brokerId; this.topicName = topicName; this.inputPendingQueue = inputPendingQueue; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/task/stage/SubscriptionTaskConnectorStage.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/task/stage/SubscriptionTaskConnectorStage.java index e5f965f71764..040eccced459 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/task/stage/SubscriptionTaskConnectorStage.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/task/stage/SubscriptionTaskConnectorStage.java @@ -20,7 +20,7 @@ package org.apache.iotdb.db.subscription.task.stage; import org.apache.iotdb.commons.pipe.config.plugin.env.PipeTaskConnectorRuntimeEnvironment; -import org.apache.iotdb.commons.pipe.task.connection.BoundedBlockingPendingQueue; +import org.apache.iotdb.commons.pipe.task.connection.UnboundedBlockingPendingQueue; import org.apache.iotdb.db.pipe.execution.PipeConnectorSubtaskExecutor; import org.apache.iotdb.db.pipe.task.stage.PipeTaskConnectorStage; import org.apache.iotdb.db.subscription.task.subtask.SubscriptionConnectorSubtaskManager; @@ -70,7 +70,7 @@ public void dropSubtask() throws PipeException { .deregister(pipeName, creationTime, regionId, connectorSubtaskId); } - public BoundedBlockingPendingQueue getPipeConnectorPendingQueue() { + public UnboundedBlockingPendingQueue getPipeConnectorPendingQueue() { return SubscriptionConnectorSubtaskManager.instance() .getPipeConnectorPendingQueue(connectorSubtaskId); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/task/subtask/SubscriptionConnectorSubtask.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/task/subtask/SubscriptionConnectorSubtask.java index 4ea4dfe96d08..bb4838b903e7 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/task/subtask/SubscriptionConnectorSubtask.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/task/subtask/SubscriptionConnectorSubtask.java @@ -19,7 +19,7 @@ package org.apache.iotdb.db.subscription.task.subtask; -import org.apache.iotdb.commons.pipe.task.connection.BoundedBlockingPendingQueue; +import org.apache.iotdb.commons.pipe.task.connection.UnboundedBlockingPendingQueue; import org.apache.iotdb.db.pipe.task.subtask.connector.PipeConnectorSubtask; import org.apache.iotdb.db.subscription.agent.SubscriptionAgent; import org.apache.iotdb.pipe.api.PipeConnector; @@ -35,7 +35,7 @@ public SubscriptionConnectorSubtask( long creationTime, String attributeSortedString, int connectorIndex, - BoundedBlockingPendingQueue inputPendingQueue, + UnboundedBlockingPendingQueue inputPendingQueue, PipeConnector outputPipeConnector, String topicName, String consumerGroupId) { @@ -69,7 +69,7 @@ public String getConsumerGroupId() { return consumerGroupId; } - public BoundedBlockingPendingQueue getInputPendingQueue() { + public UnboundedBlockingPendingQueue getInputPendingQueue() { return inputPendingQueue; } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/task/subtask/SubscriptionConnectorSubtaskLifeCycle.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/task/subtask/SubscriptionConnectorSubtaskLifeCycle.java index 8192d13e811d..14633dfafefe 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/task/subtask/SubscriptionConnectorSubtaskLifeCycle.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/task/subtask/SubscriptionConnectorSubtaskLifeCycle.java @@ -19,7 +19,7 @@ package org.apache.iotdb.db.subscription.task.subtask; -import org.apache.iotdb.commons.pipe.task.connection.BoundedBlockingPendingQueue; +import org.apache.iotdb.commons.pipe.task.connection.UnboundedBlockingPendingQueue; import org.apache.iotdb.db.pipe.execution.PipeConnectorSubtaskExecutor; import org.apache.iotdb.db.pipe.task.subtask.connector.PipeConnectorSubtask; import org.apache.iotdb.db.pipe.task.subtask.connector.PipeConnectorSubtaskLifeCycle; @@ -40,7 +40,7 @@ public class SubscriptionConnectorSubtaskLifeCycle extends PipeConnectorSubtaskL public SubscriptionConnectorSubtaskLifeCycle( PipeConnectorSubtaskExecutor executor, // SubscriptionSubtaskExecutor PipeConnectorSubtask subtask, // SubscriptionConnectorSubtask - BoundedBlockingPendingQueue pendingQueue) { + UnboundedBlockingPendingQueue pendingQueue) { super(executor, subtask, pendingQueue); runningTaskCount = 0; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/task/subtask/SubscriptionConnectorSubtaskManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/task/subtask/SubscriptionConnectorSubtaskManager.java index 8f2cbd613d57..35a4a3ab4ac8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/task/subtask/SubscriptionConnectorSubtaskManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/task/subtask/SubscriptionConnectorSubtaskManager.java @@ -19,19 +19,19 @@ package org.apache.iotdb.db.subscription.task.subtask; -import org.apache.iotdb.commons.pipe.config.PipeConfig; import org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant; import org.apache.iotdb.commons.pipe.config.constant.SystemConstant; import org.apache.iotdb.commons.pipe.config.plugin.configuraion.PipeTaskRuntimeConfiguration; import org.apache.iotdb.commons.pipe.config.plugin.env.PipeTaskConnectorRuntimeEnvironment; import org.apache.iotdb.commons.pipe.plugin.builtin.BuiltinPipePlugin; import org.apache.iotdb.commons.pipe.progress.PipeEventCommitManager; -import org.apache.iotdb.commons.pipe.task.connection.BoundedBlockingPendingQueue; +import org.apache.iotdb.commons.pipe.task.connection.UnboundedBlockingPendingQueue; import org.apache.iotdb.db.pipe.agent.PipeAgent; import org.apache.iotdb.db.pipe.execution.PipeConnectorSubtaskExecutor; import org.apache.iotdb.db.pipe.metric.PipeDataRegionEventCounter; import org.apache.iotdb.db.pipe.task.subtask.connector.PipeConnectorSubtask; import org.apache.iotdb.db.pipe.task.subtask.connector.PipeConnectorSubtaskLifeCycle; +import org.apache.iotdb.db.pipe.task.subtask.connector.PipeRealtimePriorityBlockingQueue; import org.apache.iotdb.pipe.api.PipeConnector; import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameterValidator; import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameters; @@ -86,14 +86,20 @@ public synchronized String register( environment.getRegionId(), connectorKey); + boolean realTimeFirst = + pipeConnectorParameters.getBooleanOrDefault( + Arrays.asList( + PipeConnectorConstant.CONNECTOR_REALTIME_FIRST_KEY, + PipeConnectorConstant.SINK_REALTIME_FIRST_KEY), + PipeConnectorConstant.CONNECTOR_REALTIME_FIRST_DEFAULT_VALUE); String attributeSortedString = generateAttributeSortedString(pipeConnectorParameters); attributeSortedString = "__subscription_" + attributeSortedString; if (!attributeSortedString2SubtaskLifeCycleMap.containsKey(attributeSortedString)) { - final BoundedBlockingPendingQueue pendingQueue = - new BoundedBlockingPendingQueue<>( - PipeConfig.getInstance().getPipeConnectorPendingQueueSize(), - new PipeDataRegionEventCounter()); + final UnboundedBlockingPendingQueue pendingQueue = + realTimeFirst + ? new PipeRealtimePriorityBlockingQueue() + : new UnboundedBlockingPendingQueue<>(new PipeDataRegionEventCounter()); final PipeConnector pipeConnector = PipeAgent.plugin().dataRegion().reflectConnector(pipeConnectorParameters); @@ -193,7 +199,7 @@ public synchronized void stop(final String attributeSortedString) { lifeCycle.stop(); } - public BoundedBlockingPendingQueue getPipeConnectorPendingQueue( + public UnboundedBlockingPendingQueue getPipeConnectorPendingQueue( final String attributeSortedString) { if (!attributeSortedString2SubtaskLifeCycleMap.containsKey(attributeSortedString)) { throw new PipeException( diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/execution/PipeConnectorSubtaskExecutorTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/execution/PipeConnectorSubtaskExecutorTest.java index e1f5063eacc6..e83c8500a48f 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/execution/PipeConnectorSubtaskExecutorTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/execution/PipeConnectorSubtaskExecutorTest.java @@ -19,7 +19,7 @@ package org.apache.iotdb.db.pipe.execution; -import org.apache.iotdb.commons.pipe.task.connection.BoundedBlockingPendingQueue; +import org.apache.iotdb.commons.pipe.task.connection.UnboundedBlockingPendingQueue; import org.apache.iotdb.db.pipe.task.subtask.connector.PipeConnectorSubtask; import org.apache.iotdb.pipe.api.PipeConnector; @@ -41,7 +41,7 @@ public void setUp() throws Exception { System.currentTimeMillis(), "TestAttributeSortedString", 0, - mock(BoundedBlockingPendingQueue.class), + mock(UnboundedBlockingPendingQueue.class), mock(PipeConnector.class))); } } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java index 12ef440b81b9..fc437a3de979 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java @@ -189,9 +189,6 @@ public class CommonConfig { private int pipeConnectorTransferTimeoutMs = 15 * 60 * 1000; // 15 minutes private int pipeConnectorReadFileBufferSize = 8388608; private long pipeConnectorRetryIntervalMs = 1000L; - // recommend to set this value to 3 * pipeSubtaskExecutorMaxThreadNum * - // pipeAsyncConnectorCoreClientNumber - private int pipeConnectorPendingQueueSize = 256; private boolean pipeConnectorRPCThriftCompressionEnabled = false; private int pipeAsyncConnectorSelectorNumber = 4; @@ -758,14 +755,6 @@ public void setPipeConnectorRetryIntervalMs(long pipeConnectorRetryIntervalMs) { this.pipeConnectorRetryIntervalMs = pipeConnectorRetryIntervalMs; } - public int getPipeConnectorPendingQueueSize() { - return pipeConnectorPendingQueueSize; - } - - public void setPipeConnectorPendingQueueSize(int pipeConnectorPendingQueueSize) { - this.pipeConnectorPendingQueueSize = pipeConnectorPendingQueueSize; - } - public int getPipeSubtaskExecutorBasicCheckPointIntervalByConsumedEventCount() { return pipeSubtaskExecutorBasicCheckPointIntervalByConsumedEventCount; } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonDescriptor.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonDescriptor.java index e1ec6c03dbbf..efb3420b62ca 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonDescriptor.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonDescriptor.java @@ -354,13 +354,6 @@ private void loadPipeProps(Properties properties) { properties.getProperty( "pipe_connector_retry_interval_ms", String.valueOf(config.getPipeConnectorRetryIntervalMs()))))); - config.setPipeConnectorPendingQueueSize( - Integer.parseInt( - Optional.ofNullable(properties.getProperty("pipe_sink_pending_queue_size")) - .orElse( - properties.getProperty( - "pipe_connector_pending_queue_size", - String.valueOf(config.getPipeConnectorPendingQueueSize()))))); config.setPipeConnectorRPCThriftCompressionEnabled( Boolean.parseBoolean( Optional.ofNullable(properties.getProperty("pipe_sink_rpc_thrift_compression_enabled")) diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java index 1821fe74a7c8..41331a822dda 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java @@ -111,10 +111,6 @@ public long getPipeConnectorRetryIntervalMs() { return COMMON_CONFIG.getPipeConnectorRetryIntervalMs(); } - public int getPipeConnectorPendingQueueSize() { - return COMMON_CONFIG.getPipeConnectorPendingQueueSize(); - } - public boolean isPipeConnectorRPCThriftCompressionEnabled() { return COMMON_CONFIG.isPipeConnectorRPCThriftCompressionEnabled(); } @@ -312,7 +308,6 @@ public void printAllConfigs() { LOGGER.info("PipeConnectorTransferTimeoutMs: {}", getPipeConnectorTransferTimeoutMs()); LOGGER.info("PipeConnectorReadFileBufferSize: {}", getPipeConnectorReadFileBufferSize()); LOGGER.info("PipeConnectorRetryIntervalMs: {}", getPipeConnectorRetryIntervalMs()); - LOGGER.info("PipeConnectorPendingQueueSize: {}", getPipeConnectorPendingQueueSize()); LOGGER.info( "PipeConnectorRPCThriftCompressionEnabled: {}", isPipeConnectorRPCThriftCompressionEnabled()); 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 331c4d79a2f8..c15fc4db1890 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,6 +53,10 @@ public class PipeConnectorConstant { public static final int CONNECTOR_IOTDB_PARALLEL_TASKS_DEFAULT_VALUE = PipeConfig.getInstance().getPipeSubtaskExecutorMaxThreadNum(); + public static final String CONNECTOR_REALTIME_FIRST_KEY = "connector.realtime-first"; + public static final String SINK_REALTIME_FIRST_KEY = "sink.realtime-first"; + public static final boolean CONNECTOR_REALTIME_FIRST_DEFAULT_VALUE = false; + public static final String CONNECTOR_IOTDB_BATCH_MODE_ENABLE_KEY = "connector.batch.enable"; public static final String SINK_IOTDB_BATCH_MODE_ENABLE_KEY = "sink.batch.enable"; public static final boolean CONNECTOR_IOTDB_BATCH_MODE_ENABLE_DEFAULT_VALUE = true; From cabdec75b637a9f235a55ec634f4b71b66765f65 Mon Sep 17 00:00:00 2001 From: Potato Date: Tue, 21 May 2024 21:19:14 +0800 Subject: [PATCH 25/42] Fixed restart error when DataRegion is RatisConsensus (#12554) --- .../confignode/manager/ConfigManager.java | 2 +- .../manager/consensus/ConsensusManager.java | 7 ++ .../apache/iotdb/consensus/IConsensus.java | 3 +- .../iotdb/consensus/ratis/DiskGuardian.java | 65 +++++----- .../iotdb/consensus/ratis/RatisConsensus.java | 113 ++++++++++-------- 5 files changed, 110 insertions(+), 80 deletions(-) diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java index 0c407afec2f4..2666008cd80c 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java @@ -1280,7 +1280,7 @@ public TSStatus createPeerForConsensusGroup(List configNode for (int i = 0; i < rpcTimeoutInMS / retryIntervalInMS; i++) { try { - if (consensusManager.get() == null) { + if (consensusManager.get() == null || !consensusManager.get().isInitialized()) { TimeUnit.MILLISECONDS.sleep(retryIntervalInMS); } else { // When add non Seed-ConfigNode to the ConfigNodeGroup, the parameter should be emptyList diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/consensus/ConsensusManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/consensus/ConsensusManager.java index e28aee7fdabd..361229f44354 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/consensus/ConsensusManager.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/consensus/ConsensusManager.java @@ -77,6 +77,8 @@ public class ConsensusManager { private final IManager configManager; private IConsensus consensusImpl; + private boolean isInitialized; + public ConsensusManager(IManager configManager, ConfigRegionStateMachine stateMachine) { this.configManager = configManager; setConsensusLayer(stateMachine); @@ -101,6 +103,7 @@ public void start() throws IOException { "Something wrong happened while calling consensus layer's createLocalPeer API.", e); } } + isInitialized = true; } public void close() throws IOException { @@ -444,4 +447,8 @@ private NodeManager getNodeManager() { public void manuallyTakeSnapshot() throws ConsensusException { consensusImpl.triggerSnapshot(ConfigNodeInfo.CONFIG_REGION_ID, true); } + + public boolean isInitialized() { + return isInitialized; + } } diff --git a/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/IConsensus.java b/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/IConsensus.java index 8baaec892476..643c8360e847 100644 --- a/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/IConsensus.java +++ b/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/IConsensus.java @@ -43,7 +43,8 @@ public interface IConsensus { /** - * Start the consensus module. + * Start the consensus module. Note: You should call this function immediately after initializing + * the instance, because calling other functions without start may produce unexpected errors * * @throws IOException when start consensus errors */ diff --git a/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/ratis/DiskGuardian.java b/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/ratis/DiskGuardian.java index faa0f317d1a9..ae6e597aafb1 100644 --- a/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/ratis/DiskGuardian.java +++ b/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/ratis/DiskGuardian.java @@ -143,7 +143,7 @@ public String toString() { void start() { // first schedule the snapshot daemon ScheduledExecutorUtil.safelyScheduleWithFixedDelay( - workerThread, this::snapshotDaemon, 0L, daemonIntervalMs, TimeUnit.SECONDS); + workerThread, this::snapshotDaemon, daemonIntervalMs, daemonIntervalMs, TimeUnit.SECONDS); // then schedule all checker daemons snapshotArbitrators.forEach( @@ -151,7 +151,7 @@ void start() { ScheduledExecutorUtil.safelyScheduleWithFixedDelay( workerThread, () -> checkerDaemon(checkers), - 0L, + daemonIntervalMs, interval.toLong(TimeUnit.SECONDS), TimeUnit.SECONDS)); } @@ -175,26 +175,31 @@ private void snapshotDaemon() { if (isStopped.get()) { return; } - for (RaftGroupId groupId : serverRef.get().getServer().getGroupIds()) { - if (getSnapshotFlag(groupId).get()) { - try { - serverRef.get().triggerSnapshot(Utils.fromRaftGroupIdToConsensusGroupId(groupId), false); - final boolean flagCleared = snapshotFlag.get(groupId).compareAndSet(true, false); - if (!flagCleared) { - logger.warn( - "{}: clear snapshot flag failed for group {}, please check the related implementation", + try { + for (RaftGroupId groupId : serverRef.get().getServer().getGroupIds()) { + if (getSnapshotFlag(groupId).get()) { + try { + serverRef + .get() + .triggerSnapshot(Utils.fromRaftGroupIdToConsensusGroupId(groupId), false); + final boolean flagCleared = snapshotFlag.get(groupId).compareAndSet(true, false); + if (!flagCleared) { + logger.info( + "{}: clear snapshot flag failed for group {}, please check the related implementation", + this, + groupId); + } + } catch (ConsensusException e) { + logger.info( + "{} take snapshot failed for group {} due to {}. Disk file status {}", this, - groupId); + groupId, + e, + getLatestSummary(groupId).orElse(null)); } - } catch (ConsensusException e) { - logger.warn( - "{} take snapshot failed for group {} due to {}. Disk file status {}", - this, - groupId, - e, - getLatestSummary(groupId).orElse(null)); } } + } catch (IOException ignore) { } } @@ -203,18 +208,21 @@ private void checkerDaemon(List> checkerList) { if (isStopped.get()) { return; } - for (RaftGroupId groupId : serverRef.get().getServer().getGroupIds()) { - final Optional summary = getLatestSummary(groupId); - if (summary.isPresent()) { - final Optional anyCheckerPositive = - checkerList.stream() - .map(checker -> checker.test(summary.get())) - .filter(Boolean::booleanValue) - .findAny(); - if (anyCheckerPositive.isPresent()) { - getSnapshotFlag(groupId).set(true); + try { + for (RaftGroupId groupId : serverRef.get().getServer().getGroupIds()) { + final Optional summary = getLatestSummary(groupId); + if (summary.isPresent()) { + final Optional anyCheckerPositive = + checkerList.stream() + .map(checker -> checker.test(summary.get())) + .filter(Boolean::booleanValue) + .findAny(); + if (anyCheckerPositive.isPresent()) { + getSnapshotFlag(groupId).set(true); + } } } + } catch (IOException ignore) { } } @@ -240,7 +248,6 @@ private Optional getLatestSummary(RaftGroupId groupId) { .getCurrentDir(); return new RaftLogSummary(gid, root); } catch (IOException e) { - logger.warn("{}: group not exists for {} and caught exception ", this, groupId, e); return null; } }); diff --git a/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/ratis/RatisConsensus.java b/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/ratis/RatisConsensus.java index de646de2334f..45c82332f55f 100644 --- a/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/ratis/RatisConsensus.java +++ b/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/ratis/RatisConsensus.java @@ -83,6 +83,7 @@ import org.apache.ratis.server.RaftServerConfigKeys; import org.apache.ratis.server.storage.RaftStorage; import org.apache.ratis.thirdparty.io.grpc.StatusRuntimeException; +import org.apache.ratis.util.MemoizedCheckedSupplier; import org.apache.ratis.util.TimeDuration; import org.apache.ratis.util.function.CheckedSupplier; import org.slf4j.Logger; @@ -115,8 +116,7 @@ class RatisConsensus implements IConsensus { private final RaftPeer myself; private final File storageDir; - - private final RaftServer server; + private final MemoizedCheckedSupplier server; private final RaftProperties properties = new RaftProperties(); private final RaftClientRpc clientRpc; @@ -144,8 +144,7 @@ class RatisConsensus implements IConsensus { private final ConcurrentHashMap canServeStaleRead; - public RatisConsensus(ConsensusConfig config, IStateMachine.Registry registry) - throws IOException { + public RatisConsensus(ConsensusConfig config, IStateMachine.Registry registry) { myself = Utils.fromNodeInfoAndPriorityToRaftPeer( config.getThisNodeId(), config.getThisNodeEndPoint(), DEFAULT_PRIORITY); @@ -195,24 +194,28 @@ public RatisConsensus(ConsensusConfig config, IStateMachine.Registry registry) clientRpc = new GrpcFactory(new Parameters()).newRaftClientRpc(ClientId.randomId(), properties); + // do not build server in constructor in case stateMachine is not ready server = - RaftServer.newBuilder() - .setServerId(myself.getId()) - .setProperties(properties) - .setOption(RaftStorage.StartupOption.RECOVER) - .setStateMachineRegistry( - raftGroupId -> - new ApplicationStateMachineProxy( - registry.apply(Utils.fromRaftGroupIdToConsensusGroupId(raftGroupId)), - raftGroupId, - this::onLeaderChanged)) - .build(); + MemoizedCheckedSupplier.valueOf( + () -> + RaftServer.newBuilder() + .setServerId(myself.getId()) + .setProperties(properties) + .setOption(RaftStorage.StartupOption.RECOVER) + .setStateMachineRegistry( + raftGroupId -> + new ApplicationStateMachineProxy( + registry.apply( + Utils.fromRaftGroupIdToConsensusGroupId(raftGroupId)), + raftGroupId, + this::onLeaderChanged)) + .build()); } @Override public synchronized void start() throws IOException { MetricService.getInstance().addMetricSet(this.ratisMetricSet); - server.start(); + server.get().start(); registerAndStartDiskGuardian(); } @@ -225,7 +228,7 @@ public synchronized void stop() throws IOException { Thread.currentThread().interrupt(); } finally { clientManager.close(); - server.close(); + server.get().close(); MetricService.getInstance().removeMetricSet(this.ratisMetricSet); } } @@ -252,7 +255,7 @@ private RaftClientReply writeWithRetry(CheckedSupplier server.submitClientRequest(request)); + return writeWithRetry(() -> server.get().submitClientRequest(request)); } private RaftClientReply writeRemotelyWithRetry(RatisClient client, Message message) @@ -288,8 +291,12 @@ public TSStatus write(ConsensusGroupId groupId, IConsensusRequest request) Message message = new RequestMessage(request); // 1. first try the local server - RaftClientRequest clientRequest = - buildRawRequest(raftGroupId, message, RaftClientRequest.writeRequestType()); + RaftClientRequest clientRequest; + try { + clientRequest = buildRawRequest(raftGroupId, message, RaftClientRequest.writeRequestType()); + } catch (IOException e) { + throw new RatisRequestFailedException(e); + } RaftPeer suggestedLeader = null; if ((isLeader(groupId) || raftGroup.getPeers().size() == 1) @@ -392,7 +399,7 @@ private RaftClientReply doRead( Retriable.attempt( () -> { try { - return server.submitClientRequest(request); + return server.get().submitClientRequest(request); } catch ( IOException ioe) { // IOE indicates some unexpected errors, say StatusRuntimeException @@ -402,7 +409,7 @@ private RaftClientReply doRead( // We can still retry in case it's a temporary network partition. return RaftClientReply.newBuilder() .setClientId(localFakeId) - .setServerId(server.getId()) + .setServerId(server.get().getId()) .setGroupId(request.getRaftGroupId()) .setException( new ReadIndexException( @@ -440,9 +447,11 @@ public void createLocalPeer(ConsensusGroupId groupId, List peers) RaftGroup group = buildRaftGroup(groupId, peers); try { RaftClientReply reply = - server.groupManagement( - GroupManagementRequest.newAdd( - localFakeId, myself.getId(), localFakeCallId.incrementAndGet(), group, true)); + server + .get() + .groupManagement( + GroupManagementRequest.newAdd( + localFakeId, myself.getId(), localFakeCallId.incrementAndGet(), group, true)); if (!reply.isSuccess()) { throw new RatisRequestFailedException(reply.getException()); } @@ -468,14 +477,16 @@ public void deleteLocalPeer(ConsensusGroupId groupId) throws ConsensusException RaftClientReply reply; try { reply = - server.groupManagement( - GroupManagementRequest.newRemove( - localFakeId, - myself.getId(), - localFakeCallId.incrementAndGet(), - raftGroupId, - true, - false)); + server + .get() + .groupManagement( + GroupManagementRequest.newRemove( + localFakeId, + myself.getId(), + localFakeCallId.incrementAndGet(), + raftGroupId, + true, + false)); if (!reply.isSuccess()) { throw new RatisRequestFailedException(reply.getException()); } @@ -602,7 +613,7 @@ private RaftClientReply transferLeader(RaftGroup group, RaftPeer newLeader) thro public boolean isLeader(ConsensusGroupId groupId) { RaftGroupId raftGroupId = Utils.fromConsensusGroupIdToRaftGroupId(groupId); try { - return server.getDivision(raftGroupId).getInfo().isLeader(); + return server.get().getDivision(raftGroupId).getInfo().isLeader(); } catch (IOException exception) { // if the read fails, simply return not leader logger.info("isLeader request failed with exception: ", exception); @@ -614,7 +625,7 @@ public boolean isLeader(ConsensusGroupId groupId) { public boolean isLeaderReady(ConsensusGroupId groupId) { RaftGroupId raftGroupId = Utils.fromConsensusGroupIdToRaftGroupId(groupId); try { - return server.getDivision(raftGroupId).getInfo().isLeaderReady(); + return server.get().getDivision(raftGroupId).getInfo().isLeaderReady(); } catch (IOException exception) { // if the read fails, simply return not ready logger.info("isLeaderReady request failed with exception: ", exception); @@ -626,7 +637,7 @@ public boolean isLeaderReady(ConsensusGroupId groupId) { public long getLogicalClock(ConsensusGroupId groupId) { RaftGroupId raftGroupId = Utils.fromConsensusGroupIdToRaftGroupId(groupId); try { - return server.getDivision(raftGroupId).getInfo().getCurrentTerm(); + return server.get().getDivision(raftGroupId).getInfo().getCurrentTerm(); } catch (IOException exception) { // if the read fails, simply return 0 logger.info("getLogicalClock request failed with exception: ", exception); @@ -637,7 +648,7 @@ public long getLogicalClock(ConsensusGroupId groupId) { private boolean waitUntilLeaderReady(RaftGroupId groupId) { DivisionInfo divisionInfo; try { - divisionInfo = server.getDivision(groupId).getInfo(); + divisionInfo = server.get().getDivision(groupId).getInfo(); } catch (IOException e) { // if the read fails, simply return not leader logger.info("isLeaderReady checking failed with exception: ", e); @@ -685,7 +696,7 @@ public Peer getLeader(ConsensusGroupId groupId) { RaftPeerId leaderId; try { - leaderId = server.getDivision(raftGroupId).getInfo().getLeaderId(); + leaderId = server.get().getDivision(raftGroupId).getInfo().getLeaderId(); } catch (IOException e) { logger.warn("fetch division info for group " + groupId + " failed due to: ", e); return null; @@ -700,10 +711,15 @@ public Peer getLeader(ConsensusGroupId groupId) { @Override public List getAllConsensusGroupIds() { List ids = new ArrayList<>(); - server - .getGroupIds() - .forEach(groupId -> ids.add(Utils.fromRaftGroupIdToConsensusGroupId(groupId))); - return ids; + try { + server + .get() + .getGroupIds() + .forEach(groupId -> ids.add(Utils.fromRaftGroupIdToConsensusGroupId(groupId))); + return ids; + } catch (IOException e) { + return Collections.emptyList(); + } } @Override @@ -757,7 +773,7 @@ public void triggerSnapshot(ConsensusGroupId groupId, boolean force) throws Cons final RaftClientReply reply; try { - reply = server.snapshotManagement(request); + reply = server.get().snapshotManagement(request); if (!reply.isSuccess()) { throw new RatisRequestFailedException(reply.getException()); } @@ -785,9 +801,9 @@ private void registerAndStartDiskGuardian() { } private RaftClientRequest buildRawRequest( - RaftGroupId groupId, Message message, RaftClientRequest.Type type) { + RaftGroupId groupId, Message message, RaftClientRequest.Type type) throws IOException { return RaftClientRequest.newBuilder() - .setServerId(server.getId()) + .setServerId(server.get().getId()) .setClientId(localFakeId) .setCallId(localFakeCallId.incrementAndGet()) .setGroupId(groupId) @@ -799,7 +815,7 @@ private RaftClientRequest buildRawRequest( private RaftGroup getGroupInfo(RaftGroupId raftGroupId) { RaftGroup raftGroup = null; try { - raftGroup = server.getDivision(raftGroupId).getGroup(); + raftGroup = server.get().getDivision(raftGroupId).getGroup(); RaftGroup lastSeenGroup = lastSeen.getOrDefault(raftGroupId, null); if (lastSeenGroup != null && !lastSeenGroup.equals(raftGroup)) { // delete the pooled raft-client of the out-dated group and cache the latest @@ -854,9 +870,8 @@ private void onLeaderChanged(RaftGroupMemberId groupMemberId, RaftPeerId leaderI }); } - @TestOnly - public RaftServer getServer() { - return server; + public RaftServer getServer() throws IOException { + return server.get(); } @TestOnly From 49dc7650e8021b33baafdbd9ace17bf66dc88da6 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Tue, 21 May 2024 22:07:26 +0800 Subject: [PATCH 26/42] Pipe: Support parsing pattern when tranferring schema plans (#12464) Co-authored-by: Steve Yurong Su --- .../it/autocreate/IoTDBPipeExtractorIT.java | 10 +- .../pipe/it/manual/IoTDBPipeInclusionIT.java | 49 ++ .../it/manual/IoTDBPipeMetaHistoricalIT.java | 4 + .../consensus/request/ConfigPhysicalPlan.java | 4 + .../request/ConfigPhysicalPlanType.java | 1 + .../consensus/request/auth/AuthorPlan.java | 21 +- .../payload/PipeDeleteLogicalViewPlan.java | 11 +- .../payload/PipeDeleteTimeSeriesPlan.java | 11 +- .../write/pipe/payload/PipeSetTTLPlan.java | 103 +++++ .../PipeTransferConfigSnapshotSealReq.java | 33 +- .../IoTDBConfigRegionAirGapConnector.java | 2 + .../protocol/IoTDBConfigRegionConnector.java | 2 + .../extractor/IoTDBConfigRegionExtractor.java | 18 +- ...ConfigPhysicalPlanPatternParseVisitor.java | 283 ++++++++++++ .../protocol/IoTDBConfigNodeReceiver.java | 42 +- ...PipeConfigPhysicalPlanTSStatusVisitor.java | 55 +-- .../thrift/ConfigNodeRPCServiceProcessor.java | 65 ++- .../request/ConfigPhysicalPlanSerDeTest.java | 111 +++-- .../PipeConfigNodeThriftRequestTest.java | 2 +- .../IoTDBConfigRegionExtractorTest.java | 7 +- ...igPhysicalPlanPatternParseVisitorTest.java | 408 +++++++++++++++++ .../PipeTransferSchemaSnapshotSealReq.java | 32 +- .../IoTDBSchemaRegionAirGapConnector.java | 2 + .../sync/IoTDBSchemaRegionConnector.java | 2 + .../PipeSchemaRegionWritePlanEvent.java | 4 +- .../tablet/TabletInsertionDataContainer.java | 32 +- .../tsfile/TsFileInsertionDataContainer.java | 28 +- .../tsfile/TsFileInsertionPointCounter.java | 10 +- .../dataregion/IoTDBDataRegionExtractor.java | 13 + .../IoTDBSchemaRegionExtractor.java | 15 +- .../PipePlanPatternParseVisitor.java | 282 ++++++++++++ .../pattern/CachedSchemaPatternMatcher.java | 10 +- .../thrift/IoTDBDataNodeReceiver.java | 13 +- .../visitor/PipePlanToStatementVisitor.java | 2 +- .../PipeStatementPatternParseVisitor.java | 136 ++++++ .../task/connection/PipeEventCollector.java | 19 + .../plan/node/write/DeleteDataNode.java | 21 + .../CreateAlignedTimeSeriesStatement.java | 55 +++ .../PipeDataNodeThriftRequestTest.java | 2 +- .../PipeStatementPatternParseVisitorTest.java | 161 +++++++ .../IoTDBDataRegionExtractorTest.java | 18 +- .../PipePlanPatternParseVisitorTest.java | 428 ++++++++++++++++++ .../db/pipe/pattern/IoTDBPipePatternTest.java | 46 +- .../pipe/pattern/PrefixPipePatternTest.java | 46 +- .../IoTDBNonDataRegionExtractor.java | 48 +- .../pipe/pattern/IoTDBPipePattern.java | 99 +++- .../commons/pipe/pattern/PipePattern.java | 17 +- .../pipe/pattern/PrefixPipePattern.java | 20 +- 48 files changed, 2487 insertions(+), 316 deletions(-) create mode 100644 iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/payload/PipeSetTTLPlan.java create mode 100644 iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/extractor/PipeConfigPhysicalPlanPatternParseVisitor.java create mode 100644 iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/manager/pipe/extractor/PipeConfigPhysicalPlanPatternParseVisitorTest.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/schemaregion/PipePlanPatternParseVisitor.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/visitor/PipeStatementPatternParseVisitor.java create mode 100644 iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/connector/PipeStatementPatternParseVisitorTest.java create mode 100644 iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/extractor/PipePlanPatternParseVisitorTest.java diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeExtractorIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeExtractorIT.java index 541d628aa7eb..28852463a77d 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeExtractorIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeExtractorIT.java @@ -411,7 +411,7 @@ public void testExtractorPatternMatch() throws Exception { final Map connectorAttributes = new HashMap<>(); extractorAttributes.put("extractor.pattern", null); - extractorAttributes.put("extractor.inclusion", "data"); + extractorAttributes.put("extractor.inclusion", "data.insert"); connectorAttributes.put("connector", "iotdb-thrift-connector"); connectorAttributes.put("connector.batch.enable", "false"); @@ -491,7 +491,7 @@ public void testMatchingMultipleDatabases() throws Exception { final Map connectorAttributes = new HashMap<>(); extractorAttributes.put("extractor.pattern", "root.db1"); - extractorAttributes.put("extractor.inclusion", "data"); + extractorAttributes.put("extractor.inclusion", "data.insert"); connectorAttributes.put("connector", "iotdb-thrift-connector"); connectorAttributes.put("connector.batch.enable", "false"); @@ -589,7 +589,7 @@ public void testHistoryAndRealtime() throws Exception { connectorAttributes.put("connector.ip", receiverIp); connectorAttributes.put("connector.port", Integer.toString(receiverPort)); - extractorAttributes.put("extractor.inclusion", "data"); + extractorAttributes.put("extractor.inclusion", "data.insert"); extractorAttributes.put("extractor.pattern", "root.db.d2"); extractorAttributes.put("extractor.history.enable", "false"); extractorAttributes.put("extractor.realtime.enable", "true"); @@ -675,7 +675,7 @@ public void testHistoryStartTimeAndEndTimeWorkingWithOrWithoutPattern() throws E final Map connectorAttributes = new HashMap<>(); extractorAttributes.put("extractor.pattern", "root.db.d1"); - extractorAttributes.put("extractor.inclusion", "data"); + extractorAttributes.put("extractor.inclusion", "data.insert"); extractorAttributes.put("extractor.history.enable", "true"); // 1970-01-01T08:00:02+08:00 extractorAttributes.put("extractor.history.start-time", "2000"); @@ -825,7 +825,7 @@ public void testSourceStartTimeAndEndTimeWorkingWithOrWithoutPattern() throws Ex final Map connectorAttributes = new HashMap<>(); extractorAttributes.put("source.pattern", "root.db.d1"); - extractorAttributes.put("source.inclusion", "data"); + extractorAttributes.put("source.inclusion", "data.insert"); extractorAttributes.put("source.start-time", "1970-01-01T08:00:02+08:00"); // 1970-01-01T08:00:04+08:00 extractorAttributes.put("source.end-time", "4000"); diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/IoTDBPipeInclusionIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/IoTDBPipeInclusionIT.java index c27fa7f46c4f..7a8ff6c3f156 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/IoTDBPipeInclusionIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/IoTDBPipeInclusionIT.java @@ -36,6 +36,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.Map; @RunWith(IoTDBTestRunner.class) @@ -144,6 +145,54 @@ public void testAuthExclusion() throws Exception { } } + @Test + public void testAuthInclusionWithPattern() throws Exception { + final DataNodeWrapper receiverDataNode = receiverEnv.getDataNodeWrapper(0); + + final String receiverIp = receiverDataNode.getIp(); + final int receiverPort = receiverDataNode.getPort(); + + try (final SyncConfigNodeIServiceClient client = + (SyncConfigNodeIServiceClient) senderEnv.getLeaderConfigNodeConnection()) { + final Map extractorAttributes = new HashMap<>(); + final Map processorAttributes = new HashMap<>(); + final Map connectorAttributes = new HashMap<>(); + + extractorAttributes.put("extractor.inclusion", "auth"); + extractorAttributes.put("path", "root.ln.**"); + + connectorAttributes.put("connector", "iotdb-thrift-connector"); + connectorAttributes.put("connector.ip", receiverIp); + connectorAttributes.put("connector.port", Integer.toString(receiverPort)); + + final TSStatus status = + client.createPipe( + new TCreatePipeReq("testPipe", connectorAttributes) + .setExtractorAttributes(extractorAttributes) + .setProcessorAttributes(processorAttributes)); + + Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode()); + + Assert.assertEquals( + TSStatusCode.SUCCESS_STATUS.getStatusCode(), client.startPipe("testPipe").getCode()); + + if (!TestUtils.tryExecuteNonQueriesWithRetry( + senderEnv, + Arrays.asList( + "create user `ln_write_user` 'write_pwd'", + "GRANT READ_DATA, WRITE_DATA ON root.** TO USER ln_write_user;"))) { + return; + } + + TestUtils.assertDataAlwaysOnEnv( + receiverEnv, + "LIST PRIVILEGES OF USER ln_write_user", + "ROLE,PATH,PRIVILEGES,GRANT OPTION,", + new HashSet<>( + Arrays.asList(",root.ln.**,READ_DATA,false,", ",root.ln.**,WRITE_DATA,false,"))); + } + } + @Test public void testPureDeleteInclusion() throws Exception { final DataNodeWrapper receiverDataNode = receiverEnv.getDataNodeWrapper(0); diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/IoTDBPipeMetaHistoricalIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/IoTDBPipeMetaHistoricalIT.java index 89662c19f633..70316d3b600a 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/IoTDBPipeMetaHistoricalIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/IoTDBPipeMetaHistoricalIT.java @@ -97,6 +97,7 @@ public void testTemplateInclusion() throws Exception { senderEnv, Arrays.asList( "create database root.ln", + "create database root.db", "set ttl to root.ln 3600000", "create user `thulab` 'passwd'", "create role `admin`", @@ -104,7 +105,9 @@ public void testTemplateInclusion() throws Exception { "grant read on root.** to role `admin`", "create schema template t1 (temperature FLOAT encoding=RLE, status BOOLEAN encoding=PLAIN compression=SNAPPY)", "set schema template t1 to root.ln.wf01", + "set schema template t1 to root.db.wf01", "create timeseries using schema template on root.ln.wf01.wt01", + "create timeseries using schema template on root.db.wf01.wt01", "create timeseries root.ln.wf02.wt01.status with datatype=BOOLEAN,encoding=PLAIN", // Insert large timestamp to avoid deletion by ttl "insert into root.ln.wf01.wt01(time, temperature, status) values (1800000000000, 23, true)"))) { @@ -118,6 +121,7 @@ public void testTemplateInclusion() throws Exception { extractorAttributes.put("extractor.inclusion", "data, schema"); extractorAttributes.put( "extractor.inclusion.exclusion", "schema.timeseries.ordinary, schema.ttl"); + extractorAttributes.put("extractor.path", "root.ln.**"); connectorAttributes.put("connector", "iotdb-thrift-connector"); connectorAttributes.put("connector.ip", receiverIp); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlan.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlan.java index 8ad8c9b2fd25..3c93330ae110 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlan.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlan.java @@ -82,6 +82,7 @@ import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeDeleteLogicalViewPlan; import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeDeleteTimeSeriesPlan; import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeEnrichedPlan; +import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeSetTTLPlan; import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeUnsetSchemaTemplatePlan; import org.apache.iotdb.confignode.consensus.request.write.pipe.plugin.CreatePipePluginPlan; import org.apache.iotdb.confignode.consensus.request.write.pipe.plugin.DropPipePluginPlan; @@ -492,6 +493,9 @@ public static ConfigPhysicalPlan create(ByteBuffer buffer) throws IOException { case PipeDeactivateTemplate: plan = new PipeDeactivateTemplatePlan(); break; + case PipeSetTTL: + plan = new PipeSetTTLPlan(); + break; case GetRegionId: plan = new GetRegionIdPlan(); break; diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanType.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanType.java index 031518be9448..1318c2165b46 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanType.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanType.java @@ -213,6 +213,7 @@ public enum ConfigPhysicalPlanType { PipeDeleteTimeSeries((short) 1702), PipeDeleteLogicalView((short) 1703), PipeDeactivateTemplate((short) 1704), + PipeSetTTL((short) 1705), /** Subscription */ CreateTopic((short) 1800), diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/auth/AuthorPlan.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/auth/AuthorPlan.java index 0204d013e065..be162647f8fc 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/auth/AuthorPlan.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/auth/AuthorPlan.java @@ -19,7 +19,6 @@ package org.apache.iotdb.confignode.consensus.request.auth; -import org.apache.iotdb.commons.auth.AuthException; import org.apache.iotdb.commons.auth.entity.PrivilegeType; import org.apache.iotdb.commons.exception.MetadataException; import org.apache.iotdb.commons.path.PartialPath; @@ -51,7 +50,7 @@ public class AuthorPlan extends ConfigPhysicalPlan { private String userName; private boolean grantOpt; - public AuthorPlan(ConfigPhysicalPlanType type) { + public AuthorPlan(final ConfigPhysicalPlanType type) { super(type); authorType = type; } @@ -67,18 +66,16 @@ public AuthorPlan(ConfigPhysicalPlanType type) { * @param permissions permissions * @param grantOpt with grant option, only grant statement can set grantOpt = true * @param nodeNameList node name in Path structure - * @throws AuthException Authentication Exception */ public AuthorPlan( - ConfigPhysicalPlanType authorType, - String userName, - String roleName, - String password, - String newPassword, - Set permissions, - boolean grantOpt, - List nodeNameList) - throws AuthException { + final ConfigPhysicalPlanType authorType, + final String userName, + final String roleName, + final String password, + final String newPassword, + final Set permissions, + final boolean grantOpt, + final List nodeNameList) { this(authorType); this.authorType = authorType; this.userName = userName; diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/payload/PipeDeleteLogicalViewPlan.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/payload/PipeDeleteLogicalViewPlan.java index 0db5952175a8..6865b135c7b2 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/payload/PipeDeleteLogicalViewPlan.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/payload/PipeDeleteLogicalViewPlan.java @@ -38,9 +38,8 @@ public PipeDeleteLogicalViewPlan() { super(ConfigPhysicalPlanType.PipeDeleteLogicalView); } - public PipeDeleteLogicalViewPlan(ByteBuffer patternTreeBytes) { + public PipeDeleteLogicalViewPlan(final ByteBuffer patternTreeBytes) { super(ConfigPhysicalPlanType.PipeDeleteLogicalView); - patternTreeBytes.flip(); this.patternTreeBytes = patternTreeBytes; } @@ -49,25 +48,25 @@ public ByteBuffer getPatternTreeBytes() { } @Override - protected void serializeImpl(DataOutputStream stream) throws IOException { + protected void serializeImpl(final DataOutputStream stream) throws IOException { stream.writeShort(getType().getPlanType()); ReadWriteIOUtils.write(patternTreeBytes, stream); } @Override - protected void deserializeImpl(ByteBuffer buffer) throws IOException { + protected void deserializeImpl(final ByteBuffer buffer) throws IOException { patternTreeBytes = ByteBuffer.wrap(ReadWriteIOUtils.readBinary(buffer).getValues()); } @Override - public boolean equals(Object obj) { + public boolean equals(final Object obj) { if (this == obj) { return true; } if (obj == null || getClass() != obj.getClass()) { return false; } - PipeDeleteLogicalViewPlan that = (PipeDeleteLogicalViewPlan) obj; + final PipeDeleteLogicalViewPlan that = (PipeDeleteLogicalViewPlan) obj; return Arrays.equals(patternTreeBytes.array(), that.patternTreeBytes.array()); } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/payload/PipeDeleteTimeSeriesPlan.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/payload/PipeDeleteTimeSeriesPlan.java index d97346f01550..2b651cb034d7 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/payload/PipeDeleteTimeSeriesPlan.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/payload/PipeDeleteTimeSeriesPlan.java @@ -38,9 +38,8 @@ public PipeDeleteTimeSeriesPlan() { super(ConfigPhysicalPlanType.PipeDeleteTimeSeries); } - public PipeDeleteTimeSeriesPlan(ByteBuffer patternTreeBytes) { + public PipeDeleteTimeSeriesPlan(final ByteBuffer patternTreeBytes) { super(ConfigPhysicalPlanType.PipeDeleteTimeSeries); - patternTreeBytes.flip(); this.patternTreeBytes = patternTreeBytes; } @@ -49,25 +48,25 @@ public ByteBuffer getPatternTreeBytes() { } @Override - protected void serializeImpl(DataOutputStream stream) throws IOException { + protected void serializeImpl(final DataOutputStream stream) throws IOException { stream.writeShort(getType().getPlanType()); ReadWriteIOUtils.write(patternTreeBytes, stream); } @Override - protected void deserializeImpl(ByteBuffer buffer) throws IOException { + protected void deserializeImpl(final ByteBuffer buffer) throws IOException { patternTreeBytes = ByteBuffer.wrap(ReadWriteIOUtils.readBinary(buffer).getValues()); } @Override - public boolean equals(Object obj) { + public boolean equals(final Object obj) { if (this == obj) { return true; } if (obj == null || getClass() != obj.getClass()) { return false; } - PipeDeleteTimeSeriesPlan that = (PipeDeleteTimeSeriesPlan) obj; + final PipeDeleteTimeSeriesPlan that = (PipeDeleteTimeSeriesPlan) obj; return Arrays.equals(patternTreeBytes.array(), that.patternTreeBytes.array()); } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/payload/PipeSetTTLPlan.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/payload/PipeSetTTLPlan.java new file mode 100644 index 000000000000..67ab85d898cb --- /dev/null +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/payload/PipeSetTTLPlan.java @@ -0,0 +1,103 @@ +/* + * 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.confignode.consensus.request.write.pipe.payload; + +import org.apache.iotdb.commons.pipe.extractor.IoTDBNonDataRegionExtractor; +import org.apache.iotdb.commons.pipe.pattern.IoTDBPipePattern; +import org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlan; +import org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlanType; +import org.apache.iotdb.confignode.consensus.request.write.database.SetTTLPlan; +import org.apache.iotdb.confignode.manager.pipe.event.PipeConfigRegionWritePlanEvent; +import org.apache.iotdb.confignode.manager.pipe.extractor.ConfigRegionListeningQueue; +import org.apache.iotdb.confignode.manager.pipe.receiver.protocol.IoTDBConfigNodeReceiver; + +import org.apache.tsfile.utils.ReadWriteIOUtils; + +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.List; +import java.util.Objects; + +/** + * This is a set of {@link SetTTLPlan}s generated by an {@link IoTDBPipePattern} and a normal {@link + * SetTTLPlan}. This is used to persist the "at most one" match between {@link + * PipeConfigRegionWritePlanEvent} in the {@link ConfigRegionListeningQueue} and the {@link + * IoTDBNonDataRegionExtractor#supply()} and will be executed serially in {@link + * IoTDBConfigNodeReceiver} + */ +public class PipeSetTTLPlan extends ConfigPhysicalPlan { + private List planList; + + public PipeSetTTLPlan() { + super(ConfigPhysicalPlanType.PipeSetTTL); + } + + public PipeSetTTLPlan(final List planList) { + super(ConfigPhysicalPlanType.PipeSetTTL); + this.planList = planList; + } + + public List getSetTTLPlans() { + return planList; + } + + @Override + protected void serializeImpl(final DataOutputStream stream) throws IOException { + stream.writeShort(getType().getPlanType()); + ReadWriteIOUtils.write(planList.size(), stream); + for (final SetTTLPlan setTTLPlan : planList) { + final ByteBuffer buffer = setTTLPlan.serializeToByteBuffer(); + stream.write(buffer.array(), 0, buffer.limit()); + } + } + + @Override + protected void deserializeImpl(final ByteBuffer buffer) throws IOException { + final int size = ReadWriteIOUtils.readInt(buffer); + planList = new ArrayList<>(); + for (int i = 0; i < size; ++i) { + planList.add((SetTTLPlan) ConfigPhysicalPlan.Factory.create(buffer)); + } + } + + @Override + public boolean equals(final Object obj) { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + final PipeSetTTLPlan that = (PipeSetTTLPlan) obj; + return Objects.equals(planList, that.planList); + } + + @Override + public int hashCode() { + return Objects.hash(planList); + } + + @Override + public String toString() { + return "PipeSetTTLPlan{" + "planList='" + planList + "'}"; + } +} diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/connector/payload/PipeTransferConfigSnapshotSealReq.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/connector/payload/PipeTransferConfigSnapshotSealReq.java index 0397cff3a946..07664f6523cc 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/connector/payload/PipeTransferConfigSnapshotSealReq.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/connector/payload/PipeTransferConfigSnapshotSealReq.java @@ -48,16 +48,19 @@ protected PipeRequestType getPlanType() { /////////////////////////////// Thrift /////////////////////////////// public static PipeTransferConfigSnapshotSealReq toTPipeTransferReq( - String snapshotName, - long snapshotLength, - String templateFileName, - long templateFileLength, - CNSnapshotFileType fileType, - String typeString) + final String pattern, + final String snapshotName, + final long snapshotLength, + final String templateFileName, + final long templateFileLength, + final CNSnapshotFileType fileType, + final String typeString) throws IOException { final Map parameters = new HashMap<>(); + parameters.put(ColumnHeaderConstant.PATH_PATTERN, pattern); parameters.put(FILE_TYPE, Byte.toString(fileType.getType())); parameters.put(ColumnHeaderConstant.TYPE, typeString); + return (PipeTransferConfigSnapshotSealReq) new PipeTransferConfigSnapshotSealReq() .convertToTPipeTransferReq( @@ -70,7 +73,7 @@ public static PipeTransferConfigSnapshotSealReq toTPipeTransferReq( parameters); } - public static PipeTransferConfigSnapshotSealReq fromTPipeTransferReq(TPipeTransferReq req) { + public static PipeTransferConfigSnapshotSealReq fromTPipeTransferReq(final TPipeTransferReq req) { return (PipeTransferConfigSnapshotSealReq) new PipeTransferConfigSnapshotSealReq().translateFromTPipeTransferReq(req); } @@ -78,14 +81,16 @@ public static PipeTransferConfigSnapshotSealReq fromTPipeTransferReq(TPipeTransf /////////////////////////////// Air Gap /////////////////////////////// public static byte[] toTPipeTransferBytes( - String snapshotName, - long snapshotLength, - String templateFileName, - long templateFileLength, - CNSnapshotFileType fileType, - String typeString) + final String pattern, + final String snapshotName, + final long snapshotLength, + final String templateFileName, + final long templateFileLength, + final CNSnapshotFileType fileType, + final String typeString) throws IOException { final Map parameters = new HashMap<>(); + parameters.put(ColumnHeaderConstant.PATH_PATTERN, pattern); parameters.put(FILE_TYPE, Byte.toString(fileType.getType())); parameters.put(ColumnHeaderConstant.TYPE, typeString); return new PipeTransferConfigSnapshotSealReq() @@ -102,7 +107,7 @@ public static byte[] toTPipeTransferBytes( /////////////////////////////// Object /////////////////////////////// @Override - public boolean equals(Object obj) { + public boolean equals(final Object obj) { return obj instanceof PipeTransferConfigSnapshotSealReq && super.equals(obj); } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/connector/protocol/IoTDBConfigRegionAirGapConnector.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/connector/protocol/IoTDBConfigRegionAirGapConnector.java index 3ca8031d2681..ec09b546648b 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/connector/protocol/IoTDBConfigRegionAirGapConnector.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/connector/protocol/IoTDBConfigRegionAirGapConnector.java @@ -201,6 +201,8 @@ private void doTransfer( socket, compressIfNeeded( PipeTransferConfigSnapshotSealReq.toTPipeTransferBytes( + // The pattern is surely Non-null + pipeConfigRegionSnapshotEvent.getPatternString(), snapshot.getName(), snapshot.length(), Objects.nonNull(templateFile) ? templateFile.getName() : null, 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 a2962c194f33..b199df4c390c 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 @@ -195,6 +195,8 @@ private void doTransfer(final PipeConfigRegionSnapshotEvent snapshotEvent) .pipeTransfer( compressIfNeeded( PipeTransferConfigSnapshotSealReq.toTPipeTransferReq( + // The pattern is surely Non-null + snapshotEvent.getPatternString(), snapshotFile.getName(), snapshotFile.length(), Objects.nonNull(templateFile) ? templateFile.getName() : null, diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/extractor/IoTDBConfigRegionExtractor.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/extractor/IoTDBConfigRegionExtractor.java index a5eb544c95ae..ea897465b8b7 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/extractor/IoTDBConfigRegionExtractor.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/extractor/IoTDBConfigRegionExtractor.java @@ -24,6 +24,7 @@ import org.apache.iotdb.commons.pipe.datastructure.queue.listening.AbstractPipeListeningQueue; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.commons.pipe.event.PipeSnapshotEvent; +import org.apache.iotdb.commons.pipe.event.PipeWritePlanEvent; import org.apache.iotdb.commons.pipe.extractor.IoTDBNonDataRegionExtractor; import org.apache.iotdb.commons.pipe.progress.PipeEventCommitManager; import org.apache.iotdb.confignode.conf.ConfigNodeDescriptor; @@ -38,15 +39,18 @@ import org.apache.iotdb.consensus.exception.ConsensusException; import org.apache.iotdb.pipe.api.customizer.configuration.PipeExtractorRuntimeConfiguration; import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameters; -import org.apache.iotdb.pipe.api.event.Event; import org.apache.iotdb.pipe.api.exception.PipeException; import java.util.HashSet; import java.util.Objects; +import java.util.Optional; import java.util.Set; public class IoTDBConfigRegionExtractor extends IoTDBNonDataRegionExtractor { + public static final PipeConfigPhysicalPlanPatternParseVisitor PATTERN_PARSE_VISITOR = + new PipeConfigPhysicalPlanPatternParseVisitor(); + private Set listenedTypeSet = new HashSet<>(); @Override @@ -110,7 +114,17 @@ protected long getMaxBlockingTimeMs() { } @Override - protected boolean isTypeListened(final Event event) { + protected Optional trimRealtimeEventByPipePattern( + final PipeWritePlanEvent event) { + return PATTERN_PARSE_VISITOR + .process(((PipeConfigRegionWritePlanEvent) event).getConfigPhysicalPlan(), pipePattern) + .map( + configPhysicalPlan -> + new PipeConfigRegionWritePlanEvent(configPhysicalPlan, event.isGeneratedByPipe())); + } + + @Override + protected boolean isTypeListened(final PipeWritePlanEvent event) { return listenedTypeSet.contains( ((PipeConfigRegionWritePlanEvent) event).getConfigPhysicalPlan().getType()); } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/extractor/PipeConfigPhysicalPlanPatternParseVisitor.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/extractor/PipeConfigPhysicalPlanPatternParseVisitor.java new file mode 100644 index 000000000000..e5ea0cccd3b3 --- /dev/null +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/extractor/PipeConfigPhysicalPlanPatternParseVisitor.java @@ -0,0 +1,283 @@ +/* + * 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.confignode.manager.pipe.extractor; + +import org.apache.iotdb.commons.path.PartialPath; +import org.apache.iotdb.commons.path.PathPatternTree; +import org.apache.iotdb.commons.pipe.pattern.IoTDBPipePattern; +import org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlan; +import org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlanVisitor; +import org.apache.iotdb.confignode.consensus.request.auth.AuthorPlan; +import org.apache.iotdb.confignode.consensus.request.write.database.DatabaseSchemaPlan; +import org.apache.iotdb.confignode.consensus.request.write.database.DeleteDatabasePlan; +import org.apache.iotdb.confignode.consensus.request.write.database.SetTTLPlan; +import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeDeactivateTemplatePlan; +import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeDeleteLogicalViewPlan; +import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeDeleteTimeSeriesPlan; +import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeSetTTLPlan; +import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeUnsetSchemaTemplatePlan; +import org.apache.iotdb.confignode.consensus.request.write.template.CommitSetSchemaTemplatePlan; +import org.apache.iotdb.confignode.consensus.request.write.template.CreateSchemaTemplatePlan; +import org.apache.iotdb.confignode.consensus.request.write.template.ExtendSchemaTemplatePlan; +import org.apache.iotdb.confignode.manager.pipe.event.PipeConfigRegionWritePlanEvent; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNode; +import org.apache.iotdb.db.schemaengine.template.Template; +import org.apache.iotdb.db.schemaengine.template.alter.TemplateExtendInfo; + +import org.apache.tsfile.utils.Pair; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import java.util.stream.Stream; + +/** + * The {@link PipeConfigPhysicalPlanPatternParseVisitor} will transform the schema {@link + * ConfigPhysicalPlan}s using {@link IoTDBPipePattern}. Rule: + * + *

1. All patterns in the output {@link ConfigPhysicalPlan} will be the intersection of the + * original {@link ConfigPhysicalPlan}'s patterns and the given {@link IoTDBPipePattern}. + * + *

2. If a pattern does not intersect with the {@link IoTDBPipePattern}, it's dropped. + * + *

3. If all the patterns in the {@link ConfigPhysicalPlan} is dropped, the {@link + * ConfigPhysicalPlan} is dropped. + * + *

4. The output {@link PlanNode} shall be a copied form of the original one because the original + * one is used in the {@link PipeConfigRegionWritePlanEvent} in {@link ConfigRegionListeningQueue}. + */ +public class PipeConfigPhysicalPlanPatternParseVisitor + extends ConfigPhysicalPlanVisitor, IoTDBPipePattern> { + private static final Logger LOGGER = + LoggerFactory.getLogger(PipeConfigPhysicalPlanPatternParseVisitor.class); + + @Override + public Optional visitPlan( + final ConfigPhysicalPlan plan, final IoTDBPipePattern pattern) { + return Optional.of(plan); + } + + // Note: This will return true even if the pattern matches the database precisely, + // like database is "root.db.a" and pattern is "root.db.a". In this case, none of + // the data and time series under this database will be transferred, however we + // interpret user's pattern as "send precisely the database" and transfer it + // nonetheless. + // Other matches using "matchPrefixPath" are with the same principle. + @Override + public Optional visitCreateDatabase( + final DatabaseSchemaPlan createDatabasePlan, final IoTDBPipePattern pattern) { + return pattern.matchPrefixPath(createDatabasePlan.getSchema().getName()) + ? Optional.of(createDatabasePlan) + : Optional.empty(); + } + + @Override + public Optional visitAlterDatabase( + final DatabaseSchemaPlan alterDatabasePlan, final IoTDBPipePattern pattern) { + return pattern.matchPrefixPath(alterDatabasePlan.getSchema().getName()) + ? Optional.of(alterDatabasePlan) + : Optional.empty(); + } + + @Override + public Optional visitDeleteDatabase( + final DeleteDatabasePlan deleteDatabasePlan, final IoTDBPipePattern pattern) { + return pattern.matchPrefixPath(deleteDatabasePlan.getName()) + ? Optional.of(deleteDatabasePlan) + : Optional.empty(); + } + + @Override + public Optional visitCreateSchemaTemplate( + final CreateSchemaTemplatePlan createSchemaTemplatePlan, final IoTDBPipePattern pattern) { + // This is a deserialized template and can be arbitrarily altered + final Template template = createSchemaTemplatePlan.getTemplate(); + template.getSchemaMap().keySet().removeIf(measurement -> !pattern.matchTailNode(measurement)); + return !template.getSchemaMap().isEmpty() + ? Optional.of(new CreateSchemaTemplatePlan(template.serialize().array())) + : Optional.empty(); + } + + @Override + public Optional visitCommitSetSchemaTemplate( + final CommitSetSchemaTemplatePlan commitSetSchemaTemplatePlan, + final IoTDBPipePattern pattern) { + return pattern.matchPrefixPath(commitSetSchemaTemplatePlan.getPath()) + ? Optional.of(commitSetSchemaTemplatePlan) + : Optional.empty(); + } + + @Override + public Optional visitPipeUnsetSchemaTemplate( + final PipeUnsetSchemaTemplatePlan pipeUnsetSchemaTemplatePlan, + final IoTDBPipePattern pattern) { + return pattern.matchPrefixPath(pipeUnsetSchemaTemplatePlan.getPath()) + ? Optional.of(pipeUnsetSchemaTemplatePlan) + : Optional.empty(); + } + + @Override + public Optional visitExtendSchemaTemplate( + final ExtendSchemaTemplatePlan extendSchemaTemplatePlan, final IoTDBPipePattern pattern) { + final TemplateExtendInfo extendInfo = extendSchemaTemplatePlan.getTemplateExtendInfo(); + final int[] filteredIndexes = + IntStream.range(0, extendInfo.getMeasurements().size()) + .filter(index -> pattern.matchTailNode(extendInfo.getMeasurements().get(index))) + .toArray(); + return filteredIndexes.length > 0 + ? Optional.of( + new ExtendSchemaTemplatePlan( + new TemplateExtendInfo( + extendInfo.getTemplateName(), + IoTDBPipePattern.applyIndexesOnList( + filteredIndexes, extendInfo.getMeasurements()), + IoTDBPipePattern.applyIndexesOnList(filteredIndexes, extendInfo.getDataTypes()), + IoTDBPipePattern.applyIndexesOnList(filteredIndexes, extendInfo.getEncodings()), + IoTDBPipePattern.applyIndexesOnList( + filteredIndexes, extendInfo.getCompressors())))) + : Optional.empty(); + } + + @Override + public Optional visitGrantUser( + final AuthorPlan grantUserPlan, final IoTDBPipePattern pattern) { + return visitPathRelatedAuthorPlan(grantUserPlan, pattern); + } + + @Override + public Optional visitRevokeUser( + final AuthorPlan revokeUserPlan, final IoTDBPipePattern pattern) { + return visitPathRelatedAuthorPlan(revokeUserPlan, pattern); + } + + @Override + public Optional visitGrantRole( + final AuthorPlan revokeUserPlan, final IoTDBPipePattern pattern) { + return visitPathRelatedAuthorPlan(revokeUserPlan, pattern); + } + + @Override + public Optional visitRevokeRole( + final AuthorPlan revokeUserPlan, final IoTDBPipePattern pattern) { + return visitPathRelatedAuthorPlan(revokeUserPlan, pattern); + } + + private Optional visitPathRelatedAuthorPlan( + final AuthorPlan pathRelatedAuthorPlan, final IoTDBPipePattern pattern) { + final List intersectedPaths = + pathRelatedAuthorPlan.getNodeNameList().stream() + .map(pattern::getIntersection) + .flatMap(Collection::stream) + .collect(Collectors.toList()); + return !intersectedPaths.isEmpty() + ? Optional.of( + new AuthorPlan( + pathRelatedAuthorPlan.getAuthorType(), + pathRelatedAuthorPlan.getUserName(), + pathRelatedAuthorPlan.getRoleName(), + pathRelatedAuthorPlan.getPassword(), + pathRelatedAuthorPlan.getNewPassword(), + pathRelatedAuthorPlan.getPermissions(), + pathRelatedAuthorPlan.getGrantOpt(), + intersectedPaths)) + : Optional.empty(); + } + + @Override + public Optional visitPipeDeleteTimeSeries( + final PipeDeleteTimeSeriesPlan pipeDeleteTimeSeriesPlan, final IoTDBPipePattern pattern) { + try { + final PathPatternTree intersectedTree = + pattern.getIntersection( + PathPatternTree.deserialize(pipeDeleteTimeSeriesPlan.getPatternTreeBytes())); + return !intersectedTree.isEmpty() + ? Optional.of(new PipeDeleteTimeSeriesPlan(intersectedTree.serialize())) + : Optional.empty(); + } catch (final IOException e) { + LOGGER.warn( + "Serialization failed for the delete time series plan in pipe transmission, skip transfer", + e); + return Optional.empty(); + } + } + + @Override + public Optional visitPipeDeleteLogicalView( + final PipeDeleteLogicalViewPlan pipeDeleteLogicalViewPlan, final IoTDBPipePattern pattern) { + try { + final PathPatternTree intersectedTree = + pattern.getIntersection( + PathPatternTree.deserialize(pipeDeleteLogicalViewPlan.getPatternTreeBytes())); + return !intersectedTree.isEmpty() + ? Optional.of(new PipeDeleteTimeSeriesPlan(intersectedTree.serialize())) + : Optional.empty(); + } catch (final IOException e) { + LOGGER.warn( + "Serialization failed for the delete logical view plan in pipe transmission, skip transfer", + e); + return Optional.empty(); + } + } + + @Override + public Optional visitPipeDeactivateTemplate( + final PipeDeactivateTemplatePlan pipeDeactivateTemplatePlan, final IoTDBPipePattern pattern) { + final Map> newTemplateSetInfo = + pipeDeactivateTemplatePlan.getTemplateSetInfo().entrySet().stream() + .flatMap( + entry -> + pattern.getIntersection(entry.getKey()).stream() + .map(partialPath -> new Pair<>(partialPath, entry.getValue()))) + .collect( + Collectors.toMap( + Pair::getLeft, + Pair::getRight, + (oldTemplates, newTemplates) -> + Stream.of(oldTemplates, newTemplates) + .flatMap(Collection::stream) + .distinct() + .collect(Collectors.toList()))); + return !newTemplateSetInfo.isEmpty() + ? Optional.of(new PipeDeactivateTemplatePlan(newTemplateSetInfo)) + : Optional.empty(); + } + + @Override + public Optional visitTTL( + final SetTTLPlan setTTLPlan, final IoTDBPipePattern pattern) { + final List intersectionList = + pattern.getIntersection(new PartialPath(setTTLPlan.getDatabasePathPattern())); + return !intersectionList.isEmpty() + ? Optional.of( + new PipeSetTTLPlan( + intersectionList.stream() + .map( + path -> new SetTTLPlan(Arrays.asList(path.getNodes()), setTTLPlan.getTTL())) + .collect(Collectors.toList()))) + : Optional.empty(); + } +} diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/receiver/protocol/IoTDBConfigNodeReceiver.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/receiver/protocol/IoTDBConfigNodeReceiver.java index 0a78553516a9..513c910ce114 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/receiver/protocol/IoTDBConfigNodeReceiver.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/receiver/protocol/IoTDBConfigNodeReceiver.java @@ -27,6 +27,7 @@ import org.apache.iotdb.commons.pipe.connector.payload.thrift.request.PipeTransferCompressedReq; import org.apache.iotdb.commons.pipe.connector.payload.thrift.request.PipeTransferFileSealReqV1; import org.apache.iotdb.commons.pipe.connector.payload.thrift.request.PipeTransferFileSealReqV2; +import org.apache.iotdb.commons.pipe.pattern.IoTDBPipePattern; import org.apache.iotdb.commons.pipe.receiver.IoTDBFileReceiver; import org.apache.iotdb.confignode.conf.ConfigNodeDescriptor; import org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlan; @@ -39,6 +40,7 @@ import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeDeleteLogicalViewPlan; import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeDeleteTimeSeriesPlan; import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeEnrichedPlan; +import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeSetTTLPlan; import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeUnsetSchemaTemplatePlan; import org.apache.iotdb.confignode.consensus.request.write.template.CommitSetSchemaTemplatePlan; import org.apache.iotdb.confignode.consensus.request.write.template.ExtendSchemaTemplatePlan; @@ -50,6 +52,7 @@ import org.apache.iotdb.confignode.manager.pipe.connector.payload.PipeTransferConfigSnapshotPieceReq; import org.apache.iotdb.confignode.manager.pipe.connector.payload.PipeTransferConfigSnapshotSealReq; import org.apache.iotdb.confignode.manager.pipe.event.PipeConfigRegionSnapshotEvent; +import org.apache.iotdb.confignode.manager.pipe.extractor.IoTDBConfigRegionExtractor; import org.apache.iotdb.confignode.manager.pipe.receiver.visitor.PipeConfigPhysicalPlanExceptionVisitor; import org.apache.iotdb.confignode.manager.pipe.receiver.visitor.PipeConfigPhysicalPlanTSStatusVisitor; import org.apache.iotdb.confignode.persistence.schema.CNPhysicalPlanGenerator; @@ -82,6 +85,7 @@ import java.util.Objects; import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; public class IoTDBConfigNodeReceiver extends IoTDBFileReceiver { @@ -143,7 +147,7 @@ public TPipeTransferResp receive(final TPipeTransferReq req) { receiverId.get(), status); return new TPipeTransferResp(status); - } catch (Exception e) { + } catch (final Exception e) { final String error = "Exception encountered while handling pipe transfer request. Root cause: " + e.getMessage(); @@ -157,19 +161,19 @@ public TPipeTransferResp receive(final TPipeTransferReq req) { // to notify its configurations. // Note that the sender needs not to reconstruct its client because the client // is directly linked to the preceding DataNode and has not broken. - private boolean needHandshake(PipeRequestType type) { + private boolean needHandshake(final PipeRequestType type) { return Objects.isNull(receiverFileDirWithIdSuffix.get()) && type != PipeRequestType.HANDSHAKE_CONFIGNODE_V1 && type != PipeRequestType.HANDSHAKE_CONFIGNODE_V2; } - private TPipeTransferResp handleTransferConfigPlan(PipeTransferConfigPlanReq req) + private TPipeTransferResp handleTransferConfigPlan(final PipeTransferConfigPlanReq req) throws IOException { return new TPipeTransferResp( executePlanAndClassifyExceptions(ConfigPhysicalPlan.Factory.create(req.body))); } - private TSStatus executePlanAndClassifyExceptions(ConfigPhysicalPlan plan) { + private TSStatus executePlanAndClassifyExceptions(final ConfigPhysicalPlan plan) { TSStatus result; try { result = executePlan(plan); @@ -181,7 +185,7 @@ private TSStatus executePlanAndClassifyExceptions(ConfigPhysicalPlan plan) { result); result = STATUS_VISITOR.process(plan, result); } - } catch (Exception e) { + } catch (final Exception e) { LOGGER.warn( "Receiver id = {}: Exception encountered while executing plan {}: ", receiverId.get(), @@ -192,7 +196,7 @@ private TSStatus executePlanAndClassifyExceptions(ConfigPhysicalPlan plan) { return result; } - private TSStatus executePlan(ConfigPhysicalPlan plan) throws ConsensusException { + private TSStatus executePlan(final ConfigPhysicalPlan plan) throws ConsensusException { switch (plan.getType()) { case CreateDatabase: // Here we only reserve database name and substitute the sender's local information @@ -269,6 +273,13 @@ private TSStatus executePlan(ConfigPhysicalPlan plan) throws ConsensusException .setIsGeneratedByPipe(true)); case SetTTL: return configManager.getClusterSchemaManager().setTTL((SetTTLPlan) plan, true); + case PipeSetTTL: + // The prior status won't be altered by the status visitor + return PipeReceiverStatusHandler.getPriorStatus( + ((PipeSetTTLPlan) plan) + .getSetTTLPlans().stream() + .map(this::executePlanAndClassifyExceptions) + .collect(Collectors.toList())); case DropUser: case DropRole: case GrantRole: @@ -304,13 +315,15 @@ protected String getReceiverFileBaseDir() { } @Override - protected TSStatus loadFileV1(PipeTransferFileSealReqV1 req, String fileAbsolutePath) { + protected TSStatus loadFileV1( + final PipeTransferFileSealReqV1 req, final String fileAbsolutePath) { throw new UnsupportedOperationException( "IoTDBConfigNodeReceiver does not support load file V1."); } @Override - protected TSStatus loadFileV2(PipeTransferFileSealReqV2 req, List fileAbsolutePaths) + protected TSStatus loadFileV2( + final PipeTransferFileSealReqV2 req, final List fileAbsolutePaths) throws IOException { final Map parameters = req.getParameters(); final CNPhysicalPlanGenerator generator = @@ -326,13 +339,16 @@ protected TSStatus loadFileV2(PipeTransferFileSealReqV2 req, List fileAb final Set executionTypes = PipeConfigRegionSnapshotEvent.getConfigPhysicalPlanTypeSet( parameters.get(ColumnHeaderConstant.TYPE)); + final IoTDBPipePattern pattern = + new IoTDBPipePattern(parameters.get(ColumnHeaderConstant.PATH_PATTERN)); final List results = new ArrayList<>(); while (generator.hasNext()) { - final ConfigPhysicalPlan plan = generator.next(); - if (executionTypes.contains(plan.getType())) { - // Here we apply the statements as many as possible - results.add(executePlanAndClassifyExceptions(plan)); - } + IoTDBConfigRegionExtractor.PATTERN_PARSE_VISITOR + .process(generator.next(), pattern) + .filter(configPhysicalPlan -> executionTypes.contains(configPhysicalPlan.getType())) + .ifPresent( + configPhysicalPlan -> + results.add(executePlanAndClassifyExceptions(configPhysicalPlan))); } return PipeReceiverStatusHandler.getPriorStatus(results); } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/receiver/visitor/PipeConfigPhysicalPlanTSStatusVisitor.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/receiver/visitor/PipeConfigPhysicalPlanTSStatusVisitor.java index 89a9fd4d471e..f2db9d8c3ee2 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/receiver/visitor/PipeConfigPhysicalPlanTSStatusVisitor.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/receiver/visitor/PipeConfigPhysicalPlanTSStatusVisitor.java @@ -45,12 +45,12 @@ public class PipeConfigPhysicalPlanTSStatusVisitor extends ConfigPhysicalPlanVisitor { @Override - public TSStatus visitPlan(ConfigPhysicalPlan plan, TSStatus context) { + public TSStatus visitPlan(final ConfigPhysicalPlan plan, final TSStatus context) { return context; } @Override - public TSStatus visitCreateDatabase(DatabaseSchemaPlan plan, TSStatus context) { + public TSStatus visitCreateDatabase(final DatabaseSchemaPlan plan, final TSStatus context) { if (context.getCode() == TSStatusCode.DATABASE_ALREADY_EXISTS.getStatusCode()) { if (context .getMessage() @@ -77,7 +77,7 @@ public TSStatus visitCreateDatabase(DatabaseSchemaPlan plan, TSStatus context) { } @Override - public TSStatus visitAlterDatabase(DatabaseSchemaPlan plan, TSStatus context) { + public TSStatus visitAlterDatabase(final DatabaseSchemaPlan plan, final TSStatus context) { if (context.getCode() == TSStatusCode.DATABASE_CONFIG_ERROR.getStatusCode()) { return new TSStatus(TSStatusCode.PIPE_RECEIVER_IDEMPOTENT_CONFLICT_EXCEPTION.getStatusCode()) .setMessage(context.getMessage()); @@ -89,7 +89,7 @@ public TSStatus visitAlterDatabase(DatabaseSchemaPlan plan, TSStatus context) { } @Override - public TSStatus visitDeleteDatabase(DeleteDatabasePlan plan, TSStatus context) { + public TSStatus visitDeleteDatabase(final DeleteDatabasePlan plan, final TSStatus context) { if (context.getCode() == TSStatusCode.PATH_NOT_EXIST.getStatusCode()) { return new TSStatus(TSStatusCode.PIPE_RECEIVER_IDEMPOTENT_CONFLICT_EXCEPTION.getStatusCode()) .setMessage(context.getMessage()); @@ -98,7 +98,8 @@ public TSStatus visitDeleteDatabase(DeleteDatabasePlan plan, TSStatus context) { } @Override - public TSStatus visitCreateSchemaTemplate(CreateSchemaTemplatePlan plan, TSStatus context) { + public TSStatus visitCreateSchemaTemplate( + final CreateSchemaTemplatePlan plan, final TSStatus context) { if (context.getCode() == TSStatusCode.METADATA_ERROR.getStatusCode()) { return new TSStatus(TSStatusCode.PIPE_RECEIVER_IDEMPOTENT_CONFLICT_EXCEPTION.getStatusCode()) .setMessage(context.getMessage()); @@ -107,7 +108,8 @@ public TSStatus visitCreateSchemaTemplate(CreateSchemaTemplatePlan plan, TSStatu } @Override - public TSStatus visitExtendSchemaTemplate(ExtendSchemaTemplatePlan plan, TSStatus context) { + public TSStatus visitExtendSchemaTemplate( + final ExtendSchemaTemplatePlan plan, final TSStatus context) { if (context.getCode() == TSStatusCode.MEASUREMENT_ALREADY_EXISTS_IN_TEMPLATE.getStatusCode()) { return new TSStatus(TSStatusCode.PIPE_RECEIVER_IDEMPOTENT_CONFLICT_EXCEPTION.getStatusCode()) .setMessage(context.getMessage()); @@ -119,7 +121,8 @@ public TSStatus visitExtendSchemaTemplate(ExtendSchemaTemplatePlan plan, TSStatu } @Override - public TSStatus visitCommitSetSchemaTemplate(CommitSetSchemaTemplatePlan plan, TSStatus context) { + public TSStatus visitCommitSetSchemaTemplate( + final CommitSetSchemaTemplatePlan plan, final TSStatus context) { if (context.getCode() == TSStatusCode.METADATA_ERROR.getStatusCode()) { if (context.getMessage().contains("Template already exists")) { return new TSStatus( @@ -138,7 +141,7 @@ public TSStatus visitCommitSetSchemaTemplate(CommitSetSchemaTemplatePlan plan, T @Override public TSStatus visitPipeUnsetSchemaTemplate( - PipeUnsetSchemaTemplatePlan pipeUnsetSchemaTemplatePlan, TSStatus context) { + final PipeUnsetSchemaTemplatePlan pipeUnsetSchemaTemplatePlan, final TSStatus context) { if (context.getCode() == TSStatusCode.TEMPLATE_NOT_SET.getStatusCode()) { return new TSStatus(TSStatusCode.PIPE_RECEIVER_IDEMPOTENT_CONFLICT_EXCEPTION.getStatusCode()) .setMessage(context.getMessage()); @@ -152,7 +155,7 @@ public TSStatus visitPipeUnsetSchemaTemplate( @Override public TSStatus visitDropSchemaTemplate( - DropSchemaTemplatePlan dropSchemaTemplatePlan, TSStatus context) { + final DropSchemaTemplatePlan dropSchemaTemplatePlan, final TSStatus context) { if (context.getCode() == TSStatusCode.UNDEFINED_TEMPLATE.getStatusCode()) { return new TSStatus(TSStatusCode.PIPE_RECEIVER_IDEMPOTENT_CONFLICT_EXCEPTION.getStatusCode()) .setMessage(context.getMessage()); @@ -162,7 +165,7 @@ public TSStatus visitDropSchemaTemplate( @Override public TSStatus visitPipeDeleteTimeSeries( - PipeDeleteTimeSeriesPlan pipeDeleteTimeSeriesPlan, TSStatus context) { + final PipeDeleteTimeSeriesPlan pipeDeleteTimeSeriesPlan, final TSStatus context) { if (context.getCode() == TSStatusCode.PATH_NOT_EXIST.getStatusCode()) { return new TSStatus(TSStatusCode.PIPE_RECEIVER_IDEMPOTENT_CONFLICT_EXCEPTION.getStatusCode()) .setMessage(context.getMessage()); @@ -172,7 +175,7 @@ public TSStatus visitPipeDeleteTimeSeries( @Override public TSStatus visitPipeDeleteLogicalView( - PipeDeleteLogicalViewPlan pipeDeleteLogicalViewPlan, TSStatus context) { + final PipeDeleteLogicalViewPlan pipeDeleteLogicalViewPlan, final TSStatus context) { if (context.getCode() == TSStatusCode.PATH_NOT_EXIST.getStatusCode()) { return new TSStatus(TSStatusCode.PIPE_RECEIVER_IDEMPOTENT_CONFLICT_EXCEPTION.getStatusCode()) .setMessage(context.getMessage()); @@ -182,7 +185,7 @@ public TSStatus visitPipeDeleteLogicalView( @Override public TSStatus visitPipeDeactivateTemplate( - PipeDeactivateTemplatePlan pipeDeactivateTemplatePlan, TSStatus context) { + final PipeDeactivateTemplatePlan pipeDeactivateTemplatePlan, final TSStatus context) { if (context.getCode() == TSStatusCode.TEMPLATE_NOT_ACTIVATED.getStatusCode()) { return new TSStatus(TSStatusCode.PIPE_RECEIVER_IDEMPOTENT_CONFLICT_EXCEPTION.getStatusCode()) .setMessage(context.getMessage()); @@ -191,7 +194,7 @@ public TSStatus visitPipeDeactivateTemplate( } @Override - public TSStatus visitCreateUser(AuthorPlan plan, TSStatus context) { + public TSStatus visitCreateUser(final AuthorPlan plan, final TSStatus context) { if (context.getCode() == TSStatusCode.USER_ALREADY_EXIST.getStatusCode()) { return new TSStatus(TSStatusCode.PIPE_RECEIVER_IDEMPOTENT_CONFLICT_EXCEPTION.getStatusCode()) .setMessage(context.getMessage()); @@ -200,7 +203,7 @@ public TSStatus visitCreateUser(AuthorPlan plan, TSStatus context) { } @Override - public TSStatus visitCreateRawUser(AuthorPlan plan, TSStatus context) { + public TSStatus visitCreateRawUser(final AuthorPlan plan, final TSStatus context) { if (context.getCode() == TSStatusCode.USER_ALREADY_EXIST.getStatusCode()) { return new TSStatus(TSStatusCode.PIPE_RECEIVER_IDEMPOTENT_CONFLICT_EXCEPTION.getStatusCode()) .setMessage(context.getMessage()); @@ -209,7 +212,7 @@ public TSStatus visitCreateRawUser(AuthorPlan plan, TSStatus context) { } @Override - public TSStatus visitUpdateUser(AuthorPlan plan, TSStatus context) { + public TSStatus visitUpdateUser(final AuthorPlan plan, final TSStatus context) { if (context.getCode() == TSStatusCode.USER_NOT_EXIST.getStatusCode()) { return new TSStatus(TSStatusCode.PIPE_RECEIVER_USER_CONFLICT_EXCEPTION.getStatusCode()) .setMessage(context.getMessage()); @@ -218,7 +221,7 @@ public TSStatus visitUpdateUser(AuthorPlan plan, TSStatus context) { } @Override - public TSStatus visitDropUser(AuthorPlan plan, TSStatus context) { + public TSStatus visitDropUser(final AuthorPlan plan, final TSStatus context) { if (context.getCode() == TSStatusCode.USER_NOT_EXIST.getStatusCode()) { return new TSStatus(TSStatusCode.PIPE_RECEIVER_IDEMPOTENT_CONFLICT_EXCEPTION.getStatusCode()) .setMessage(context.getMessage()); @@ -227,7 +230,7 @@ public TSStatus visitDropUser(AuthorPlan plan, TSStatus context) { } @Override - public TSStatus visitGrantUser(AuthorPlan plan, TSStatus context) { + public TSStatus visitGrantUser(final AuthorPlan plan, final TSStatus context) { if (context.getCode() == TSStatusCode.NO_PERMISSION.getStatusCode()) { // Admin user return new TSStatus(TSStatusCode.PIPE_RECEIVER_IDEMPOTENT_CONFLICT_EXCEPTION.getStatusCode()) @@ -241,7 +244,7 @@ public TSStatus visitGrantUser(AuthorPlan plan, TSStatus context) { } @Override - public TSStatus visitRevokeUser(AuthorPlan plan, TSStatus context) { + public TSStatus visitRevokeUser(final AuthorPlan plan, final TSStatus context) { if (context.getCode() == TSStatusCode.NOT_HAS_PRIVILEGE.getStatusCode()) { return new TSStatus(TSStatusCode.PIPE_RECEIVER_IDEMPOTENT_CONFLICT_EXCEPTION.getStatusCode()) .setMessage(context.getMessage()); @@ -255,7 +258,7 @@ public TSStatus visitRevokeUser(AuthorPlan plan, TSStatus context) { } @Override - public TSStatus visitCreateRole(AuthorPlan plan, TSStatus context) { + public TSStatus visitCreateRole(final AuthorPlan plan, final TSStatus context) { if (context.getCode() == TSStatusCode.ROLE_ALREADY_EXIST.getStatusCode()) { return new TSStatus(TSStatusCode.PIPE_RECEIVER_IDEMPOTENT_CONFLICT_EXCEPTION.getStatusCode()) .setMessage(context.getMessage()); @@ -264,7 +267,7 @@ public TSStatus visitCreateRole(AuthorPlan plan, TSStatus context) { } @Override - public TSStatus visitDropRole(AuthorPlan plan, TSStatus context) { + public TSStatus visitDropRole(final AuthorPlan plan, final TSStatus context) { if (context.getCode() == TSStatusCode.ROLE_NOT_EXIST.getStatusCode()) { return new TSStatus(TSStatusCode.PIPE_RECEIVER_IDEMPOTENT_CONFLICT_EXCEPTION.getStatusCode()) .setMessage(context.getMessage()); @@ -273,7 +276,7 @@ public TSStatus visitDropRole(AuthorPlan plan, TSStatus context) { } @Override - public TSStatus visitGrantRole(AuthorPlan plan, TSStatus context) { + public TSStatus visitGrantRole(final AuthorPlan plan, final TSStatus context) { if (context.getCode() == TSStatusCode.ROLE_NOT_EXIST.getStatusCode()) { return new TSStatus(TSStatusCode.PIPE_RECEIVER_USER_CONFLICT_EXCEPTION.getStatusCode()) .setMessage(context.getMessage()); @@ -282,7 +285,7 @@ public TSStatus visitGrantRole(AuthorPlan plan, TSStatus context) { } @Override - public TSStatus visitRevokeRole(AuthorPlan plan, TSStatus context) { + public TSStatus visitRevokeRole(final AuthorPlan plan, final TSStatus context) { if (context.getCode() == TSStatusCode.NOT_HAS_PRIVILEGE.getStatusCode()) { return new TSStatus(TSStatusCode.PIPE_RECEIVER_IDEMPOTENT_CONFLICT_EXCEPTION.getStatusCode()) .setMessage(context.getMessage()); @@ -294,7 +297,7 @@ public TSStatus visitRevokeRole(AuthorPlan plan, TSStatus context) { } @Override - public TSStatus visitGrantRoleToUser(AuthorPlan plan, TSStatus context) { + public TSStatus visitGrantRoleToUser(final AuthorPlan plan, final TSStatus context) { if (context.getCode() == TSStatusCode.USER_ALREADY_HAS_ROLE.getStatusCode()) { return new TSStatus(TSStatusCode.PIPE_RECEIVER_IDEMPOTENT_CONFLICT_EXCEPTION.getStatusCode()) .setMessage(context.getMessage()); @@ -303,7 +306,7 @@ public TSStatus visitGrantRoleToUser(AuthorPlan plan, TSStatus context) { } @Override - public TSStatus visitRevokeRoleFromUser(AuthorPlan plan, TSStatus context) { + public TSStatus visitRevokeRoleFromUser(final AuthorPlan plan, final TSStatus context) { if (context.getCode() == TSStatusCode.USER_NOT_HAS_ROLE.getStatusCode()) { return new TSStatus(TSStatusCode.PIPE_RECEIVER_IDEMPOTENT_CONFLICT_EXCEPTION.getStatusCode()) .setMessage(context.getMessage()); @@ -315,9 +318,9 @@ public TSStatus visitRevokeRoleFromUser(AuthorPlan plan, TSStatus context) { } @Override - public TSStatus visitTTL(SetTTLPlan plan, TSStatus context) { + public TSStatus visitTTL(final SetTTLPlan plan, final TSStatus context) { if (context.getCode() == TSStatusCode.DATABASE_NOT_EXIST.getStatusCode()) { - return new TSStatus(TSStatusCode.PIPE_RECEIVER_USER_CONFLICT_EXCEPTION.getStatusCode()) + return new TSStatus(TSStatusCode.PIPE_RECEIVER_IDEMPOTENT_CONFLICT_EXCEPTION.getStatusCode()) .setMessage(context.getMessage()); } return super.visitTTL(plan, context); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java index 6cb52ef7f278..afb2f8415de9 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java @@ -26,7 +26,6 @@ import org.apache.iotdb.common.rpc.thrift.TSetSpaceQuotaReq; import org.apache.iotdb.common.rpc.thrift.TSetTTLReq; import org.apache.iotdb.common.rpc.thrift.TSetThrottleQuotaReq; -import org.apache.iotdb.commons.auth.AuthException; import org.apache.iotdb.commons.conf.CommonDescriptor; import org.apache.iotdb.commons.consensus.ConsensusGroupId; import org.apache.iotdb.commons.path.PartialPath; @@ -545,52 +544,42 @@ public TDataPartitionTableResp getOrCreateDataPartitionTable(TDataPartitionReq r } @Override - public TSStatus operatePermission(TAuthorizerReq req) { + public TSStatus operatePermission(final TAuthorizerReq req) { if (req.getAuthorType() < 0 || req.getAuthorType() >= AuthorType.values().length) { throw new IndexOutOfBoundsException("Invalid Author Type ordinal"); } - AuthorPlan plan = null; - try { - plan = - new AuthorPlan( - ConfigPhysicalPlanType.values()[ - req.getAuthorType() + ConfigPhysicalPlanType.CreateUser.ordinal()], - req.getUserName(), - req.getRoleName(), - req.getPassword(), - req.getNewPassword(), - req.getPermissions(), - req.isGrantOpt(), - AuthUtils.deserializePartialPathList(ByteBuffer.wrap(req.getNodeNameList()))); - } catch (AuthException e) { - LOGGER.error(e.getMessage()); - } - return configManager.operatePermission(plan); + return configManager.operatePermission( + new AuthorPlan( + ConfigPhysicalPlanType.values()[ + req.getAuthorType() + ConfigPhysicalPlanType.CreateUser.ordinal()], + req.getUserName(), + req.getRoleName(), + req.getPassword(), + req.getNewPassword(), + req.getPermissions(), + req.isGrantOpt(), + AuthUtils.deserializePartialPathList(ByteBuffer.wrap(req.getNodeNameList())))); } @Override - public TAuthorizerResp queryPermission(TAuthorizerReq req) { + public TAuthorizerResp queryPermission(final TAuthorizerReq req) { if (req.getAuthorType() < 0 || req.getAuthorType() >= AuthorType.values().length) { throw new IndexOutOfBoundsException("Invalid Author Type ordinal"); } - AuthorPlan plan = null; - try { - plan = - new AuthorPlan( - ConfigPhysicalPlanType.values()[ - req.getAuthorType() + ConfigPhysicalPlanType.CreateUser.ordinal()], - req.getUserName(), - req.getRoleName(), - req.getPassword(), - req.getNewPassword(), - req.getPermissions(), - req.isGrantOpt(), - AuthUtils.deserializePartialPathList(ByteBuffer.wrap(req.getNodeNameList()))); - } catch (AuthException e) { - LOGGER.error(e.getMessage()); - } - PermissionInfoResp dataSet = (PermissionInfoResp) configManager.queryPermission(plan); - TAuthorizerResp resp = new TAuthorizerResp(dataSet.getStatus()); + final PermissionInfoResp dataSet = + (PermissionInfoResp) + configManager.queryPermission( + new AuthorPlan( + ConfigPhysicalPlanType.values()[ + req.getAuthorType() + ConfigPhysicalPlanType.CreateUser.ordinal()], + req.getUserName(), + req.getRoleName(), + req.getPassword(), + req.getNewPassword(), + req.getPermissions(), + req.isGrantOpt(), + AuthUtils.deserializePartialPathList(ByteBuffer.wrap(req.getNodeNameList())))); + final TAuthorizerResp resp = new TAuthorizerResp(dataSet.getStatus()); resp.setMemberInfo(dataSet.getMemberList()); resp.setPermissionInfo(dataSet.getPermissionInfoResp()); resp.setTag(dataSet.getTag()); diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanSerDeTest.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanSerDeTest.java index b6c57369f673..5c60f2dc3d3d 100644 --- a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanSerDeTest.java +++ b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanSerDeTest.java @@ -33,7 +33,6 @@ import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot; import org.apache.iotdb.common.rpc.thrift.TTimedQuota; import org.apache.iotdb.common.rpc.thrift.ThrottleType; -import org.apache.iotdb.commons.auth.AuthException; import org.apache.iotdb.commons.consensus.index.impl.IoTProgressIndex; import org.apache.iotdb.commons.consensus.index.impl.MinimumProgressIndex; import org.apache.iotdb.commons.exception.IllegalPathException; @@ -111,6 +110,7 @@ import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeDeleteLogicalViewPlan; import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeDeleteTimeSeriesPlan; import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeEnrichedPlan; +import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeSetTTLPlan; import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeUnsetSchemaTemplatePlan; import org.apache.iotdb.confignode.consensus.request.write.pipe.plugin.CreatePipePluginPlan; import org.apache.iotdb.confignode.consensus.request.write.pipe.plugin.DropPipePluginPlan; @@ -145,6 +145,7 @@ import org.apache.iotdb.confignode.consensus.request.write.sync.ShowPipePlanV1; import org.apache.iotdb.confignode.consensus.request.write.template.CreateSchemaTemplatePlan; import org.apache.iotdb.confignode.consensus.request.write.template.DropSchemaTemplatePlan; +import org.apache.iotdb.confignode.consensus.request.write.template.ExtendSchemaTemplatePlan; import org.apache.iotdb.confignode.consensus.request.write.template.PreUnsetSchemaTemplatePlan; import org.apache.iotdb.confignode.consensus.request.write.template.RollbackPreUnsetSchemaTemplatePlan; import org.apache.iotdb.confignode.consensus.request.write.template.SetSchemaTemplatePlan; @@ -166,6 +167,7 @@ import org.apache.iotdb.confignode.rpc.thrift.TTimeSlotList; import org.apache.iotdb.confignode.rpc.thrift.TTriggerState; import org.apache.iotdb.db.schemaengine.template.Template; +import org.apache.iotdb.db.schemaengine.template.alter.TemplateExtendInfo; import org.apache.iotdb.trigger.api.enums.FailureStrategy; import org.apache.iotdb.trigger.api.enums.TriggerEvent; @@ -560,11 +562,12 @@ public void GetOrCreateDataPartitionPlanTest() throws IOException { } @Test - public void AuthorPlanTest() throws IOException, AuthException, IllegalPathException { - + public void AuthorPlanTest() throws IOException, IllegalPathException { AuthorPlan req0; AuthorPlan req1; Set permissions = new HashSet<>(); + permissions.add(1); + permissions.add(2); // create user req0 = @@ -966,6 +969,20 @@ private Template newSchemaTemplate(String name) throws IllegalPathException { return new Template(name, measurements, dataTypes, encodings, compressors); } + @Test + public void ExtendSchemaTemplatePlanTest() throws IOException { + final ExtendSchemaTemplatePlan plan = + new ExtendSchemaTemplatePlan( + new TemplateExtendInfo( + "template_name", + Arrays.asList( + "template_name" + "_" + "temperature", "template_name" + "_" + "status"), + Arrays.asList(TSDataType.FLOAT, TSDataType.BOOLEAN), + Arrays.asList(TSEncoding.RLE, TSEncoding.PLAIN), + Arrays.asList(CompressionType.SNAPPY, CompressionType.SNAPPY))); + Assert.assertEquals(plan, ConfigPhysicalPlan.Factory.create(plan.serializeToByteBuffer())); + } + @Test public void GetSchemaTemplatePlanTest() throws IOException { GetSchemaTemplatePlan getSchemaTemplatePlan = new GetSchemaTemplatePlan("template1"); @@ -1869,72 +1886,78 @@ public void updateClusterIdPlanTest() throws IOException { @Test public void pipeEnrichedPlanTest() throws IOException { - DatabaseSchemaPlan req0 = - new DatabaseSchemaPlan( - ConfigPhysicalPlanType.CreateDatabase, - new TDatabaseSchema() - .setName("sg") - .setTTL(Long.MAX_VALUE) - .setSchemaReplicationFactor(3) - .setDataReplicationFactor(3) - .setTimePartitionInterval(604800)); - PipeEnrichedPlan plan = new PipeEnrichedPlan(req0); - PipeEnrichedPlan plan1 = - (PipeEnrichedPlan) ConfigPhysicalPlan.Factory.create(plan.serializeToByteBuffer()); - Assert.assertEquals(plan, plan1); + final PipeEnrichedPlan plan = + new PipeEnrichedPlan( + new DatabaseSchemaPlan( + ConfigPhysicalPlanType.CreateDatabase, + new TDatabaseSchema() + .setName("sg") + .setTTL(Long.MAX_VALUE) + .setSchemaReplicationFactor(3) + .setDataReplicationFactor(3) + .setTimePartitionInterval(604800))); + Assert.assertEquals(plan, ConfigPhysicalPlan.Factory.create(plan.serializeToByteBuffer())); } @Test public void pipeUnsetSchemaTemplatePlanTest() throws IOException { - PipeUnsetSchemaTemplatePlan pipeUnsetSchemaTemplatePlan = + final PipeUnsetSchemaTemplatePlan pipeUnsetSchemaTemplatePlan = new PipeUnsetSchemaTemplatePlan("template0", "root.sg"); - PipeUnsetSchemaTemplatePlan deserializedPlan = - (PipeUnsetSchemaTemplatePlan) - ConfigPhysicalPlan.Factory.create(pipeUnsetSchemaTemplatePlan.serializeToByteBuffer()); - Assert.assertEquals(pipeUnsetSchemaTemplatePlan, deserializedPlan); + Assert.assertEquals( + pipeUnsetSchemaTemplatePlan, + ConfigPhysicalPlan.Factory.create(pipeUnsetSchemaTemplatePlan.serializeToByteBuffer())); } @Test public void pipeDeleteTimeSeriesPlanTest() throws IOException, IllegalPathException { - PathPatternTree patternTree = new PathPatternTree(); + final PathPatternTree patternTree = new PathPatternTree(); patternTree.appendPathPattern(new PartialPath("root.**.s1")); patternTree.constructTree(); - PipeDeleteTimeSeriesPlan pipeDeleteTimeSeriesPlan = + final PipeDeleteTimeSeriesPlan pipeDeleteTimeSeriesPlan = new PipeDeleteTimeSeriesPlan(patternTree.serialize()); - PipeDeleteTimeSeriesPlan deserializedPlan = - (PipeDeleteTimeSeriesPlan) - ConfigPhysicalPlan.Factory.create(pipeDeleteTimeSeriesPlan.serializeToByteBuffer()); - Assert.assertEquals(pipeDeleteTimeSeriesPlan, deserializedPlan); + Assert.assertEquals( + pipeDeleteTimeSeriesPlan, + ConfigPhysicalPlan.Factory.create(pipeDeleteTimeSeriesPlan.serializeToByteBuffer())); } @Test public void pipeDeleteLogicalViewPlanTest() throws IOException, IllegalPathException { - PathPatternTree patternTree = new PathPatternTree(); + final PathPatternTree patternTree = new PathPatternTree(); patternTree.appendPathPattern(new PartialPath("root.**.s1")); patternTree.constructTree(); - PipeDeleteLogicalViewPlan pipeDeleteLogicalViewPlan = + final PipeDeleteLogicalViewPlan pipeDeleteLogicalViewPlan = new PipeDeleteLogicalViewPlan(patternTree.serialize()); - PipeDeleteLogicalViewPlan deserializedPlan = - (PipeDeleteLogicalViewPlan) - ConfigPhysicalPlan.Factory.create(pipeDeleteLogicalViewPlan.serializeToByteBuffer()); - Assert.assertEquals(pipeDeleteLogicalViewPlan, deserializedPlan); + Assert.assertEquals( + pipeDeleteLogicalViewPlan, + ConfigPhysicalPlan.Factory.create(pipeDeleteLogicalViewPlan.serializeToByteBuffer())); } @Test public void pipeDeactivateTemplatePlanTest() throws IllegalPathException, IOException { - Map> templateSetInfo = new HashMap<>(); - templateSetInfo.put( - new PartialPath("root.**.s1"), - Collections.singletonList(newSchemaTemplate("template_name"))); - - PipeDeactivateTemplatePlan pipeDeactivateTemplatePlan = - new PipeDeactivateTemplatePlan(templateSetInfo); - PipeDeactivateTemplatePlan deserializedPlan = - (PipeDeactivateTemplatePlan) - ConfigPhysicalPlan.Factory.create(pipeDeactivateTemplatePlan.serializeToByteBuffer()); - Assert.assertEquals(pipeDeactivateTemplatePlan, deserializedPlan); + final PipeDeactivateTemplatePlan pipeDeactivateTemplatePlan = + new PipeDeactivateTemplatePlan( + new HashMap>() { + { + put( + new PartialPath("root.**.s1"), + Collections.singletonList(newSchemaTemplate("template_name"))); + } + }); + Assert.assertEquals( + pipeDeactivateTemplatePlan, + ConfigPhysicalPlan.Factory.create(pipeDeactivateTemplatePlan.serializeToByteBuffer())); + } + + @Test + public void pipeSetTTLPlanTest() throws IOException { + final PipeSetTTLPlan plan = + new PipeSetTTLPlan( + Arrays.asList( + new SetTTLPlan(Arrays.asList("root", "db", "**", "a", "**"), Long.MAX_VALUE), + new SetTTLPlan(Arrays.asList("root", "db", "a", "**"), Long.MAX_VALUE))); + Assert.assertEquals(plan, ConfigPhysicalPlan.Factory.create(plan.serializeToByteBuffer())); } @Test diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/manager/pipe/connector/PipeConfigNodeThriftRequestTest.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/manager/pipe/connector/PipeConfigNodeThriftRequestTest.java index 934fa214e06f..c2c04f945299 100644 --- a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/manager/pipe/connector/PipeConfigNodeThriftRequestTest.java +++ b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/manager/pipe/connector/PipeConfigNodeThriftRequestTest.java @@ -89,7 +89,7 @@ public void testPipeTransferConfigSnapshotSealReq() throws IOException { PipeTransferConfigSnapshotSealReq req = PipeTransferConfigSnapshotSealReq.toTPipeTransferReq( - snapshotName, 100, templateInfoName, 10, fileType, typeString); + "root.**", snapshotName, 100, templateInfoName, 10, fileType, typeString); PipeTransferConfigSnapshotSealReq deserializeReq = PipeTransferConfigSnapshotSealReq.fromTPipeTransferReq(req); diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/manager/pipe/extractor/IoTDBConfigRegionExtractorTest.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/manager/pipe/extractor/IoTDBConfigRegionExtractorTest.java index 46f514dbaf16..f3ccf42a0c22 100644 --- a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/manager/pipe/extractor/IoTDBConfigRegionExtractorTest.java +++ b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/manager/pipe/extractor/IoTDBConfigRegionExtractorTest.java @@ -30,9 +30,8 @@ public class IoTDBConfigRegionExtractorTest { @Test - public void testIoTDBSchemaExtractor() { - IoTDBConfigRegionExtractor extractor = new IoTDBConfigRegionExtractor(); - try { + public void testIoTDBConfigExtractor() { + try (final IoTDBConfigRegionExtractor extractor = new IoTDBConfigRegionExtractor()) { extractor.validate( new PipeParameterValidator( new PipeParameters( @@ -52,7 +51,7 @@ public void testIoTDBSchemaExtractor() { PipeExtractorConstant.EXTRACTOR_INCLUSION_DEFAULT_VALUE); } }))); - } catch (Exception e) { + } catch (final Exception e) { Assert.fail(); } } diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/manager/pipe/extractor/PipeConfigPhysicalPlanPatternParseVisitorTest.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/manager/pipe/extractor/PipeConfigPhysicalPlanPatternParseVisitorTest.java new file mode 100644 index 000000000000..a2ff81a9da44 --- /dev/null +++ b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/manager/pipe/extractor/PipeConfigPhysicalPlanPatternParseVisitorTest.java @@ -0,0 +1,408 @@ +/* + * 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.confignode.manager.pipe.extractor; + +import org.apache.iotdb.commons.exception.IllegalPathException; +import org.apache.iotdb.commons.path.PartialPath; +import org.apache.iotdb.commons.path.PathPatternTree; +import org.apache.iotdb.commons.pipe.pattern.IoTDBPipePattern; +import org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlanType; +import org.apache.iotdb.confignode.consensus.request.auth.AuthorPlan; +import org.apache.iotdb.confignode.consensus.request.write.database.DatabaseSchemaPlan; +import org.apache.iotdb.confignode.consensus.request.write.database.DeleteDatabasePlan; +import org.apache.iotdb.confignode.consensus.request.write.database.SetTTLPlan; +import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeDeactivateTemplatePlan; +import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeDeleteLogicalViewPlan; +import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeDeleteTimeSeriesPlan; +import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeSetTTLPlan; +import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeUnsetSchemaTemplatePlan; +import org.apache.iotdb.confignode.consensus.request.write.template.CommitSetSchemaTemplatePlan; +import org.apache.iotdb.confignode.consensus.request.write.template.CreateSchemaTemplatePlan; +import org.apache.iotdb.confignode.consensus.request.write.template.ExtendSchemaTemplatePlan; +import org.apache.iotdb.confignode.rpc.thrift.TDatabaseSchema; +import org.apache.iotdb.db.schemaengine.template.Template; +import org.apache.iotdb.db.schemaengine.template.alter.TemplateExtendInfo; + +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.file.metadata.enums.CompressionType; +import org.apache.tsfile.file.metadata.enums.TSEncoding; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.stream.Collectors; + +public class PipeConfigPhysicalPlanPatternParseVisitorTest { + + private final IoTDBPipePattern prefixPathPattern = new IoTDBPipePattern("root.db.device.**"); + private final IoTDBPipePattern fullPathPattern = new IoTDBPipePattern("root.db.device.s1"); + + @Test + public void testCreateDatabase() { + final DatabaseSchemaPlan createDatabasePlan = + new DatabaseSchemaPlan( + ConfigPhysicalPlanType.CreateDatabase, new TDatabaseSchema("root.db")); + final DatabaseSchemaPlan createDatabasePlanToFilter = + new DatabaseSchemaPlan( + ConfigPhysicalPlanType.CreateDatabase, new TDatabaseSchema("root.db1")); + + Assert.assertEquals( + createDatabasePlan, + IoTDBConfigRegionExtractor.PATTERN_PARSE_VISITOR + .visitCreateDatabase(createDatabasePlan, prefixPathPattern) + .orElseThrow(AssertionError::new)); + Assert.assertFalse( + IoTDBConfigRegionExtractor.PATTERN_PARSE_VISITOR + .visitCreateDatabase(createDatabasePlanToFilter, prefixPathPattern) + .isPresent()); + } + + @Test + public void testAlterDatabase() { + final DatabaseSchemaPlan alterDatabasePlan = + new DatabaseSchemaPlan( + ConfigPhysicalPlanType.AlterDatabase, new TDatabaseSchema("root.db")); + final DatabaseSchemaPlan alterDatabasePlanToFilter = + new DatabaseSchemaPlan( + ConfigPhysicalPlanType.AlterDatabase, new TDatabaseSchema("root.db1")); + + Assert.assertEquals( + alterDatabasePlan, + IoTDBConfigRegionExtractor.PATTERN_PARSE_VISITOR + .visitAlterDatabase(alterDatabasePlan, prefixPathPattern) + .orElseThrow(AssertionError::new)); + Assert.assertFalse( + IoTDBConfigRegionExtractor.PATTERN_PARSE_VISITOR + .visitAlterDatabase(alterDatabasePlanToFilter, prefixPathPattern) + .isPresent()); + } + + @Test + public void testDeleteDatabase() { + final DeleteDatabasePlan deleteDatabasePlan = new DeleteDatabasePlan("root.db"); + final DeleteDatabasePlan deleteDatabasePlanToFilter = new DeleteDatabasePlan("root.db1"); + + Assert.assertEquals( + deleteDatabasePlan, + IoTDBConfigRegionExtractor.PATTERN_PARSE_VISITOR + .visitDeleteDatabase(deleteDatabasePlan, prefixPathPattern) + .orElseThrow(AssertionError::new)); + Assert.assertFalse( + IoTDBConfigRegionExtractor.PATTERN_PARSE_VISITOR + .visitDeleteDatabase(deleteDatabasePlanToFilter, prefixPathPattern) + .isPresent()); + } + + @Test + public void testCreateSchemaTemplate() throws IllegalPathException { + final CreateSchemaTemplatePlan createSchemaTemplatePlan = + new CreateSchemaTemplatePlan( + new Template( + "template_name", + Arrays.asList("s1", "s2"), + Arrays.asList(TSDataType.FLOAT, TSDataType.BOOLEAN), + Arrays.asList(TSEncoding.RLE, TSEncoding.PLAIN), + Arrays.asList(CompressionType.SNAPPY, CompressionType.SNAPPY)) + .serialize() + .array()); + + Assert.assertEquals( + createSchemaTemplatePlan, + IoTDBConfigRegionExtractor.PATTERN_PARSE_VISITOR + .visitCreateSchemaTemplate(createSchemaTemplatePlan, prefixPathPattern) + .orElseThrow(AssertionError::new)); + + final CreateSchemaTemplatePlan parsedTemplatePlan = + (CreateSchemaTemplatePlan) + IoTDBConfigRegionExtractor.PATTERN_PARSE_VISITOR + .visitCreateSchemaTemplate(createSchemaTemplatePlan, fullPathPattern) + .orElseThrow(AssertionError::new); + Assert.assertEquals( + Collections.singleton("s1"), parsedTemplatePlan.getTemplate().getSchemaMap().keySet()); + Assert.assertEquals( + createSchemaTemplatePlan.getTemplate().getSchemaMap().get("s1"), + parsedTemplatePlan.getTemplate().getSchemaMap().get("s1")); + } + + @Test + public void testCommitSetSchemaTemplate() { + final CommitSetSchemaTemplatePlan setSchemaTemplatePlanOnPrefix = + new CommitSetSchemaTemplatePlan("t1", "root.db"); + final CommitSetSchemaTemplatePlan setSchemaTemplatePlanOnFullPath = + new CommitSetSchemaTemplatePlan("t1", "root.db.device.s1"); + + Assert.assertEquals( + setSchemaTemplatePlanOnPrefix, + IoTDBConfigRegionExtractor.PATTERN_PARSE_VISITOR + .visitCommitSetSchemaTemplate(setSchemaTemplatePlanOnPrefix, fullPathPattern) + .orElseThrow(AssertionError::new)); + Assert.assertEquals( + setSchemaTemplatePlanOnFullPath, + IoTDBConfigRegionExtractor.PATTERN_PARSE_VISITOR + .visitCommitSetSchemaTemplate(setSchemaTemplatePlanOnFullPath, fullPathPattern) + .orElseThrow(AssertionError::new)); + } + + @Test + public void testPipeUnsetSchemaTemplate() { + final PipeUnsetSchemaTemplatePlan pipeUnsetSchemaTemplatePlanOnPrefix = + new PipeUnsetSchemaTemplatePlan("t1", "root.db"); + final PipeUnsetSchemaTemplatePlan pipeUnsetSchemaTemplatePlanOrFullPath = + new PipeUnsetSchemaTemplatePlan("t1", "root.db.device.s1"); + + Assert.assertEquals( + pipeUnsetSchemaTemplatePlanOnPrefix, + IoTDBConfigRegionExtractor.PATTERN_PARSE_VISITOR + .visitPipeUnsetSchemaTemplate(pipeUnsetSchemaTemplatePlanOnPrefix, fullPathPattern) + .orElseThrow(AssertionError::new)); + Assert.assertEquals( + pipeUnsetSchemaTemplatePlanOrFullPath, + IoTDBConfigRegionExtractor.PATTERN_PARSE_VISITOR + .visitPipeUnsetSchemaTemplate(pipeUnsetSchemaTemplatePlanOrFullPath, fullPathPattern) + .orElseThrow(AssertionError::new)); + } + + @Test + public void testExtendSchemaTemplate() { + final ExtendSchemaTemplatePlan extendSchemaTemplatePlan = + new ExtendSchemaTemplatePlan( + new TemplateExtendInfo( + "template_name", + Arrays.asList("s1", "s2"), + Arrays.asList(TSDataType.FLOAT, TSDataType.BOOLEAN), + Arrays.asList(TSEncoding.RLE, TSEncoding.PLAIN), + Arrays.asList(CompressionType.SNAPPY, CompressionType.SNAPPY))); + + final ExtendSchemaTemplatePlan parsedTemplatePlan = + (ExtendSchemaTemplatePlan) + IoTDBConfigRegionExtractor.PATTERN_PARSE_VISITOR + .visitExtendSchemaTemplate(extendSchemaTemplatePlan, fullPathPattern) + .orElseThrow(AssertionError::new); + Assert.assertEquals( + Collections.singletonList("s1"), + parsedTemplatePlan.getTemplateExtendInfo().getMeasurements()); + Assert.assertEquals( + extendSchemaTemplatePlan.getTemplateExtendInfo().getTemplateName(), + parsedTemplatePlan.getTemplateExtendInfo().getTemplateName()); + Assert.assertEquals( + extendSchemaTemplatePlan.getTemplateExtendInfo().getDataTypes().get(0), + parsedTemplatePlan.getTemplateExtendInfo().getDataTypes().get(0)); + Assert.assertEquals( + extendSchemaTemplatePlan.getTemplateExtendInfo().getEncodings().get(0), + parsedTemplatePlan.getTemplateExtendInfo().getEncodings().get(0)); + Assert.assertEquals( + extendSchemaTemplatePlan.getTemplateExtendInfo().getCompressors().get(0), + parsedTemplatePlan.getTemplateExtendInfo().getCompressors().get(0)); + } + + @Test + public void testGrantUser() throws IllegalPathException { + Assert.assertEquals( + Collections.singletonList(new PartialPath("root.db.device.**")), + ((AuthorPlan) + IoTDBConfigRegionExtractor.PATTERN_PARSE_VISITOR + .visitGrantUser( + new AuthorPlan( + ConfigPhysicalPlanType.GrantUser, + "tempUser", + "", + "", + "", + new HashSet<>(Arrays.asList(1, 2)), + true, + Arrays.asList( + new PartialPath("root.db.**"), new PartialPath("root.abc.**"))), + prefixPathPattern) + .orElseThrow(AssertionError::new)) + .getNodeNameList()); + } + + @Test + public void testRevokeUser() throws IllegalPathException { + Assert.assertEquals( + Collections.singletonList(new PartialPath("root.db.device.**")), + ((AuthorPlan) + IoTDBConfigRegionExtractor.PATTERN_PARSE_VISITOR + .visitRevokeUser( + new AuthorPlan( + ConfigPhysicalPlanType.RevokeUser, + "tempUser", + "", + "", + "", + new HashSet<>(Arrays.asList(1, 2)), + false, + Arrays.asList( + new PartialPath("root.db.**"), new PartialPath("root.abc.**"))), + prefixPathPattern) + .orElseThrow(AssertionError::new)) + .getNodeNameList()); + } + + @Test + public void testGrantRole() throws IllegalPathException { + Assert.assertEquals( + Collections.singletonList(new PartialPath("root.db.device.**")), + ((AuthorPlan) + IoTDBConfigRegionExtractor.PATTERN_PARSE_VISITOR + .visitGrantRole( + new AuthorPlan( + ConfigPhysicalPlanType.GrantRole, + "", + "tempRole", + "", + "", + new HashSet<>(Arrays.asList(1, 2)), + true, + Arrays.asList( + new PartialPath("root.db.**"), new PartialPath("root.abc.**"))), + prefixPathPattern) + .orElseThrow(AssertionError::new)) + .getNodeNameList()); + } + + @Test + public void testRevokeRole() throws IllegalPathException { + Assert.assertEquals( + Collections.singletonList(new PartialPath("root.db.device.**")), + ((AuthorPlan) + IoTDBConfigRegionExtractor.PATTERN_PARSE_VISITOR + .visitRevokeRole( + new AuthorPlan( + ConfigPhysicalPlanType.RevokeRole, + "", + "tempRole", + "", + "", + new HashSet<>(Arrays.asList(1, 2)), + false, + Arrays.asList( + new PartialPath("root.db.**"), new PartialPath("root.abc.**"))), + prefixPathPattern) + .orElseThrow(AssertionError::new)) + .getNodeNameList()); + } + + @Test + public void testPipeDeleteTimeSeries() throws IllegalPathException, IOException { + final PathPatternTree patternTree = new PathPatternTree(); + patternTree.appendPathPattern(new PartialPath("root.*.device.s1")); + patternTree.constructTree(); + + Assert.assertEquals( + Collections.singletonList(new PartialPath("root.db.device.s1")), + PathPatternTree.deserialize( + ((PipeDeleteTimeSeriesPlan) + IoTDBConfigRegionExtractor.PATTERN_PARSE_VISITOR + .visitPipeDeleteTimeSeries( + new PipeDeleteTimeSeriesPlan(patternTree.serialize()), + prefixPathPattern) + .orElseThrow(AssertionError::new)) + .getPatternTreeBytes()) + .getAllPathPatterns()); + } + + @Test + public void testPipeDeleteLogicalView() throws IllegalPathException, IOException { + final PathPatternTree patternTree = new PathPatternTree(); + patternTree.appendPathPattern(new PartialPath("root.*.device.s1")); + patternTree.constructTree(); + + Assert.assertEquals( + Collections.singletonList(new PartialPath("root.db.device.s1")), + PathPatternTree.deserialize( + ((PipeDeleteTimeSeriesPlan) + IoTDBConfigRegionExtractor.PATTERN_PARSE_VISITOR + .visitPipeDeleteLogicalView( + new PipeDeleteLogicalViewPlan(patternTree.serialize()), + prefixPathPattern) + .orElseThrow(AssertionError::new)) + .getPatternTreeBytes()) + .getAllPathPatterns()); + } + + @Test + public void testPipeDeactivateTemplate() throws IllegalPathException { + final Template template1 = newSchemaTemplate("template1"); + final Template template2 = newSchemaTemplate("template2"); + final Template template3 = newSchemaTemplate("template3"); + + Assert.assertEquals( + new HashMap>() { + { + put( + new PartialPath("root.db.device.s1"), + Arrays.asList(template1, template2, template3)); + } + }, + ((PipeDeactivateTemplatePlan) + IoTDBConfigRegionExtractor.PATTERN_PARSE_VISITOR + .visitPipeDeactivateTemplate( + new PipeDeactivateTemplatePlan( + new HashMap>() { + { + put( + new PartialPath("root.*.device.s1"), + Collections.singletonList(template1)); + put( + new PartialPath("root.db.*.s1"), + Arrays.asList(template2, template3)); + } + }), + prefixPathPattern) + .orElseThrow(AssertionError::new)) + .getTemplateSetInfo()); + } + + private Template newSchemaTemplate(final String name) throws IllegalPathException { + return new Template( + name, + Arrays.asList(name + "_" + "temperature", name + "_" + "status"), + Arrays.asList(TSDataType.FLOAT, TSDataType.BOOLEAN), + Arrays.asList(TSEncoding.RLE, TSEncoding.PLAIN), + Arrays.asList(CompressionType.SNAPPY, CompressionType.SNAPPY)); + } + + @Test + public void testSetTTL() throws IllegalPathException { + final List plans = + ((PipeSetTTLPlan) + IoTDBConfigRegionExtractor.PATTERN_PARSE_VISITOR + .visitTTL( + new SetTTLPlan(Arrays.asList("root", "*", "device", "s1"), Long.MAX_VALUE), + prefixPathPattern) + .orElseThrow(AssertionError::new)) + .getSetTTLPlans(); + + Assert.assertEquals( + Collections.singletonList(new PartialPath("root.db.device.s1")), + plans.stream() + .map(setTTLPlan -> new PartialPath(setTTLPlan.getDatabasePathPattern())) + .collect(Collectors.toList())); + Assert.assertEquals( + Collections.singletonList(Long.MAX_VALUE), + plans.stream().map(SetTTLPlan::getTTL).collect(Collectors.toList())); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferSchemaSnapshotSealReq.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferSchemaSnapshotSealReq.java index 9b896a00203f..70835e0ea4f7 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferSchemaSnapshotSealReq.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferSchemaSnapshotSealReq.java @@ -45,14 +45,16 @@ protected PipeRequestType getPlanType() { /////////////////////////////// Thrift /////////////////////////////// public static PipeTransferSchemaSnapshotSealReq toTPipeTransferReq( - String mTreeSnapshotName, - long mTreeSnapshotLength, - String tLogName, - long tLogLength, - String databaseName, - String typeString) + final String pattern, + final String mTreeSnapshotName, + final long mTreeSnapshotLength, + final String tLogName, + final long tLogLength, + final String databaseName, + final String typeString) throws IOException { final Map parameters = new HashMap<>(); + parameters.put(ColumnHeaderConstant.PATH_PATTERN, pattern); parameters.put(ColumnHeaderConstant.DATABASE, databaseName); parameters.put(ColumnHeaderConstant.TYPE, typeString); return (PipeTransferSchemaSnapshotSealReq) @@ -67,7 +69,7 @@ public static PipeTransferSchemaSnapshotSealReq toTPipeTransferReq( parameters); } - public static PipeTransferSchemaSnapshotSealReq fromTPipeTransferReq(TPipeTransferReq req) { + public static PipeTransferSchemaSnapshotSealReq fromTPipeTransferReq(final TPipeTransferReq req) { return (PipeTransferSchemaSnapshotSealReq) new PipeTransferSchemaSnapshotSealReq().translateFromTPipeTransferReq(req); } @@ -75,14 +77,16 @@ public static PipeTransferSchemaSnapshotSealReq fromTPipeTransferReq(TPipeTransf /////////////////////////////// Air Gap /////////////////////////////// public static byte[] toTPipeTransferBytes( - String mTreeSnapshotName, - long mTreeSnapshotLength, - String tLogName, - long tLogLength, - String databaseName, - String typeString) + final String pattern, + final String mTreeSnapshotName, + final long mTreeSnapshotLength, + final String tLogName, + final long tLogLength, + final String databaseName, + final String typeString) throws IOException { final Map parameters = new HashMap<>(); + parameters.put(ColumnHeaderConstant.PATH_PATTERN, pattern); parameters.put(ColumnHeaderConstant.DATABASE, databaseName); parameters.put(ColumnHeaderConstant.TYPE, typeString); return new PipeTransferSchemaSnapshotSealReq() @@ -99,7 +103,7 @@ public static byte[] toTPipeTransferBytes( /////////////////////////////// Object /////////////////////////////// @Override - public boolean equals(Object obj) { + public boolean equals(final Object obj) { return obj instanceof PipeTransferSchemaSnapshotSealReq && super.equals(obj); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/airgap/IoTDBSchemaRegionAirGapConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/airgap/IoTDBSchemaRegionAirGapConnector.java index d4e6706a5a4e..f390994e4f0f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/airgap/IoTDBSchemaRegionAirGapConnector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/airgap/IoTDBSchemaRegionAirGapConnector.java @@ -116,6 +116,8 @@ private void doTransfer( socket, compressIfNeeded( PipeTransferSchemaSnapshotSealReq.toTPipeTransferBytes( + // The pattern is surely Non-null + pipeSchemaRegionSnapshotEvent.getPatternString(), mtreeSnapshotFile.getName(), mtreeSnapshotFile.length(), Objects.nonNull(tagLogSnapshotFile) ? tagLogSnapshotFile.getName() : null, diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBSchemaRegionConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBSchemaRegionConnector.java index 667050320d07..2b2cba03f20d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBSchemaRegionConnector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBSchemaRegionConnector.java @@ -105,6 +105,8 @@ private void doTransfer(final PipeSchemaRegionSnapshotEvent snapshotEvent) .pipeTransfer( compressIfNeeded( PipeTransferSchemaSnapshotSealReq.toTPipeTransferReq( + // The pattern is surely Non-null + snapshotEvent.getPatternString(), mTreeSnapshotFile.getName(), mTreeSnapshotFile.length(), Objects.nonNull(tagLogSnapshotFile) ? tagLogSnapshotFile.getName() : null, diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/schema/PipeSchemaRegionWritePlanEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/schema/PipeSchemaRegionWritePlanEvent.java index 6e62fe80610f..6e0b5cccf866 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/schema/PipeSchemaRegionWritePlanEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/schema/PipeSchemaRegionWritePlanEvent.java @@ -88,14 +88,14 @@ public void deserializeFromByteBuffer(final ByteBuffer buffer) { @Override public String toString() { - return String.format("PipeConfigRegionWritePlanEvent{planNode=%s}", planNode) + return String.format("PipeSchemaRegionWritePlanEvent{planNode=%s}", planNode) + " - " + super.toString(); } @Override public String coreReportMessage() { - return String.format("PipeConfigRegionWritePlanEvent{planNode=%s}", planNode) + return String.format("PipeSchemaRegionWritePlanEvent{planNode=%s}", planNode) + " - " + super.coreReportMessage(); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/TabletInsertionDataContainer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/TabletInsertionDataContainer.java index e95bc5373b16..83fa0497f18f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/TabletInsertionDataContainer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/TabletInsertionDataContainer.java @@ -89,10 +89,10 @@ public class TabletInsertionDataContainer { } public TabletInsertionDataContainer( - PipeTaskMeta pipeTaskMeta, - EnrichedEvent sourceEvent, - InsertNode insertNode, - PipePattern pattern) { + final PipeTaskMeta pipeTaskMeta, + final EnrichedEvent sourceEvent, + final InsertNode insertNode, + final PipePattern pattern) { this.pipeTaskMeta = pipeTaskMeta; this.sourceEvent = sourceEvent; @@ -107,11 +107,11 @@ public TabletInsertionDataContainer( } public TabletInsertionDataContainer( - PipeTaskMeta pipeTaskMeta, - EnrichedEvent sourceEvent, - Tablet tablet, - boolean isAligned, - PipePattern pattern) { + final PipeTaskMeta pipeTaskMeta, + final EnrichedEvent sourceEvent, + final Tablet tablet, + final boolean isAligned, + final PipePattern pattern) { this.pipeTaskMeta = pipeTaskMeta; this.sourceEvent = sourceEvent; @@ -119,7 +119,7 @@ public TabletInsertionDataContainer( } @TestOnly - public TabletInsertionDataContainer(InsertNode insertNode, PipePattern pattern) { + public TabletInsertionDataContainer(final InsertNode insertNode, final PipePattern pattern) { this(null, null, insertNode, pattern); } @@ -133,7 +133,7 @@ public void markAsNeedToReport() { //////////////////////////// parse //////////////////////////// - private void parse(InsertRowNode insertRowNode, PipePattern pattern) { + private void parse(final InsertRowNode insertRowNode, final PipePattern pattern) { final int originColumnSize = insertRowNode.getMeasurements().length; final Integer[] originColumnIndex2FilteredColumnIndexMapperList = new Integer[originColumnSize]; @@ -200,7 +200,7 @@ private void parse(InsertRowNode insertRowNode, PipePattern pattern) { } } - private void parse(InsertTabletNode insertTabletNode, PipePattern pattern) { + private void parse(final InsertTabletNode insertTabletNode, final PipePattern pattern) { final int originColumnSize = insertTabletNode.getMeasurements().length; final Integer[] originColumnIndex2FilteredColumnIndexMapperList = new Integer[originColumnSize]; @@ -283,7 +283,7 @@ private void parse(InsertTabletNode insertTabletNode, PipePattern pattern) { } } - private void parse(Tablet tablet, boolean isAligned, PipePattern pattern) { + private void parse(final Tablet tablet, final boolean isAligned, final PipePattern pattern) { final int originColumnSize = tablet.getSchemas().size(); final Integer[] originColumnIndex2FilteredColumnIndexMapperList = new Integer[originColumnSize]; @@ -375,9 +375,9 @@ private void parse(Tablet tablet, boolean isAligned, PipePattern pattern) { } private void generateColumnIndexMapper( - String[] originMeasurementList, - PipePattern pattern, - Integer[] originColumnIndex2FilteredColumnIndexMapperList) { + final String[] originMeasurementList, + final PipePattern pattern, + final Integer[] originColumnIndex2FilteredColumnIndexMapperList) { final int originColumnSize = originMeasurementList.length; // case 1: for example, pattern is root.a.b or pattern is null and device is root.a.b.c diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/TsFileInsertionDataContainer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/TsFileInsertionDataContainer.java index 2081a540feac..36dfc3df4589 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/TsFileInsertionDataContainer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/TsFileInsertionDataContainer.java @@ -78,17 +78,18 @@ public class TsFileInsertionDataContainer implements AutoCloseable { private boolean shouldParsePattern = false; public TsFileInsertionDataContainer( - File tsFile, PipePattern pattern, long startTime, long endTime) throws IOException { + final File tsFile, final PipePattern pattern, final long startTime, final long endTime) + throws IOException { this(tsFile, pattern, startTime, endTime, null, null); } public TsFileInsertionDataContainer( - File tsFile, - PipePattern pattern, - long startTime, - long endTime, - PipeTaskMeta pipeTaskMeta, - EnrichedEvent sourceEvent) + final File tsFile, + final PipePattern pattern, + final long startTime, + final long endTime, + final PipeTaskMeta pipeTaskMeta, + final EnrichedEvent sourceEvent) throws IOException { this.pattern = pattern; timeFilterExpression = @@ -135,16 +136,17 @@ public TsFileInsertionDataContainer( // No longer need this. Help GC. tsFileSequenceReader.clearCachedDeviceMetadata(); - } catch (Exception e) { + } catch (final Exception e) { close(); throw e; } } private Map> filterDeviceMeasurementsMapByPattern( - Map> originalDeviceMeasurementsMap) { + final Map> originalDeviceMeasurementsMap) { final Map> filteredDeviceMeasurementsMap = new HashMap<>(); - for (Map.Entry> entry : originalDeviceMeasurementsMap.entrySet()) { + for (final Map.Entry> entry : + originalDeviceMeasurementsMap.entrySet()) { final String deviceId = ((PlainDeviceID) entry.getKey()).toStringID(); // case 1: for example, pattern is root.a.b or pattern is null and device is root.a.b.c @@ -223,7 +225,7 @@ public boolean hasNext() { ((PlainDeviceID) entry.getKey()).toStringID(), entry.getValue(), timeFilterExpression); - } catch (IOException e) { + } catch (final IOException e) { close(); throw new PipeException("failed to create TsFileInsertionDataTabletIterator", e); } @@ -279,7 +281,7 @@ public void close() { if (tsFileReader != null) { tsFileReader.close(); } - } catch (IOException e) { + } catch (final IOException e) { LOGGER.warn("Failed to close TsFileReader", e); } @@ -287,7 +289,7 @@ public void close() { if (tsFileSequenceReader != null) { tsFileSequenceReader.close(); } - } catch (IOException e) { + } catch (final IOException e) { LOGGER.warn("Failed to close TsFileSequenceReader", e); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/TsFileInsertionPointCounter.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/TsFileInsertionPointCounter.java index 086b03c45583..f76c220dd2b0 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/TsFileInsertionPointCounter.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/TsFileInsertionPointCounter.java @@ -52,7 +52,8 @@ public class TsFileInsertionPointCounter implements AutoCloseable { private long count = 0; - public TsFileInsertionPointCounter(File tsFile, PipePattern pattern) throws IOException { + public TsFileInsertionPointCounter(final File tsFile, final PipePattern pattern) + throws IOException { this.pattern = pattern; try { @@ -69,7 +70,7 @@ public TsFileInsertionPointCounter(File tsFile, PipePattern pattern) throws IOEx // No longer need this. Help GC. tsFileSequenceReader.clearCachedDeviceMetadata(); - } catch (Exception e) { + } catch (final Exception e) { close(); throw e; } @@ -80,7 +81,8 @@ private Map> filterDeviceMeasurementsMapByPattern() throw tsFileSequenceReader.getDeviceMeasurementsMap(); final Map> filteredDeviceMeasurementsMap = new HashMap<>(); - for (Map.Entry> entry : originalDeviceMeasurementsMap.entrySet()) { + for (final Map.Entry> entry : + originalDeviceMeasurementsMap.entrySet()) { final String deviceId = ((PlainDeviceID) entry.getKey()).toStringID(); // case 1: for example, pattern is root.a.b or pattern is null and device is root.a.b.c @@ -162,7 +164,7 @@ public void close() { if (tsFileSequenceReader != null) { tsFileSequenceReader.close(); } - } catch (IOException e) { + } catch (final IOException e) { LOGGER.warn("Failed to close TsFileSequenceReader", e); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/IoTDBDataRegionExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/IoTDBDataRegionExtractor.java index b8ee7eb06202..d89b223df5b3 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/IoTDBDataRegionExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/IoTDBDataRegionExtractor.java @@ -22,6 +22,7 @@ import org.apache.iotdb.commons.consensus.DataRegionId; import org.apache.iotdb.commons.exception.IllegalPathException; import org.apache.iotdb.commons.pipe.extractor.IoTDBExtractor; +import org.apache.iotdb.commons.pipe.pattern.IoTDBPipePattern; import org.apache.iotdb.commons.pipe.pattern.PipePattern; import org.apache.iotdb.consensus.ConsensusFactory; import org.apache.iotdb.db.conf.IoTDBDescriptor; @@ -94,6 +95,7 @@ public class IoTDBDataRegionExtractor extends IoTDBExtractor { private boolean hasNoExtractionNeed = true; private boolean shouldExtractInsertion = false; + private boolean shouldExtractDeletion = false; @Override public void validate(final PipeParameterValidator validator) throws Exception { @@ -108,6 +110,7 @@ public void validate(final PipeParameterValidator validator) throws Exception { } hasNoExtractionNeed = false; shouldExtractInsertion = insertionDeletionListeningOptionPair.getLeft(); + shouldExtractDeletion = insertionDeletionListeningOptionPair.getRight(); if (insertionDeletionListeningOptionPair.getLeft().equals(true) && IoTDBDescriptor.getInstance() @@ -214,6 +217,16 @@ private void validatePattern(final PipePattern pattern) { if (!pattern.isLegal()) { throw new IllegalArgumentException(String.format("Pattern \"%s\" is illegal.", pattern)); } + + if (shouldExtractDeletion + && !(pattern instanceof IoTDBPipePattern + && (((IoTDBPipePattern) pattern).isPrefix() + || ((IoTDBPipePattern) pattern).isFullPath()))) { + throw new IllegalArgumentException( + String.format( + "The path pattern %s is not valid for the source. Only prefix or full path is allowed.", + pattern)); + } } private void constructHistoricalExtractor() { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/schemaregion/IoTDBSchemaRegionExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/schemaregion/IoTDBSchemaRegionExtractor.java index a0798e3818fc..3682095999b3 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/schemaregion/IoTDBSchemaRegionExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/schemaregion/IoTDBSchemaRegionExtractor.java @@ -23,6 +23,7 @@ import org.apache.iotdb.commons.pipe.datastructure.queue.listening.AbstractPipeListeningQueue; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.commons.pipe.event.PipeSnapshotEvent; +import org.apache.iotdb.commons.pipe.event.PipeWritePlanEvent; import org.apache.iotdb.commons.pipe.extractor.IoTDBNonDataRegionExtractor; import org.apache.iotdb.consensus.ConsensusFactory; import org.apache.iotdb.consensus.exception.ConsensusException; @@ -40,14 +41,16 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.pipe.PipeOperateSchemaQueueNode; import org.apache.iotdb.pipe.api.customizer.configuration.PipeExtractorRuntimeConfiguration; import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameters; -import org.apache.iotdb.pipe.api.event.Event; import org.apache.iotdb.pipe.api.exception.PipeException; import java.util.HashSet; import java.util.Objects; +import java.util.Optional; import java.util.Set; public class IoTDBSchemaRegionExtractor extends IoTDBNonDataRegionExtractor { + public static final PipePlanPatternParseVisitor PATTERN_PARSE_VISITOR = + new PipePlanPatternParseVisitor(); private SchemaRegionId schemaRegionId; @@ -120,13 +123,21 @@ protected long getMaxBlockingTimeMs() { return 0; } + @Override + protected Optional trimRealtimeEventByPipePattern( + final PipeWritePlanEvent event) { + return PATTERN_PARSE_VISITOR + .process(((PipeSchemaRegionWritePlanEvent) event).getPlanNode(), pipePattern) + .map(planNode -> new PipeSchemaRegionWritePlanEvent(planNode, event.isGeneratedByPipe())); + } + @Override protected AbstractPipeListeningQueue getListeningQueue() { return PipeAgent.runtime().schemaListener(schemaRegionId); } @Override - protected boolean isTypeListened(final Event event) { + protected boolean isTypeListened(final PipeWritePlanEvent event) { final PlanNode planNode = ((PipeSchemaRegionWritePlanEvent) event).getPlanNode(); return listenedTypeSet.contains( (planNode.getType() == PlanNodeType.ALTER_TIME_SERIES diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/schemaregion/PipePlanPatternParseVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/schemaregion/PipePlanPatternParseVisitor.java new file mode 100644 index 000000000000..150775884aa8 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/schemaregion/PipePlanPatternParseVisitor.java @@ -0,0 +1,282 @@ +/* + * 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.db.pipe.extractor.schemaregion; + +import org.apache.iotdb.commons.path.PartialPath; +import org.apache.iotdb.commons.pipe.pattern.IoTDBPipePattern; +import org.apache.iotdb.commons.schema.view.viewExpression.ViewExpression; +import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionWritePlanEvent; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanVisitor; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.metedata.write.ActivateTemplateNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.metedata.write.AlterTimeSeriesNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.metedata.write.BatchActivateTemplateNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.metedata.write.CreateAlignedTimeSeriesNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.metedata.write.CreateMultiTimeSeriesNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.metedata.write.CreateTimeSeriesNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.metedata.write.InternalBatchActivateTemplateNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.metedata.write.InternalCreateMultiTimeSeriesNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.metedata.write.InternalCreateTimeSeriesNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.metedata.write.MeasurementGroup; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.metedata.write.view.AlterLogicalViewNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.metedata.write.view.CreateLogicalViewNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.DeleteDataNode; + +import org.apache.tsfile.utils.Pair; + +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +/** + * The {@link PipePlanPatternParseVisitor} will transform the schema {@link PlanNode}s using {@link + * IoTDBPipePattern}. Rule: + * + *

1. All patterns in the output {@link PlanNode} will be the intersection of the original {@link + * PlanNode}'s patterns and the given {@link IoTDBPipePattern}. + * + *

2. If a pattern does not intersect with the {@link IoTDBPipePattern}, it's dropped. + * + *

3. If all the patterns in the {@link PlanNode} is dropped, the {@link PlanNode} is dropped. + * + *

4. The output {@link PlanNode} shall be a copied form of the original one because the original + * one is used in the {@link PipeSchemaRegionWritePlanEvent} in {@link SchemaRegionListeningQueue}. + */ +public class PipePlanPatternParseVisitor extends PlanVisitor, IoTDBPipePattern> { + @Override + public Optional visitPlan(final PlanNode node, final IoTDBPipePattern pattern) { + return Optional.of(node); + } + + @Override + public Optional visitCreateTimeSeries( + final CreateTimeSeriesNode node, final IoTDBPipePattern pattern) { + return pattern.matchesMeasurement(node.getPath().getDevice(), node.getPath().getMeasurement()) + ? Optional.of(node) + : Optional.empty(); + } + + @Override + public Optional visitCreateAlignedTimeSeries( + final CreateAlignedTimeSeriesNode node, final IoTDBPipePattern pattern) { + final int[] filteredIndexes = + IntStream.range(0, node.getMeasurements().size()) + .filter( + index -> + pattern.matchesMeasurement( + node.getDevicePath().getFullPath(), node.getMeasurements().get(index))) + .toArray(); + return filteredIndexes.length > 0 + ? Optional.of( + new CreateAlignedTimeSeriesNode( + node.getPlanNodeId(), + node.getDevicePath(), + IoTDBPipePattern.applyIndexesOnList(filteredIndexes, node.getMeasurements()), + IoTDBPipePattern.applyIndexesOnList(filteredIndexes, node.getDataTypes()), + IoTDBPipePattern.applyIndexesOnList(filteredIndexes, node.getEncodings()), + IoTDBPipePattern.applyIndexesOnList(filteredIndexes, node.getCompressors()), + IoTDBPipePattern.applyIndexesOnList(filteredIndexes, node.getAliasList()), + IoTDBPipePattern.applyIndexesOnList(filteredIndexes, node.getTagsList()), + IoTDBPipePattern.applyIndexesOnList(filteredIndexes, node.getAttributesList()))) + : Optional.empty(); + } + + @Override + public Optional visitCreateMultiTimeSeries( + final CreateMultiTimeSeriesNode node, final IoTDBPipePattern pattern) { + final Map filteredMeasurementGroupMap = + node.getMeasurementGroupMap().entrySet().stream() + .filter(entry -> pattern.matchPrefixPath(entry.getKey().getFullPath())) + .map( + entry -> + new Pair<>( + entry.getKey(), + trimMeasurementGroup( + entry.getKey().getFullPath(), entry.getValue(), pattern))) + .filter(pair -> Objects.nonNull(pair.getRight())) + .collect(Collectors.toMap(Pair::getLeft, Pair::getRight)); + return !filteredMeasurementGroupMap.isEmpty() + ? Optional.of( + new CreateMultiTimeSeriesNode(node.getPlanNodeId(), filteredMeasurementGroupMap)) + : Optional.empty(); + } + + private static MeasurementGroup trimMeasurementGroup( + final String device, final MeasurementGroup group, final IoTDBPipePattern pattern) { + final int[] filteredIndexes = + IntStream.range(0, group.size()) + .filter(index -> pattern.matchesMeasurement(device, group.getMeasurements().get(index))) + .toArray(); + if (filteredIndexes.length == 0) { + return null; + } + final MeasurementGroup targetMeasurementGroup = new MeasurementGroup(); + Arrays.stream(filteredIndexes) + .forEach( + index -> { + targetMeasurementGroup.addMeasurement( + group.getMeasurements().get(index), + group.getDataTypes().get(index), + group.getEncodings().get(index), + group.getCompressors().get(index)); + targetMeasurementGroup.addTags(group.getTagsList().get(index)); + targetMeasurementGroup.addAttributes(group.getAttributesList().get(index)); + targetMeasurementGroup.addAlias(group.getAliasList().get(index)); + targetMeasurementGroup.addProps(group.getPropsList().get(index)); + }); + return targetMeasurementGroup; + } + + @Override + public Optional visitAlterTimeSeries( + final AlterTimeSeriesNode node, final IoTDBPipePattern pattern) { + return pattern.matchesMeasurement(node.getPath().getDevice(), node.getPath().getMeasurement()) + ? Optional.of(node) + : Optional.empty(); + } + + @Override + public Optional visitInternalCreateTimeSeries( + final InternalCreateTimeSeriesNode node, final IoTDBPipePattern pattern) { + final MeasurementGroup group = + pattern.matchPrefixPath(node.getDevicePath().getFullPath()) + ? trimMeasurementGroup( + node.getDevicePath().getFullPath(), node.getMeasurementGroup(), pattern) + : null; + return Objects.nonNull(group) + ? Optional.of( + new InternalCreateTimeSeriesNode( + node.getPlanNodeId(), node.getDevicePath(), group, node.isAligned())) + : Optional.empty(); + } + + @Override + public Optional visitActivateTemplate( + final ActivateTemplateNode node, final IoTDBPipePattern pattern) { + return pattern.matchDevice(node.getActivatePath().getFullPath()) + ? Optional.of(node) + : Optional.empty(); + } + + @Override + public Optional visitInternalBatchActivateTemplate( + final InternalBatchActivateTemplateNode node, final IoTDBPipePattern pattern) { + final Map> filteredTemplateActivationMap = + node.getTemplateActivationMap().entrySet().stream() + .filter(entry -> pattern.matchDevice(entry.getKey().getFullPath())) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + return !filteredTemplateActivationMap.isEmpty() + ? Optional.of( + new InternalBatchActivateTemplateNode( + node.getPlanNodeId(), filteredTemplateActivationMap)) + : Optional.empty(); + } + + @Override + public Optional visitInternalCreateMultiTimeSeries( + final InternalCreateMultiTimeSeriesNode node, final IoTDBPipePattern pattern) { + final Map> filteredDeviceMap = + node.getDeviceMap().entrySet().stream() + .filter(entry -> pattern.matchPrefixPath(entry.getKey().getFullPath())) + .map( + entry -> + new Pair<>( + entry.getKey(), + new Pair<>( + entry.getValue().getLeft(), + trimMeasurementGroup( + entry.getKey().getFullPath(), + entry.getValue().getRight(), + pattern)))) + .filter(pair -> Objects.nonNull(pair.getRight().getRight())) + .collect(Collectors.toMap(Pair::getLeft, Pair::getRight)); + return !filteredDeviceMap.isEmpty() + ? Optional.of( + new InternalCreateMultiTimeSeriesNode(node.getPlanNodeId(), filteredDeviceMap)) + : Optional.empty(); + } + + @Override + public Optional visitBatchActivateTemplate( + final BatchActivateTemplateNode node, final IoTDBPipePattern pattern) { + final Map> filteredTemplateActivationMap = + node.getTemplateActivationMap().entrySet().stream() + .filter(entry -> pattern.matchDevice(entry.getKey().getFullPath())) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + return !filteredTemplateActivationMap.isEmpty() + ? Optional.of( + new BatchActivateTemplateNode(node.getPlanNodeId(), filteredTemplateActivationMap)) + : Optional.empty(); + } + + @Override + public Optional visitCreateLogicalView( + final CreateLogicalViewNode node, final IoTDBPipePattern pattern) { + final Map filteredViewPathToSourceMap = + node.getViewPathToSourceExpressionMap().entrySet().stream() + .filter( + entry -> + pattern.matchesMeasurement( + entry.getKey().getDevice(), entry.getKey().getMeasurement())) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + return !filteredViewPathToSourceMap.isEmpty() + ? Optional.of(new CreateLogicalViewNode(node.getPlanNodeId(), filteredViewPathToSourceMap)) + : Optional.empty(); + } + + @Override + public Optional visitAlterLogicalView( + final AlterLogicalViewNode node, final IoTDBPipePattern pattern) { + final Map filteredViewPathToSourceMap = + node.getViewPathToSourceMap().entrySet().stream() + .filter( + entry -> + pattern.matchesMeasurement( + entry.getKey().getDevice(), entry.getKey().getMeasurement())) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + return !filteredViewPathToSourceMap.isEmpty() + ? Optional.of(new AlterLogicalViewNode(node.getPlanNodeId(), filteredViewPathToSourceMap)) + : Optional.empty(); + } + + @Override + public Optional visitDeleteData( + final DeleteDataNode node, final IoTDBPipePattern pattern) { + final List intersectedPaths = + node.getPathList().stream() + .map(pattern::getIntersection) + .flatMap(Collection::stream) + .distinct() + .collect(Collectors.toList()); + return !intersectedPaths.isEmpty() + ? Optional.of( + new DeleteDataNode( + node.getPlanNodeId(), + intersectedPaths, + node.getDeleteStartTime(), + node.getDeleteEndTime())) + : Optional.empty(); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/pattern/CachedSchemaPatternMatcher.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/pattern/CachedSchemaPatternMatcher.java index 1de43b7042e1..4be44a0fecc4 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/pattern/CachedSchemaPatternMatcher.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/pattern/CachedSchemaPatternMatcher.java @@ -61,7 +61,7 @@ public CachedSchemaPatternMatcher() { } @Override - public void register(PipeRealtimeDataRegionExtractor extractor) { + public void register(final PipeRealtimeDataRegionExtractor extractor) { lock.writeLock().lock(); try { extractors.add(extractor); @@ -72,7 +72,7 @@ public void register(PipeRealtimeDataRegionExtractor extractor) { } @Override - public void deregister(PipeRealtimeDataRegionExtractor extractor) { + public void deregister(final PipeRealtimeDataRegionExtractor extractor) { lock.writeLock().lock(); try { extractors.remove(extractor); @@ -93,7 +93,7 @@ public int getRegisterCount() { } @Override - public Set match(PipeRealtimeEvent event) { + public Set match(final PipeRealtimeEvent event) { final Set matchedExtractors = new HashSet<>(); lock.readLock().lock(); @@ -176,10 +176,10 @@ public Set match(PipeRealtimeEvent event) { return matchedExtractors; } - protected Set filterExtractorsByDevice(String device) { + protected Set filterExtractorsByDevice(final String device) { final Set filteredExtractors = new HashSet<>(); - for (PipeRealtimeDataRegionExtractor extractor : extractors) { + for (final PipeRealtimeDataRegionExtractor extractor : extractors) { // Return if the extractor only extract deletion if (!extractor.shouldExtractInsertion()) { continue; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/thrift/IoTDBDataNodeReceiver.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/thrift/IoTDBDataNodeReceiver.java index dfd8be444b64..55fe372ea8f3 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/thrift/IoTDBDataNodeReceiver.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/thrift/IoTDBDataNodeReceiver.java @@ -28,6 +28,7 @@ import org.apache.iotdb.commons.pipe.connector.payload.thrift.request.PipeTransferCompressedReq; import org.apache.iotdb.commons.pipe.connector.payload.thrift.request.PipeTransferFileSealReqV1; import org.apache.iotdb.commons.pipe.connector.payload.thrift.request.PipeTransferFileSealReqV2; +import org.apache.iotdb.commons.pipe.pattern.IoTDBPipePattern; import org.apache.iotdb.commons.pipe.receiver.IoTDBFileReceiver; import org.apache.iotdb.db.auth.AuthorityChecker; import org.apache.iotdb.db.conf.IoTDBConfig; @@ -50,6 +51,7 @@ import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionSnapshotEvent; import org.apache.iotdb.db.pipe.receiver.visitor.PipePlanToStatementVisitor; import org.apache.iotdb.db.pipe.receiver.visitor.PipeStatementExceptionVisitor; +import org.apache.iotdb.db.pipe.receiver.visitor.PipeStatementPatternParseVisitor; import org.apache.iotdb.db.pipe.receiver.visitor.PipeStatementTSStatusVisitor; import org.apache.iotdb.db.pipe.receiver.visitor.PipeStatementToBatchVisitor; import org.apache.iotdb.db.protocol.session.SessionManager; @@ -112,6 +114,8 @@ public class IoTDBDataNodeReceiver extends IoTDBFileReceiver { new PipeStatementTSStatusVisitor(); private static final PipeStatementExceptionVisitor STATEMENT_EXCEPTION_VISITOR = new PipeStatementExceptionVisitor(); + private static final PipeStatementPatternParseVisitor STATEMENT_PATTERN_PARSE_VISITOR = + new PipeStatementPatternParseVisitor(); private final PipeStatementToBatchVisitor batchVisitor = new PipeStatementToBatchVisitor(); // Used for data transfer: confignode (cluster A) -> datanode (cluster B) -> confignode (cluster @@ -207,7 +211,7 @@ public synchronized TPipeTransferResp receive(final TPipeTransferReq req) { receiverId.get(), status); return new TPipeTransferResp(status); - } catch (Exception e) { + } catch (final Exception e) { final String error = String.format("Exception %s encountered while handling request %s.", e.getMessage(), req); LOGGER.warn("Receiver id = {}: {}", receiverId.get(), error, e); @@ -303,6 +307,8 @@ private TSStatus loadSchemaSnapShot( final Set executionTypes = PipeSchemaRegionSnapshotEvent.getStatementTypeSet( parameters.get(ColumnHeaderConstant.TYPE)); + final IoTDBPipePattern pattern = + new IoTDBPipePattern(parameters.get(ColumnHeaderConstant.PATH_PATTERN)); // Clear to avoid previous exceptions batchVisitor.clear(); @@ -316,8 +322,9 @@ private TSStatus loadSchemaSnapShot( // The statements do not contain AlterLogicalViewStatements // Here we apply the statements as many as possible // Even if there are failed statements - batchVisitor - .process(originalStatement, null) + STATEMENT_PATTERN_PARSE_VISITOR + .process(originalStatement, pattern) + .flatMap(parsedStatement -> batchVisitor.process(parsedStatement, null)) .ifPresent(statement -> results.add(executeStatementAndClassifyExceptions(statement))); } batchVisitor.getRemainBatches().stream() diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/visitor/PipePlanToStatementVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/visitor/PipePlanToStatementVisitor.java index 32e3e6947ed8..593e3715c88e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/visitor/PipePlanToStatementVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/visitor/PipePlanToStatementVisitor.java @@ -257,7 +257,7 @@ public CreateLogicalViewStatement visitCreateLogicalView( // We do not support AlterLogicalViewNode parsing and use direct rpc instead @Override - public DeleteDataStatement visitDeleteData(DeleteDataNode node, final Void context) { + public DeleteDataStatement visitDeleteData(final DeleteDataNode node, final Void context) { final DeleteDataStatement statement = new DeleteDataStatement(); statement.setDeleteEndTime(node.getDeleteEndTime()); statement.setDeleteStartTime(node.getDeleteStartTime()); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/visitor/PipeStatementPatternParseVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/visitor/PipeStatementPatternParseVisitor.java new file mode 100644 index 000000000000..1fd404375f4a --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/visitor/PipeStatementPatternParseVisitor.java @@ -0,0 +1,136 @@ +/* + * 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.db.pipe.receiver.visitor; + +import org.apache.iotdb.commons.pipe.pattern.IoTDBPipePattern; +import org.apache.iotdb.db.queryengine.plan.statement.Statement; +import org.apache.iotdb.db.queryengine.plan.statement.StatementNode; +import org.apache.iotdb.db.queryengine.plan.statement.StatementVisitor; +import org.apache.iotdb.db.queryengine.plan.statement.metadata.CreateAlignedTimeSeriesStatement; +import org.apache.iotdb.db.queryengine.plan.statement.metadata.CreateTimeSeriesStatement; +import org.apache.iotdb.db.queryengine.plan.statement.metadata.template.ActivateTemplateStatement; +import org.apache.iotdb.db.queryengine.plan.statement.metadata.view.CreateLogicalViewStatement; +import org.apache.iotdb.db.tools.schema.SRStatementGenerator; + +import java.util.Arrays; +import java.util.Optional; +import java.util.stream.IntStream; + +/** + * The {@link PipeStatementPatternParseVisitor} will transform the schema {@link Statement}s using + * {@link IoTDBPipePattern}. Rule: + * + *

1. All patterns in the output {@link Statement} will be the intersection of the original + * {@link Statement}'s patterns and the given {@link IoTDBPipePattern}. + * + *

2. If a pattern does not intersect with the {@link IoTDBPipePattern}, it's dropped. + * + *

3. If all the patterns in the {@link Statement} is dropped, the {@link Statement} is dropped. + * + *

4. The output {@link Statement} can be the altered form of the original one because it's read + * from the {@link SRStatementGenerator} and will no longer be used. + */ +public class PipeStatementPatternParseVisitor + extends StatementVisitor, IoTDBPipePattern> { + @Override + public Optional visitNode( + final StatementNode statement, final IoTDBPipePattern pattern) { + return Optional.of((Statement) statement); + } + + @Override + public Optional visitCreateTimeseries( + final CreateTimeSeriesStatement statement, final IoTDBPipePattern pattern) { + return pattern.matchesMeasurement( + statement.getPath().getDevice(), statement.getPath().getMeasurement()) + ? Optional.of(statement) + : Optional.empty(); + } + + @Override + public Optional visitCreateAlignedTimeseries( + final CreateAlignedTimeSeriesStatement statement, final IoTDBPipePattern pattern) { + final int[] filteredIndexes = + IntStream.range(0, statement.getMeasurements().size()) + .filter( + index -> + pattern.matchesMeasurement( + statement.getDevicePath().getFullPath(), + statement.getMeasurements().get(index))) + .toArray(); + if (filteredIndexes.length == 0) { + return Optional.empty(); + } + final CreateAlignedTimeSeriesStatement targetCreateAlignedTimeSeriesStatement = + new CreateAlignedTimeSeriesStatement(); + targetCreateAlignedTimeSeriesStatement.setDevicePath(statement.getDevicePath()); + Arrays.stream(filteredIndexes) + .forEach( + index -> { + targetCreateAlignedTimeSeriesStatement.addMeasurement( + statement.getMeasurements().get(index)); + targetCreateAlignedTimeSeriesStatement.addDataType( + statement.getDataTypes().get(index)); + targetCreateAlignedTimeSeriesStatement.addEncoding( + statement.getEncodings().get(index)); + targetCreateAlignedTimeSeriesStatement.addCompressor( + statement.getCompressors().get(index)); + targetCreateAlignedTimeSeriesStatement.addTagsList( + statement.getTagsList().get(index)); + targetCreateAlignedTimeSeriesStatement.addAttributesList( + statement.getAttributesList().get(index)); + targetCreateAlignedTimeSeriesStatement.addAliasList( + statement.getAliasList().get(index)); + }); + return Optional.of(targetCreateAlignedTimeSeriesStatement); + } + + @Override + public Optional visitActivateTemplate( + final ActivateTemplateStatement activateTemplateStatement, final IoTDBPipePattern pattern) { + return pattern.matchDevice(activateTemplateStatement.getPath().getFullPath()) + ? Optional.of(activateTemplateStatement) + : Optional.empty(); + } + + @Override + public Optional visitCreateLogicalView( + final CreateLogicalViewStatement createLogicalViewStatement, final IoTDBPipePattern pattern) { + final int[] filteredIndexes = + IntStream.range(0, createLogicalViewStatement.getTargetPathList().size()) + .filter( + index -> + pattern.matchesMeasurement( + createLogicalViewStatement.getTargetPathList().get(index).getDevice(), + createLogicalViewStatement.getTargetPathList().get(index).getMeasurement())) + .toArray(); + if (filteredIndexes.length == 0) { + return Optional.empty(); + } + createLogicalViewStatement.setTargetFullPaths( + IoTDBPipePattern.applyIndexesOnList( + filteredIndexes, createLogicalViewStatement.getTargetPathList())); + createLogicalViewStatement.setViewExpressions( + IoTDBPipePattern.applyIndexesOnList( + filteredIndexes, createLogicalViewStatement.getViewExpressions())); + + return Optional.of(createLogicalViewStatement); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/connection/PipeEventCollector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/connection/PipeEventCollector.java index cdd735985719..a3f2227e126b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/connection/PipeEventCollector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/connection/PipeEventCollector.java @@ -21,12 +21,15 @@ import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.commons.pipe.event.ProgressReportEvent; +import org.apache.iotdb.commons.pipe.pattern.IoTDBPipePattern; import org.apache.iotdb.commons.pipe.progress.PipeEventCommitManager; import org.apache.iotdb.commons.pipe.task.connection.UnboundedBlockingPendingQueue; import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent; +import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionWritePlanEvent; import org.apache.iotdb.db.pipe.event.common.tablet.PipeInsertNodeTabletInsertionEvent; import org.apache.iotdb.db.pipe.event.common.tablet.PipeRawTabletInsertionEvent; import org.apache.iotdb.db.pipe.event.common.tsfile.PipeTsFileInsertionEvent; +import org.apache.iotdb.db.pipe.extractor.schemaregion.IoTDBSchemaRegionExtractor; import org.apache.iotdb.pipe.api.collector.EventCollector; import org.apache.iotdb.pipe.api.event.Event; import org.apache.iotdb.pipe.api.event.dml.insertion.TabletInsertionEvent; @@ -67,6 +70,8 @@ public void collect(final Event event) { parseAndCollectEvent((PipeRawTabletInsertionEvent) event); } else if (event instanceof PipeTsFileInsertionEvent) { parseAndCollectEvent((PipeTsFileInsertionEvent) event); + } else if (event instanceof PipeSchemaRegionWritePlanEvent) { + parseAndCollectEvent((PipeSchemaRegionWritePlanEvent) event); } else if (!(event instanceof ProgressReportEvent)) { collectEvent(event); } @@ -121,6 +126,20 @@ private void parseAndCollectEvent(final PipeTsFileInsertionEvent sourceEvent) th } } + private void parseAndCollectEvent(final PipeSchemaRegionWritePlanEvent deleteDataEvent) { + IoTDBSchemaRegionExtractor.PATTERN_PARSE_VISITOR + .process(deleteDataEvent.getPlanNode(), (IoTDBPipePattern) deleteDataEvent.getPipePattern()) + .map( + planNode -> + new PipeSchemaRegionWritePlanEvent( + planNode, + deleteDataEvent.getPipeName(), + deleteDataEvent.getPipeTaskMeta(), + deleteDataEvent.getPipePattern(), + deleteDataEvent.isGeneratedByPipe())) + .ifPresent(this::collectEvent); + } + private void collectEvent(final Event event) { collectInvocationCount.incrementAndGet(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/DeleteDataNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/DeleteDataNode.java index 08a80133926c..877187be5a63 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/DeleteDataNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/DeleteDataNode.java @@ -48,6 +48,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.stream.Collectors; import static org.apache.iotdb.commons.conf.IoTDBConstant.MULTI_LEVEL_PATH_WILDCARD; @@ -251,6 +252,26 @@ public void setRegionReplicaSet(TRegionReplicaSet regionReplicaSet) { this.regionReplicaSet = regionReplicaSet; } + @Override + public boolean equals(final Object obj) { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + final DeleteDataNode that = (DeleteDataNode) obj; + return this.getPlanNodeId().equals(that.getPlanNodeId()) + && Objects.equals(this.pathList, that.pathList) + && Objects.equals(this.deleteStartTime, that.deleteStartTime) + && Objects.equals(this.deleteEndTime, that.deleteEndTime); + } + + @Override + public int hashCode() { + return Objects.hash(getPlanNodeId(), pathList, deleteStartTime, deleteEndTime); + } + public String toString() { return String.format( "DeleteDataNode-%s[ Paths: %s, Region: %s ]", diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/CreateAlignedTimeSeriesStatement.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/CreateAlignedTimeSeriesStatement.java index 5c441253697f..81ee69f0bd2c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/CreateAlignedTimeSeriesStatement.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/CreateAlignedTimeSeriesStatement.java @@ -35,6 +35,7 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.Objects; /** * CREATE ALIGNED TIMESERIES statement. @@ -178,4 +179,58 @@ public void addAttributesList(Map attributes) { public R accept(StatementVisitor visitor, C context) { return visitor.visitCreateAlignedTimeseries(this, context); } + + @Override + public boolean equals(final Object obj) { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + final CreateAlignedTimeSeriesStatement that = (CreateAlignedTimeSeriesStatement) obj; + return Objects.equals(this.devicePath, that.devicePath) + && Objects.equals(this.measurements, that.measurements) + && Objects.equals(this.dataTypes, that.dataTypes) + && Objects.equals(this.encodings, that.encodings) + && Objects.equals(this.compressors, that.compressors) + && Objects.equals(this.aliasList, that.aliasList) + && Objects.equals(this.tagsList, that.tagsList) + && Objects.equals(this.attributesList, that.attributesList); + } + + @Override + public int hashCode() { + return Objects.hash( + devicePath, + measurements, + dataTypes, + encodings, + compressors, + aliasList, + tagsList, + attributesList); + } + + @Override + public String toString() { + return "CreateAlignedTimeSeriesStatement{" + + "devicePath='" + + devicePath + + "', measurements=" + + measurements + + ", dataTypes='" + + dataTypes + + "', encodings=" + + encodings + + "', compressors=" + + compressors + + "', aliasList=" + + aliasList + + "', tagsList=" + + tagsList + + "', attributesList=" + + attributesList + + "'}"; + } } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/connector/PipeDataNodeThriftRequestTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/connector/PipeDataNodeThriftRequestTest.java index fd3e06f45881..0d8e5718c9eb 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/connector/PipeDataNodeThriftRequestTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/connector/PipeDataNodeThriftRequestTest.java @@ -323,7 +323,7 @@ public void testPipeTransferSchemaSnapshotSealReq() throws IOException { PipeTransferSchemaSnapshotSealReq req = PipeTransferSchemaSnapshotSealReq.toTPipeTransferReq( - mTreeSnapshotName, 100, tLogName, 10, databaseName, typeString); + "root.**", mTreeSnapshotName, 100, tLogName, 10, databaseName, typeString); PipeTransferSchemaSnapshotSealReq deserializeReq = PipeTransferSchemaSnapshotSealReq.fromTPipeTransferReq(req); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/connector/PipeStatementPatternParseVisitorTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/connector/PipeStatementPatternParseVisitorTest.java new file mode 100644 index 000000000000..b11e9a0e3e6f --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/connector/PipeStatementPatternParseVisitorTest.java @@ -0,0 +1,161 @@ +/* + * 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.db.pipe.connector; + +import org.apache.iotdb.commons.exception.IllegalPathException; +import org.apache.iotdb.commons.path.PartialPath; +import org.apache.iotdb.commons.pipe.pattern.IoTDBPipePattern; +import org.apache.iotdb.commons.schema.view.viewExpression.leaf.TimeSeriesViewOperand; +import org.apache.iotdb.db.pipe.receiver.visitor.PipeStatementPatternParseVisitor; +import org.apache.iotdb.db.queryengine.plan.statement.metadata.CreateAlignedTimeSeriesStatement; +import org.apache.iotdb.db.queryengine.plan.statement.metadata.CreateTimeSeriesStatement; +import org.apache.iotdb.db.queryengine.plan.statement.metadata.template.ActivateTemplateStatement; +import org.apache.iotdb.db.queryengine.plan.statement.metadata.view.CreateLogicalViewStatement; + +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.file.metadata.enums.CompressionType; +import org.apache.tsfile.file.metadata.enums.TSEncoding; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; + +public class PipeStatementPatternParseVisitorTest { + + private final IoTDBPipePattern prefixPathPattern = new IoTDBPipePattern("root.db.device.**"); + private final IoTDBPipePattern fullPathPattern = new IoTDBPipePattern("root.db.device.s1"); + + @Test + public void testCreateTimeSeries() throws IllegalPathException { + final CreateTimeSeriesStatement createTimeSeriesStatement = new CreateTimeSeriesStatement(); + createTimeSeriesStatement.setPath(new PartialPath("root.db.device.s1")); + createTimeSeriesStatement.setDataType(TSDataType.FLOAT); + createTimeSeriesStatement.setEncoding(TSEncoding.RLE); + createTimeSeriesStatement.setCompressor(CompressionType.SNAPPY); + createTimeSeriesStatement.setProps(new HashMap<>()); + createTimeSeriesStatement.setTags(new HashMap<>()); + createTimeSeriesStatement.setAttributes(new HashMap<>()); + createTimeSeriesStatement.setAlias("a1"); + + final CreateTimeSeriesStatement createTimeSeriesStatementToFilter = + new CreateTimeSeriesStatement(); + createTimeSeriesStatementToFilter.setPath(new PartialPath("root.db1.device.s1")); + createTimeSeriesStatementToFilter.setDataType(TSDataType.FLOAT); + createTimeSeriesStatementToFilter.setEncoding(TSEncoding.RLE); + createTimeSeriesStatementToFilter.setCompressor(CompressionType.SNAPPY); + createTimeSeriesStatementToFilter.setProps(new HashMap<>()); + createTimeSeriesStatementToFilter.setTags(new HashMap<>()); + createTimeSeriesStatementToFilter.setAttributes(new HashMap<>()); + createTimeSeriesStatementToFilter.setAlias("a2"); + + Assert.assertEquals( + createTimeSeriesStatement, + new PipeStatementPatternParseVisitor() + .visitCreateTimeseries(createTimeSeriesStatement, prefixPathPattern) + .orElseThrow(AssertionError::new)); + Assert.assertFalse( + new PipeStatementPatternParseVisitor() + .visitCreateTimeseries(createTimeSeriesStatementToFilter, prefixPathPattern) + .isPresent()); + } + + @Test + public void testCreateAlignedTimeSeries() throws IllegalPathException { + final CreateAlignedTimeSeriesStatement expectedCreateAlignedTimeSeriesStatement = + new CreateAlignedTimeSeriesStatement(); + expectedCreateAlignedTimeSeriesStatement.setDevicePath(new PartialPath("root.db.device")); + expectedCreateAlignedTimeSeriesStatement.setMeasurements(Collections.singletonList("s1")); + expectedCreateAlignedTimeSeriesStatement.setDataTypes( + Collections.singletonList(TSDataType.FLOAT)); + expectedCreateAlignedTimeSeriesStatement.setEncodings( + Collections.singletonList(TSEncoding.RLE)); + expectedCreateAlignedTimeSeriesStatement.setCompressors( + Collections.singletonList(CompressionType.SNAPPY)); + expectedCreateAlignedTimeSeriesStatement.setTagsList( + Collections.singletonList(new HashMap<>())); + expectedCreateAlignedTimeSeriesStatement.setAttributesList( + Collections.singletonList(new HashMap<>())); + expectedCreateAlignedTimeSeriesStatement.setAliasList(Collections.singletonList("a1")); + + final CreateAlignedTimeSeriesStatement originalCreateAlignedTimeSeriesStatement = + new CreateAlignedTimeSeriesStatement(); + originalCreateAlignedTimeSeriesStatement.setDevicePath(new PartialPath("root.db.device")); + originalCreateAlignedTimeSeriesStatement.setMeasurements(Arrays.asList("s1", "s2")); + originalCreateAlignedTimeSeriesStatement.setDataTypes( + Arrays.asList(TSDataType.FLOAT, TSDataType.BOOLEAN)); + originalCreateAlignedTimeSeriesStatement.setEncodings( + Arrays.asList(TSEncoding.RLE, TSEncoding.PLAIN)); + originalCreateAlignedTimeSeriesStatement.setCompressors( + Arrays.asList(CompressionType.SNAPPY, CompressionType.SNAPPY)); + originalCreateAlignedTimeSeriesStatement.setTagsList( + Arrays.asList(new HashMap<>(), new HashMap<>())); + originalCreateAlignedTimeSeriesStatement.setAttributesList( + Arrays.asList(new HashMap<>(), new HashMap<>())); + originalCreateAlignedTimeSeriesStatement.setAliasList(Arrays.asList("a1", "a2")); + + Assert.assertEquals( + expectedCreateAlignedTimeSeriesStatement, + new PipeStatementPatternParseVisitor() + .visitCreateAlignedTimeseries(originalCreateAlignedTimeSeriesStatement, fullPathPattern) + .orElseThrow(AssertionError::new)); + } + + @Test + public void testActivateTemplate() throws IllegalPathException { + final ActivateTemplateStatement activateTemplateStatement = + new ActivateTemplateStatement(new PartialPath("root.db.device")); + final ActivateTemplateStatement activateTemplateStatementToFilter = + new ActivateTemplateStatement(new PartialPath("root.db")); + + Assert.assertEquals( + activateTemplateStatement, + new PipeStatementPatternParseVisitor() + .visitActivateTemplate(activateTemplateStatement, prefixPathPattern) + .orElseThrow(AssertionError::new)); + Assert.assertFalse( + new PipeStatementPatternParseVisitor() + .visitActivateTemplate(activateTemplateStatementToFilter, prefixPathPattern) + .isPresent()); + } + + @Test + public void testCreateLogicalView() throws IllegalPathException { + final CreateLogicalViewStatement createLogicalViewStatement = new CreateLogicalViewStatement(); + createLogicalViewStatement.setTargetFullPaths( + Arrays.asList(new PartialPath("root.db.device.a1"), new PartialPath("root.db1.device.a1"))); + createLogicalViewStatement.setViewExpressions( + Arrays.asList( + new TimeSeriesViewOperand("root.sg1.d1"), new TimeSeriesViewOperand("root.sg1.d2"))); + + final CreateLogicalViewStatement targetLogicalViewStatement = + (CreateLogicalViewStatement) + new PipeStatementPatternParseVisitor() + .visitCreateLogicalView(createLogicalViewStatement, prefixPathPattern) + .orElseThrow(AssertionError::new); + Assert.assertEquals( + Collections.singletonList(new PartialPath("root.db.device.a1")), + targetLogicalViewStatement.getPaths()); + Assert.assertEquals( + Collections.singletonList(new TimeSeriesViewOperand("root.sg1.d1")), + targetLogicalViewStatement.getViewExpressions()); + } +} diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/extractor/IoTDBDataRegionExtractorTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/extractor/IoTDBDataRegionExtractorTest.java index bdf63161b966..8b133b673913 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/extractor/IoTDBDataRegionExtractorTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/extractor/IoTDBDataRegionExtractorTest.java @@ -32,8 +32,7 @@ public class IoTDBDataRegionExtractorTest { @Test public void testIoTDBDataRegionExtractor() { - final IoTDBDataRegionExtractor extractor = new IoTDBDataRegionExtractor(); - try { + try (final IoTDBDataRegionExtractor extractor = new IoTDBDataRegionExtractor()) { extractor.validate( new PipeParameterValidator( new PipeParameters( @@ -102,4 +101,19 @@ public Exception testIoTDBDataRegionExtractorWithPattern(final String pattern) { } return null; } + + @Test(expected = IllegalArgumentException.class) + public void testIoTDBDataRegionExtractorWithDeletionAndPattern() throws Exception { + try (final IoTDBDataRegionExtractor extractor = new IoTDBDataRegionExtractor()) { + extractor.validate( + new PipeParameterValidator( + new PipeParameters( + new HashMap() { + { + put(PipeExtractorConstant.EXTRACTOR_PATTERN_KEY, "root.db"); + put(PipeExtractorConstant.EXTRACTOR_INCLUSION_KEY, "data.delete"); + } + }))); + } + } } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/extractor/PipePlanPatternParseVisitorTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/extractor/PipePlanPatternParseVisitorTest.java new file mode 100644 index 000000000000..2cd3e02822d0 --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/extractor/PipePlanPatternParseVisitorTest.java @@ -0,0 +1,428 @@ +/* + * 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.db.pipe.extractor; + +import org.apache.iotdb.commons.exception.IllegalPathException; +import org.apache.iotdb.commons.path.PartialPath; +import org.apache.iotdb.commons.pipe.pattern.IoTDBPipePattern; +import org.apache.iotdb.commons.schema.view.viewExpression.ViewExpression; +import org.apache.iotdb.commons.schema.view.viewExpression.leaf.TimeSeriesViewOperand; +import org.apache.iotdb.db.pipe.extractor.schemaregion.IoTDBSchemaRegionExtractor; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.metedata.write.ActivateTemplateNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.metedata.write.AlterTimeSeriesNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.metedata.write.BatchActivateTemplateNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.metedata.write.CreateAlignedTimeSeriesNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.metedata.write.CreateMultiTimeSeriesNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.metedata.write.CreateTimeSeriesNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.metedata.write.InternalBatchActivateTemplateNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.metedata.write.InternalCreateMultiTimeSeriesNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.metedata.write.InternalCreateTimeSeriesNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.metedata.write.MeasurementGroup; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.metedata.write.view.AlterLogicalViewNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.metedata.write.view.CreateLogicalViewNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.DeleteDataNode; +import org.apache.iotdb.db.queryengine.plan.statement.metadata.AlterTimeSeriesStatement; + +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.file.metadata.enums.CompressionType; +import org.apache.tsfile.file.metadata.enums.TSEncoding; +import org.apache.tsfile.utils.Pair; +import org.junit.Assert; +import org.junit.Test; + +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +public class PipePlanPatternParseVisitorTest { + private final IoTDBPipePattern prefixPathPattern = new IoTDBPipePattern("root.db.device.**"); + private final IoTDBPipePattern fullPathPattern = new IoTDBPipePattern("root.db.device.s1"); + + @Test + public void testCreateTimeSeries() throws IllegalPathException { + final CreateTimeSeriesNode createTimeSeriesNode = + new CreateTimeSeriesNode( + new PlanNodeId("2024-04-30-1"), + new PartialPath("root.db.device.s1"), + TSDataType.FLOAT, + TSEncoding.RLE, + CompressionType.SNAPPY, + new HashMap<>(), + new HashMap<>(), + new HashMap<>(), + "a1"); + final CreateTimeSeriesNode createTimeSeriesNodeToFilter = + new CreateTimeSeriesNode( + new PlanNodeId("2024-04-30-2"), + new PartialPath("root.db1.device.s1"), + TSDataType.FLOAT, + TSEncoding.RLE, + CompressionType.SNAPPY, + new HashMap<>(), + new HashMap<>(), + new HashMap<>(), + "a1"); + + Assert.assertEquals( + createTimeSeriesNode, + IoTDBSchemaRegionExtractor.PATTERN_PARSE_VISITOR + .visitCreateTimeSeries(createTimeSeriesNode, prefixPathPattern) + .orElseThrow(AssertionError::new)); + Assert.assertFalse( + IoTDBSchemaRegionExtractor.PATTERN_PARSE_VISITOR + .visitCreateTimeSeries(createTimeSeriesNodeToFilter, prefixPathPattern) + .isPresent()); + } + + @Test + public void testCreateAlignedTimeSeries() throws IllegalPathException { + Assert.assertEquals( + new CreateAlignedTimeSeriesNode( + new PlanNodeId("2024-04-30-1"), + new PartialPath("root.db.device"), + Collections.singletonList("s1"), + Collections.singletonList(TSDataType.FLOAT), + Collections.singletonList(TSEncoding.RLE), + Collections.singletonList(CompressionType.SNAPPY), + Collections.singletonList("a1"), + Collections.singletonList(new HashMap<>()), + Collections.singletonList(new HashMap<>())), + IoTDBSchemaRegionExtractor.PATTERN_PARSE_VISITOR + .visitCreateAlignedTimeSeries( + new CreateAlignedTimeSeriesNode( + new PlanNodeId("2024-04-30-1"), + new PartialPath("root.db.device"), + Arrays.asList("s1", "s2"), + Arrays.asList(TSDataType.FLOAT, TSDataType.BOOLEAN), + Arrays.asList(TSEncoding.RLE, TSEncoding.PLAIN), + Arrays.asList(CompressionType.SNAPPY, CompressionType.SNAPPY), + Arrays.asList("a1", "a2"), + Arrays.asList(new HashMap<>(), new HashMap<>()), + Arrays.asList(new HashMap<>(), new HashMap<>())), + fullPathPattern) + .orElseThrow(AssertionError::new)); + } + + @Test + public void testCreateMultiTimeSeries() throws IllegalPathException { + Assert.assertEquals( + new CreateMultiTimeSeriesNode( + new PlanNodeId("2024-04-30-1"), + Collections.singletonList(new PartialPath("root.db.device.s1")), + Collections.singletonList(TSDataType.FLOAT), + Collections.singletonList(TSEncoding.RLE), + Collections.singletonList(CompressionType.SNAPPY), + Collections.singletonList(new HashMap<>()), + Collections.singletonList("a1"), + Collections.singletonList(new HashMap<>()), + Collections.singletonList(new HashMap<>())), + IoTDBSchemaRegionExtractor.PATTERN_PARSE_VISITOR + .visitCreateMultiTimeSeries( + new CreateMultiTimeSeriesNode( + new PlanNodeId("2024-04-30-1"), + Arrays.asList( + new PartialPath("root.db.device.s1"), + new PartialPath("root.db1.device.s1")), + Arrays.asList(TSDataType.FLOAT, TSDataType.BOOLEAN), + Arrays.asList(TSEncoding.RLE, TSEncoding.PLAIN), + Arrays.asList(CompressionType.SNAPPY, CompressionType.SNAPPY), + Arrays.asList(new HashMap<>(), new HashMap<>()), + Arrays.asList("a1", "a2"), + Arrays.asList(new HashMap<>(), new HashMap<>()), + Arrays.asList(new HashMap<>(), new HashMap<>())), + fullPathPattern) + .orElseThrow(AssertionError::new)); + } + + @Test + public void testAlterTimeSeries() throws IllegalPathException { + final Map attributesMap = + new HashMap() { + { + put("k1", "v1"); + } + }; + final AlterTimeSeriesNode alterTimeSeriesNode = + new AlterTimeSeriesNode( + new PlanNodeId("2024-04-30-1"), + new PartialPath("root.db.device.s1"), + AlterTimeSeriesStatement.AlterType.ADD_ATTRIBUTES, + attributesMap, + "", + new HashMap<>(), + attributesMap, + false); + final AlterTimeSeriesNode alterTimeSeriesNodeToFilter = + new AlterTimeSeriesNode( + new PlanNodeId("2024-04-30-2"), + new PartialPath("root.db1.device.s1"), + AlterTimeSeriesStatement.AlterType.ADD_ATTRIBUTES, + attributesMap, + "", + new HashMap<>(), + attributesMap, + false); + + Assert.assertEquals( + alterTimeSeriesNode, + IoTDBSchemaRegionExtractor.PATTERN_PARSE_VISITOR + .visitAlterTimeSeries(alterTimeSeriesNode, prefixPathPattern) + .orElseThrow(AssertionError::new)); + Assert.assertFalse( + IoTDBSchemaRegionExtractor.PATTERN_PARSE_VISITOR + .visitAlterTimeSeries(alterTimeSeriesNodeToFilter, prefixPathPattern) + .isPresent()); + } + + @Test + public void testInternalCreateTimeSeries() throws IllegalPathException { + final MeasurementGroup expectedMeasurementGroup = new MeasurementGroup(); + expectedMeasurementGroup.addMeasurement( + "s1", TSDataType.FLOAT, TSEncoding.RLE, CompressionType.SNAPPY); + expectedMeasurementGroup.addProps(new HashMap<>()); + expectedMeasurementGroup.addAlias("a1"); + expectedMeasurementGroup.addTags(new HashMap<>()); + expectedMeasurementGroup.addAttributes(new HashMap<>()); + + final ByteBuffer byteBuffer = ByteBuffer.allocate(1000); + expectedMeasurementGroup.serialize(byteBuffer); + byteBuffer.flip(); + final MeasurementGroup originalMeasurementGroup = new MeasurementGroup(); + originalMeasurementGroup.deserialize(byteBuffer); + + originalMeasurementGroup.addMeasurement( + "s2", TSDataType.BOOLEAN, TSEncoding.PLAIN, CompressionType.SNAPPY); + originalMeasurementGroup.addProps(new HashMap<>()); + originalMeasurementGroup.addAlias("a2"); + originalMeasurementGroup.addTags(new HashMap<>()); + originalMeasurementGroup.addAttributes(new HashMap<>()); + + Assert.assertEquals( + new InternalCreateTimeSeriesNode( + new PlanNodeId("2024-04-30-1"), + new PartialPath("root.db.device"), + expectedMeasurementGroup, + true), + IoTDBSchemaRegionExtractor.PATTERN_PARSE_VISITOR + .visitInternalCreateTimeSeries( + new InternalCreateTimeSeriesNode( + new PlanNodeId("2024-04-30-1"), + new PartialPath("root.db.device"), + originalMeasurementGroup, + true), + fullPathPattern) + .orElseThrow(AssertionError::new)); + } + + @Test + public void testActivateTemplate() throws IllegalPathException { + final ActivateTemplateNode activateTemplateNode = + new ActivateTemplateNode( + new PlanNodeId("2024-04-30-1"), new PartialPath("root.db.device"), 3, 1); + final ActivateTemplateNode activateTemplateNodeToFilter = + new ActivateTemplateNode(new PlanNodeId("2024-04-30-2"), new PartialPath("root.db"), 2, 1); + + Assert.assertEquals( + activateTemplateNode, + IoTDBSchemaRegionExtractor.PATTERN_PARSE_VISITOR + .visitActivateTemplate(activateTemplateNode, prefixPathPattern) + .orElseThrow(AssertionError::new)); + Assert.assertFalse( + IoTDBSchemaRegionExtractor.PATTERN_PARSE_VISITOR + .visitActivateTemplate(activateTemplateNodeToFilter, prefixPathPattern) + .isPresent()); + } + + @Test + public void testInternalBatchActivateTemplate() throws IllegalPathException { + Assert.assertEquals( + new InternalBatchActivateTemplateNode( + new PlanNodeId("2024-04-30-1"), + new HashMap>() { + { + put(new PartialPath("root.db.device"), new Pair<>(1, 1)); + } + }), + IoTDBSchemaRegionExtractor.PATTERN_PARSE_VISITOR + .visitInternalBatchActivateTemplate( + new InternalBatchActivateTemplateNode( + new PlanNodeId("2024-04-30-1"), + new HashMap>() { + { + put(new PartialPath("root.db.device"), new Pair<>(1, 1)); + put(new PartialPath("root.db"), new Pair<>(2, 2)); + } + }), + fullPathPattern) + .orElseThrow(AssertionError::new)); + } + + @Test + public void testInternalCreateMultiTimeSeries() throws IllegalPathException { + final MeasurementGroup expectedMeasurementGroup = new MeasurementGroup(); + expectedMeasurementGroup.addMeasurement( + "s1", TSDataType.FLOAT, TSEncoding.RLE, CompressionType.SNAPPY); + expectedMeasurementGroup.addProps(new HashMap<>()); + expectedMeasurementGroup.addAlias("a1"); + expectedMeasurementGroup.addTags(new HashMap<>()); + expectedMeasurementGroup.addAttributes(new HashMap<>()); + + final ByteBuffer byteBuffer = ByteBuffer.allocate(1000); + expectedMeasurementGroup.serialize(byteBuffer); + byteBuffer.flip(); + final MeasurementGroup originalMeasurementGroup = new MeasurementGroup(); + originalMeasurementGroup.deserialize(byteBuffer); + + originalMeasurementGroup.addMeasurement( + "s2", TSDataType.BOOLEAN, TSEncoding.PLAIN, CompressionType.SNAPPY); + originalMeasurementGroup.addProps(new HashMap<>()); + originalMeasurementGroup.addAlias("a2"); + originalMeasurementGroup.addTags(new HashMap<>()); + originalMeasurementGroup.addAttributes(new HashMap<>()); + + Assert.assertEquals( + new InternalCreateMultiTimeSeriesNode( + new PlanNodeId("2024-04-30-1"), + new HashMap>() { + { + put(new PartialPath("root.db.device"), new Pair<>(false, expectedMeasurementGroup)); + } + }), + IoTDBSchemaRegionExtractor.PATTERN_PARSE_VISITOR + .visitInternalCreateMultiTimeSeries( + new InternalCreateMultiTimeSeriesNode( + new PlanNodeId("2024-04-30-1"), + new HashMap>() { + { + put( + new PartialPath("root.db.device"), + new Pair<>(false, originalMeasurementGroup)); + put( + new PartialPath("root.db1.device"), + new Pair<>(false, originalMeasurementGroup)); + } + }), + fullPathPattern) + .orElseThrow(AssertionError::new)); + } + + @Test + public void testBatchActivateTemplate() throws IllegalPathException { + Assert.assertEquals( + new BatchActivateTemplateNode( + new PlanNodeId("2024-04-30-1"), + new HashMap>() { + { + put(new PartialPath("root.db.device"), new Pair<>(1, 1)); + } + }), + IoTDBSchemaRegionExtractor.PATTERN_PARSE_VISITOR + .visitBatchActivateTemplate( + new BatchActivateTemplateNode( + new PlanNodeId("2024-04-30-1"), + new HashMap>() { + { + put(new PartialPath("root.db.device"), new Pair<>(1, 1)); + put(new PartialPath("root.db"), new Pair<>(2, 2)); + } + }), + fullPathPattern) + .orElseThrow(AssertionError::new)); + } + + @Test + public void testCreateLogicalView() throws IllegalPathException { + Assert.assertEquals( + new CreateLogicalViewNode( + new PlanNodeId("2024-04-30-1"), + new HashMap() { + { + put(new PartialPath("root.db.device.a1"), new TimeSeriesViewOperand("root.sg1.d1")); + } + }), + IoTDBSchemaRegionExtractor.PATTERN_PARSE_VISITOR + .visitCreateLogicalView( + new CreateLogicalViewNode( + new PlanNodeId("2024-04-30-1"), + new HashMap() { + { + put( + new PartialPath("root.db.device.a1"), + new TimeSeriesViewOperand("root.sg1.d1")); + put( + new PartialPath("root.db1.device.a1"), + new TimeSeriesViewOperand("root.sg1.d2")); + } + }), + prefixPathPattern) + .orElseThrow(AssertionError::new)); + } + + @Test + public void testAlterLogicalView() throws IllegalPathException { + Assert.assertEquals( + new AlterLogicalViewNode( + new PlanNodeId("2024-04-30-1"), + new HashMap() { + { + put(new PartialPath("root.db.device.a1"), new TimeSeriesViewOperand("root.sg1.d1")); + } + }), + IoTDBSchemaRegionExtractor.PATTERN_PARSE_VISITOR + .visitAlterLogicalView( + new AlterLogicalViewNode( + new PlanNodeId("2024-04-30-1"), + new HashMap() { + { + put( + new PartialPath("root.db.device.a1"), + new TimeSeriesViewOperand("root.sg1.d1")); + put( + new PartialPath("root.db1.device.a1"), + new TimeSeriesViewOperand("root.sg1.d2")); + } + }), + prefixPathPattern) + .orElseThrow(AssertionError::new)); + } + + @Test + public void testDeleteData() throws IllegalPathException { + Assert.assertEquals( + new DeleteDataNode( + new PlanNodeId("2024-04-30-1"), + Collections.singletonList(new PartialPath("root.db.device.s1")), + Long.MIN_VALUE, + Long.MAX_VALUE), + IoTDBSchemaRegionExtractor.PATTERN_PARSE_VISITOR + .visitDeleteData( + new DeleteDataNode( + new PlanNodeId("2024-04-30-1"), + Arrays.asList( + new PartialPath("root.*.device.s1"), new PartialPath("root.db.*.s1")), + Long.MIN_VALUE, + Long.MAX_VALUE), + prefixPathPattern) + .orElseThrow(AssertionError::new)); + } +} diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/pattern/IoTDBPipePatternTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/pattern/IoTDBPipePatternTest.java index 5c74f3ea21c1..db42d713012d 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/pattern/IoTDBPipePatternTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/pattern/IoTDBPipePatternTest.java @@ -30,48 +30,48 @@ public class IoTDBPipePatternTest { @Test public void testIotdbPipePattern() { // Test legal and illegal pattern - String[] legalPatterns = { + final String[] legalPatterns = { "root", "root.db", "root.db.d1.s", "root.db.`1`", "root.*.d.*s.s", }; - String[] illegalPatterns = { + final String[] illegalPatterns = { "root.", "roo", "", "root..", "root./", }; - for (String s : legalPatterns) { + for (final String s : legalPatterns) { Assert.assertTrue(new IoTDBPipePattern(s).isLegal()); } - for (String t : illegalPatterns) { + for (final String t : illegalPatterns) { try { Assert.assertFalse(new IoTDBPipePattern(t).isLegal()); - } catch (Exception e) { + } catch (final Exception e) { Assert.assertTrue(e instanceof PipeException); } } // Test pattern cover db - String db = "root.db"; - String[] patternsCoverDb = { + final String db = "root.db"; + final String[] patternsCoverDb = { "root.**", "root.db.**", "root.*db*.**", }; - String[] patternsNotCoverDb = { + final String[] patternsNotCoverDb = { "root.db", "root.*", "root.*.*", "root.db.*.**", "root.db.d1", "root.**.db.**", }; - for (String s : patternsCoverDb) { + for (final String s : patternsCoverDb) { Assert.assertTrue(new IoTDBPipePattern(s).coversDb(db)); } - for (String t : patternsNotCoverDb) { + for (final String t : patternsNotCoverDb) { Assert.assertFalse(new IoTDBPipePattern(t).coversDb(db)); } - String device = "root.db.d1"; + final String device = "root.db.d1"; // Test pattern cover device - String[] patternsCoverDevice = { + final String[] patternsCoverDevice = { "root.**", "root.db.**", "root.*.*.*", "root.db.d1.*", "root.*db*.*d*.*", "root.**.*1.*", }; - String[] patternsNotCoverDevice = { + final String[] patternsNotCoverDevice = { "root.*", "root.*.*", "root.db.d1", "root.db.d2.*", "root.**.d2.**", }; - for (String s : patternsCoverDevice) { + for (final String s : patternsCoverDevice) { Assert.assertTrue(new IoTDBPipePattern(s).coversDevice(device)); } for (String t : patternsNotCoverDevice) { @@ -79,31 +79,31 @@ public void testIotdbPipePattern() { } // Test pattern may overlap with device - String[] patternsOverlapWithDevice = { + final String[] patternsOverlapWithDevice = { "root.db.**", "root.db.d1", "root.db.d1.*", "root.db.d1.s1", "root.**.d2.**", "root.*.d*.**", }; - String[] patternsNotOverlapWithDevice = { + final String[] patternsNotOverlapWithDevice = { "root.db.d2.**", "root.db2.d1.**", "root.db.db.d1.**", }; - for (String s : patternsOverlapWithDevice) { + for (final String s : patternsOverlapWithDevice) { Assert.assertTrue(new IoTDBPipePattern(s).mayOverlapWithDevice(device)); } - for (String t : patternsNotOverlapWithDevice) { + for (final String t : patternsNotOverlapWithDevice) { Assert.assertFalse(new IoTDBPipePattern(t).mayOverlapWithDevice(device)); } // Test pattern match measurement - String measurement = "s1"; - String[] patternsMatchMeasurement = { + final String measurement = "s1"; + final String[] patternsMatchMeasurement = { "root.db.d1.s1", "root.db.d1.*", }; - String[] patternsNotMatchMeasurement = { + final String[] patternsNotMatchMeasurement = { "root.db.d1", "root.db.d1", "root.db.d1.*.*", }; - for (String s : patternsMatchMeasurement) { + for (final String s : patternsMatchMeasurement) { Assert.assertTrue(new IoTDBPipePattern(s).matchesMeasurement(device, measurement)); } - for (String t : patternsNotMatchMeasurement) { + for (final String t : patternsNotMatchMeasurement) { Assert.assertFalse(new IoTDBPipePattern(t).matchesMeasurement(device, measurement)); } } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/pattern/PrefixPipePatternTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/pattern/PrefixPipePatternTest.java index cb327d2dac05..d81c50a57ce0 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/pattern/PrefixPipePatternTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/pattern/PrefixPipePatternTest.java @@ -29,76 +29,76 @@ public class PrefixPipePatternTest { @Test public void testPrefixPipePattern() { // Test legal and illegal pattern - String[] legalPatterns = { + final String[] legalPatterns = { "root", "root.", "root.db", "root.db.d1.s", "root.db.`1`", }; - String[] illegalPatterns = { + final String[] illegalPatterns = { "roo", "", "root..", "root./", }; - for (String s : legalPatterns) { + for (final String s : legalPatterns) { Assert.assertTrue(new PrefixPipePattern(s).isLegal()); } - for (String t : illegalPatterns) { + for (final String t : illegalPatterns) { Assert.assertFalse(new PrefixPipePattern(t).isLegal()); } // Test pattern cover db - String db = "root.db"; - String[] patternsCoverDb = { + final String db = "root.db"; + final String[] patternsCoverDb = { "root", "root.", "root.d", "root.db", }; - String[] patternsNotCoverDb = { + final String[] patternsNotCoverDb = { "root.**", "root.db.", }; - for (String s : patternsCoverDb) { + for (final String s : patternsCoverDb) { Assert.assertTrue(new PrefixPipePattern(s).coversDb(db)); } - for (String t : patternsNotCoverDb) { + for (final String t : patternsNotCoverDb) { Assert.assertFalse(new PrefixPipePattern(t).coversDb(db)); } - String device = "root.db.d1"; + final String device = "root.db.d1"; // Test pattern cover device - String[] patternsCoverDevice = { + final String[] patternsCoverDevice = { "root", "root.", "root.d", "root.db", "root.db.", "root.db.d", "root.db.d1", }; - String[] patternsNotCoverDevice = { + final String[] patternsNotCoverDevice = { "root.db.d1.", "root.db.d1.s1", "root.**", "root.db.d2", }; - for (String s : patternsCoverDevice) { + for (final String s : patternsCoverDevice) { Assert.assertTrue(new PrefixPipePattern(s).coversDevice(device)); } - for (String t : patternsNotCoverDevice) { + for (final String t : patternsNotCoverDevice) { Assert.assertFalse(new PrefixPipePattern(t).coversDevice(device)); } // Test pattern may overlap with device - String[] patternsOverlapWithDevice = { + final String[] patternsOverlapWithDevice = { "root", "root.db.d1", "root.db.d1.", "root.db.d1.s1", }; - String[] patternsNotOverlapWithDevice = { + final String[] patternsNotOverlapWithDevice = { "root.db.d2", "root.**", }; - for (String s : patternsOverlapWithDevice) { + for (final String s : patternsOverlapWithDevice) { Assert.assertTrue(new PrefixPipePattern(s).mayOverlapWithDevice(device)); } - for (String t : patternsNotOverlapWithDevice) { + for (final String t : patternsNotOverlapWithDevice) { Assert.assertFalse(new PrefixPipePattern(t).mayOverlapWithDevice(device)); } // Test pattern match measurement - String measurement = "s1"; - String[] patternsMatchMeasurement = { + final String measurement = "s1"; + final String[] patternsMatchMeasurement = { "root.db.d1", "root.db.d1.", "root.db.d1.s", "root.db.d1.s1", }; - String[] patternsNotMatchMeasurement = { + final String[] patternsNotMatchMeasurement = { "root.db.d1.s11", "root.db.d1.s2", }; - for (String s : patternsMatchMeasurement) { + for (final String s : patternsMatchMeasurement) { Assert.assertTrue(new PrefixPipePattern(s).matchesMeasurement(device, measurement)); } - for (String t : patternsNotMatchMeasurement) { + for (final String t : patternsNotMatchMeasurement) { Assert.assertFalse(new PrefixPipePattern(t).matchesMeasurement(device, measurement)); } } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/extractor/IoTDBNonDataRegionExtractor.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/extractor/IoTDBNonDataRegionExtractor.java index e41957458950..9d6ce0ffde81 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/extractor/IoTDBNonDataRegionExtractor.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/extractor/IoTDBNonDataRegionExtractor.java @@ -26,7 +26,12 @@ import org.apache.iotdb.commons.pipe.datastructure.queue.listening.AbstractPipeListeningQueue; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.commons.pipe.event.PipeSnapshotEvent; +import org.apache.iotdb.commons.pipe.event.PipeWritePlanEvent; import org.apache.iotdb.commons.pipe.event.ProgressReportEvent; +import org.apache.iotdb.commons.pipe.pattern.IoTDBPipePattern; +import org.apache.iotdb.commons.pipe.pattern.PipePattern; +import org.apache.iotdb.pipe.api.customizer.configuration.PipeExtractorRuntimeConfiguration; +import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameters; import org.apache.iotdb.pipe.api.event.Event; import org.apache.iotdb.pipe.api.exception.PipeException; @@ -35,10 +40,13 @@ import java.util.LinkedList; import java.util.List; import java.util.Objects; +import java.util.Optional; import java.util.concurrent.atomic.AtomicBoolean; public abstract class IoTDBNonDataRegionExtractor extends IoTDBExtractor { + protected IoTDBPipePattern pipePattern; + private List historicalEvents = new LinkedList<>(); // A fixed size initialized only when the historicalEvents are first // filled. Used only for metric framework. @@ -53,6 +61,24 @@ public abstract class IoTDBNonDataRegionExtractor extends IoTDBExtractor { protected abstract AbstractPipeListeningQueue getListeningQueue(); + @Override + public void customize( + final PipeParameters parameters, final PipeExtractorRuntimeConfiguration configuration) + throws Exception { + super.customize(parameters, configuration); + + final PipePattern pattern = PipePattern.parsePipePatternFromSourceParameters(parameters); + if (!(pattern instanceof IoTDBPipePattern + && (((IoTDBPipePattern) pattern).isPrefix() + || ((IoTDBPipePattern) pattern).isFullPath()))) { + throw new IllegalArgumentException( + String.format( + "The path pattern %s is not valid for the source. Only prefix or full path is allowed.", + pattern.getPattern())); + } + pipePattern = (IoTDBPipePattern) pattern; + } + @Override public void start() throws Exception { if (hasBeenStarted.get() || hasBeenClosed.get()) { @@ -128,7 +154,7 @@ public EnrichedEvent supply() throws Exception { historicalEvents .remove(0) .shallowCopySelfAndBindPipeTaskMetaForProgressReport( - pipeName, pipeTaskMeta, null, Long.MIN_VALUE, Long.MAX_VALUE); + pipeName, pipeTaskMeta, pipePattern, Long.MIN_VALUE, Long.MAX_VALUE); if (historicalEvents.isEmpty()) { // We only report progress for the last snapshot event. @@ -143,23 +169,27 @@ public EnrichedEvent supply() throws Exception { } // Realtime - EnrichedEvent realtimeEvent = (EnrichedEvent) iterator.next(getMaxBlockingTimeMs()); + PipeWritePlanEvent realtimeEvent = (PipeWritePlanEvent) iterator.next(getMaxBlockingTimeMs()); if (Objects.isNull(realtimeEvent)) { return null; } - if (!isTypeListened(realtimeEvent) + realtimeEvent = trimRealtimeEventByPipePattern(realtimeEvent).orElse(null); + if (Objects.isNull(realtimeEvent) + || !isTypeListened(realtimeEvent) || (!isForwardingPipeRequests && realtimeEvent.isGeneratedByPipe())) { final ProgressReportEvent event = - new ProgressReportEvent(pipeName, pipeTaskMeta, null, Long.MIN_VALUE, Long.MAX_VALUE); + new ProgressReportEvent( + pipeName, pipeTaskMeta, pipePattern, Long.MIN_VALUE, Long.MAX_VALUE); event.bindProgressIndex(new MetaProgressIndex(iterator.getNextIndex() - 1)); event.increaseReferenceCount(IoTDBNonDataRegionExtractor.class.getName()); return event; } realtimeEvent = - realtimeEvent.shallowCopySelfAndBindPipeTaskMetaForProgressReport( - pipeName, pipeTaskMeta, null, Long.MIN_VALUE, Long.MAX_VALUE); + (PipeWritePlanEvent) + realtimeEvent.shallowCopySelfAndBindPipeTaskMetaForProgressReport( + pipeName, pipeTaskMeta, pipePattern, Long.MIN_VALUE, Long.MAX_VALUE); realtimeEvent.bindProgressIndex(new MetaProgressIndex(iterator.getNextIndex() - 1)); realtimeEvent.increaseReferenceCount(IoTDBNonDataRegionExtractor.class.getName()); return realtimeEvent; @@ -167,7 +197,11 @@ public EnrichedEvent supply() throws Exception { protected abstract long getMaxBlockingTimeMs(); - protected abstract boolean isTypeListened(final Event event); + // The trimmed event shall be non-null. + protected abstract Optional trimRealtimeEventByPipePattern( + final PipeWritePlanEvent event); + + protected abstract boolean isTypeListened(final PipeWritePlanEvent event); protected abstract void confineHistoricalEventTransferTypes(final PipeSnapshotEvent event); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/pattern/IoTDBPipePattern.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/pattern/IoTDBPipePattern.java index ec4192adff37..6c82c7e3d9cc 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/pattern/IoTDBPipePattern.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/pattern/IoTDBPipePattern.java @@ -22,26 +22,37 @@ import org.apache.iotdb.commons.conf.IoTDBConstant; import org.apache.iotdb.commons.exception.IllegalPathException; import org.apache.iotdb.commons.path.PartialPath; +import org.apache.iotdb.commons.path.PathPatternTree; +import org.apache.iotdb.commons.path.PathPatternUtil; import org.apache.iotdb.commons.pipe.config.constant.PipeExtractorConstant; import org.apache.iotdb.commons.utils.PathUtils; import org.apache.iotdb.pipe.api.exception.PipeException; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; import java.util.Objects; +import java.util.stream.Collectors; public class IoTDBPipePattern extends PipePattern { private final PartialPath patternPartialPath; - public IoTDBPipePattern(String pattern) { + public IoTDBPipePattern(final String pattern) { super(pattern); try { patternPartialPath = new PartialPath(getPattern()); - } catch (IllegalPathException e) { + } catch (final IllegalPathException e) { throw new PipeException("Illegal IoTDBPipePattern: " + getPattern(), e); } } + public static List applyIndexesOnList( + final int[] filteredIndexes, final List originalList) { + return Arrays.stream(filteredIndexes).mapToObj(originalList::get).collect(Collectors.toList()); + } + @Override public String getDefaultPattern() { return PipeExtractorConstant.EXTRACTOR_PATTERN_IOTDB_DEFAULT_VALUE; @@ -56,44 +67,44 @@ public boolean isLegal() { try { PathUtils.isLegalPath(pattern); return true; - } catch (IllegalPathException e) { + } catch (final IllegalPathException e) { return false; } } @Override - public boolean coversDb(String db) { + public boolean coversDb(final String db) { try { return patternPartialPath.include( new PartialPath(db, IoTDBConstant.MULTI_LEVEL_PATH_WILDCARD)); - } catch (IllegalPathException e) { + } catch (final IllegalPathException e) { return false; } } @Override - public boolean coversDevice(String device) { + public boolean coversDevice(final String device) { try { return patternPartialPath.include( new PartialPath(device, IoTDBConstant.ONE_LEVEL_PATH_WILDCARD)); - } catch (IllegalPathException e) { + } catch (final IllegalPathException e) { return false; } } @Override - public boolean mayOverlapWithDevice(String device) { + public boolean mayOverlapWithDevice(final String device) { try { // Another way is to use patternPath.overlapWith("device.*"), // there will be no false positives but time cost may be higher. return patternPartialPath.matchPrefixPath(new PartialPath(device)); - } catch (IllegalPathException e) { + } catch (final IllegalPathException e) { return false; } } @Override - public boolean matchesMeasurement(String device, String measurement) { + public boolean matchesMeasurement(final String device, final String measurement) { // For aligned timeseries, empty measurement is an alias of the time column. if (Objects.isNull(measurement) || measurement.isEmpty()) { return false; @@ -101,11 +112,77 @@ public boolean matchesMeasurement(String device, String measurement) { try { return patternPartialPath.matchFullPath(new PartialPath(device, measurement)); - } catch (IllegalPathException e) { + } catch (final IllegalPathException e) { + return false; + } + } + + /** + * Check if the {@link PipePattern} matches the given prefix path. + * + *

NOTE: In schema transmission, {@link #mayOverlapWithDevice(String)} can be used to detect + * whether the given path can act as a parent path of the {@link PipePattern}, and to transmit + * possibly used schemas like database creation and template setting. + */ + public boolean matchPrefixPath(final String path) { + return mayOverlapWithDevice(path); + } + + /** + * This is the precise form of the device overlap and is used only be device template transfer. + */ + public boolean matchDevice(final String devicePath) { + try { + return patternPartialPath.overlapWith(new PartialPath(devicePath, "*")); + } catch (final IllegalPathException e) { return false; } } + /** + * Return if the given tail node matches the pattern's tail node. Caller shall ensure that it is a + * prefix or full path pattern. + */ + public boolean matchTailNode(final String tailNode) { + return !isFullPath() || patternPartialPath.getTailNode().equals(tailNode); + } + + /** + * Get the intersection of the given {@link PartialPath} and the {@link PipePattern}, Only used by + * schema transmission. Caller shall ensure that it is a prefix or full path pattern. + */ + public List getIntersection(final PartialPath partialPath) { + if (isFullPath()) { + return partialPath.matchFullPath(patternPartialPath) + ? Collections.singletonList(patternPartialPath) + : Collections.emptyList(); + } + return partialPath.intersectWithPrefixPattern(patternPartialPath); + } + + /** + * Get the intersection of the given {@link PathPatternTree} and the {@link PipePattern}. Only + * used by schema transmission. Caller shall ensure that it is a prefix or full path pattern. + */ + public PathPatternTree getIntersection(final PathPatternTree patternTree) { + final PathPatternTree thisPatternTree = new PathPatternTree(); + thisPatternTree.appendPathPattern(patternPartialPath); + thisPatternTree.constructTree(); + return patternTree.intersectWithFullPathPrefixTree(thisPatternTree); + } + + public boolean isPrefix() { + return PathPatternUtil.isMultiLevelMatchWildcard(patternPartialPath.getTailNode()) + && !new PartialPath( + Arrays.copyOfRange( + patternPartialPath.getNodes(), 0, patternPartialPath.getNodeLength() - 1)) + .hasWildcard(); + } + + public boolean isFullPath() { + return !patternPartialPath.hasWildcard(); + } + @Override public String toString() { return "IoTDBPipePattern" + super.toString(); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/pattern/PipePattern.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/pattern/PipePattern.java index 04be541f6252..803c24621cf4 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/pattern/PipePattern.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/pattern/PipePattern.java @@ -41,7 +41,7 @@ public abstract class PipePattern { protected final String pattern; - protected PipePattern(String pattern) { + protected PipePattern(final String pattern) { this.pattern = pattern != null ? pattern : getDefaultPattern(); } @@ -58,7 +58,8 @@ public boolean isRoot() { * * @return The interpreted {@link PipePattern} which is not null. */ - public static PipePattern parsePipePatternFromSourceParameters(PipeParameters sourceParameters) { + public static PipePattern parsePipePatternFromSourceParameters( + final PipeParameters sourceParameters) { final String path = sourceParameters.getStringByKeys(EXTRACTOR_PATH_KEY, SOURCE_PATH_KEY); // 1. If "source.path" is specified, it will be interpreted as an IoTDB-style path, @@ -95,7 +96,7 @@ public static PipePattern parsePipePatternFromSourceParameters(PipeParameters so // 3. If neither "source.path" nor "source.pattern" is specified, // this pipe source will match all data. - return new PrefixPipePattern(null); + return new IoTDBPipePattern(null); } public abstract String getDefaultPattern(); @@ -104,10 +105,10 @@ public static PipePattern parsePipePatternFromSourceParameters(PipeParameters so public abstract boolean isLegal(); /** Check if this pattern matches all time-series under a database. */ - public abstract boolean coversDb(String db); + public abstract boolean coversDb(final String db); /** Check if a device's all measurements are covered by this pattern. */ - public abstract boolean coversDevice(String device); + public abstract boolean coversDevice(final String device); /** * Check if a device may have some measurements matched by the pattern. @@ -117,14 +118,14 @@ public static PipePattern parsePipePatternFromSourceParameters(PipeParameters so *

NOTE2: this is just a loose check and may have false positives. To further check if a * measurement matches the pattern, please use {@link PipePattern#matchesMeasurement} after this. */ - public abstract boolean mayOverlapWithDevice(String device); + public abstract boolean mayOverlapWithDevice(final String device); /** * Check if a full path with device and measurement can be matched by pattern. * - *

NOTE: this is only called when {@link PipePattern#mayOverlapWithDevice} is true. + *

NOTE: this is only called when {@link PipePattern#mayOverlapWithDevice(String)} is true. */ - public abstract boolean matchesMeasurement(String device, String measurement); + public abstract boolean matchesMeasurement(final String device, final String measurement); @Override public String toString() { diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/pattern/PrefixPipePattern.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/pattern/PrefixPipePattern.java index 5e9c3a5aa441..52f70057052f 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/pattern/PrefixPipePattern.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/pattern/PrefixPipePattern.java @@ -30,7 +30,7 @@ public class PrefixPipePattern extends PipePattern { - public PrefixPipePattern(String pattern) { + public PrefixPipePattern(final String pattern) { super(pattern); } @@ -47,25 +47,25 @@ public boolean isLegal() { try { PathUtils.isLegalPath(pattern); - } catch (IllegalPathException e) { + } catch (final IllegalPathException e) { try { if ("root".equals(pattern) || "root.".equals(pattern)) { return true; } // Split the pattern to nodes. - String[] pathNodes = StringUtils.splitPreserveAllTokens(pattern, "\\."); + final String[] pathNodes = StringUtils.splitPreserveAllTokens(pattern, "\\."); // Check whether the pattern without last node is legal. PathUtils.splitPathToDetachedNodes( String.join(".", Arrays.copyOfRange(pathNodes, 0, pathNodes.length - 1))); - String lastNode = pathNodes[pathNodes.length - 1]; + final String lastNode = pathNodes[pathNodes.length - 1]; // Check whether the last node is legal. if (!"".equals(lastNode)) { Double.parseDouble(lastNode); } - } catch (Exception ignored) { + } catch (final Exception ignored) { return false; } } @@ -73,19 +73,19 @@ public boolean isLegal() { } @Override - public boolean coversDb(String db) { + public boolean coversDb(final String db) { return pattern.length() <= db.length() && db.startsWith(pattern); } @Override - public boolean coversDevice(String device) { + public boolean coversDevice(final String device) { // for example, pattern is root.a.b and device is root.a.b.c // in this case, the extractor can be matched without checking the measurements return pattern.length() <= device.length() && device.startsWith(pattern); } @Override - public boolean mayOverlapWithDevice(String device) { + public boolean mayOverlapWithDevice(final String device) { return ( // for example, pattern is root.a.b and device is root.a.b.c // in this case, the extractor can be matched without checking the measurements @@ -97,7 +97,7 @@ public boolean mayOverlapWithDevice(String device) { } @Override - public boolean matchesMeasurement(String device, String measurement) { + public boolean matchesMeasurement(final String device, final String measurement) { // We assume that the device is already matched. if (pattern.length() <= device.length()) { return true; @@ -106,7 +106,7 @@ public boolean matchesMeasurement(String device, String measurement) { // For example, pattern is "root.a.b.c", device is "root.a.b", // then measurements "c" and "cc" can be matched, // measurements "d" or "dc" can't be matched. - String dotAndMeasurement = TsFileConstant.PATH_SEPARATOR + measurement; + final String dotAndMeasurement = TsFileConstant.PATH_SEPARATOR + measurement; return // low cost check comes first pattern.length() <= device.length() + dotAndMeasurement.length() From bb80f6cea14a1e1a76713a4e91248681c9b7cea8 Mon Sep 17 00:00:00 2001 From: Haonan Date: Wed, 22 May 2024 10:45:21 +0800 Subject: [PATCH 27/42] Fix Python IT (#12567) --- iotdb-client/client-py/requirements_dev.txt | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/iotdb-client/client-py/requirements_dev.txt b/iotdb-client/client-py/requirements_dev.txt index 38fe8ccd816c..8068e8c747fc 100644 --- a/iotdb-client/client-py/requirements_dev.txt +++ b/iotdb-client/client-py/requirements_dev.txt @@ -23,6 +23,7 @@ flake8==3.9.0 black==24.3.0 # Testcontainer testcontainers==3.4.2 +requests<2.32.0 # For releases twine==3.4.1 -wheel==0.38.1 \ No newline at end of file +wheel==0.38.1 From 8150f7bde1dc0e21649258bc11694e8494234282 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 22 May 2024 12:49:15 +0800 Subject: [PATCH 28/42] Pipe: Adjust the reloading logic to restart all the pipes when deleted tsfiles are linked too many (#12566) --- .../agent/task/PipeDataNodeTaskAgent.java | 39 ++++++++++--------- 1 file changed, 21 insertions(+), 18 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java index 2671de28f198..366024d2906e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java @@ -355,26 +355,29 @@ private void restartAllStuckPipesInternal() { continue; } - if (!extractors.get(0).isStreamMode() - || extractors.stream() - .noneMatch(IoTDBDataRegionExtractor::hasConsumedAllHistoricalTsFiles)) { - // Extractors of this pipe might not pin too much MemTables, - // still need to check if linked-and-deleted TsFile count exceeds limit. - if ((CONFIG.isEnableSeqSpaceCompaction() - || CONFIG.isEnableUnseqSpaceCompaction() - || CONFIG.isEnableCrossSpaceCompaction()) - && mayDeletedTsFileSizeReachDangerousThreshold()) { - LOGGER.warn( - "Pipe {} needs to restart because too many TsFiles are out-of-date.", - pipeMeta.getStaticMeta()); - stuckPipes.add(pipeMeta); - } + // Extractors of this pipe might not pin too much MemTables, + // still need to check if linked-and-deleted TsFile count exceeds limit. + // Typically, if deleted tsFiles are too abundant all pipes may need to restart. + if ((CONFIG.isEnableSeqSpaceCompaction() + || CONFIG.isEnableUnseqSpaceCompaction() + || CONFIG.isEnableCrossSpaceCompaction()) + && mayDeletedTsFileSizeReachDangerousThreshold()) { + LOGGER.warn( + "Pipe {} needs to restart because too many TsFiles are out-of-date.", + pipeMeta.getStaticMeta()); + stuckPipes.add(pipeMeta); continue; } - if (mayMemTablePinnedCountReachDangerousThreshold() || mayWalSizeReachThrottleThreshold()) { - // Extractors of this pipe may be stuck and pinning too much MemTables. - LOGGER.warn("Pipe {} may be stuck.", pipeMeta.getStaticMeta()); + // Only restart the stream mode pipes for releasing memTables. + if (extractors.get(0).isStreamMode() + && extractors.stream().anyMatch(IoTDBDataRegionExtractor::hasConsumedAllHistoricalTsFiles) + && (mayMemTablePinnedCountReachDangerousThreshold() + || mayWalSizeReachThrottleThreshold())) { + // Extractors of this pipe may be stuck and is pinning too many MemTables. + LOGGER.warn( + "Pipe {} needs to restart because too many memTables are pinned.", + pipeMeta.getStaticMeta()); stuckPipes.add(pipeMeta); } } @@ -402,7 +405,7 @@ private boolean mayDeletedTsFileSizeReachDangerousThreshold() { && linkedButDeletedTsFileSize > PipeConfig.getInstance().getPipeMaxAllowedLinkedDeletedTsFileDiskUsagePercentage() * totalDisk; - } catch (Exception e) { + } catch (final Exception e) { LOGGER.warn("Failed to judge if deleted TsFile size reaches dangerous threshold.", e); return false; } From 7d4dd9e1427324816143192437121efe9f2a03a5 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 22 May 2024 14:19:20 +0800 Subject: [PATCH 29/42] Pipe: Fix insertNode.getDevicePath() is not handled correctly for insertRowsNode (NPE) (#12569) --- .../iotdb/pipe/it/autocreate/IoTDBPipeClusterIT.java | 8 ++++---- .../pipe/it/autocreate/IoTDBPipeExtractorIT.java | 5 ++--- .../builder/PipeTransferBatchReqBuilder.java | 3 ++- .../thrift/async/IoTDBDataRegionAsyncConnector.java | 5 ++++- .../PipeTransferTabletInsertNodeEventHandler.java | 3 ++- .../thrift/sync/IoTDBDataRegionSyncConnector.java | 9 +++++++-- .../tablet/PipeInsertNodeTabletInsertionEvent.java | 11 ++++++++++- 7 files changed, 31 insertions(+), 13 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeClusterIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeClusterIT.java index e90c7ecf8ade..7ebf56c60356 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeClusterIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeClusterIT.java @@ -915,9 +915,9 @@ public void testNegativeTimestamp() throws Exception { if (!TestUtils.tryExecuteNonQueriesWithRetry( senderEnv, Arrays.asList( - "insert into root.db.d1(time, s1) values (-123, 3)", - "insert into root.db.d1(time, s1) values (now(), 3)", - "flush"))) { + // Test the correctness of insertRowsNode transmission + "insert into root.db.d1(time, s1) values (-122, 3)", + "insert into root.db.d1(time, s1) values (-123, 3), (now(), 3)"))) { return; } @@ -925,7 +925,7 @@ public void testNegativeTimestamp() throws Exception { receiverEnv, "select count(*) from root.**", "count(root.db.d1.s1),", - Collections.singleton("5,")); + Collections.singleton("6,")); } } } diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeExtractorIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeExtractorIT.java index 28852463a77d..b0504f8b88e6 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeExtractorIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeExtractorIT.java @@ -632,8 +632,7 @@ public void testHistoryAndRealtime() throws Exception { "insert into root.db.d1 (time, at1) values (2, 11)", "insert into root.db.d2 (time, at1) values (2, 21)", "insert into root.db.d3 (time, at1) values (2, 31)", - "insert into root.db.d4 (time, at1) values (2, 41)", - "flush"))) { + "insert into root.db.d4 (time, at1) values (2, 41), (3, 51)"))) { return; } @@ -646,7 +645,7 @@ public void testHistoryAndRealtime() throws Exception { receiverEnv, "select count(*) from root.** where time >= 2", "count(root.db.d4.at1),count(root.db.d2.at1),count(root.db.d3.at1),", - Collections.singleton("1,1,0,")); + Collections.singleton("2,1,0,")); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/builder/PipeTransferBatchReqBuilder.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/builder/PipeTransferBatchReqBuilder.java index 2b19bb1275dc..087c969b89a6 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/builder/PipeTransferBatchReqBuilder.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/builder/PipeTransferBatchReqBuilder.java @@ -114,7 +114,8 @@ public synchronized Pair onEvent(final TabletInsertio } else if (event instanceof PipeInsertNodeTabletInsertionEvent) { final InsertNode insertNode = ((PipeInsertNodeTabletInsertionEvent) event).getInsertNodeViaCacheIfPossible(); - if (Objects.nonNull(insertNode)) { + // insertNode.getDevicePath() is null for InsertRowsNode + if (Objects.nonNull(insertNode) && Objects.nonNull(insertNode.getDevicePath())) { deviceId = insertNode.getDevicePath().getFullPath(); } } 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 dcab0b69b3c5..632d929e3724 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 @@ -196,7 +196,10 @@ private void transferWithoutCheck(final TabletInsertionEvent tabletInsertionEven pipeInsertNodeTabletInsertionEvent, pipeTransferReq, this); transfer( - Objects.nonNull(insertNode) ? insertNode.getDevicePath().getFullPath() : null, + // insertNode.getDevicePath() is null for InsertRowsNode + Objects.nonNull(insertNode) && Objects.nonNull(insertNode.getDevicePath()) + ? insertNode.getDevicePath().getFullPath() + : null, pipeTransferInsertNodeReqHandler); } else { // tabletInsertionEvent instanceof PipeRawTabletInsertionEvent final PipeRawTabletInsertionEvent pipeRawTabletInsertionEvent = diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/handler/PipeTransferTabletInsertNodeEventHandler.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/handler/PipeTransferTabletInsertNodeEventHandler.java index 0d22722c4fcc..21cc976be77f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/handler/PipeTransferTabletInsertNodeEventHandler.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/handler/PipeTransferTabletInsertNodeEventHandler.java @@ -49,7 +49,8 @@ protected void doTransfer(AsyncPipeDataTransferServiceClient client, TPipeTransf protected void updateLeaderCache(TSStatus status) { final InsertNode insertNode = ((PipeInsertNodeTabletInsertionEvent) event).getInsertNodeViaCacheIfPossible(); - if (insertNode != null) { + // insertNode.getDevicePath() is null for InsertRowsNode + if (insertNode != null && insertNode.getDevicePath() != null) { connector.updateLeaderCache( insertNode.getDevicePath().getFullPath(), status.getRedirectNode()); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBDataRegionSyncConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBDataRegionSyncConnector.java index cb93bffd4111..12e9438f245e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBDataRegionSyncConnector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBDataRegionSyncConnector.java @@ -238,7 +238,11 @@ private void doTransfer( insertNode = pipeInsertNodeTabletInsertionEvent.getInsertNodeViaCacheIfPossible(); if (insertNode != null) { - clientAndStatus = clientManager.getClient(insertNode.getDevicePath().getFullPath()); + clientAndStatus = + // insertNode.getDevicePath() is null for InsertRowsNode + Objects.nonNull(insertNode.getDevicePath()) + ? clientManager.getClient(insertNode.getDevicePath().getFullPath()) + : clientManager.getClient(); resp = clientAndStatus .getLeft() @@ -277,7 +281,8 @@ private void doTransfer( pipeInsertNodeTabletInsertionEvent.coreReportMessage(), status), pipeInsertNodeTabletInsertionEvent.toString()); } - if (insertNode != null && status.isSetRedirectNode()) { + // insertNode.getDevicePath() is null for InsertRowsNode + if (insertNode != null && insertNode.getDevicePath() != null && status.isSetRedirectNode()) { clientManager.updateLeaderCache( insertNode.getDevicePath().getFullPath(), status.getRedirectNode()); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeInsertNodeTabletInsertionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeInsertNodeTabletInsertionEvent.java index dad53f2b69de..437d8b97d509 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeInsertNodeTabletInsertionEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeInsertNodeTabletInsertionEvent.java @@ -25,6 +25,7 @@ import org.apache.iotdb.commons.pipe.pattern.PipePattern; import org.apache.iotdb.commons.pipe.task.meta.PipeTaskMeta; import org.apache.iotdb.db.pipe.resource.PipeResourceManager; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeType; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertRowNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertRowsNode; @@ -303,7 +304,15 @@ public boolean shouldParsePattern() { final InsertNode node = getInsertNodeViaCacheIfPossible(); return super.shouldParsePattern() && Objects.nonNull(pipePattern) - && (Objects.isNull(node) || !pipePattern.coversDevice(node.getDevicePath().getFullPath())); + && (Objects.isNull(node) + || (node.getType() == PlanNodeType.INSERT_ROWS + ? ((InsertRowsNode) node) + .getInsertRowNodeList().stream() + .anyMatch( + insertRowNode -> + !pipePattern.coversDevice( + insertRowNode.getDevicePath().getFullPath())) + : !pipePattern.coversDevice(node.getDevicePath().getFullPath()))); } public List toRawTabletInsertionEvents() { From f1d8fce46c2e967d89fc9e07c796a39b7645c80a Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 22 May 2024 19:00:06 +0800 Subject: [PATCH 30/42] Pipe: Fix PipeSetTTLPlan is not handled correctly (#12571) --- .../iotdb/pipe/it/manual/IoTDBPipeMetaHistoricalIT.java | 5 ++--- .../manager/pipe/event/PipeConfigRegionSnapshotEvent.java | 1 + .../pipe/extractor/ConfigRegionListeningFilter.java | 8 ++++++-- .../PipeConfigPhysicalPlanPatternParseVisitor.java | 6 +++++- .../persistence/executor/ConfigPlanExecutor.java | 8 +++++--- .../PipeConfigPhysicalPlanPatternParseVisitorTest.java | 4 ++-- 6 files changed, 21 insertions(+), 11 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/IoTDBPipeMetaHistoricalIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/IoTDBPipeMetaHistoricalIT.java index 70316d3b600a..858af4cb5766 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/IoTDBPipeMetaHistoricalIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/IoTDBPipeMetaHistoricalIT.java @@ -119,8 +119,7 @@ public void testTemplateInclusion() throws Exception { final Map connectorAttributes = new HashMap<>(); extractorAttributes.put("extractor.inclusion", "data, schema"); - extractorAttributes.put( - "extractor.inclusion.exclusion", "schema.timeseries.ordinary, schema.ttl"); + extractorAttributes.put("extractor.inclusion.exclusion", "schema.timeseries.ordinary"); extractorAttributes.put("extractor.path", "root.ln.**"); connectorAttributes.put("connector", "iotdb-thrift-connector"); @@ -154,7 +153,7 @@ public void testTemplateInclusion() throws Exception { "Database,TTL,SchemaReplicationFactor,DataReplicationFactor,TimePartitionInterval,", // Receiver's SchemaReplicationFactor/DataReplicationFactor shall be 3/2 regardless of the // sender - Collections.singleton("root.ln,null,3,2,604800000,")); + Collections.singleton("root.ln,3600000,3,2,604800000,")); TestUtils.assertDataEventuallyOnEnv( receiverEnv, "select * from root.**", diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/event/PipeConfigRegionSnapshotEvent.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/event/PipeConfigRegionSnapshotEvent.java index 407dec4887e7..9e7a252eb6e7 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/event/PipeConfigRegionSnapshotEvent.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/event/PipeConfigRegionSnapshotEvent.java @@ -78,6 +78,7 @@ public class PipeConfigRegionSnapshotEvent extends PipeSnapshotEvent { Arrays.asList( ConfigPhysicalPlanType.CreateDatabase.getPlanType(), ConfigPhysicalPlanType.SetTTL.getPlanType(), + ConfigPhysicalPlanType.PipeSetTTL.getPlanType(), ConfigPhysicalPlanType.CreateSchemaTemplate.getPlanType(), ConfigPhysicalPlanType.CommitSetSchemaTemplate.getPlanType())))); } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/extractor/ConfigRegionListeningFilter.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/extractor/ConfigRegionListeningFilter.java index 2f1ee641ba6e..f782f3e06171 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/extractor/ConfigRegionListeningFilter.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/extractor/ConfigRegionListeningFilter.java @@ -83,7 +83,9 @@ public class ConfigRegionListeningFilter { Collections.singletonList(ConfigPhysicalPlanType.DropSchemaTemplate)); OPTION_PLAN_MAP.put( new PartialPath("schema.timeseries.template.unset"), - Collections.singletonList(ConfigPhysicalPlanType.PipeUnsetTemplate)); + Collections.unmodifiableList( + Arrays.asList( + ConfigPhysicalPlanType.UnsetTemplate, ConfigPhysicalPlanType.PipeUnsetTemplate))); OPTION_PLAN_MAP.put( new PartialPath("schema.timeseries.ordinary.drop"), @@ -96,7 +98,9 @@ public class ConfigRegionListeningFilter { Collections.singletonList(ConfigPhysicalPlanType.PipeDeactivateTemplate)); OPTION_PLAN_MAP.put( - new PartialPath("schema.ttl"), Collections.singletonList(ConfigPhysicalPlanType.SetTTL)); + new PartialPath("schema.ttl"), + Collections.unmodifiableList( + Arrays.asList(ConfigPhysicalPlanType.SetTTL, ConfigPhysicalPlanType.PipeSetTTL))); OPTION_PLAN_MAP.put( new PartialPath("auth.role.create"), diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/extractor/PipeConfigPhysicalPlanPatternParseVisitor.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/extractor/PipeConfigPhysicalPlanPatternParseVisitor.java index e5ea0cccd3b3..93424df84166 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/extractor/PipeConfigPhysicalPlanPatternParseVisitor.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/extractor/PipeConfigPhysicalPlanPatternParseVisitor.java @@ -48,6 +48,7 @@ import java.io.IOException; import java.util.Arrays; import java.util.Collection; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Optional; @@ -269,8 +270,11 @@ public Optional visitPipeDeactivateTemplate( @Override public Optional visitTTL( final SetTTLPlan setTTLPlan, final IoTDBPipePattern pattern) { + final PartialPath databasePath = new PartialPath(setTTLPlan.getDatabasePathPattern()); final List intersectionList = - pattern.getIntersection(new PartialPath(setTTLPlan.getDatabasePathPattern())); + pattern.matchPrefixPath(databasePath.getFullPath()) + ? Collections.singletonList(databasePath) + : pattern.getIntersection(databasePath); return !intersectionList.isEmpty() ? Optional.of( new PipeSetTTLPlan( diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/executor/ConfigPlanExecutor.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/executor/ConfigPlanExecutor.java index f47634065650..aa65375290a1 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/executor/ConfigPlanExecutor.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/executor/ConfigPlanExecutor.java @@ -507,9 +507,11 @@ public TSStatus executeNonQueryPlan(ConfigPhysicalPlan physicalPlan) // Will not be actually executed. return new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode()); case PipeUnsetTemplate: - // PipeUnsetTemplate plan will not be written here, and exists only after pipe sender - // collects UnsetTemplatePlan and before receiver calls ConfigManager. - throw new UnsupportedOperationException("PipeUnsetTemplate is not supported."); + case PipeSetTTL: + // PipeUnsetTemplate/PipeSetTTL plan will not be written here, and exists only after pipe + // sender collects UnsetTemplatePlan/SetTTLPlan and before receiver calls ConfigManager. + throw new UnsupportedOperationException( + String.format("Plan type %s is not supported.", physicalPlan.getType())); case TestOnly: return new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode()); default: diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/manager/pipe/extractor/PipeConfigPhysicalPlanPatternParseVisitorTest.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/manager/pipe/extractor/PipeConfigPhysicalPlanPatternParseVisitorTest.java index a2ff81a9da44..5b341d11da80 100644 --- a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/manager/pipe/extractor/PipeConfigPhysicalPlanPatternParseVisitorTest.java +++ b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/manager/pipe/extractor/PipeConfigPhysicalPlanPatternParseVisitorTest.java @@ -391,13 +391,13 @@ public void testSetTTL() throws IllegalPathException { ((PipeSetTTLPlan) IoTDBConfigRegionExtractor.PATTERN_PARSE_VISITOR .visitTTL( - new SetTTLPlan(Arrays.asList("root", "*", "device", "s1"), Long.MAX_VALUE), + new SetTTLPlan(Arrays.asList("root", "db", "**"), Long.MAX_VALUE), prefixPathPattern) .orElseThrow(AssertionError::new)) .getSetTTLPlans(); Assert.assertEquals( - Collections.singletonList(new PartialPath("root.db.device.s1")), + Collections.singletonList(new PartialPath("root.db.device.**")), plans.stream() .map(setTTLPlan -> new PartialPath(setTTLPlan.getDatabasePathPattern())) .collect(Collectors.toList())); From 9a29af7ce1df06940eff3124240d5de60357dec9 Mon Sep 17 00:00:00 2001 From: YangCaiyin Date: Thu, 23 May 2024 08:34:49 +0800 Subject: [PATCH 31/42] [IOTDB-6325] Support RegionScan for active metadata queries [BE Part One] --- .../execution/driver/DataDriver.java | 25 +- .../execution/driver/DataDriverContext.java | 4 +- .../fragment/FragmentInstanceContext.java | 150 ++++++-- .../execution/fragment/QueryContext.java | 76 +++-- .../source/AbstractDataSourceOperator.java | 5 +- .../source/AlignedSeriesScanOperator.java | 5 +- .../operator/source/DataSourceOperator.java | 4 +- .../utils/ResourceByPathUtils.java | 83 +---- .../storageengine/dataregion/DataRegion.java | 208 +++++++++-- .../dataregion/IDataRegionForQuery.java | 20 +- .../dataregion/VirtualDataRegion.java | 27 ++ .../dataregion/memtable/AbstractMemTable.java | 311 +++++++++++++++++ .../memtable/AlignedWritableMemChunk.java | 4 + .../dataregion/memtable/IMemTable.java | 19 ++ .../dataregion/memtable/TsFileProcessor.java | 322 ++++++++++++++++++ .../dataregion/read/IQueryDataSource.java | 27 ++ .../dataregion/read/QueryDataSource.java | 11 +- .../read/QueryDataSourceForRegionScan.java | 64 ++++ .../dataregion/read/QueryDataSourceType.java | 25 ++ .../read/filescan/IChunkHandle.java | 49 +++ .../read/filescan/IFileScanHandle.java | 99 ++++++ .../impl/ClosedFileScanHandleImpl.java | 211 ++++++++++++ .../impl/DiskAlignedChunkHandleImpl.java | 81 +++++ .../filescan/impl/DiskChunkHandleImpl.java | 137 ++++++++ .../impl/MemAlignedChunkHandleImpl.java | 72 ++++ .../filescan/impl/MemChunkHandleImpl.java | 56 +++ .../impl/UnclosedFileScanHandleImpl.java | 180 ++++++++++ .../filescan/model/AbstractChunkOffset.java | 49 +++ .../model/AbstractDeviceChunkMetaData.java | 36 ++ .../filescan/model/AlignedChunkOffset.java | 52 +++ .../model/AlignedDeviceChunkMetaData.java | 44 +++ .../read/filescan/model/ChunkOffset.java | 48 +++ .../filescan/model/DeviceChunkMetaData.java | 45 +++ .../filescan/model/DeviceStartEndTime.java | 46 +++ .../utils/SharedTimeDataBuffer.java | 91 +++++ .../TsFileDeviceStartEndTimeIterator.java | 52 +++ .../iotdb/db/utils/ModificationUtils.java | 97 ++++++ .../db/utils/datastructure/AlignedTVList.java | 7 +- .../db/utils/datastructure/BinaryTVList.java | 3 +- .../db/utils/datastructure/BooleanTVList.java | 3 +- .../db/utils/datastructure/DoubleTVList.java | 3 +- .../db/utils/datastructure/FloatTVList.java | 3 +- .../db/utils/datastructure/IntTVList.java | 3 +- .../db/utils/datastructure/LongTVList.java | 3 +- .../iotdb/db/utils/datastructure/TVList.java | 18 +- .../db/metadata/path/PatternTreeMapTest.java | 55 +++ .../iotdb/commons/path/PatternTreeMap.java | 41 +++ pom.xml | 2 +- 48 files changed, 2774 insertions(+), 202 deletions(-) create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/IQueryDataSource.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/QueryDataSourceForRegionScan.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/QueryDataSourceType.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/IChunkHandle.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/IFileScanHandle.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/impl/ClosedFileScanHandleImpl.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/impl/DiskAlignedChunkHandleImpl.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/impl/DiskChunkHandleImpl.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/impl/MemAlignedChunkHandleImpl.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/impl/MemChunkHandleImpl.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/impl/UnclosedFileScanHandleImpl.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/model/AbstractChunkOffset.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/model/AbstractDeviceChunkMetaData.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/model/AlignedChunkOffset.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/model/AlignedDeviceChunkMetaData.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/model/ChunkOffset.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/model/DeviceChunkMetaData.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/model/DeviceStartEndTime.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/SharedTimeDataBuffer.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/TsFileDeviceStartEndTimeIterator.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/driver/DataDriver.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/driver/DataDriver.java index ee4a0ad6d416..a225793437f8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/driver/DataDriver.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/driver/DataDriver.java @@ -23,7 +23,7 @@ import org.apache.iotdb.db.queryengine.execution.operator.Operator; import org.apache.iotdb.db.queryengine.execution.operator.source.DataSourceOperator; import org.apache.iotdb.db.queryengine.plan.planner.plan.FragmentInstance; -import org.apache.iotdb.db.storageengine.dataregion.read.QueryDataSource; +import org.apache.iotdb.db.storageengine.dataregion.read.IQueryDataSource; import com.google.common.util.concurrent.SettableFuture; @@ -68,13 +68,15 @@ protected boolean init(SettableFuture blockedFuture) { } /** - * Init seq file list and unseq file list in {@link QueryDataSource} and set it into each + * Init seq file list and unseq file list in {@link + * org.apache.iotdb.db.storageengine.dataregion.read.QueryDataSource} and set it into each * SourceNode. * * @throws QueryProcessException while failed to init query resource, QueryProcessException will * be thrown - * @throws IllegalStateException if {@link QueryDataSource} is null after initialization, - * IllegalStateException will be thrown + * @throws IllegalStateException if {@link + * org.apache.iotdb.db.storageengine.dataregion.read.QueryDataSource} is null after + * initialization, IllegalStateException will be thrown */ private void initialize() throws QueryProcessException { long startTime = System.nanoTime(); @@ -82,7 +84,7 @@ private void initialize() throws QueryProcessException { List sourceOperators = ((DataDriverContext) driverContext).getSourceOperators(); if (sourceOperators != null && !sourceOperators.isEmpty()) { - QueryDataSource dataSource = initQueryDataSource(); + IQueryDataSource dataSource = initQueryDataSource(); if (dataSource == null) { // If this driver is being initialized, meanwhile the whole FI was aborted or cancelled // for some reasons, we may get null QueryDataSource here. @@ -92,14 +94,7 @@ private void initialize() throws QueryProcessException { sourceOperators.forEach( sourceOperator -> { // Construct QueryDataSource for source operator - QueryDataSource queryDataSource = - new QueryDataSource(dataSource.getSeqResources(), dataSource.getUnseqResources()); - - queryDataSource.setSingleDevice(dataSource.isSingleDevice()); - - queryDataSource.setDataTTL(dataSource.getDataTTL()); - - sourceOperator.initQueryDataSource(queryDataSource); + sourceOperator.initQueryDataSource(dataSource.clone()); }); } @@ -118,12 +113,12 @@ protected void releaseResource() { /** * The method is called in mergeLock() when executing query. This method will get all the {@link - * QueryDataSource} needed for this query. + * org.apache.iotdb.db.storageengine.dataregion.read.QueryDataSource} needed for this query. * * @throws QueryProcessException while failed to init query resource, QueryProcessException will * be thrown */ - private QueryDataSource initQueryDataSource() throws QueryProcessException { + private IQueryDataSource initQueryDataSource() throws QueryProcessException { return ((DataDriverContext) driverContext).getSharedQueryDataSource(); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/driver/DataDriverContext.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/driver/DataDriverContext.java index c51c02c08715..335133b19072 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/driver/DataDriverContext.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/driver/DataDriverContext.java @@ -24,7 +24,7 @@ import org.apache.iotdb.db.queryengine.execution.fragment.FragmentInstanceContext; import org.apache.iotdb.db.queryengine.execution.operator.source.DataSourceOperator; import org.apache.iotdb.db.storageengine.dataregion.IDataRegionForQuery; -import org.apache.iotdb.db.storageengine.dataregion.read.QueryDataSource; +import org.apache.iotdb.db.storageengine.dataregion.read.IQueryDataSource; import java.util.ArrayList; import java.util.List; @@ -69,7 +69,7 @@ public IDataRegionForQuery getDataRegion() { return getFragmentInstanceContext().getDataRegion(); } - public QueryDataSource getSharedQueryDataSource() throws QueryProcessException { + public IQueryDataSource getSharedQueryDataSource() throws QueryProcessException { return getFragmentInstanceContext().getSharedQueryDataSource(); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java index 3d72278cc13e..7ec5503dda80 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java @@ -29,11 +29,15 @@ import org.apache.iotdb.db.queryengine.metric.SeriesScanCostMetricSet; import org.apache.iotdb.db.queryengine.plan.planner.plan.TimePredicate; import org.apache.iotdb.db.storageengine.dataregion.IDataRegionForQuery; +import org.apache.iotdb.db.storageengine.dataregion.read.IQueryDataSource; import org.apache.iotdb.db.storageengine.dataregion.read.QueryDataSource; +import org.apache.iotdb.db.storageengine.dataregion.read.QueryDataSourceForRegionScan; +import org.apache.iotdb.db.storageengine.dataregion.read.QueryDataSourceType; import org.apache.iotdb.db.storageengine.dataregion.read.control.FileReaderManager; import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; import org.apache.iotdb.mpp.rpc.thrift.TFetchFragmentInstanceStatisticsResp; +import org.apache.tsfile.file.metadata.IDeviceID; import org.apache.tsfile.read.filter.basic.Filter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -41,7 +45,6 @@ import java.time.ZoneId; import java.util.ArrayList; import java.util.HashSet; -import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Optional; @@ -64,8 +67,11 @@ public class FragmentInstanceContext extends QueryContext { // it will only be used once, after sharedQueryDataSource being inited, it will be set to null private List sourcePaths; + // Used for region scan, relating methods are to be added. + private Map devicePathsToAligned; + // Shared by all scan operators in this fragment instance to avoid memory problem - private QueryDataSource sharedQueryDataSource; + private IQueryDataSource sharedQueryDataSource; /** closed tsfile used in this fragment instance. */ private Set closedFilePaths; @@ -80,6 +86,8 @@ public class FragmentInstanceContext extends QueryContext { // empty for zero time partitions private List timePartitions; + private QueryDataSourceType queryDataSourceType = QueryDataSourceType.SERIES_SCAN; + private final AtomicLong startNanos = new AtomicLong(); private final AtomicLong endNanos = new AtomicLong(); @@ -153,6 +161,10 @@ public static FragmentInstanceContext createFragmentInstanceContextForCompaction return new FragmentInstanceContext(queryId); } + public void setQueryDataSourceType(QueryDataSourceType queryDataSourceType) { + this.queryDataSourceType = queryDataSourceType; + } + @TestOnly public static FragmentInstanceContext createFragmentInstanceContext( FragmentInstanceId id, FragmentInstanceStateMachine stateMachine) { @@ -374,8 +386,31 @@ public void initQueryDataSource(List sourcePaths) throws QueryProce if (sharedQueryDataSource != null) { closedFilePaths = new HashSet<>(); unClosedFilePaths = new HashSet<>(); - addUsedFilesForQuery(sharedQueryDataSource); - sharedQueryDataSource.setSingleDevice(selectedDeviceIdSet.size() == 1); + addUsedFilesForQuery((QueryDataSource) sharedQueryDataSource); + ((QueryDataSource) sharedQueryDataSource).setSingleDevice(selectedDeviceIdSet.size() == 1); + } + } finally { + setInitQueryDataSourceCost(System.nanoTime() - startTime); + dataRegion.readUnlock(); + } + } + + public void initRegionScanQueryDataSource(Map devicePathToAligned) + throws QueryProcessException { + long startTime = System.nanoTime(); + dataRegion.readLock(); + try { + this.sharedQueryDataSource = + dataRegion.queryForDeviceRegionScan( + devicePathToAligned, + this, + globalTimeFilter != null ? globalTimeFilter.copy() : null, + timePartitions); + + if (sharedQueryDataSource != null) { + closedFilePaths = new HashSet<>(); + unClosedFilePaths = new HashSet<>(); + addUsedFilesForRegionQuery((QueryDataSourceForRegionScan) sharedQueryDataSource); } } finally { setInitQueryDataSourceCost(System.nanoTime() - startTime); @@ -383,50 +418,111 @@ public void initQueryDataSource(List sourcePaths) throws QueryProce } } - public synchronized QueryDataSource getSharedQueryDataSource() throws QueryProcessException { + public void initRegionScanQueryDataSource(List pathList) + throws QueryProcessException { + long startTime = System.nanoTime(); + dataRegion.readLock(); + try { + this.sharedQueryDataSource = + dataRegion.queryForSeriesRegionScan( + pathList, + this, + globalTimeFilter != null ? globalTimeFilter.copy() : null, + timePartitions); + + if (sharedQueryDataSource != null) { + closedFilePaths = new HashSet<>(); + unClosedFilePaths = new HashSet<>(); + addUsedFilesForRegionQuery((QueryDataSourceForRegionScan) sharedQueryDataSource); + } + } finally { + setInitQueryDataSourceCost(System.nanoTime() - startTime); + dataRegion.readUnlock(); + } + } + + public synchronized IQueryDataSource getSharedQueryDataSource() throws QueryProcessException { if (sharedQueryDataSource == null) { - initQueryDataSource(sourcePaths); - // friendly for gc - sourcePaths = null; + switch (queryDataSourceType) { + case SERIES_SCAN: + initQueryDataSource(sourcePaths); + // Friendly for gc + sourcePaths = null; + break; + case DEVICE_REGION_SCAN: + initRegionScanQueryDataSource(devicePathsToAligned); + devicePathsToAligned = null; + break; + case TIME_SERIES_REGION_SCAN: + initRegionScanQueryDataSource(sourcePaths); + sourcePaths = null; + break; + default: + throw new QueryProcessException( + "Unsupported query data source type: " + queryDataSourceType); + } } return sharedQueryDataSource; } + /** Lock and check if tsFileResource is deleted */ + private boolean processTsFileResource(TsFileResource tsFileResource, boolean isClosed) { + addFilePathToMap(tsFileResource, isClosed); + // this file may be deleted just before we lock it + if (tsFileResource.isDeleted()) { + Set pathSet = isClosed ? closedFilePaths : unClosedFilePaths; + // This resource may be removed by other threads of this query. + if (pathSet.remove(tsFileResource)) { + FileReaderManager.getInstance().decreaseFileReaderReference(tsFileResource, isClosed); + } + return true; + } else { + return false; + } + } + /** Add the unique file paths to closeddFilePathsMap and unClosedFilePathsMap. */ private void addUsedFilesForQuery(QueryDataSource dataSource) { // sequence data - addUsedFilesForQuery(dataSource.getSeqResources()); + dataSource + .getSeqResources() + .removeIf( + tsFileResource -> processTsFileResource(tsFileResource, tsFileResource.isClosed())); // Record statistics of seqFiles unclosedSeqFileNum = unClosedFilePaths.size(); closedSeqFileNum = closedFilePaths.size(); // unsequence data - addUsedFilesForQuery(dataSource.getUnseqResources()); + dataSource + .getUnseqResources() + .removeIf( + tsFileResource -> processTsFileResource(tsFileResource, tsFileResource.isClosed())); // Record statistics of files of unseqFiles unclosedUnseqFileNum = unClosedFilePaths.size() - unclosedSeqFileNum; closedUnseqFileNum = closedFilePaths.size() - closedSeqFileNum; } - private void addUsedFilesForQuery(List resources) { - Iterator iterator = resources.iterator(); - while (iterator.hasNext()) { - TsFileResource tsFileResource = iterator.next(); - boolean isClosed = tsFileResource.isClosed(); - addFilePathToMap(tsFileResource, isClosed); - - // this file may be deleted just before we lock it - if (tsFileResource.isDeleted()) { - Set pathSet = isClosed ? closedFilePaths : unClosedFilePaths; - // This resource may be removed by other threads of this query. - if (pathSet.remove(tsFileResource)) { - FileReaderManager.getInstance().decreaseFileReaderReference(tsFileResource, isClosed); - } - iterator.remove(); - } - } + private void addUsedFilesForRegionQuery(QueryDataSourceForRegionScan dataSource) { + dataSource + .getSeqFileScanHandles() + .removeIf( + fileScanHandle -> + processTsFileResource(fileScanHandle.getTsResource(), fileScanHandle.isClosed())); + + unclosedSeqFileNum = unClosedFilePaths.size(); + closedSeqFileNum = closedFilePaths.size(); + + dataSource + .getUnseqFileScanHandles() + .removeIf( + fileScanHandle -> + processTsFileResource(fileScanHandle.getTsResource(), fileScanHandle.isClosed())); + + unclosedUnseqFileNum = unClosedFilePaths.size() - unclosedSeqFileNum; + closedUnseqFileNum = closedFilePaths.size() - closedSeqFileNum; } /** diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/QueryContext.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/QueryContext.java index e774cd6de982..a9b5c2991808 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/QueryContext.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/QueryContext.java @@ -19,6 +19,7 @@ package org.apache.iotdb.db.queryengine.execution.fragment; +import org.apache.iotdb.commons.exception.IllegalPathException; import org.apache.iotdb.commons.path.AlignedPath; import org.apache.iotdb.commons.path.PartialPath; import org.apache.iotdb.commons.path.PatternTreeMap; @@ -30,6 +31,7 @@ import org.apache.iotdb.db.utils.datastructure.PatternTreeMapFactory.ModsSerializer; import org.apache.tsfile.file.metadata.IChunkMetadata; +import org.apache.tsfile.file.metadata.IDeviceID; import java.util.ArrayList; import java.util.Collections; @@ -79,34 +81,70 @@ public QueryContext(long queryId, boolean debug, long startTime, long timeout) { this.timeout = timeout; } - /** - * Find the modifications of timeseries 'path' in 'modFile'. If they are not in the cache, read - * them from 'modFile' and put then into the cache. - */ - public List getPathModifications(TsFileResource tsFileResource, PartialPath path) { - // if the mods file does not exist, do not add it to the cache + private boolean checkIfModificationExists(TsFileResource tsFileResource) { if (nonExistentModFiles.contains(tsFileResource.getTsFileID())) { - return Collections.emptyList(); + return false; } ModificationFile modFile = tsFileResource.getModFile(); if (!modFile.exists()) { nonExistentModFiles.add(tsFileResource.getTsFileID()); + return false; + } + return true; + } + + private PatternTreeMap getAllModifications( + ModificationFile modFile) { + return fileModCache.computeIfAbsent( + modFile.getFilePath(), + k -> { + PatternTreeMap modifications = + PatternTreeMapFactory.getModsPatternTreeMap(); + for (Modification modification : modFile.getModificationsIter()) { + modifications.append(modification.getPath(), modification); + } + return modifications; + }); + } + + public List getPathModifications( + TsFileResource tsFileResource, IDeviceID deviceID, String measurement) + throws IllegalPathException { + // if the mods file does not exist, do not add it to the cache + if (!checkIfModificationExists(tsFileResource)) { + return Collections.emptyList(); + } + + return ModificationFile.sortAndMerge( + getAllModifications(tsFileResource.getModFile()) + .getOverlapped(new PartialPath(deviceID, measurement))); + } + + public List getPathModifications(TsFileResource tsFileResource, IDeviceID deviceID) + throws IllegalPathException { + // if the mods file does not exist, do not add it to the cache + if (!checkIfModificationExists(tsFileResource)) { + return Collections.emptyList(); + } + + return ModificationFile.sortAndMerge( + getAllModifications(tsFileResource.getModFile()) + .getDeviceOverlapped(new PartialPath(deviceID))); + } + + /** + * Find the modifications of timeseries 'path' in 'modFile'. If they are not in the cache, read + * them from 'modFile' and put then into the cache. + */ + public List getPathModifications(TsFileResource tsFileResource, PartialPath path) { + // if the mods file does not exist, do not add it to the cache + if (!checkIfModificationExists(tsFileResource)) { return Collections.emptyList(); } - PatternTreeMap allModifications = - fileModCache.computeIfAbsent( - modFile.getFilePath(), - k -> { - PatternTreeMap modifications = - PatternTreeMapFactory.getModsPatternTreeMap(); - for (Modification modification : modFile.getModificationsIter()) { - modifications.append(modification.getPath(), modification); - } - return modifications; - }); - return ModificationFile.sortAndMerge(allModifications.getOverlapped(path)); + return ModificationFile.sortAndMerge( + getAllModifications(tsFileResource.getModFile()).getOverlapped(path)); } /** diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/AbstractDataSourceOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/AbstractDataSourceOperator.java index 1d257eef50a4..47fe0acbf877 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/AbstractDataSourceOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/AbstractDataSourceOperator.java @@ -19,6 +19,7 @@ package org.apache.iotdb.db.queryengine.execution.operator.source; +import org.apache.iotdb.db.storageengine.dataregion.read.IQueryDataSource; import org.apache.iotdb.db.storageengine.dataregion.read.QueryDataSource; import org.apache.tsfile.enums.TSDataType; @@ -34,8 +35,8 @@ public abstract class AbstractDataSourceOperator extends AbstractSourceOperator protected TsBlockBuilder resultTsBlockBuilder; @Override - public void initQueryDataSource(QueryDataSource dataSource) { - seriesScanUtil.initQueryDataSource(dataSource); + public void initQueryDataSource(IQueryDataSource dataSource) { + seriesScanUtil.initQueryDataSource((QueryDataSource) dataSource); resultTsBlockBuilder = new TsBlockBuilder(getResultDataTypes()); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/AlignedSeriesScanOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/AlignedSeriesScanOperator.java index be4d3f439317..22cb94b9ec19 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/AlignedSeriesScanOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/AlignedSeriesScanOperator.java @@ -25,6 +25,7 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; import org.apache.iotdb.db.queryengine.plan.planner.plan.parameter.SeriesScanOptions; import org.apache.iotdb.db.queryengine.plan.statement.component.Ordering; +import org.apache.iotdb.db.storageengine.dataregion.read.IQueryDataSource; import org.apache.iotdb.db.storageengine.dataregion.read.QueryDataSource; import org.apache.tsfile.block.column.Column; @@ -224,8 +225,8 @@ protected List getResultDataTypes() { } @Override - public void initQueryDataSource(QueryDataSource dataSource) { - seriesScanUtil.initQueryDataSource(dataSource); + public void initQueryDataSource(IQueryDataSource dataSource) { + seriesScanUtil.initQueryDataSource((QueryDataSource) dataSource); resultTsBlockBuilder = new TsBlockBuilder(getResultDataTypes()); resultTsBlockBuilder.setMaxTsBlockLineNumber(this.maxTsBlockLineNum); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/DataSourceOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/DataSourceOperator.java index 3cecdd3a8a87..ec03d19a69bb 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/DataSourceOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/DataSourceOperator.java @@ -19,9 +19,9 @@ package org.apache.iotdb.db.queryengine.execution.operator.source; -import org.apache.iotdb.db.storageengine.dataregion.read.QueryDataSource; +import org.apache.iotdb.db.storageengine.dataregion.read.IQueryDataSource; public interface DataSourceOperator extends SourceOperator { - void initQueryDataSource(QueryDataSource dataSource); + void initQueryDataSource(IQueryDataSource dataSource); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java index 36ba50ef6950..812edc6b43bb 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java @@ -31,7 +31,6 @@ import org.apache.iotdb.db.storageengine.dataregion.memtable.IWritableMemChunk; import org.apache.iotdb.db.storageengine.dataregion.memtable.IWritableMemChunkGroup; import org.apache.iotdb.db.storageengine.dataregion.memtable.ReadOnlyMemChunk; -import org.apache.iotdb.db.storageengine.dataregion.modification.Deletion; import org.apache.iotdb.db.storageengine.dataregion.modification.Modification; import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; import org.apache.iotdb.db.utils.ModificationUtils; @@ -90,20 +89,6 @@ public abstract ReadOnlyMemChunk getReadOnlyMemChunkFromMemTable( public abstract List getVisibleMetadataListFromWriter( RestorableTsFileIOWriter writer, TsFileResource tsFileResource, QueryContext context); - - /** get modifications from a memtable. */ - protected List getModificationsForMemtable( - IMemTable memTable, List> modsToMemtable) { - List modifications = new ArrayList<>(); - boolean foundMemtable = false; - for (Pair entry : modsToMemtable) { - if (foundMemtable || entry.right.equals(memTable)) { - modifications.add(entry.left); - foundMemtable = true; - } - } - return modifications; - } } class AlignedResourceByPathUtils extends ResourceByPathUtils { @@ -218,7 +203,9 @@ public ReadOnlyMemChunk getReadOnlyMemChunkFromMemTable( TVList alignedTvListCopy = alignedMemChunk.getSortedTvListForQuery(partialPath.getSchemaList()); List> deletionList = null; if (modsToMemtable != null) { - deletionList = constructDeletionList(memTable, modsToMemtable, timeLowerBound); + deletionList = + ModificationUtils.constructDeletionList( + partialPath, memTable, modsToMemtable, timeLowerBound); } return new AlignedReadOnlyMemChunk( context, getMeasurementSchema(), alignedTvListCopy, deletionList); @@ -245,28 +232,6 @@ public VectorMeasurementSchema getMeasurementSchema() { partialPath.getSchemaList().get(0).getCompressor()); } - private List> constructDeletionList( - IMemTable memTable, List> modsToMemtable, long timeLowerBound) { - List> deletionList = new ArrayList<>(); - for (String measurement : partialPath.getMeasurementList()) { - List columnDeletionList = new ArrayList<>(); - columnDeletionList.add(new TimeRange(Long.MIN_VALUE, timeLowerBound)); - for (Modification modification : getModificationsForMemtable(memTable, modsToMemtable)) { - if (modification instanceof Deletion) { - Deletion deletion = (Deletion) modification; - PartialPath fullPath = partialPath.concatNode(measurement); - if (deletion.getPath().matchFullPath(fullPath) - && deletion.getEndTime() > timeLowerBound) { - long lowerBound = Math.max(deletion.getStartTime(), timeLowerBound); - columnDeletionList.add(new TimeRange(lowerBound, deletion.getEndTime())); - } - } - } - deletionList.add(TimeRange.sortAndMerge(columnDeletionList)); - } - return deletionList; - } - @Override public List getVisibleMetadataListFromWriter( RestorableTsFileIOWriter writer, TsFileResource tsFileResource, QueryContext context) { @@ -371,7 +336,9 @@ public ReadOnlyMemChunk getReadOnlyMemChunkFromMemTable( TVList chunkCopy = memChunk.getSortedTvListForQuery(); List deletionList = null; if (modsToMemtable != null) { - deletionList = constructDeletionList(memTable, modsToMemtable, timeLowerBound); + deletionList = + ModificationUtils.constructDeletionList( + partialPath, memTable, modsToMemtable, timeLowerBound); } return new ReadOnlyMemChunk( context, @@ -383,44 +350,6 @@ public ReadOnlyMemChunk getReadOnlyMemChunkFromMemTable( deletionList); } - /** - * construct a deletion list from a memtable. - * - * @param memTable memtable - * @param timeLowerBound time watermark - */ - private List constructDeletionList( - IMemTable memTable, List> modsToMemtable, long timeLowerBound) { - List deletionList = new ArrayList<>(); - deletionList.add(new TimeRange(Long.MIN_VALUE, timeLowerBound)); - for (Modification modification : getModificationsForMemtable(memTable, modsToMemtable)) { - if (modification instanceof Deletion) { - Deletion deletion = (Deletion) modification; - if (deletion.getPath().matchFullPath(partialPath) - && deletion.getEndTime() > timeLowerBound) { - long lowerBound = Math.max(deletion.getStartTime(), timeLowerBound); - deletionList.add(new TimeRange(lowerBound, deletion.getEndTime())); - } - } - } - return TimeRange.sortAndMerge(deletionList); - } - - /** get modifications from a memtable. */ - @Override - protected List getModificationsForMemtable( - IMemTable memTable, List> modsToMemtable) { - List modifications = new ArrayList<>(); - boolean foundMemtable = false; - for (Pair entry : modsToMemtable) { - if (foundMemtable || entry.right.equals(memTable)) { - modifications.add(entry.left); - foundMemtable = true; - } - } - return modifications; - } - @Override public List getVisibleMetadataListFromWriter( RestorableTsFileIOWriter writer, TsFileResource tsFileResource, QueryContext context) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java index 8a7f9ff663bf..c5cd3423e746 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java @@ -81,8 +81,12 @@ import org.apache.iotdb.db.storageengine.dataregion.memtable.TsFileProcessorInfo; import org.apache.iotdb.db.storageengine.dataregion.modification.Deletion; import org.apache.iotdb.db.storageengine.dataregion.modification.ModificationFile; +import org.apache.iotdb.db.storageengine.dataregion.read.IQueryDataSource; import org.apache.iotdb.db.storageengine.dataregion.read.QueryDataSource; +import org.apache.iotdb.db.storageengine.dataregion.read.QueryDataSourceForRegionScan; import org.apache.iotdb.db.storageengine.dataregion.read.control.FileReaderManager; +import org.apache.iotdb.db.storageengine.dataregion.read.filescan.IFileScanHandle; +import org.apache.iotdb.db.storageengine.dataregion.read.filescan.impl.ClosedFileScanHandleImpl; import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileManager; import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResourceStatus; @@ -1857,34 +1861,6 @@ public QueryDataSource query( } } - /** lock the read lock of the insert lock */ - @Override - public void readLock() { - // apply read lock for SG insert lock to prevent inconsistent with concurrently writing memtable - insertLock.readLock().lock(); - // apply read lock for TsFileResource list - tsFileManager.readLock(); - } - - /** unlock the read lock of insert lock */ - @Override - public void readUnlock() { - tsFileManager.readUnlock(); - insertLock.readLock().unlock(); - } - - /** lock the write lock of the insert lock */ - public void writeLock(String holder) { - insertLock.writeLock().lock(); - insertWriteLockHolder = holder; - } - - /** unlock the write lock of the insert lock */ - public void writeUnlock() { - insertWriteLockHolder = ""; - insertLock.writeLock().unlock(); - } - /** * @param tsFileResources includes sealed and unsealed tsfile resources * @return fill unsealed tsfile resources with memory data and ChunkMetadataList of data in disk @@ -1907,7 +1883,7 @@ private List getFileResourceListForQuery( (globalTimeFilter == null ? "null" : globalTimeFilter)); } - List tsfileResourcesForQuery = new ArrayList<>(); + List tsFileResourcesForQuery = new ArrayList<>(); long timeLowerBound = dataTTL != Long.MAX_VALUE ? CommonDateTimeUtils.currentTime() - dataTTL : Long.MIN_VALUE; @@ -1925,9 +1901,9 @@ private List getFileResourceListForQuery( closeQueryLock.readLock().lock(); try { if (tsFileResource.isClosed()) { - tsfileResourcesForQuery.add(tsFileResource); + tsFileResourcesForQuery.add(tsFileResource); } else { - tsFileResource.getProcessor().query(pathList, context, tsfileResourcesForQuery); + tsFileResource.getProcessor().query(pathList, context, tsFileResourcesForQuery); } } catch (IOException e) { throw new MetadataException(e); @@ -1935,7 +1911,175 @@ private List getFileResourceListForQuery( closeQueryLock.readLock().unlock(); } } - return tsfileResourcesForQuery; + return tsFileResourcesForQuery; + } + + @Override + public IQueryDataSource queryForSeriesRegionScan( + List pathList, + QueryContext queryContext, + Filter globalTimeFilter, + List timePartitions) + throws QueryProcessException { + try { + List seqFileScanHandles = + getFileHandleListForQuery( + tsFileManager.getTsFileList(true, timePartitions, globalTimeFilter), + pathList, + queryContext, + globalTimeFilter, + true); + List unseqFileScanHandles = + getFileHandleListForQuery( + tsFileManager.getTsFileList(false, timePartitions, globalTimeFilter), + pathList, + queryContext, + globalTimeFilter, + false); + + QUERY_RESOURCE_METRIC_SET.recordQueryResourceNum(SEQUENCE_TSFILE, seqFileScanHandles.size()); + QUERY_RESOURCE_METRIC_SET.recordQueryResourceNum( + UNSEQUENCE_TSFILE, unseqFileScanHandles.size()); + + QueryDataSourceForRegionScan dataSource = + new QueryDataSourceForRegionScan(seqFileScanHandles, unseqFileScanHandles); + dataSource.setDataTTL(dataTTL); + return dataSource; + } catch (MetadataException e) { + throw new QueryProcessException(e); + } + } + + private List getFileHandleListForQuery( + Collection tsFileResources, + List partialPaths, + QueryContext context, + Filter globalTimeFilter, + boolean isSeq) + throws MetadataException { + + List fileScanHandles = new ArrayList<>(); + + long timeLowerBound = + dataTTL != Long.MAX_VALUE ? CommonDateTimeUtils.currentTime() - dataTTL : Long.MIN_VALUE; + context.setQueryTimeLowerBound(timeLowerBound); + + for (TsFileResource tsFileResource : tsFileResources) { + if (!tsFileResource.isSatisfied(null, globalTimeFilter, isSeq, dataTTL, context.isDebug())) { + continue; + } + closeQueryLock.readLock().lock(); + try { + if (tsFileResource.isClosed()) { + fileScanHandles.add(new ClosedFileScanHandleImpl(tsFileResource, context)); + } else { + tsFileResource + .getProcessor() + .queryForSeriesRegionScan(partialPaths, context, fileScanHandles); + } + } finally { + closeQueryLock.readLock().unlock(); + } + } + return fileScanHandles; + } + + @Override + public IQueryDataSource queryForDeviceRegionScan( + Map devicePathToAligned, + QueryContext queryContext, + Filter globalTimeFilter, + List timePartitions) + throws QueryProcessException { + try { + List seqFileScanHandles = + getFileHandleListForQuery( + tsFileManager.getTsFileList(true, timePartitions, globalTimeFilter), + devicePathToAligned, + queryContext, + globalTimeFilter, + true); + List unseqFileScanHandles = + getFileHandleListForQuery( + tsFileManager.getTsFileList(false, timePartitions, globalTimeFilter), + devicePathToAligned, + queryContext, + globalTimeFilter, + false); + + QUERY_RESOURCE_METRIC_SET.recordQueryResourceNum(SEQUENCE_TSFILE, seqFileScanHandles.size()); + QUERY_RESOURCE_METRIC_SET.recordQueryResourceNum( + UNSEQUENCE_TSFILE, unseqFileScanHandles.size()); + + QueryDataSourceForRegionScan dataSource = + new QueryDataSourceForRegionScan(seqFileScanHandles, unseqFileScanHandles); + dataSource.setDataTTL(dataTTL); + return dataSource; + } catch (MetadataException e) { + throw new QueryProcessException(e); + } + } + + private List getFileHandleListForQuery( + Collection tsFileResources, + Map devicePathToAligned, + QueryContext context, + Filter globalTimeFilter, + boolean isSeq) + throws MetadataException { + + List fileScanHandles = new ArrayList<>(); + + long timeLowerBound = + dataTTL != Long.MAX_VALUE ? CommonDateTimeUtils.currentTime() - dataTTL : Long.MIN_VALUE; + context.setQueryTimeLowerBound(timeLowerBound); + + for (TsFileResource tsFileResource : tsFileResources) { + if (!tsFileResource.isSatisfied(null, globalTimeFilter, isSeq, dataTTL, context.isDebug())) { + continue; + } + closeQueryLock.readLock().lock(); + try { + if (tsFileResource.isClosed()) { + fileScanHandles.add(new ClosedFileScanHandleImpl(tsFileResource, context)); + } else { + tsFileResource + .getProcessor() + .queryForDeviceRegionScan(devicePathToAligned, context, fileScanHandles); + } + } finally { + closeQueryLock.readLock().unlock(); + } + } + return fileScanHandles; + } + + /** lock the read lock of the insert lock */ + @Override + public void readLock() { + // apply read lock for SG insert lock to prevent inconsistent with concurrently writing memtable + insertLock.readLock().lock(); + // apply read lock for TsFileResource list + tsFileManager.readLock(); + } + + /** unlock the read lock of insert lock */ + @Override + public void readUnlock() { + tsFileManager.readUnlock(); + insertLock.readLock().unlock(); + } + + /** lock the write lock of the insert lock */ + public void writeLock(String holder) { + insertLock.writeLock().lock(); + insertWriteLockHolder = holder; + } + + /** unlock the write lock of the insert lock */ + public void writeUnlock() { + insertWriteLockHolder = ""; + insertLock.writeLock().unlock(); } /** Seperate tsfiles in TsFileManager to sealedList and unsealedList. */ diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/IDataRegionForQuery.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/IDataRegionForQuery.java index a0a52cf867b4..045ac288b57f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/IDataRegionForQuery.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/IDataRegionForQuery.java @@ -21,11 +21,14 @@ import org.apache.iotdb.commons.path.PartialPath; import org.apache.iotdb.db.exception.query.QueryProcessException; import org.apache.iotdb.db.queryengine.execution.fragment.QueryContext; +import org.apache.iotdb.db.storageengine.dataregion.read.IQueryDataSource; import org.apache.iotdb.db.storageengine.dataregion.read.QueryDataSource; +import org.apache.tsfile.file.metadata.IDeviceID; import org.apache.tsfile.read.filter.basic.Filter; import java.util.List; +import java.util.Map; /** It's an interface that storage engine must provide for query engine */ public interface IDataRegionForQuery { @@ -35,7 +38,7 @@ public interface IDataRegionForQuery { void readUnlock(); - /** Get satisfied QueryDataSource from DataRegion */ + /** Get satisfied QueryDataSource from DataRegion for seriesScan */ QueryDataSource query( List pathList, String singleDeviceId, @@ -44,6 +47,21 @@ QueryDataSource query( List timePartitions) throws QueryProcessException; + /** Get satisfied QueryDataSource from DataRegion for regionScan */ + IQueryDataSource queryForDeviceRegionScan( + Map devicePathToAligned, + QueryContext queryContext, + Filter globalTimeFilter, + List timePartitions) + throws QueryProcessException; + + IQueryDataSource queryForSeriesRegionScan( + List pathList, + QueryContext queryContext, + Filter globalTimeFilter, + List timePartitions) + throws QueryProcessException; + /** Get TTL of this DataRegion */ long getDataTTL(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/VirtualDataRegion.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/VirtualDataRegion.java index 6be5449f9b40..45c1ad75c0e0 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/VirtualDataRegion.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/VirtualDataRegion.java @@ -21,12 +21,16 @@ import org.apache.iotdb.commons.path.PartialPath; import org.apache.iotdb.db.exception.query.QueryProcessException; import org.apache.iotdb.db.queryengine.execution.fragment.QueryContext; +import org.apache.iotdb.db.storageengine.dataregion.read.IQueryDataSource; import org.apache.iotdb.db.storageengine.dataregion.read.QueryDataSource; +import org.apache.iotdb.db.storageengine.dataregion.read.QueryDataSourceForRegionScan; +import org.apache.tsfile.file.metadata.IDeviceID; import org.apache.tsfile.read.filter.basic.Filter; import java.util.Collections; import java.util.List; +import java.util.Map; /** * It's a virtual data region used for query which contains time series that don't belong to any @@ -39,6 +43,9 @@ public class VirtualDataRegion implements IDataRegionForQuery { private static final QueryDataSource EMPTY_QUERY_DATA_SOURCE = new QueryDataSource(Collections.emptyList(), Collections.emptyList()); + private static final QueryDataSourceForRegionScan EMPTY_REGION_QUERY_DATA_SOURCE = + new QueryDataSourceForRegionScan(Collections.emptyList(), Collections.emptyList()); + public static VirtualDataRegion getInstance() { return VirtualDataRegion.InstanceHolder.INSTANCE; } @@ -64,6 +71,26 @@ public QueryDataSource query( return EMPTY_QUERY_DATA_SOURCE; } + @Override + public IQueryDataSource queryForDeviceRegionScan( + Map devicePathToAligned, + QueryContext queryContext, + Filter globalTimeFilter, + List timePartitions) + throws QueryProcessException { + return EMPTY_REGION_QUERY_DATA_SOURCE; + } + + @Override + public IQueryDataSource queryForSeriesRegionScan( + List pathList, + QueryContext queryContext, + Filter globalTimeFilter, + List timePartitions) + throws QueryProcessException { + return EMPTY_REGION_QUERY_DATA_SOURCE; + } + @Override public long getDataTTL() { return Long.MAX_VALUE; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AbstractMemTable.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AbstractMemTable.java index 2182cec0007e..550b35eeee23 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AbstractMemTable.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AbstractMemTable.java @@ -20,6 +20,9 @@ package org.apache.iotdb.db.storageengine.dataregion.memtable; import org.apache.iotdb.commons.exception.IllegalPathException; +import org.apache.iotdb.commons.exception.MetadataException; +import org.apache.iotdb.commons.path.AlignedPath; +import org.apache.iotdb.commons.path.MeasurementPath; import org.apache.iotdb.commons.path.PartialPath; import org.apache.iotdb.commons.service.metric.MetricService; import org.apache.iotdb.commons.service.metric.enums.Metric; @@ -34,14 +37,25 @@ import org.apache.iotdb.db.storageengine.dataregion.flush.FlushStatus; import org.apache.iotdb.db.storageengine.dataregion.flush.NotifyFlushMemTable; import org.apache.iotdb.db.storageengine.dataregion.modification.Modification; +import org.apache.iotdb.db.storageengine.dataregion.read.filescan.IChunkHandle; +import org.apache.iotdb.db.storageengine.dataregion.read.filescan.impl.MemAlignedChunkHandleImpl; +import org.apache.iotdb.db.storageengine.dataregion.read.filescan.impl.MemChunkHandleImpl; import org.apache.iotdb.db.storageengine.dataregion.wal.buffer.IWALByteBufferView; import org.apache.iotdb.db.storageengine.dataregion.wal.utils.WALWriteUtils; import org.apache.iotdb.db.utils.MemUtils; +import org.apache.iotdb.db.utils.ModificationUtils; +import org.apache.iotdb.db.utils.datastructure.AlignedTVList; +import org.apache.iotdb.db.utils.datastructure.TVList; import org.apache.iotdb.metrics.utils.MetricLevel; import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.file.metadata.ChunkMetadata; +import org.apache.tsfile.file.metadata.IChunkMetadata; import org.apache.tsfile.file.metadata.IDeviceID; import org.apache.tsfile.file.metadata.PlainDeviceID; +import org.apache.tsfile.file.metadata.statistics.TimeStatistics; +import org.apache.tsfile.read.common.TimeRange; +import org.apache.tsfile.utils.BitMap; import org.apache.tsfile.utils.Pair; import org.apache.tsfile.utils.ReadWriteIOUtils; import org.apache.tsfile.write.schema.IMeasurementSchema; @@ -49,6 +63,7 @@ import java.io.DataInputStream; import java.io.IOException; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -57,6 +72,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; +import java.util.stream.LongStream; public abstract class AbstractMemTable implements IMemTable { /** Each memTable node has a unique int value identifier, init when recovering wal. */ @@ -465,6 +481,301 @@ public ReadOnlyMemChunk query( .getReadOnlyMemChunkFromMemTable(context, this, modsToMemtable, ttlLowerBound); } + @Override + public void queryForSeriesRegionScan( + PartialPath fullPath, + long ttlLowerBound, + Map> chunkMetaDataMap, + Map> memChunkHandleMap, + List> modsToMemTabled) { + + IDeviceID deviceID = DeviceIDFactory.getInstance().getDeviceID(fullPath.getDevicePath()); + String measurementId = fullPath.getMeasurement(); + Map memTableMap = getMemTableMap(); + + // check If MemTable Contains this path + if (!memTableMap.containsKey(deviceID) || !memTableMap.get(deviceID).contains(measurementId)) { + return; + } + + if (fullPath instanceof MeasurementPath) { + List deletionList = new ArrayList<>(); + if (modsToMemTabled != null) { + deletionList = + ModificationUtils.constructDeletionList( + (MeasurementPath) fullPath, this, modsToMemTabled, ttlLowerBound); + } + getMemChunkHandleFromMemTable( + deviceID, measurementId, chunkMetaDataMap, memChunkHandleMap, deletionList); + } else { + List> deletionList = new ArrayList<>(); + if (modsToMemTabled != null) { + deletionList = + ModificationUtils.constructDeletionList( + (AlignedPath) fullPath, this, modsToMemTabled, ttlLowerBound); + } + + getMemAlignedChunkHandleFromMemTable( + deviceID, + ((AlignedPath) fullPath).getSchemaList(), + chunkMetaDataMap, + memChunkHandleMap, + deletionList); + } + } + + @Override + public void queryForDeviceRegionScan( + IDeviceID deviceID, + boolean isAligned, + long ttlLowerBound, + Map> chunkMetadataMap, + Map> memChunkHandleMap, + List> modsToMemTabled) + throws MetadataException { + + Map memTableMap = getMemTableMap(); + + // check If MemTable Contains this path + if (!memTableMap.containsKey(deviceID)) { + return; + } + + IWritableMemChunkGroup writableMemChunkGroup = memTableMap.get(deviceID); + if (isAligned) { + getMemAlignedChunkHandleFromMemTable( + deviceID, + (AlignedWritableMemChunkGroup) writableMemChunkGroup, + chunkMetadataMap, + memChunkHandleMap, + ttlLowerBound, + modsToMemTabled); + } else { + getMemChunkHandleFromMemTable( + deviceID, + (WritableMemChunkGroup) writableMemChunkGroup, + chunkMetadataMap, + memChunkHandleMap, + ttlLowerBound, + modsToMemTabled); + } + } + + private void getMemChunkHandleFromMemTable( + IDeviceID deviceID, + String measurementId, + Map> chunkMetadataMap, + Map> memChunkHandleMap, + List deletionList) { + + IWritableMemChunk memChunk = memTableMap.get(deviceID).getMemChunkMap().get(measurementId); + + TVList tvListCopy = memChunk.getSortedTvListForQuery(); + long[] timestamps = filterDeletedTimestamp(tvListCopy, deletionList); + + chunkMetadataMap + .computeIfAbsent(measurementId, k -> new ArrayList<>()) + .add( + buildChunkMetaDataForMemoryChunk( + measurementId, + timestamps[0], + timestamps[timestamps.length - 1], + Collections.emptyList())); + memChunkHandleMap + .computeIfAbsent(measurementId, k -> new ArrayList<>()) + .add(new MemChunkHandleImpl(timestamps)); + } + + private void getMemAlignedChunkHandleFromMemTable( + IDeviceID deviceID, + List schemaList, + Map> chunkMetadataList, + Map> memChunkHandleMap, + List> deletionList) { + + AlignedWritableMemChunk alignedMemChunk = + ((AlignedWritableMemChunkGroup) memTableMap.get(deviceID)).getAlignedMemChunk(); + + boolean containsMeasurement = false; + for (IMeasurementSchema measurementSchema : schemaList) { + if (alignedMemChunk.containsMeasurement(measurementSchema.getMeasurementId())) { + containsMeasurement = true; + break; + } + } + if (!containsMeasurement) { + return; + } + + AlignedTVList alignedTVListCopy = + (AlignedTVList) alignedMemChunk.getSortedTvListForQuery(schemaList); + + buildAlignedMemChunkHandle( + alignedTVListCopy, deletionList, schemaList, chunkMetadataList, memChunkHandleMap); + } + + private void getMemAlignedChunkHandleFromMemTable( + IDeviceID deviceID, + AlignedWritableMemChunkGroup writableMemChunkGroup, + Map> chunkMetadataList, + Map> memChunkHandleMap, + long ttlLowerBound, + List> modsToMemTabled) + throws IllegalPathException { + + AlignedWritableMemChunk memChunk = writableMemChunkGroup.getAlignedMemChunk(); + List schemaList = memChunk.getSchemaList(); + + AlignedTVList alignedTVListCopy = (AlignedTVList) memChunk.getSortedTvListForQuery(schemaList); + + List> deletionList = new ArrayList<>(); + if (modsToMemTabled != null) { + for (IMeasurementSchema schema : schemaList) { + deletionList.add( + ModificationUtils.constructDeletionList( + new MeasurementPath(deviceID, schema.getMeasurementId(), schema), + this, + modsToMemTabled, + ttlLowerBound)); + } + } + buildAlignedMemChunkHandle( + alignedTVListCopy, deletionList, schemaList, chunkMetadataList, memChunkHandleMap); + } + + private void getMemChunkHandleFromMemTable( + IDeviceID deviceID, + WritableMemChunkGroup writableMemChunkGroup, + Map> chunkMetadataMap, + Map> memChunkHandleMap, + long ttlLowerBound, + List> modsToMemTabled) + throws IllegalPathException { + + for (Entry entry : + writableMemChunkGroup.getMemChunkMap().entrySet()) { + + String measurementId = entry.getKey(); + IWritableMemChunk writableMemChunk = entry.getValue(); + TVList tvListCopy = writableMemChunk.getSortedTvListForQuery(); + + List deletionList = new ArrayList<>(); + if (modsToMemTabled != null) { + deletionList = + ModificationUtils.constructDeletionList( + new MeasurementPath(deviceID, measurementId, null), + this, + modsToMemTabled, + ttlLowerBound); + } + long[] timestamps = filterDeletedTimestamp(tvListCopy, deletionList); + chunkMetadataMap + .computeIfAbsent(measurementId, k -> new ArrayList<>()) + .add( + buildChunkMetaDataForMemoryChunk( + measurementId, + timestamps[0], + timestamps[timestamps.length - 1], + Collections.emptyList())); + memChunkHandleMap + .computeIfAbsent(measurementId, k -> new ArrayList<>()) + .add(new MemChunkHandleImpl(timestamps)); + } + } + + private void buildAlignedMemChunkHandle( + AlignedTVList alignedTVList, + List> deletionList, + List schemaList, + Map> chunkMetadataList, + Map> chunkHandleMap) { + + List> bitMaps = alignedTVList.getBitMaps(); + long[] timestamps = + alignedTVList.getTimestamps().stream().flatMapToLong(LongStream::of).toArray(); + for (int i = 0; i < schemaList.size(); i++) { + String measurement = schemaList.get(i).getMeasurementId(); + long[] startEndTime = calculateStartEndTime(timestamps, bitMaps.get(i)); + chunkMetadataList + .computeIfAbsent(measurement, k -> new ArrayList<>()) + .add( + buildChunkMetaDataForMemoryChunk( + measurement, startEndTime[0], startEndTime[1], deletionList.get(i))); + chunkHandleMap + .computeIfAbsent(measurement, k -> new ArrayList<>()) + .add( + new MemAlignedChunkHandleImpl( + timestamps, bitMaps.get(i), deletionList.get(i), startEndTime)); + } + } + + private long[] calculateStartEndTime(long[] timestamps, List bitMaps) { + long startTime = -1; + for (int i = 0; i < bitMaps.size(); i++) { + BitMap bitMap = bitMaps.get(i); + for (int j = 0; j < bitMap.getSize(); j++) { + if (!bitMap.isMarked(j)) { + startTime = timestamps[i]; + break; + } + } + if (startTime != -1) { + break; + } + } + + long endTime = -1; + for (int i = bitMaps.size() - 1; i >= 0; i--) { + BitMap bitMap = bitMaps.get(i); + for (int j = bitMap.getSize() - 1; j >= 0; j--) { + if (!bitMap.isMarked(j)) { + endTime = timestamps[i]; + break; + } + } + if (endTime != -1) { + break; + } + } + return new long[] {startTime, endTime}; + } + + private IChunkMetadata buildChunkMetaDataForMemoryChunk( + String measurement, long startTime, long endTime, List deletionList) { + TimeStatistics timeStatistics = new TimeStatistics(); + timeStatistics.setStartTime(startTime); + timeStatistics.setEndTime(endTime); + + // ChunkMetaData for memory is only used to get time statistics, the dataType is irrelevant. + IChunkMetadata chunkMetadata = + new ChunkMetadata(measurement, TSDataType.UNKNOWN, 0, timeStatistics); + for (TimeRange timeRange : deletionList) { + chunkMetadata.insertIntoSortedDeletions(timeRange); + } + return chunkMetadata; + } + + private long[] filterDeletedTimestamp(TVList tvList, List deletionList) { + if (deletionList.isEmpty()) { + return tvList.getTimestamps().stream().flatMapToLong(LongStream::of).toArray(); + } + + long lastTime = -1; + int[] deletionCursor = {0}; + int rowCount = tvList.rowCount(); + List result = new ArrayList<>(); + + for (int i = 0; i < rowCount; i++) { + long curTime = tvList.getTime(i); + if (!ModificationUtils.isPointDeleted(curTime, deletionList, deletionCursor) + && (i == rowCount - 1 || curTime != lastTime)) { + result.add(curTime); + } + lastTime = curTime; + } + return result.stream().mapToLong(Long::longValue).toArray(); + } + /** * Delete data by path and timeStamp. * diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java index 3160668b7c74..bf9fd47d93e9 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java @@ -537,4 +537,8 @@ public static AlignedWritableMemChunk deserialize(DataInputStream stream) throws AlignedTVList list = (AlignedTVList) TVList.deserialize(stream); return new AlignedWritableMemChunk(schemaList, list); } + + public List getSchemaList() { + return schemaList; + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IMemTable.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IMemTable.java index 8a43573950cd..d68d1ef010d4 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IMemTable.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IMemTable.java @@ -27,8 +27,10 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertTabletNode; import org.apache.iotdb.db.storageengine.dataregion.flush.FlushStatus; import org.apache.iotdb.db.storageengine.dataregion.modification.Modification; +import org.apache.iotdb.db.storageengine.dataregion.read.filescan.IChunkHandle; import org.apache.iotdb.db.storageengine.dataregion.wal.buffer.WALEntryValue; +import org.apache.tsfile.file.metadata.IChunkMetadata; import org.apache.tsfile.file.metadata.IDeviceID; import org.apache.tsfile.utils.Pair; import org.apache.tsfile.write.schema.IMeasurementSchema; @@ -122,6 +124,23 @@ ReadOnlyMemChunk query( List> modsToMemtabled) throws IOException, QueryProcessException, MetadataException; + void queryForSeriesRegionScan( + PartialPath fullPath, + long ttlLowerBound, + Map> chunkMetadataMap, + Map> memChunkHandleMap, + List> modsToMemtabled) + throws IOException, QueryProcessException, MetadataException; + + void queryForDeviceRegionScan( + IDeviceID deviceID, + boolean isAligned, + long ttlLowerBound, + Map> chunkMetadataMap, + Map> memChunkHandleMap, + List> modsToMemtabled) + throws IOException, QueryProcessException, MetadataException; + /** putBack all the memory resources. */ void clear(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/TsFileProcessor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/TsFileProcessor.java index 819d5bae8a3d..8f771f3ce039 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/TsFileProcessor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/TsFileProcessor.java @@ -21,6 +21,7 @@ import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.commons.conf.CommonDescriptor; +import org.apache.iotdb.commons.exception.IllegalPathException; import org.apache.iotdb.commons.exception.MetadataException; import org.apache.iotdb.commons.path.AlignedPath; import org.apache.iotdb.commons.path.PartialPath; @@ -54,7 +55,13 @@ import org.apache.iotdb.db.storageengine.dataregion.flush.NotifyFlushMemTable; import org.apache.iotdb.db.storageengine.dataregion.modification.Deletion; import org.apache.iotdb.db.storageengine.dataregion.modification.Modification; +import org.apache.iotdb.db.storageengine.dataregion.read.filescan.IChunkHandle; +import org.apache.iotdb.db.storageengine.dataregion.read.filescan.IFileScanHandle; +import org.apache.iotdb.db.storageengine.dataregion.read.filescan.impl.DiskAlignedChunkHandleImpl; +import org.apache.iotdb.db.storageengine.dataregion.read.filescan.impl.DiskChunkHandleImpl; +import org.apache.iotdb.db.storageengine.dataregion.read.filescan.impl.UnclosedFileScanHandleImpl; import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; +import org.apache.iotdb.db.storageengine.dataregion.utils.SharedTimeDataBuffer; import org.apache.iotdb.db.storageengine.dataregion.wal.WALManager; import org.apache.iotdb.db.storageengine.dataregion.wal.node.IWALNode; import org.apache.iotdb.db.storageengine.dataregion.wal.utils.listener.AbstractResultListener; @@ -63,12 +70,14 @@ import org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager; import org.apache.iotdb.db.storageengine.rescon.memory.SystemInfo; import org.apache.iotdb.db.utils.MemUtils; +import org.apache.iotdb.db.utils.ModificationUtils; import org.apache.iotdb.db.utils.datastructure.AlignedTVList; import org.apache.iotdb.db.utils.datastructure.TVList; import org.apache.iotdb.rpc.RpcUtils; import org.apache.iotdb.rpc.TSStatusCode; import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.file.metadata.AlignedChunkMetadata; import org.apache.tsfile.file.metadata.ChunkMetadata; import org.apache.tsfile.file.metadata.IChunkMetadata; import org.apache.tsfile.file.metadata.IDeviceID; @@ -1612,6 +1621,319 @@ public String getStorageGroupName() { return storageGroupName; } + private void processAlignedChunkMetaDataFromFlushedMemTable( + AlignedChunkMetadata alignedChunkMetadata, + Map> measurementToChunkMetaMap, + Map> measurementToChunkHandleMap, + String filePath) { + SharedTimeDataBuffer sharedTimeDataBuffer = + new SharedTimeDataBuffer(alignedChunkMetadata.getTimeChunkMetadata()); + for (IChunkMetadata valueChunkMetaData : alignedChunkMetadata.getValueChunkMetadataList()) { + measurementToChunkMetaMap + .computeIfAbsent(valueChunkMetaData.getMeasurementUid(), k -> new ArrayList<>()) + .add(valueChunkMetaData); + measurementToChunkHandleMap + .computeIfAbsent(valueChunkMetaData.getMeasurementUid(), k -> new ArrayList<>()) + .add( + new DiskAlignedChunkHandleImpl( + filePath, + false, + valueChunkMetaData.getOffsetOfChunkHeader(), + valueChunkMetaData.getStatistics(), + sharedTimeDataBuffer)); + } + } + + private void processChunkMetaDataFromFlushedMemTable( + ChunkMetadata chunkMetadata, + Map> measurementToChunkMetaMap, + Map> measurementToChunkHandleMap, + String filePath) { + measurementToChunkMetaMap + .computeIfAbsent(chunkMetadata.getMeasurementUid(), k -> new ArrayList<>()) + .add(chunkMetadata); + measurementToChunkHandleMap + .computeIfAbsent(chunkMetadata.getMeasurementUid(), k -> new ArrayList<>()) + .add( + new DiskChunkHandleImpl( + filePath, + false, + chunkMetadata.getOffsetOfChunkHeader(), + chunkMetadata.getStatistics())); + } + + private void buildChunkHandleForFlushedMemTable( + List chunkMetadataList, + Map> measurementToChunkMetaList, + Map> measurementToChunkHandleList) { + for (IChunkMetadata chunkMetadata : chunkMetadataList) { + if (chunkMetadata instanceof AlignedChunkMetadata) { + processAlignedChunkMetaDataFromFlushedMemTable( + (AlignedChunkMetadata) chunkMetadata, + measurementToChunkMetaList, + measurementToChunkHandleList, + this.tsFileResource.getTsFilePath()); + } else { + processChunkMetaDataFromFlushedMemTable( + (ChunkMetadata) chunkMetadata, + measurementToChunkMetaList, + measurementToChunkHandleList, + this.tsFileResource.getTsFilePath()); + } + } + } + + private int searchTimeChunkMetaDataIndexAndSetModifications( + List> chunkMetaDataList, + IDeviceID deviceID, + List> modifications, + QueryContext context) + throws QueryProcessException { + int timeChunkMetaDataIndex = -1; + for (int i = 0; i < chunkMetaDataList.size(); i++) { + List chunkMetadata = chunkMetaDataList.get(i); + String measurement = chunkMetadata.get(0).getMeasurementUid(); + // measurement = "" means this is a timeChunkMetadata + if (measurement.isEmpty()) { + timeChunkMetaDataIndex = i; + continue; + } + + try { + modifications.add(context.getPathModifications(tsFileResource, deviceID, measurement)); + } catch (IllegalPathException e) { + throw new QueryProcessException(e.getMessage()); + } + } + return timeChunkMetaDataIndex; + } + + private List getVisibleMetadataListFromWriterByDeviceID( + QueryContext queryContext, IDeviceID deviceID) throws IllegalPathException { + + List> chunkMetaDataListForDevice = + writer.getVisibleMetadataList(deviceID, null); + List processedChunkMetadataForOneDevice = new ArrayList<>(); + for (List chunkMetadataList : chunkMetaDataListForDevice) { + if (chunkMetadataList.isEmpty()) { + continue; + } + ModificationUtils.modifyChunkMetaData( + chunkMetadataList, + queryContext.getPathModifications( + tsFileResource, deviceID, chunkMetadataList.get(0).getMeasurementUid())); + chunkMetadataList.removeIf(queryContext::chunkNotSatisfy); + processedChunkMetadataForOneDevice.addAll(chunkMetadataList); + } + return new ArrayList<>(processedChunkMetadataForOneDevice); + } + + private List getAlignedVisibleMetadataListFromWriterByDeviceID( + QueryContext queryContext, IDeviceID deviceID) throws QueryProcessException { + + List alignedChunkMetadataForOneDevice = new ArrayList<>(); + List> modifications = new ArrayList<>(); + List> chunkMetaDataListForDevice = + writer.getVisibleMetadataList(deviceID, null); + + int timeChunkMetadataListIndex = + searchTimeChunkMetaDataIndexAndSetModifications( + chunkMetaDataListForDevice, deviceID, modifications, queryContext); + if (timeChunkMetadataListIndex == -1) { + throw new QueryProcessException("TimeChunkMetadata in aligned device should not be empty"); + } + List timeChunkMetadataList = + chunkMetaDataListForDevice.get(timeChunkMetadataListIndex); + + for (int i = 0; i < timeChunkMetadataList.size(); i++) { + List valuesChunkMetadata = new ArrayList<>(); + boolean exits = false; + for (int j = 0; j < chunkMetaDataListForDevice.size(); j++) { + List chunkMetadataList = chunkMetaDataListForDevice.get(j); + // Filter timeChunkMetadata + if (j == timeChunkMetadataListIndex || chunkMetadataList.isEmpty()) { + continue; + } + boolean currentExist = i < chunkMetadataList.size(); + exits = (exits || currentExist); + valuesChunkMetadata.add(currentExist ? chunkMetadataList.get(i) : null); + } + if (exits) { + alignedChunkMetadataForOneDevice.add( + new AlignedChunkMetadata(timeChunkMetadataList.get(i), valuesChunkMetadata)); + } + } + + ModificationUtils.modifyAlignedChunkMetaData(alignedChunkMetadataForOneDevice, modifications); + alignedChunkMetadataForOneDevice.removeIf(queryContext::chunkNotSatisfy); + return new ArrayList<>(alignedChunkMetadataForOneDevice); + } + + public void queryForSeriesRegionScan( + List pathList, + QueryContext queryContext, + List fileScanHandlesForQuery) { + long startTime = System.nanoTime(); + try { + Map>> deviceToMemChunkHandleMap = new HashMap<>(); + Map>> deviceToChunkMetadataListMap = + new HashMap<>(); + flushQueryLock.readLock().lock(); + try { + for (PartialPath seriesPath : pathList) { + Map> measurementToChunkMetaList = new HashMap<>(); + Map> measurementToChunkHandleList = new HashMap<>(); + for (IMemTable flushingMemTable : flushingMemTables) { + if (flushingMemTable.isSignalMemTable()) { + continue; + } + flushingMemTable.queryForSeriesRegionScan( + seriesPath, + queryContext.getQueryTimeLowerBound(), + measurementToChunkMetaList, + measurementToChunkHandleList, + modsToMemtable); + if (workMemTable != null) { + workMemTable.queryForSeriesRegionScan( + seriesPath, + queryContext.getQueryTimeLowerBound(), + measurementToChunkMetaList, + measurementToChunkHandleList, + null); + } + + // Some memTable have been flushed already, so we need to get the chunk metadata from + // writer and build chunk handle for disk scanning + buildChunkHandleForFlushedMemTable( + ResourceByPathUtils.getResourceInstance(seriesPath) + .getVisibleMetadataListFromWriter(writer, tsFileResource, queryContext), + measurementToChunkMetaList, + measurementToChunkHandleList); + + IDeviceID devicePath = + DeviceIDFactory.getInstance().getDeviceID(seriesPath.getDevice()); + if (!measurementToChunkHandleList.isEmpty() || !measurementToChunkMetaList.isEmpty()) { + deviceToMemChunkHandleMap.put(devicePath, measurementToChunkHandleList); + deviceToChunkMetadataListMap.put(devicePath, measurementToChunkMetaList); + } + } + } + } catch (QueryProcessException | MetadataException | IOException e) { + logger.error( + "{}: {} get ReadOnlyMemChunk has error", + storageGroupName, + tsFileResource.getTsFile().getName(), + e); + } finally { + QUERY_RESOURCE_METRICS.recordQueryResourceNum(FLUSHING_MEMTABLE, flushingMemTables.size()); + QUERY_RESOURCE_METRICS.recordQueryResourceNum( + WORKING_MEMTABLE, workMemTable != null ? 1 : 0); + + flushQueryLock.readLock().unlock(); + if (logger.isDebugEnabled()) { + logger.debug( + "{}: {} release flushQueryLock", + storageGroupName, + tsFileResource.getTsFile().getName()); + } + } + if (!deviceToMemChunkHandleMap.isEmpty() || !deviceToChunkMetadataListMap.isEmpty()) { + fileScanHandlesForQuery.add( + new UnclosedFileScanHandleImpl( + deviceToChunkMetadataListMap, deviceToMemChunkHandleMap, tsFileResource)); + } + } finally { + QUERY_EXECUTION_METRICS.recordExecutionCost( + GET_QUERY_RESOURCE_FROM_MEM, System.nanoTime() - startTime); + } + } + + /** + * Construct IFileScanHandle for data in memtable and the other ones in flushing memtables. Then + * get the related ChunkMetadata of data on disk. + */ + public void queryForDeviceRegionScan( + Map devicePathToAligned, + QueryContext queryContext, + List fileScanHandlesForQuery) { + long startTime = System.nanoTime(); + try { + Map>> deviceToMemChunkHandleMap = new HashMap<>(); + Map>> deviceToChunkMetadataListMap = + new HashMap<>(); + flushQueryLock.readLock().lock(); + try { + for (Map.Entry entry : devicePathToAligned.entrySet()) { + IDeviceID devicePath = entry.getKey(); + boolean isAligned = entry.getValue(); + Map> measurementToChunkMetadataList = new HashMap<>(); + Map> measurementToMemChunkHandleList = new HashMap<>(); + for (IMemTable flushingMemTable : flushingMemTables) { + if (flushingMemTable.isSignalMemTable()) { + continue; + } + flushingMemTable.queryForDeviceRegionScan( + devicePath, + isAligned, + queryContext.getQueryTimeLowerBound(), + measurementToChunkMetadataList, + measurementToMemChunkHandleList, + modsToMemtable); + } + if (workMemTable != null) { + workMemTable.queryForDeviceRegionScan( + devicePath, + isAligned, + queryContext.getQueryTimeLowerBound(), + measurementToChunkMetadataList, + measurementToMemChunkHandleList, + null); + } + + buildChunkHandleForFlushedMemTable( + isAligned + ? getAlignedVisibleMetadataListFromWriterByDeviceID(queryContext, devicePath) + : getVisibleMetadataListFromWriterByDeviceID(queryContext, devicePath), + measurementToChunkMetadataList, + measurementToMemChunkHandleList); + + if (!measurementToMemChunkHandleList.isEmpty() + || !measurementToChunkMetadataList.isEmpty()) { + deviceToMemChunkHandleMap.put(devicePath, measurementToMemChunkHandleList); + deviceToChunkMetadataListMap.put(devicePath, measurementToChunkMetadataList); + } + } + } catch (QueryProcessException | MetadataException | IOException e) { + logger.error( + "{}: {} get ReadOnlyMemChunk has error", + storageGroupName, + tsFileResource.getTsFile().getName(), + e); + } finally { + QUERY_RESOURCE_METRICS.recordQueryResourceNum(FLUSHING_MEMTABLE, flushingMemTables.size()); + QUERY_RESOURCE_METRICS.recordQueryResourceNum( + WORKING_MEMTABLE, workMemTable != null ? 1 : 0); + + flushQueryLock.readLock().unlock(); + if (logger.isDebugEnabled()) { + logger.debug( + "{}: {} release flushQueryLock", + storageGroupName, + tsFileResource.getTsFile().getName()); + } + } + + if (!deviceToMemChunkHandleMap.isEmpty() || !deviceToChunkMetadataListMap.isEmpty()) { + fileScanHandlesForQuery.add( + new UnclosedFileScanHandleImpl( + deviceToChunkMetadataListMap, deviceToMemChunkHandleMap, tsFileResource)); + } + } finally { + QUERY_EXECUTION_METRICS.recordExecutionCost( + GET_QUERY_RESOURCE_FROM_MEM, System.nanoTime() - startTime); + } + } + /** * Get the chunk(s) in the memtable (one from work memtable and the other ones in flushing * memtables and then compact them into one TimeValuePairSorter). Then get the related diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/IQueryDataSource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/IQueryDataSource.java new file mode 100644 index 000000000000..06ca60551751 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/IQueryDataSource.java @@ -0,0 +1,27 @@ +/* + * 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.db.storageengine.dataregion.read; + +public interface IQueryDataSource { + + IQueryDataSource clone(); + + long getDataTTL(); +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/QueryDataSource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/QueryDataSource.java index 01e0b76345b0..afe89c14c26d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/QueryDataSource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/QueryDataSource.java @@ -33,7 +33,7 @@ * The QueryDataSource contains all the seq and unseq TsFileResources for one timeseries in one * read. */ -public class QueryDataSource { +public class QueryDataSource implements IQueryDataSource { /** * TsFileResources used by read job. @@ -90,10 +90,19 @@ public List getUnseqResources() { return unseqResources; } + @Override public long getDataTTL() { return dataTTL; } + @Override + public IQueryDataSource clone() { + QueryDataSource queryDataSource = new QueryDataSource(getSeqResources(), getUnseqResources()); + queryDataSource.setSingleDevice(isSingleDevice()); + queryDataSource.setDataTTL(getDataTTL()); + return queryDataSource; + } + public void setDataTTL(long dataTTL) { this.dataTTL = dataTTL; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/QueryDataSourceForRegionScan.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/QueryDataSourceForRegionScan.java new file mode 100644 index 000000000000..257e294276cb --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/QueryDataSourceForRegionScan.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.db.storageengine.dataregion.read; + +import org.apache.iotdb.db.storageengine.dataregion.read.filescan.IFileScanHandle; + +import java.util.List; + +public class QueryDataSourceForRegionScan implements IQueryDataSource { + + private final List seqFileScanHandle; + + private final List unseqFileScanHandles; + + private long dataTTL = Long.MAX_VALUE; + + public QueryDataSourceForRegionScan( + List seqFileScanHandle, List unseqFileScanHandles) { + this.seqFileScanHandle = seqFileScanHandle; + this.unseqFileScanHandles = unseqFileScanHandles; + } + + public List getSeqFileScanHandles() { + return seqFileScanHandle; + } + + public List getUnseqFileScanHandles() { + return unseqFileScanHandles; + } + + @Override + public IQueryDataSource clone() { + QueryDataSourceForRegionScan queryDataSourceForRegionScan = + new QueryDataSourceForRegionScan(seqFileScanHandle, unseqFileScanHandles); + queryDataSourceForRegionScan.setDataTTL(getDataTTL()); + return queryDataSourceForRegionScan; + } + + @Override + public long getDataTTL() { + return dataTTL; + } + + public void setDataTTL(long dataTTL) { + this.dataTTL = dataTTL; + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/QueryDataSourceType.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/QueryDataSourceType.java new file mode 100644 index 000000000000..d80ce27ad7a1 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/QueryDataSourceType.java @@ -0,0 +1,25 @@ +/* + * 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.db.storageengine.dataregion.read; + +public enum QueryDataSourceType { + SERIES_SCAN, + DEVICE_REGION_SCAN, + TIME_SERIES_REGION_SCAN +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/IChunkHandle.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/IChunkHandle.java new file mode 100644 index 000000000000..b5ced3f3d374 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/IChunkHandle.java @@ -0,0 +1,49 @@ +/* + * 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.db.storageengine.dataregion.read.filescan; + +import java.io.IOException; + +/** This interface is used to handle the scan of chunks in TSFile. */ +public interface IChunkHandle { + + /** + * Check If there is more pages to be scanned in Chunk. If so, move to next page and return true + */ + boolean hasNextPage() throws IOException; + + /** Skip the current page */ + void skipCurrentPage(); + + /** + * Get the statistics time of page in Chunk. + * + * @return start time and end time of page. + */ + long[] getPageStatisticsTime(); + + /** + * Scan the data in the page and get the timestamp. It will cause disk IO if tsFile is not in + * memory. + * + * @return the iterator of timestamp. + */ + long[] getDataTime() throws IOException; +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/IFileScanHandle.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/IFileScanHandle.java new file mode 100644 index 000000000000..53c7fcb918d2 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/IFileScanHandle.java @@ -0,0 +1,99 @@ +/* + * 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.db.storageengine.dataregion.read.filescan; + +import org.apache.iotdb.commons.exception.IllegalPathException; +import org.apache.iotdb.db.storageengine.dataregion.read.filescan.model.AbstractChunkOffset; +import org.apache.iotdb.db.storageengine.dataregion.read.filescan.model.AbstractDeviceChunkMetaData; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; +import org.apache.iotdb.db.storageengine.dataregion.utils.TsFileDeviceStartEndTimeIterator; + +import org.apache.tsfile.file.metadata.IDeviceID; +import org.apache.tsfile.file.metadata.statistics.Statistics; + +import java.io.IOException; +import java.io.Serializable; +import java.util.Iterator; +import java.util.List; + +/** + * IFileScanHandle will supply interfaces for metadata checking and chunk scanning for specified one + * TsFile. + */ +public interface IFileScanHandle { + + /** + * Get timeIndex for devices in current TsFile. + * + * @return the iterator of DeviceStartEndTime, which includes devicePath and startEndTime of this + * devicePath. + */ + TsFileDeviceStartEndTimeIterator getDeviceStartEndTimeIterator() throws IOException; + + /** + * Check whether timestamp is deleted in specified device. + * + * @param deviceID the devicePath needs to be checked. + * @param timeArray time value needed to be checked, which should be ordered. + * @return A boolean array, which indicates whether the timestamp in timeArray is deleted. + */ + boolean[] isDeviceTimeDeleted(IDeviceID deviceID, long[] timeArray) throws IllegalPathException; + + /** + * Get all the chunkMetaData in current TsFile. ChunkMetaData will be organized in device level. + * + * @return the iterator of DeviceChunkMetaData, which includes the devicePath, measurementId and + * relating chunkMetaDataList. + */ + Iterator getAllDeviceChunkMetaData() throws IOException; + + /** + * Check whether timestamp in specified timeSeries is deleted. + * + * @param deviceID the devicePath needs to be checked. + * @param timeSeriesName the timeSeries needs to be checked. + * @param timeArray time value needed to be checked, which should be ordered. + * @return A boolean array, which indicates whether the timestamp in timeArray is deleted. + */ + boolean[] isTimeSeriesTimeDeleted(IDeviceID deviceID, String timeSeriesName, long[] timeArray) + throws IllegalPathException; + + /** + * Get the chunkHandles of chunks needed to be scanned. ChunkHandles are used to read chunk. + * + * @param chunkInfoList the list of ChunkOffset, which decides the chunk needed to be scanned. + * @param statisticsList the list of Statistics, which will be used when there is only one page in + * chunk. + * @return the iterator of IChunkHandle. + */ + Iterator getChunkHandles( + List chunkInfoList, + List> statisticsList) + throws IOException; + + /** If the TsFile of this handle is closed. */ + boolean isClosed(); + + /** If the TsFile of this handle is deleted. */ + boolean isDeleted(); + + /** Get TsFileResource of current TsFile. */ + TsFileResource getTsResource(); +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/impl/ClosedFileScanHandleImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/impl/ClosedFileScanHandleImpl.java new file mode 100644 index 000000000000..01efa19f2c70 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/impl/ClosedFileScanHandleImpl.java @@ -0,0 +1,211 @@ +/* + * 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.db.storageengine.dataregion.read.filescan.impl; + +import org.apache.iotdb.commons.exception.IllegalPathException; +import org.apache.iotdb.db.queryengine.execution.fragment.QueryContext; +import org.apache.iotdb.db.storageengine.dataregion.modification.Deletion; +import org.apache.iotdb.db.storageengine.dataregion.modification.Modification; +import org.apache.iotdb.db.storageengine.dataregion.read.control.FileReaderManager; +import org.apache.iotdb.db.storageengine.dataregion.read.filescan.IChunkHandle; +import org.apache.iotdb.db.storageengine.dataregion.read.filescan.IFileScanHandle; +import org.apache.iotdb.db.storageengine.dataregion.read.filescan.model.AbstractChunkOffset; +import org.apache.iotdb.db.storageengine.dataregion.read.filescan.model.AbstractDeviceChunkMetaData; +import org.apache.iotdb.db.storageengine.dataregion.read.filescan.model.AlignedDeviceChunkMetaData; +import org.apache.iotdb.db.storageengine.dataregion.read.filescan.model.DeviceChunkMetaData; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.timeindex.DeviceTimeIndex; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.timeindex.ITimeIndex; +import org.apache.iotdb.db.storageengine.dataregion.utils.TsFileDeviceStartEndTimeIterator; +import org.apache.iotdb.db.utils.ModificationUtils; + +import org.apache.tsfile.file.metadata.AlignedChunkMetadata; +import org.apache.tsfile.file.metadata.IChunkMetadata; +import org.apache.tsfile.file.metadata.IDeviceID; +import org.apache.tsfile.file.metadata.statistics.Statistics; +import org.apache.tsfile.read.TsFileDeviceIterator; +import org.apache.tsfile.read.TsFileSequenceReader; +import org.apache.tsfile.read.common.TimeRange; +import org.apache.tsfile.utils.Pair; + +import java.io.IOException; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +public class ClosedFileScanHandleImpl implements IFileScanHandle { + + private final TsFileResource tsFileResource; + private final QueryContext queryContext; + // Used to cache the modifications of each timeseries + private final Map>> deviceToModifications; + + public ClosedFileScanHandleImpl(TsFileResource tsFileResource, QueryContext context) { + this.tsFileResource = tsFileResource; + this.queryContext = context; + this.deviceToModifications = new HashMap<>(); + } + + @Override + public TsFileDeviceStartEndTimeIterator getDeviceStartEndTimeIterator() throws IOException { + ITimeIndex timeIndex = tsFileResource.getTimeIndex(); + return timeIndex instanceof DeviceTimeIndex + ? new TsFileDeviceStartEndTimeIterator((DeviceTimeIndex) timeIndex) + : new TsFileDeviceStartEndTimeIterator(tsFileResource.buildDeviceTimeIndex()); + } + + @Override + public boolean[] isDeviceTimeDeleted(IDeviceID deviceID, long[] timeArray) + throws IllegalPathException { + boolean[] result = new boolean[2]; + List modifications = queryContext.getPathModifications(tsFileResource, deviceID); + List timeRangeList = + modifications.stream() + .filter(Deletion.class::isInstance) + .map(Deletion.class::cast) + .map(Deletion::getTimeRange) + .collect(Collectors.toList()); + + int[] deleteCursor = {0}; + for (int i = 0; i < timeArray.length; i++) { + result[i] = ModificationUtils.isPointDeleted(timeArray[i], timeRangeList, deleteCursor); + } + return result; + } + + private boolean[] calculateBooleanArray(List timeRangeList, long[] timeArray) { + boolean[] result = new boolean[timeArray.length]; + int[] deleteCursor = {0}; + for (int i = 0; i < timeArray.length; i++) { + result[i] = ModificationUtils.isPointDeleted(timeArray[i], timeRangeList, deleteCursor); + } + return result; + } + + @Override + public boolean[] isTimeSeriesTimeDeleted( + IDeviceID deviceID, String timeSeriesName, long[] timeArray) throws IllegalPathException { + + if (deviceToModifications.containsKey(deviceID) + && deviceToModifications.get(deviceID).containsKey(timeSeriesName)) { + return calculateBooleanArray( + deviceToModifications.get(deviceID).get(timeSeriesName), timeArray); + } + + List modifications = + queryContext.getPathModifications(tsFileResource, deviceID, timeSeriesName); + List timeRangeList = + modifications.stream() + .filter(Deletion.class::isInstance) + .map(Deletion.class::cast) + .map(Deletion::getTimeRange) + .collect(Collectors.toList()); + deviceToModifications + .computeIfAbsent(deviceID, k -> new HashMap<>()) + .put(timeSeriesName, timeRangeList); + return calculateBooleanArray(timeRangeList, timeArray); + } + + @Override + public Iterator getAllDeviceChunkMetaData() throws IOException { + + TsFileSequenceReader tsFileReader = FileReaderManager.getInstance().get(getFilePath(), true); + TsFileDeviceIterator deviceIterator = tsFileReader.getAllDevicesIteratorWithIsAligned(); + + List deviceChunkMetaDataList = new LinkedList<>(); + // Traverse each device in current tsFile and get all the relating chunkMetaData + while (deviceIterator.hasNext()) { + Pair deviceIDWithIsAligned = deviceIterator.next(); + Map, Pair>> metadataForDevice = + tsFileReader.getTimeseriesMetadataOffsetByDevice( + deviceIterator.getFirstMeasurementNodeOfCurrentDevice(), + Collections.emptySet(), + true); + if (!deviceIDWithIsAligned.right) { + // device is not aligned + deviceChunkMetaDataList.add( + new DeviceChunkMetaData( + deviceIDWithIsAligned.left, + metadataForDevice.values().stream() + .flatMap(pair -> pair.getLeft().stream()) + .collect(Collectors.toList()))); + } else { + // device is aligned + List timeChunkMetaData = metadataForDevice.get("").getLeft(); + List> valueMetaDataList = new ArrayList<>(); + for (Map.Entry, Pair>> pair : + metadataForDevice.entrySet()) { + // Skip timeChunkMetaData + if (pair.getKey().isEmpty()) { + continue; + } + valueMetaDataList.add(pair.getValue().getLeft()); + } + + List alignedDeviceChunkMetaData = new ArrayList<>(); + for (int i = 0; i < timeChunkMetaData.size(); i++) { + alignedDeviceChunkMetaData.add( + new AlignedChunkMetadata(timeChunkMetaData.get(i), valueMetaDataList.get(i))); + } + deviceChunkMetaDataList.add( + new AlignedDeviceChunkMetaData(deviceIDWithIsAligned.left, alignedDeviceChunkMetaData)); + } + } + return deviceChunkMetaDataList.iterator(); + } + + @Override + public Iterator getChunkHandles( + List chunkInfoList, + List> statisticsList) { + String filePath = tsFileResource.getTsFilePath(); + List chunkHandleList = new ArrayList<>(); + for (int i = 0; i < chunkInfoList.size(); i++) { + AbstractChunkOffset chunkOffset = chunkInfoList.get(i); + chunkHandleList.add(chunkOffset.generateChunkHandle(filePath, statisticsList.get(i))); + } + return chunkHandleList.iterator(); + } + + @Override + public boolean isClosed() { + return true; + } + + @Override + public boolean isDeleted() { + return tsFileResource.isDeleted(); + } + + public String getFilePath() { + return tsFileResource.getTsFilePath(); + } + + @Override + public TsFileResource getTsResource() { + return tsFileResource; + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/impl/DiskAlignedChunkHandleImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/impl/DiskAlignedChunkHandleImpl.java new file mode 100644 index 000000000000..50baf1b85e8a --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/impl/DiskAlignedChunkHandleImpl.java @@ -0,0 +1,81 @@ +/* + * 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.db.storageengine.dataregion.read.filescan.impl; + +import org.apache.iotdb.db.storageengine.dataregion.utils.SharedTimeDataBuffer; + +import org.apache.tsfile.file.metadata.statistics.Statistics; +import org.apache.tsfile.read.TsFileSequenceReader; +import org.apache.tsfile.read.reader.chunk.ChunkReader; +import org.apache.tsfile.utils.ReadWriteIOUtils; + +import java.io.IOException; +import java.io.Serializable; +import java.nio.ByteBuffer; + +public class DiskAlignedChunkHandleImpl extends DiskChunkHandleImpl { + private static final int MASK = 0x80; + + private final SharedTimeDataBuffer sharedTimeDataBuffer; + private int pageIndex = 0; + + public DiskAlignedChunkHandleImpl( + String filePath, + boolean isTsFileClosed, + long offset, + Statistics chunkStatistic, + SharedTimeDataBuffer sharedTimeDataBuffer) { + super(filePath, isTsFileClosed, offset, chunkStatistic); + this.sharedTimeDataBuffer = sharedTimeDataBuffer; + } + + @Override + protected void init(TsFileSequenceReader reader) throws IOException { + sharedTimeDataBuffer.init(reader); + super.init(reader); + } + + @Override + public long[] getDataTime() throws IOException { + ByteBuffer currentPageDataBuffer = + ChunkReader.deserializePageData( + this.currentPageHeader, this.currentChunkDataBuffer, this.currentChunkHeader); + int size = ReadWriteIOUtils.readInt(currentPageDataBuffer); + byte[] bitmap = new byte[(size + 7) / 8]; + currentPageDataBuffer.get(bitmap); + + long[] timeData = sharedTimeDataBuffer.getPageTime(pageIndex); + if (timeData.length != size) { + throw new UnsupportedOperationException("Time data size not match"); + } + + long[] validTimeList = new long[(int) currentPageHeader.getNumOfValues()]; + for (int i = 0; i < size; i++) { + if (((bitmap[i / 8] & 0xFF) & (MASK >>> (i % 8))) == 0) { + continue; + } + long timestamp = timeData[i]; + validTimeList[i] = timestamp; + } + + pageIndex++; + return validTimeList; + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/impl/DiskChunkHandleImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/impl/DiskChunkHandleImpl.java new file mode 100644 index 000000000000..429cef7ee5b7 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/impl/DiskChunkHandleImpl.java @@ -0,0 +1,137 @@ +/* + * 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.db.storageengine.dataregion.read.filescan.impl; + +import org.apache.iotdb.db.storageengine.dataregion.read.control.FileReaderManager; +import org.apache.iotdb.db.storageengine.dataregion.read.filescan.IChunkHandle; + +import org.apache.tsfile.common.conf.TSFileDescriptor; +import org.apache.tsfile.encoding.decoder.Decoder; +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.file.MetaMarker; +import org.apache.tsfile.file.header.ChunkHeader; +import org.apache.tsfile.file.header.PageHeader; +import org.apache.tsfile.file.metadata.enums.TSEncoding; +import org.apache.tsfile.file.metadata.statistics.Statistics; +import org.apache.tsfile.read.TsFileSequenceReader; +import org.apache.tsfile.read.common.Chunk; +import org.apache.tsfile.read.reader.chunk.ChunkReader; +import org.apache.tsfile.utils.ReadWriteForEncodingUtils; + +import java.io.IOException; +import java.io.Serializable; +import java.nio.ByteBuffer; + +/** It will receive a list of offset and execute sequential scan of TsFile for chunkData. */ +public class DiskChunkHandleImpl implements IChunkHandle { + private final boolean tsFileClosed; + private final String filePath; + protected ChunkHeader currentChunkHeader; + protected PageHeader currentPageHeader; + protected ByteBuffer currentChunkDataBuffer; + protected long offset; + + private final Decoder defaultTimeDecoder = + Decoder.getDecoderByType( + TSEncoding.valueOf(TSFileDescriptor.getInstance().getConfig().getTimeEncoder()), + TSDataType.INT64); + + // Page will reuse chunkStatistics if there is only one page in chunk + protected final Statistics chunkStatistic; + + public DiskChunkHandleImpl( + String filePath, + boolean isTsFileClosed, + long offset, + Statistics chunkStatistics) { + this.chunkStatistic = chunkStatistics; + this.offset = offset; + this.filePath = filePath; + this.tsFileClosed = isTsFileClosed; + } + + protected void init(TsFileSequenceReader reader) throws IOException { + if (currentChunkDataBuffer != null) { + return; + } + Chunk chunk = reader.readMemChunk(offset); + this.currentChunkDataBuffer = chunk.getData(); + this.currentChunkHeader = chunk.getHeader(); + } + + // Check if there is more pages to be scanned in Chunk. + // If so, deserialize the page header + @Override + public boolean hasNextPage() throws IOException { + // read chunk from disk if needed + if (currentChunkDataBuffer == null) { + TsFileSequenceReader reader = FileReaderManager.getInstance().get(filePath, tsFileClosed); + init(reader); + } + + if (!currentChunkDataBuffer.hasRemaining()) { + return false; + } + // If there is only one page, page statistics is not stored in the chunk header, which is the + // same as chunkStatistics + if ((byte) (this.currentChunkHeader.getChunkType() & 0x3F) + == MetaMarker.ONLY_ONE_PAGE_CHUNK_HEADER) { + currentPageHeader = + PageHeader.deserializeFrom(this.currentChunkDataBuffer, this.chunkStatistic); + } else { + currentPageHeader = + PageHeader.deserializeFrom( + this.currentChunkDataBuffer, this.currentChunkHeader.getDataType()); + } + return true; + } + + @Override + public void skipCurrentPage() { + currentChunkDataBuffer.position( + currentChunkDataBuffer.position() + currentPageHeader.getCompressedSize()); + } + + @Override + public long[] getPageStatisticsTime() { + return new long[] {currentPageHeader.getStartTime(), currentPageHeader.getEndTime()}; + } + + @Override + public long[] getDataTime() throws IOException { + ByteBuffer currentPageDataBuffer = + ChunkReader.deserializePageData( + currentPageHeader, this.currentChunkDataBuffer, this.currentChunkHeader); + int timeBufferLength = ReadWriteForEncodingUtils.readUnsignedVarInt(currentPageDataBuffer); + ByteBuffer timeBuffer = currentPageDataBuffer.slice(); + timeBuffer.limit(timeBufferLength); + + return convertToTimeArray(timeBuffer); + } + + private long[] convertToTimeArray(ByteBuffer timeBuffer) throws IOException { + long[] timeArray = new long[(int) currentPageHeader.getNumOfValues()]; + int index = 0; + while (defaultTimeDecoder.hasNext(timeBuffer)) { + timeArray[index++] = defaultTimeDecoder.readLong(timeBuffer); + } + return timeArray; + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/impl/MemAlignedChunkHandleImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/impl/MemAlignedChunkHandleImpl.java new file mode 100644 index 000000000000..868950c717ec --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/impl/MemAlignedChunkHandleImpl.java @@ -0,0 +1,72 @@ +/* + * 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.db.storageengine.dataregion.read.filescan.impl; + +import org.apache.iotdb.db.utils.ModificationUtils; + +import org.apache.tsfile.read.common.TimeRange; +import org.apache.tsfile.utils.BitMap; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.ARRAY_SIZE; + +public class MemAlignedChunkHandleImpl extends MemChunkHandleImpl { + + private final List bitMapOfValue; + private final List deletionList; + // start time and end time of the chunk according to bitMap + private final long[] startEndTime; + + public MemAlignedChunkHandleImpl( + long[] dataOfTimestamp, + List bitMapOfValue, + List deletionList, + long[] startEndTime) { + super(dataOfTimestamp); + this.bitMapOfValue = bitMapOfValue; + this.deletionList = deletionList; + this.startEndTime = startEndTime; + } + + @Override + public long[] getPageStatisticsTime() { + return startEndTime; + } + + @Override + public long[] getDataTime() throws IOException { + List timeList = new ArrayList<>(); + int[] deletionCursor = {0}; + for (int i = 0; i < dataOfTimestamp.length; i++) { + int arrayIndex = i / ARRAY_SIZE; + int elementIndex = i % ARRAY_SIZE; + if (!bitMapOfValue.get(arrayIndex).isMarked(elementIndex) + && !ModificationUtils.isPointDeleted(dataOfTimestamp[i], deletionList, deletionCursor) + && (i == dataOfTimestamp.length - 1 || dataOfTimestamp[i] != dataOfTimestamp[i + 1])) { + timeList.add(dataOfTimestamp[i]); + } + } + hasRead = true; + return timeList.stream().mapToLong(Long::longValue).toArray(); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/impl/MemChunkHandleImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/impl/MemChunkHandleImpl.java new file mode 100644 index 000000000000..e84dd071a40e --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/impl/MemChunkHandleImpl.java @@ -0,0 +1,56 @@ +/* + * 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.db.storageengine.dataregion.read.filescan.impl; + +import org.apache.iotdb.db.storageengine.dataregion.read.filescan.IChunkHandle; + +import java.io.IOException; + +public class MemChunkHandleImpl implements IChunkHandle { + protected final long[] dataOfTimestamp; + + protected boolean hasRead = false; + + public MemChunkHandleImpl(long[] dataOfTimestamp) { + this.dataOfTimestamp = dataOfTimestamp; + } + + @Override + public boolean hasNextPage() throws IOException { + return !hasRead; + } + + // MemChunk only has one page in handle + @Override + public void skipCurrentPage() { + hasRead = true; + } + + @Override + public long[] getPageStatisticsTime() { + return new long[] {dataOfTimestamp[0], dataOfTimestamp[dataOfTimestamp.length - 1]}; + } + + @Override + public long[] getDataTime() throws IOException { + hasRead = true; + return dataOfTimestamp; + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/impl/UnclosedFileScanHandleImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/impl/UnclosedFileScanHandleImpl.java new file mode 100644 index 000000000000..fef83dd0d7eb --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/impl/UnclosedFileScanHandleImpl.java @@ -0,0 +1,180 @@ +/* + * 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.db.storageengine.dataregion.read.filescan.impl; + +import org.apache.iotdb.db.storageengine.dataregion.read.filescan.IChunkHandle; +import org.apache.iotdb.db.storageengine.dataregion.read.filescan.IFileScanHandle; +import org.apache.iotdb.db.storageengine.dataregion.read.filescan.model.AbstractChunkOffset; +import org.apache.iotdb.db.storageengine.dataregion.read.filescan.model.AbstractDeviceChunkMetaData; +import org.apache.iotdb.db.storageengine.dataregion.read.filescan.model.AlignedDeviceChunkMetaData; +import org.apache.iotdb.db.storageengine.dataregion.read.filescan.model.ChunkOffset; +import org.apache.iotdb.db.storageengine.dataregion.read.filescan.model.DeviceChunkMetaData; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.timeindex.DeviceTimeIndex; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.timeindex.ITimeIndex; +import org.apache.iotdb.db.storageengine.dataregion.utils.TsFileDeviceStartEndTimeIterator; +import org.apache.iotdb.db.utils.ModificationUtils; + +import org.apache.tsfile.file.metadata.AlignedChunkMetadata; +import org.apache.tsfile.file.metadata.IChunkMetadata; +import org.apache.tsfile.file.metadata.IDeviceID; +import org.apache.tsfile.file.metadata.statistics.Statistics; + +import java.io.IOException; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +public class UnclosedFileScanHandleImpl implements IFileScanHandle { + + private final TsFileResource tsFileResource; + private final Map>> deviceToChunkMetadataMap; + private final Map>> deviceToMemChunkHandleMap; + + public UnclosedFileScanHandleImpl( + Map>> deviceToChunkMetadataMap, + Map>> deviceToMemChunkHandleMap, + TsFileResource tsFileResource) { + this.deviceToChunkMetadataMap = deviceToChunkMetadataMap; + this.deviceToMemChunkHandleMap = deviceToMemChunkHandleMap; + this.tsFileResource = tsFileResource; + } + + @Override + public TsFileDeviceStartEndTimeIterator getDeviceStartEndTimeIterator() throws IOException { + ITimeIndex timeIndex = tsFileResource.getTimeIndex(); + return timeIndex instanceof DeviceTimeIndex + ? new TsFileDeviceStartEndTimeIterator((DeviceTimeIndex) timeIndex) + : new TsFileDeviceStartEndTimeIterator(tsFileResource.buildDeviceTimeIndex()); + } + + @Override + public boolean[] isDeviceTimeDeleted(IDeviceID deviceID, long[] timeArray) { + Map> chunkMetadataMap = deviceToChunkMetadataMap.get(deviceID); + boolean[] result = new boolean[timeArray.length]; + + chunkMetadataMap.values().stream() + .flatMap(List::stream) + .map(IChunkMetadata::getDeleteIntervalList) + .filter(deleteIntervalList -> !deleteIntervalList.isEmpty()) + .forEach( + timeRangeList -> { + int[] deleteCursor = {0}; + for (int i = 0; i < timeArray.length; i++) { + if (!result[i] + && ModificationUtils.isPointDeleted( + timeArray[i], timeRangeList, deleteCursor)) { + result[i] = true; + } + } + }); + return result; + } + + @Override + public Iterator getAllDeviceChunkMetaData() throws IOException { + List deviceChunkMetaDataList = new ArrayList<>(); + for (Map.Entry>> entry : + deviceToChunkMetadataMap.entrySet()) { + IDeviceID deviceID = entry.getKey(); + Map> chunkMetadataList = entry.getValue(); + if (chunkMetadataList.isEmpty()) { + continue; + } + + boolean isAligned = chunkMetadataList.containsKey(""); + if (isAligned) { + List alignedChunkMetadataList = new ArrayList<>(); + List timeChunkMetadataList = chunkMetadataList.get(""); + List> valueChunkMetadataList = + new ArrayList<>(chunkMetadataList.values()); + for (int i = 0; i < timeChunkMetadataList.size(); i++) { + alignedChunkMetadataList.add( + new AlignedChunkMetadata( + timeChunkMetadataList.get(i), valueChunkMetadataList.get(i))); + } + deviceChunkMetaDataList.add( + new AlignedDeviceChunkMetaData(deviceID, alignedChunkMetadataList)); + } else { + for (Map.Entry> measurementMetaData : + chunkMetadataList.entrySet()) { + deviceChunkMetaDataList.add( + new DeviceChunkMetaData(deviceID, measurementMetaData.getValue())); + } + } + } + return deviceChunkMetaDataList.iterator(); + } + + @Override + public boolean[] isTimeSeriesTimeDeleted( + IDeviceID deviceID, String timeSeriesName, long[] timeArray) { + List chunkMetadataList = + deviceToChunkMetadataMap.get(deviceID).get(timeSeriesName); + boolean[] result = new boolean[timeArray.length]; + chunkMetadataList.stream() + .map(IChunkMetadata::getDeleteIntervalList) + .filter(deleteIntervalList -> !deleteIntervalList.isEmpty()) + .forEach( + timeRangeList -> { + int[] deleteCursor = {0}; + for (int i = 0; i < timeArray.length; i++) { + if (!result[i] + && ModificationUtils.isPointDeleted( + timeArray[i], timeRangeList, deleteCursor)) { + result[i] = true; + } + } + }); + return result; + } + + @Override + public Iterator getChunkHandles( + List chunkInfoList, + List> statisticsList) { + List chunkHandleList = new ArrayList<>(); + for (AbstractChunkOffset chunkOffsetInfo : chunkInfoList) { + List chunkHandle = + deviceToMemChunkHandleMap + .get(chunkOffsetInfo.getDevicePath()) + .get(((ChunkOffset) chunkOffsetInfo).getMeasurement()); + chunkHandleList.addAll(chunkHandle); + } + return chunkHandleList.iterator(); + } + + @Override + public boolean isClosed() { + return false; + } + + @Override + public boolean isDeleted() { + return tsFileResource.isDeleted(); + } + + @Override + public TsFileResource getTsResource() { + return tsFileResource; + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/model/AbstractChunkOffset.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/model/AbstractChunkOffset.java new file mode 100644 index 000000000000..93c804c7322e --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/model/AbstractChunkOffset.java @@ -0,0 +1,49 @@ +/* + * 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.db.storageengine.dataregion.read.filescan.model; + +import org.apache.iotdb.db.storageengine.dataregion.read.filescan.IChunkHandle; + +import org.apache.tsfile.file.metadata.IDeviceID; +import org.apache.tsfile.file.metadata.statistics.Statistics; + +import java.io.Serializable; + +public abstract class AbstractChunkOffset { + + private final IDeviceID devicePath; + private final long offSet; + + protected AbstractChunkOffset(long offSet, IDeviceID devicePath) { + this.offSet = offSet; + this.devicePath = devicePath; + } + + public abstract IChunkHandle generateChunkHandle( + String filePath, Statistics statistics); + + public IDeviceID getDevicePath() { + return devicePath; + } + + public long getOffSet() { + return offSet; + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/model/AbstractDeviceChunkMetaData.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/model/AbstractDeviceChunkMetaData.java new file mode 100644 index 000000000000..cac85810634c --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/model/AbstractDeviceChunkMetaData.java @@ -0,0 +1,36 @@ +/* + * 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.db.storageengine.dataregion.read.filescan.model; + +import org.apache.tsfile.file.metadata.IDeviceID; + +public abstract class AbstractDeviceChunkMetaData { + private final IDeviceID devicePath; + + public AbstractDeviceChunkMetaData(IDeviceID devicePath) { + this.devicePath = devicePath; + } + + public IDeviceID getDevicePath() { + return devicePath; + } + + public abstract boolean isAligned(); +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/model/AlignedChunkOffset.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/model/AlignedChunkOffset.java new file mode 100644 index 000000000000..a145d3e47aaa --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/model/AlignedChunkOffset.java @@ -0,0 +1,52 @@ +/* + * 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.db.storageengine.dataregion.read.filescan.model; + +import org.apache.iotdb.db.storageengine.dataregion.read.filescan.IChunkHandle; +import org.apache.iotdb.db.storageengine.dataregion.read.filescan.impl.DiskAlignedChunkHandleImpl; +import org.apache.iotdb.db.storageengine.dataregion.utils.SharedTimeDataBuffer; + +import org.apache.tsfile.file.metadata.IDeviceID; +import org.apache.tsfile.file.metadata.statistics.Statistics; + +import java.io.Serializable; + +public class AlignedChunkOffset extends AbstractChunkOffset { + + // Used by aligned device to share the same time buffer + private final SharedTimeDataBuffer sharedTimeDataBuffer; + + public AlignedChunkOffset( + long offSet, IDeviceID devicePath, SharedTimeDataBuffer sharedTimeDataBuffer) { + super(offSet, devicePath); + this.sharedTimeDataBuffer = sharedTimeDataBuffer; + } + + public SharedTimeDataBuffer getSharedTimeDataBuffer() { + return sharedTimeDataBuffer; + } + + @Override + public IChunkHandle generateChunkHandle( + String filePath, Statistics statistics) { + return new DiskAlignedChunkHandleImpl( + filePath, true, getOffSet(), statistics, sharedTimeDataBuffer); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/model/AlignedDeviceChunkMetaData.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/model/AlignedDeviceChunkMetaData.java new file mode 100644 index 000000000000..39fd103d0ab9 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/model/AlignedDeviceChunkMetaData.java @@ -0,0 +1,44 @@ +/* + * 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.db.storageengine.dataregion.read.filescan.model; + +import org.apache.tsfile.file.metadata.AlignedChunkMetadata; +import org.apache.tsfile.file.metadata.IDeviceID; + +import java.util.List; + +public class AlignedDeviceChunkMetaData extends AbstractDeviceChunkMetaData { + + List alignedChunkMetadataList; + + public AlignedDeviceChunkMetaData( + IDeviceID devicePath, List alignedChunkMetadataList) { + super(devicePath); + this.alignedChunkMetadataList = alignedChunkMetadataList; + } + + public List getAlignedChunkMetadataList() { + return alignedChunkMetadataList; + } + + @Override + public boolean isAligned() { + return true; + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/model/ChunkOffset.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/model/ChunkOffset.java new file mode 100644 index 000000000000..7d2ec8a63e27 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/model/ChunkOffset.java @@ -0,0 +1,48 @@ +/* + * 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.db.storageengine.dataregion.read.filescan.model; + +import org.apache.iotdb.db.storageengine.dataregion.read.filescan.IChunkHandle; +import org.apache.iotdb.db.storageengine.dataregion.read.filescan.impl.DiskChunkHandleImpl; + +import org.apache.tsfile.file.metadata.IDeviceID; +import org.apache.tsfile.file.metadata.statistics.Statistics; + +import java.io.Serializable; + +public class ChunkOffset extends AbstractChunkOffset { + + private final String measurement; + + public ChunkOffset(long offset, IDeviceID deviceID, String measurement) { + super(offset, deviceID); + this.measurement = measurement; + } + + public String getMeasurement() { + return measurement; + } + + @Override + public IChunkHandle generateChunkHandle( + String filePath, Statistics statistics) { + return new DiskChunkHandleImpl(filePath, true, getOffSet(), statistics); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/model/DeviceChunkMetaData.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/model/DeviceChunkMetaData.java new file mode 100644 index 000000000000..373412603393 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/model/DeviceChunkMetaData.java @@ -0,0 +1,45 @@ +/* + * 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.db.storageengine.dataregion.read.filescan.model; + +import org.apache.tsfile.file.metadata.IChunkMetadata; +import org.apache.tsfile.file.metadata.IDeviceID; + +import java.util.List; + +public class DeviceChunkMetaData extends AbstractDeviceChunkMetaData { + + private final List measurementChunkMetadata; + + public DeviceChunkMetaData( + IDeviceID devicePath, List measurementChunkMetadataMap) { + super(devicePath); + this.measurementChunkMetadata = measurementChunkMetadataMap; + } + + public List getMeasurementChunkMetadataMap() { + return measurementChunkMetadata; + } + + @Override + public boolean isAligned() { + return false; + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/model/DeviceStartEndTime.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/model/DeviceStartEndTime.java new file mode 100644 index 000000000000..3c036baddb71 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/model/DeviceStartEndTime.java @@ -0,0 +1,46 @@ +/* + * 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.db.storageengine.dataregion.read.filescan.model; + +import org.apache.tsfile.file.metadata.IDeviceID; + +public class DeviceStartEndTime { + private final IDeviceID devicePath; + private final long startTime; + private final long endTime; + + public DeviceStartEndTime(IDeviceID devicePath, long startTime, long endTime) { + this.devicePath = devicePath; + this.startTime = startTime; + this.endTime = endTime; + } + + public IDeviceID getDevicePath() { + return devicePath; + } + + public long getEndTime() { + return endTime; + } + + public long getStartTime() { + return startTime; + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/SharedTimeDataBuffer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/SharedTimeDataBuffer.java new file mode 100644 index 000000000000..606037b529da --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/SharedTimeDataBuffer.java @@ -0,0 +1,91 @@ +/* + * 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.db.storageengine.dataregion.utils; + +import org.apache.tsfile.common.conf.TSFileDescriptor; +import org.apache.tsfile.encoding.decoder.Decoder; +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.file.header.ChunkHeader; +import org.apache.tsfile.file.header.PageHeader; +import org.apache.tsfile.file.metadata.IChunkMetadata; +import org.apache.tsfile.file.metadata.enums.TSEncoding; +import org.apache.tsfile.read.TsFileSequenceReader; +import org.apache.tsfile.read.common.Chunk; +import org.apache.tsfile.read.reader.chunk.ChunkReader; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.List; + +public class SharedTimeDataBuffer { + private ByteBuffer timeBuffer; + private final IChunkMetadata timeChunkMetaData; + private ChunkHeader timeChunkHeader; + private final List timeData; + private final Decoder defaultTimeDecoder = + Decoder.getDecoderByType( + TSEncoding.valueOf(TSFileDescriptor.getInstance().getConfig().getTimeEncoder()), + TSDataType.INT64); + + public SharedTimeDataBuffer(IChunkMetadata timeChunkMetaData) { + this.timeChunkMetaData = timeChunkMetaData; + this.timeData = new ArrayList<>(); + } + + // It should be called first before other methods in sharedTimeBuffer. + public void init(TsFileSequenceReader reader) throws IOException { + if (timeBuffer != null) { + return; + } + Chunk timeChunk = reader.readMemChunk(timeChunkMetaData.getOffsetOfChunkHeader()); + timeChunkHeader = timeChunk.getHeader(); + timeBuffer = timeChunk.getData(); + } + + public long[] getPageTime(int pageId) throws IOException { + int size = timeData.size(); + if (pageId < size) { + return timeData.get(pageId); + } else if (pageId == size) { + loadPageData(); + return timeData.get(pageId); + } else { + throw new UnsupportedOperationException( + "PageId in SharedTimeDataBuffer should be incremental."); + } + } + + private void loadPageData() throws IOException { + if (!timeBuffer.hasRemaining()) { + throw new UnsupportedOperationException("No more data in SharedTimeDataBuffer"); + } + PageHeader timePageHeader = + PageHeader.deserializeFrom(timeBuffer, timeChunkHeader.getDataType()); + ByteBuffer timePageData = + ChunkReader.deserializePageData(timePageHeader, timeBuffer, timeChunkHeader); + long[] pageData = new long[(int) timePageHeader.getNumOfValues()]; + int index = 0; + while (defaultTimeDecoder.hasNext(timePageData)) { + pageData[index] = defaultTimeDecoder.readLong(timePageData); + } + timeData.add(pageData); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/TsFileDeviceStartEndTimeIterator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/TsFileDeviceStartEndTimeIterator.java new file mode 100644 index 000000000000..c641ec72032b --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/TsFileDeviceStartEndTimeIterator.java @@ -0,0 +1,52 @@ +/* + * 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.db.storageengine.dataregion.utils; + +import org.apache.iotdb.db.storageengine.dataregion.read.filescan.model.DeviceStartEndTime; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.timeindex.DeviceTimeIndex; + +import org.apache.tsfile.file.metadata.IDeviceID; + +import java.util.Iterator; + +/** + * This class is used to iterate over the devicesTimeIndex in a TsFile to get the start and end + * times of each device. + */ +public class TsFileDeviceStartEndTimeIterator { + + private final DeviceTimeIndex deviceTimeIndex; + private final Iterator currentDevice; + + public TsFileDeviceStartEndTimeIterator(DeviceTimeIndex deviceTimeIndex) { + this.deviceTimeIndex = deviceTimeIndex; + this.currentDevice = deviceTimeIndex.getDevices().iterator(); + } + + public boolean hasNext() { + return currentDevice.hasNext(); + } + + public DeviceStartEndTime next() { + IDeviceID deviceID = currentDevice.next(); + return new DeviceStartEndTime( + deviceID, deviceTimeIndex.getStartTime(deviceID), deviceTimeIndex.getEndTime(deviceID)); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/ModificationUtils.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/ModificationUtils.java index 230d33dc7a7e..9e69c6d5cd34 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/ModificationUtils.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/ModificationUtils.java @@ -19,13 +19,19 @@ package org.apache.iotdb.db.utils; +import org.apache.iotdb.commons.path.AlignedPath; +import org.apache.iotdb.commons.path.MeasurementPath; +import org.apache.iotdb.commons.path.PartialPath; +import org.apache.iotdb.db.storageengine.dataregion.memtable.IMemTable; import org.apache.iotdb.db.storageengine.dataregion.modification.Deletion; import org.apache.iotdb.db.storageengine.dataregion.modification.Modification; import org.apache.tsfile.file.metadata.AlignedChunkMetadata; import org.apache.tsfile.file.metadata.IChunkMetadata; import org.apache.tsfile.read.common.TimeRange; +import org.apache.tsfile.utils.Pair; +import java.util.ArrayList; import java.util.List; public class ModificationUtils { @@ -140,10 +146,101 @@ public static void modifyAlignedChunkMetaData( }); } + // Check whether the timestamp is deleted in deletionList + // Timestamp and deletionList need to be ordered, and deleteCursor is array whose size is 1 stands + // for the index of the deletionList + public static boolean isPointDeleted( + long timestamp, List deletionList, int[] deleteCursor) { + if (deleteCursor.length != 1) { + throw new IllegalArgumentException("deleteCursor should be an array whose size is 1"); + } + while (deletionList != null && deleteCursor[0] < deletionList.size()) { + if (deletionList.get(deleteCursor[0]).contains(timestamp)) { + return true; + } else if (deletionList.get(deleteCursor[0]).getMax() < timestamp) { + deleteCursor[0]++; + } else { + return false; + } + } + return false; + } + + public static boolean isPointDeleted(long timestamp, List deletionList) { + int[] deleteCursor = {0}; + return isPointDeleted(timestamp, deletionList, deleteCursor); + } + private static void doModifyChunkMetaData(Modification modification, IChunkMetadata metaData) { if (modification instanceof Deletion) { Deletion deletion = (Deletion) modification; metaData.insertIntoSortedDeletions(deletion.getTimeRange()); } } + + /** Methods for modification in memory table */ + public static List> constructDeletionList( + AlignedPath partialPath, + IMemTable memTable, + List> modsToMemtable, + long timeLowerBound) { + List> deletionList = new ArrayList<>(); + for (String measurement : partialPath.getMeasurementList()) { + List columnDeletionList = new ArrayList<>(); + columnDeletionList.add(new TimeRange(Long.MIN_VALUE, timeLowerBound)); + for (Modification modification : + ModificationUtils.getModificationsForMemtable(memTable, modsToMemtable)) { + if (modification instanceof Deletion) { + Deletion deletion = (Deletion) modification; + PartialPath fullPath = partialPath.concatNode(measurement); + if (deletion.getPath().matchFullPath(fullPath) + && deletion.getEndTime() > timeLowerBound) { + long lowerBound = Math.max(deletion.getStartTime(), timeLowerBound); + columnDeletionList.add(new TimeRange(lowerBound, deletion.getEndTime())); + } + } + } + deletionList.add(TimeRange.sortAndMerge(columnDeletionList)); + } + return deletionList; + } + + /** + * construct a deletion list from a memtable. + * + * @param memTable memtable + * @param timeLowerBound time watermark + */ + public static List constructDeletionList( + MeasurementPath partialPath, + IMemTable memTable, + List> modsToMemtable, + long timeLowerBound) { + List deletionList = new ArrayList<>(); + deletionList.add(new TimeRange(Long.MIN_VALUE, timeLowerBound)); + for (Modification modification : getModificationsForMemtable(memTable, modsToMemtable)) { + if (modification instanceof Deletion) { + Deletion deletion = (Deletion) modification; + if (deletion.getPath().matchFullPath(partialPath) + && deletion.getEndTime() > timeLowerBound) { + long lowerBound = Math.max(deletion.getStartTime(), timeLowerBound); + deletionList.add(new TimeRange(lowerBound, deletion.getEndTime())); + } + } + } + return TimeRange.sortAndMerge(deletionList); + } + + private static List getModificationsForMemtable( + IMemTable memTable, List> modsToMemtable) { + List modifications = new ArrayList<>(); + boolean foundMemtable = false; + for (Pair entry : modsToMemtable) { + if (foundMemtable || entry.right.equals(memTable)) { + modifications.add(entry.left); + foundMemtable = true; + } + } + return modifications; + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java index fedd841470a8..59fd47091389 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java @@ -47,6 +47,7 @@ import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.ARRAY_SIZE; import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.TVLIST_SORT_ALGORITHM; import static org.apache.iotdb.db.utils.MemUtils.getBinarySize; +import static org.apache.iotdb.db.utils.ModificationUtils.isPointDeleted; import static org.apache.tsfile.utils.RamUsageEstimator.NUM_BYTES_ARRAY_HEADER; import static org.apache.tsfile.utils.RamUsageEstimator.NUM_BYTES_OBJECT_REF; @@ -939,7 +940,7 @@ public TsBlock buildTsBlock( // value columns for (int columnIndex = 0; columnIndex < dataTypes.size(); columnIndex++) { - int deleteCursor = 0; + int[] deleteCursor = {0}; // Pair of Time and Index Pair lastValidPointIndexForTimeDupCheck = null; if (Objects.nonNull(timeDuplicateInfo)) { @@ -1260,4 +1261,8 @@ public BitMap getRowBitMap() { return new BitMap(rowCount, rowBitsArr); } + + public List> getBitMaps() { + return bitMaps; + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BinaryTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BinaryTVList.java index 8f90296d774e..b11d7f14f54e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BinaryTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BinaryTVList.java @@ -40,6 +40,7 @@ import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.ARRAY_SIZE; import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.TVLIST_SORT_ALGORITHM; import static org.apache.iotdb.db.utils.MemUtils.getBinarySize; +import static org.apache.iotdb.db.utils.ModificationUtils.isPointDeleted; public abstract class BinaryTVList extends TVList { // list of primitive array, add 1 when expanded -> Binary primitive array @@ -185,7 +186,7 @@ protected void writeValidValuesIntoTsBlock( int floatPrecision, TSEncoding encoding, List deletionList) { - Integer deleteCursor = 0; + int[] deleteCursor = {0}; for (int i = 0; i < rowCount; i++) { if (!isPointDeleted(getTime(i), deletionList, deleteCursor) && (i == rowCount - 1 || getTime(i) != getTime(i + 1))) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BooleanTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BooleanTVList.java index 28e1b54dc8eb..00b0c3de8fd6 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BooleanTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BooleanTVList.java @@ -38,6 +38,7 @@ import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.ARRAY_SIZE; import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.TVLIST_SORT_ALGORITHM; +import static org.apache.iotdb.db.utils.ModificationUtils.isPointDeleted; public abstract class BooleanTVList extends TVList { // list of primitive array, add 1 when expanded -> Binary primitive array @@ -144,7 +145,7 @@ protected void writeValidValuesIntoTsBlock( int floatPrecision, TSEncoding encoding, List deletionList) { - Integer deleteCursor = 0; + int[] deleteCursor = {0}; for (int i = 0; i < rowCount; i++) { if (!isPointDeleted(getTime(i), deletionList, deleteCursor) && (i == rowCount - 1 || getTime(i) != getTime(i + 1))) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/DoubleTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/DoubleTVList.java index c72150db27e4..b7cc3336d10a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/DoubleTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/DoubleTVList.java @@ -38,6 +38,7 @@ import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.ARRAY_SIZE; import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.TVLIST_SORT_ALGORITHM; +import static org.apache.iotdb.db.utils.ModificationUtils.isPointDeleted; public abstract class DoubleTVList extends TVList { // list of primitive array, add 1 when expanded -> double primitive array @@ -147,7 +148,7 @@ protected void writeValidValuesIntoTsBlock( int floatPrecision, TSEncoding encoding, List deletionList) { - Integer deleteCursor = 0; + int[] deleteCursor = {0}; for (int i = 0; i < rowCount; i++) { if (!isPointDeleted(getTime(i), deletionList, deleteCursor) && (i == rowCount - 1 || getTime(i) != getTime(i + 1))) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/FloatTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/FloatTVList.java index 86e2b36d8816..43a208ecccfd 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/FloatTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/FloatTVList.java @@ -38,6 +38,7 @@ import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.ARRAY_SIZE; import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.TVLIST_SORT_ALGORITHM; +import static org.apache.iotdb.db.utils.ModificationUtils.isPointDeleted; public abstract class FloatTVList extends TVList { // list of primitive array, add 1 when expanded -> float primitive array @@ -147,7 +148,7 @@ protected void writeValidValuesIntoTsBlock( int floatPrecision, TSEncoding encoding, List deletionList) { - Integer deleteCursor = 0; + int[] deleteCursor = {0}; for (int i = 0; i < rowCount; i++) { if (!isPointDeleted(getTime(i), deletionList, deleteCursor) && (i == rowCount - 1 || getTime(i) != getTime(i + 1))) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/IntTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/IntTVList.java index 033e88a9e1cb..162f1fd1112b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/IntTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/IntTVList.java @@ -37,6 +37,7 @@ import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.ARRAY_SIZE; import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.TVLIST_SORT_ALGORITHM; +import static org.apache.iotdb.db.utils.ModificationUtils.isPointDeleted; public abstract class IntTVList extends TVList { // list of primitive array, add 1 when expanded -> int primitive array @@ -142,7 +143,7 @@ protected void writeValidValuesIntoTsBlock( int floatPrecision, TSEncoding encoding, List deletionList) { - Integer deleteCursor = 0; + int[] deleteCursor = {0}; for (int i = 0; i < rowCount; i++) { if (!isPointDeleted(getTime(i), deletionList, deleteCursor) && (i == rowCount - 1 || getTime(i) != getTime(i + 1))) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/LongTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/LongTVList.java index 4adaa0b978c5..4c8f4ccea9e7 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/LongTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/LongTVList.java @@ -37,6 +37,7 @@ import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.ARRAY_SIZE; import static org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager.TVLIST_SORT_ALGORITHM; +import static org.apache.iotdb.db.utils.ModificationUtils.isPointDeleted; public abstract class LongTVList extends TVList { // list of primitive array, add 1 when expanded -> long primitive array @@ -142,7 +143,7 @@ protected void writeValidValuesIntoTsBlock( int floatPrecision, TSEncoding encoding, List deletionList) { - Integer deleteCursor = 0; + int[] deleteCursor = {0}; for (int i = 0; i < rowCount; i++) { if (!isPointDeleted(getTime(i), deletionList, deleteCursor) && (i == rowCount - 1 || getTime(i) != getTime(i + 1))) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java index 90956eb12b6d..b179f681bafb 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java @@ -360,20 +360,6 @@ protected abstract void writeValidValuesIntoTsBlock( TSEncoding encoding, List deletionList); - protected boolean isPointDeleted( - long timestamp, List deletionList, Integer deleteCursor) { - while (deletionList != null && deleteCursor < deletionList.size()) { - if (deletionList.get(deleteCursor).contains(timestamp)) { - return true; - } else if (deletionList.get(deleteCursor).getMax() < timestamp) { - deleteCursor++; - } else { - return false; - } - } - return false; - } - protected float roundValueWithGivenPrecision( float value, int floatPrecision, TSEncoding encoding) { if (!Float.isNaN(value) && (encoding == TSEncoding.RLE || encoding == TSEncoding.TS_2DIFF)) { @@ -414,4 +400,8 @@ public static TVList deserialize(DataInputStream stream) throws IOException { } return null; } + + public List getTimestamps() { + return timestamps; + } } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/metadata/path/PatternTreeMapTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/metadata/path/PatternTreeMapTest.java index 92f41526fc2e..a3450f72332b 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/metadata/path/PatternTreeMapTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/metadata/path/PatternTreeMapTest.java @@ -55,6 +55,11 @@ public void stringAppendPatternTreeMapTest() throws IllegalPathException { patternTreeMap.append(new PartialPath("root.**"), "J"); patternTreeMap.append(new PartialPath("root.**.**"), "K"); + checkOverlappedByDevice( + patternTreeMap, + new PartialPath("root.sg1.d1"), + Arrays.asList("A", "B", "C", "D", "E", "F", "G", "H", "I", "J", "K")); + checkOverlapped( patternTreeMap, new PartialPath("root.sg1.d1.s1"), @@ -160,6 +165,47 @@ public void modificationPatternTreeMapTest() throws IllegalPathException { patternTreeMap, new PartialPath("root.sg1.d1.s2"), Collections.singletonList(new Deletion(new PartialPath("root.**"), 5, 10, 100))); + + patternTreeMap.append( + new PartialPath("root.sg1.d2.s1"), + new Deletion(new PartialPath("root.sg1.d2.s1"), 4, 4, 6)); + patternTreeMap.append( + new PartialPath("root.**.s2"), new Deletion(new PartialPath("root.**.s2"), 4, 4, 6)); + patternTreeMap.append( + new PartialPath("root.sg1.d1.s3"), + new Deletion(new PartialPath("root.sg1.d1.s3"), 4, 5, 6)); + patternTreeMap.append( + new PartialPath("root.sg1.d1.*"), new Deletion(new PartialPath("root.sg1.d1.*"), 8, 4, 6)); + patternTreeMap.append( + new PartialPath("root.sg1.d1.*.d3.s5"), + new Deletion(new PartialPath("root.sg1.d1.*.d3.s5"), 2, 4, 6)); + patternTreeMap.append( + new PartialPath("root.sg1.d1.*.d3.s4"), + new Deletion(new PartialPath("root.sg1.d1.*.d3.s4"), 3, 4, 6)); + + checkOverlappedByDevice( + patternTreeMap, + new PartialPath("root.sg1.d1"), + Arrays.asList( + new Deletion(new PartialPath("root.sg1.d1.s1"), 1, 1, 3), + new Deletion(new PartialPath("root.sg1.d1.s1"), 1, 6, 10), + new Deletion(new PartialPath("root.**.s1"), 5, 10, 100), + new Deletion(new PartialPath("root.**.s2"), 4, 4, 6), + new Deletion(new PartialPath("root.sg1.d1.s3"), 4, 5, 6), + new Deletion(new PartialPath("root.**.s1"), 10, 100, 200), + new Deletion(new PartialPath("root.sg1.d1.*"), 8, 4, 6), + new Deletion(new PartialPath("root.**"), 5, 10, 100))); + + checkOverlappedByDevice( + patternTreeMap, + new PartialPath("root.sg1.d1.t1.d3"), + Arrays.asList( + new Deletion(new PartialPath("root.**.s1"), 5, 10, 100), + new Deletion(new PartialPath("root.**.s2"), 4, 4, 6), + new Deletion(new PartialPath("root.**.s1"), 10, 100, 200), + new Deletion(new PartialPath("root.**"), 5, 10, 100), + new Deletion(new PartialPath("root.sg1.d1.*.d3.s5"), 2, 4, 6), + new Deletion(new PartialPath("root.sg1.d1.*.d3.s4"), 3, 4, 6))); } private void checkOverlapped( @@ -187,4 +233,13 @@ private void checkOverlappedByDeviceMeasurements( } } } + + private void checkOverlappedByDevice( + PatternTreeMap patternTreeMap, PartialPath devicePath, List expectedList) { + Set resultSet = new HashSet<>(patternTreeMap.getDeviceOverlapped(devicePath)); + Assert.assertEquals(expectedList.size(), resultSet.size()); + for (T o : expectedList) { + Assert.assertTrue(resultSet.contains(o)); + } + } } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/path/PatternTreeMap.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/path/PatternTreeMap.java index 130c605c1950..dc35cb5a4c4f 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/path/PatternTreeMap.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/path/PatternTreeMap.java @@ -205,4 +205,45 @@ private void searchOverlapped( searchOverlapped(child, deviceNodes, pos + 1, measurements, resultSet); } } + + /** + * Get a list of value lists related to PathPattern that overlapped with device. + * + *

Attention!: The results may contain imprecise and redundant values. Values that appear in + * the result set are not necessarily belong to current device, but those that do not appear are + * definitely not included. + * + * @param devicePath device path without wildcard + * @return de-duplicated value list + */ + public List getDeviceOverlapped(PartialPath devicePath) { + Set resultSet = new HashSet<>(); + searchDeviceOverlapped(root, devicePath.getNodes(), 0, resultSet); + return new ArrayList<>(resultSet); + } + + /** + * Recursive method for search overlapped pattern for devicePath. + * + * @param node current PathPatternNode + * @param deviceNodes pathNodes of device + * @param pos current index of deviceNodes + * @param resultSet result set + */ + private void searchDeviceOverlapped( + PathPatternNode node, String[] deviceNodes, int pos, Set resultSet) { + if (pos == deviceNodes.length - 1) { + resultSet.addAll(node.getValues()); + for (PathPatternNode child : node.getChildren().values()) { + resultSet.addAll(child.getValues()); + } + return; + } + if (node.isMultiLevelWildcard()) { + searchDeviceOverlapped(node, deviceNodes, pos + 1, resultSet); + } + for (PathPatternNode child : node.getMatchChildren(deviceNodes[pos + 1])) { + searchDeviceOverlapped(child, deviceNodes, pos + 1, resultSet); + } + } } diff --git a/pom.xml b/pom.xml index f3054f579aa4..4f5d319baf94 100644 --- a/pom.xml +++ b/pom.xml @@ -185,7 +185,7 @@ 1.9 0.11.1 1.5.5-5 - 1.0.1-6cbbba7-SNAPSHOT + 1.0.1-ee39f42-SNAPSHOT