From ea1b05f344ce330b0bf0523d5e908bb27a2e2c3b Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Mon, 8 Apr 2024 23:25:54 +0800 Subject: [PATCH 01/93] setup topic config --- .../rpc/subscription/config/TopicConfig.java | 44 +++++++++++++++++++ .../subscription/config/TopicConstant.java | 5 ++- .../executor/ClusterConfigTaskExecutor.java | 36 ++++++++++----- .../subscription/meta/topic/TopicMeta.java | 2 + 4 files changed, 74 insertions(+), 13 deletions(-) diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/config/TopicConfig.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/config/TopicConfig.java index 8e1fc70ab1c9..147b282794ca 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/config/TopicConfig.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/config/TopicConfig.java @@ -81,6 +81,15 @@ public Map getAttributesWithTimeRange(long creationTime) { return attributesWithTimeRange; } + public Map getAttributesWithSourceRealtimeMode() { + if (TopicConstant.FORMAT_TS_FILE_READER_VALUE.equals( + attributes.getOrDefault(TopicConstant.FORMAT_KEY, TopicConstant.FORMAT_DEFAULT_VALUE))) { + return Collections.singletonMap("realtime.mode", "batch"); + } + + return Collections.singletonMap("realtime.mode", "hybrid"); + } + public Map getAttributesWithProcessorPrefix() { Map attributesWithProcessorPrefix = new HashMap<>(); attributes.forEach( @@ -91,4 +100,39 @@ public Map getAttributesWithProcessorPrefix() { }); return attributesWithProcessorPrefix; } + + public boolean isValid() { + if (!TopicConstant.FORMAT_TS_FILE_READER_VALUE.equals( + attributes.getOrDefault(TopicConstant.FORMAT_KEY, TopicConstant.FORMAT_DEFAULT_VALUE))) { + return true; + } + + // check processor + if (!getAttributesWithProcessorPrefix().isEmpty()) { + return false; + } + + // check time range + final String startTime = + attributes.getOrDefault(TopicConstant.START_TIME_KEY, String.valueOf(Long.MIN_VALUE)); + if (!String.valueOf(Long.MIN_VALUE).equals(startTime)) { + return false; + } + final String endTime = + attributes.getOrDefault(TopicConstant.END_TIME_KEY, String.valueOf(Long.MAX_VALUE)); + if (!String.valueOf(Long.MAX_VALUE).equals(endTime)) { + return false; + } + + // check path or pattern + if (!TopicConstant.PATH_DEFAULT_VALUE.equals( + attributes.getOrDefault(TopicConstant.PATH_KEY, TopicConstant.PATH_DEFAULT_VALUE)) + || !TopicConstant.PATTERN_DEFAULT_VALUE.equals( + attributes.getOrDefault( + TopicConstant.PATTERN_KEY, TopicConstant.PATTERN_DEFAULT_VALUE))) { + return false; + } + + return true; + } } diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/config/TopicConstant.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/config/TopicConstant.java index 1bb8b7d92ebf..27f3d771117c 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/config/TopicConstant.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/config/TopicConstant.java @@ -24,13 +24,16 @@ public class TopicConstant { public static final String PATH_KEY = "path"; public static final String PATH_DEFAULT_VALUE = "root.**"; public static final String PATTERN_KEY = "pattern"; + public static final String PATTERN_DEFAULT_VALUE = "root"; public static final String START_TIME_KEY = "start-time"; public static final String END_TIME_KEY = "end-time"; public static final String NOW_TIME_VALUE = "now"; public static final String FORMAT_KEY = "format"; - public static final String FORMAT_DEFAULT_VALUE = "log"; + public static final String FORMAT_SESSION_DATA_SET_VALUE = "SessionDataSet"; + public static final String FORMAT_TS_FILE_READER_VALUE = "TsFileReader"; + public static final String FORMAT_DEFAULT_VALUE = FORMAT_SESSION_DATA_SET_VALUE; private TopicConstant() { throw new IllegalStateException("Utility class"); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java index e844a547093c..ad69124e1fc4 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java @@ -214,6 +214,7 @@ import org.apache.iotdb.rpc.RpcUtils; import org.apache.iotdb.rpc.StatementExecutionException; import org.apache.iotdb.rpc.TSStatusCode; +import org.apache.iotdb.rpc.subscription.config.TopicConfig; import org.apache.iotdb.service.rpc.thrift.TPipeTransferReq; import org.apache.iotdb.service.rpc.thrift.TPipeTransferResp; import org.apache.iotdb.trigger.api.Trigger; @@ -1671,7 +1672,7 @@ public SettableFuture createPipe(CreatePipeStatement createPip if (createPipeStatement.getPipeName().startsWith(PipeStaticMeta.SYSTEM_PIPE_PREFIX)) { String exceptionMessage = String.format( - "Failed to create pipe %s in config node, pipe name starting with \"%s\" are not allowed to be created", + "Failed to create pipe %s, pipe name starting with \"%s\" are not allowed to be created.", createPipeStatement.getPipeName(), PipeStaticMeta.SYSTEM_PIPE_PREFIX); LOGGER.warn(exceptionMessage); future.setException( @@ -1726,7 +1727,7 @@ public SettableFuture alterPipe(AlterPipeStatement alterPipeSt if (alterPipeStatement.getPipeName().startsWith(PipeStaticMeta.SYSTEM_PIPE_PREFIX)) { String exceptionMessage = String.format( - "Failed to alter pipe %s in config node, pipe name starting with \"%s\" are not allowed to be altered", + "Failed to alter pipe %s, pipe name starting with \"%s\" are not allowed to be altered.", alterPipeStatement.getPipeName(), PipeStaticMeta.SYSTEM_PIPE_PREFIX); LOGGER.warn(exceptionMessage); future.setException( @@ -1782,7 +1783,7 @@ public SettableFuture startPipe(StartPipeStatement startPipeSt if (startPipeStatement.getPipeName().startsWith(PipeStaticMeta.SYSTEM_PIPE_PREFIX)) { String exceptionMessage = String.format( - "Failed to start pipe %s in config node, pipe name starting with \"%s\" are not allowed to be started", + "Failed to start pipe %s, pipe name starting with \"%s\" are not allowed to be started.", startPipeStatement.getPipeName(), PipeStaticMeta.SYSTEM_PIPE_PREFIX); LOGGER.warn(exceptionMessage); future.setException( @@ -1814,7 +1815,7 @@ public SettableFuture dropPipe(DropPipeStatement dropPipeState if (dropPipeStatement.getPipeName().startsWith(PipeStaticMeta.SYSTEM_PIPE_PREFIX)) { String exceptionMessage = String.format( - "Failed to drop pipe %s in config node, pipe name starting with \"%s\" are not allowed to be dropped", + "Failed to drop pipe %s, pipe name starting with \"%s\" are not allowed to be dropped.", dropPipeStatement.getPipeName(), PipeStaticMeta.SYSTEM_PIPE_PREFIX); LOGGER.warn(exceptionMessage); future.setException( @@ -1846,7 +1847,7 @@ public SettableFuture stopPipe(StopPipeStatement stopPipeState if (stopPipeStatement.getPipeName().startsWith(PipeStaticMeta.SYSTEM_PIPE_PREFIX)) { String exceptionMessage = String.format( - "Failed to stop pipe %s in config node, pipe name starting with \"%s\" are not allowed to be stopped", + "Failed to stop pipe %s, pipe name starting with \"%s\" are not allowed to be stopped.", stopPipeStatement.getPipeName(), PipeStaticMeta.SYSTEM_PIPE_PREFIX); LOGGER.warn(exceptionMessage); future.setException( @@ -1928,18 +1929,29 @@ public SettableFuture showSubscriptions( @Override public SettableFuture createTopic(CreateTopicStatement createTopicStatement) { final SettableFuture future = SettableFuture.create(); + + final String topicName = createTopicStatement.getTopicName(); + final Map topicAttributes = createTopicStatement.getTopicAttributes(); + + // Validate topic config + final TopicConfig topicConfig = new TopicConfig(topicAttributes); + if (!topicConfig.isValid()) { + final String exceptionMessage = + String.format( + "Failed to create topic %s, topic config %s is invalid.", topicName, topicConfig); + LOGGER.warn(exceptionMessage); + future.setException( + new IoTDBException(exceptionMessage, TSStatusCode.PIPE_ERROR.getStatusCode())); + return future; + } + try (final ConfigNodeClient configNodeClient = CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) { final TCreateTopicReq req = - new TCreateTopicReq() - .setTopicName(createTopicStatement.getTopicName()) - .setTopicAttributes(createTopicStatement.getTopicAttributes()); + new TCreateTopicReq().setTopicName(topicName).setTopicAttributes(topicAttributes); final TSStatus tsStatus = configNodeClient.createTopic(req); if (TSStatusCode.SUCCESS_STATUS.getStatusCode() != tsStatus.getCode()) { - LOGGER.warn( - "Failed to create topic {} in config node, status is {}.", - createTopicStatement.getTopicName(), - tsStatus); + LOGGER.warn("Failed to create topic {} in config node, status is {}.", topicName, tsStatus); future.setException(new IoTDBException(tsStatus.message, tsStatus.code)); } else { future.set(new ConfigTaskResult(TSStatusCode.SUCCESS_STATUS)); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/subscription/meta/topic/TopicMeta.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/subscription/meta/topic/TopicMeta.java index 05cd0b535c50..212964ac2963 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/subscription/meta/topic/TopicMeta.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/subscription/meta/topic/TopicMeta.java @@ -179,6 +179,8 @@ public Map generateExtractorAttributes() { extractorAttributes.putAll(config.getAttributesWithSourcePathOrPattern()); // time extractorAttributes.putAll(config.getAttributesWithTimeRange(creationTime)); + // realtime mode + extractorAttributes.putAll(config.getAttributesWithSourceRealtimeMode()); return extractorAttributes; } From 2f333831eeacadfc63717460a6795c3e9600b060 Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Tue, 9 Apr 2024 20:03:28 +0800 Subject: [PATCH 02/93] improve SubscriptionSessionExample --- .../iotdb/SubscriptionSessionExample.java | 36 +++++++++---------- 1 file changed, 18 insertions(+), 18 deletions(-) diff --git a/example/session/src/main/java/org/apache/iotdb/SubscriptionSessionExample.java b/example/session/src/main/java/org/apache/iotdb/SubscriptionSessionExample.java index f31eac1df6c7..27a6035e6636 100644 --- a/example/session/src/main/java/org/apache/iotdb/SubscriptionSessionExample.java +++ b/example/session/src/main/java/org/apache/iotdb/SubscriptionSessionExample.java @@ -39,7 +39,7 @@ public class SubscriptionSessionExample { private static final String LOCAL_HOST = "127.0.0.1"; - public static void main(String[] args) throws Exception { + public static void main(final String[] args) throws Exception { session = new Session.Builder() .host(LOCAL_HOST) @@ -51,7 +51,7 @@ public static void main(String[] args) throws Exception { session.open(false); // Insert some historical data - long currentTime = System.currentTimeMillis(); + final long currentTime = System.currentTimeMillis(); for (int i = 0; i < 100; ++i) { session.executeNonQueryStatement( String.format("insert into root.db.d1(time, s1, s2) values (%s, 1, 2)", i)); @@ -63,28 +63,28 @@ public static void main(String[] args) throws Exception { session.executeNonQueryStatement("flush"); // Create topic - try (SubscriptionSession subscriptionSession = new SubscriptionSession(LOCAL_HOST, 6667)) { + try (final SubscriptionSession subscriptionSession = new SubscriptionSession(LOCAL_HOST, 6667)) { subscriptionSession.open(); subscriptionSession.createTopic("topic1"); subscriptionSession.createTopic("topic2"); } // Subscription: property-style ctor - Properties config = new Properties(); - config.put(ConsumerConstant.CONSUMER_ID_KEY, "c1"); - config.put(ConsumerConstant.CONSUMER_GROUP_ID_KEY, "cg1"); - SubscriptionPullConsumer consumer1 = new SubscriptionPullConsumer(config); + final Properties properties = new Properties(); + properties.put(ConsumerConstant.CONSUMER_ID_KEY, "c1"); + properties.put(ConsumerConstant.CONSUMER_GROUP_ID_KEY, "cg1"); + final SubscriptionPullConsumer consumer1 = new SubscriptionPullConsumer(properties); consumer1.open(); consumer1.subscribe("topic1"); while (true) { Thread.sleep(1000); // Wait for some time - List messages = consumer1.poll(Duration.ofMillis(10000)); + final List messages = consumer1.poll(Duration.ofMillis(10000)); if (messages.isEmpty()) { break; } - for (SubscriptionMessage message : messages) { - SubscriptionSessionDataSets payload = (SubscriptionSessionDataSets) message.getPayload(); - for (SubscriptionSessionDataSet dataSet : payload) { + for (final SubscriptionMessage message : messages) { + final SubscriptionSessionDataSets payload = (SubscriptionSessionDataSets) message.getPayload(); + for (final SubscriptionSessionDataSet dataSet : payload) { System.out.println(dataSet.getColumnNames()); System.out.println(dataSet.getColumnTypes()); while (dataSet.hasNext()) { @@ -96,7 +96,7 @@ public static void main(String[] args) throws Exception { } // Show topics and subscriptions - try (SubscriptionSession subscriptionSession = new SubscriptionSession(LOCAL_HOST, 6667)) { + try (final SubscriptionSession subscriptionSession = new SubscriptionSession(LOCAL_HOST, 6667)) { subscriptionSession.open(); subscriptionSession.getTopics().forEach((System.out::println)); subscriptionSession.getSubscriptions().forEach((System.out::println)); @@ -106,7 +106,7 @@ public static void main(String[] args) throws Exception { consumer1.close(); // Subscription: builder-style ctor - try (SubscriptionPullConsumer consumer2 = + try (final SubscriptionPullConsumer consumer2 = new SubscriptionPullConsumer.Builder() .consumerId("c2") .consumerGroupId("cg2") @@ -116,13 +116,13 @@ public static void main(String[] args) throws Exception { consumer2.subscribe("topic2"); while (true) { Thread.sleep(1000); // wait some time - List messages = consumer2.poll(Duration.ofMillis(10000)); + final List messages = consumer2.poll(Duration.ofMillis(10000)); if (messages.isEmpty()) { break; } - for (SubscriptionMessage message : messages) { - SubscriptionSessionDataSets payload = (SubscriptionSessionDataSets) message.getPayload(); - for (SubscriptionSessionDataSet dataSet : payload) { + for (final SubscriptionMessage message : messages) { + final SubscriptionSessionDataSets payload = (SubscriptionSessionDataSets) message.getPayload(); + for (final SubscriptionSessionDataSet dataSet : payload) { System.out.println(dataSet.getColumnNames()); System.out.println(dataSet.getColumnTypes()); while (dataSet.hasNext()) { @@ -136,7 +136,7 @@ public static void main(String[] args) throws Exception { } // Query - SessionDataSet dataSet = session.executeQueryStatement("select ** from root.**"); + final SessionDataSet dataSet = session.executeQueryStatement("select ** from root.**"); while (dataSet.hasNext()) { System.out.println(dataSet.next()); } From d3e9976f40543c9e74396febd6e7214bbca11282 Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Wed, 10 Apr 2024 00:28:44 +0800 Subject: [PATCH 03/93] intro req and resp for poll tsfile --- .../iotdb/SubscriptionSessionExample.java | 12 +- .../org/apache/iotdb/rpc/TSStatusCode.java | 1 + .../request/PipeSubscribePollTsFileReq.java | 115 +++++++++++++++ .../request/PipeSubscribeRequestType.java | 1 + .../response/PipeSubscribePollResp.java | 80 +++++++---- .../PipeSubscribePollTsFilePieceResp.java | 131 ++++++++++++++++++ .../PipeSubscribePollTsFileSealResp.java | 118 ++++++++++++++++ .../response/PipeSubscribeResponseType.java | 1 - .../receiver/SubscriptionReceiverV1.java | 26 +++- 9 files changed, 452 insertions(+), 33 deletions(-) create mode 100644 iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribePollTsFileReq.java create mode 100644 iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollTsFilePieceResp.java create mode 100644 iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollTsFileSealResp.java diff --git a/example/session/src/main/java/org/apache/iotdb/SubscriptionSessionExample.java b/example/session/src/main/java/org/apache/iotdb/SubscriptionSessionExample.java index 27a6035e6636..f50fb938c1f5 100644 --- a/example/session/src/main/java/org/apache/iotdb/SubscriptionSessionExample.java +++ b/example/session/src/main/java/org/apache/iotdb/SubscriptionSessionExample.java @@ -63,7 +63,8 @@ public static void main(final String[] args) throws Exception { session.executeNonQueryStatement("flush"); // Create topic - try (final SubscriptionSession subscriptionSession = new SubscriptionSession(LOCAL_HOST, 6667)) { + try (final SubscriptionSession subscriptionSession = + new SubscriptionSession(LOCAL_HOST, 6667)) { subscriptionSession.open(); subscriptionSession.createTopic("topic1"); subscriptionSession.createTopic("topic2"); @@ -83,7 +84,8 @@ public static void main(final String[] args) throws Exception { break; } for (final SubscriptionMessage message : messages) { - final SubscriptionSessionDataSets payload = (SubscriptionSessionDataSets) message.getPayload(); + final SubscriptionSessionDataSets payload = + (SubscriptionSessionDataSets) message.getPayload(); for (final SubscriptionSessionDataSet dataSet : payload) { System.out.println(dataSet.getColumnNames()); System.out.println(dataSet.getColumnTypes()); @@ -96,7 +98,8 @@ public static void main(final String[] args) throws Exception { } // Show topics and subscriptions - try (final SubscriptionSession subscriptionSession = new SubscriptionSession(LOCAL_HOST, 6667)) { + try (final SubscriptionSession subscriptionSession = + new SubscriptionSession(LOCAL_HOST, 6667)) { subscriptionSession.open(); subscriptionSession.getTopics().forEach((System.out::println)); subscriptionSession.getSubscriptions().forEach((System.out::println)); @@ -121,7 +124,8 @@ public static void main(final String[] args) throws Exception { break; } for (final SubscriptionMessage message : messages) { - final SubscriptionSessionDataSets payload = (SubscriptionSessionDataSets) message.getPayload(); + final SubscriptionSessionDataSets payload = + (SubscriptionSessionDataSets) message.getPayload(); for (final SubscriptionSessionDataSet dataSet : payload) { System.out.println(dataSet.getColumnNames()); System.out.println(dataSet.getColumnTypes()); diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java index e434852e870d..f70c07ab1c51 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java @@ -239,6 +239,7 @@ public enum TSStatusCode { SUBSCRIPTION_UNSUBSCRIBE_ERROR(1908), SUBSCRIPTION_MISSING_CUSTOMER(1909), SHOW_SUBSCRIPTION_ERROR(1910), + SUBSCRIPTION_POLL_TS_FILE_ERROR(1911), // Topic CREATE_TOPIC_ERROR(2000), diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribePollTsFileReq.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribePollTsFileReq.java new file mode 100644 index 000000000000..979a706ac0c7 --- /dev/null +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribePollTsFileReq.java @@ -0,0 +1,115 @@ +/* + * 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.rpc.subscription.payload.request; + +import org.apache.iotdb.service.rpc.thrift.TPipeSubscribeReq; +import org.apache.iotdb.tsfile.utils.PublicBAOS; +import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; + +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Objects; + +public class PipeSubscribePollTsFileReq extends TPipeSubscribeReq { + + public static final long ERROR_END_OFFSET = -1; + + private transient String topicName; + + private transient String fileName; + + private transient long endWritingOffset; + + public String getTopicName() { + return topicName; + } + + public String getFileName() { + return fileName; + } + + public long getEndWritingOffset() { + return endWritingOffset; + } + + /////////////////////////////// Thrift /////////////////////////////// + + public static PipeSubscribePollTsFileReq toTPipeSubscribeReq( + String topicName, String fileName, long endWritingOffset) throws IOException { + final PipeSubscribePollTsFileReq req = new PipeSubscribePollTsFileReq(); + + req.topicName = topicName; + req.fileName = fileName; + req.endWritingOffset = endWritingOffset; + + req.version = PipeSubscribeRequestVersion.VERSION_1.getVersion(); + req.type = PipeSubscribeRequestType.POLL_TS_FILE.getType(); + try (final PublicBAOS byteArrayOutputStream = new PublicBAOS(); + final DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream)) { + ReadWriteIOUtils.write(topicName, outputStream); + ReadWriteIOUtils.write(fileName, outputStream); + ReadWriteIOUtils.write(endWritingOffset, outputStream); + req.body = ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size()); + } + + return req; + } + + public static PipeSubscribePollTsFileReq fromTPipeSubscribeReq(TPipeSubscribeReq pollTsFileReq) { + final PipeSubscribePollTsFileReq req = new PipeSubscribePollTsFileReq(); + + if (Objects.nonNull(pollTsFileReq.body) && pollTsFileReq.body.hasRemaining()) { + req.topicName = ReadWriteIOUtils.readString(pollTsFileReq.body); + req.fileName = ReadWriteIOUtils.readString(pollTsFileReq.body); + req.endWritingOffset = ReadWriteIOUtils.readLong(pollTsFileReq.body); + } + + req.version = pollTsFileReq.version; + req.type = pollTsFileReq.type; + req.body = pollTsFileReq.body; + + return req; + } + + /////////////////////////////// Object /////////////////////////////// + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + PipeSubscribePollTsFileReq that = (PipeSubscribePollTsFileReq) obj; + return Objects.equals(this.topicName, that.topicName) + && Objects.equals(this.fileName, that.fileName) + && this.endWritingOffset == that.endWritingOffset + && this.version == that.version + && this.type == that.type + && Objects.equals(this.body, that.body); + } + + @Override + public int hashCode() { + return Objects.hash(topicName, fileName, endWritingOffset, version, type, body); + } +} diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribeRequestType.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribeRequestType.java index f9827e8f7067..96ea90d6d8fc 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribeRequestType.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribeRequestType.java @@ -31,6 +31,7 @@ public enum PipeSubscribeRequestType { CLOSE((short) 4), SUBSCRIBE((short) 5), UNSUBSCRIBE((short) 6), + POLL_TS_FILE((short) 7), ; private final short type; diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollResp.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollResp.java index 0d1ed4eb0e2f..074571dbea5a 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollResp.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollResp.java @@ -26,12 +26,15 @@ import org.apache.iotdb.service.rpc.thrift.TPipeSubscribeResp; import org.apache.iotdb.tsfile.utils.Pair; import org.apache.iotdb.tsfile.utils.PublicBAOS; +import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; import java.io.DataOutputStream; import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.stream.Collectors; @@ -39,10 +42,16 @@ public class PipeSubscribePollResp extends TPipeSubscribeResp { private transient List enrichedTabletsList = new ArrayList<>(); + private transient Map topicNameToTsFileNameMap = new HashMap<>(); + public List getEnrichedTabletsList() { return enrichedTabletsList; } + public Map getTopicNameToTsFileNameMap() { + return topicNameToTsFileNameMap; + } + /////////////////////////////// Thrift /////////////////////////////// /** @@ -50,17 +59,23 @@ public List getEnrichedTabletsList() { * server. */ public static PipeSubscribePollResp toTPipeSubscribeResp( - TSStatus status, List> enrichedTabletsWithByteBufferList) { + TSStatus status, + List> enrichedTabletsWithByteBufferList, + Map topicNameToTsFileNameMap) { final PipeSubscribePollResp resp = new PipeSubscribePollResp(); resp.enrichedTabletsList = enrichedTabletsWithByteBufferList.stream().map(Pair::getRight).collect(Collectors.toList()); + resp.topicNameToTsFileNameMap = topicNameToTsFileNameMap; resp.status = status; resp.version = PipeSubscribeResponseVersion.VERSION_1.getVersion(); - resp.type = PipeSubscribeResponseType.POLL_TABLETS.getType(); + resp.type = PipeSubscribeResponseType.ACK.getType(); try { - resp.body = serializeEnrichedTabletsWithByteBufferList(enrichedTabletsWithByteBufferList); + resp.body = new ArrayList<>(); + resp.body.add(serializeTopicNameToTsFileNameMap(topicNameToTsFileNameMap)); + resp.body.addAll( + serializeEnrichedTabletsWithByteBufferList(enrichedTabletsWithByteBufferList)); } catch (IOException e) { resp.status = RpcUtils.getStatus(TSStatusCode.SUBSCRIPTION_POLL_ERROR, e.getMessage()); } @@ -68,26 +83,20 @@ public static PipeSubscribePollResp toTPipeSubscribeResp( return resp; } - public static PipeSubscribePollResp directToTPipeSubscribeResp( - TSStatus status, List byteBuffers) { - final PipeSubscribePollResp resp = new PipeSubscribePollResp(); - - resp.status = status; - resp.version = PipeSubscribeResponseVersion.VERSION_1.getVersion(); - resp.type = PipeSubscribeResponseType.POLL_TABLETS.getType(); - resp.body = byteBuffers; - - return resp; - } - /** Deserialize `TPipeSubscribeResp` to obtain parameters, called by the subscription client. */ public static PipeSubscribePollResp fromTPipeSubscribeResp(TPipeSubscribeResp pollResp) { final PipeSubscribePollResp resp = new PipeSubscribePollResp(); + boolean isFirst = true; if (Objects.nonNull(pollResp.body)) { - for (ByteBuffer byteBuffer : pollResp.body) { + for (final ByteBuffer byteBuffer : pollResp.body) { if (Objects.nonNull(byteBuffer) && byteBuffer.hasRemaining()) { - resp.enrichedTabletsList.add(EnrichedTablets.deserialize(byteBuffer)); + if (isFirst) { + resp.topicNameToTsFileNameMap = deserializeTopicNameToTsFileNameMap(byteBuffer); + isFirst = false; + } else { + resp.enrichedTabletsList.add(EnrichedTablets.deserialize(byteBuffer)); + } } } } @@ -102,15 +111,6 @@ public static PipeSubscribePollResp fromTPipeSubscribeResp(TPipeSubscribeResp po /////////////////////////////// Utility /////////////////////////////// - public static List serializeEnrichedTabletsList( - List enrichedTabletsList) throws IOException { - List byteBufferList = new ArrayList<>(); - for (EnrichedTablets enrichedTablets : enrichedTabletsList) { - byteBufferList.add(serializeEnrichedTablets(enrichedTablets)); - } - return byteBufferList; - } - public static List serializeEnrichedTabletsWithByteBufferList( List> enrichedTabletsWithByteBufferList) throws IOException { @@ -135,6 +135,31 @@ public static ByteBuffer serializeEnrichedTablets(EnrichedTablets enrichedTablet } } + public static ByteBuffer serializeTopicNameToTsFileNameMap( + Map topicNameToTsFileNameMap) throws IOException { + try (final PublicBAOS byteArrayOutputStream = new PublicBAOS(); + final DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream)) { + ReadWriteIOUtils.write(topicNameToTsFileNameMap.size(), outputStream); + for (final Map.Entry topicNameToTsFileName : + topicNameToTsFileNameMap.entrySet()) { + ReadWriteIOUtils.write(topicNameToTsFileName.getKey(), outputStream); + ReadWriteIOUtils.write(topicNameToTsFileName.getValue(), outputStream); + } + return ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size()); + } + } + + public static Map deserializeTopicNameToTsFileNameMap(ByteBuffer buffer) { + final Map topicNameToTsFileNameMap = new HashMap<>(); + final int size = ReadWriteIOUtils.readInt(buffer); + for (int i = 0; i < size; i++) { + final String topicName = ReadWriteIOUtils.readString(buffer); + final String tsFileName = ReadWriteIOUtils.readString(buffer); + topicNameToTsFileNameMap.put(topicName, tsFileName); + } + return topicNameToTsFileNameMap; + } + /////////////////////////////// Object /////////////////////////////// @Override @@ -147,6 +172,7 @@ public boolean equals(Object obj) { } PipeSubscribePollResp that = (PipeSubscribePollResp) obj; return Objects.equals(this.enrichedTabletsList, that.enrichedTabletsList) + && Objects.equals(this.topicNameToTsFileNameMap, that.topicNameToTsFileNameMap) && Objects.equals(this.status, that.status) && this.version == that.version && this.type == that.type @@ -155,6 +181,6 @@ public boolean equals(Object obj) { @Override public int hashCode() { - return Objects.hash(enrichedTabletsList, status, version, type, body); + return Objects.hash(enrichedTabletsList, topicNameToTsFileNameMap, status, version, type, body); } } diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollTsFilePieceResp.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollTsFilePieceResp.java new file mode 100644 index 000000000000..a05ffb8a51b9 --- /dev/null +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollTsFilePieceResp.java @@ -0,0 +1,131 @@ +/* + * 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.rpc.subscription.payload.response; + +import org.apache.iotdb.common.rpc.thrift.TSStatus; +import org.apache.iotdb.rpc.RpcUtils; +import org.apache.iotdb.rpc.TSStatusCode; +import org.apache.iotdb.service.rpc.thrift.TPipeSubscribeResp; +import org.apache.iotdb.tsfile.utils.Binary; +import org.apache.iotdb.tsfile.utils.PublicBAOS; +import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; + +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Collections; +import java.util.Objects; + +public class PipeSubscribePollTsFilePieceResp extends TPipeSubscribeResp { + + private transient String fileName; + private transient long startWritingOffset; + private transient byte[] filePiece; + + public final String getFileName() { + return fileName; + } + + public final long getStartWritingOffset() { + return startWritingOffset; + } + + public final byte[] getFilePiece() { + return filePiece; + } + + /////////////////////////////// Thrift /////////////////////////////// + + public static PipeSubscribePollTsFilePieceResp toTPipeSubscribeResp( + TSStatus status, String fileName, long startWritingOffset, byte[] filePiece) { + final PipeSubscribePollTsFilePieceResp resp = new PipeSubscribePollTsFilePieceResp(); + + resp.fileName = fileName; + resp.startWritingOffset = startWritingOffset; + resp.filePiece = filePiece; + + resp.status = status; + resp.version = PipeSubscribeResponseVersion.VERSION_1.getVersion(); + resp.type = PipeSubscribeResponseType.ACK.getType(); + try (final PublicBAOS byteArrayOutputStream = new PublicBAOS(); + final DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream)) { + ReadWriteIOUtils.write(fileName, outputStream); + ReadWriteIOUtils.write(startWritingOffset, outputStream); + ReadWriteIOUtils.write(new Binary(filePiece), outputStream); + resp.body = + Collections.singletonList( + ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size())); + } catch (final IOException e) { + resp.status = + RpcUtils.getStatus(TSStatusCode.SUBSCRIPTION_POLL_TS_FILE_ERROR, e.getMessage()); + } + + return resp; + } + + public static PipeSubscribePollTsFilePieceResp fromTPipeSubscribeResp( + TPipeSubscribeResp pollTsFilePieceResp) { + final PipeSubscribePollTsFilePieceResp resp = new PipeSubscribePollTsFilePieceResp(); + + if (Objects.nonNull(pollTsFilePieceResp.body)) { + for (final ByteBuffer byteBuffer : pollTsFilePieceResp.body) { + if (Objects.nonNull(byteBuffer) && byteBuffer.hasRemaining()) { + resp.fileName = ReadWriteIOUtils.readString(byteBuffer); + resp.startWritingOffset = ReadWriteIOUtils.readLong(byteBuffer); + resp.filePiece = ReadWriteIOUtils.readBinary(byteBuffer).getValues(); + } + } + } + + resp.status = pollTsFilePieceResp.status; + resp.version = pollTsFilePieceResp.version; + resp.type = pollTsFilePieceResp.type; + resp.body = pollTsFilePieceResp.body; + + return resp; + } + + /////////////////////////////// Object /////////////////////////////// + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + final PipeSubscribePollTsFilePieceResp that = (PipeSubscribePollTsFilePieceResp) obj; + return Objects.equals(this.fileName, that.fileName) + && this.startWritingOffset == that.startWritingOffset + && Arrays.equals(this.filePiece, that.filePiece) + && Objects.equals(this.status, that.status) + && this.version == that.version + && this.type == that.type + && Objects.equals(this.body, that.body); + } + + @Override + public int hashCode() { + return Objects.hash( + fileName, startWritingOffset, Arrays.hashCode(filePiece), status, version, type, body); + } +} diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollTsFileSealResp.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollTsFileSealResp.java new file mode 100644 index 000000000000..5490ef98f693 --- /dev/null +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollTsFileSealResp.java @@ -0,0 +1,118 @@ +/* + * 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.rpc.subscription.payload.response; + +import org.apache.iotdb.common.rpc.thrift.TSStatus; +import org.apache.iotdb.rpc.RpcUtils; +import org.apache.iotdb.rpc.TSStatusCode; +import org.apache.iotdb.service.rpc.thrift.TPipeSubscribeResp; +import org.apache.iotdb.tsfile.utils.PublicBAOS; +import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; + +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Collections; +import java.util.Objects; + +public class PipeSubscribePollTsFileSealResp extends TPipeSubscribeResp { + + private transient String fileName; + private transient long fileLength; + + public final String getFileName() { + return fileName; + } + + public final long getFileLength() { + return fileLength; + } + + /////////////////////////////// Thrift /////////////////////////////// + + public static PipeSubscribePollTsFileSealResp toTPipeSubscribeResp( + TSStatus status, String fileName, long fileLength) { + final PipeSubscribePollTsFileSealResp resp = new PipeSubscribePollTsFileSealResp(); + + resp.fileName = fileName; + resp.fileLength = fileLength; + + resp.status = status; + resp.version = PipeSubscribeResponseVersion.VERSION_1.getVersion(); + resp.type = PipeSubscribeResponseType.ACK.getType(); + try (final PublicBAOS byteArrayOutputStream = new PublicBAOS(); + final DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream)) { + ReadWriteIOUtils.write(fileName, outputStream); + ReadWriteIOUtils.write(fileLength, outputStream); + resp.body = + Collections.singletonList( + ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size())); + } catch (final IOException e) { + resp.status = + RpcUtils.getStatus(TSStatusCode.SUBSCRIPTION_POLL_TS_FILE_ERROR, e.getMessage()); + } + + return resp; + } + + public static PipeSubscribePollTsFileSealResp fromTPipeSubscribeResp( + TPipeSubscribeResp pollTsFileSealResp) { + final PipeSubscribePollTsFileSealResp resp = new PipeSubscribePollTsFileSealResp(); + + if (Objects.nonNull(pollTsFileSealResp.body)) { + for (final ByteBuffer byteBuffer : pollTsFileSealResp.body) { + if (Objects.nonNull(byteBuffer) && byteBuffer.hasRemaining()) { + resp.fileName = ReadWriteIOUtils.readString(byteBuffer); + resp.fileLength = ReadWriteIOUtils.readLong(byteBuffer); + } + } + } + + resp.status = pollTsFileSealResp.status; + resp.version = pollTsFileSealResp.version; + resp.type = pollTsFileSealResp.type; + resp.body = pollTsFileSealResp.body; + + return resp; + } + + /////////////////////////////// Object /////////////////////////////// + + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + final PipeSubscribePollTsFileSealResp that = (PipeSubscribePollTsFileSealResp) obj; + return Objects.equals(this.fileName, that.fileName) + && fileLength == that.fileLength + && Objects.equals(this.status, that.status) + && this.version == that.version + && this.type == that.type + && Objects.equals(this.body, that.body); + } + + @Override + public int hashCode() { + return Objects.hash(fileName, fileLength, status, version, type, body); + } +} diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribeResponseType.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribeResponseType.java index aff0ed99e787..44acc999205c 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribeResponseType.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribeResponseType.java @@ -25,7 +25,6 @@ public enum PipeSubscribeResponseType { ACK((short) 0), - POLL_TABLETS((short) 1), ; private final short type; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java index d0e35211816a..9054f1c72ce0 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java @@ -45,6 +45,7 @@ import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribeHandshakeReq; import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribeHeartbeatReq; import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribePollReq; +import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribePollTsFileReq; import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribeRequestType; import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribeRequestVersion; import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribeSubscribeReq; @@ -67,6 +68,7 @@ import org.slf4j.LoggerFactory; import java.nio.ByteBuffer; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Objects; @@ -119,6 +121,9 @@ public final TPipeSubscribeResp handle(TPipeSubscribeReq req) { PipeSubscribeUnsubscribeReq.fromTPipeSubscribeReq(req)); case POLL: return handlePipeSubscribePoll(PipeSubscribePollReq.fromTPipeSubscribeReq(req)); + case POLL_TS_FILE: + return handlePipeSubscribePollTsFile( + PipeSubscribePollTsFileReq.fromTPipeSubscribeReq(req)); case COMMIT: return handlePipeSubscribeCommit(PipeSubscribeCommitReq.fromTPipeSubscribeReq(req)); case CLOSE: @@ -350,11 +355,30 @@ private TPipeSubscribeResp handlePipeSubscribePollInternal(PipeSubscribePollReq .collect(Collectors.toList()); TPipeSubscribeResp resp = PipeSubscribePollResp.toTPipeSubscribeResp( - RpcUtils.SUCCESS_STATUS, enrichedTabletsWithByteBufferList); + RpcUtils.SUCCESS_STATUS, enrichedTabletsWithByteBufferList, new HashMap<>()); events.forEach(SerializedEnrichedEvent::resetByteBuffer); return resp; } + private TPipeSubscribeResp handlePipeSubscribePollTsFile(PipeSubscribePollTsFileReq req) { + try { + return handlePipeSubscribePollTsFileInternal(req); + } catch (SubscriptionException e) { + final String exceptionMessage = + String.format( + "Subscription: something unexpected happened when polling tsfile: %s, req: %s", + e.getMessage(), req); + LOGGER.warn(exceptionMessage); + return PipeSubscribeHandshakeResp.toTPipeSubscribeResp( + RpcUtils.getStatus(TSStatusCode.SUBSCRIPTION_POLL_ERROR, exceptionMessage)); + } + } + + // TODO + private TPipeSubscribeResp handlePipeSubscribePollTsFileInternal(PipeSubscribePollTsFileReq req) { + return null; + } + private TPipeSubscribeResp handlePipeSubscribeCommit(PipeSubscribeCommitReq req) { try { return handlePipeSubscribeCommitInternal(req); From cf8f4bac3b40e3576945419006b3743de6888525 Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Wed, 10 Apr 2024 23:46:59 +0800 Subject: [PATCH 04/93] backup --- .../response/PipeSubscribePollResp.java | 2 +- .../PipeSubscribePollTsFilePieceResp.java | 2 +- .../PipeSubscribePollTsFileSealResp.java | 15 +++- .../response/PipeSubscribeResponseType.java | 4 + .../agent/SubscriptionBrokerAgent.java | 4 +- .../broker/SubscriptionBroker.java | 8 +- ...iptionPrefetchingEnrichedTabletsQueue.java | 12 +++ .../broker/SubscriptionPrefetchingQueue.java | 22 ++--- .../SubscriptionPrefetchingTsFileQueue.java | 31 +++++++ ...zableEnrichedTabletsSubscriptionEvent.java | 80 +++++++++++++++++++ .../SubscriptionEvent.java} | 70 +++------------- .../event/TsFileSubscriptionEvent.java | 12 +++ .../receiver/SubscriptionReceiverV1.java | 8 +- 13 files changed, 189 insertions(+), 81 deletions(-) create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingEnrichedTabletsQueue.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SerializableEnrichedTabletsSubscriptionEvent.java rename iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/{broker/SerializedEnrichedEvent.java => event/SubscriptionEvent.java} (58%) create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/TsFileSubscriptionEvent.java diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollResp.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollResp.java index 074571dbea5a..497f04fdcab0 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollResp.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollResp.java @@ -70,7 +70,7 @@ public static PipeSubscribePollResp toTPipeSubscribeResp( resp.status = status; resp.version = PipeSubscribeResponseVersion.VERSION_1.getVersion(); - resp.type = PipeSubscribeResponseType.ACK.getType(); + resp.type = PipeSubscribeResponseType.POLL.getType(); try { resp.body = new ArrayList<>(); resp.body.add(serializeTopicNameToTsFileNameMap(topicNameToTsFileNameMap)); diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollTsFilePieceResp.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollTsFilePieceResp.java index a05ffb8a51b9..11135a48c4ad 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollTsFilePieceResp.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollTsFilePieceResp.java @@ -64,7 +64,7 @@ public static PipeSubscribePollTsFilePieceResp toTPipeSubscribeResp( resp.status = status; resp.version = PipeSubscribeResponseVersion.VERSION_1.getVersion(); - resp.type = PipeSubscribeResponseType.ACK.getType(); + resp.type = PipeSubscribeResponseType.POLL_TS_FILE_PIECE.getType(); try (final PublicBAOS byteArrayOutputStream = new PublicBAOS(); final DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream)) { ReadWriteIOUtils.write(fileName, outputStream); diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollTsFileSealResp.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollTsFileSealResp.java index 5490ef98f693..c41a475c9623 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollTsFileSealResp.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollTsFileSealResp.java @@ -36,6 +36,7 @@ public class PipeSubscribePollTsFileSealResp extends TPipeSubscribeResp { private transient String fileName; private transient long fileLength; + private transient String subscriptionCommitId; public final String getFileName() { return fileName; @@ -45,22 +46,28 @@ public final long getFileLength() { return fileLength; } + public String getSubscriptionCommitId() { + return subscriptionCommitId; + } + /////////////////////////////// Thrift /////////////////////////////// public static PipeSubscribePollTsFileSealResp toTPipeSubscribeResp( - TSStatus status, String fileName, long fileLength) { + TSStatus status, String fileName, long fileLength, String subscriptionCommitId) { final PipeSubscribePollTsFileSealResp resp = new PipeSubscribePollTsFileSealResp(); resp.fileName = fileName; resp.fileLength = fileLength; + resp.subscriptionCommitId = subscriptionCommitId; resp.status = status; resp.version = PipeSubscribeResponseVersion.VERSION_1.getVersion(); - resp.type = PipeSubscribeResponseType.ACK.getType(); + resp.type = PipeSubscribeResponseType.POLL_TS_FILE_SEAL.getType(); try (final PublicBAOS byteArrayOutputStream = new PublicBAOS(); final DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream)) { ReadWriteIOUtils.write(fileName, outputStream); ReadWriteIOUtils.write(fileLength, outputStream); + ReadWriteIOUtils.write(subscriptionCommitId, outputStream); resp.body = Collections.singletonList( ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size())); @@ -81,6 +88,7 @@ public static PipeSubscribePollTsFileSealResp fromTPipeSubscribeResp( if (Objects.nonNull(byteBuffer) && byteBuffer.hasRemaining()) { resp.fileName = ReadWriteIOUtils.readString(byteBuffer); resp.fileLength = ReadWriteIOUtils.readLong(byteBuffer); + resp.subscriptionCommitId = ReadWriteIOUtils.readString(byteBuffer); } } } @@ -105,6 +113,7 @@ public boolean equals(Object obj) { final PipeSubscribePollTsFileSealResp that = (PipeSubscribePollTsFileSealResp) obj; return Objects.equals(this.fileName, that.fileName) && fileLength == that.fileLength + && Objects.equals(this.subscriptionCommitId, that.subscriptionCommitId) && Objects.equals(this.status, that.status) && this.version == that.version && this.type == that.type @@ -113,6 +122,6 @@ public boolean equals(Object obj) { @Override public int hashCode() { - return Objects.hash(fileName, fileLength, status, version, type, body); + return Objects.hash(fileName, fileLength, subscriptionCommitId, status, version, type, body); } } diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribeResponseType.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribeResponseType.java index 44acc999205c..55a93e35dfa3 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribeResponseType.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribeResponseType.java @@ -25,6 +25,10 @@ public enum PipeSubscribeResponseType { ACK((short) 0), + + POLL((short) 1), + POLL_TS_FILE_PIECE((short) 1), + POLL_TS_FILE_SEAL((short) 1), ; private final short type; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionBrokerAgent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionBrokerAgent.java index 7b75a8dd5126..09888a642c55 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionBrokerAgent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionBrokerAgent.java @@ -19,8 +19,8 @@ package org.apache.iotdb.db.subscription.agent; -import org.apache.iotdb.db.subscription.broker.SerializedEnrichedEvent; import org.apache.iotdb.db.subscription.broker.SubscriptionBroker; +import org.apache.iotdb.db.subscription.event.SerializableEnrichedTabletsSubscriptionEvent; import org.apache.iotdb.db.subscription.task.subtask.SubscriptionConnectorSubtask; import org.apache.iotdb.db.subscription.timer.SubscriptionPollTimer; import org.apache.iotdb.rpc.subscription.config.ConsumerConfig; @@ -44,7 +44,7 @@ public class SubscriptionBrokerAgent { //////////////////////////// provided for subscription agent //////////////////////////// - public List poll( + public List poll( ConsumerConfig consumerConfig, Set topicNames, SubscriptionPollTimer timer) { String consumerGroupId = consumerConfig.getConsumerGroupId(); SubscriptionBroker broker = consumerGroupIdToSubscriptionBroker.get(consumerGroupId); 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..5bedd567c202 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 @@ -20,6 +20,7 @@ package org.apache.iotdb.db.subscription.broker; import org.apache.iotdb.commons.pipe.task.connection.BoundedBlockingPendingQueue; +import org.apache.iotdb.db.subscription.event.SerializableEnrichedTabletsSubscriptionEvent; import org.apache.iotdb.db.subscription.timer.SubscriptionPollTimer; import org.apache.iotdb.pipe.api.event.Event; @@ -52,14 +53,15 @@ public boolean isEmpty() { //////////////////////////// provided for SubscriptionBrokerAgent //////////////////////////// - public List poll(Set topicNames, SubscriptionPollTimer timer) { - List events = new ArrayList<>(); + public List poll( + Set topicNames, SubscriptionPollTimer timer) { + List events = new ArrayList<>(); for (Map.Entry entry : topicNameToPrefetchingQueue.entrySet()) { String topicName = entry.getKey(); SubscriptionPrefetchingQueue prefetchingQueue = entry.getValue(); if (topicNames.contains(topicName)) { - SerializedEnrichedEvent event = prefetchingQueue.poll(timer); + SerializableEnrichedTabletsSubscriptionEvent event = prefetchingQueue.poll(timer); if (Objects.nonNull(event)) { events.add(event); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingEnrichedTabletsQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingEnrichedTabletsQueue.java new file mode 100644 index 000000000000..f5bafc3637ed --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingEnrichedTabletsQueue.java @@ -0,0 +1,12 @@ +package org.apache.iotdb.db.subscription.broker; + +import org.apache.iotdb.commons.pipe.task.connection.BoundedBlockingPendingQueue; +import org.apache.iotdb.pipe.api.event.Event; + +public class SubscriptionPrefetchingEnrichedTabletsQueue extends SubscriptionPrefetchingQueue { + + public SubscriptionPrefetchingEnrichedTabletsQueue( + String brokerId, String topicName, BoundedBlockingPendingQueue inputPendingQueue) { + super(brokerId, topicName, inputPendingQueue); + } +} 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 c156bea5a43b..eabaed668064 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 @@ -28,6 +28,8 @@ 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.subscription.event.SerializableEnrichedTabletsSubscriptionEvent; +import org.apache.iotdb.db.subscription.event.SubscriptionEvent; import org.apache.iotdb.db.subscription.timer.SubscriptionPollTimer; import org.apache.iotdb.pipe.api.event.Event; import org.apache.iotdb.pipe.api.event.dml.insertion.TabletInsertionEvent; @@ -46,16 +48,16 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; -public class SubscriptionPrefetchingQueue { +public abstract class SubscriptionPrefetchingQueue { - private static final Logger LOGGER = LoggerFactory.getLogger(SubscriptionBroker.class); + private static final Logger LOGGER = LoggerFactory.getLogger(SubscriptionPrefetchingQueue.class); private final String brokerId; // consumer group id private final String topicName; private final BoundedBlockingPendingQueue inputPendingQueue; - private final Map uncommittedEvents; - private final LinkedBlockingQueue prefetchingQueue; + private final Map uncommittedEvents; + private final LinkedBlockingQueue prefetchingQueue; private final AtomicLong subscriptionCommitIdGenerator = new AtomicLong(0); @@ -70,13 +72,13 @@ public SubscriptionPrefetchingQueue( /////////////////////////////// provided for SubscriptionBroker /////////////////////////////// - public SerializedEnrichedEvent poll(SubscriptionPollTimer timer) { + public SerializableEnrichedTabletsSubscriptionEvent poll(SubscriptionPollTimer timer) { if (prefetchingQueue.isEmpty()) { prefetchOnce(SubscriptionConfig.getInstance().getSubscriptionMaxTabletsPerPrefetching()); // without serializeOnce here } - SerializedEnrichedEvent currentEvent; + SerializableEnrichedTabletsSubscriptionEvent currentEvent; try { while (Objects.nonNull( currentEvent = @@ -109,7 +111,7 @@ public SerializedEnrichedEvent poll(SubscriptionPollTimer timer) { public void commit(List subscriptionCommitIds) { for (String subscriptionCommitId : subscriptionCommitIds) { - SerializedEnrichedEvent event = uncommittedEvents.get(subscriptionCommitId); + SubscriptionEvent event = uncommittedEvents.get(subscriptionCommitId); if (Objects.isNull(event)) { LOGGER.warn( "Subscription: subscription commit id [{}] does not exist, it may have been committed or something unexpected happened", @@ -177,8 +179,8 @@ private void prefetchOnce(long limit) { if (!tablets.isEmpty()) { String subscriptionCommitId = generateSubscriptionCommitId(); - SerializedEnrichedEvent enrichedEvent = - new SerializedEnrichedEvent( + SerializableEnrichedTabletsSubscriptionEvent enrichedEvent = + new SerializableEnrichedTabletsSubscriptionEvent( new EnrichedTablets(topicName, tablets, subscriptionCommitId), enrichedEvents); uncommittedEvents.put(subscriptionCommitId, enrichedEvent); // before enqueuing the event prefetchingQueue.add(enrichedEvent); @@ -189,7 +191,7 @@ private void serializeOnce() { long size = prefetchingQueue.size(); long count = 0; - SerializedEnrichedEvent currentEvent; + SerializableEnrichedTabletsSubscriptionEvent currentEvent; try { while (Objects.nonNull( currentEvent = diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java new file mode 100644 index 000000000000..92c1c94c65eb --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java @@ -0,0 +1,31 @@ +/* + * 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.subscription.broker; + +import org.apache.iotdb.commons.pipe.task.connection.BoundedBlockingPendingQueue; +import org.apache.iotdb.pipe.api.event.Event; + +public class SubscriptionPrefetchingTsFileQueue extends SubscriptionPrefetchingQueue { + + public SubscriptionPrefetchingTsFileQueue( + String brokerId, String topicName, BoundedBlockingPendingQueue inputPendingQueue) { + super(brokerId, topicName, inputPendingQueue); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SerializableEnrichedTabletsSubscriptionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SerializableEnrichedTabletsSubscriptionEvent.java new file mode 100644 index 000000000000..dbb4a4c158e7 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SerializableEnrichedTabletsSubscriptionEvent.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.db.subscription.event; + +import org.apache.iotdb.commons.pipe.event.EnrichedEvent; +import org.apache.iotdb.rpc.subscription.payload.EnrichedTablets; +import org.apache.iotdb.rpc.subscription.payload.response.PipeSubscribePollResp; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.List; +import java.util.Objects; + +public class SerializableEnrichedTabletsSubscriptionEvent extends SubscriptionEvent { + + private static final Logger LOGGER = + LoggerFactory.getLogger(SerializableEnrichedTabletsSubscriptionEvent.class); + + private final EnrichedTablets enrichedTablets; + + private ByteBuffer byteBuffer; // serialized EnrichedTablets + + public SerializableEnrichedTabletsSubscriptionEvent( + EnrichedTablets enrichedTablets, List enrichedEvents) { + super(enrichedEvents, enrichedTablets.getSubscriptionCommitId()); + this.enrichedTablets = enrichedTablets; + } + + //////////////////////////// serialization //////////////////////////// + + public EnrichedTablets getEnrichedTablets() { + return enrichedTablets; + } + + /** @return true -> byte buffer is not null */ + public boolean serialize() { + if (Objects.isNull(byteBuffer)) { + try { + byteBuffer = PipeSubscribePollResp.serializeEnrichedTablets(enrichedTablets); + return true; + } catch (IOException e) { + LOGGER.warn( + "Subscription: something unexpected happened when serializing EnrichedTablets {}, exception is {}", + byteBuffer, + e.getMessage()); + } + return false; + } + return true; + } + + public ByteBuffer getByteBuffer() { + return byteBuffer; + } + + public void resetByteBuffer() { + // maybe friendly for gc + byteBuffer = null; + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SerializedEnrichedEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionEvent.java similarity index 58% rename from iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SerializedEnrichedEvent.java rename to iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionEvent.java index 419579713fd4..5c04b64e65db 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SerializedEnrichedEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionEvent.java @@ -17,95 +17,51 @@ * under the License. */ -package org.apache.iotdb.db.subscription.broker; +package org.apache.iotdb.db.subscription.event; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.commons.subscription.config.SubscriptionConfig; -import org.apache.iotdb.rpc.subscription.payload.EnrichedTablets; -import org.apache.iotdb.rpc.subscription.payload.response.PipeSubscribePollResp; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.nio.ByteBuffer; import java.util.List; -import java.util.Objects; - -public class SerializedEnrichedEvent { - private static final Logger LOGGER = LoggerFactory.getLogger(SerializedEnrichedEvent.class); +public abstract class SubscriptionEvent { private static final long INVALID_TIMESTAMP = -1; - private final EnrichedTablets enrichedTablets; private final List enrichedEvents; + private final String subscriptionCommitId; private long lastPolledTimestamp; private long committedTimestamp; - private ByteBuffer byteBuffer; // serialized EnrichedTablets - - public SerializedEnrichedEvent( - EnrichedTablets enrichedTablets, List enrichedEvents) { - this.enrichedTablets = enrichedTablets; + public SubscriptionEvent(List enrichedEvents, String subscriptionCommitId) { this.enrichedEvents = enrichedEvents; + this.subscriptionCommitId = subscriptionCommitId; + this.lastPolledTimestamp = INVALID_TIMESTAMP; this.committedTimestamp = INVALID_TIMESTAMP; } - //////////////////////////// serialization //////////////////////////// - - public EnrichedTablets getEnrichedTablets() { - return enrichedTablets; - } - - /** @return true -> byte buffer is not null */ - public boolean serialize() { - if (Objects.isNull(byteBuffer)) { - try { - byteBuffer = PipeSubscribePollResp.serializeEnrichedTablets(enrichedTablets); - return true; - } catch (IOException e) { - LOGGER.warn( - "Subscription: something unexpected happened when serializing EnrichedTablets {}, exception is {}", - byteBuffer, - e.getMessage()); - } - return false; - } - return true; - } + //////////////////////////// commit //////////////////////////// - public ByteBuffer getByteBuffer() { - return byteBuffer; + public void recordCommittedTimestamp() { + committedTimestamp = System.currentTimeMillis(); } - public void resetByteBuffer() { - // maybe friendly for gc - byteBuffer = null; + public boolean isCommitted() { + return committedTimestamp != INVALID_TIMESTAMP; } - //////////////////////////// commit //////////////////////////// - public String getSubscriptionCommitId() { - return enrichedTablets.getSubscriptionCommitId(); + return subscriptionCommitId; } public void decreaseReferenceCount() { - for (EnrichedEvent enrichedEvent : enrichedEvents) { + for (final EnrichedEvent enrichedEvent : enrichedEvents) { enrichedEvent.decreaseReferenceCount(this.getClass().getName(), true); } } - public void recordCommittedTimestamp() { - committedTimestamp = System.currentTimeMillis(); - } - - public boolean isCommitted() { - return committedTimestamp != INVALID_TIMESTAMP; - } - //////////////////////////// pollable //////////////////////////// public void recordLastPolledTimestamp() { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/TsFileSubscriptionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/TsFileSubscriptionEvent.java new file mode 100644 index 000000000000..8de05d436f02 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/TsFileSubscriptionEvent.java @@ -0,0 +1,12 @@ +package org.apache.iotdb.db.subscription.event; + +import org.apache.iotdb.commons.pipe.event.EnrichedEvent; + +import java.util.List; + +public class TsFileSubscriptionEvent extends SubscriptionEvent { + + public TsFileSubscriptionEvent(List enrichedEvents, String subscriptionCommitId) { + super(enrichedEvents, subscriptionCommitId); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java index 96c9b8bb54ed..62eda351cd1f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java @@ -33,7 +33,7 @@ import org.apache.iotdb.db.protocol.client.ConfigNodeClientManager; import org.apache.iotdb.db.protocol.client.ConfigNodeInfo; import org.apache.iotdb.db.subscription.agent.SubscriptionAgent; -import org.apache.iotdb.db.subscription.broker.SerializedEnrichedEvent; +import org.apache.iotdb.db.subscription.event.SerializableEnrichedTabletsSubscriptionEvent; import org.apache.iotdb.db.subscription.timer.SubscriptionPollTimer; import org.apache.iotdb.rpc.RpcUtils; import org.apache.iotdb.rpc.TSStatusCode; @@ -330,11 +330,11 @@ private TPipeSubscribeResp handlePipeSubscribePollInternal(PipeSubscribePollReq : Math.max( req.getTimeoutMs(), SubscriptionConfig.getInstance().getSubscriptionMinPollTimeoutMs())); - List events = + List events = SubscriptionAgent.broker().poll(consumerConfig, topicNames, timer); List subscriptionCommitIds = events.stream() - .map(SerializedEnrichedEvent::getSubscriptionCommitId) + .map(SerializableEnrichedTabletsSubscriptionEvent::getSubscriptionCommitId) .collect(Collectors.toList()); if (timer.isExpired()) { @@ -356,7 +356,7 @@ private TPipeSubscribeResp handlePipeSubscribePollInternal(PipeSubscribePollReq TPipeSubscribeResp resp = PipeSubscribePollResp.toTPipeSubscribeResp( RpcUtils.SUCCESS_STATUS, enrichedTabletsWithByteBufferList, new HashMap<>()); - events.forEach(SerializedEnrichedEvent::resetByteBuffer); + events.forEach(SerializableEnrichedTabletsSubscriptionEvent::resetByteBuffer); return resp; } From 04d4b693f26716d83188f0bc9381d09abaecc3b1 Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Thu, 11 Apr 2024 01:25:59 +0800 Subject: [PATCH 05/93] backup --- .../request/PipeSubscribePollTsFileReq.java | 2 - .../agent/SubscriptionBrokerAgent.java | 47 ++--- .../agent/SubscriptionTopicAgent.java | 13 ++ .../broker/SubscriptionBroker.java | 43 ++-- ...iptionPrefetchingEnrichedTabletsQueue.java | 180 ++++++++++++++++- .../broker/SubscriptionPrefetchingQueue.java | 187 ++---------------- .../SubscriptionPrefetchingTsFileQueue.java | 61 +++++- ...zableEnrichedTabletsSubscriptionEvent.java | 4 +- .../subscription/event/SubscriptionEvent.java | 3 +- .../event/TsFileSubscriptionEvent.java | 28 ++- .../receiver/SubscriptionReceiverV1.java | 168 +++++++++------- 11 files changed, 452 insertions(+), 284 deletions(-) diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribePollTsFileReq.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribePollTsFileReq.java index 979a706ac0c7..eb9be53d1600 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribePollTsFileReq.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribePollTsFileReq.java @@ -30,8 +30,6 @@ public class PipeSubscribePollTsFileReq extends TPipeSubscribeReq { - public static final long ERROR_END_OFFSET = -1; - private transient String topicName; private transient String fileName; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionBrokerAgent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionBrokerAgent.java index 09888a642c55..4e15ece6dc72 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionBrokerAgent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionBrokerAgent.java @@ -20,7 +20,7 @@ package org.apache.iotdb.db.subscription.agent; import org.apache.iotdb.db.subscription.broker.SubscriptionBroker; -import org.apache.iotdb.db.subscription.event.SerializableEnrichedTabletsSubscriptionEvent; +import org.apache.iotdb.db.subscription.event.SubscriptionEvent; import org.apache.iotdb.db.subscription.task.subtask.SubscriptionConnectorSubtask; import org.apache.iotdb.db.subscription.timer.SubscriptionPollTimer; import org.apache.iotdb.rpc.subscription.config.ConsumerConfig; @@ -44,10 +44,12 @@ public class SubscriptionBrokerAgent { //////////////////////////// provided for subscription agent //////////////////////////// - public List poll( - ConsumerConfig consumerConfig, Set topicNames, SubscriptionPollTimer timer) { - String consumerGroupId = consumerConfig.getConsumerGroupId(); - SubscriptionBroker broker = consumerGroupIdToSubscriptionBroker.get(consumerGroupId); + public List poll( + final ConsumerConfig consumerConfig, + final Set topicNames, + final SubscriptionPollTimer timer) { + final String consumerGroupId = consumerConfig.getConsumerGroupId(); + final SubscriptionBroker broker = consumerGroupIdToSubscriptionBroker.get(consumerGroupId); if (Objects.isNull(broker)) { LOGGER.warn( "Subscription: broker bound to consumer group [{}] does not exist", consumerGroupId); @@ -58,9 +60,10 @@ public List poll( } public void commit( - ConsumerConfig consumerConfig, Map> topicNameToSubscriptionCommitIds) { - String consumerGroupId = consumerConfig.getConsumerGroupId(); - SubscriptionBroker broker = consumerGroupIdToSubscriptionBroker.get(consumerGroupId); + final ConsumerConfig consumerConfig, + final Map> topicNameToSubscriptionCommitIds) { + final String consumerGroupId = consumerConfig.getConsumerGroupId(); + final SubscriptionBroker broker = consumerGroupIdToSubscriptionBroker.get(consumerGroupId); if (Objects.isNull(broker)) { LOGGER.warn( "Subscription: broker bound to consumer group [{}] does not exist", consumerGroupId); @@ -71,18 +74,18 @@ public void commit( /////////////////////////////// broker /////////////////////////////// - public boolean isBrokerExist(String consumerGroupId) { + public boolean isBrokerExist(final String consumerGroupId) { return consumerGroupIdToSubscriptionBroker.containsKey(consumerGroupId); } - public synchronized void createBroker(String consumerGroupId) { - SubscriptionBroker broker = new SubscriptionBroker(consumerGroupId); + public synchronized void createBroker(final String consumerGroupId) { + final SubscriptionBroker broker = new SubscriptionBroker(consumerGroupId); consumerGroupIdToSubscriptionBroker.put(consumerGroupId, broker); } /** @return true -> if drop broker success */ - public synchronized boolean dropBroker(String consumerGroupId) { - SubscriptionBroker broker = consumerGroupIdToSubscriptionBroker.get(consumerGroupId); + public synchronized boolean dropBroker(final String consumerGroupId) { + final SubscriptionBroker broker = consumerGroupIdToSubscriptionBroker.get(consumerGroupId); if (Objects.isNull(broker)) { LOGGER.warn( "Subscription: broker bound to consumer group [{}] does not exist", consumerGroupId); @@ -101,9 +104,9 @@ public synchronized boolean dropBroker(String consumerGroupId) { /////////////////////////////// prefetching queue /////////////////////////////// - public void bindPrefetchingQueue(SubscriptionConnectorSubtask subtask) { - String consumerGroupId = subtask.getConsumerGroupId(); - SubscriptionBroker broker = consumerGroupIdToSubscriptionBroker.get(consumerGroupId); + public void bindPrefetchingQueue(final SubscriptionConnectorSubtask subtask) { + final String consumerGroupId = subtask.getConsumerGroupId(); + final SubscriptionBroker broker = consumerGroupIdToSubscriptionBroker.get(consumerGroupId); if (Objects.isNull(broker)) { LOGGER.warn("Subscription: consumer group [{}] does not exist", consumerGroupId); return; @@ -111,9 +114,9 @@ public void bindPrefetchingQueue(SubscriptionConnectorSubtask subtask) { broker.bindPrefetchingQueue(subtask.getTopicName(), subtask.getInputPendingQueue()); } - public void unbindPrefetchingQueue(SubscriptionConnectorSubtask subtask) { - String consumerGroupId = subtask.getConsumerGroupId(); - SubscriptionBroker broker = consumerGroupIdToSubscriptionBroker.get(consumerGroupId); + public void unbindPrefetchingQueue(final SubscriptionConnectorSubtask subtask) { + final String consumerGroupId = subtask.getConsumerGroupId(); + final SubscriptionBroker broker = consumerGroupIdToSubscriptionBroker.get(consumerGroupId); if (Objects.isNull(broker)) { LOGGER.warn("Subscription: consumer group [{}] does not exist", consumerGroupId); return; @@ -121,9 +124,9 @@ public void unbindPrefetchingQueue(SubscriptionConnectorSubtask subtask) { broker.unbindPrefetchingQueue(subtask.getTopicName()); } - public void executePrefetch(SubscriptionConnectorSubtask subtask) { - String consumerGroupId = subtask.getConsumerGroupId(); - SubscriptionBroker broker = consumerGroupIdToSubscriptionBroker.get(consumerGroupId); + public void executePrefetch(final SubscriptionConnectorSubtask subtask) { + final String consumerGroupId = subtask.getConsumerGroupId(); + final SubscriptionBroker broker = consumerGroupIdToSubscriptionBroker.get(consumerGroupId); if (Objects.isNull(broker)) { LOGGER.warn("Subscription: consumer group [{}] does not exist", consumerGroupId); return; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionTopicAgent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionTopicAgent.java index 1a4c6697f825..5190f58550bf 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionTopicAgent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionTopicAgent.java @@ -22,6 +22,7 @@ import org.apache.iotdb.commons.subscription.meta.topic.TopicMeta; import org.apache.iotdb.commons.subscription.meta.topic.TopicMetaKeeper; import org.apache.iotdb.mpp.rpc.thrift.TPushTopicMetaRespExceptionMessage; +import org.apache.iotdb.rpc.subscription.config.TopicConstant; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -137,4 +138,16 @@ public boolean isTopicExisted(String topicName) { releaseReadLock(); } } + + public String getTopicFormat(String topicName) { + acquireReadLock(); + try { + return topicMetaKeeper + .getTopicMeta(topicName) + .getConfig() + .getStringOrDefault(TopicConstant.FORMAT_KEY, TopicConstant.FORMAT_DEFAULT_VALUE); + } finally { + releaseReadLock(); + } + } } 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 5bedd567c202..e8929b3cdc14 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 @@ -20,9 +20,11 @@ package org.apache.iotdb.db.subscription.broker; import org.apache.iotdb.commons.pipe.task.connection.BoundedBlockingPendingQueue; -import org.apache.iotdb.db.subscription.event.SerializableEnrichedTabletsSubscriptionEvent; +import org.apache.iotdb.db.subscription.agent.SubscriptionAgent; +import org.apache.iotdb.db.subscription.event.SubscriptionEvent; import org.apache.iotdb.db.subscription.timer.SubscriptionPollTimer; import org.apache.iotdb.pipe.api.event.Event; +import org.apache.iotdb.rpc.subscription.config.TopicConstant; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -42,7 +44,7 @@ public class SubscriptionBroker { private final Map topicNameToPrefetchingQueue; - public SubscriptionBroker(String brokerId) { + public SubscriptionBroker(final String brokerId) { this.brokerId = brokerId; this.topicNameToPrefetchingQueue = new ConcurrentHashMap<>(); } @@ -53,15 +55,15 @@ public boolean isEmpty() { //////////////////////////// provided for SubscriptionBrokerAgent //////////////////////////// - public List poll( - Set topicNames, SubscriptionPollTimer timer) { - List events = new ArrayList<>(); - for (Map.Entry entry : + public List poll( + final Set topicNames, final SubscriptionPollTimer timer) { + final List events = new ArrayList<>(); + for (final Map.Entry entry : topicNameToPrefetchingQueue.entrySet()) { - String topicName = entry.getKey(); - SubscriptionPrefetchingQueue prefetchingQueue = entry.getValue(); + final String topicName = entry.getKey(); + final SubscriptionPrefetchingQueue prefetchingQueue = entry.getValue(); if (topicNames.contains(topicName)) { - SerializableEnrichedTabletsSubscriptionEvent event = prefetchingQueue.poll(timer); + final SubscriptionEvent event = prefetchingQueue.poll(timer); if (Objects.nonNull(event)) { events.add(event); } @@ -74,8 +76,9 @@ public List poll( return events; } - public void commit(Map> topicNameToSubscriptionCommitIds) { - for (Map.Entry> entry : topicNameToSubscriptionCommitIds.entrySet()) { + public void commit(final Map> topicNameToSubscriptionCommitIds) { + for (final Map.Entry> entry : + topicNameToSubscriptionCommitIds.entrySet()) { final String topicName = entry.getKey(); final SubscriptionPrefetchingQueue prefetchingQueue = topicNameToPrefetchingQueue.get(topicName); @@ -91,7 +94,7 @@ public void commit(Map> topicNameToSubscriptionCommitIds) { /////////////////////////////// prefetching queue /////////////////////////////// public void bindPrefetchingQueue( - String topicName, BoundedBlockingPendingQueue inputPendingQueue) { + final String topicName, final BoundedBlockingPendingQueue inputPendingQueue) { final SubscriptionPrefetchingQueue prefetchingQueue = topicNameToPrefetchingQueue.get(topicName); if (Objects.nonNull(prefetchingQueue)) { @@ -99,11 +102,19 @@ public void bindPrefetchingQueue( "Subscription: prefetching queue bound to topic [{}] has already existed", topicName); return; } - topicNameToPrefetchingQueue.put( - topicName, new SubscriptionPrefetchingQueue(brokerId, topicName, inputPendingQueue)); + final String topicFormat = SubscriptionAgent.topic().getTopicFormat(topicName); + if (TopicConstant.FORMAT_TS_FILE_READER_VALUE.equals(topicFormat)) { + topicNameToPrefetchingQueue.put( + topicName, + new SubscriptionPrefetchingTsFileQueue(brokerId, topicName, inputPendingQueue)); + } else { + topicNameToPrefetchingQueue.put( + topicName, + new SubscriptionPrefetchingEnrichedTabletsQueue(brokerId, topicName, inputPendingQueue)); + } } - public void unbindPrefetchingQueue(String topicName) { + public void unbindPrefetchingQueue(final String topicName) { final SubscriptionPrefetchingQueue prefetchingQueue = topicNameToPrefetchingQueue.get(topicName); if (Objects.isNull(prefetchingQueue)) { @@ -114,7 +125,7 @@ public void unbindPrefetchingQueue(String topicName) { topicNameToPrefetchingQueue.remove(topicName); } - public void executePrefetch(String topicName) { + public void executePrefetch(final String topicName) { final SubscriptionPrefetchingQueue prefetchingQueue = topicNameToPrefetchingQueue.get(topicName); if (Objects.isNull(prefetchingQueue)) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingEnrichedTabletsQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingEnrichedTabletsQueue.java index f5bafc3637ed..e9891d55676f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingEnrichedTabletsQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingEnrichedTabletsQueue.java @@ -1,12 +1,190 @@ 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.subscription.config.SubscriptionConfig; +import org.apache.iotdb.db.pipe.event.UserDefinedEnrichedEvent; +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.subscription.event.SerializableEnrichedTabletsSubscriptionEvent; +import org.apache.iotdb.db.subscription.event.SubscriptionEvent; +import org.apache.iotdb.db.subscription.timer.SubscriptionPollTimer; import org.apache.iotdb.pipe.api.event.Event; +import org.apache.iotdb.pipe.api.event.dml.insertion.TabletInsertionEvent; +import org.apache.iotdb.rpc.subscription.payload.EnrichedTablets; +import org.apache.iotdb.tsfile.write.record.Tablet; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.List; +import java.util.Objects; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; public class SubscriptionPrefetchingEnrichedTabletsQueue extends SubscriptionPrefetchingQueue { + private static final Logger LOGGER = + LoggerFactory.getLogger(SubscriptionPrefetchingEnrichedTabletsQueue.class); + + private final LinkedBlockingQueue prefetchingQueue; + public SubscriptionPrefetchingEnrichedTabletsQueue( - String brokerId, String topicName, BoundedBlockingPendingQueue inputPendingQueue) { + final String brokerId, + final String topicName, + final BoundedBlockingPendingQueue inputPendingQueue) { super(brokerId, topicName, inputPendingQueue); + + this.prefetchingQueue = new LinkedBlockingQueue<>(); + } + + @Override + public SubscriptionEvent poll(final SubscriptionPollTimer timer) { + if (prefetchingQueue.isEmpty()) { + prefetchOnce(SubscriptionConfig.getInstance().getSubscriptionMaxTabletsPerPrefetching()); + // without serializeOnce here + } + + SerializableEnrichedTabletsSubscriptionEvent currentEvent; + try { + while (Objects.nonNull( + currentEvent = + prefetchingQueue.poll( + SubscriptionConfig.getInstance().getSubscriptionPollMaxBlockingTimeMs(), + TimeUnit.MILLISECONDS))) { + if (currentEvent.isCommitted()) { + continue; + } + // Re-enqueue the uncommitted event at the end of the queue. + prefetchingQueue.add(currentEvent); + // timeout control + timer.update(); + if (timer.isExpired()) { + break; + } + if (!currentEvent.pollable()) { + continue; + } + currentEvent.recordLastPolledTimestamp(); + return currentEvent; + } + } catch (final InterruptedException e) { + Thread.currentThread().interrupt(); + LOGGER.warn("Subscription: Interrupted while polling events.", e); + } + + return null; + } + + @Override + public void executePrefetch() { + prefetchOnce(SubscriptionConfig.getInstance().getSubscriptionMaxTabletsPerPrefetching()); + serializeOnce(); + } + + // TODO: use org.apache.iotdb.db.pipe.resource.memory.PipeMemoryManager.calculateTabletSizeInBytes + // for limit control + private void prefetchOnce(final long limit) { + final List tablets = new ArrayList<>(); + final List enrichedEvents = new ArrayList<>(); + + Event event; + while (Objects.nonNull( + event = UserDefinedEnrichedEvent.maybeOf(inputPendingQueue.waitedPoll()))) { + if (!(event instanceof EnrichedEvent)) { + LOGGER.warn("Subscription: Only support prefetch EnrichedEvent. Ignore {}.", event); + continue; + } + + if (event instanceof TabletInsertionEvent) { + final Tablet tablet = convertToTablet((TabletInsertionEvent) event); + if (Objects.isNull(tablet)) { + continue; + } + tablets.add(tablet); + enrichedEvents.add((EnrichedEvent) event); + if (tablets.size() >= limit) { + break; + } + } else if (event instanceof PipeTsFileInsertionEvent) { + for (final TabletInsertionEvent tabletInsertionEvent : + ((PipeTsFileInsertionEvent) event).toTabletInsertionEvents()) { + final Tablet tablet = convertToTablet(tabletInsertionEvent); + if (Objects.isNull(tablet)) { + continue; + } + tablets.add(tablet); + } + enrichedEvents.add((EnrichedEvent) event); + if (tablets.size() >= limit) { + break; + } + } else { + // TODO: + // - PipeHeartbeatEvent: ignored? (may affect pipe metrics) + // - UserDefinedEnrichedEvent: ignored? + // - Others: events related to meta sync, safe to ignore + LOGGER.warn("Subscription: Ignore EnrichedEvent {} when prefetching.", event); + } + } + + if (!tablets.isEmpty()) { + final String subscriptionCommitId = generateSubscriptionCommitId(); + final SerializableEnrichedTabletsSubscriptionEvent enrichedEvent = + new SerializableEnrichedTabletsSubscriptionEvent( + enrichedEvents, new EnrichedTablets(topicName, tablets, subscriptionCommitId)); + uncommittedEvents.put(subscriptionCommitId, enrichedEvent); // before enqueuing the event + prefetchingQueue.add(enrichedEvent); + } + } + + private void serializeOnce() { + final long size = prefetchingQueue.size(); + long count = 0; + + SerializableEnrichedTabletsSubscriptionEvent currentEvent; + try { + while (Objects.nonNull( + currentEvent = + prefetchingQueue.poll( + SubscriptionConfig.getInstance().getSubscriptionSerializeMaxBlockingTimeMs(), + TimeUnit.MILLISECONDS))) { + if (currentEvent.isCommitted()) { + continue; + } + // Re-enqueue the uncommitted event at the end of the queue. + prefetchingQueue.add(currentEvent); + // limit control + if (count >= size) { + break; + } + count++; + // Serialize the uncommitted and pollable event. + if (currentEvent.pollable()) { + // No need to concern whether serialization is successful. + currentEvent.serialize(); + } + } + } catch (final InterruptedException e) { + Thread.currentThread().interrupt(); + LOGGER.warn("Subscription: Interrupted while serializing events.", e); + } + } + + /////////////////////////////// utility /////////////////////////////// + + private Tablet convertToTablet(final TabletInsertionEvent tabletInsertionEvent) { + if (tabletInsertionEvent instanceof PipeInsertNodeTabletInsertionEvent) { + return ((PipeInsertNodeTabletInsertionEvent) tabletInsertionEvent).convertToTablet(); + } else if (tabletInsertionEvent instanceof PipeRawTabletInsertionEvent) { + return ((PipeRawTabletInsertionEvent) tabletInsertionEvent).convertToTablet(); + } + + LOGGER.warn( + "Subscription: Only support convert PipeInsertNodeTabletInsertionEvent or PipeRawTabletInsertionEvent to tablet. Ignore {}.", + tabletInsertionEvent); + return null; } } 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 eabaed668064..6a68552b0dec 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 @@ -19,99 +19,53 @@ 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.subscription.config.SubscriptionConfig; import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.pipe.agent.PipeAgent; -import org.apache.iotdb.db.pipe.event.UserDefinedEnrichedEvent; -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.subscription.event.SerializableEnrichedTabletsSubscriptionEvent; import org.apache.iotdb.db.subscription.event.SubscriptionEvent; import org.apache.iotdb.db.subscription.timer.SubscriptionPollTimer; import org.apache.iotdb.pipe.api.event.Event; -import org.apache.iotdb.pipe.api.event.dml.insertion.TabletInsertionEvent; -import org.apache.iotdb.rpc.subscription.payload.EnrichedTablets; -import org.apache.iotdb.tsfile.write.record.Tablet; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.Objects; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; public abstract class SubscriptionPrefetchingQueue { private static final Logger LOGGER = LoggerFactory.getLogger(SubscriptionPrefetchingQueue.class); - private final String brokerId; // consumer group id - private final String topicName; - private final BoundedBlockingPendingQueue inputPendingQueue; - - private final Map uncommittedEvents; - private final LinkedBlockingQueue prefetchingQueue; + protected final String brokerId; // consumer group id + protected final String topicName; + protected final BoundedBlockingPendingQueue inputPendingQueue; + protected final Map uncommittedEvents; private final AtomicLong subscriptionCommitIdGenerator = new AtomicLong(0); public SubscriptionPrefetchingQueue( - String brokerId, String topicName, BoundedBlockingPendingQueue inputPendingQueue) { + final String brokerId, + final String topicName, + final BoundedBlockingPendingQueue inputPendingQueue) { this.brokerId = brokerId; this.topicName = topicName; this.inputPendingQueue = inputPendingQueue; + this.uncommittedEvents = new ConcurrentHashMap<>(); - this.prefetchingQueue = new LinkedBlockingQueue<>(); } - /////////////////////////////// provided for SubscriptionBroker /////////////////////////////// - - public SerializableEnrichedTabletsSubscriptionEvent poll(SubscriptionPollTimer timer) { - if (prefetchingQueue.isEmpty()) { - prefetchOnce(SubscriptionConfig.getInstance().getSubscriptionMaxTabletsPerPrefetching()); - // without serializeOnce here - } + public abstract SubscriptionEvent poll(SubscriptionPollTimer timer); - SerializableEnrichedTabletsSubscriptionEvent currentEvent; - try { - while (Objects.nonNull( - currentEvent = - prefetchingQueue.poll( - SubscriptionConfig.getInstance().getSubscriptionPollMaxBlockingTimeMs(), - TimeUnit.MILLISECONDS))) { - if (currentEvent.isCommitted()) { - continue; - } - // Re-enqueue the uncommitted event at the end of the queue. - prefetchingQueue.add(currentEvent); - // timeout control - timer.update(); - if (timer.isExpired()) { - break; - } - if (!currentEvent.pollable()) { - continue; - } - currentEvent.recordLastPolledTimestamp(); - return currentEvent; - } - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - LOGGER.warn("Subscription: Interrupted while polling events.", e); - } + public abstract void executePrefetch(); - return null; - } + /////////////////////////////// commit /////////////////////////////// - public void commit(List subscriptionCommitIds) { - for (String subscriptionCommitId : subscriptionCommitIds) { - SubscriptionEvent event = uncommittedEvents.get(subscriptionCommitId); + public void commit(final List subscriptionCommitIds) { + for (final String subscriptionCommitId : subscriptionCommitIds) { + final SubscriptionEvent event = uncommittedEvents.get(subscriptionCommitId); if (Objects.isNull(event)) { LOGGER.warn( "Subscription: subscription commit id [{}] does not exist, it may have been committed or something unexpected happened", @@ -124,118 +78,7 @@ public void commit(List subscriptionCommitIds) { } } - /////////////////////////////// prefetch /////////////////////////////// - - public void executePrefetch() { - prefetchOnce(SubscriptionConfig.getInstance().getSubscriptionMaxTabletsPerPrefetching()); - serializeOnce(); - } - - // TODO: use org.apache.iotdb.db.pipe.resource.memory.PipeMemoryManager.calculateTabletSizeInBytes - // for limit control - private void prefetchOnce(long limit) { - List tablets = new ArrayList<>(); - List enrichedEvents = new ArrayList<>(); - - Event event; - while (Objects.nonNull( - event = UserDefinedEnrichedEvent.maybeOf(inputPendingQueue.waitedPoll()))) { - if (!(event instanceof EnrichedEvent)) { - LOGGER.warn("Subscription: Only support prefetch EnrichedEvent. Ignore {}.", event); - continue; - } - - if (event instanceof TabletInsertionEvent) { - Tablet tablet = convertToTablet((TabletInsertionEvent) event); - if (Objects.isNull(tablet)) { - continue; - } - tablets.add(tablet); - enrichedEvents.add((EnrichedEvent) event); - if (tablets.size() >= limit) { - break; - } - } else if (event instanceof PipeTsFileInsertionEvent) { - for (TabletInsertionEvent tabletInsertionEvent : - ((PipeTsFileInsertionEvent) event).toTabletInsertionEvents()) { - Tablet tablet = convertToTablet(tabletInsertionEvent); - if (Objects.isNull(tablet)) { - continue; - } - tablets.add(tablet); - } - enrichedEvents.add((EnrichedEvent) event); - if (tablets.size() >= limit) { - break; - } - } else { - // TODO: - // - PipeHeartbeatEvent: ignored? (may affect pipe metrics) - // - UserDefinedEnrichedEvent: ignored? - // - Others: events related to meta sync, safe to ignore - LOGGER.warn("Subscription: Ignore EnrichedEvent {} when prefetching.", event); - } - } - - if (!tablets.isEmpty()) { - String subscriptionCommitId = generateSubscriptionCommitId(); - SerializableEnrichedTabletsSubscriptionEvent enrichedEvent = - new SerializableEnrichedTabletsSubscriptionEvent( - new EnrichedTablets(topicName, tablets, subscriptionCommitId), enrichedEvents); - uncommittedEvents.put(subscriptionCommitId, enrichedEvent); // before enqueuing the event - prefetchingQueue.add(enrichedEvent); - } - } - - private void serializeOnce() { - long size = prefetchingQueue.size(); - long count = 0; - - SerializableEnrichedTabletsSubscriptionEvent currentEvent; - try { - while (Objects.nonNull( - currentEvent = - prefetchingQueue.poll( - SubscriptionConfig.getInstance().getSubscriptionSerializeMaxBlockingTimeMs(), - TimeUnit.MILLISECONDS))) { - if (currentEvent.isCommitted()) { - continue; - } - // Re-enqueue the uncommitted event at the end of the queue. - prefetchingQueue.add(currentEvent); - // limit control - if (count >= size) { - break; - } - count++; - // Serialize the uncommitted and pollable event. - if (currentEvent.pollable()) { - // No need to concern whether serialization is successful. - currentEvent.serialize(); - } - } - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - LOGGER.warn("Subscription: Interrupted while serializing events.", e); - } - } - - /////////////////////////////// utility /////////////////////////////// - - private Tablet convertToTablet(TabletInsertionEvent tabletInsertionEvent) { - if (tabletInsertionEvent instanceof PipeInsertNodeTabletInsertionEvent) { - return ((PipeInsertNodeTabletInsertionEvent) tabletInsertionEvent).convertToTablet(); - } else if (tabletInsertionEvent instanceof PipeRawTabletInsertionEvent) { - return ((PipeRawTabletInsertionEvent) tabletInsertionEvent).convertToTablet(); - } - - LOGGER.warn( - "Subscription: Only support convert PipeInsertNodeTabletInsertionEvent or PipeRawTabletInsertionEvent to tablet. Ignore {}.", - tabletInsertionEvent); - return null; - } - - private String generateSubscriptionCommitId() { + protected String generateSubscriptionCommitId() { // subscription commit id format: {DataNodeId}#{RebootTimes}#{TopicName}_{BrokerId}#{Id} // Recording data node ID and reboot times to address potential stale commit IDs caused by // leader transfers or restarts. diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java index 92c1c94c65eb..0ae5d29343ac 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java @@ -20,12 +20,71 @@ package org.apache.iotdb.db.subscription.broker; import org.apache.iotdb.commons.pipe.task.connection.BoundedBlockingPendingQueue; +import org.apache.iotdb.db.pipe.event.UserDefinedEnrichedEvent; +import org.apache.iotdb.db.pipe.event.common.tsfile.PipeTsFileInsertionEvent; +import org.apache.iotdb.db.subscription.event.SubscriptionEvent; +import org.apache.iotdb.db.subscription.event.TsFileSubscriptionEvent; +import org.apache.iotdb.db.subscription.timer.SubscriptionPollTimer; import org.apache.iotdb.pipe.api.event.Event; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collections; +import java.util.Objects; +import java.util.concurrent.atomic.AtomicReference; + public class SubscriptionPrefetchingTsFileQueue extends SubscriptionPrefetchingQueue { + private static final Logger LOGGER = + LoggerFactory.getLogger(SubscriptionPrefetchingTsFileQueue.class); + + private final AtomicReference tsFileInsertionEventRef; + public SubscriptionPrefetchingTsFileQueue( - String brokerId, String topicName, BoundedBlockingPendingQueue inputPendingQueue) { + final String brokerId, + final String topicName, + final BoundedBlockingPendingQueue inputPendingQueue) { super(brokerId, topicName, inputPendingQueue); + + this.tsFileInsertionEventRef = new AtomicReference<>(); + } + + @Override + public SubscriptionEvent poll(final SubscriptionPollTimer timer) { + if (Objects.nonNull(tsFileInsertionEventRef.get())) { + return null; + } + + Event event; + while (Objects.nonNull( + event = UserDefinedEnrichedEvent.maybeOf(inputPendingQueue.waitedPoll()))) { + if (!(event instanceof PipeTsFileInsertionEvent)) { + LOGGER.warn( + "Subscription: SubscriptionPrefetchingTsFileQueue only support poll PipeTsFileInsertionEvent. Ignore {}.", + event); + continue; + } + + final PipeTsFileInsertionEvent tsFileInsertionEvent = (PipeTsFileInsertionEvent) event; + tsFileInsertionEventRef.set(tsFileInsertionEvent); + + final String subscriptionCommitId = generateSubscriptionCommitId(); + final TsFileSubscriptionEvent tsFileSubscriptionEvent = + new TsFileSubscriptionEvent( + Collections.singletonList((PipeTsFileInsertionEvent) event), + subscriptionCommitId, + topicName, + tsFileInsertionEvent.getTsFile().getName()); + uncommittedEvents.put(subscriptionCommitId, tsFileSubscriptionEvent); + return tsFileSubscriptionEvent; + } + + return null; + } + + @Override + public void executePrefetch() { + // do nothing now } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SerializableEnrichedTabletsSubscriptionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SerializableEnrichedTabletsSubscriptionEvent.java index dbb4a4c158e7..886c8db71ae4 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SerializableEnrichedTabletsSubscriptionEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SerializableEnrichedTabletsSubscriptionEvent.java @@ -41,7 +41,7 @@ public class SerializableEnrichedTabletsSubscriptionEvent extends SubscriptionEv private ByteBuffer byteBuffer; // serialized EnrichedTablets public SerializableEnrichedTabletsSubscriptionEvent( - EnrichedTablets enrichedTablets, List enrichedEvents) { + final List enrichedEvents, final EnrichedTablets enrichedTablets) { super(enrichedEvents, enrichedTablets.getSubscriptionCommitId()); this.enrichedTablets = enrichedTablets; } @@ -58,7 +58,7 @@ public boolean serialize() { try { byteBuffer = PipeSubscribePollResp.serializeEnrichedTablets(enrichedTablets); return true; - } catch (IOException e) { + } catch (final IOException e) { LOGGER.warn( "Subscription: something unexpected happened when serializing EnrichedTablets {}, exception is {}", byteBuffer, diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionEvent.java index 5c04b64e65db..4b1afd0a6227 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionEvent.java @@ -34,7 +34,8 @@ public abstract class SubscriptionEvent { private long lastPolledTimestamp; private long committedTimestamp; - public SubscriptionEvent(List enrichedEvents, String subscriptionCommitId) { + public SubscriptionEvent( + final List enrichedEvents, final String subscriptionCommitId) { this.enrichedEvents = enrichedEvents; this.subscriptionCommitId = subscriptionCommitId; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/TsFileSubscriptionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/TsFileSubscriptionEvent.java index 8de05d436f02..d198e61fb438 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/TsFileSubscriptionEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/TsFileSubscriptionEvent.java @@ -6,7 +6,33 @@ public class TsFileSubscriptionEvent extends SubscriptionEvent { - public TsFileSubscriptionEvent(List enrichedEvents, String subscriptionCommitId) { + private final String topicName; + private final String fileName; + + private final long fileLength; + private final long startWritingOffset; + private final byte[] filePiece; + + public TsFileSubscriptionEvent( + final List enrichedEvents, + final String subscriptionCommitId, + final String topicName, + final String fileName) { super(enrichedEvents, subscriptionCommitId); + + this.topicName = topicName; + this.fileName = fileName; + + this.fileLength = -1; + this.startWritingOffset = -1; + this.filePiece = null; + } + + public String getTopicName() { + return topicName; + } + + public String getFileName() { + return fileName; } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java index 62eda351cd1f..91c0e856d40a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java @@ -34,6 +34,8 @@ import org.apache.iotdb.db.protocol.client.ConfigNodeInfo; import org.apache.iotdb.db.subscription.agent.SubscriptionAgent; import org.apache.iotdb.db.subscription.event.SerializableEnrichedTabletsSubscriptionEvent; +import org.apache.iotdb.db.subscription.event.SubscriptionEvent; +import org.apache.iotdb.db.subscription.event.TsFileSubscriptionEvent; import org.apache.iotdb.db.subscription.timer.SubscriptionPollTimer; import org.apache.iotdb.rpc.RpcUtils; import org.apache.iotdb.rpc.TSStatusCode; @@ -68,7 +70,6 @@ import org.slf4j.LoggerFactory; import java.nio.ByteBuffer; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Objects; @@ -106,7 +107,7 @@ public void handleExit() { } @Override - public final TPipeSubscribeResp handle(TPipeSubscribeReq req) { + public final TPipeSubscribeResp handle(final TPipeSubscribeReq req) { final short reqType = req.getType(); if (PipeSubscribeRequestType.isValidatedRequestType(reqType)) { switch (PipeSubscribeRequestType.valueOf(reqType)) { @@ -144,10 +145,10 @@ public final TPipeSubscribeResp handle(TPipeSubscribeReq req) { PipeSubscribeResponseType.ACK.getType()); } - private TPipeSubscribeResp handlePipeSubscribeHandshake(PipeSubscribeHandshakeReq req) { + private TPipeSubscribeResp handlePipeSubscribeHandshake(final PipeSubscribeHandshakeReq req) { try { return handlePipeSubscribeHandshakeInternal(req); - } catch (SubscriptionException e) { + } catch (final SubscriptionException e) { final String exceptionMessage = String.format( "Subscription: something unexpected happened when handshaking: %s, req: %s", @@ -158,11 +159,11 @@ private TPipeSubscribeResp handlePipeSubscribeHandshake(PipeSubscribeHandshakeRe } } - private TPipeSubscribeResp handlePipeSubscribeHandshakeInternal(PipeSubscribeHandshakeReq req) - throws SubscriptionException { + private TPipeSubscribeResp handlePipeSubscribeHandshakeInternal( + final PipeSubscribeHandshakeReq req) throws SubscriptionException { // set consumer config thread local - ConsumerConfig existedConsumerConfig = consumerConfigThreadLocal.get(); - ConsumerConfig consumerConfig = req.getConsumerConfig(); + final ConsumerConfig existedConsumerConfig = consumerConfigThreadLocal.get(); + final ConsumerConfig consumerConfig = req.getConsumerConfig(); if (Objects.isNull(existedConsumerConfig)) { consumerConfigThreadLocal.set(consumerConfig); @@ -188,7 +189,7 @@ private TPipeSubscribeResp handlePipeSubscribeHandshakeInternal(PipeSubscribeHan consumerConfig); } - int dataNodeId = IoTDBDescriptor.getInstance().getConfig().getDataNodeId(); + final int dataNodeId = IoTDBDescriptor.getInstance().getConfig().getDataNodeId(); LOGGER.info( "Subscription: consumer {} handshake successfully, data node id: {}", req.getConsumerConfig(), @@ -196,10 +197,10 @@ private TPipeSubscribeResp handlePipeSubscribeHandshakeInternal(PipeSubscribeHan return PipeSubscribeHandshakeResp.toTPipeSubscribeResp(RpcUtils.SUCCESS_STATUS, dataNodeId); } - private TPipeSubscribeResp handlePipeSubscribeHeartbeat(PipeSubscribeHeartbeatReq req) { + private TPipeSubscribeResp handlePipeSubscribeHeartbeat(final PipeSubscribeHeartbeatReq req) { try { return handlePipeSubscribeHeartbeatInternal(req); - } catch (SubscriptionException e) { + } catch (final SubscriptionException e) { final String exceptionMessage = String.format( "Subscription: something unexpected happened when heartbeat: %s, req: %s", @@ -210,9 +211,10 @@ private TPipeSubscribeResp handlePipeSubscribeHeartbeat(PipeSubscribeHeartbeatRe } } - private TPipeSubscribeResp handlePipeSubscribeHeartbeatInternal(PipeSubscribeHeartbeatReq req) { + private TPipeSubscribeResp handlePipeSubscribeHeartbeatInternal( + final PipeSubscribeHeartbeatReq req) { // check consumer config thread local - ConsumerConfig consumerConfig = consumerConfigThreadLocal.get(); + final ConsumerConfig consumerConfig = consumerConfigThreadLocal.get(); if (Objects.isNull(consumerConfig)) { LOGGER.warn( "Subscription: missing consumer config when handling PipeSubscribeHeartbeatReq: {}", req); @@ -225,10 +227,10 @@ private TPipeSubscribeResp handlePipeSubscribeHeartbeatInternal(PipeSubscribeHea return PipeSubscribeHeartbeatResp.toTPipeSubscribeResp(RpcUtils.SUCCESS_STATUS); } - private TPipeSubscribeResp handlePipeSubscribeSubscribe(PipeSubscribeSubscribeReq req) { + private TPipeSubscribeResp handlePipeSubscribeSubscribe(final PipeSubscribeSubscribeReq req) { try { return handlePipeSubscribeSubscribeInternal(req); - } catch (SubscriptionException e) { + } catch (final SubscriptionException e) { final String exceptionMessage = String.format( "Subscription: something unexpected happened when subscribing: %s, req: %s", @@ -239,9 +241,10 @@ private TPipeSubscribeResp handlePipeSubscribeSubscribe(PipeSubscribeSubscribeRe } } - private TPipeSubscribeResp handlePipeSubscribeSubscribeInternal(PipeSubscribeSubscribeReq req) { + private TPipeSubscribeResp handlePipeSubscribeSubscribeInternal( + final PipeSubscribeSubscribeReq req) { // check consumer config thread local - ConsumerConfig consumerConfig = consumerConfigThreadLocal.get(); + final ConsumerConfig consumerConfig = consumerConfigThreadLocal.get(); if (Objects.isNull(consumerConfig)) { LOGGER.warn( "Subscription: missing consumer config when handling PipeSubscribeSubscribeReq: {}", req); @@ -249,17 +252,17 @@ private TPipeSubscribeResp handlePipeSubscribeSubscribeInternal(PipeSubscribeSub } // subscribe topics - Set topicNames = req.getTopicNames(); + final Set topicNames = req.getTopicNames(); subscribe(consumerConfig, topicNames); LOGGER.info("Subscription: consumer {} subscribe {} successfully", consumerConfig, topicNames); return PipeSubscribeSubscribeResp.toTPipeSubscribeResp(RpcUtils.SUCCESS_STATUS); } - private TPipeSubscribeResp handlePipeSubscribeUnsubscribe(PipeSubscribeUnsubscribeReq req) { + private TPipeSubscribeResp handlePipeSubscribeUnsubscribe(final PipeSubscribeUnsubscribeReq req) { try { return handlePipeSubscribeUnsubscribeInternal(req); - } catch (SubscriptionException e) { + } catch (final SubscriptionException e) { final String exceptionMessage = String.format( "Subscription: something unexpected happened when unsubscribing: %s, req: %s", @@ -271,9 +274,9 @@ private TPipeSubscribeResp handlePipeSubscribeUnsubscribe(PipeSubscribeUnsubscri } private TPipeSubscribeResp handlePipeSubscribeUnsubscribeInternal( - PipeSubscribeUnsubscribeReq req) { + final PipeSubscribeUnsubscribeReq req) { // check consumer config thread local - ConsumerConfig consumerConfig = consumerConfigThreadLocal.get(); + final ConsumerConfig consumerConfig = consumerConfigThreadLocal.get(); if (Objects.isNull(consumerConfig)) { LOGGER.warn( "Subscription: missing consumer config when handling PipeSubscribeUnsubscribeReq: {}", @@ -282,7 +285,7 @@ private TPipeSubscribeResp handlePipeSubscribeUnsubscribeInternal( } // unsubscribe topics - Set topicNames = req.getTopicNames(); + final Set topicNames = req.getTopicNames(); unsubscribe(consumerConfig, topicNames); LOGGER.info( @@ -290,10 +293,10 @@ private TPipeSubscribeResp handlePipeSubscribeUnsubscribeInternal( return PipeSubscribeUnsubscribeResp.toTPipeSubscribeResp(RpcUtils.SUCCESS_STATUS); } - private TPipeSubscribeResp handlePipeSubscribePoll(PipeSubscribePollReq req) { + private TPipeSubscribeResp handlePipeSubscribePoll(final PipeSubscribePollReq req) { try { return handlePipeSubscribePollInternal(req); - } catch (SubscriptionException e) { + } catch (final SubscriptionException e) { final String exceptionMessage = String.format( "Subscription: something unexpected happened when polling: %s, req: %s", @@ -304,16 +307,16 @@ private TPipeSubscribeResp handlePipeSubscribePoll(PipeSubscribePollReq req) { } } - private TPipeSubscribeResp handlePipeSubscribePollInternal(PipeSubscribePollReq req) { + private TPipeSubscribeResp handlePipeSubscribePollInternal(final PipeSubscribePollReq req) { // check consumer config thread local - ConsumerConfig consumerConfig = consumerConfigThreadLocal.get(); + final ConsumerConfig consumerConfig = consumerConfigThreadLocal.get(); if (Objects.isNull(consumerConfig)) { LOGGER.warn( "Subscription: missing consumer config when handling PipeSubscribePollReq: {}", req); return SUBSCRIPTION_MISSING_CUSTOMER_RESP; } - // poll + // get topic names Set topicNames = req.getTopicNames(); if (topicNames.isEmpty()) { // poll all subscribed topics @@ -322,7 +325,9 @@ private TPipeSubscribeResp handlePipeSubscribePollInternal(PipeSubscribePollReq .getTopicsSubscribedByConsumer( consumerConfig.getConsumerGroupId(), consumerConfig.getConsumerId()); } - SubscriptionPollTimer timer = + + // poll + final SubscriptionPollTimer timer = new SubscriptionPollTimer( System.currentTimeMillis(), req.getTimeoutMs() == 0 @@ -330,40 +335,70 @@ private TPipeSubscribeResp handlePipeSubscribePollInternal(PipeSubscribePollReq : Math.max( req.getTimeoutMs(), SubscriptionConfig.getInstance().getSubscriptionMinPollTimeoutMs())); - List events = + final List events = SubscriptionAgent.broker().poll(consumerConfig, topicNames, timer); - List subscriptionCommitIds = + + // filter SerializableEnrichedTabletsSubscriptionEvent + final List enrichedTabletsSubscriptionEvents = events.stream() - .map(SerializableEnrichedTabletsSubscriptionEvent::getSubscriptionCommitId) + .filter(event -> event instanceof SerializableEnrichedTabletsSubscriptionEvent) + .map(event -> (SerializableEnrichedTabletsSubscriptionEvent) event) .collect(Collectors.toList()); + // get subscriptionCommitIds from SerializableEnrichedTabletsSubscriptionEvent + final List subscriptionCommitIds = + enrichedTabletsSubscriptionEvents.stream() + .map(SubscriptionEvent::getSubscriptionCommitId) + .collect(Collectors.toList()); + + // get enrichedTablets with byte buffer from SerializableEnrichedTabletsSubscriptionEvent + final List> enrichedTabletsWithByteBufferList = + enrichedTabletsSubscriptionEvents.stream() + .map(event -> new Pair<>(event.getByteBuffer(), event.getEnrichedTablets())) + .collect(Collectors.toList()); + + // filter TsFileSubscriptionEvent + final List tsFileSubscriptionEvents = + events.stream() + .filter(event -> event instanceof TsFileSubscriptionEvent) + .map(event -> (TsFileSubscriptionEvent) event) + .collect(Collectors.toList()); + + final Map topicNameToTsFileNameMap = + tsFileSubscriptionEvents.stream() + .collect( + Collectors.toMap( + TsFileSubscriptionEvent::getTopicName, TsFileSubscriptionEvent::getFileName)); + + // generate response + final TPipeSubscribeResp resp = + PipeSubscribePollResp.toTPipeSubscribeResp( + RpcUtils.SUCCESS_STATUS, enrichedTabletsWithByteBufferList, topicNameToTsFileNameMap); + + // reset byte buffer + enrichedTabletsSubscriptionEvents.forEach( + SerializableEnrichedTabletsSubscriptionEvent::resetByteBuffer); + + // check timer if (timer.isExpired()) { LOGGER.warn( "Subscription: timeout happened when consumer {} poll topics {}", consumerConfig, topicNames); } + LOGGER.info( "Subscription: consumer {} poll topics {} successfully, commit ids: {}", consumerConfig, topicNames, subscriptionCommitIds); - - List> enrichedTabletsWithByteBufferList = - events.stream() - .map(event -> new Pair<>(event.getByteBuffer(), event.getEnrichedTablets())) - .collect(Collectors.toList()); - TPipeSubscribeResp resp = - PipeSubscribePollResp.toTPipeSubscribeResp( - RpcUtils.SUCCESS_STATUS, enrichedTabletsWithByteBufferList, new HashMap<>()); - events.forEach(SerializableEnrichedTabletsSubscriptionEvent::resetByteBuffer); return resp; } - private TPipeSubscribeResp handlePipeSubscribePollTsFile(PipeSubscribePollTsFileReq req) { + private TPipeSubscribeResp handlePipeSubscribePollTsFile(final PipeSubscribePollTsFileReq req) { try { return handlePipeSubscribePollTsFileInternal(req); - } catch (SubscriptionException e) { + } catch (final SubscriptionException e) { final String exceptionMessage = String.format( "Subscription: something unexpected happened when polling tsfile: %s, req: %s", @@ -375,14 +410,15 @@ private TPipeSubscribeResp handlePipeSubscribePollTsFile(PipeSubscribePollTsFile } // TODO - private TPipeSubscribeResp handlePipeSubscribePollTsFileInternal(PipeSubscribePollTsFileReq req) { + private TPipeSubscribeResp handlePipeSubscribePollTsFileInternal( + final PipeSubscribePollTsFileReq req) { return null; } - private TPipeSubscribeResp handlePipeSubscribeCommit(PipeSubscribeCommitReq req) { + private TPipeSubscribeResp handlePipeSubscribeCommit(final PipeSubscribeCommitReq req) { try { return handlePipeSubscribeCommitInternal(req); - } catch (SubscriptionException e) { + } catch (final SubscriptionException e) { final String exceptionMessage = String.format( "Subscription: something unexpected happened when committing: %s, req: %s", @@ -393,9 +429,9 @@ private TPipeSubscribeResp handlePipeSubscribeCommit(PipeSubscribeCommitReq req) } } - private TPipeSubscribeResp handlePipeSubscribeCommitInternal(PipeSubscribeCommitReq req) { + private TPipeSubscribeResp handlePipeSubscribeCommitInternal(final PipeSubscribeCommitReq req) { // check consumer config thread local - ConsumerConfig consumerConfig = consumerConfigThreadLocal.get(); + final ConsumerConfig consumerConfig = consumerConfigThreadLocal.get(); if (Objects.isNull(consumerConfig)) { LOGGER.warn( "Subscription: missing consumer config when handling PipeSubscribeCommitReq: {}", req); @@ -403,7 +439,7 @@ private TPipeSubscribeResp handlePipeSubscribeCommitInternal(PipeSubscribeCommit } // commit - Map> topicNameToSubscriptionCommitIds = + final Map> topicNameToSubscriptionCommitIds = req.getTopicNameToSubscriptionCommitIds(); SubscriptionAgent.broker().commit(consumerConfig, topicNameToSubscriptionCommitIds); @@ -414,10 +450,10 @@ private TPipeSubscribeResp handlePipeSubscribeCommitInternal(PipeSubscribeCommit return PipeSubscribeCommitResp.toTPipeSubscribeResp(RpcUtils.SUCCESS_STATUS); } - private TPipeSubscribeResp handlePipeSubscribeClose(PipeSubscribeCloseReq req) { + private TPipeSubscribeResp handlePipeSubscribeClose(final PipeSubscribeCloseReq req) { try { return handlePipeSubscribeCloseInternal(req); - } catch (SubscriptionException e) { + } catch (final SubscriptionException e) { final String exceptionMessage = String.format( "Subscription: something unexpected happened when closing: %s, req: %s", @@ -428,9 +464,9 @@ private TPipeSubscribeResp handlePipeSubscribeClose(PipeSubscribeCloseReq req) { } } - private TPipeSubscribeResp handlePipeSubscribeCloseInternal(PipeSubscribeCloseReq req) { + private TPipeSubscribeResp handlePipeSubscribeCloseInternal(final PipeSubscribeCloseReq req) { // check consumer config thread local - ConsumerConfig consumerConfig = consumerConfigThreadLocal.get(); + final ConsumerConfig consumerConfig = consumerConfigThreadLocal.get(); if (Objects.isNull(consumerConfig)) { LOGGER.warn( "Subscription: missing consumer config when handling PipeSubscribeCloseReq: {}", req); @@ -438,7 +474,7 @@ private TPipeSubscribeResp handlePipeSubscribeCloseInternal(PipeSubscribeCloseRe } // unsubscribe all subscribed topics - Set topics = + final Set topics = SubscriptionAgent.consumer() .getTopicsSubscribedByConsumer( consumerConfig.getConsumerGroupId(), consumerConfig.getConsumerId()); @@ -466,8 +502,8 @@ private TPipeSubscribeResp handlePipeSubscribeCloseInternal(PipeSubscribeCloseRe //////////////////////////// consumer operations //////////////////////////// - private void createConsumer(ConsumerConfig consumerConfig) throws SubscriptionException { - try (ConfigNodeClient configNodeClient = + private void createConsumer(final ConsumerConfig consumerConfig) throws SubscriptionException { + try (final ConfigNodeClient configNodeClient = CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) { final TCreateConsumerReq req = new TCreateConsumerReq() @@ -483,7 +519,7 @@ private void createConsumer(ConsumerConfig consumerConfig) throws SubscriptionEx LOGGER.warn(exceptionMessage); throw new SubscriptionException(exceptionMessage); } - } catch (ClientManagerException | TException e) { + } catch (final ClientManagerException | TException e) { final String exceptionMessage = String.format( "Subscription: Failed to create consumer %s in config node, exception is %s.", @@ -493,8 +529,8 @@ private void createConsumer(ConsumerConfig consumerConfig) throws SubscriptionEx } } - private void dropConsumer(ConsumerConfig consumerConfig) throws SubscriptionException { - try (ConfigNodeClient configNodeClient = + private void dropConsumer(final ConsumerConfig consumerConfig) throws SubscriptionException { + try (final ConfigNodeClient configNodeClient = CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) { final TCloseConsumerReq req = new TCloseConsumerReq() @@ -509,7 +545,7 @@ private void dropConsumer(ConsumerConfig consumerConfig) throws SubscriptionExce LOGGER.warn(exceptionMessage); throw new SubscriptionException(exceptionMessage); } - } catch (ClientManagerException | TException e) { + } catch (final ClientManagerException | TException e) { final String exceptionMessage = String.format( "Subscription: Failed to close consumer %s in config node, exception is %s.", @@ -521,9 +557,9 @@ private void dropConsumer(ConsumerConfig consumerConfig) throws SubscriptionExce // TODO: broker TTL if no consumer in consumer group } - private void subscribe(ConsumerConfig consumerConfig, Set topicNames) + private void subscribe(final ConsumerConfig consumerConfig, final Set topicNames) throws SubscriptionException { - try (ConfigNodeClient configNodeClient = + try (final ConfigNodeClient configNodeClient = CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) { final TSubscribeReq req = new TSubscribeReq() @@ -539,7 +575,7 @@ private void subscribe(ConsumerConfig consumerConfig, Set topicNames) LOGGER.warn(exceptionMessage); throw new SubscriptionException(exceptionMessage); } - } catch (ClientManagerException | TException e) { + } catch (final ClientManagerException | TException e) { final String exceptionMessage = String.format( "Subscription: Failed to subscribe topics %s for consumer %s in config node, exception is %s.", @@ -549,9 +585,9 @@ private void subscribe(ConsumerConfig consumerConfig, Set topicNames) } } - private void unsubscribe(ConsumerConfig consumerConfig, Set topicNames) + private void unsubscribe(final ConsumerConfig consumerConfig, final Set topicNames) throws SubscriptionException { - try (ConfigNodeClient configNodeClient = + try (final ConfigNodeClient configNodeClient = CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) { final TUnsubscribeReq req = new TUnsubscribeReq() @@ -567,7 +603,7 @@ private void unsubscribe(ConsumerConfig consumerConfig, Set topicNames) LOGGER.warn(exceptionMessage); throw new SubscriptionException(exceptionMessage); } - } catch (ClientManagerException | TException e) { + } catch (final ClientManagerException | TException e) { final String exceptionMessage = String.format( "Subscription: Failed to unsubscribe topics %s for consumer %s in config node, exception is %s.", From 4eefc00f7cefb2188464c63d6c3443986e5b0a19 Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Thu, 11 Apr 2024 17:28:00 +0800 Subject: [PATCH 06/93] backup --- .../payload/{ => common}/EnrichedTablets.java | 52 +++----- .../payload/common/EnrichedTsFileInfo.java | 5 + .../common/SubscriptionCommitContext.java | 126 ++++++++++++++++++ .../common/SubscriptionPolledRawMessage.java | 7 + .../response/PipeSubscribePollResp.java | 4 +- .../subscription/SubscriptionMessage.java | 6 +- .../SubscriptionPullConsumer.java | 2 +- .../SubscriptionSessionConnection.java | 11 +- ...iptionPrefetchingEnrichedTabletsQueue.java | 9 +- .../broker/SubscriptionPrefetchingQueue.java | 22 ++- .../SubscriptionPrefetchingTsFileQueue.java | 19 ++- ...zableEnrichedTabletsSubscriptionEvent.java | 10 +- .../subscription/event/SubscriptionEvent.java | 11 +- .../event/TsFileSubscriptionEvent.java | 12 ++ .../receiver/SubscriptionReceiverV1.java | 34 ++--- 15 files changed, 228 insertions(+), 102 deletions(-) rename iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/{ => common}/EnrichedTablets.java (56%) create mode 100644 iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/EnrichedTsFileInfo.java create mode 100644 iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionCommitContext.java create mode 100644 iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledRawMessage.java diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/EnrichedTablets.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/EnrichedTablets.java similarity index 56% rename from iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/EnrichedTablets.java rename to iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/EnrichedTablets.java index 9bcae958d684..ecfeb02a3ec0 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/EnrichedTablets.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/EnrichedTablets.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.iotdb.rpc.subscription.payload; +package org.apache.iotdb.rpc.subscription.payload.common; import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; import org.apache.iotdb.tsfile.write.record.Tablet; @@ -31,67 +31,49 @@ public class EnrichedTablets { - private transient String topicName; - private transient String subscriptionCommitId; - private transient List tablets; - - public String getTopicName() { - return topicName; - } - - public String getSubscriptionCommitId() { - return subscriptionCommitId; - } + private final transient SubscriptionCommitContext commitContext; + private final transient List tablets; public List getTablets() { return tablets; } - public EnrichedTablets() { - this.tablets = new ArrayList<>(); - } - - public EnrichedTablets(String topicName, List tablets, String subscriptionCommitId) { - this.topicName = topicName; + public EnrichedTablets(final SubscriptionCommitContext commitContext, final List tablets) { + this.commitContext = commitContext; this.tablets = tablets; - this.subscriptionCommitId = subscriptionCommitId; } /////////////////////////////// de/ser /////////////////////////////// - public void serialize(DataOutputStream stream) throws IOException { - ReadWriteIOUtils.write(topicName, stream); - ReadWriteIOUtils.write(subscriptionCommitId, stream); + public void serialize(final DataOutputStream stream) throws IOException { ReadWriteIOUtils.write(tablets.size(), stream); - for (Tablet tablet : tablets) { + for (final Tablet tablet : tablets) { tablet.serialize(stream); } } - public static EnrichedTablets deserialize(ByteBuffer buffer) { - final EnrichedTablets enrichedTablets = new EnrichedTablets(); - enrichedTablets.topicName = ReadWriteIOUtils.readString(buffer); - enrichedTablets.subscriptionCommitId = ReadWriteIOUtils.readString(buffer); - int size = ReadWriteIOUtils.readInt(buffer); + public static EnrichedTablets deserialize(final ByteBuffer buffer) { + final SubscriptionCommitContext commitContext = SubscriptionCommitContext.deserialize(buffer); + final List tablets = new ArrayList<>(); + final int size = ReadWriteIOUtils.readInt(buffer); for (int i = 0; i < size; ++i) { - enrichedTablets.tablets.add(Tablet.deserialize(buffer)); + tablets.add(Tablet.deserialize(buffer)); } - return enrichedTablets; + return new EnrichedTablets(commitContext, tablets); } /////////////////////////////// Object /////////////////////////////// @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; } - EnrichedTablets that = (EnrichedTablets) obj; - return Objects.equals(this.topicName, that.topicName) - && Objects.equals(this.subscriptionCommitId, that.subscriptionCommitId) + final EnrichedTablets that = (EnrichedTablets) obj; + return Objects.equals(this.commitContext, that.commitContext) && Objects.equals(this.tablets, that.tablets); } @@ -99,6 +81,6 @@ public boolean equals(Object obj) { public int hashCode() { // Considering that the Tablet class has not implemented the hashCode method, the tablets member // should not be included when calculating the hashCode of EnrichedTablets. - return Objects.hash(topicName, subscriptionCommitId); + return Objects.hash(commitContext); } } diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/EnrichedTsFileInfo.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/EnrichedTsFileInfo.java new file mode 100644 index 000000000000..68a1b3b13cf0 --- /dev/null +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/EnrichedTsFileInfo.java @@ -0,0 +1,5 @@ +package org.apache.iotdb.rpc.subscription.payload.common; + +public class EnrichedTsFileInfo { + +} diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionCommitContext.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionCommitContext.java new file mode 100644 index 000000000000..3188cf25d657 --- /dev/null +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionCommitContext.java @@ -0,0 +1,126 @@ +/* + * 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.rpc.subscription.payload.common; + +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Objects; +import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; + +public class SubscriptionCommitContext { + + private final int dataNodeId; + + private final int rebootTimes; + + private final String topicName; + + private final String consumerGroupId; + + private final long commitId; + + public SubscriptionCommitContext(final int dataNodeId, final int rebootTimes, final String topicName, + final String consumerGroupId, final long commitId) { + this.dataNodeId = dataNodeId; + this.rebootTimes = rebootTimes; + this.topicName = topicName; + this.consumerGroupId = consumerGroupId; + this.commitId = commitId; + } + + public int getDataNodeId() { + return dataNodeId; + } + + public int getRebootTimes() { + return rebootTimes; + } + + public String getTopicName() { + return topicName; + } + + public String getConsumerGroupId() { + return consumerGroupId; + } + + public long getCommitId() { + return commitId; + } + + /////////////////////////////// de/ser /////////////////////////////// + + public void serialize(final DataOutputStream stream) throws IOException { + ReadWriteIOUtils.write(dataNodeId, stream); + ReadWriteIOUtils.write(rebootTimes, stream); + ReadWriteIOUtils.write(topicName, stream); + ReadWriteIOUtils.write(consumerGroupId, stream); + ReadWriteIOUtils.write(commitId, stream); + } + + public static SubscriptionCommitContext deserialize(final ByteBuffer buffer) { + final int dataNodeId = ReadWriteIOUtils.readInt(buffer); + final int rebootTimes = ReadWriteIOUtils.readInt(buffer); + final String topicName = ReadWriteIOUtils.readString(buffer); + final String consumerGroupId = ReadWriteIOUtils.readString(buffer); + final long commitId = ReadWriteIOUtils.readLong(buffer); + return new SubscriptionCommitContext(dataNodeId, rebootTimes, topicName, consumerGroupId, commitId); + } + + /////////////////////////////// Object /////////////////////////////// + + @Override + public boolean equals(final Object obj) { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + final SubscriptionCommitContext that = (SubscriptionCommitContext) obj; + return + this.dataNodeId == that.dataNodeId && + this.rebootTimes == that.rebootTimes && + Objects.equals(this.topicName, that.topicName) + && Objects.equals(this.consumerGroupId, that.consumerGroupId) + && Objects.equals(this.commitId, that.commitId); + } + + @Override + public int hashCode() { + return Objects.hash(dataNodeId, rebootTimes, topicName, consumerGroupId, commitId); + } + + @Override + public String toString() { + return "SubscriptionCommitContext{dataNodeId=" + + dataNodeId + + ", rebootTimes=" + + rebootTimes + + ", topicName=" + + topicName + + ", consumerGroupId=" + + consumerGroupId + + ", commitId=" + + commitId + + "}"; + } +} diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledRawMessage.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledRawMessage.java new file mode 100644 index 000000000000..e235da8b7977 --- /dev/null +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledRawMessage.java @@ -0,0 +1,7 @@ +package org.apache.iotdb.rpc.subscription.payload.common; + +public class SubscriptionPolledRawMessage { + + private final transient SubscriptionCommitContext commitContext; + +} diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollResp.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollResp.java index 497f04fdcab0..675b7dd8d7ca 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollResp.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollResp.java @@ -22,7 +22,7 @@ import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.rpc.RpcUtils; import org.apache.iotdb.rpc.TSStatusCode; -import org.apache.iotdb.rpc.subscription.payload.EnrichedTablets; +import org.apache.iotdb.rpc.subscription.payload.common.EnrichedTablets; import org.apache.iotdb.service.rpc.thrift.TPipeSubscribeResp; import org.apache.iotdb.tsfile.utils.Pair; import org.apache.iotdb.tsfile.utils.PublicBAOS; @@ -40,6 +40,8 @@ public class PipeSubscribePollResp extends TPipeSubscribeResp { + private transient List<> + private transient List enrichedTabletsList = new ArrayList<>(); private transient Map topicNameToTsFileNameMap = new HashMap<>(); diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionMessage.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionMessage.java index 35507f8c7286..ce7715966790 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionMessage.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionMessage.java @@ -19,7 +19,7 @@ package org.apache.iotdb.session.subscription; -import org.apache.iotdb.rpc.subscription.payload.EnrichedTablets; +import org.apache.iotdb.rpc.subscription.payload.common.EnrichedTablets; import java.util.Objects; @@ -37,6 +37,10 @@ public SubscriptionMessage(EnrichedTablets tablets) { this.subscriptionCommitId = tablets.getSubscriptionCommitId(); } + public SubscriptionMessage(String topicName, String fileName) { + + } + public String getTopicName() { return topicName; } diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java index 892ae5a44bf3..0462f94aa789 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java @@ -23,7 +23,7 @@ import org.apache.iotdb.rpc.StatementExecutionException; import org.apache.iotdb.rpc.subscription.config.ConsumerConstant; import org.apache.iotdb.rpc.subscription.exception.SubscriptionException; -import org.apache.iotdb.rpc.subscription.payload.EnrichedTablets; +import org.apache.iotdb.rpc.subscription.payload.common.EnrichedTablets; import org.apache.thrift.TException; import org.slf4j.Logger; diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionSessionConnection.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionSessionConnection.java index 891cd0fc4ffd..11cc600fa3b3 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionSessionConnection.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionSessionConnection.java @@ -25,7 +25,7 @@ import org.apache.iotdb.rpc.RpcUtils; import org.apache.iotdb.rpc.StatementExecutionException; import org.apache.iotdb.rpc.subscription.config.ConsumerConfig; -import org.apache.iotdb.rpc.subscription.payload.EnrichedTablets; +import org.apache.iotdb.rpc.subscription.payload.common.EnrichedTablets; import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribeCloseReq; import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribeCommitReq; import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribeHandshakeReq; @@ -124,13 +124,8 @@ public void unsubscribe(Set topicNames) RpcUtils.verifySuccess(resp.status); } - public List poll(Set topicNames) - throws TException, IOException, StatementExecutionException { - TPipeSubscribeResp resp = - client.pipeSubscribe(PipeSubscribePollReq.toTPipeSubscribeReq(topicNames, 0)); - RpcUtils.verifySuccess(resp.status); - PipeSubscribePollResp pollResp = PipeSubscribePollResp.fromTPipeSubscribeResp(resp); - return pollResp.getEnrichedTabletsList(); + public List poll(Set topicNames) throws TException, IOException, StatementExecutionException { + return poll(topicNames, 0); } public List poll(Set topicNames, long timeoutMs) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingEnrichedTabletsQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingEnrichedTabletsQueue.java index e9891d55676f..f40e0aef4653 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingEnrichedTabletsQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingEnrichedTabletsQueue.java @@ -12,7 +12,8 @@ import org.apache.iotdb.db.subscription.timer.SubscriptionPollTimer; import org.apache.iotdb.pipe.api.event.Event; import org.apache.iotdb.pipe.api.event.dml.insertion.TabletInsertionEvent; -import org.apache.iotdb.rpc.subscription.payload.EnrichedTablets; +import org.apache.iotdb.rpc.subscription.payload.common.EnrichedTablets; +import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionCommitContext; import org.apache.iotdb.tsfile.write.record.Tablet; import org.slf4j.Logger; @@ -131,11 +132,11 @@ private void prefetchOnce(final long limit) { } if (!tablets.isEmpty()) { - final String subscriptionCommitId = generateSubscriptionCommitId(); + final SubscriptionCommitContext commitContext = generateSubscriptionCommitContext(); final SerializableEnrichedTabletsSubscriptionEvent enrichedEvent = new SerializableEnrichedTabletsSubscriptionEvent( - enrichedEvents, new EnrichedTablets(topicName, tablets, subscriptionCommitId)); - uncommittedEvents.put(subscriptionCommitId, enrichedEvent); // before enqueuing the event + enrichedEvents, commitContext, new EnrichedTablets(commitContext, tablets)); + uncommittedEvents.put(commitContext, enrichedEvent); // before enqueuing the event prefetchingQueue.add(enrichedEvent); } } 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 6a68552b0dec..ec0ad772a914 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 @@ -26,6 +26,7 @@ import org.apache.iotdb.db.subscription.timer.SubscriptionPollTimer; import org.apache.iotdb.pipe.api.event.Event; +import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionCommitContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -43,7 +44,7 @@ public abstract class SubscriptionPrefetchingQueue { protected final String topicName; protected final BoundedBlockingPendingQueue inputPendingQueue; - protected final Map uncommittedEvents; + protected final Map uncommittedEvents; private final AtomicLong subscriptionCommitIdGenerator = new AtomicLong(0); public SubscriptionPrefetchingQueue( @@ -78,18 +79,15 @@ public void commit(final List subscriptionCommitIds) { } } - protected String generateSubscriptionCommitId() { - // subscription commit id format: {DataNodeId}#{RebootTimes}#{TopicName}_{BrokerId}#{Id} + protected SubscriptionCommitContext generateSubscriptionCommitContext() { // Recording data node ID and reboot times to address potential stale commit IDs caused by // leader transfers or restarts. - return IoTDBDescriptor.getInstance().getConfig().getDataNodeId() - + "#" - + PipeAgent.runtime().getRebootTimes() - + "#" - + topicName - + "_" - + brokerId - + "#" - + subscriptionCommitIdGenerator.getAndIncrement(); + return new SubscriptionCommitContext( + IoTDBDescriptor.getInstance().getConfig().getDataNodeId(), + PipeAgent.runtime().getRebootTimes(), + topicName, + brokerId, + subscriptionCommitIdGenerator.getAndIncrement() + ); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java index 0ae5d29343ac..1ae47ba513aa 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java @@ -39,7 +39,7 @@ public class SubscriptionPrefetchingTsFileQueue extends SubscriptionPrefetchingQ private static final Logger LOGGER = LoggerFactory.getLogger(SubscriptionPrefetchingTsFileQueue.class); - private final AtomicReference tsFileInsertionEventRef; + private final AtomicReference eventRef; public SubscriptionPrefetchingTsFileQueue( final String brokerId, @@ -47,12 +47,12 @@ public SubscriptionPrefetchingTsFileQueue( final BoundedBlockingPendingQueue inputPendingQueue) { super(brokerId, topicName, inputPendingQueue); - this.tsFileInsertionEventRef = new AtomicReference<>(); + this.eventRef = new AtomicReference<>(); } @Override public SubscriptionEvent poll(final SubscriptionPollTimer timer) { - if (Objects.nonNull(tsFileInsertionEventRef.get())) { + if (Objects.nonNull(eventRef.get())) { return null; } @@ -67,17 +67,16 @@ public SubscriptionEvent poll(final SubscriptionPollTimer timer) { } final PipeTsFileInsertionEvent tsFileInsertionEvent = (PipeTsFileInsertionEvent) event; - tsFileInsertionEventRef.set(tsFileInsertionEvent); - - final String subscriptionCommitId = generateSubscriptionCommitId(); - final TsFileSubscriptionEvent tsFileSubscriptionEvent = + final String subscriptionCommitId = generateSubscriptionCommitContext(); + final TsFileSubscriptionEvent subscriptionEvent = new TsFileSubscriptionEvent( - Collections.singletonList((PipeTsFileInsertionEvent) event), + Collections.singletonList(tsFileInsertionEvent), subscriptionCommitId, topicName, tsFileInsertionEvent.getTsFile().getName()); - uncommittedEvents.put(subscriptionCommitId, tsFileSubscriptionEvent); - return tsFileSubscriptionEvent; + eventRef.set(subscriptionEvent); + uncommittedEvents.put(subscriptionCommitId, subscriptionEvent); + return subscriptionEvent; } return null; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SerializableEnrichedTabletsSubscriptionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SerializableEnrichedTabletsSubscriptionEvent.java index 886c8db71ae4..4fd0d044cdbb 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SerializableEnrichedTabletsSubscriptionEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SerializableEnrichedTabletsSubscriptionEvent.java @@ -20,7 +20,8 @@ package org.apache.iotdb.db.subscription.event; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; -import org.apache.iotdb.rpc.subscription.payload.EnrichedTablets; +import org.apache.iotdb.rpc.subscription.payload.common.EnrichedTablets; +import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionCommitContext; import org.apache.iotdb.rpc.subscription.payload.response.PipeSubscribePollResp; import org.slf4j.Logger; @@ -41,8 +42,11 @@ public class SerializableEnrichedTabletsSubscriptionEvent extends SubscriptionEv private ByteBuffer byteBuffer; // serialized EnrichedTablets public SerializableEnrichedTabletsSubscriptionEvent( - final List enrichedEvents, final EnrichedTablets enrichedTablets) { - super(enrichedEvents, enrichedTablets.getSubscriptionCommitId()); + final List enrichedEvents, + final SubscriptionCommitContext commitContext, + final EnrichedTablets enrichedTablets) { + super(enrichedEvents); + this.enrichedTablets = enrichedTablets; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionEvent.java index 4b1afd0a6227..871a07342066 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionEvent.java @@ -23,21 +23,22 @@ import org.apache.iotdb.commons.subscription.config.SubscriptionConfig; import java.util.List; +import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionCommitContext; public abstract class SubscriptionEvent { private static final long INVALID_TIMESTAMP = -1; private final List enrichedEvents; - private final String subscriptionCommitId; + private final SubscriptionCommitContext commitContext; private long lastPolledTimestamp; private long committedTimestamp; public SubscriptionEvent( - final List enrichedEvents, final String subscriptionCommitId) { + final List enrichedEvents, final SubscriptionCommitContext commitContext) { this.enrichedEvents = enrichedEvents; - this.subscriptionCommitId = subscriptionCommitId; + this.commitContext = commitContext; this.lastPolledTimestamp = INVALID_TIMESTAMP; this.committedTimestamp = INVALID_TIMESTAMP; @@ -53,10 +54,6 @@ public boolean isCommitted() { return committedTimestamp != INVALID_TIMESTAMP; } - public String getSubscriptionCommitId() { - return subscriptionCommitId; - } - public void decreaseReferenceCount() { for (final EnrichedEvent enrichedEvent : enrichedEvents) { enrichedEvent.decreaseReferenceCount(this.getClass().getName(), true); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/TsFileSubscriptionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/TsFileSubscriptionEvent.java index d198e61fb438..4c7c7932e01c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/TsFileSubscriptionEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/TsFileSubscriptionEvent.java @@ -35,4 +35,16 @@ public String getTopicName() { public String getFileName() { return fileName; } + + public long getFileLength() { + return fileLength; + } + + public long getStartWritingOffset() { + return startWritingOffset; + } + + public byte[] getFilePiece() { + return filePiece; + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java index 91c0e856d40a..78d98f7d63c6 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java @@ -41,7 +41,7 @@ import org.apache.iotdb.rpc.TSStatusCode; import org.apache.iotdb.rpc.subscription.config.ConsumerConfig; import org.apache.iotdb.rpc.subscription.exception.SubscriptionException; -import org.apache.iotdb.rpc.subscription.payload.EnrichedTablets; +import org.apache.iotdb.rpc.subscription.payload.common.EnrichedTablets; import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribeCloseReq; import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribeCommitReq; import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribeHandshakeReq; @@ -345,13 +345,7 @@ private TPipeSubscribeResp handlePipeSubscribePollInternal(final PipeSubscribePo .map(event -> (SerializableEnrichedTabletsSubscriptionEvent) event) .collect(Collectors.toList()); - // get subscriptionCommitIds from SerializableEnrichedTabletsSubscriptionEvent - final List subscriptionCommitIds = - enrichedTabletsSubscriptionEvents.stream() - .map(SubscriptionEvent::getSubscriptionCommitId) - .collect(Collectors.toList()); - - // get enrichedTablets with byte buffer from SerializableEnrichedTabletsSubscriptionEvent + // get enriched tablets with byte buffer from SerializableEnrichedTabletsSubscriptionEvent final List> enrichedTabletsWithByteBufferList = enrichedTabletsSubscriptionEvents.stream() .map(event -> new Pair<>(event.getByteBuffer(), event.getEnrichedTablets())) @@ -364,21 +358,13 @@ private TPipeSubscribeResp handlePipeSubscribePollInternal(final PipeSubscribePo .map(event -> (TsFileSubscriptionEvent) event) .collect(Collectors.toList()); + // generate topicNameToTsFileNameMap final Map topicNameToTsFileNameMap = tsFileSubscriptionEvents.stream() .collect( Collectors.toMap( TsFileSubscriptionEvent::getTopicName, TsFileSubscriptionEvent::getFileName)); - // generate response - final TPipeSubscribeResp resp = - PipeSubscribePollResp.toTPipeSubscribeResp( - RpcUtils.SUCCESS_STATUS, enrichedTabletsWithByteBufferList, topicNameToTsFileNameMap); - - // reset byte buffer - enrichedTabletsSubscriptionEvents.forEach( - SerializableEnrichedTabletsSubscriptionEvent::resetByteBuffer); - // check timer if (timer.isExpired()) { LOGGER.warn( @@ -388,10 +374,18 @@ private TPipeSubscribeResp handlePipeSubscribePollInternal(final PipeSubscribePo } LOGGER.info( - "Subscription: consumer {} poll topics {} successfully, commit ids: {}", + "Subscription: consumer {} poll topics {} successfully", consumerConfig, - topicNames, - subscriptionCommitIds); + topicNames); + + // generate response + final TPipeSubscribeResp resp = + PipeSubscribePollResp.toTPipeSubscribeResp( + RpcUtils.SUCCESS_STATUS, enrichedTabletsWithByteBufferList, topicNameToTsFileNameMap); + + // reset byte buffer + enrichedTabletsSubscriptionEvents.forEach( + SerializableEnrichedTabletsSubscriptionEvent::resetByteBuffer); return resp; } From f424311c22c3263576547d1a291333b0fa6d061d Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Thu, 11 Apr 2024 20:13:16 +0800 Subject: [PATCH 07/93] backup --- .../common/SubscriptionPolledRawMessage.java | 7 - .../common/SubscriptionRawMessage.java | 63 +++++++++ .../common/SubscriptionRawMessagePayload.java | 18 +++ .../common/SubscriptionRawMessageType.java | 40 ++++++ .../payload/common/TabletsMessagePayload.java | 39 ++++++ .../request/PipeSubscribeRequestType.java | 2 +- .../response/PipeSubscribePollResp.java | 64 ++------- .../PipeSubscribePollTsFilePieceResp.java | 131 ------------------ .../PipeSubscribePollTsFileSealResp.java | 127 ----------------- .../response/PipeSubscribeResponseType.java | 8 +- 10 files changed, 172 insertions(+), 327 deletions(-) delete mode 100644 iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledRawMessage.java create mode 100644 iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionRawMessage.java create mode 100644 iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionRawMessagePayload.java create mode 100644 iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionRawMessageType.java create mode 100644 iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TabletsMessagePayload.java delete mode 100644 iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollTsFilePieceResp.java delete mode 100644 iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollTsFileSealResp.java diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledRawMessage.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledRawMessage.java deleted file mode 100644 index e235da8b7977..000000000000 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledRawMessage.java +++ /dev/null @@ -1,7 +0,0 @@ -package org.apache.iotdb.rpc.subscription.payload.common; - -public class SubscriptionPolledRawMessage { - - private final transient SubscriptionCommitContext commitContext; - -} diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionRawMessage.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionRawMessage.java new file mode 100644 index 000000000000..fca7bde08623 --- /dev/null +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionRawMessage.java @@ -0,0 +1,63 @@ +package org.apache.iotdb.rpc.subscription.payload.common; + +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.List; +import org.apache.iotdb.tsfile.utils.PublicBAOS; +import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; +import org.apache.iotdb.tsfile.write.record.Tablet; + +public class SubscriptionRawMessage { + + private final transient short messageType; + + private final transient SubscriptionRawMessagePayload messagePayload; + + private final transient SubscriptionCommitContext commitContext; + + public SubscriptionRawMessage( + short messageType, + SubscriptionRawMessagePayload messagePayload, + SubscriptionCommitContext commitContext) { + this.messageType = messageType; + this.messagePayload = messagePayload; + this.commitContext = commitContext; + } + + /////////////////////////////// de/ser /////////////////////////////// + + public static ByteBuffer serialize(SubscriptionRawMessage message) throws IOException { + try (final PublicBAOS byteArrayOutputStream = new PublicBAOS(); + final DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream)) { + message.serialize(outputStream); + return ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size()); + } + } + + private void serialize(final DataOutputStream stream) throws IOException { + ReadWriteIOUtils.write(messageType, stream); + messagePayload.serialize(stream); + commitContext.serialize(stream); + } + + public static SubscriptionRawMessage deserialize(final ByteBuffer buffer) { + final short messageType = ReadWriteIOUtils.readShort(buffer); + final SubscriptionRawMessagePayload messagePayload; + if (SubscriptionRawMessageType.isValidatedMessageType(messageType)) { + switch (SubscriptionRawMessageType.valueOf(messageType)) { + case TABLETS: + messagePayload = new TabletsMessagePayload().deserialize(buffer); + break; + default: + messagePayload = null; + } + } else { + messagePayload = null; + } + + final SubscriptionCommitContext commitContext = SubscriptionCommitContext.deserialize(buffer); + return new SubscriptionRawMessage(messageType, messagePayload, commitContext); + } +} diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionRawMessagePayload.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionRawMessagePayload.java new file mode 100644 index 000000000000..f948b8340716 --- /dev/null +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionRawMessagePayload.java @@ -0,0 +1,18 @@ +package org.apache.iotdb.rpc.subscription.payload.common; + +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; + +public interface SubscriptionRawMessagePayload { + + void serialize(final DataOutputStream stream) throws IOException; + + SubscriptionRawMessagePayload deserialize(final ByteBuffer buffer); + + boolean equals(final Object obj); + + int hashCode(); + + String toString(); +} diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionRawMessageType.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionRawMessageType.java new file mode 100644 index 000000000000..8d334379040c --- /dev/null +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionRawMessageType.java @@ -0,0 +1,40 @@ +package org.apache.iotdb.rpc.subscription.payload.common; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; + +public enum SubscriptionRawMessageType { + TABLETS((short) 0), + + TS_FILE_PREPARE((short) 1), + TS_FILE_PIECE((short) 2), + TS_FILE_SEAL((short) 3), + ; + + private final short type; + + SubscriptionRawMessageType(short type) { + this.type = type; + } + + public short getType() { + return type; + } + + private static final Map TYPE_MAP = + Arrays.stream(SubscriptionRawMessageType.values()) + .collect( + HashMap::new, + (typeMap, messageType) -> typeMap.put(messageType.getType(), + messageType), + HashMap::putAll); + + public static boolean isValidatedMessageType(short type) { + return TYPE_MAP.containsKey(type); + } + + public static SubscriptionRawMessageType valueOf(short type) { + return TYPE_MAP.get(type); + } +} diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TabletsMessagePayload.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TabletsMessagePayload.java new file mode 100644 index 000000000000..b05601cbd1af --- /dev/null +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TabletsMessagePayload.java @@ -0,0 +1,39 @@ +package org.apache.iotdb.rpc.subscription.payload.common; + +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.List; +import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; +import org.apache.iotdb.tsfile.write.record.Tablet; + +public class TabletsMessagePayload implements SubscriptionRawMessagePayload { + + private transient List tablets = new ArrayList<>(); + + public TabletsMessagePayload() {} + + public TabletsMessagePayload(List tablets) { + this.tablets = tablets; + } + + @Override + public void serialize(DataOutputStream stream) throws IOException { + ReadWriteIOUtils.write(tablets.size(), stream); + for (final Tablet tablet : tablets) { + tablet.serialize(stream); + } + } + + @Override + public SubscriptionRawMessagePayload deserialize(ByteBuffer buffer) { + final List tablets = new ArrayList<>(); + final int size = ReadWriteIOUtils.readInt(buffer); + for (int i = 0; i < size; ++i) { + tablets.add(Tablet.deserialize(buffer)); + } + this.tablets = tablets; + return this; + } +} diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribeRequestType.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribeRequestType.java index 96ea90d6d8fc..60fefd7ecb82 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribeRequestType.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribeRequestType.java @@ -48,7 +48,7 @@ public short getType() { Arrays.stream(PipeSubscribeRequestType.values()) .collect( HashMap::new, - (typeMap, pipeRequestType) -> typeMap.put(pipeRequestType.getType(), pipeRequestType), + (typeMap, requestType) -> typeMap.put(requestType.getType(), requestType), HashMap::putAll); public static boolean isValidatedRequestType(short type) { diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollResp.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollResp.java index 675b7dd8d7ca..272d91db6990 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollResp.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollResp.java @@ -23,6 +23,7 @@ import org.apache.iotdb.rpc.RpcUtils; import org.apache.iotdb.rpc.TSStatusCode; import org.apache.iotdb.rpc.subscription.payload.common.EnrichedTablets; +import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionRawMessage; import org.apache.iotdb.service.rpc.thrift.TPipeSubscribeResp; import org.apache.iotdb.tsfile.utils.Pair; import org.apache.iotdb.tsfile.utils.PublicBAOS; @@ -40,19 +41,7 @@ public class PipeSubscribePollResp extends TPipeSubscribeResp { - private transient List<> - - private transient List enrichedTabletsList = new ArrayList<>(); - - private transient Map topicNameToTsFileNameMap = new HashMap<>(); - - public List getEnrichedTabletsList() { - return enrichedTabletsList; - } - - public Map getTopicNameToTsFileNameMap() { - return topicNameToTsFileNameMap; - } + private transient List messages = new ArrayList<>(); /////////////////////////////// Thrift /////////////////////////////// @@ -61,23 +50,19 @@ public Map getTopicNameToTsFileNameMap() { * server. */ public static PipeSubscribePollResp toTPipeSubscribeResp( - TSStatus status, - List> enrichedTabletsWithByteBufferList, - Map topicNameToTsFileNameMap) { + TSStatus status, List messages) { final PipeSubscribePollResp resp = new PipeSubscribePollResp(); - resp.enrichedTabletsList = - enrichedTabletsWithByteBufferList.stream().map(Pair::getRight).collect(Collectors.toList()); - resp.topicNameToTsFileNameMap = topicNameToTsFileNameMap; + resp.messages = messages; resp.status = status; resp.version = PipeSubscribeResponseVersion.VERSION_1.getVersion(); - resp.type = PipeSubscribeResponseType.POLL.getType(); + resp.type = PipeSubscribeResponseType.ACK.getType(); try { resp.body = new ArrayList<>(); - resp.body.add(serializeTopicNameToTsFileNameMap(topicNameToTsFileNameMap)); - resp.body.addAll( - serializeEnrichedTabletsWithByteBufferList(enrichedTabletsWithByteBufferList)); + for (final SubscriptionRawMessage message: messages) { + resp.body.add(SubscriptionRawMessage.serialize(message)); + } } catch (IOException e) { resp.status = RpcUtils.getStatus(TSStatusCode.SUBSCRIPTION_POLL_ERROR, e.getMessage()); } @@ -89,16 +74,10 @@ public static PipeSubscribePollResp toTPipeSubscribeResp( public static PipeSubscribePollResp fromTPipeSubscribeResp(TPipeSubscribeResp pollResp) { final PipeSubscribePollResp resp = new PipeSubscribePollResp(); - boolean isFirst = true; if (Objects.nonNull(pollResp.body)) { for (final ByteBuffer byteBuffer : pollResp.body) { if (Objects.nonNull(byteBuffer) && byteBuffer.hasRemaining()) { - if (isFirst) { - resp.topicNameToTsFileNameMap = deserializeTopicNameToTsFileNameMap(byteBuffer); - isFirst = false; - } else { - resp.enrichedTabletsList.add(EnrichedTablets.deserialize(byteBuffer)); - } + resp.messages.add(SubscriptionRawMessage.deserialize(byteBuffer)); } } } @@ -137,31 +116,6 @@ public static ByteBuffer serializeEnrichedTablets(EnrichedTablets enrichedTablet } } - public static ByteBuffer serializeTopicNameToTsFileNameMap( - Map topicNameToTsFileNameMap) throws IOException { - try (final PublicBAOS byteArrayOutputStream = new PublicBAOS(); - final DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream)) { - ReadWriteIOUtils.write(topicNameToTsFileNameMap.size(), outputStream); - for (final Map.Entry topicNameToTsFileName : - topicNameToTsFileNameMap.entrySet()) { - ReadWriteIOUtils.write(topicNameToTsFileName.getKey(), outputStream); - ReadWriteIOUtils.write(topicNameToTsFileName.getValue(), outputStream); - } - return ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size()); - } - } - - public static Map deserializeTopicNameToTsFileNameMap(ByteBuffer buffer) { - final Map topicNameToTsFileNameMap = new HashMap<>(); - final int size = ReadWriteIOUtils.readInt(buffer); - for (int i = 0; i < size; i++) { - final String topicName = ReadWriteIOUtils.readString(buffer); - final String tsFileName = ReadWriteIOUtils.readString(buffer); - topicNameToTsFileNameMap.put(topicName, tsFileName); - } - return topicNameToTsFileNameMap; - } - /////////////////////////////// Object /////////////////////////////// @Override diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollTsFilePieceResp.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollTsFilePieceResp.java deleted file mode 100644 index 11135a48c4ad..000000000000 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollTsFilePieceResp.java +++ /dev/null @@ -1,131 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iotdb.rpc.subscription.payload.response; - -import org.apache.iotdb.common.rpc.thrift.TSStatus; -import org.apache.iotdb.rpc.RpcUtils; -import org.apache.iotdb.rpc.TSStatusCode; -import org.apache.iotdb.service.rpc.thrift.TPipeSubscribeResp; -import org.apache.iotdb.tsfile.utils.Binary; -import org.apache.iotdb.tsfile.utils.PublicBAOS; -import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; - -import java.io.DataOutputStream; -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.Arrays; -import java.util.Collections; -import java.util.Objects; - -public class PipeSubscribePollTsFilePieceResp extends TPipeSubscribeResp { - - private transient String fileName; - private transient long startWritingOffset; - private transient byte[] filePiece; - - public final String getFileName() { - return fileName; - } - - public final long getStartWritingOffset() { - return startWritingOffset; - } - - public final byte[] getFilePiece() { - return filePiece; - } - - /////////////////////////////// Thrift /////////////////////////////// - - public static PipeSubscribePollTsFilePieceResp toTPipeSubscribeResp( - TSStatus status, String fileName, long startWritingOffset, byte[] filePiece) { - final PipeSubscribePollTsFilePieceResp resp = new PipeSubscribePollTsFilePieceResp(); - - resp.fileName = fileName; - resp.startWritingOffset = startWritingOffset; - resp.filePiece = filePiece; - - resp.status = status; - resp.version = PipeSubscribeResponseVersion.VERSION_1.getVersion(); - resp.type = PipeSubscribeResponseType.POLL_TS_FILE_PIECE.getType(); - try (final PublicBAOS byteArrayOutputStream = new PublicBAOS(); - final DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream)) { - ReadWriteIOUtils.write(fileName, outputStream); - ReadWriteIOUtils.write(startWritingOffset, outputStream); - ReadWriteIOUtils.write(new Binary(filePiece), outputStream); - resp.body = - Collections.singletonList( - ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size())); - } catch (final IOException e) { - resp.status = - RpcUtils.getStatus(TSStatusCode.SUBSCRIPTION_POLL_TS_FILE_ERROR, e.getMessage()); - } - - return resp; - } - - public static PipeSubscribePollTsFilePieceResp fromTPipeSubscribeResp( - TPipeSubscribeResp pollTsFilePieceResp) { - final PipeSubscribePollTsFilePieceResp resp = new PipeSubscribePollTsFilePieceResp(); - - if (Objects.nonNull(pollTsFilePieceResp.body)) { - for (final ByteBuffer byteBuffer : pollTsFilePieceResp.body) { - if (Objects.nonNull(byteBuffer) && byteBuffer.hasRemaining()) { - resp.fileName = ReadWriteIOUtils.readString(byteBuffer); - resp.startWritingOffset = ReadWriteIOUtils.readLong(byteBuffer); - resp.filePiece = ReadWriteIOUtils.readBinary(byteBuffer).getValues(); - } - } - } - - resp.status = pollTsFilePieceResp.status; - resp.version = pollTsFilePieceResp.version; - resp.type = pollTsFilePieceResp.type; - resp.body = pollTsFilePieceResp.body; - - return resp; - } - - /////////////////////////////// Object /////////////////////////////// - - @Override - public boolean equals(Object obj) { - if (this == obj) { - return true; - } - if (obj == null || getClass() != obj.getClass()) { - return false; - } - final PipeSubscribePollTsFilePieceResp that = (PipeSubscribePollTsFilePieceResp) obj; - return Objects.equals(this.fileName, that.fileName) - && this.startWritingOffset == that.startWritingOffset - && Arrays.equals(this.filePiece, that.filePiece) - && Objects.equals(this.status, that.status) - && this.version == that.version - && this.type == that.type - && Objects.equals(this.body, that.body); - } - - @Override - public int hashCode() { - return Objects.hash( - fileName, startWritingOffset, Arrays.hashCode(filePiece), status, version, type, body); - } -} diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollTsFileSealResp.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollTsFileSealResp.java deleted file mode 100644 index c41a475c9623..000000000000 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollTsFileSealResp.java +++ /dev/null @@ -1,127 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iotdb.rpc.subscription.payload.response; - -import org.apache.iotdb.common.rpc.thrift.TSStatus; -import org.apache.iotdb.rpc.RpcUtils; -import org.apache.iotdb.rpc.TSStatusCode; -import org.apache.iotdb.service.rpc.thrift.TPipeSubscribeResp; -import org.apache.iotdb.tsfile.utils.PublicBAOS; -import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; - -import java.io.DataOutputStream; -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.Collections; -import java.util.Objects; - -public class PipeSubscribePollTsFileSealResp extends TPipeSubscribeResp { - - private transient String fileName; - private transient long fileLength; - private transient String subscriptionCommitId; - - public final String getFileName() { - return fileName; - } - - public final long getFileLength() { - return fileLength; - } - - public String getSubscriptionCommitId() { - return subscriptionCommitId; - } - - /////////////////////////////// Thrift /////////////////////////////// - - public static PipeSubscribePollTsFileSealResp toTPipeSubscribeResp( - TSStatus status, String fileName, long fileLength, String subscriptionCommitId) { - final PipeSubscribePollTsFileSealResp resp = new PipeSubscribePollTsFileSealResp(); - - resp.fileName = fileName; - resp.fileLength = fileLength; - resp.subscriptionCommitId = subscriptionCommitId; - - resp.status = status; - resp.version = PipeSubscribeResponseVersion.VERSION_1.getVersion(); - resp.type = PipeSubscribeResponseType.POLL_TS_FILE_SEAL.getType(); - try (final PublicBAOS byteArrayOutputStream = new PublicBAOS(); - final DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream)) { - ReadWriteIOUtils.write(fileName, outputStream); - ReadWriteIOUtils.write(fileLength, outputStream); - ReadWriteIOUtils.write(subscriptionCommitId, outputStream); - resp.body = - Collections.singletonList( - ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size())); - } catch (final IOException e) { - resp.status = - RpcUtils.getStatus(TSStatusCode.SUBSCRIPTION_POLL_TS_FILE_ERROR, e.getMessage()); - } - - return resp; - } - - public static PipeSubscribePollTsFileSealResp fromTPipeSubscribeResp( - TPipeSubscribeResp pollTsFileSealResp) { - final PipeSubscribePollTsFileSealResp resp = new PipeSubscribePollTsFileSealResp(); - - if (Objects.nonNull(pollTsFileSealResp.body)) { - for (final ByteBuffer byteBuffer : pollTsFileSealResp.body) { - if (Objects.nonNull(byteBuffer) && byteBuffer.hasRemaining()) { - resp.fileName = ReadWriteIOUtils.readString(byteBuffer); - resp.fileLength = ReadWriteIOUtils.readLong(byteBuffer); - resp.subscriptionCommitId = ReadWriteIOUtils.readString(byteBuffer); - } - } - } - - resp.status = pollTsFileSealResp.status; - resp.version = pollTsFileSealResp.version; - resp.type = pollTsFileSealResp.type; - resp.body = pollTsFileSealResp.body; - - return resp; - } - - /////////////////////////////// Object /////////////////////////////// - - public boolean equals(Object obj) { - if (this == obj) { - return true; - } - if (obj == null || getClass() != obj.getClass()) { - return false; - } - final PipeSubscribePollTsFileSealResp that = (PipeSubscribePollTsFileSealResp) obj; - return Objects.equals(this.fileName, that.fileName) - && fileLength == that.fileLength - && Objects.equals(this.subscriptionCommitId, that.subscriptionCommitId) - && Objects.equals(this.status, that.status) - && this.version == that.version - && this.type == that.type - && Objects.equals(this.body, that.body); - } - - @Override - public int hashCode() { - return Objects.hash(fileName, fileLength, subscriptionCommitId, status, version, type, body); - } -} diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribeResponseType.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribeResponseType.java index 55a93e35dfa3..6b9f4bfe9713 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribeResponseType.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribeResponseType.java @@ -25,10 +25,6 @@ public enum PipeSubscribeResponseType { ACK((short) 0), - - POLL((short) 1), - POLL_TS_FILE_PIECE((short) 1), - POLL_TS_FILE_SEAL((short) 1), ; private final short type; @@ -45,10 +41,10 @@ public short getType() { Arrays.stream(PipeSubscribeResponseType.values()) .collect( HashMap::new, - (typeMap, pipeRequestType) -> typeMap.put(pipeRequestType.getType(), pipeRequestType), + (typeMap, responseType) -> typeMap.put(responseType.getType(), responseType), HashMap::putAll); - public static boolean isValidatedRequestType(short type) { + public static boolean isValidatedResponseType(short type) { return TYPE_MAP.containsKey(type); } From 3de85c92f6ce44031ff6a39268f4297b727f5385 Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Fri, 12 Apr 2024 00:59:15 +0800 Subject: [PATCH 08/93] backup --- .../payload/common/EnrichedTablets.java | 3 +- .../payload/common/EnrichedTsFileInfo.java | 5 -- .../common/SubscriptionCommitContext.java | 33 ++++---- .../common/SubscriptionRawMessage.java | 23 +++-- .../common/SubscriptionRawMessageType.java | 5 +- .../payload/common/TabletsMessagePayload.java | 44 +++++++++- .../common/TsFileInfoMessagePayload.java | 29 +++++++ .../request/PipeSubscribeCommitReq.java | 30 +++---- .../response/PipeSubscribePollResp.java | 15 ++-- .../subscription/SubscriptionMessage.java | 4 +- .../SubscriptionSessionConnection.java | 9 +- .../agent/SubscriptionBrokerAgent.java | 6 +- .../broker/SubscriptionBroker.java | 12 +-- .../broker/SubscriptionPrefetchingQueue.java | 27 +++--- ... SubscriptionPrefetchingTabletsQueue.java} | 33 ++++---- .../SubscriptionPrefetchingTsFileQueue.java | 23 +++-- ...zableEnrichedTabletsSubscriptionEvent.java | 84 ------------------- .../subscription/event/SubscriptionEvent.java | 14 ++-- .../event/TsFileSubscriptionEvent.java | 50 ----------- .../receiver/SubscriptionReceiverV1.java | 69 +++++++-------- 20 files changed, 227 insertions(+), 291 deletions(-) delete mode 100644 iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/EnrichedTsFileInfo.java create mode 100644 iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileInfoMessagePayload.java rename iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/{SubscriptionPrefetchingEnrichedTabletsQueue.java => SubscriptionPrefetchingTabletsQueue.java} (84%) delete mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SerializableEnrichedTabletsSubscriptionEvent.java delete mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/TsFileSubscriptionEvent.java diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/EnrichedTablets.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/EnrichedTablets.java index ecfeb02a3ec0..09eaa359f593 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/EnrichedTablets.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/EnrichedTablets.java @@ -38,7 +38,8 @@ public List getTablets() { return tablets; } - public EnrichedTablets(final SubscriptionCommitContext commitContext, final List tablets) { + public EnrichedTablets( + final SubscriptionCommitContext commitContext, final List tablets) { this.commitContext = commitContext; this.tablets = tablets; } diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/EnrichedTsFileInfo.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/EnrichedTsFileInfo.java deleted file mode 100644 index 68a1b3b13cf0..000000000000 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/EnrichedTsFileInfo.java +++ /dev/null @@ -1,5 +0,0 @@ -package org.apache.iotdb.rpc.subscription.payload.common; - -public class EnrichedTsFileInfo { - -} diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionCommitContext.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionCommitContext.java index 3188cf25d657..f2c9e61bf03f 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionCommitContext.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionCommitContext.java @@ -19,26 +19,31 @@ package org.apache.iotdb.rpc.subscription.payload.common; +import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; + import java.io.DataOutputStream; import java.io.IOException; import java.nio.ByteBuffer; import java.util.Objects; -import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; public class SubscriptionCommitContext { private final int dataNodeId; - + private final int rebootTimes; - + private final String topicName; - + private final String consumerGroupId; - + private final long commitId; - public SubscriptionCommitContext(final int dataNodeId, final int rebootTimes, final String topicName, - final String consumerGroupId, final long commitId) { + public SubscriptionCommitContext( + final int dataNodeId, + final int rebootTimes, + final String topicName, + final String consumerGroupId, + final long commitId) { this.dataNodeId = dataNodeId; this.rebootTimes = rebootTimes; this.topicName = topicName; @@ -82,7 +87,8 @@ public static SubscriptionCommitContext deserialize(final ByteBuffer buffer) { final String topicName = ReadWriteIOUtils.readString(buffer); final String consumerGroupId = ReadWriteIOUtils.readString(buffer); final long commitId = ReadWriteIOUtils.readLong(buffer); - return new SubscriptionCommitContext(dataNodeId, rebootTimes, topicName, consumerGroupId, commitId); + return new SubscriptionCommitContext( + dataNodeId, rebootTimes, topicName, consumerGroupId, commitId); } /////////////////////////////// Object /////////////////////////////// @@ -96,10 +102,9 @@ public boolean equals(final Object obj) { return false; } final SubscriptionCommitContext that = (SubscriptionCommitContext) obj; - return - this.dataNodeId == that.dataNodeId && - this.rebootTimes == that.rebootTimes && - Objects.equals(this.topicName, that.topicName) + return this.dataNodeId == that.dataNodeId + && this.rebootTimes == that.rebootTimes + && Objects.equals(this.topicName, that.topicName) && Objects.equals(this.consumerGroupId, that.consumerGroupId) && Objects.equals(this.commitId, that.commitId); } @@ -108,14 +113,14 @@ public boolean equals(final Object obj) { public int hashCode() { return Objects.hash(dataNodeId, rebootTimes, topicName, consumerGroupId, commitId); } - + @Override public String toString() { return "SubscriptionCommitContext{dataNodeId=" + dataNodeId + ", rebootTimes=" + rebootTimes - + ", topicName=" + + ", topicName=" + topicName + ", consumerGroupId=" + consumerGroupId diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionRawMessage.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionRawMessage.java index fca7bde08623..25aa41444f25 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionRawMessage.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionRawMessage.java @@ -1,13 +1,11 @@ package org.apache.iotdb.rpc.subscription.payload.common; +import org.apache.iotdb.tsfile.utils.PublicBAOS; +import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; + import java.io.DataOutputStream; import java.io.IOException; import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.List; -import org.apache.iotdb.tsfile.utils.PublicBAOS; -import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; -import org.apache.iotdb.tsfile.write.record.Tablet; public class SubscriptionRawMessage { @@ -26,6 +24,18 @@ public SubscriptionRawMessage( this.commitContext = commitContext; } + public short getMessageType() { + return messageType; + } + + public SubscriptionRawMessagePayload getMessagePayload() { + return messagePayload; + } + + public SubscriptionCommitContext getCommitContext() { + return commitContext; + } + /////////////////////////////// de/ser /////////////////////////////// public static ByteBuffer serialize(SubscriptionRawMessage message) throws IOException { @@ -50,6 +60,9 @@ public static SubscriptionRawMessage deserialize(final ByteBuffer buffer) { case TABLETS: messagePayload = new TabletsMessagePayload().deserialize(buffer); break; + case TS_FILE_INFO: + messagePayload = new TsFileInfoMessagePayload().deserialize(buffer); + break; default: messagePayload = null; } diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionRawMessageType.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionRawMessageType.java index 8d334379040c..b4700e12d03b 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionRawMessageType.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionRawMessageType.java @@ -7,7 +7,7 @@ public enum SubscriptionRawMessageType { TABLETS((short) 0), - TS_FILE_PREPARE((short) 1), + TS_FILE_INFO((short) 1), TS_FILE_PIECE((short) 2), TS_FILE_SEAL((short) 3), ; @@ -26,8 +26,7 @@ public short getType() { Arrays.stream(SubscriptionRawMessageType.values()) .collect( HashMap::new, - (typeMap, messageType) -> typeMap.put(messageType.getType(), - messageType), + (typeMap, messageType) -> typeMap.put(messageType.getType(), messageType), HashMap::putAll); public static boolean isValidatedMessageType(short type) { diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TabletsMessagePayload.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TabletsMessagePayload.java index b05601cbd1af..3dc112dd3209 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TabletsMessagePayload.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TabletsMessagePayload.java @@ -1,16 +1,26 @@ package org.apache.iotdb.rpc.subscription.payload.common; +import org.apache.iotdb.tsfile.utils.PublicBAOS; +import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; +import org.apache.iotdb.tsfile.write.record.Tablet; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.io.DataOutputStream; import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.List; -import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; -import org.apache.iotdb.tsfile.write.record.Tablet; +import java.util.Objects; public class TabletsMessagePayload implements SubscriptionRawMessagePayload { - private transient List tablets = new ArrayList<>(); + private static final Logger LOGGER = LoggerFactory.getLogger(TabletsMessagePayload.class); + + protected transient List tablets = new ArrayList<>(); + + private ByteBuffer byteBuffer; // serialized tablets public TabletsMessagePayload() {} @@ -36,4 +46,32 @@ public SubscriptionRawMessagePayload deserialize(ByteBuffer buffer) { this.tablets = tablets; return this; } + + //////////////////////////// serialization //////////////////////////// + + /** @return true -> byte buffer is not null */ + public boolean trySerialize() { + if (Objects.isNull(byteBuffer)) { + try { + try (final PublicBAOS byteArrayOutputStream = new PublicBAOS(); + final DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream)) { + serialize(outputStream); + byteBuffer = + ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size()); + } + return true; + } catch (final IOException e) { + LOGGER.warn( + "Subscription: something unexpected happened when serializing Tablets, exception is {}", + e.getMessage()); + } + return false; + } + return true; + } + + public void resetByteBuffer() { + // maybe friendly for gc + byteBuffer = null; + } } diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileInfoMessagePayload.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileInfoMessagePayload.java new file mode 100644 index 000000000000..b8ce41dcef74 --- /dev/null +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileInfoMessagePayload.java @@ -0,0 +1,29 @@ +package org.apache.iotdb.rpc.subscription.payload.common; + +import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; + +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; + +public class TsFileInfoMessagePayload implements SubscriptionRawMessagePayload { + + protected transient String fileName; + + public TsFileInfoMessagePayload() {} + + public TsFileInfoMessagePayload(String fileName) { + this.fileName = fileName; + } + + @Override + public void serialize(DataOutputStream stream) throws IOException { + ReadWriteIOUtils.write(fileName, stream); + } + + @Override + public SubscriptionRawMessagePayload deserialize(ByteBuffer buffer) { + this.fileName = ReadWriteIOUtils.readString(buffer); + return this; + } +} diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribeCommitReq.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribeCommitReq.java index 89c844c7ebd5..5e51377c88ee 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribeCommitReq.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribeCommitReq.java @@ -19,6 +19,7 @@ package org.apache.iotdb.rpc.subscription.payload.request; +import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionCommitContext; import org.apache.iotdb.service.rpc.thrift.TPipeSubscribeReq; import org.apache.iotdb.tsfile.utils.PublicBAOS; import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; @@ -26,17 +27,16 @@ import java.io.DataOutputStream; import java.io.IOException; import java.nio.ByteBuffer; -import java.util.HashMap; +import java.util.ArrayList; import java.util.List; -import java.util.Map; import java.util.Objects; public class PipeSubscribeCommitReq extends TPipeSubscribeReq { - private transient Map> topicNameToSubscriptionCommitIds = new HashMap<>(); + private transient List commitContexts = new ArrayList<>(); - public Map> getTopicNameToSubscriptionCommitIds() { - return topicNameToSubscriptionCommitIds; + public List getCommitContexts() { + return commitContexts; } /////////////////////////////// Thrift /////////////////////////////// @@ -46,19 +46,18 @@ public Map> getTopicNameToSubscriptionCommitIds() { * client. */ public static PipeSubscribeCommitReq toTPipeSubscribeReq( - Map> topicNameToSubscriptionCommitIds) throws IOException { + List commitContexts) throws IOException { final PipeSubscribeCommitReq req = new PipeSubscribeCommitReq(); - req.topicNameToSubscriptionCommitIds = topicNameToSubscriptionCommitIds; + req.commitContexts = commitContexts; req.version = PipeSubscribeRequestVersion.VERSION_1.getVersion(); req.type = PipeSubscribeRequestType.COMMIT.getType(); try (final PublicBAOS byteArrayOutputStream = new PublicBAOS(); final DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream)) { - ReadWriteIOUtils.write(topicNameToSubscriptionCommitIds.size(), outputStream); - for (Map.Entry> entry : topicNameToSubscriptionCommitIds.entrySet()) { - ReadWriteIOUtils.write(entry.getKey(), outputStream); - ReadWriteIOUtils.writeStringList(entry.getValue(), outputStream); + ReadWriteIOUtils.write(commitContexts.size(), outputStream); + for (final SubscriptionCommitContext commitContext : commitContexts) { + commitContext.serialize(outputStream); } req.body = ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size()); } @@ -73,9 +72,7 @@ public static PipeSubscribeCommitReq fromTPipeSubscribeReq(TPipeSubscribeReq com if (Objects.nonNull(commitReq.body) && commitReq.body.hasRemaining()) { int size = ReadWriteIOUtils.readInt(commitReq.body); for (int i = 0; i < size; ++i) { - String topicName = ReadWriteIOUtils.readString(commitReq.body); - List subscriptionCommitIds = ReadWriteIOUtils.readStringList(commitReq.body); - req.topicNameToSubscriptionCommitIds.put(topicName, subscriptionCommitIds); + req.commitContexts.add(SubscriptionCommitContext.deserialize(commitReq.body)); } } @@ -97,8 +94,7 @@ public boolean equals(Object obj) { return false; } PipeSubscribeCommitReq that = (PipeSubscribeCommitReq) obj; - return Objects.equals( - this.topicNameToSubscriptionCommitIds, that.topicNameToSubscriptionCommitIds) + return Objects.equals(this.commitContexts, that.commitContexts) && this.version == that.version && this.type == that.type && Objects.equals(this.body, that.body); @@ -106,6 +102,6 @@ public boolean equals(Object obj) { @Override public int hashCode() { - return Objects.hash(topicNameToSubscriptionCommitIds, version, type, body); + return Objects.hash(commitContexts, version, type, body); } } diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollResp.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollResp.java index 272d91db6990..c05e1ebba539 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollResp.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollResp.java @@ -27,22 +27,22 @@ import org.apache.iotdb.service.rpc.thrift.TPipeSubscribeResp; import org.apache.iotdb.tsfile.utils.Pair; import org.apache.iotdb.tsfile.utils.PublicBAOS; -import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; import java.io.DataOutputStream; import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; -import java.util.HashMap; import java.util.List; -import java.util.Map; import java.util.Objects; -import java.util.stream.Collectors; public class PipeSubscribePollResp extends TPipeSubscribeResp { private transient List messages = new ArrayList<>(); + public List getMessages() { + return messages; + } + /////////////////////////////// Thrift /////////////////////////////// /** @@ -60,7 +60,7 @@ public static PipeSubscribePollResp toTPipeSubscribeResp( resp.type = PipeSubscribeResponseType.ACK.getType(); try { resp.body = new ArrayList<>(); - for (final SubscriptionRawMessage message: messages) { + for (final SubscriptionRawMessage message : messages) { resp.body.add(SubscriptionRawMessage.serialize(message)); } } catch (IOException e) { @@ -127,8 +127,7 @@ public boolean equals(Object obj) { return false; } PipeSubscribePollResp that = (PipeSubscribePollResp) obj; - return Objects.equals(this.enrichedTabletsList, that.enrichedTabletsList) - && Objects.equals(this.topicNameToTsFileNameMap, that.topicNameToTsFileNameMap) + return Objects.equals(this.messages, that.messages) && Objects.equals(this.status, that.status) && this.version == that.version && this.type == that.type @@ -137,6 +136,6 @@ public boolean equals(Object obj) { @Override public int hashCode() { - return Objects.hash(enrichedTabletsList, topicNameToTsFileNameMap, status, version, type, body); + return Objects.hash(messages, status, version, type, body); } } diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionMessage.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionMessage.java index ce7715966790..6ef7c781e004 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionMessage.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionMessage.java @@ -37,9 +37,7 @@ public SubscriptionMessage(EnrichedTablets tablets) { this.subscriptionCommitId = tablets.getSubscriptionCommitId(); } - public SubscriptionMessage(String topicName, String fileName) { - - } + public SubscriptionMessage(String topicName, String fileName) {} public String getTopicName() { return topicName; diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionSessionConnection.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionSessionConnection.java index 11cc600fa3b3..87211b15b9b4 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionSessionConnection.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionSessionConnection.java @@ -25,7 +25,7 @@ import org.apache.iotdb.rpc.RpcUtils; import org.apache.iotdb.rpc.StatementExecutionException; import org.apache.iotdb.rpc.subscription.config.ConsumerConfig; -import org.apache.iotdb.rpc.subscription.payload.common.EnrichedTablets; +import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionRawMessage; import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribeCloseReq; import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribeCommitReq; import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribeHandshakeReq; @@ -124,17 +124,18 @@ public void unsubscribe(Set topicNames) RpcUtils.verifySuccess(resp.status); } - public List poll(Set topicNames) throws TException, IOException, StatementExecutionException { + public List poll(Set topicNames) + throws TException, IOException, StatementExecutionException { return poll(topicNames, 0); } - public List poll(Set topicNames, long timeoutMs) + public List poll(Set topicNames, long timeoutMs) throws TException, IOException, StatementExecutionException { TPipeSubscribeResp resp = client.pipeSubscribe(PipeSubscribePollReq.toTPipeSubscribeReq(topicNames, timeoutMs)); RpcUtils.verifySuccess(resp.status); PipeSubscribePollResp pollResp = PipeSubscribePollResp.fromTPipeSubscribeResp(resp); - return pollResp.getEnrichedTabletsList(); + return pollResp.getMessages(); } public void commitSync(Map> topicNameToSubscriptionCommitIds) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionBrokerAgent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionBrokerAgent.java index 4e15ece6dc72..452b86b7a231 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionBrokerAgent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionBrokerAgent.java @@ -24,6 +24,7 @@ import org.apache.iotdb.db.subscription.task.subtask.SubscriptionConnectorSubtask; import org.apache.iotdb.db.subscription.timer.SubscriptionPollTimer; import org.apache.iotdb.rpc.subscription.config.ConsumerConfig; +import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionCommitContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -60,8 +61,7 @@ public List poll( } public void commit( - final ConsumerConfig consumerConfig, - final Map> topicNameToSubscriptionCommitIds) { + final ConsumerConfig consumerConfig, final List commitContexts) { final String consumerGroupId = consumerConfig.getConsumerGroupId(); final SubscriptionBroker broker = consumerGroupIdToSubscriptionBroker.get(consumerGroupId); if (Objects.isNull(broker)) { @@ -69,7 +69,7 @@ public void commit( "Subscription: broker bound to consumer group [{}] does not exist", consumerGroupId); return; } - broker.commit(topicNameToSubscriptionCommitIds); + broker.commit(commitContexts); } /////////////////////////////// broker /////////////////////////////// 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 e8929b3cdc14..72d0aa0f392b 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 @@ -25,6 +25,7 @@ import org.apache.iotdb.db.subscription.timer.SubscriptionPollTimer; import org.apache.iotdb.pipe.api.event.Event; import org.apache.iotdb.rpc.subscription.config.TopicConstant; +import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionCommitContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -76,10 +77,9 @@ public List poll( return events; } - public void commit(final Map> topicNameToSubscriptionCommitIds) { - for (final Map.Entry> entry : - topicNameToSubscriptionCommitIds.entrySet()) { - final String topicName = entry.getKey(); + public void commit(final List commitContexts) { + for (final SubscriptionCommitContext commitContext : commitContexts) { + final String topicName = commitContext.getTopicName(); final SubscriptionPrefetchingQueue prefetchingQueue = topicNameToPrefetchingQueue.get(topicName); if (Objects.isNull(prefetchingQueue)) { @@ -87,7 +87,7 @@ public void commit(final Map> topicNameToSubscriptionCommit "Subscription: prefetching queue bound to topic [{}] does not exist", topicName); continue; } - prefetchingQueue.commit(entry.getValue()); + prefetchingQueue.commit(commitContext); } } @@ -110,7 +110,7 @@ public void bindPrefetchingQueue( } else { topicNameToPrefetchingQueue.put( topicName, - new SubscriptionPrefetchingEnrichedTabletsQueue(brokerId, topicName, inputPendingQueue)); + new SubscriptionPrefetchingTabletsQueue(brokerId, topicName, inputPendingQueue)); } } 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 ec0ad772a914..100b7421c800 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 @@ -25,12 +25,11 @@ import org.apache.iotdb.db.subscription.event.SubscriptionEvent; import org.apache.iotdb.db.subscription.timer.SubscriptionPollTimer; import org.apache.iotdb.pipe.api.event.Event; - import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionCommitContext; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.List; import java.util.Map; import java.util.Objects; import java.util.concurrent.ConcurrentHashMap; @@ -64,19 +63,16 @@ public SubscriptionPrefetchingQueue( /////////////////////////////// commit /////////////////////////////// - public void commit(final List subscriptionCommitIds) { - for (final String subscriptionCommitId : subscriptionCommitIds) { - final SubscriptionEvent event = uncommittedEvents.get(subscriptionCommitId); - if (Objects.isNull(event)) { - LOGGER.warn( - "Subscription: subscription commit id [{}] does not exist, it may have been committed or something unexpected happened", - subscriptionCommitId); - continue; - } - event.decreaseReferenceCount(); - event.recordCommittedTimestamp(); - uncommittedEvents.remove(subscriptionCommitId); + public void commit(final SubscriptionCommitContext commitContext) { + final SubscriptionEvent event = uncommittedEvents.get(commitContext); + if (Objects.isNull(event)) { + LOGGER.warn( + "Subscription: subscription commit context [{}] does not exist, it may have been committed or something unexpected happened", + commitContext); } + event.decreaseReferenceCount(); + event.recordCommittedTimestamp(); + uncommittedEvents.remove(commitContext); } protected SubscriptionCommitContext generateSubscriptionCommitContext() { @@ -87,7 +83,6 @@ protected SubscriptionCommitContext generateSubscriptionCommitContext() { PipeAgent.runtime().getRebootTimes(), topicName, brokerId, - subscriptionCommitIdGenerator.getAndIncrement() - ); + subscriptionCommitIdGenerator.getAndIncrement()); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingEnrichedTabletsQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTabletsQueue.java similarity index 84% rename from iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingEnrichedTabletsQueue.java rename to iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTabletsQueue.java index f40e0aef4653..68480a3ca537 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingEnrichedTabletsQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTabletsQueue.java @@ -7,13 +7,14 @@ 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.subscription.event.SerializableEnrichedTabletsSubscriptionEvent; import org.apache.iotdb.db.subscription.event.SubscriptionEvent; import org.apache.iotdb.db.subscription.timer.SubscriptionPollTimer; import org.apache.iotdb.pipe.api.event.Event; import org.apache.iotdb.pipe.api.event.dml.insertion.TabletInsertionEvent; -import org.apache.iotdb.rpc.subscription.payload.common.EnrichedTablets; import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionCommitContext; +import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionRawMessage; +import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionRawMessageType; +import org.apache.iotdb.rpc.subscription.payload.common.TabletsMessagePayload; import org.apache.iotdb.tsfile.write.record.Tablet; import org.slf4j.Logger; @@ -25,14 +26,14 @@ import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.TimeUnit; -public class SubscriptionPrefetchingEnrichedTabletsQueue extends SubscriptionPrefetchingQueue { +public class SubscriptionPrefetchingTabletsQueue extends SubscriptionPrefetchingQueue { private static final Logger LOGGER = - LoggerFactory.getLogger(SubscriptionPrefetchingEnrichedTabletsQueue.class); + LoggerFactory.getLogger(SubscriptionPrefetchingTabletsQueue.class); - private final LinkedBlockingQueue prefetchingQueue; + private final LinkedBlockingQueue prefetchingQueue; - public SubscriptionPrefetchingEnrichedTabletsQueue( + public SubscriptionPrefetchingTabletsQueue( final String brokerId, final String topicName, final BoundedBlockingPendingQueue inputPendingQueue) { @@ -48,7 +49,7 @@ public SubscriptionEvent poll(final SubscriptionPollTimer timer) { // without serializeOnce here } - SerializableEnrichedTabletsSubscriptionEvent currentEvent; + SubscriptionEvent currentEvent; try { while (Objects.nonNull( currentEvent = @@ -133,11 +134,15 @@ private void prefetchOnce(final long limit) { if (!tablets.isEmpty()) { final SubscriptionCommitContext commitContext = generateSubscriptionCommitContext(); - final SerializableEnrichedTabletsSubscriptionEvent enrichedEvent = - new SerializableEnrichedTabletsSubscriptionEvent( - enrichedEvents, commitContext, new EnrichedTablets(commitContext, tablets)); - uncommittedEvents.put(commitContext, enrichedEvent); // before enqueuing the event - prefetchingQueue.add(enrichedEvent); + final SubscriptionEvent subscriptionEvent = + new SubscriptionEvent( + enrichedEvents, + new SubscriptionRawMessage( + SubscriptionRawMessageType.TABLETS.getType(), + new TabletsMessagePayload(tablets), + commitContext)); + uncommittedEvents.put(commitContext, subscriptionEvent); // before enqueuing the event + prefetchingQueue.add(subscriptionEvent); } } @@ -145,7 +150,7 @@ private void serializeOnce() { final long size = prefetchingQueue.size(); long count = 0; - SerializableEnrichedTabletsSubscriptionEvent currentEvent; + SubscriptionEvent currentEvent; try { while (Objects.nonNull( currentEvent = @@ -165,7 +170,7 @@ private void serializeOnce() { // Serialize the uncommitted and pollable event. if (currentEvent.pollable()) { // No need to concern whether serialization is successful. - currentEvent.serialize(); + ((TabletsMessagePayload) currentEvent.getMessage().getMessagePayload()).trySerialize(); } } } catch (final InterruptedException e) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java index 1ae47ba513aa..777c6d0fe049 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java @@ -23,9 +23,12 @@ import org.apache.iotdb.db.pipe.event.UserDefinedEnrichedEvent; import org.apache.iotdb.db.pipe.event.common.tsfile.PipeTsFileInsertionEvent; import org.apache.iotdb.db.subscription.event.SubscriptionEvent; -import org.apache.iotdb.db.subscription.event.TsFileSubscriptionEvent; import org.apache.iotdb.db.subscription.timer.SubscriptionPollTimer; import org.apache.iotdb.pipe.api.event.Event; +import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionCommitContext; +import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionRawMessage; +import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionRawMessageType; +import org.apache.iotdb.rpc.subscription.payload.common.TsFileInfoMessagePayload; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -39,7 +42,7 @@ public class SubscriptionPrefetchingTsFileQueue extends SubscriptionPrefetchingQ private static final Logger LOGGER = LoggerFactory.getLogger(SubscriptionPrefetchingTsFileQueue.class); - private final AtomicReference eventRef; + private final AtomicReference eventRef; public SubscriptionPrefetchingTsFileQueue( final String brokerId, @@ -67,15 +70,17 @@ public SubscriptionEvent poll(final SubscriptionPollTimer timer) { } final PipeTsFileInsertionEvent tsFileInsertionEvent = (PipeTsFileInsertionEvent) event; - final String subscriptionCommitId = generateSubscriptionCommitContext(); - final TsFileSubscriptionEvent subscriptionEvent = - new TsFileSubscriptionEvent( + final SubscriptionCommitContext commitContext = generateSubscriptionCommitContext(); + + final SubscriptionEvent subscriptionEvent = + new SubscriptionEvent( Collections.singletonList(tsFileInsertionEvent), - subscriptionCommitId, - topicName, - tsFileInsertionEvent.getTsFile().getName()); + new SubscriptionRawMessage( + SubscriptionRawMessageType.TS_FILE_INFO.getType(), + new TsFileInfoMessagePayload(tsFileInsertionEvent.getTsFile().getName()), + commitContext)); eventRef.set(subscriptionEvent); - uncommittedEvents.put(subscriptionCommitId, subscriptionEvent); + uncommittedEvents.put(commitContext, subscriptionEvent); return subscriptionEvent; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SerializableEnrichedTabletsSubscriptionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SerializableEnrichedTabletsSubscriptionEvent.java deleted file mode 100644 index 4fd0d044cdbb..000000000000 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SerializableEnrichedTabletsSubscriptionEvent.java +++ /dev/null @@ -1,84 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iotdb.db.subscription.event; - -import org.apache.iotdb.commons.pipe.event.EnrichedEvent; -import org.apache.iotdb.rpc.subscription.payload.common.EnrichedTablets; -import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionCommitContext; -import org.apache.iotdb.rpc.subscription.payload.response.PipeSubscribePollResp; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.List; -import java.util.Objects; - -public class SerializableEnrichedTabletsSubscriptionEvent extends SubscriptionEvent { - - private static final Logger LOGGER = - LoggerFactory.getLogger(SerializableEnrichedTabletsSubscriptionEvent.class); - - private final EnrichedTablets enrichedTablets; - - private ByteBuffer byteBuffer; // serialized EnrichedTablets - - public SerializableEnrichedTabletsSubscriptionEvent( - final List enrichedEvents, - final SubscriptionCommitContext commitContext, - final EnrichedTablets enrichedTablets) { - super(enrichedEvents); - - this.enrichedTablets = enrichedTablets; - } - - //////////////////////////// serialization //////////////////////////// - - public EnrichedTablets getEnrichedTablets() { - return enrichedTablets; - } - - /** @return true -> byte buffer is not null */ - public boolean serialize() { - if (Objects.isNull(byteBuffer)) { - try { - byteBuffer = PipeSubscribePollResp.serializeEnrichedTablets(enrichedTablets); - return true; - } catch (final IOException e) { - LOGGER.warn( - "Subscription: something unexpected happened when serializing EnrichedTablets {}, exception is {}", - byteBuffer, - e.getMessage()); - } - return false; - } - return true; - } - - public ByteBuffer getByteBuffer() { - return byteBuffer; - } - - public void resetByteBuffer() { - // maybe friendly for gc - byteBuffer = null; - } -} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionEvent.java index 871a07342066..29ec5a3d0e76 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionEvent.java @@ -21,29 +21,33 @@ import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.commons.subscription.config.SubscriptionConfig; +import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionRawMessage; import java.util.List; -import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionCommitContext; -public abstract class SubscriptionEvent { +public class SubscriptionEvent { private static final long INVALID_TIMESTAMP = -1; private final List enrichedEvents; - private final SubscriptionCommitContext commitContext; + private final SubscriptionRawMessage message; private long lastPolledTimestamp; private long committedTimestamp; public SubscriptionEvent( - final List enrichedEvents, final SubscriptionCommitContext commitContext) { + final List enrichedEvents, final SubscriptionRawMessage message) { this.enrichedEvents = enrichedEvents; - this.commitContext = commitContext; + this.message = message; this.lastPolledTimestamp = INVALID_TIMESTAMP; this.committedTimestamp = INVALID_TIMESTAMP; } + public SubscriptionRawMessage getMessage() { + return message; + } + //////////////////////////// commit //////////////////////////// public void recordCommittedTimestamp() { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/TsFileSubscriptionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/TsFileSubscriptionEvent.java deleted file mode 100644 index 4c7c7932e01c..000000000000 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/TsFileSubscriptionEvent.java +++ /dev/null @@ -1,50 +0,0 @@ -package org.apache.iotdb.db.subscription.event; - -import org.apache.iotdb.commons.pipe.event.EnrichedEvent; - -import java.util.List; - -public class TsFileSubscriptionEvent extends SubscriptionEvent { - - private final String topicName; - private final String fileName; - - private final long fileLength; - private final long startWritingOffset; - private final byte[] filePiece; - - public TsFileSubscriptionEvent( - final List enrichedEvents, - final String subscriptionCommitId, - final String topicName, - final String fileName) { - super(enrichedEvents, subscriptionCommitId); - - this.topicName = topicName; - this.fileName = fileName; - - this.fileLength = -1; - this.startWritingOffset = -1; - this.filePiece = null; - } - - public String getTopicName() { - return topicName; - } - - public String getFileName() { - return fileName; - } - - public long getFileLength() { - return fileLength; - } - - public long getStartWritingOffset() { - return startWritingOffset; - } - - public byte[] getFilePiece() { - return filePiece; - } -} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java index 78d98f7d63c6..32caa47bcccf 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java @@ -33,15 +33,16 @@ import org.apache.iotdb.db.protocol.client.ConfigNodeClientManager; import org.apache.iotdb.db.protocol.client.ConfigNodeInfo; import org.apache.iotdb.db.subscription.agent.SubscriptionAgent; -import org.apache.iotdb.db.subscription.event.SerializableEnrichedTabletsSubscriptionEvent; import org.apache.iotdb.db.subscription.event.SubscriptionEvent; -import org.apache.iotdb.db.subscription.event.TsFileSubscriptionEvent; import org.apache.iotdb.db.subscription.timer.SubscriptionPollTimer; import org.apache.iotdb.rpc.RpcUtils; import org.apache.iotdb.rpc.TSStatusCode; import org.apache.iotdb.rpc.subscription.config.ConsumerConfig; import org.apache.iotdb.rpc.subscription.exception.SubscriptionException; -import org.apache.iotdb.rpc.subscription.payload.common.EnrichedTablets; +import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionCommitContext; +import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionRawMessage; +import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionRawMessageType; +import org.apache.iotdb.rpc.subscription.payload.common.TabletsMessagePayload; import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribeCloseReq; import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribeCommitReq; import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribeHandshakeReq; @@ -63,15 +64,12 @@ import org.apache.iotdb.rpc.subscription.payload.response.PipeSubscribeUnsubscribeResp; import org.apache.iotdb.service.rpc.thrift.TPipeSubscribeReq; import org.apache.iotdb.service.rpc.thrift.TPipeSubscribeResp; -import org.apache.iotdb.tsfile.utils.Pair; import org.apache.thrift.TException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.nio.ByteBuffer; import java.util.List; -import java.util.Map; import java.util.Objects; import java.util.Set; import java.util.stream.Collectors; @@ -338,33 +336,14 @@ private TPipeSubscribeResp handlePipeSubscribePollInternal(final PipeSubscribePo final List events = SubscriptionAgent.broker().poll(consumerConfig, topicNames, timer); - // filter SerializableEnrichedTabletsSubscriptionEvent - final List enrichedTabletsSubscriptionEvents = - events.stream() - .filter(event -> event instanceof SerializableEnrichedTabletsSubscriptionEvent) - .map(event -> (SerializableEnrichedTabletsSubscriptionEvent) event) - .collect(Collectors.toList()); - - // get enriched tablets with byte buffer from SerializableEnrichedTabletsSubscriptionEvent - final List> enrichedTabletsWithByteBufferList = - enrichedTabletsSubscriptionEvents.stream() - .map(event -> new Pair<>(event.getByteBuffer(), event.getEnrichedTablets())) - .collect(Collectors.toList()); + final List rawMessages = + events.stream().map(SubscriptionEvent::getMessage).collect(Collectors.toList()); - // filter TsFileSubscriptionEvent - final List tsFileSubscriptionEvents = - events.stream() - .filter(event -> event instanceof TsFileSubscriptionEvent) - .map(event -> (TsFileSubscriptionEvent) event) + final List commitContexts = + rawMessages.stream() + .map(SubscriptionRawMessage::getCommitContext) .collect(Collectors.toList()); - // generate topicNameToTsFileNameMap - final Map topicNameToTsFileNameMap = - tsFileSubscriptionEvents.stream() - .collect( - Collectors.toMap( - TsFileSubscriptionEvent::getTopicName, TsFileSubscriptionEvent::getFileName)); - // check timer if (timer.isExpired()) { LOGGER.warn( @@ -374,18 +353,27 @@ private TPipeSubscribeResp handlePipeSubscribePollInternal(final PipeSubscribePo } LOGGER.info( - "Subscription: consumer {} poll topics {} successfully", + "Subscription: consumer {} poll topics {} successfully, commit contexts: {}", consumerConfig, - topicNames); + topicNames, + commitContexts); + + // serialize byte buffer + rawMessages.stream() + .filter( + (message -> message.getMessageType() == SubscriptionRawMessageType.TABLETS.getType())) + .forEach(message -> ((TabletsMessagePayload) message.getMessagePayload()).trySerialize()); // generate response final TPipeSubscribeResp resp = - PipeSubscribePollResp.toTPipeSubscribeResp( - RpcUtils.SUCCESS_STATUS, enrichedTabletsWithByteBufferList, topicNameToTsFileNameMap); + PipeSubscribePollResp.toTPipeSubscribeResp(RpcUtils.SUCCESS_STATUS, rawMessages); // reset byte buffer - enrichedTabletsSubscriptionEvents.forEach( - SerializableEnrichedTabletsSubscriptionEvent::resetByteBuffer); + rawMessages.stream() + .filter( + (message -> message.getMessageType() == SubscriptionRawMessageType.TABLETS.getType())) + .forEach( + message -> ((TabletsMessagePayload) message.getMessagePayload()).resetByteBuffer()); return resp; } @@ -433,14 +421,13 @@ private TPipeSubscribeResp handlePipeSubscribeCommitInternal(final PipeSubscribe } // commit - final Map> topicNameToSubscriptionCommitIds = - req.getTopicNameToSubscriptionCommitIds(); - SubscriptionAgent.broker().commit(consumerConfig, topicNameToSubscriptionCommitIds); + final List commitContexts = req.getCommitContexts(); + SubscriptionAgent.broker().commit(consumerConfig, commitContexts); LOGGER.info( - "Subscription: consumer commit {} successfully, commit ids: {}", + "Subscription: consumer commit {} successfully, commit contexts: {}", consumerConfig, - topicNameToSubscriptionCommitIds); + commitContexts); return PipeSubscribeCommitResp.toTPipeSubscribeResp(RpcUtils.SUCCESS_STATUS); } From 9fc78943b287667d63cb3ef45a26e50c889d7f24 Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Fri, 12 Apr 2024 15:21:43 +0800 Subject: [PATCH 09/93] backup --- .../payload/common/EnrichedTablets.java | 87 ------------------- .../common/SubscriptionCommitContext.java | 7 +- .../common/SubscriptionRawMessage.java | 62 +++++++++++++ .../common/SubscriptionRawMessagePayload.java | 19 ++++ .../common/SubscriptionRawMessageType.java | 19 ++++ .../payload/common/TabletsMessagePayload.java | 60 +++++-------- .../common/TsFileInfoMessagePayload.java | 19 ++++ .../response/PipeSubscribePollResp.java | 36 ++------ .../subscription/SubscriptionMessage.java | 43 +++------ .../SubscriptionPullConsumer.java | 27 +++--- .../SubscriptionRawMessageParser.java | 23 +++++ .../SubscriptionSessionConnection.java | 5 +- .../SubscriptionPrefetchingTabletsQueue.java | 2 +- .../receiver/SubscriptionReceiverV1.java | 19 +--- 14 files changed, 208 insertions(+), 220 deletions(-) delete mode 100644 iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/EnrichedTablets.java create mode 100644 iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionRawMessageParser.java diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/EnrichedTablets.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/EnrichedTablets.java deleted file mode 100644 index 09eaa359f593..000000000000 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/EnrichedTablets.java +++ /dev/null @@ -1,87 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iotdb.rpc.subscription.payload.common; - -import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; -import org.apache.iotdb.tsfile.write.record.Tablet; - -import java.io.DataOutputStream; -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.List; -import java.util.Objects; - -public class EnrichedTablets { - - private final transient SubscriptionCommitContext commitContext; - private final transient List tablets; - - public List getTablets() { - return tablets; - } - - public EnrichedTablets( - final SubscriptionCommitContext commitContext, final List tablets) { - this.commitContext = commitContext; - this.tablets = tablets; - } - - /////////////////////////////// de/ser /////////////////////////////// - - public void serialize(final DataOutputStream stream) throws IOException { - ReadWriteIOUtils.write(tablets.size(), stream); - for (final Tablet tablet : tablets) { - tablet.serialize(stream); - } - } - - public static EnrichedTablets deserialize(final ByteBuffer buffer) { - final SubscriptionCommitContext commitContext = SubscriptionCommitContext.deserialize(buffer); - final List tablets = new ArrayList<>(); - final int size = ReadWriteIOUtils.readInt(buffer); - for (int i = 0; i < size; ++i) { - tablets.add(Tablet.deserialize(buffer)); - } - return new EnrichedTablets(commitContext, tablets); - } - - /////////////////////////////// Object /////////////////////////////// - - @Override - public boolean equals(final Object obj) { - if (this == obj) { - return true; - } - if (obj == null || getClass() != obj.getClass()) { - return false; - } - final EnrichedTablets that = (EnrichedTablets) obj; - return Objects.equals(this.commitContext, that.commitContext) - && Objects.equals(this.tablets, that.tablets); - } - - @Override - public int hashCode() { - // Considering that the Tablet class has not implemented the hashCode method, the tablets member - // should not be included when calculating the hashCode of EnrichedTablets. - return Objects.hash(commitContext); - } -} diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionCommitContext.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionCommitContext.java index f2c9e61bf03f..c99d32b749f1 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionCommitContext.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionCommitContext.java @@ -26,7 +26,7 @@ import java.nio.ByteBuffer; import java.util.Objects; -public class SubscriptionCommitContext { +public class SubscriptionCommitContext implements Comparable { private final int dataNodeId; @@ -128,4 +128,9 @@ public String toString() { + commitId + "}"; } + + @Override + public int compareTo(SubscriptionCommitContext commitContext) { + return 0; + } } diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionRawMessage.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionRawMessage.java index 25aa41444f25..9b35014d829c 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionRawMessage.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionRawMessage.java @@ -1,20 +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.rpc.subscription.payload.common; import org.apache.iotdb.tsfile.utils.PublicBAOS; import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.io.DataOutputStream; import java.io.IOException; import java.nio.ByteBuffer; +import java.util.Objects; public class SubscriptionRawMessage { + private static final Logger LOGGER = LoggerFactory.getLogger(SubscriptionRawMessage.class); + private final transient short messageType; private final transient SubscriptionRawMessagePayload messagePayload; private final transient SubscriptionCommitContext commitContext; + private ByteBuffer byteBuffer; + public SubscriptionRawMessage( short messageType, SubscriptionRawMessagePayload messagePayload, @@ -36,9 +63,16 @@ public SubscriptionCommitContext getCommitContext() { return commitContext; } + public ByteBuffer getByteBuffer() { + return byteBuffer; + } + /////////////////////////////// de/ser /////////////////////////////// public static ByteBuffer serialize(SubscriptionRawMessage message) throws IOException { + if (Objects.nonNull(message.byteBuffer)) { + return message.byteBuffer; + } try (final PublicBAOS byteArrayOutputStream = new PublicBAOS(); final DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream)) { message.serialize(outputStream); @@ -73,4 +107,32 @@ public static SubscriptionRawMessage deserialize(final ByteBuffer buffer) { final SubscriptionCommitContext commitContext = SubscriptionCommitContext.deserialize(buffer); return new SubscriptionRawMessage(messageType, messagePayload, commitContext); } + + //////////////////////////// serialization //////////////////////////// + + /** @return true -> byte buffer is not null */ + public boolean trySerialize() { + if (Objects.isNull(byteBuffer)) { + try { + try (final PublicBAOS byteArrayOutputStream = new PublicBAOS(); + final DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream)) { + serialize(outputStream); + byteBuffer = + ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size()); + } + return true; + } catch (final IOException e) { + LOGGER.warn( + "Subscription: something unexpected happened when serializing Tablets, exception is {}", + e.getMessage()); + } + return false; + } + return true; + } + + public void resetByteBuffer() { + // maybe friendly for gc + byteBuffer = null; + } } diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionRawMessagePayload.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionRawMessagePayload.java index f948b8340716..2c194e5500e1 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionRawMessagePayload.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionRawMessagePayload.java @@ -1,3 +1,22 @@ +/* + * 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.rpc.subscription.payload.common; import java.io.DataOutputStream; diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionRawMessageType.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionRawMessageType.java index b4700e12d03b..62fbea9a17ee 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionRawMessageType.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionRawMessageType.java @@ -1,3 +1,22 @@ +/* + * 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.rpc.subscription.payload.common; import java.util.Arrays; diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TabletsMessagePayload.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TabletsMessagePayload.java index 3dc112dd3209..d7f49cfb1830 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TabletsMessagePayload.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TabletsMessagePayload.java @@ -1,33 +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.rpc.subscription.payload.common; -import org.apache.iotdb.tsfile.utils.PublicBAOS; import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; import org.apache.iotdb.tsfile.write.record.Tablet; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.DataOutputStream; import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.List; -import java.util.Objects; public class TabletsMessagePayload implements SubscriptionRawMessagePayload { - private static final Logger LOGGER = LoggerFactory.getLogger(TabletsMessagePayload.class); - protected transient List tablets = new ArrayList<>(); - private ByteBuffer byteBuffer; // serialized tablets - public TabletsMessagePayload() {} public TabletsMessagePayload(List tablets) { this.tablets = tablets; } + public List getTablets() { + return tablets; + } + @Override public void serialize(DataOutputStream stream) throws IOException { ReadWriteIOUtils.write(tablets.size(), stream); @@ -46,32 +60,4 @@ public SubscriptionRawMessagePayload deserialize(ByteBuffer buffer) { this.tablets = tablets; return this; } - - //////////////////////////// serialization //////////////////////////// - - /** @return true -> byte buffer is not null */ - public boolean trySerialize() { - if (Objects.isNull(byteBuffer)) { - try { - try (final PublicBAOS byteArrayOutputStream = new PublicBAOS(); - final DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream)) { - serialize(outputStream); - byteBuffer = - ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size()); - } - return true; - } catch (final IOException e) { - LOGGER.warn( - "Subscription: something unexpected happened when serializing Tablets, exception is {}", - e.getMessage()); - } - return false; - } - return true; - } - - public void resetByteBuffer() { - // maybe friendly for gc - byteBuffer = null; - } } diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileInfoMessagePayload.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileInfoMessagePayload.java index b8ce41dcef74..4972bf2714f9 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileInfoMessagePayload.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileInfoMessagePayload.java @@ -1,3 +1,22 @@ +/* + * 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.rpc.subscription.payload.common; import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollResp.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollResp.java index c05e1ebba539..b06f890874c0 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollResp.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollResp.java @@ -22,13 +22,9 @@ import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.rpc.RpcUtils; import org.apache.iotdb.rpc.TSStatusCode; -import org.apache.iotdb.rpc.subscription.payload.common.EnrichedTablets; import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionRawMessage; import org.apache.iotdb.service.rpc.thrift.TPipeSubscribeResp; -import org.apache.iotdb.tsfile.utils.Pair; -import org.apache.iotdb.tsfile.utils.PublicBAOS; -import java.io.DataOutputStream; import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; @@ -61,7 +57,11 @@ public static PipeSubscribePollResp toTPipeSubscribeResp( try { resp.body = new ArrayList<>(); for (final SubscriptionRawMessage message : messages) { - resp.body.add(SubscriptionRawMessage.serialize(message)); + if (Objects.nonNull(message.getByteBuffer())) { + resp.body.add(message.getByteBuffer()); + } else { + resp.body.add(SubscriptionRawMessage.serialize(message)); + } } } catch (IOException e) { resp.status = RpcUtils.getStatus(TSStatusCode.SUBSCRIPTION_POLL_ERROR, e.getMessage()); @@ -90,32 +90,6 @@ public static PipeSubscribePollResp fromTPipeSubscribeResp(TPipeSubscribeResp po return resp; } - /////////////////////////////// Utility /////////////////////////////// - - public static List serializeEnrichedTabletsWithByteBufferList( - List> enrichedTabletsWithByteBufferList) - throws IOException { - List byteBufferList = new ArrayList<>(); - for (Pair enrichedTabletsWithByteBuffer : - enrichedTabletsWithByteBufferList) { - if (Objects.nonNull(enrichedTabletsWithByteBuffer.getLeft())) { - byteBufferList.add(enrichedTabletsWithByteBuffer.getLeft()); - } else { - byteBufferList.add(serializeEnrichedTablets(enrichedTabletsWithByteBuffer.getRight())); - } - } - return byteBufferList; - } - - public static ByteBuffer serializeEnrichedTablets(EnrichedTablets enrichedTablets) - throws IOException { - try (final PublicBAOS byteArrayOutputStream = new PublicBAOS(); - final DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream)) { - enrichedTablets.serialize(outputStream); - return ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size()); - } - } - /////////////////////////////// Object /////////////////////////////// @Override diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionMessage.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionMessage.java index 6ef7c781e004..7b7506908352 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionMessage.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionMessage.java @@ -19,44 +19,32 @@ package org.apache.iotdb.session.subscription; -import org.apache.iotdb.rpc.subscription.payload.common.EnrichedTablets; +import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionCommitContext; +import org.apache.iotdb.tsfile.write.record.Tablet; +import java.util.List; import java.util.Objects; public class SubscriptionMessage implements Comparable { + private final SubscriptionCommitContext commitContext; + // TODO: support more data format private final SubscriptionMessagePayload payload; - private final String topicName; - private final String subscriptionCommitId; - - public SubscriptionMessage(EnrichedTablets tablets) { - this.payload = new SubscriptionSessionDataSets(tablets.getTablets()); - this.topicName = tablets.getTopicName(); - this.subscriptionCommitId = tablets.getSubscriptionCommitId(); + public SubscriptionMessage(SubscriptionCommitContext commitContext, List tablets) { + this.commitContext = commitContext; + this.payload = new SubscriptionSessionDataSets(tablets); } - public SubscriptionMessage(String topicName, String fileName) {} - - public String getTopicName() { - return topicName; + public SubscriptionCommitContext getCommitContext() { + return commitContext; } public SubscriptionMessagePayload getPayload() { return payload; } - String getSubscriptionCommitId() { - // make it package-private - return subscriptionCommitId; - } - - int parseDataNodeIdFromSubscriptionCommitId() { - // make it package-private - return Integer.parseInt(subscriptionCommitId.split("#")[0]); - } - /////////////////////////////// override /////////////////////////////// @Override @@ -68,20 +56,17 @@ public boolean equals(Object obj) { return false; } SubscriptionMessage that = (SubscriptionMessage) obj; - return Objects.equals(this.topicName, that.topicName) - && Objects.equals(this.subscriptionCommitId, that.subscriptionCommitId); + return Objects.equals(this.payload, that.payload) + && Objects.equals(this.commitContext, that.commitContext); } @Override public int hashCode() { - return Objects.hash(topicName, subscriptionCommitId); + return Objects.hash(payload, commitContext); } @Override public int compareTo(SubscriptionMessage that) { - if (this.topicName.compareTo(that.topicName) == 0) { - return this.subscriptionCommitId.compareTo(that.subscriptionCommitId); - } - return this.topicName.compareTo(that.topicName); + return this.commitContext.compareTo(that.commitContext); } } diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java index 0462f94aa789..6114caaac9a7 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java @@ -23,7 +23,8 @@ import org.apache.iotdb.rpc.StatementExecutionException; import org.apache.iotdb.rpc.subscription.config.ConsumerConstant; import org.apache.iotdb.rpc.subscription.exception.SubscriptionException; -import org.apache.iotdb.rpc.subscription.payload.common.EnrichedTablets; +import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionCommitContext; +import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionRawMessage; import org.apache.thrift.TException; import org.slf4j.Logger; @@ -147,20 +148,20 @@ public List poll(Set topicNames, Duration timeoutMs public List poll(Set topicNames, long timeoutMs) throws TException, IOException, StatementExecutionException { - List enrichedTabletsList = new ArrayList<>(); + List rawMessages = new ArrayList<>(); acquireReadLock(); try { for (final SubscriptionProvider provider : getAllAvailableProviders()) { // TODO: network timeout - enrichedTabletsList.addAll(provider.getSessionConnection().poll(topicNames, timeoutMs)); + rawMessages.addAll(provider.getSessionConnection().poll(topicNames, timeoutMs)); } } finally { releaseReadLock(); } List messages = - enrichedTabletsList.stream().map(SubscriptionMessage::new).collect(Collectors.toList()); + rawMessages.stream().map(SubscriptionRawMessageParser::parse).collect(Collectors.toList()); if (autoCommit) { long currentTimestamp = System.currentTimeMillis(); @@ -183,17 +184,15 @@ public void commitSync(SubscriptionMessage message) public void commitSync(Iterable messages) throws TException, IOException, StatementExecutionException, IoTDBConnectionException { - Map>> dataNodeIdToTopicNameToSubscriptionCommitIds = + Map> dataNodeIdToSubscriptionCommitContexts = new HashMap<>(); for (SubscriptionMessage message : messages) { - dataNodeIdToTopicNameToSubscriptionCommitIds - .computeIfAbsent( - message.parseDataNodeIdFromSubscriptionCommitId(), (id) -> new HashMap<>()) - .computeIfAbsent(message.getTopicName(), (topicName) -> new ArrayList<>()) - .add(message.getSubscriptionCommitId()); + dataNodeIdToSubscriptionCommitContexts + .computeIfAbsent(message.getCommitContext().getDataNodeId(), (id) -> new ArrayList<>()) + .add(message.getCommitContext()); } - for (Map.Entry>> entry : - dataNodeIdToTopicNameToSubscriptionCommitIds.entrySet()) { + for (Map.Entry> entry : + dataNodeIdToSubscriptionCommitContexts.entrySet()) { commitSyncInternal(entry.getKey(), entry.getValue()); } } @@ -201,7 +200,7 @@ public void commitSync(Iterable messages) /////////////////////////////// utility /////////////////////////////// private void commitSyncInternal( - int dataNodeId, Map> topicNameToSubscriptionCommitIds) + int dataNodeId, List subscriptionCommitContexts) throws TException, IOException, StatementExecutionException, IoTDBConnectionException { acquireReadLock(); try { @@ -212,7 +211,7 @@ private void commitSyncInternal( "something unexpected happened when commit messages to subscription provider with data node id %s, the subscription provider may be unavailable or not existed", dataNodeId)); } - provider.getSessionConnection().commitSync(topicNameToSubscriptionCommitIds); + provider.getSessionConnection().commitSync(subscriptionCommitContexts); } finally { releaseReadLock(); } diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionRawMessageParser.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionRawMessageParser.java new file mode 100644 index 000000000000..8e4657478b60 --- /dev/null +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionRawMessageParser.java @@ -0,0 +1,23 @@ +package org.apache.iotdb.session.subscription; + +import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionRawMessage; +import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionRawMessageType; +import org.apache.iotdb.rpc.subscription.payload.common.TabletsMessagePayload; + +public class SubscriptionRawMessageParser { + + private SubscriptionRawMessageParser() {} + + public static SubscriptionMessage parse(SubscriptionRawMessage rawMessage) { + short messageType = rawMessage.getMessageType(); + if (SubscriptionRawMessageType.isValidatedMessageType(messageType)) { + switch (SubscriptionRawMessageType.valueOf(messageType)) { + case TABLETS: + return new SubscriptionMessage( + rawMessage.getCommitContext(), + ((TabletsMessagePayload) rawMessage.getMessagePayload()).getTablets()); + } + } + return null; + } +} diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionSessionConnection.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionSessionConnection.java index 87211b15b9b4..5fa118f8f76e 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionSessionConnection.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionSessionConnection.java @@ -25,6 +25,7 @@ import org.apache.iotdb.rpc.RpcUtils; import org.apache.iotdb.rpc.StatementExecutionException; import org.apache.iotdb.rpc.subscription.config.ConsumerConfig; +import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionCommitContext; import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionRawMessage; import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribeCloseReq; import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribeCommitReq; @@ -138,11 +139,11 @@ public List poll(Set topicNames, long timeoutMs) return pollResp.getMessages(); } - public void commitSync(Map> topicNameToSubscriptionCommitIds) + public void commitSync(List subscriptionCommitContexts) throws TException, IOException, StatementExecutionException { TPipeSubscribeResp resp = client.pipeSubscribe( - PipeSubscribeCommitReq.toTPipeSubscribeReq(topicNameToSubscriptionCommitIds)); + PipeSubscribeCommitReq.toTPipeSubscribeReq(subscriptionCommitContexts)); RpcUtils.verifySuccess(resp.status); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTabletsQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTabletsQueue.java index 68480a3ca537..367498fbb164 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTabletsQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTabletsQueue.java @@ -170,7 +170,7 @@ private void serializeOnce() { // Serialize the uncommitted and pollable event. if (currentEvent.pollable()) { // No need to concern whether serialization is successful. - ((TabletsMessagePayload) currentEvent.getMessage().getMessagePayload()).trySerialize(); + currentEvent.getMessage().trySerialize(); } } } catch (final InterruptedException e) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java index 32caa47bcccf..c7ec728fed1c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java @@ -41,8 +41,6 @@ import org.apache.iotdb.rpc.subscription.exception.SubscriptionException; import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionCommitContext; import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionRawMessage; -import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionRawMessageType; -import org.apache.iotdb.rpc.subscription.payload.common.TabletsMessagePayload; import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribeCloseReq; import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribeCommitReq; import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribeHandshakeReq; @@ -358,23 +356,8 @@ private TPipeSubscribeResp handlePipeSubscribePollInternal(final PipeSubscribePo topicNames, commitContexts); - // serialize byte buffer - rawMessages.stream() - .filter( - (message -> message.getMessageType() == SubscriptionRawMessageType.TABLETS.getType())) - .forEach(message -> ((TabletsMessagePayload) message.getMessagePayload()).trySerialize()); - // generate response - final TPipeSubscribeResp resp = - PipeSubscribePollResp.toTPipeSubscribeResp(RpcUtils.SUCCESS_STATUS, rawMessages); - - // reset byte buffer - rawMessages.stream() - .filter( - (message -> message.getMessageType() == SubscriptionRawMessageType.TABLETS.getType())) - .forEach( - message -> ((TabletsMessagePayload) message.getMessagePayload()).resetByteBuffer()); - return resp; + return PipeSubscribePollResp.toTPipeSubscribeResp(RpcUtils.SUCCESS_STATUS, rawMessages); } private TPipeSubscribeResp handlePipeSubscribePollTsFile(final PipeSubscribePollTsFileReq req) { From 2bd171a815594e05a5b0358f2acbbc75220c7df9 Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Fri, 12 Apr 2024 15:22:40 +0800 Subject: [PATCH 10/93] fix --- .../SubscriptionRawMessageParser.java | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionRawMessageParser.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionRawMessageParser.java index 8e4657478b60..f3659de29ac4 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionRawMessageParser.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionRawMessageParser.java @@ -1,3 +1,22 @@ +/* + * 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.session.subscription; import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionRawMessage; From 22aced35598c70b3119c70b44cf0d07de7eb59e0 Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Fri, 12 Apr 2024 15:40:30 +0800 Subject: [PATCH 11/93] fix --- iotdb-client/service-rpc/pom.xml | 4 ++++ .../common/SubscriptionRawMessage.java | 2 +- .../SubscriptionPrefetchingTabletsQueue.java | 19 +++++++++++++++++++ 3 files changed, 24 insertions(+), 1 deletion(-) diff --git a/iotdb-client/service-rpc/pom.xml b/iotdb-client/service-rpc/pom.xml index e66f399a5ad0..58a76584990a 100644 --- a/iotdb-client/service-rpc/pom.xml +++ b/iotdb-client/service-rpc/pom.xml @@ -80,6 +80,10 @@ junit test + + org.slf4j + slf4j-api + diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionRawMessage.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionRawMessage.java index 9b35014d829c..f7e7923ffa7f 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionRawMessage.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionRawMessage.java @@ -123,7 +123,7 @@ public boolean trySerialize() { return true; } catch (final IOException e) { LOGGER.warn( - "Subscription: something unexpected happened when serializing Tablets, exception is {}", + "Subscription: something unexpected happened when serializing SubscriptionRawMessage, exception is {}", e.getMessage()); } return false; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTabletsQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTabletsQueue.java index 367498fbb164..5b58ea2cdd32 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTabletsQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTabletsQueue.java @@ -1,3 +1,22 @@ +/* + * 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.subscription.broker; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; From 4f85fb6b01a05cd2ae76106c2a877b8ab618a37e Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Sat, 13 Apr 2024 16:59:13 +0800 Subject: [PATCH 12/93] refact --- .../subscription/config/ConsumerConfig.java | 11 +- .../payload/common/PollMessagePayload.java | 35 ++++++ .../common/PollTsFileMessagePayload.java | 43 +++++++ .../common/SubscriptionCommitContext.java | 10 ++ ...d.java => SubscriptionMessagePayload.java} | 4 +- .../common/SubscriptionPollMessage.java | 98 +++++++++++++++ .../common/SubscriptionPollMessageType.java | 36 ++++++ ...ge.java => SubscriptionPolledMessage.java} | 24 ++-- ...ava => SubscriptionPolledMessageType.java} | 10 +- .../payload/common/TabletsMessagePayload.java | 4 +- .../common/TsFileInfoMessagePayload.java | 4 +- .../request/PipeSubscribeHandshakeReq.java | 9 +- .../payload/request/PipeSubscribePollReq.java | 39 ++---- .../request/PipeSubscribePollTsFileReq.java | 113 ------------------ .../request/PipeSubscribeRequestType.java | 1 - .../response/PipeSubscribeHandshakeResp.java | 11 +- .../response/PipeSubscribePollResp.java | 14 +-- .../subscription/SubscriptionMessage.java | 5 + .../SubscriptionPullConsumer.java | 16 ++- .../SubscriptionRawMessageParser.java | 10 +- .../SubscriptionSessionConnection.java | 12 +- .../SubscriptionSessionDataSets.java | 7 +- .../SubscriptionTsFileReader.java | 31 +++++ .../SubscriptionPrefetchingTabletsQueue.java | 8 +- .../SubscriptionPrefetchingTsFileQueue.java | 8 +- .../subscription/event/SubscriptionEvent.java | 8 +- .../receiver/SubscriptionReceiverV1.java | 74 ++++++------ 27 files changed, 391 insertions(+), 254 deletions(-) create mode 100644 iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/PollMessagePayload.java create mode 100644 iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/PollTsFileMessagePayload.java rename iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/{SubscriptionRawMessagePayload.java => SubscriptionMessagePayload.java} (90%) create mode 100644 iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPollMessage.java create mode 100644 iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPollMessageType.java rename iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/{SubscriptionRawMessage.java => SubscriptionPolledMessage.java} (84%) rename iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/{SubscriptionRawMessageType.java => SubscriptionPolledMessageType.java} (83%) delete mode 100644 iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribePollTsFileReq.java create mode 100644 iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionTsFileReader.java diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/config/ConsumerConfig.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/config/ConsumerConfig.java index 7915e1ef1e35..a3c58d8acdc0 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/config/ConsumerConfig.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/config/ConsumerConfig.java @@ -20,6 +20,7 @@ package org.apache.iotdb.rpc.subscription.config; import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameters; +import org.apache.iotdb.tsfile.utils.PublicBAOS; import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; import java.io.DataOutputStream; @@ -40,7 +41,15 @@ public ConsumerConfig(Map attributes) { /////////////////////////////// de/ser /////////////////////////////// - public void serialize(DataOutputStream stream) throws IOException { + public static ByteBuffer serialize(ConsumerConfig consumerConfig) throws IOException { + try (final PublicBAOS byteArrayOutputStream = new PublicBAOS(); + final DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream)) { + consumerConfig.serialize(outputStream); + return ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size()); + } + } + + private void serialize(DataOutputStream stream) throws IOException { ReadWriteIOUtils.write(attributes, stream); } diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/PollMessagePayload.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/PollMessagePayload.java new file mode 100644 index 000000000000..8446368fa2a1 --- /dev/null +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/PollMessagePayload.java @@ -0,0 +1,35 @@ +package org.apache.iotdb.rpc.subscription.payload.common; + +import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; + +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.HashSet; +import java.util.Set; + +public class PollMessagePayload implements SubscriptionMessagePayload { + + private transient Set topicNames = new HashSet<>(); + + public PollMessagePayload() {} + + public PollMessagePayload(Set topicNames) { + this.topicNames = topicNames; + } + + public Set getTopicNames() { + return topicNames; + } + + @Override + public void serialize(DataOutputStream stream) throws IOException { + ReadWriteIOUtils.writeObjectSet(topicNames, stream); + } + + @Override + public SubscriptionMessagePayload deserialize(ByteBuffer buffer) { + topicNames = ReadWriteIOUtils.readObjectSet(buffer); + return this; + } +} diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/PollTsFileMessagePayload.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/PollTsFileMessagePayload.java new file mode 100644 index 000000000000..d2047e4703d1 --- /dev/null +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/PollTsFileMessagePayload.java @@ -0,0 +1,43 @@ +package org.apache.iotdb.rpc.subscription.payload.common; + +import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; + +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; + +public class PollTsFileMessagePayload implements SubscriptionMessagePayload { + + private transient String topicName; + + private transient String fileName; + + private transient long endWritingOffset; + + public String getTopicName() { + return topicName; + } + + public String getFileName() { + return fileName; + } + + public long getEndWritingOffset() { + return endWritingOffset; + } + + @Override + public void serialize(DataOutputStream stream) throws IOException { + ReadWriteIOUtils.write(topicName, stream); + ReadWriteIOUtils.write(fileName, stream); + ReadWriteIOUtils.write(endWritingOffset, stream); + } + + @Override + public SubscriptionMessagePayload deserialize(ByteBuffer buffer) { + topicName = ReadWriteIOUtils.readString(buffer); + fileName = ReadWriteIOUtils.readString(buffer); + endWritingOffset = ReadWriteIOUtils.readLong(buffer); + return this; + } +} diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionCommitContext.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionCommitContext.java index c99d32b749f1..37be5aa97a50 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionCommitContext.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionCommitContext.java @@ -19,6 +19,7 @@ package org.apache.iotdb.rpc.subscription.payload.common; +import org.apache.iotdb.tsfile.utils.PublicBAOS; import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; import java.io.DataOutputStream; @@ -73,6 +74,14 @@ public long getCommitId() { /////////////////////////////// de/ser /////////////////////////////// + public static ByteBuffer serialize(SubscriptionCommitContext commitContext) throws IOException { + try (final PublicBAOS byteArrayOutputStream = new PublicBAOS(); + final DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream)) { + commitContext.serialize(outputStream); + return ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size()); + } + } + public void serialize(final DataOutputStream stream) throws IOException { ReadWriteIOUtils.write(dataNodeId, stream); ReadWriteIOUtils.write(rebootTimes, stream); @@ -131,6 +140,7 @@ public String toString() { @Override public int compareTo(SubscriptionCommitContext commitContext) { + // TODO return 0; } } diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionRawMessagePayload.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionMessagePayload.java similarity index 90% rename from iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionRawMessagePayload.java rename to iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionMessagePayload.java index 2c194e5500e1..dadb3f8201c2 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionRawMessagePayload.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionMessagePayload.java @@ -23,11 +23,11 @@ import java.io.IOException; import java.nio.ByteBuffer; -public interface SubscriptionRawMessagePayload { +public interface SubscriptionMessagePayload { void serialize(final DataOutputStream stream) throws IOException; - SubscriptionRawMessagePayload deserialize(final ByteBuffer buffer); + SubscriptionMessagePayload deserialize(final ByteBuffer buffer); boolean equals(final Object obj); diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPollMessage.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPollMessage.java new file mode 100644 index 000000000000..708c15dcc2a6 --- /dev/null +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPollMessage.java @@ -0,0 +1,98 @@ +/* + * 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.rpc.subscription.payload.common; + +import org.apache.iotdb.tsfile.utils.PublicBAOS; +import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; + +public class SubscriptionPollMessage { + + private static final Logger LOGGER = LoggerFactory.getLogger(SubscriptionPolledMessage.class); + + private final transient short messageType; + + private final transient SubscriptionMessagePayload messagePayload; + + private final transient long timeoutMs; + + public SubscriptionPollMessage( + short messageType, SubscriptionMessagePayload messagePayload, long timeoutMs) { + this.messageType = messageType; + this.messagePayload = messagePayload; + this.timeoutMs = timeoutMs; + } + + public short getMessageType() { + return messageType; + } + + public SubscriptionMessagePayload getMessagePayload() { + return messagePayload; + } + + public long getTimeoutMs() { + return timeoutMs; + } + + //////////////////////////// serialization //////////////////////////// + + public static ByteBuffer serialize(SubscriptionPollMessage message) throws IOException { + try (final PublicBAOS byteArrayOutputStream = new PublicBAOS(); + final DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream)) { + message.serialize(outputStream); + return ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size()); + } + } + + private void serialize(final DataOutputStream stream) throws IOException { + ReadWriteIOUtils.write(messageType, stream); + messagePayload.serialize(stream); + ReadWriteIOUtils.write(timeoutMs, stream); + } + + public static SubscriptionPollMessage deserialize(final ByteBuffer buffer) { + final short messageType = ReadWriteIOUtils.readShort(buffer); + final SubscriptionMessagePayload messagePayload; + if (SubscriptionPollMessageType.isValidatedMessageType(messageType)) { + switch (SubscriptionPollMessageType.valueOf(messageType)) { + case POLL: + messagePayload = new PollMessagePayload().deserialize(buffer); + break; + case POLL_TS_FILE: + messagePayload = new PollTsFileMessagePayload().deserialize(buffer); + break; + default: + messagePayload = null; + } + } else { + messagePayload = null; + } + + final long timeoutMs = ReadWriteIOUtils.readLong(buffer); + return new SubscriptionPollMessage(messageType, messagePayload, timeoutMs); + } +} diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPollMessageType.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPollMessageType.java new file mode 100644 index 000000000000..2d5ba173d7e4 --- /dev/null +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPollMessageType.java @@ -0,0 +1,36 @@ +package org.apache.iotdb.rpc.subscription.payload.common; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; + +public enum SubscriptionPollMessageType { + POLL((short) 0), + POLL_TS_FILE((short) 1), + ; + + private final short type; + + SubscriptionPollMessageType(short type) { + this.type = type; + } + + public short getType() { + return type; + } + + private static final Map TYPE_MAP = + Arrays.stream(SubscriptionPollMessageType.values()) + .collect( + HashMap::new, + (typeMap, messageType) -> typeMap.put(messageType.getType(), messageType), + HashMap::putAll); + + public static boolean isValidatedMessageType(short type) { + return TYPE_MAP.containsKey(type); + } + + public static SubscriptionPollMessageType valueOf(short type) { + return TYPE_MAP.get(type); + } +} diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionRawMessage.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledMessage.java similarity index 84% rename from iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionRawMessage.java rename to iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledMessage.java index f7e7923ffa7f..c835bd32ab92 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionRawMessage.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledMessage.java @@ -30,21 +30,21 @@ import java.nio.ByteBuffer; import java.util.Objects; -public class SubscriptionRawMessage { +public class SubscriptionPolledMessage { - private static final Logger LOGGER = LoggerFactory.getLogger(SubscriptionRawMessage.class); + private static final Logger LOGGER = LoggerFactory.getLogger(SubscriptionPolledMessage.class); private final transient short messageType; - private final transient SubscriptionRawMessagePayload messagePayload; + private final transient SubscriptionMessagePayload messagePayload; private final transient SubscriptionCommitContext commitContext; private ByteBuffer byteBuffer; - public SubscriptionRawMessage( + public SubscriptionPolledMessage( short messageType, - SubscriptionRawMessagePayload messagePayload, + SubscriptionMessagePayload messagePayload, SubscriptionCommitContext commitContext) { this.messageType = messageType; this.messagePayload = messagePayload; @@ -55,7 +55,7 @@ public short getMessageType() { return messageType; } - public SubscriptionRawMessagePayload getMessagePayload() { + public SubscriptionMessagePayload getMessagePayload() { return messagePayload; } @@ -69,7 +69,7 @@ public ByteBuffer getByteBuffer() { /////////////////////////////// de/ser /////////////////////////////// - public static ByteBuffer serialize(SubscriptionRawMessage message) throws IOException { + public static ByteBuffer serialize(SubscriptionPolledMessage message) throws IOException { if (Objects.nonNull(message.byteBuffer)) { return message.byteBuffer; } @@ -86,11 +86,11 @@ private void serialize(final DataOutputStream stream) throws IOException { commitContext.serialize(stream); } - public static SubscriptionRawMessage deserialize(final ByteBuffer buffer) { + public static SubscriptionPolledMessage deserialize(final ByteBuffer buffer) { final short messageType = ReadWriteIOUtils.readShort(buffer); - final SubscriptionRawMessagePayload messagePayload; - if (SubscriptionRawMessageType.isValidatedMessageType(messageType)) { - switch (SubscriptionRawMessageType.valueOf(messageType)) { + final SubscriptionMessagePayload messagePayload; + if (SubscriptionPolledMessageType.isValidatedMessageType(messageType)) { + switch (SubscriptionPolledMessageType.valueOf(messageType)) { case TABLETS: messagePayload = new TabletsMessagePayload().deserialize(buffer); break; @@ -105,7 +105,7 @@ public static SubscriptionRawMessage deserialize(final ByteBuffer buffer) { } final SubscriptionCommitContext commitContext = SubscriptionCommitContext.deserialize(buffer); - return new SubscriptionRawMessage(messageType, messagePayload, commitContext); + return new SubscriptionPolledMessage(messageType, messagePayload, commitContext); } //////////////////////////// serialization //////////////////////////// diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionRawMessageType.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledMessageType.java similarity index 83% rename from iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionRawMessageType.java rename to iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledMessageType.java index 62fbea9a17ee..cc6a1026bec7 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionRawMessageType.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledMessageType.java @@ -23,7 +23,7 @@ import java.util.HashMap; import java.util.Map; -public enum SubscriptionRawMessageType { +public enum SubscriptionPolledMessageType { TABLETS((short) 0), TS_FILE_INFO((short) 1), @@ -33,7 +33,7 @@ public enum SubscriptionRawMessageType { private final short type; - SubscriptionRawMessageType(short type) { + SubscriptionPolledMessageType(short type) { this.type = type; } @@ -41,8 +41,8 @@ public short getType() { return type; } - private static final Map TYPE_MAP = - Arrays.stream(SubscriptionRawMessageType.values()) + private static final Map TYPE_MAP = + Arrays.stream(SubscriptionPolledMessageType.values()) .collect( HashMap::new, (typeMap, messageType) -> typeMap.put(messageType.getType(), messageType), @@ -52,7 +52,7 @@ public static boolean isValidatedMessageType(short type) { return TYPE_MAP.containsKey(type); } - public static SubscriptionRawMessageType valueOf(short type) { + public static SubscriptionPolledMessageType valueOf(short type) { return TYPE_MAP.get(type); } } diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TabletsMessagePayload.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TabletsMessagePayload.java index d7f49cfb1830..232befe6662b 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TabletsMessagePayload.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TabletsMessagePayload.java @@ -28,7 +28,7 @@ import java.util.ArrayList; import java.util.List; -public class TabletsMessagePayload implements SubscriptionRawMessagePayload { +public class TabletsMessagePayload implements SubscriptionMessagePayload { protected transient List tablets = new ArrayList<>(); @@ -51,7 +51,7 @@ public void serialize(DataOutputStream stream) throws IOException { } @Override - public SubscriptionRawMessagePayload deserialize(ByteBuffer buffer) { + public SubscriptionMessagePayload deserialize(ByteBuffer buffer) { final List tablets = new ArrayList<>(); final int size = ReadWriteIOUtils.readInt(buffer); for (int i = 0; i < size; ++i) { diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileInfoMessagePayload.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileInfoMessagePayload.java index 4972bf2714f9..d2c83175c546 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileInfoMessagePayload.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileInfoMessagePayload.java @@ -25,7 +25,7 @@ import java.io.IOException; import java.nio.ByteBuffer; -public class TsFileInfoMessagePayload implements SubscriptionRawMessagePayload { +public class TsFileInfoMessagePayload implements SubscriptionMessagePayload { protected transient String fileName; @@ -41,7 +41,7 @@ public void serialize(DataOutputStream stream) throws IOException { } @Override - public SubscriptionRawMessagePayload deserialize(ByteBuffer buffer) { + public SubscriptionMessagePayload deserialize(ByteBuffer buffer) { this.fileName = ReadWriteIOUtils.readString(buffer); return this; } diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribeHandshakeReq.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribeHandshakeReq.java index f0dc2d1880d5..55c236bd9d88 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribeHandshakeReq.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribeHandshakeReq.java @@ -21,11 +21,8 @@ import org.apache.iotdb.rpc.subscription.config.ConsumerConfig; import org.apache.iotdb.service.rpc.thrift.TPipeSubscribeReq; -import org.apache.iotdb.tsfile.utils.PublicBAOS; -import java.io.DataOutputStream; import java.io.IOException; -import java.nio.ByteBuffer; import java.util.Objects; public class PipeSubscribeHandshakeReq extends TPipeSubscribeReq { @@ -50,11 +47,7 @@ public static PipeSubscribeHandshakeReq toTPipeSubscribeReq(ConsumerConfig consu req.version = PipeSubscribeRequestVersion.VERSION_1.getVersion(); req.type = PipeSubscribeRequestType.HANDSHAKE.getType(); - try (final PublicBAOS byteArrayOutputStream = new PublicBAOS(); - final DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream)) { - consumerConfig.serialize(outputStream); - req.body = ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size()); - } + req.body = ConsumerConfig.serialize(consumerConfig); return req; } diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribePollReq.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribePollReq.java index 0188633239f7..696bd1a26b9e 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribePollReq.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribePollReq.java @@ -19,29 +19,18 @@ package org.apache.iotdb.rpc.subscription.payload.request; +import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPollMessage; import org.apache.iotdb.service.rpc.thrift.TPipeSubscribeReq; -import org.apache.iotdb.tsfile.utils.PublicBAOS; -import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; -import java.io.DataOutputStream; import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.HashSet; import java.util.Objects; -import java.util.Set; public class PipeSubscribePollReq extends TPipeSubscribeReq { - private transient Set topicNames = new HashSet<>(); + private transient SubscriptionPollMessage pollMessage; - private transient long timeoutMs; - - public Set getTopicNames() { - return topicNames; - } - - public long getTimeoutMs() { - return timeoutMs; + public SubscriptionPollMessage getPollMessage() { + return pollMessage; } /////////////////////////////// Thrift /////////////////////////////// @@ -50,21 +39,15 @@ public long getTimeoutMs() { * Serialize the incoming parameters into `PipeSubscribePollReq`, called by the subscription * client. */ - public static PipeSubscribePollReq toTPipeSubscribeReq(Set topicNames, long timeoutMs) + public static PipeSubscribePollReq toTPipeSubscribeReq(SubscriptionPollMessage pollMessage) throws IOException { final PipeSubscribePollReq req = new PipeSubscribePollReq(); - req.topicNames = topicNames; - req.timeoutMs = timeoutMs; + req.pollMessage = pollMessage; req.version = PipeSubscribeRequestVersion.VERSION_1.getVersion(); req.type = PipeSubscribeRequestType.POLL.getType(); - try (final PublicBAOS byteArrayOutputStream = new PublicBAOS(); - final DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream)) { - ReadWriteIOUtils.writeObjectSet(topicNames, outputStream); - ReadWriteIOUtils.write(timeoutMs, outputStream); - req.body = ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size()); - } + req.body = SubscriptionPollMessage.serialize(pollMessage); return req; } @@ -74,8 +57,7 @@ public static PipeSubscribePollReq fromTPipeSubscribeReq(TPipeSubscribeReq pollR final PipeSubscribePollReq req = new PipeSubscribePollReq(); if (Objects.nonNull(pollReq.body) && pollReq.body.hasRemaining()) { - req.topicNames = ReadWriteIOUtils.readObjectSet(pollReq.body); - req.timeoutMs = ReadWriteIOUtils.readLong(pollReq.body); + req.pollMessage = SubscriptionPollMessage.deserialize(pollReq.body); } req.version = pollReq.version; @@ -96,8 +78,7 @@ public boolean equals(Object obj) { return false; } PipeSubscribePollReq that = (PipeSubscribePollReq) obj; - return Objects.equals(this.topicNames, that.topicNames) - && this.timeoutMs == that.timeoutMs + return Objects.equals(this.pollMessage, that.pollMessage) && this.version == that.version && this.type == that.type && Objects.equals(this.body, that.body); @@ -105,6 +86,6 @@ public boolean equals(Object obj) { @Override public int hashCode() { - return Objects.hash(topicNames, timeoutMs, version, type, body); + return Objects.hash(pollMessage, pollMessage, version, type, body); } } diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribePollTsFileReq.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribePollTsFileReq.java deleted file mode 100644 index eb9be53d1600..000000000000 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribePollTsFileReq.java +++ /dev/null @@ -1,113 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iotdb.rpc.subscription.payload.request; - -import org.apache.iotdb.service.rpc.thrift.TPipeSubscribeReq; -import org.apache.iotdb.tsfile.utils.PublicBAOS; -import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; - -import java.io.DataOutputStream; -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.Objects; - -public class PipeSubscribePollTsFileReq extends TPipeSubscribeReq { - - private transient String topicName; - - private transient String fileName; - - private transient long endWritingOffset; - - public String getTopicName() { - return topicName; - } - - public String getFileName() { - return fileName; - } - - public long getEndWritingOffset() { - return endWritingOffset; - } - - /////////////////////////////// Thrift /////////////////////////////// - - public static PipeSubscribePollTsFileReq toTPipeSubscribeReq( - String topicName, String fileName, long endWritingOffset) throws IOException { - final PipeSubscribePollTsFileReq req = new PipeSubscribePollTsFileReq(); - - req.topicName = topicName; - req.fileName = fileName; - req.endWritingOffset = endWritingOffset; - - req.version = PipeSubscribeRequestVersion.VERSION_1.getVersion(); - req.type = PipeSubscribeRequestType.POLL_TS_FILE.getType(); - try (final PublicBAOS byteArrayOutputStream = new PublicBAOS(); - final DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream)) { - ReadWriteIOUtils.write(topicName, outputStream); - ReadWriteIOUtils.write(fileName, outputStream); - ReadWriteIOUtils.write(endWritingOffset, outputStream); - req.body = ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size()); - } - - return req; - } - - public static PipeSubscribePollTsFileReq fromTPipeSubscribeReq(TPipeSubscribeReq pollTsFileReq) { - final PipeSubscribePollTsFileReq req = new PipeSubscribePollTsFileReq(); - - if (Objects.nonNull(pollTsFileReq.body) && pollTsFileReq.body.hasRemaining()) { - req.topicName = ReadWriteIOUtils.readString(pollTsFileReq.body); - req.fileName = ReadWriteIOUtils.readString(pollTsFileReq.body); - req.endWritingOffset = ReadWriteIOUtils.readLong(pollTsFileReq.body); - } - - req.version = pollTsFileReq.version; - req.type = pollTsFileReq.type; - req.body = pollTsFileReq.body; - - return req; - } - - /////////////////////////////// Object /////////////////////////////// - - @Override - public boolean equals(Object obj) { - if (this == obj) { - return true; - } - if (obj == null || getClass() != obj.getClass()) { - return false; - } - PipeSubscribePollTsFileReq that = (PipeSubscribePollTsFileReq) obj; - return Objects.equals(this.topicName, that.topicName) - && Objects.equals(this.fileName, that.fileName) - && this.endWritingOffset == that.endWritingOffset - && this.version == that.version - && this.type == that.type - && Objects.equals(this.body, that.body); - } - - @Override - public int hashCode() { - return Objects.hash(topicName, fileName, endWritingOffset, version, type, body); - } -} diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribeRequestType.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribeRequestType.java index 60fefd7ecb82..037481127a14 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribeRequestType.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribeRequestType.java @@ -31,7 +31,6 @@ public enum PipeSubscribeRequestType { CLOSE((short) 4), SUBSCRIBE((short) 5), UNSUBSCRIBE((short) 6), - POLL_TS_FILE((short) 7), ; private final short type; diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribeHandshakeResp.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribeHandshakeResp.java index 932c88a33741..fbf95914b4b2 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribeHandshakeResp.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribeHandshakeResp.java @@ -34,7 +34,6 @@ public class PipeSubscribeHandshakeResp extends TPipeSubscribeResp { - // dataNodeId -> clientRpcEndPoint private transient int dataNodeId; public int getDataNodeId() { @@ -79,10 +78,12 @@ public static PipeSubscribeHandshakeResp fromTPipeSubscribeResp( TPipeSubscribeResp handshakeResp) { final PipeSubscribeHandshakeResp resp = new PipeSubscribeHandshakeResp(); - if (Objects.nonNull(handshakeResp.body) && !handshakeResp.body.isEmpty()) { - ByteBuffer byteBuffer = handshakeResp.body.get(0); - if (byteBuffer.hasRemaining()) { - resp.dataNodeId = ReadWriteIOUtils.readInt(byteBuffer); + if (Objects.nonNull(handshakeResp.body)) { + for (final ByteBuffer byteBuffer : handshakeResp.body) { + if (Objects.nonNull(byteBuffer) && byteBuffer.hasRemaining()) { + resp.dataNodeId = ReadWriteIOUtils.readInt(byteBuffer); + break; + } } } diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollResp.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollResp.java index b06f890874c0..335c8f8ca8f9 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollResp.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollResp.java @@ -22,7 +22,7 @@ import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.rpc.RpcUtils; import org.apache.iotdb.rpc.TSStatusCode; -import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionRawMessage; +import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPolledMessage; import org.apache.iotdb.service.rpc.thrift.TPipeSubscribeResp; import java.io.IOException; @@ -33,9 +33,9 @@ public class PipeSubscribePollResp extends TPipeSubscribeResp { - private transient List messages = new ArrayList<>(); + private transient List messages = new ArrayList<>(); - public List getMessages() { + public List getMessages() { return messages; } @@ -46,7 +46,7 @@ public List getMessages() { * server. */ public static PipeSubscribePollResp toTPipeSubscribeResp( - TSStatus status, List messages) { + TSStatus status, List messages) { final PipeSubscribePollResp resp = new PipeSubscribePollResp(); resp.messages = messages; @@ -56,11 +56,11 @@ public static PipeSubscribePollResp toTPipeSubscribeResp( resp.type = PipeSubscribeResponseType.ACK.getType(); try { resp.body = new ArrayList<>(); - for (final SubscriptionRawMessage message : messages) { + for (final SubscriptionPolledMessage message : messages) { if (Objects.nonNull(message.getByteBuffer())) { resp.body.add(message.getByteBuffer()); } else { - resp.body.add(SubscriptionRawMessage.serialize(message)); + resp.body.add(SubscriptionPolledMessage.serialize(message)); } } } catch (IOException e) { @@ -77,7 +77,7 @@ public static PipeSubscribePollResp fromTPipeSubscribeResp(TPipeSubscribeResp po if (Objects.nonNull(pollResp.body)) { for (final ByteBuffer byteBuffer : pollResp.body) { if (Objects.nonNull(byteBuffer) && byteBuffer.hasRemaining()) { - resp.messages.add(SubscriptionRawMessage.deserialize(byteBuffer)); + resp.messages.add(SubscriptionPolledMessage.deserialize(byteBuffer)); } } } diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionMessage.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionMessage.java index 7b7506908352..fc026d66b5ea 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionMessage.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionMessage.java @@ -37,6 +37,11 @@ public SubscriptionMessage(SubscriptionCommitContext commitContext, List this.payload = new SubscriptionSessionDataSets(tablets); } + public SubscriptionMessage(SubscriptionCommitContext commitContext, String fileName) { + this.commitContext = commitContext; + this.payload = new SubscriptionTsFileReader(fileName); + } + public SubscriptionCommitContext getCommitContext() { return commitContext; } diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java index 6114caaac9a7..770bed8748c8 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java @@ -23,8 +23,11 @@ import org.apache.iotdb.rpc.StatementExecutionException; import org.apache.iotdb.rpc.subscription.config.ConsumerConstant; import org.apache.iotdb.rpc.subscription.exception.SubscriptionException; +import org.apache.iotdb.rpc.subscription.payload.common.PollMessagePayload; import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionCommitContext; -import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionRawMessage; +import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPollMessage; +import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPollMessageType; +import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPolledMessage; import org.apache.thrift.TException; import org.slf4j.Logger; @@ -148,13 +151,20 @@ public List poll(Set topicNames, Duration timeoutMs public List poll(Set topicNames, long timeoutMs) throws TException, IOException, StatementExecutionException { - List rawMessages = new ArrayList<>(); + List rawMessages = new ArrayList<>(); acquireReadLock(); try { for (final SubscriptionProvider provider : getAllAvailableProviders()) { // TODO: network timeout - rawMessages.addAll(provider.getSessionConnection().poll(topicNames, timeoutMs)); + rawMessages.addAll( + provider + .getSessionConnection() + .poll( + new SubscriptionPollMessage( + SubscriptionPollMessageType.POLL.getType(), + new PollMessagePayload(topicNames), + timeoutMs))); } } finally { releaseReadLock(); diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionRawMessageParser.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionRawMessageParser.java index f3659de29ac4..e8499e8f7a20 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionRawMessageParser.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionRawMessageParser.java @@ -19,18 +19,18 @@ package org.apache.iotdb.session.subscription; -import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionRawMessage; -import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionRawMessageType; +import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPolledMessage; +import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPolledMessageType; import org.apache.iotdb.rpc.subscription.payload.common.TabletsMessagePayload; public class SubscriptionRawMessageParser { private SubscriptionRawMessageParser() {} - public static SubscriptionMessage parse(SubscriptionRawMessage rawMessage) { + public static SubscriptionMessage parse(SubscriptionPolledMessage rawMessage) { short messageType = rawMessage.getMessageType(); - if (SubscriptionRawMessageType.isValidatedMessageType(messageType)) { - switch (SubscriptionRawMessageType.valueOf(messageType)) { + if (SubscriptionPolledMessageType.isValidatedMessageType(messageType)) { + switch (SubscriptionPolledMessageType.valueOf(messageType)) { case TABLETS: return new SubscriptionMessage( rawMessage.getCommitContext(), diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionSessionConnection.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionSessionConnection.java index 5fa118f8f76e..e6d957f84b73 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionSessionConnection.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionSessionConnection.java @@ -26,7 +26,8 @@ import org.apache.iotdb.rpc.StatementExecutionException; import org.apache.iotdb.rpc.subscription.config.ConsumerConfig; import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionCommitContext; -import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionRawMessage; +import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPollMessage; +import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPolledMessage; import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribeCloseReq; import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribeCommitReq; import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribeHandshakeReq; @@ -125,15 +126,10 @@ public void unsubscribe(Set topicNames) RpcUtils.verifySuccess(resp.status); } - public List poll(Set topicNames) - throws TException, IOException, StatementExecutionException { - return poll(topicNames, 0); - } - - public List poll(Set topicNames, long timeoutMs) + public List poll(SubscriptionPollMessage pollMessage) throws TException, IOException, StatementExecutionException { TPipeSubscribeResp resp = - client.pipeSubscribe(PipeSubscribePollReq.toTPipeSubscribeReq(topicNames, timeoutMs)); + client.pipeSubscribe(PipeSubscribePollReq.toTPipeSubscribeReq(pollMessage)); RpcUtils.verifySuccess(resp.status); PipeSubscribePollResp pollResp = PipeSubscribePollResp.fromTPipeSubscribeResp(resp); return pollResp.getMessages(); diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionSessionDataSets.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionSessionDataSets.java index 87633457cdc3..44ed814fc38d 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionSessionDataSets.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionSessionDataSets.java @@ -30,18 +30,21 @@ public class SubscriptionSessionDataSets private final List dataSetList; + private final List tablets; + public SubscriptionSessionDataSets(List tablets) { this.dataSetList = new ArrayList<>(); - tablets.forEach((tablet -> this.dataSetList.add(new SubscriptionSessionDataSet(tablet)))); + this.tablets = tablets; } @Override public Iterator iterator() { + tablets.forEach((tablet -> this.dataSetList.add(new SubscriptionSessionDataSet(tablet)))); return dataSetList.iterator(); } public Iterator tabletIterator() { - return dataSetList.stream().map((SubscriptionSessionDataSet::getTablet)).iterator(); + return tablets.iterator(); } @Override diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionTsFileReader.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionTsFileReader.java new file mode 100644 index 000000000000..07cbce223b99 --- /dev/null +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionTsFileReader.java @@ -0,0 +1,31 @@ +package org.apache.iotdb.session.subscription; + +import org.apache.iotdb.rpc.StatementExecutionException; +import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionCommitContext; +import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPolledMessage; +import org.apache.iotdb.tsfile.read.TsFileReader; + +import org.apache.thrift.TException; + +import java.io.IOException; + +public class SubscriptionTsFileReader implements SubscriptionMessagePayload { + + private final String fileName; + + private TsFileReader reader; + + public SubscriptionTsFileReader(String fileName) { + this.fileName = fileName; + } + + public void open(SubscriptionPullConsumer consumer, SubscriptionCommitContext commitContext) + throws TException, IOException, StatementExecutionException { + SubscriptionPolledMessage rawMessage; + } + + @Override + public void close() throws Exception { + reader.close(); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTabletsQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTabletsQueue.java index 5b58ea2cdd32..b891d4c62211 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTabletsQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTabletsQueue.java @@ -31,8 +31,8 @@ import org.apache.iotdb.pipe.api.event.Event; import org.apache.iotdb.pipe.api.event.dml.insertion.TabletInsertionEvent; import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionCommitContext; -import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionRawMessage; -import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionRawMessageType; +import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPolledMessage; +import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPolledMessageType; import org.apache.iotdb.rpc.subscription.payload.common.TabletsMessagePayload; import org.apache.iotdb.tsfile.write.record.Tablet; @@ -156,8 +156,8 @@ private void prefetchOnce(final long limit) { final SubscriptionEvent subscriptionEvent = new SubscriptionEvent( enrichedEvents, - new SubscriptionRawMessage( - SubscriptionRawMessageType.TABLETS.getType(), + new SubscriptionPolledMessage( + SubscriptionPolledMessageType.TABLETS.getType(), new TabletsMessagePayload(tablets), commitContext)); uncommittedEvents.put(commitContext, subscriptionEvent); // before enqueuing the event diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java index 777c6d0fe049..3f3a038f31be 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java @@ -26,8 +26,8 @@ import org.apache.iotdb.db.subscription.timer.SubscriptionPollTimer; import org.apache.iotdb.pipe.api.event.Event; import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionCommitContext; -import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionRawMessage; -import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionRawMessageType; +import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPolledMessage; +import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPolledMessageType; import org.apache.iotdb.rpc.subscription.payload.common.TsFileInfoMessagePayload; import org.slf4j.Logger; @@ -75,8 +75,8 @@ public SubscriptionEvent poll(final SubscriptionPollTimer timer) { final SubscriptionEvent subscriptionEvent = new SubscriptionEvent( Collections.singletonList(tsFileInsertionEvent), - new SubscriptionRawMessage( - SubscriptionRawMessageType.TS_FILE_INFO.getType(), + new SubscriptionPolledMessage( + SubscriptionPolledMessageType.TS_FILE_INFO.getType(), new TsFileInfoMessagePayload(tsFileInsertionEvent.getTsFile().getName()), commitContext)); eventRef.set(subscriptionEvent); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionEvent.java index 29ec5a3d0e76..6011aae89306 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionEvent.java @@ -21,7 +21,7 @@ import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.commons.subscription.config.SubscriptionConfig; -import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionRawMessage; +import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPolledMessage; import java.util.List; @@ -30,13 +30,13 @@ public class SubscriptionEvent { private static final long INVALID_TIMESTAMP = -1; private final List enrichedEvents; - private final SubscriptionRawMessage message; + private final SubscriptionPolledMessage message; private long lastPolledTimestamp; private long committedTimestamp; public SubscriptionEvent( - final List enrichedEvents, final SubscriptionRawMessage message) { + final List enrichedEvents, final SubscriptionPolledMessage message) { this.enrichedEvents = enrichedEvents; this.message = message; @@ -44,7 +44,7 @@ public SubscriptionEvent( this.committedTimestamp = INVALID_TIMESTAMP; } - public SubscriptionRawMessage getMessage() { + public SubscriptionPolledMessage getMessage() { return message; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java index a1ec9259c88f..f90c0b552e16 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java @@ -40,14 +40,17 @@ import org.apache.iotdb.rpc.TSStatusCode; import org.apache.iotdb.rpc.subscription.config.ConsumerConfig; import org.apache.iotdb.rpc.subscription.exception.SubscriptionException; +import org.apache.iotdb.rpc.subscription.payload.common.PollMessagePayload; +import org.apache.iotdb.rpc.subscription.payload.common.PollTsFileMessagePayload; import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionCommitContext; -import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionRawMessage; +import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPollMessage; +import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPollMessageType; +import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPolledMessage; import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribeCloseReq; import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribeCommitReq; import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribeHandshakeReq; import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribeHeartbeatReq; import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribePollReq; -import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribePollTsFileReq; import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribeRequestType; import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribeRequestVersion; import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribeSubscribeReq; @@ -119,9 +122,6 @@ public final TPipeSubscribeResp handle(final TPipeSubscribeReq req) { PipeSubscribeUnsubscribeReq.fromTPipeSubscribeReq(req)); case POLL: return handlePipeSubscribePoll(PipeSubscribePollReq.fromTPipeSubscribeReq(req)); - case POLL_TS_FILE: - return handlePipeSubscribePollTsFile( - PipeSubscribePollTsFileReq.fromTPipeSubscribeReq(req)); case COMMIT: return handlePipeSubscribeCommit(PipeSubscribeCommitReq.fromTPipeSubscribeReq(req)); case CLOSE: @@ -293,8 +293,31 @@ private TPipeSubscribeResp handlePipeSubscribeUnsubscribeInternal( } private TPipeSubscribeResp handlePipeSubscribePoll(final PipeSubscribePollReq req) { + final ConsumerConfig consumerConfig = consumerConfigThreadLocal.get(); + if (Objects.isNull(consumerConfig)) { + LOGGER.warn( + "Subscription: missing consumer config when handling PipeSubscribePollReq: {}", req); + return SUBSCRIPTION_MISSING_CUSTOMER_RESP; + } + try { - return handlePipeSubscribePollInternal(req); + SubscriptionPollMessage pollMessage = req.getPollMessage(); + short messageType = pollMessage.getMessageType(); + if (SubscriptionPollMessageType.isValidatedMessageType(messageType)) { + switch (SubscriptionPollMessageType.valueOf(messageType)) { + case POLL: + return handlePipeSubscribePollInternal( + consumerConfig, + (PollMessagePayload) pollMessage.getMessagePayload(), + pollMessage.getTimeoutMs()); + case POLL_TS_FILE: + return handlePipeSubscribePollTsFileInternal( + consumerConfig, + (PollTsFileMessagePayload) pollMessage.getMessagePayload(), + pollMessage.getTimeoutMs()); + } + } + throw new SubscriptionException("..."); } catch (final SubscriptionException e) { final String exceptionMessage = String.format( @@ -306,17 +329,9 @@ private TPipeSubscribeResp handlePipeSubscribePoll(final PipeSubscribePollReq re } } - private TPipeSubscribeResp handlePipeSubscribePollInternal(final PipeSubscribePollReq req) { - // check consumer config thread local - final ConsumerConfig consumerConfig = consumerConfigThreadLocal.get(); - if (Objects.isNull(consumerConfig)) { - LOGGER.warn( - "Subscription: missing consumer config when handling PipeSubscribePollReq: {}", req); - return SUBSCRIPTION_MISSING_CUSTOMER_RESP; - } - - // get topic names - Set topicNames = req.getTopicNames(); + private TPipeSubscribeResp handlePipeSubscribePollInternal( + ConsumerConfig consumerConfig, PollMessagePayload messagePayload, long timeoutMs) { + Set topicNames = messagePayload.getTopicNames(); if (topicNames.isEmpty()) { // poll all subscribed topics topicNames = @@ -331,20 +346,19 @@ private TPipeSubscribeResp handlePipeSubscribePollInternal(final PipeSubscribePo final SubscriptionPollTimer timer = new SubscriptionPollTimer( System.currentTimeMillis(), - req.getTimeoutMs() == 0 + timeoutMs == 0 ? SubscriptionConfig.getInstance().getSubscriptionDefaultPollTimeoutMs() : Math.max( - req.getTimeoutMs(), - SubscriptionConfig.getInstance().getSubscriptionMinPollTimeoutMs())); + timeoutMs, SubscriptionConfig.getInstance().getSubscriptionMinPollTimeoutMs())); final List events = SubscriptionAgent.broker().poll(consumerConfig, topicNames, timer); - final List rawMessages = + final List rawMessages = events.stream().map(SubscriptionEvent::getMessage).collect(Collectors.toList()); final List commitContexts = rawMessages.stream() - .map(SubscriptionRawMessage::getCommitContext) + .map(SubscriptionPolledMessage::getCommitContext) .collect(Collectors.toList()); // check timer @@ -365,23 +379,9 @@ private TPipeSubscribeResp handlePipeSubscribePollInternal(final PipeSubscribePo return PipeSubscribePollResp.toTPipeSubscribeResp(RpcUtils.SUCCESS_STATUS, rawMessages); } - private TPipeSubscribeResp handlePipeSubscribePollTsFile(final PipeSubscribePollTsFileReq req) { - try { - return handlePipeSubscribePollTsFileInternal(req); - } catch (final SubscriptionException e) { - final String exceptionMessage = - String.format( - "Subscription: something unexpected happened when polling tsfile: %s, req: %s", - e.getMessage(), req); - LOGGER.warn(exceptionMessage); - return PipeSubscribeHandshakeResp.toTPipeSubscribeResp( - RpcUtils.getStatus(TSStatusCode.SUBSCRIPTION_POLL_ERROR, exceptionMessage)); - } - } - // TODO private TPipeSubscribeResp handlePipeSubscribePollTsFileInternal( - final PipeSubscribePollTsFileReq req) { + ConsumerConfig consumerConfig, PollTsFileMessagePayload messagePayload, long timeoutMs) { return null; } From 1113eae91f0d8b6b4808a198e1b5b6bc120af930 Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Sat, 13 Apr 2024 17:34:58 +0800 Subject: [PATCH 13/93] fix --- .../payload/common/PollMessagePayload.java | 19 +++++++++++++++++++ .../common/PollTsFileMessagePayload.java | 19 +++++++++++++++++++ .../common/SubscriptionPollMessageType.java | 19 +++++++++++++++++++ .../SubscriptionTsFileReader.java | 19 +++++++++++++++++++ 4 files changed, 76 insertions(+) diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/PollMessagePayload.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/PollMessagePayload.java index 8446368fa2a1..6f8f08a3e9ec 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/PollMessagePayload.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/PollMessagePayload.java @@ -1,3 +1,22 @@ +/* + * 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.rpc.subscription.payload.common; import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/PollTsFileMessagePayload.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/PollTsFileMessagePayload.java index d2047e4703d1..e8956250de8b 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/PollTsFileMessagePayload.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/PollTsFileMessagePayload.java @@ -1,3 +1,22 @@ +/* + * 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.rpc.subscription.payload.common; import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPollMessageType.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPollMessageType.java index 2d5ba173d7e4..bfdefa58e864 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPollMessageType.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPollMessageType.java @@ -1,3 +1,22 @@ +/* + * 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.rpc.subscription.payload.common; import java.util.Arrays; diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionTsFileReader.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionTsFileReader.java index 07cbce223b99..942203fee5eb 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionTsFileReader.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionTsFileReader.java @@ -1,3 +1,22 @@ +/* + * 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.session.subscription; import org.apache.iotdb.rpc.StatementExecutionException; From 7d5bc5af9c20c72a4e12b5480418d655df150dd8 Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Tue, 16 Apr 2024 17:33:10 +0800 Subject: [PATCH 14/93] backup --- .../common/PollTsFileMessagePayload.java | 8 ++ .../common/SubscriptionPolledMessage.java | 6 ++ .../common/TsFileInfoMessagePayload.java | 6 +- .../common/TsFilePieceMessagePayload.java | 60 ++++++++++++++ .../common/TsFileSealMessagePayload.java | 53 +++++++++++++ .../subscription/SubscriptionConsumer.java | 7 +- .../subscription/SubscriptionMessage.java | 1 - .../SubscriptionMessagePayload.java | 5 +- ...a => SubscriptionPolledMessageParser.java} | 26 ++++-- .../SubscriptionPullConsumer.java | 33 +++++++- .../SubscriptionSessionDataSets.java | 7 +- .../SubscriptionTsFileReader.java | 14 +--- .../agent/SubscriptionBrokerAgent.java | 12 +++ .../broker/SubscriptionBroker.java | 15 ++++ .../SubscriptionPrefetchingTsFileQueue.java | 79 ++++++++++++++++++- .../subscription/event/SubscriptionEvent.java | 4 + .../receiver/SubscriptionReceiverV1.java | 72 ++++++++++++----- .../iotdb/commons/conf/CommonConfig.java | 9 +++ .../iotdb/commons/conf/CommonDescriptor.java | 5 ++ .../config/SubscriptionConfig.java | 5 ++ 20 files changed, 379 insertions(+), 48 deletions(-) create mode 100644 iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFilePieceMessagePayload.java create mode 100644 iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileSealMessagePayload.java rename iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/{SubscriptionRawMessageParser.java => SubscriptionPolledMessageParser.java} (60%) diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/PollTsFileMessagePayload.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/PollTsFileMessagePayload.java index e8956250de8b..7f73cc46aa69 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/PollTsFileMessagePayload.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/PollTsFileMessagePayload.java @@ -45,6 +45,14 @@ public long getEndWritingOffset() { return endWritingOffset; } + public PollTsFileMessagePayload() {} + + public PollTsFileMessagePayload(String topicName, String fileName, long endWritingOffset) { + this.topicName = topicName; + this.fileName = fileName; + this.endWritingOffset = endWritingOffset; + } + @Override public void serialize(DataOutputStream stream) throws IOException { ReadWriteIOUtils.write(topicName, stream); diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledMessage.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledMessage.java index c835bd32ab92..cefa0b1c5109 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledMessage.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledMessage.java @@ -97,6 +97,12 @@ public static SubscriptionPolledMessage deserialize(final ByteBuffer buffer) { case TS_FILE_INFO: messagePayload = new TsFileInfoMessagePayload().deserialize(buffer); break; + case TS_FILE_PIECE: + messagePayload = new TsFilePieceMessagePayload().deserialize(buffer); + break; + case TS_FILE_SEAL: + messagePayload = new TsFileSealMessagePayload().deserialize(buffer); + break; default: messagePayload = null; } diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileInfoMessagePayload.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileInfoMessagePayload.java index d2c83175c546..ec834ef9d3d1 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileInfoMessagePayload.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileInfoMessagePayload.java @@ -27,7 +27,11 @@ public class TsFileInfoMessagePayload implements SubscriptionMessagePayload { - protected transient String fileName; + private transient String fileName; + + public String getFileName() { + return fileName; + } public TsFileInfoMessagePayload() {} diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFilePieceMessagePayload.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFilePieceMessagePayload.java new file mode 100644 index 000000000000..9ff79b7de06b --- /dev/null +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFilePieceMessagePayload.java @@ -0,0 +1,60 @@ +/* + * 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.rpc.subscription.payload.common; + +import org.apache.iotdb.tsfile.utils.Binary; +import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; + +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; + +public class TsFilePieceMessagePayload implements SubscriptionMessagePayload { + + private transient String fileName; + + private transient long endWritingOffset; + + private transient byte[] filePiece; + + public TsFilePieceMessagePayload() {} + + public TsFilePieceMessagePayload(String fileName, long endWritingOffset, byte[] filePiece) { + this.fileName = fileName; + this.endWritingOffset = endWritingOffset; + this.filePiece = filePiece; + } + + @Override + public void serialize(DataOutputStream stream) throws IOException { + ReadWriteIOUtils.write(fileName, stream); + ReadWriteIOUtils.write(endWritingOffset, stream); + ReadWriteIOUtils.write(new Binary(filePiece), stream); + } + + @Override + public SubscriptionMessagePayload deserialize(ByteBuffer buffer) { + this.fileName = ReadWriteIOUtils.readString(buffer); + this.endWritingOffset = ReadWriteIOUtils.readLong(buffer); + final int size = ReadWriteIOUtils.readInt(buffer); + this.filePiece = ReadWriteIOUtils.readBytes(buffer, size); + return this; + } +} diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileSealMessagePayload.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileSealMessagePayload.java new file mode 100644 index 000000000000..d9baf23610db --- /dev/null +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileSealMessagePayload.java @@ -0,0 +1,53 @@ +/* + * 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.rpc.subscription.payload.common; + +import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; + +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; + +public class TsFileSealMessagePayload implements SubscriptionMessagePayload { + + private transient String fileName; + + private transient long fileLength; + + public TsFileSealMessagePayload() {} + + public TsFileSealMessagePayload(String fileName, long fileLength) { + this.fileName = fileName; + this.fileLength = fileLength; + } + + @Override + public void serialize(DataOutputStream stream) throws IOException { + ReadWriteIOUtils.write(fileName, stream); + ReadWriteIOUtils.write(fileLength, stream); + } + + @Override + public SubscriptionMessagePayload deserialize(ByteBuffer buffer) { + this.fileName = ReadWriteIOUtils.readString(buffer); + this.fileLength = ReadWriteIOUtils.readLong(buffer); + return this; + } +} diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java index 4f93c97e7954..a6964ce6e662 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java @@ -40,8 +40,7 @@ import java.util.Objects; import java.util.Properties; import java.util.Set; -import java.util.SortedMap; -import java.util.concurrent.ConcurrentSkipListMap; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; @@ -67,8 +66,8 @@ public abstract class SubscriptionConsumer implements AutoCloseable { private final long heartbeatIntervalMs; private final long endpointsSyncIntervalMs; - private final SortedMap subscriptionProviders = - new ConcurrentSkipListMap<>(); + private final Map subscriptionProviders = + new ConcurrentHashMap<>(); private final ReentrantReadWriteLock subscriptionProvidersLock = new ReentrantReadWriteLock(true); private ScheduledExecutorService heartbeatWorkerExecutor; diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionMessage.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionMessage.java index fc026d66b5ea..60485f03ad8d 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionMessage.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionMessage.java @@ -29,7 +29,6 @@ public class SubscriptionMessage implements Comparable { private final SubscriptionCommitContext commitContext; - // TODO: support more data format private final SubscriptionMessagePayload payload; public SubscriptionMessage(SubscriptionCommitContext commitContext, List tablets) { diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionMessagePayload.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionMessagePayload.java index 41068c701493..5953f4a654bf 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionMessagePayload.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionMessagePayload.java @@ -19,4 +19,7 @@ package org.apache.iotdb.session.subscription; -public interface SubscriptionMessagePayload extends AutoCloseable {} +public interface SubscriptionMessagePayload extends AutoCloseable { + + void open() throws Exception; +} diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionRawMessageParser.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPolledMessageParser.java similarity index 60% rename from iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionRawMessageParser.java rename to iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPolledMessageParser.java index e8499e8f7a20..49fe43f2b59f 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionRawMessageParser.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPolledMessageParser.java @@ -22,19 +22,33 @@ import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPolledMessage; import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPolledMessageType; import org.apache.iotdb.rpc.subscription.payload.common.TabletsMessagePayload; +import org.apache.iotdb.rpc.subscription.payload.common.TsFileInfoMessagePayload; -public class SubscriptionRawMessageParser { +public class SubscriptionPolledMessageParser { - private SubscriptionRawMessageParser() {} + private SubscriptionPolledMessageParser() {} - public static SubscriptionMessage parse(SubscriptionPolledMessage rawMessage) { - short messageType = rawMessage.getMessageType(); + public static SubscriptionMessage parse( + SubscriptionPullConsumer consumer, SubscriptionPolledMessage polledMessage) { + short messageType = polledMessage.getMessageType(); if (SubscriptionPolledMessageType.isValidatedMessageType(messageType)) { switch (SubscriptionPolledMessageType.valueOf(messageType)) { case TABLETS: return new SubscriptionMessage( - rawMessage.getCommitContext(), - ((TabletsMessagePayload) rawMessage.getMessagePayload()).getTablets()); + polledMessage.getCommitContext(), + ((TabletsMessagePayload) polledMessage.getMessagePayload()).getTablets()); + case TS_FILE_INFO: + // TODO + try { + consumer.pollTsFile( + polledMessage.getCommitContext().getDataNodeId(), + polledMessage.getCommitContext().getTopicName(), + ((TsFileInfoMessagePayload) polledMessage.getMessagePayload()).getFileName(), + 0, + 0L); + } catch (Exception e) { + + } } } return null; diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java index 770bed8748c8..511fb04566f1 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java @@ -24,6 +24,7 @@ import org.apache.iotdb.rpc.subscription.config.ConsumerConstant; import org.apache.iotdb.rpc.subscription.exception.SubscriptionException; import org.apache.iotdb.rpc.subscription.payload.common.PollMessagePayload; +import org.apache.iotdb.rpc.subscription.payload.common.PollTsFileMessagePayload; import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionCommitContext; import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPollMessage; import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPollMessageType; @@ -151,13 +152,13 @@ public List poll(Set topicNames, Duration timeoutMs public List poll(Set topicNames, long timeoutMs) throws TException, IOException, StatementExecutionException { - List rawMessages = new ArrayList<>(); + List polledMessages = new ArrayList<>(); acquireReadLock(); try { for (final SubscriptionProvider provider : getAllAvailableProviders()) { // TODO: network timeout - rawMessages.addAll( + polledMessages.addAll( provider .getSessionConnection() .poll( @@ -171,7 +172,9 @@ public List poll(Set topicNames, long timeoutMs) } List messages = - rawMessages.stream().map(SubscriptionRawMessageParser::parse).collect(Collectors.toList()); + polledMessages.stream() + .map(polledMessage -> SubscriptionPolledMessageParser.parse(this, polledMessage)) + .collect(Collectors.toList()); if (autoCommit) { long currentTimestamp = System.currentTimeMillis(); @@ -187,6 +190,30 @@ public List poll(Set topicNames, long timeoutMs) return messages; } + List pollTsFile( + int dataNodeId, String topicName, String fileName, long endWritingOffset, long timeoutMs) + throws TException, IOException, StatementExecutionException, IoTDBConnectionException { + acquireReadLock(); + try { + final SubscriptionProvider provider = getProvider(dataNodeId); + if (Objects.isNull(provider) || !provider.isAvailable()) { + throw new IoTDBConnectionException( + String.format( + "something unexpected happened when poll tsfile from subscription provider with data node id %s, the subscription provider may be unavailable or not existed", + dataNodeId)); + } + return provider + .getSessionConnection() + .poll( + new SubscriptionPollMessage( + SubscriptionPollMessageType.POLL_TS_FILE.getType(), + new PollTsFileMessagePayload(topicName, fileName, endWritingOffset), + timeoutMs)); + } finally { + releaseReadLock(); + } + } + public void commitSync(SubscriptionMessage message) throws TException, IOException, StatementExecutionException, IoTDBConnectionException { commitSync(Collections.singletonList(message)); diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionSessionDataSets.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionSessionDataSets.java index 44ed814fc38d..94a61a90d6af 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionSessionDataSets.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionSessionDataSets.java @@ -35,11 +35,11 @@ public class SubscriptionSessionDataSets public SubscriptionSessionDataSets(List tablets) { this.dataSetList = new ArrayList<>(); this.tablets = tablets; + tablets.forEach((tablet -> this.dataSetList.add(new SubscriptionSessionDataSet(tablet)))); } @Override public Iterator iterator() { - tablets.forEach((tablet -> this.dataSetList.add(new SubscriptionSessionDataSet(tablet)))); return dataSetList.iterator(); } @@ -47,6 +47,11 @@ public Iterator tabletIterator() { return tablets.iterator(); } + @Override + public void open() { + // do nothing + } + @Override public void close() throws Exception { for (SubscriptionSessionDataSet dataSet : dataSetList) { diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionTsFileReader.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionTsFileReader.java index 942203fee5eb..95179c351fc0 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionTsFileReader.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionTsFileReader.java @@ -19,14 +19,8 @@ package org.apache.iotdb.session.subscription; -import org.apache.iotdb.rpc.StatementExecutionException; -import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionCommitContext; -import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPolledMessage; import org.apache.iotdb.tsfile.read.TsFileReader; - -import org.apache.thrift.TException; - -import java.io.IOException; +import org.apache.iotdb.tsfile.read.TsFileSequenceReader; public class SubscriptionTsFileReader implements SubscriptionMessagePayload { @@ -38,9 +32,9 @@ public SubscriptionTsFileReader(String fileName) { this.fileName = fileName; } - public void open(SubscriptionPullConsumer consumer, SubscriptionCommitContext commitContext) - throws TException, IOException, StatementExecutionException { - SubscriptionPolledMessage rawMessage; + @Override + public void open() throws Exception { + reader = new TsFileReader(new TsFileSequenceReader(fileName)); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionBrokerAgent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionBrokerAgent.java index 452b86b7a231..89b5107aefc9 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionBrokerAgent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionBrokerAgent.java @@ -60,6 +60,18 @@ public List poll( return broker.poll(topicNames, timer); } + public List pollTsFile( + ConsumerConfig consumerConfig, String topicName, String fileName, long endWritingOffset) { + final String consumerGroupId = consumerConfig.getConsumerGroupId(); + final SubscriptionBroker broker = consumerGroupIdToSubscriptionBroker.get(consumerGroupId); + if (Objects.isNull(broker)) { + LOGGER.warn( + "Subscription: broker bound to consumer group [{}] does not exist", consumerGroupId); + return Collections.emptyList(); + } + return broker.pollTsFile(topicName, fileName, endWritingOffset); + } + public void commit( final ConsumerConfig consumerConfig, final List commitContexts) { final String consumerGroupId = consumerConfig.getConsumerGroupId(); 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 72d0aa0f392b..5f57ddafe2bc 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 @@ -31,6 +31,7 @@ import org.slf4j.LoggerFactory; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Objects; @@ -77,6 +78,20 @@ public List poll( return events; } + public List pollTsFile( + String topicName, String fileName, long endWritingOffset) { + SubscriptionPrefetchingQueue prefetchingQueue = topicNameToPrefetchingQueue.get(topicName); + if (Objects.isNull(prefetchingQueue)) { + return null; + } + if (!(prefetchingQueue instanceof SubscriptionPrefetchingTsFileQueue)) { + return null; + } + return Collections.singletonList( + ((SubscriptionPrefetchingTsFileQueue) prefetchingQueue) + .pollTsFile(fileName, endWritingOffset)); + } + public void commit(final List commitContexts) { for (final SubscriptionCommitContext commitContext : commitContexts) { final String topicName = commitContext.getTopicName(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java index 3f3a038f31be..c9fe9946aa26 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java @@ -19,21 +19,30 @@ 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.subscription.config.SubscriptionConfig; import org.apache.iotdb.db.pipe.event.UserDefinedEnrichedEvent; import org.apache.iotdb.db.pipe.event.common.tsfile.PipeTsFileInsertionEvent; import org.apache.iotdb.db.subscription.event.SubscriptionEvent; import org.apache.iotdb.db.subscription.timer.SubscriptionPollTimer; import org.apache.iotdb.pipe.api.event.Event; import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionCommitContext; +import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionMessagePayload; import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPolledMessage; import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPolledMessageType; import org.apache.iotdb.rpc.subscription.payload.common.TsFileInfoMessagePayload; +import org.apache.iotdb.rpc.subscription.payload.common.TsFilePieceMessagePayload; +import org.apache.iotdb.rpc.subscription.payload.common.TsFileSealMessagePayload; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.IOException; +import java.io.RandomAccessFile; +import java.util.Arrays; import java.util.Collections; +import java.util.List; import java.util.Objects; import java.util.concurrent.atomic.AtomicReference; @@ -80,13 +89,81 @@ public SubscriptionEvent poll(final SubscriptionPollTimer timer) { new TsFileInfoMessagePayload(tsFileInsertionEvent.getTsFile().getName()), commitContext)); eventRef.set(subscriptionEvent); - uncommittedEvents.put(commitContext, subscriptionEvent); + // don't allow commit now return subscriptionEvent; } return null; } + public SubscriptionEvent pollTsFile(String fileName, long endWritingOffset) { + final SubscriptionEvent event = eventRef.get(); + if (Objects.isNull(event)) { + return null; + } + + final SubscriptionPolledMessage polledMessage = event.getMessage(); + final List enrichedEvents = event.getEnrichedEvents(); + final PipeTsFileInsertionEvent tsFileInsertionEvent = + (PipeTsFileInsertionEvent) enrichedEvents.get(0); + final SubscriptionMessagePayload messagePayload = polledMessage.getMessagePayload(); + final SubscriptionCommitContext commitContext = polledMessage.getCommitContext(); + + if (messagePayload instanceof TsFileInfoMessagePayload) { + if (!fileName.equals(((TsFileInfoMessagePayload) messagePayload).getFileName())) { + return null; + } + } else if (messagePayload instanceof TsFilePieceMessagePayload) { + final int readFileBufferSize = + SubscriptionConfig.getInstance().getSubscriptionReadFileBufferSize(); + final byte[] readBuffer = new byte[readFileBufferSize]; + try (final RandomAccessFile reader = + new RandomAccessFile(tsFileInsertionEvent.getTsFile(), "r")) { + while (true) { + reader.seek(endWritingOffset); + final int readLength = reader.read(readBuffer); + if (readLength == -1) { + break; + } + final byte[] filePiece = + readLength == readFileBufferSize + ? readBuffer + : Arrays.copyOfRange(readBuffer, 0, readLength); + + // poll tsfile piece + final SubscriptionEvent newEvent = + new SubscriptionEvent( + Collections.singletonList(tsFileInsertionEvent), + new SubscriptionPolledMessage( + SubscriptionPolledMessageType.TS_FILE_PIECE.getType(), + new TsFilePieceMessagePayload( + fileName, endWritingOffset + readLength, filePiece), + commitContext)); + eventRef.set(newEvent); + return newEvent; + } + + // poll tsfile seal + final SubscriptionEvent newEvent = + new SubscriptionEvent( + Collections.singletonList(tsFileInsertionEvent), + new SubscriptionPolledMessage( + SubscriptionPolledMessageType.TS_FILE_SEAL.getType(), + new TsFileSealMessagePayload( + fileName, tsFileInsertionEvent.getTsFile().length()), + commitContext)); + eventRef.set(newEvent); + // allow commit now + uncommittedEvents.put(commitContext, newEvent); + return newEvent; + } catch (IOException e) { + LOGGER.warn(e.getMessage()); + } + } + + return null; + } + @Override public void executePrefetch() { // do nothing now diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionEvent.java index 6011aae89306..9e539ce695e0 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionEvent.java @@ -44,6 +44,10 @@ public SubscriptionEvent( this.committedTimestamp = INVALID_TIMESTAMP; } + public List getEnrichedEvents() { + return enrichedEvents; + } + public SubscriptionPolledMessage getMessage() { return message; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java index f90c0b552e16..ae52ff1ec50c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java @@ -303,18 +303,25 @@ private TPipeSubscribeResp handlePipeSubscribePoll(final PipeSubscribePollReq re try { SubscriptionPollMessage pollMessage = req.getPollMessage(); short messageType = pollMessage.getMessageType(); + + long timeoutMs = pollMessage.getTimeoutMs(); + final SubscriptionPollTimer timer = + new SubscriptionPollTimer( + System.currentTimeMillis(), + timeoutMs == 0 + ? SubscriptionConfig.getInstance().getSubscriptionDefaultPollTimeoutMs() + : Math.max( + timeoutMs, + SubscriptionConfig.getInstance().getSubscriptionMinPollTimeoutMs())); + if (SubscriptionPollMessageType.isValidatedMessageType(messageType)) { switch (SubscriptionPollMessageType.valueOf(messageType)) { case POLL: return handlePipeSubscribePollInternal( - consumerConfig, - (PollMessagePayload) pollMessage.getMessagePayload(), - pollMessage.getTimeoutMs()); + consumerConfig, (PollMessagePayload) pollMessage.getMessagePayload(), timer); case POLL_TS_FILE: return handlePipeSubscribePollTsFileInternal( - consumerConfig, - (PollTsFileMessagePayload) pollMessage.getMessagePayload(), - pollMessage.getTimeoutMs()); + consumerConfig, (PollTsFileMessagePayload) pollMessage.getMessagePayload(), timer); } } throw new SubscriptionException("..."); @@ -330,7 +337,9 @@ private TPipeSubscribeResp handlePipeSubscribePoll(final PipeSubscribePollReq re } private TPipeSubscribeResp handlePipeSubscribePollInternal( - ConsumerConfig consumerConfig, PollMessagePayload messagePayload, long timeoutMs) { + ConsumerConfig consumerConfig, + PollMessagePayload messagePayload, + SubscriptionPollTimer timer) { Set topicNames = messagePayload.getTopicNames(); if (topicNames.isEmpty()) { // poll all subscribed topics @@ -343,21 +352,14 @@ private TPipeSubscribeResp handlePipeSubscribePollInternal( } // poll - final SubscriptionPollTimer timer = - new SubscriptionPollTimer( - System.currentTimeMillis(), - timeoutMs == 0 - ? SubscriptionConfig.getInstance().getSubscriptionDefaultPollTimeoutMs() - : Math.max( - timeoutMs, SubscriptionConfig.getInstance().getSubscriptionMinPollTimeoutMs())); final List events = SubscriptionAgent.broker().poll(consumerConfig, topicNames, timer); - final List rawMessages = + final List polledMessages = events.stream().map(SubscriptionEvent::getMessage).collect(Collectors.toList()); final List commitContexts = - rawMessages.stream() + polledMessages.stream() .map(SubscriptionPolledMessage::getCommitContext) .collect(Collectors.toList()); @@ -376,13 +378,43 @@ private TPipeSubscribeResp handlePipeSubscribePollInternal( commitContexts); // generate response - return PipeSubscribePollResp.toTPipeSubscribeResp(RpcUtils.SUCCESS_STATUS, rawMessages); + return PipeSubscribePollResp.toTPipeSubscribeResp(RpcUtils.SUCCESS_STATUS, polledMessages); } - // TODO private TPipeSubscribeResp handlePipeSubscribePollTsFileInternal( - ConsumerConfig consumerConfig, PollTsFileMessagePayload messagePayload, long timeoutMs) { - return null; + ConsumerConfig consumerConfig, + PollTsFileMessagePayload messagePayload, + SubscriptionPollTimer timer) { + // TODO: timer + + // poll + final List events = + SubscriptionAgent.broker() + .pollTsFile( + consumerConfig, + messagePayload.getTopicName(), + messagePayload.getFileName(), + messagePayload.getEndWritingOffset()); + + final List polledMessages = + events.stream().map(SubscriptionEvent::getMessage).collect(Collectors.toList()); + + final List commitContexts = + polledMessages.stream() + .map(SubscriptionPolledMessage::getCommitContext) + .collect(Collectors.toList()); + + // check timer + if (timer.isExpired()) { + LOGGER.warn("Subscription: timeout happened when consumer {} poll tsfile", consumerConfig); + } + + LOGGER.info( + "Subscription: consumer {} poll tsfile, commit contexts: {}", + consumerConfig, + commitContexts); + + return PipeSubscribePollResp.toTPipeSubscribeResp(RpcUtils.SUCCESS_STATUS, polledMessages); } private TPipeSubscribeResp handlePipeSubscribeCommit(final PipeSubscribeCommitReq req) { 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 22a8616ee95e..1148aa79609c 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 @@ -237,6 +237,7 @@ public class CommonConfig { private int subscriptionRecycleUncommittedEventIntervalMs = 240000; // 240s private long subscriptionDefaultPollTimeoutMs = 30000; private long subscriptionMinPollTimeoutMs = 500; + private int subscriptionReadFileBufferSize = 8388608; /** Whether to use persistent schema mode. */ private String schemaEngineMode = "Memory"; @@ -1032,6 +1033,14 @@ public void setSubscriptionMinPollTimeoutMs(long subscriptionMinPollTimeoutMs) { this.subscriptionMinPollTimeoutMs = subscriptionMinPollTimeoutMs; } + public int getSubscriptionReadFileBufferSize() { + return subscriptionReadFileBufferSize; + } + + public void setSubscriptionReadFileBufferSize(int subscriptionReadFileBufferSize) { + this.subscriptionReadFileBufferSize = subscriptionReadFileBufferSize; + } + public String getSchemaEngineMode() { return schemaEngineMode; } 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 4e347ccd41a1..c9cd97640414 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 @@ -562,6 +562,11 @@ private void loadSubscriptionProps(Properties properties) { properties.getProperty( "subscription_min_poll_timeout_ms", String.valueOf(config.getSubscriptionMinPollTimeoutMs())))); + config.setSubscriptionReadFileBufferSize( + Integer.parseInt( + properties.getProperty( + "subscription_read_file_buffer_size", + String.valueOf(config.getSubscriptionReadFileBufferSize())))); } public void loadGlobalConfig(TGlobalConfig globalConfig) { diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/subscription/config/SubscriptionConfig.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/subscription/config/SubscriptionConfig.java index a6ba77b34cae..afa4346d2622 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/subscription/config/SubscriptionConfig.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/subscription/config/SubscriptionConfig.java @@ -63,6 +63,10 @@ public long getSubscriptionMinPollTimeoutMs() { return COMMON_CONFIG.getSubscriptionMinPollTimeoutMs(); } + public int getSubscriptionReadFileBufferSize() { + return COMMON_CONFIG.getSubscriptionReadFileBufferSize(); + } + /////////////////////////////// Utils /////////////////////////////// private static final Logger LOGGER = LoggerFactory.getLogger(SubscriptionConfig.class); @@ -82,6 +86,7 @@ public void printAllConfigs() { getSubscriptionRecycleUncommittedEventIntervalMs()); LOGGER.info("SubscriptionDefaultPollTimeoutMs: {}", getSubscriptionDefaultPollTimeoutMs()); LOGGER.info("SubscriptionMinPollTimeoutMs: {}", getSubscriptionMinPollTimeoutMs()); + LOGGER.info("SubscriptionReadFileBufferSize: {}", getSubscriptionReadFileBufferSize()); } /////////////////////////////// Singleton /////////////////////////////// From 1cf8d99c76ce918f2fd537ad64030b8ab164b386 Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Tue, 16 Apr 2024 19:29:17 +0800 Subject: [PATCH 15/93] backup --- .../common/TsFilePieceMessagePayload.java | 12 ++ .../common/TsFileSealMessagePayload.java | 8 ++ .../subscription/SubscriptionConsumer.java | 28 ++++- .../SubscriptionMessagePayload.java | 5 +- .../SubscriptionPolledMessageParser.java | 56 --------- .../SubscriptionPullConsumer.java | 114 ++++++++++++++++-- .../SubscriptionSessionDataSets.java | 12 -- .../SubscriptionTsFileReader.java | 14 +-- 8 files changed, 155 insertions(+), 94 deletions(-) delete mode 100644 iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPolledMessageParser.java diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFilePieceMessagePayload.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFilePieceMessagePayload.java index 9ff79b7de06b..872b2757a8de 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFilePieceMessagePayload.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFilePieceMessagePayload.java @@ -34,6 +34,18 @@ public class TsFilePieceMessagePayload implements SubscriptionMessagePayload { private transient byte[] filePiece; + public String getFileName() { + return fileName; + } + + public long getEndWritingOffset() { + return endWritingOffset; + } + + public byte[] getFilePiece() { + return filePiece; + } + public TsFilePieceMessagePayload() {} public TsFilePieceMessagePayload(String fileName, long endWritingOffset, byte[] filePiece) { diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileSealMessagePayload.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileSealMessagePayload.java index d9baf23610db..2354174eddc3 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileSealMessagePayload.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileSealMessagePayload.java @@ -31,6 +31,14 @@ public class TsFileSealMessagePayload implements SubscriptionMessagePayload { private transient long fileLength; + public String getFileName() { + return fileName; + } + + public long getFileLength() { + return fileLength; + } + public TsFileSealMessagePayload() {} public TsFileSealMessagePayload(String fileName, long fileLength) { diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java index a6964ce6e662..188244da7637 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java @@ -24,13 +24,19 @@ import org.apache.iotdb.rpc.IoTDBConnectionException; import org.apache.iotdb.rpc.StatementExecutionException; import org.apache.iotdb.rpc.subscription.config.ConsumerConstant; +import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionCommitContext; import org.apache.iotdb.session.util.SessionUtils; +import org.apache.iotdb.tsfile.utils.Pair; import org.apache.thrift.TException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.File; import java.io.IOException; +import java.io.RandomAccessFile; +import java.nio.file.Files; +import java.nio.file.Path; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -60,8 +66,8 @@ public abstract class SubscriptionConsumer implements AutoCloseable { private final String username; private final String password; - private final String consumerId; - private final String consumerGroupId; + protected final String consumerId; + protected final String consumerGroupId; private final long heartbeatIntervalMs; private final long endpointsSyncIntervalMs; @@ -83,6 +89,24 @@ public String getConsumerGroupId() { return consumerGroupId; } + /////////////////////////////// tsfile dir /////////////////////////////// + + protected Path subscribedTsFileBaseDirPath; + + protected final Map> + commitContextToTsFile = new ConcurrentHashMap<>(); + + public Path getTsFileDir(String topicName) throws IOException { + if (Objects.isNull(subscribedTsFileBaseDirPath)) { + subscribedTsFileBaseDirPath = + Files.createTempDirectory( + String.format("subscribedTsFile_%s_%s", consumerId, consumerGroupId)); + } + Path dirPath = subscribedTsFileBaseDirPath.resolve(topicName); + Files.createDirectories(dirPath); + return dirPath; + } + /////////////////////////////// ctor /////////////////////////////// protected SubscriptionConsumer(Builder builder) { diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionMessagePayload.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionMessagePayload.java index 5953f4a654bf..0408c9da45a9 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionMessagePayload.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionMessagePayload.java @@ -19,7 +19,4 @@ package org.apache.iotdb.session.subscription; -public interface SubscriptionMessagePayload extends AutoCloseable { - - void open() throws Exception; -} +public interface SubscriptionMessagePayload {} diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPolledMessageParser.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPolledMessageParser.java deleted file mode 100644 index 49fe43f2b59f..000000000000 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPolledMessageParser.java +++ /dev/null @@ -1,56 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iotdb.session.subscription; - -import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPolledMessage; -import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPolledMessageType; -import org.apache.iotdb.rpc.subscription.payload.common.TabletsMessagePayload; -import org.apache.iotdb.rpc.subscription.payload.common.TsFileInfoMessagePayload; - -public class SubscriptionPolledMessageParser { - - private SubscriptionPolledMessageParser() {} - - public static SubscriptionMessage parse( - SubscriptionPullConsumer consumer, SubscriptionPolledMessage polledMessage) { - short messageType = polledMessage.getMessageType(); - if (SubscriptionPolledMessageType.isValidatedMessageType(messageType)) { - switch (SubscriptionPolledMessageType.valueOf(messageType)) { - case TABLETS: - return new SubscriptionMessage( - polledMessage.getCommitContext(), - ((TabletsMessagePayload) polledMessage.getMessagePayload()).getTablets()); - case TS_FILE_INFO: - // TODO - try { - consumer.pollTsFile( - polledMessage.getCommitContext().getDataNodeId(), - polledMessage.getCommitContext().getTopicName(), - ((TsFileInfoMessagePayload) polledMessage.getMessagePayload()).getFileName(), - 0, - 0L); - } catch (Exception e) { - - } - } - } - return null; - } -} diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java index 511fb04566f1..8fa88f2dc0d0 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java @@ -29,12 +29,22 @@ import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPollMessage; import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPollMessageType; import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPolledMessage; +import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPolledMessageType; +import org.apache.iotdb.rpc.subscription.payload.common.TabletsMessagePayload; +import org.apache.iotdb.rpc.subscription.payload.common.TsFileInfoMessagePayload; +import org.apache.iotdb.rpc.subscription.payload.common.TsFilePieceMessagePayload; +import org.apache.iotdb.rpc.subscription.payload.common.TsFileSealMessagePayload; +import org.apache.iotdb.tsfile.utils.Pair; import org.apache.thrift.TException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.File; import java.io.IOException; +import java.io.RandomAccessFile; +import java.nio.file.Files; +import java.nio.file.Path; import java.time.Duration; import java.util.ArrayList; import java.util.Collections; @@ -51,7 +61,6 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.stream.Collectors; public class SubscriptionPullConsumer extends SubscriptionConsumer { @@ -65,6 +74,11 @@ public class SubscriptionPullConsumer extends SubscriptionConsumer { private final AtomicBoolean isClosed = new AtomicBoolean(true); + @Override + boolean isClosed() { + return isClosed.get(); + } + /////////////////////////////// ctor /////////////////////////////// public SubscriptionPullConsumer(SubscriptionPullConsumer.Builder builder) { @@ -171,10 +185,37 @@ public List poll(Set topicNames, long timeoutMs) releaseReadLock(); } - List messages = - polledMessages.stream() - .map(polledMessage -> SubscriptionPolledMessageParser.parse(this, polledMessage)) - .collect(Collectors.toList()); + final List messages = new ArrayList<>(); + for (final SubscriptionPolledMessage polledMessage : polledMessages) { + final short messageType = polledMessage.getMessageType(); + if (SubscriptionPolledMessageType.isValidatedMessageType(messageType)) { + switch (SubscriptionPolledMessageType.valueOf(messageType)) { + case TABLETS: + messages.add( + new SubscriptionMessage( + polledMessage.getCommitContext(), + ((TabletsMessagePayload) polledMessage.getMessagePayload()).getTablets())); + break; + case TS_FILE_INFO: + try { + final SubscriptionMessage message = + pollTsFile( + polledMessage.getCommitContext(), + ((TsFileInfoMessagePayload) polledMessage.getMessagePayload()).getFileName(), + timeoutMs); + if (Objects.isNull(message)) { + throw new Exception("xxx"); + } + messages.add(message); + } catch (Exception e) { + LOGGER.warn(e.getMessage()); + } + break; + default: + break; + } + } + } if (autoCommit) { long currentTimestamp = System.currentTimeMillis(); @@ -190,7 +231,64 @@ public List poll(Set topicNames, long timeoutMs) return messages; } - List pollTsFile( + private SubscriptionMessage pollTsFile( + SubscriptionCommitContext commitContext, String fileName, long timeoutMs) + throws TException, IOException, StatementExecutionException, IoTDBConnectionException { + final int dataNodeId = commitContext.getDataNodeId(); + final String topicName = commitContext.getTopicName(); + + final Path filePath = getTsFileDir(topicName).resolve(fileName); + Files.createFile(filePath); + final File file = filePath.toFile(); + final RandomAccessFile fileWriter = new RandomAccessFile(file, "rw"); + commitContextToTsFile.put(commitContext, new Pair<>(file, fileWriter)); + + long endWritingOffset = 0; + while (true) { + final List polledMessages = + pollTsFileInternal(dataNodeId, topicName, fileName, endWritingOffset, timeoutMs); + if (Objects.isNull(polledMessages) || polledMessages.size() != 1) { + return null; + } + final SubscriptionPolledMessage polledMessage = polledMessages.get(0); + final short messageType = polledMessage.getMessageType(); + if (SubscriptionPolledMessageType.isValidatedMessageType(messageType)) { + switch (SubscriptionPolledMessageType.valueOf(messageType)) { + case TS_FILE_PIECE: + { + final TsFilePieceMessagePayload messagePayload = + (TsFilePieceMessagePayload) polledMessage.getMessagePayload(); + if (!fileName.equals(messagePayload.getFileName())) { + return null; + } + fileWriter.write(messagePayload.getFilePiece()); + fileWriter.getFD().sync(); + endWritingOffset = messagePayload.getEndWritingOffset(); + break; + } + case TS_FILE_SEAL: + { + final TsFileSealMessagePayload messagePayload = + (TsFileSealMessagePayload) polledMessage.getMessagePayload(); + if (!fileName.equals(messagePayload.getFileName())) { + return null; + } + if (fileWriter.length() != messagePayload.getFileLength()) { + return null; + } + fileWriter.getFD().sync(); + fileWriter.close(); + commitContextToTsFile.remove(commitContext); + break; + } + default: + break; + } + } + } + } + + private List pollTsFileInternal( int dataNodeId, String topicName, String fileName, long endWritingOffset, long timeoutMs) throws TException, IOException, StatementExecutionException, IoTDBConnectionException { acquireReadLock(); @@ -296,10 +394,6 @@ private void commitAllUncommittedMessages() { } } - boolean isClosed() { - return isClosed.get(); - } - long getAutoCommitIntervalMs() { return autoCommitIntervalMs; } diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionSessionDataSets.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionSessionDataSets.java index 94a61a90d6af..a5359b1b9b12 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionSessionDataSets.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionSessionDataSets.java @@ -46,16 +46,4 @@ public Iterator iterator() { public Iterator tabletIterator() { return tablets.iterator(); } - - @Override - public void open() { - // do nothing - } - - @Override - public void close() throws Exception { - for (SubscriptionSessionDataSet dataSet : dataSetList) { - dataSet.close(); - } - } } diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionTsFileReader.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionTsFileReader.java index 95179c351fc0..7b9b8aa94ded 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionTsFileReader.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionTsFileReader.java @@ -22,23 +22,17 @@ import org.apache.iotdb.tsfile.read.TsFileReader; import org.apache.iotdb.tsfile.read.TsFileSequenceReader; +import java.io.IOException; + public class SubscriptionTsFileReader implements SubscriptionMessagePayload { private final String fileName; - private TsFileReader reader; - public SubscriptionTsFileReader(String fileName) { this.fileName = fileName; } - @Override - public void open() throws Exception { - reader = new TsFileReader(new TsFileSequenceReader(fileName)); - } - - @Override - public void close() throws Exception { - reader.close(); + public TsFileReader open() throws IOException { + return new TsFileReader(new TsFileSequenceReader(fileName)); } } From 5cb1762bb695517dbb70fb0750cf77e6b9a8b9d8 Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Tue, 16 Apr 2024 21:47:14 +0800 Subject: [PATCH 16/93] backup --- .../payload/common/PollMessagePayload.java | 25 ++++ .../common/PollTsFileMessagePayload.java | 33 +++++ .../common/SubscriptionMessagePayload.java | 6 - .../common/SubscriptionPollMessage.java | 7 +- .../common/SubscriptionPolledMessage.java | 7 +- .../payload/common/TabletsMessagePayload.java | 23 +++ .../common/TsFileInfoMessagePayload.java | 23 +++ .../common/TsFilePieceMessagePayload.java | 32 ++++ .../common/TsFileSealMessagePayload.java | 24 +++ .../subscription/SubscriptionConsumer.java | 2 +- .../SubscriptionPullConsumer.java | 32 +++- .../broker/SubscriptionBroker.java | 10 +- .../SubscriptionPrefetchingTsFileQueue.java | 140 +++++++++++++----- .../receiver/SubscriptionReceiverV1.java | 4 +- 14 files changed, 308 insertions(+), 60 deletions(-) diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/PollMessagePayload.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/PollMessagePayload.java index 6f8f08a3e9ec..8bd08c4c7d48 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/PollMessagePayload.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/PollMessagePayload.java @@ -25,6 +25,7 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.HashSet; +import java.util.Objects; import java.util.Set; public class PollMessagePayload implements SubscriptionMessagePayload { @@ -51,4 +52,28 @@ public SubscriptionMessagePayload deserialize(ByteBuffer buffer) { topicNames = ReadWriteIOUtils.readObjectSet(buffer); return this; } + + /////////////////////////////// Object /////////////////////////////// + + @Override + public boolean equals(final Object obj) { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + final PollMessagePayload that = (PollMessagePayload) obj; + return Objects.equals(this.topicNames, that.topicNames); + } + + @Override + public int hashCode() { + return Objects.hash(topicNames); + } + + @Override + public String toString() { + return "PollMessagePayload{topicNames=" + topicNames + "}"; + } } diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/PollTsFileMessagePayload.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/PollTsFileMessagePayload.java index 7f73cc46aa69..658416eaad7c 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/PollTsFileMessagePayload.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/PollTsFileMessagePayload.java @@ -24,6 +24,7 @@ import java.io.DataOutputStream; import java.io.IOException; import java.nio.ByteBuffer; +import java.util.Objects; public class PollTsFileMessagePayload implements SubscriptionMessagePayload { @@ -67,4 +68,36 @@ public SubscriptionMessagePayload deserialize(ByteBuffer buffer) { endWritingOffset = ReadWriteIOUtils.readLong(buffer); return this; } + + /////////////////////////////// Object /////////////////////////////// + + @Override + public boolean equals(final Object obj) { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + final PollTsFileMessagePayload that = (PollTsFileMessagePayload) obj; + return Objects.equals(this.topicName, that.topicName) + && Objects.equals(this.fileName, that.fileName) + && Objects.equals(this.endWritingOffset, that.endWritingOffset); + } + + @Override + public int hashCode() { + return Objects.hash(topicName, fileName, endWritingOffset); + } + + @Override + public String toString() { + return "PollTsFileMessagePayload{topicName=" + + topicName + + ", fileName=" + + fileName + + ", endWritingOffset=" + + endWritingOffset + + "}"; + } } diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionMessagePayload.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionMessagePayload.java index dadb3f8201c2..cfcde495fe88 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionMessagePayload.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionMessagePayload.java @@ -28,10 +28,4 @@ public interface SubscriptionMessagePayload { void serialize(final DataOutputStream stream) throws IOException; SubscriptionMessagePayload deserialize(final ByteBuffer buffer); - - boolean equals(final Object obj); - - int hashCode(); - - String toString(); } diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPollMessage.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPollMessage.java index 708c15dcc2a6..39520277d8a5 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPollMessage.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPollMessage.java @@ -76,7 +76,7 @@ private void serialize(final DataOutputStream stream) throws IOException { public static SubscriptionPollMessage deserialize(final ByteBuffer buffer) { final short messageType = ReadWriteIOUtils.readShort(buffer); - final SubscriptionMessagePayload messagePayload; + SubscriptionMessagePayload messagePayload = null; if (SubscriptionPollMessageType.isValidatedMessageType(messageType)) { switch (SubscriptionPollMessageType.valueOf(messageType)) { case POLL: @@ -86,10 +86,11 @@ public static SubscriptionPollMessage deserialize(final ByteBuffer buffer) { messagePayload = new PollTsFileMessagePayload().deserialize(buffer); break; default: - messagePayload = null; + LOGGER.warn("unexpected message type: {}, message payload will be null", messageType); + break; } } else { - messagePayload = null; + LOGGER.warn("unexpected message type: {}, message payload will be null", messageType); } final long timeoutMs = ReadWriteIOUtils.readLong(buffer); diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledMessage.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledMessage.java index cefa0b1c5109..c66b23dec276 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledMessage.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledMessage.java @@ -88,7 +88,7 @@ private void serialize(final DataOutputStream stream) throws IOException { public static SubscriptionPolledMessage deserialize(final ByteBuffer buffer) { final short messageType = ReadWriteIOUtils.readShort(buffer); - final SubscriptionMessagePayload messagePayload; + SubscriptionMessagePayload messagePayload = null; if (SubscriptionPolledMessageType.isValidatedMessageType(messageType)) { switch (SubscriptionPolledMessageType.valueOf(messageType)) { case TABLETS: @@ -104,10 +104,11 @@ public static SubscriptionPolledMessage deserialize(final ByteBuffer buffer) { messagePayload = new TsFileSealMessagePayload().deserialize(buffer); break; default: - messagePayload = null; + LOGGER.warn("unexpected message type: {}, message payload will be null", messageType); + break; } } else { - messagePayload = null; + LOGGER.warn("unexpected message type: {}, message payload will be null", messageType); } final SubscriptionCommitContext commitContext = SubscriptionCommitContext.deserialize(buffer); diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TabletsMessagePayload.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TabletsMessagePayload.java index 232befe6662b..18258a476458 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TabletsMessagePayload.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TabletsMessagePayload.java @@ -27,6 +27,7 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.List; +import java.util.Objects; public class TabletsMessagePayload implements SubscriptionMessagePayload { @@ -60,4 +61,26 @@ public SubscriptionMessagePayload deserialize(ByteBuffer buffer) { this.tablets = tablets; return this; } + + @Override + public boolean equals(final Object obj) { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + final TabletsMessagePayload that = (TabletsMessagePayload) obj; + return Objects.equals(this.tablets, that.tablets); + } + + @Override + public int hashCode() { + return Objects.hash(tablets); + } + + @Override + public String toString() { + return "TabletsMessagePayload{tablets=" + tablets + "}"; + } } diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileInfoMessagePayload.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileInfoMessagePayload.java index ec834ef9d3d1..4b32914a76a4 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileInfoMessagePayload.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileInfoMessagePayload.java @@ -24,6 +24,7 @@ import java.io.DataOutputStream; import java.io.IOException; import java.nio.ByteBuffer; +import java.util.Objects; public class TsFileInfoMessagePayload implements SubscriptionMessagePayload { @@ -49,4 +50,26 @@ public SubscriptionMessagePayload deserialize(ByteBuffer buffer) { this.fileName = ReadWriteIOUtils.readString(buffer); return this; } + + @Override + public boolean equals(final Object obj) { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + final TsFileInfoMessagePayload that = (TsFileInfoMessagePayload) obj; + return Objects.equals(this.fileName, that.fileName); + } + + @Override + public int hashCode() { + return Objects.hash(fileName); + } + + @Override + public String toString() { + return "TsFileInfoMessagePayload{fileName=" + fileName + "}"; + } } diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFilePieceMessagePayload.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFilePieceMessagePayload.java index 872b2757a8de..36660b69334a 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFilePieceMessagePayload.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFilePieceMessagePayload.java @@ -25,6 +25,8 @@ import java.io.DataOutputStream; import java.io.IOException; import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Objects; public class TsFilePieceMessagePayload implements SubscriptionMessagePayload { @@ -69,4 +71,34 @@ public SubscriptionMessagePayload deserialize(ByteBuffer buffer) { this.filePiece = ReadWriteIOUtils.readBytes(buffer, size); return this; } + + @Override + public boolean equals(final Object obj) { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + final TsFilePieceMessagePayload that = (TsFilePieceMessagePayload) obj; + return Objects.equals(this.fileName, that.fileName) + && Objects.equals(this.endWritingOffset, that.endWritingOffset) + && Arrays.equals(this.filePiece, that.filePiece); + } + + @Override + public int hashCode() { + return Objects.hash(fileName, endWritingOffset, filePiece); + } + + @Override + public String toString() { + return "TsFilePieceMessagePayload{fileName=" + + fileName + + ", endWritingOffset=" + + endWritingOffset + + ", filePiece=" + + Arrays.toString(filePiece) + + "}"; + } } diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileSealMessagePayload.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileSealMessagePayload.java index 2354174eddc3..dfe6d491e14e 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileSealMessagePayload.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileSealMessagePayload.java @@ -24,6 +24,7 @@ import java.io.DataOutputStream; import java.io.IOException; import java.nio.ByteBuffer; +import java.util.Objects; public class TsFileSealMessagePayload implements SubscriptionMessagePayload { @@ -58,4 +59,27 @@ public SubscriptionMessagePayload deserialize(ByteBuffer buffer) { this.fileLength = ReadWriteIOUtils.readLong(buffer); return this; } + + @Override + public boolean equals(final Object obj) { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + final TsFileSealMessagePayload that = (TsFileSealMessagePayload) obj; + return Objects.equals(this.fileName, that.fileName) + && Objects.equals(this.fileLength, that.fileLength); + } + + @Override + public int hashCode() { + return Objects.hash(fileName, fileLength); + } + + @Override + public String toString() { + return "TsFileSealMessagePayload{fileName=" + fileName + ", fileLength=" + fileLength + "}"; + } } diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java index 188244da7637..e7544c549a52 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java @@ -102,7 +102,7 @@ public Path getTsFileDir(String topicName) throws IOException { Files.createTempDirectory( String.format("subscribedTsFile_%s_%s", consumerId, consumerGroupId)); } - Path dirPath = subscribedTsFileBaseDirPath.resolve(topicName); + final Path dirPath = subscribedTsFileBaseDirPath.resolve(topicName); Files.createDirectories(dirPath); return dirPath; } diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java index 8fa88f2dc0d0..340219b2e608 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java @@ -204,7 +204,7 @@ public List poll(Set topicNames, long timeoutMs) ((TsFileInfoMessagePayload) polledMessage.getMessagePayload()).getFileName(), timeoutMs); if (Objects.isNull(message)) { - throw new Exception("xxx"); + throw new Exception("poll empty tsfile, will retry later..."); } messages.add(message); } catch (Exception e) { @@ -212,8 +212,11 @@ public List poll(Set topicNames, long timeoutMs) } break; default: + LOGGER.warn("unexpected message type: {}", messageType); break; } + } else { + LOGGER.warn("unexpected message type: {}", messageType); } } @@ -243,6 +246,8 @@ private SubscriptionMessage pollTsFile( final RandomAccessFile fileWriter = new RandomAccessFile(file, "rw"); commitContextToTsFile.put(commitContext, new Pair<>(file, fileWriter)); + LOGGER.info("start poll tsfile: {}", file.getAbsolutePath()); + long endWritingOffset = 0; while (true) { final List polledMessages = @@ -251,6 +256,9 @@ private SubscriptionMessage pollTsFile( return null; } final SubscriptionPolledMessage polledMessage = polledMessages.get(0); + if (Objects.isNull(polledMessage)) { + return null; + } final short messageType = polledMessage.getMessageType(); if (SubscriptionPolledMessageType.isValidatedMessageType(messageType)) { switch (SubscriptionPolledMessageType.valueOf(messageType)) { @@ -258,11 +266,17 @@ private SubscriptionMessage pollTsFile( { final TsFilePieceMessagePayload messagePayload = (TsFilePieceMessagePayload) polledMessage.getMessagePayload(); + if (Objects.isNull(messagePayload)) { + return null; + } + // check file name if (!fileName.equals(messagePayload.getFileName())) { return null; } + // write file piece fileWriter.write(messagePayload.getFilePiece()); fileWriter.getFD().sync(); + // update offset endWritingOffset = messagePayload.getEndWritingOffset(); break; } @@ -270,20 +284,32 @@ private SubscriptionMessage pollTsFile( { final TsFileSealMessagePayload messagePayload = (TsFileSealMessagePayload) polledMessage.getMessagePayload(); + if (Objects.isNull(messagePayload)) { + return null; + } + // check file name if (!fileName.equals(messagePayload.getFileName())) { return null; } + // check file length if (fileWriter.length() != messagePayload.getFileLength()) { return null; } + // sync and close fileWriter.getFD().sync(); fileWriter.close(); commitContextToTsFile.remove(commitContext); - break; + LOGGER.info("successfully poll tsfile: {}", file.getAbsolutePath()); + // generate subscription message + return new SubscriptionMessage(commitContext, fileName); } default: - break; + LOGGER.warn("unexpected message type: {}", messageType); + return null; } + } else { + LOGGER.warn("unexpected message type: {}", messageType); + return null; } } } 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 5f57ddafe2bc..08d989ec8200 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 @@ -31,7 +31,6 @@ import org.slf4j.LoggerFactory; import java.util.ArrayList; -import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Objects; @@ -87,9 +86,14 @@ public List pollTsFile( if (!(prefetchingQueue instanceof SubscriptionPrefetchingTsFileQueue)) { return null; } - return Collections.singletonList( + final List events = new ArrayList<>(); + final SubscriptionEvent event = ((SubscriptionPrefetchingTsFileQueue) prefetchingQueue) - .pollTsFile(fileName, endWritingOffset)); + .pollTsFile(fileName, endWritingOffset); + if (Objects.nonNull(event)) { + events.add(event); + } + return events; } public void commit(final List commitContexts) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java index c9fe9946aa26..0570cf68fd07 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java @@ -102,63 +102,123 @@ public SubscriptionEvent pollTsFile(String fileName, long endWritingOffset) { return null; } - final SubscriptionPolledMessage polledMessage = event.getMessage(); final List enrichedEvents = event.getEnrichedEvents(); + if (Objects.isNull(enrichedEvents) || enrichedEvents.size() != 1) { + return null; + } + final PipeTsFileInsertionEvent tsFileInsertionEvent = (PipeTsFileInsertionEvent) enrichedEvents.get(0); + if (Objects.isNull(tsFileInsertionEvent)) { + return null; + } + + final SubscriptionPolledMessage polledMessage = event.getMessage(); + if (Objects.isNull(polledMessage)) { + return null; + } + final SubscriptionMessagePayload messagePayload = polledMessage.getMessagePayload(); + if (Objects.isNull(messagePayload)) { + return null; + } + final SubscriptionCommitContext commitContext = polledMessage.getCommitContext(); + if (Objects.isNull(commitContext)) { + return null; + } - if (messagePayload instanceof TsFileInfoMessagePayload) { - if (!fileName.equals(((TsFileInfoMessagePayload) messagePayload).getFileName())) { - return null; - } - } else if (messagePayload instanceof TsFilePieceMessagePayload) { - final int readFileBufferSize = - SubscriptionConfig.getInstance().getSubscriptionReadFileBufferSize(); - final byte[] readBuffer = new byte[readFileBufferSize]; - try (final RandomAccessFile reader = - new RandomAccessFile(tsFileInsertionEvent.getTsFile(), "r")) { - while (true) { - reader.seek(endWritingOffset); - final int readLength = reader.read(readBuffer); - if (readLength == -1) { - break; + final short messageType = polledMessage.getMessageType(); + if (SubscriptionPolledMessageType.isValidatedMessageType(messageType)) { + switch (SubscriptionPolledMessageType.valueOf(messageType)) { + case TS_FILE_INFO: + // check file name + if (!fileName.equals(((TsFileInfoMessagePayload) messagePayload).getFileName())) { + LOGGER.warn( + "inconsistent file name, current is {}, incoming is {}", + fileName, + ((TsFileInfoMessagePayload) messagePayload).getFileName()); + return null; + } + // check offset + if (endWritingOffset != 0) { + LOGGER.warn("reset file {} offset to {}", fileName, endWritingOffset); + } + break; + case TS_FILE_PIECE: + // check file name + if (!fileName.equals(((TsFilePieceMessagePayload) messagePayload).getFileName())) { + LOGGER.warn( + "inconsistent file name, current is {}, incoming is {}", + fileName, + ((TsFilePieceMessagePayload) messagePayload).getFileName()); + return null; + } + // check offset + if (endWritingOffset + != ((TsFilePieceMessagePayload) messagePayload).getEndWritingOffset()) { + LOGGER.warn("reset file {} offset to {}", fileName, endWritingOffset); } - final byte[] filePiece = - readLength == readFileBufferSize - ? readBuffer - : Arrays.copyOfRange(readBuffer, 0, readLength); - - // poll tsfile piece - final SubscriptionEvent newEvent = - new SubscriptionEvent( - Collections.singletonList(tsFileInsertionEvent), - new SubscriptionPolledMessage( - SubscriptionPolledMessageType.TS_FILE_PIECE.getType(), - new TsFilePieceMessagePayload( - fileName, endWritingOffset + readLength, filePiece), - commitContext)); - eventRef.set(newEvent); - return newEvent; + break; + case TS_FILE_SEAL: + LOGGER.warn("reset file {} offset to {}", fileName, endWritingOffset); + uncommittedEvents.remove(commitContext); + break; + default: + LOGGER.warn("unexpected message type: {}", messageType); + return null; + } + } else { + LOGGER.warn("unexpected message type: {}", messageType); + return null; + } + + final int readFileBufferSize = + SubscriptionConfig.getInstance().getSubscriptionReadFileBufferSize(); + final byte[] readBuffer = new byte[readFileBufferSize]; + try (final RandomAccessFile reader = + new RandomAccessFile(tsFileInsertionEvent.getTsFile(), "r")) { + while (true) { + reader.seek(endWritingOffset); + final int readLength = reader.read(readBuffer); + if (readLength == -1) { + break; } - // poll tsfile seal + final byte[] filePiece = + readLength == readFileBufferSize + ? readBuffer + : Arrays.copyOfRange(readBuffer, 0, readLength); + + // poll tsfile piece final SubscriptionEvent newEvent = new SubscriptionEvent( Collections.singletonList(tsFileInsertionEvent), new SubscriptionPolledMessage( - SubscriptionPolledMessageType.TS_FILE_SEAL.getType(), - new TsFileSealMessagePayload( - fileName, tsFileInsertionEvent.getTsFile().length()), + SubscriptionPolledMessageType.TS_FILE_PIECE.getType(), + new TsFilePieceMessagePayload( + fileName, endWritingOffset + readLength, filePiece), commitContext)); + eventRef.set(newEvent); - // allow commit now - uncommittedEvents.put(commitContext, newEvent); return newEvent; - } catch (IOException e) { - LOGGER.warn(e.getMessage()); } + + // poll tsfile seal + final SubscriptionEvent newEvent = + new SubscriptionEvent( + Collections.singletonList(tsFileInsertionEvent), + new SubscriptionPolledMessage( + SubscriptionPolledMessageType.TS_FILE_SEAL.getType(), + new TsFileSealMessagePayload(fileName, tsFileInsertionEvent.getTsFile().length()), + commitContext)); + eventRef.set(newEvent); + + // allow commit now + uncommittedEvents.put(commitContext, newEvent); + return newEvent; + } catch (IOException e) { + LOGGER.warn(e.getMessage()); } return null; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java index ae52ff1ec50c..584bfecba58c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java @@ -322,9 +322,11 @@ private TPipeSubscribeResp handlePipeSubscribePoll(final PipeSubscribePollReq re case POLL_TS_FILE: return handlePipeSubscribePollTsFileInternal( consumerConfig, (PollTsFileMessagePayload) pollMessage.getMessagePayload(), timer); + default: + break; } } - throw new SubscriptionException("..."); + throw new SubscriptionException(String.format("unexpected message type: %s", messageType)); } catch (final SubscriptionException e) { final String exceptionMessage = String.format( From bae11d146bbc7a92885adaa7ed62b0248048c26b Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Tue, 16 Apr 2024 22:05:18 +0800 Subject: [PATCH 17/93] improve example --- .../iotdb/SubscriptionSessionExample.java | 119 ++++++++++++------ .../it/dual/IoTDBSubscriptionTsFileIT.java | 24 ++++ 2 files changed, 107 insertions(+), 36 deletions(-) create mode 100644 integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionTsFileIT.java diff --git a/example/session/src/main/java/org/apache/iotdb/SubscriptionSessionExample.java b/example/session/src/main/java/org/apache/iotdb/SubscriptionSessionExample.java index 0c674f41b796..cdca993ec9d7 100644 --- a/example/session/src/main/java/org/apache/iotdb/SubscriptionSessionExample.java +++ b/example/session/src/main/java/org/apache/iotdb/SubscriptionSessionExample.java @@ -22,12 +22,18 @@ import org.apache.iotdb.isession.SessionDataSet; import org.apache.iotdb.isession.util.Version; import org.apache.iotdb.rpc.subscription.config.ConsumerConstant; +import org.apache.iotdb.rpc.subscription.config.TopicConstant; import org.apache.iotdb.session.Session; import org.apache.iotdb.session.subscription.SubscriptionMessage; import org.apache.iotdb.session.subscription.SubscriptionPullConsumer; import org.apache.iotdb.session.subscription.SubscriptionSession; import org.apache.iotdb.session.subscription.SubscriptionSessionDataSet; import org.apache.iotdb.session.subscription.SubscriptionSessionDataSets; +import org.apache.iotdb.session.subscription.SubscriptionTsFileReader; +import org.apache.iotdb.tsfile.read.TsFileReader; +import org.apache.iotdb.tsfile.read.common.Path; +import org.apache.iotdb.tsfile.read.expression.QueryExpression; +import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet; import java.time.Duration; import java.util.Collections; @@ -38,13 +44,18 @@ public class SubscriptionSessionExample { private static Session session; - private static final String LOCAL_HOST = "127.0.0.1"; + private static final String HOST = "127.0.0.1"; + private static final int PORT = 6667; - public static void main(String[] args) throws Exception { + private static final String TOPIC_1 = "topic1"; + private static final String TOPIC_2 = "`topic2`"; + + private static void prepareData() throws Exception { + // Open session session = new Session.Builder() - .host(LOCAL_HOST) - .port(6667) + .host(HOST) + .port(PORT) .username("root") .password("root") .version(Version.V_1_0) @@ -52,7 +63,7 @@ public static void main(String[] args) throws Exception { session.open(false); // Insert some historical data - long currentTime = System.currentTimeMillis(); + final long currentTime = System.currentTimeMillis(); for (int i = 0; i < 100; ++i) { session.executeNonQueryStatement( String.format("insert into root.db.d1(time, s1, s2) values (%s, 1, 2)", i)); @@ -63,31 +74,63 @@ public static void main(String[] args) throws Exception { } session.executeNonQueryStatement("flush"); - // Create topic - final String topic1 = "topic1"; - final String topic2 = "`topic2`"; - try (SubscriptionSession subscriptionSession = new SubscriptionSession(LOCAL_HOST, 6667)) { + // Close session + session.close(); + session = null; + } + + private static void dataQuery() throws Exception { + // Open session + session = + new Session.Builder() + .host(HOST) + .port(PORT) + .username("root") + .password("root") + .version(Version.V_1_0) + .build(); + session.open(false); + + // Query + final SessionDataSet dataSet = session.executeQueryStatement("select ** from root.**"); + while (dataSet.hasNext()) { + System.out.println(dataSet.next()); + } + + // Close session + session.close(); + session = null; + } + + private static void createTopics() throws Exception { + // Create topics + try (final SubscriptionSession subscriptionSession = new SubscriptionSession(HOST, PORT)) { subscriptionSession.open(); - subscriptionSession.createTopic(topic1); - subscriptionSession.createTopic(topic2); + subscriptionSession.createTopic(TOPIC_1); + final Properties config = new Properties(); + config.put(TopicConstant.FORMAT_KEY, TopicConstant.FORMAT_TS_FILE_READER_VALUE); + subscriptionSession.createTopic(TOPIC_2, config); } + } + private static void subscriptionExample1() throws Exception { // Subscription: property-style ctor - Properties config = new Properties(); + final Properties config = new Properties(); config.put(ConsumerConstant.CONSUMER_ID_KEY, "c1"); config.put(ConsumerConstant.CONSUMER_GROUP_ID_KEY, "cg1"); - SubscriptionPullConsumer consumer1 = new SubscriptionPullConsumer(config); + final SubscriptionPullConsumer consumer1 = new SubscriptionPullConsumer(config); consumer1.open(); - consumer1.subscribe(topic1); + consumer1.subscribe(TOPIC_1); while (true) { Thread.sleep(1000); // Wait for some time - List messages = consumer1.poll(Duration.ofMillis(10000)); + final List messages = consumer1.poll(Duration.ofMillis(10000)); if (messages.isEmpty()) { break; } - for (SubscriptionMessage message : messages) { - SubscriptionSessionDataSets payload = (SubscriptionSessionDataSets) message.getPayload(); - for (SubscriptionSessionDataSet dataSet : payload) { + for (final SubscriptionMessage message : messages) { + final SubscriptionSessionDataSets dataSets = + (SubscriptionSessionDataSets) message.getPayload(); + for (final SubscriptionSessionDataSet dataSet : dataSets) { System.out.println(dataSet.getColumnNames()); System.out.println(dataSet.getColumnTypes()); while (dataSet.hasNext()) { @@ -99,36 +142,39 @@ public static void main(String[] args) throws Exception { } // Show topics and subscriptions - try (SubscriptionSession subscriptionSession = new SubscriptionSession(LOCAL_HOST, 6667)) { + try (final SubscriptionSession subscriptionSession = new SubscriptionSession(HOST, PORT)) { subscriptionSession.open(); subscriptionSession.getTopics().forEach((System.out::println)); subscriptionSession.getSubscriptions().forEach((System.out::println)); } - consumer1.unsubscribe(topic1); + consumer1.unsubscribe(TOPIC_1); consumer1.close(); + } + private static void subscriptionExample2() throws Exception { // Subscription: builder-style ctor - try (SubscriptionPullConsumer consumer2 = + try (final SubscriptionPullConsumer consumer2 = new SubscriptionPullConsumer.Builder() .consumerId("c2") .consumerGroupId("cg2") .autoCommit(false) .buildPullConsumer()) { consumer2.open(); - consumer2.subscribe(topic2); + consumer2.subscribe(TOPIC_2); while (true) { Thread.sleep(1000); // wait some time - List messages = - consumer2.poll(Collections.singleton(topic2), Duration.ofMillis(10000)); + final List messages = + consumer2.poll(Collections.singleton(TOPIC_2), Duration.ofMillis(10000)); if (messages.isEmpty()) { break; } - for (SubscriptionMessage message : messages) { - SubscriptionSessionDataSets payload = (SubscriptionSessionDataSets) message.getPayload(); - for (SubscriptionSessionDataSet dataSet : payload) { - System.out.println(dataSet.getColumnNames()); - System.out.println(dataSet.getColumnTypes()); + for (final SubscriptionMessage message : messages) { + final SubscriptionTsFileReader reader = (SubscriptionTsFileReader) message.getPayload(); + try (final TsFileReader tsFileReader = reader.open()) { + final Path path = new Path("root.db.d1", "s1", true); + QueryDataSet dataSet = + tsFileReader.query(QueryExpression.create(Collections.singletonList(path), null)); while (dataSet.hasNext()) { System.out.println(dataSet.next()); } @@ -136,14 +182,15 @@ public static void main(String[] args) throws Exception { } consumer2.commitSync(messages); } - consumer2.unsubscribe(topic2); + consumer2.unsubscribe(TOPIC_2); } + } - // Query - SessionDataSet dataSet = session.executeQueryStatement("select ** from root.**"); - while (dataSet.hasNext()) { - System.out.println(dataSet.next()); - } - session.close(); + public static void main(final String[] args) throws Exception { + prepareData(); + dataQuery(); + createTopics(); + subscriptionExample1(); + subscriptionExample2(); } } diff --git a/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionTsFileIT.java b/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionTsFileIT.java new file mode 100644 index 000000000000..618c36effd91 --- /dev/null +++ b/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionTsFileIT.java @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.subscription.it.dual; + +public class IoTDBSubscriptionTsFileIT { + +} From ab1f04c5bbfecf28ea0dcdeccae260a6bbca9965 Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Tue, 16 Apr 2024 22:18:36 +0800 Subject: [PATCH 18/93] fix --- .../java/org/apache/iotdb/SubscriptionSessionExample.java | 2 +- .../iotdb/session/subscription/SubscriptionMessage.java | 4 ++-- .../session/subscription/SubscriptionPullConsumer.java | 2 +- .../session/subscription/SubscriptionTsFileReader.java | 8 ++++---- 4 files changed, 8 insertions(+), 8 deletions(-) diff --git a/example/session/src/main/java/org/apache/iotdb/SubscriptionSessionExample.java b/example/session/src/main/java/org/apache/iotdb/SubscriptionSessionExample.java index cdca993ec9d7..70f4f2268631 100644 --- a/example/session/src/main/java/org/apache/iotdb/SubscriptionSessionExample.java +++ b/example/session/src/main/java/org/apache/iotdb/SubscriptionSessionExample.java @@ -173,7 +173,7 @@ private static void subscriptionExample2() throws Exception { final SubscriptionTsFileReader reader = (SubscriptionTsFileReader) message.getPayload(); try (final TsFileReader tsFileReader = reader.open()) { final Path path = new Path("root.db.d1", "s1", true); - QueryDataSet dataSet = + final QueryDataSet dataSet = tsFileReader.query(QueryExpression.create(Collections.singletonList(path), null)); while (dataSet.hasNext()) { System.out.println(dataSet.next()); diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionMessage.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionMessage.java index 60485f03ad8d..8add0127e31d 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionMessage.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionMessage.java @@ -36,9 +36,9 @@ public SubscriptionMessage(SubscriptionCommitContext commitContext, List this.payload = new SubscriptionSessionDataSets(tablets); } - public SubscriptionMessage(SubscriptionCommitContext commitContext, String fileName) { + public SubscriptionMessage(SubscriptionCommitContext commitContext, String filePath) { this.commitContext = commitContext; - this.payload = new SubscriptionTsFileReader(fileName); + this.payload = new SubscriptionTsFileReader(filePath); } public SubscriptionCommitContext getCommitContext() { diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java index 340219b2e608..146489afbee9 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java @@ -301,7 +301,7 @@ private SubscriptionMessage pollTsFile( commitContextToTsFile.remove(commitContext); LOGGER.info("successfully poll tsfile: {}", file.getAbsolutePath()); // generate subscription message - return new SubscriptionMessage(commitContext, fileName); + return new SubscriptionMessage(commitContext, file.getAbsolutePath()); } default: LOGGER.warn("unexpected message type: {}", messageType); diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionTsFileReader.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionTsFileReader.java index 7b9b8aa94ded..fa7645e91f49 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionTsFileReader.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionTsFileReader.java @@ -26,13 +26,13 @@ public class SubscriptionTsFileReader implements SubscriptionMessagePayload { - private final String fileName; + private final String filePath; - public SubscriptionTsFileReader(String fileName) { - this.fileName = fileName; + public SubscriptionTsFileReader(String filePath) { + this.filePath = filePath; } public TsFileReader open() throws IOException { - return new TsFileReader(new TsFileSequenceReader(fileName)); + return new TsFileReader(new TsFileSequenceReader(filePath)); } } From 1681b89b71e02b4828a3c4a72283313b16bfd166 Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Wed, 17 Apr 2024 12:44:45 +0800 Subject: [PATCH 19/93] improve IT --- .../it/IoTDBSubscriptionITConstant.java | 11 ++ .../it/dual/AbstractSubscriptionDualIT.java | 7 +- .../IoTDBSubscriptionConsumerGroupIT.java | 21 ++- .../it/dual/IoTDBSubscriptionTopicIT.java | 75 ++++------- .../it/dual/IoTDBSubscriptionTsFileIT.java | 24 ---- .../it/local/IoTDBSubscriptionBasicIT.java | 123 ++++++++++++++++-- .../it/local/IoTDBSubscriptionRestartIT.java | 55 ++++---- .../common/SubscriptionCommitContext.java | 11 +- .../subscription/SubscriptionMessage.java | 15 ++- .../subscription/SubscriptionMessageType.java | 36 +++++ 10 files changed, 237 insertions(+), 141 deletions(-) create mode 100644 integration-test/src/test/java/org/apache/iotdb/subscription/it/IoTDBSubscriptionITConstant.java delete mode 100644 integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionTsFileIT.java create mode 100644 iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionMessageType.java diff --git a/integration-test/src/test/java/org/apache/iotdb/subscription/it/IoTDBSubscriptionITConstant.java b/integration-test/src/test/java/org/apache/iotdb/subscription/it/IoTDBSubscriptionITConstant.java new file mode 100644 index 000000000000..5df67d22b755 --- /dev/null +++ b/integration-test/src/test/java/org/apache/iotdb/subscription/it/IoTDBSubscriptionITConstant.java @@ -0,0 +1,11 @@ +package org.apache.iotdb.subscription.it; + +public class IoTDBSubscriptionITConstant { + + public static final long AWAITILITY_POLL_DELAY_SECOND = 1L; + public static final long AWAITILITY_POLL_INTERVAL_SECOND = 2L; + public static final long AWAITILITY_AT_MOST_SECOND = 240L; + + public static final long SLEEP_NS = 1_000_000_000L; + public static final long POLL_TIMEOUT_MS = 10_000L; +} 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..26e3e01a8280 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 @@ -36,13 +36,16 @@ public void setUp() { senderEnv = MultiEnvFactory.getEnv(0); receiverEnv = MultiEnvFactory.getEnv(1); + // enable auto create schema senderEnv.getConfig().getCommonConfig().setAutoCreateSchemaEnabled(true); receiverEnv.getConfig().getCommonConfig().setAutoCreateSchemaEnabled(true); + + // for IoTDBSubscriptionConsumerGroupIT 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().getConfigNodeConfig().setConnectionTimeoutMs(600_000); + receiverEnv.getConfig().getConfigNodeConfig().setConnectionTimeoutMs(600_000); senderEnv.initClusterEnvironment(); receiverEnv.initClusterEnvironment(); diff --git a/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionConsumerGroupIT.java b/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionConsumerGroupIT.java index d5f2d09f34b2..0865c7c214c6 100644 --- a/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionConsumerGroupIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionConsumerGroupIT.java @@ -32,6 +32,7 @@ import org.apache.iotdb.session.subscription.SubscriptionPullConsumer; import org.apache.iotdb.session.subscription.SubscriptionSessionDataSet; import org.apache.iotdb.session.subscription.SubscriptionSessionDataSets; +import org.apache.iotdb.subscription.it.IoTDBSubscriptionITConstant; import org.apache.iotdb.tsfile.read.common.RowRecord; import org.apache.iotdb.tsfile.utils.Pair; @@ -46,7 +47,6 @@ import java.sql.Connection; import java.sql.Statement; -import java.time.Duration; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -57,6 +57,7 @@ import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.locks.LockSupport; import java.util.stream.Collectors; import static org.junit.Assert.fail; @@ -723,16 +724,9 @@ private void pollMessagesAndCheck( () -> { try (final SubscriptionPullConsumer consumer = consumers.get(index)) { while (!isClosed.get()) { - try { - Thread.sleep(1000); // wait some time - } catch (final InterruptedException e) { - break; - } + LockSupport.parkNanos(IoTDBSubscriptionITConstant.SLEEP_NS); // wait some time final List messages = - consumer.poll(Duration.ofMillis(10000)); - if (messages.isEmpty()) { - continue; - } + consumer.poll(IoTDBSubscriptionITConstant.POLL_TIMEOUT_MS); for (final SubscriptionMessage message : messages) { final SubscriptionSessionDataSets payload = (SubscriptionSessionDataSets) message.getPayload(); @@ -770,9 +764,10 @@ private void pollMessagesAndCheck( final Statement statement = connection.createStatement()) { // Keep retrying if there are execution failures Awaitility.await() - .pollDelay(1, TimeUnit.SECONDS) - .pollInterval(1, TimeUnit.SECONDS) - .atMost(180, TimeUnit.SECONDS) + .pollDelay(IoTDBSubscriptionITConstant.AWAITILITY_POLL_DELAY_SECOND, TimeUnit.SECONDS) + .pollInterval( + IoTDBSubscriptionITConstant.AWAITILITY_POLL_INTERVAL_SECOND, TimeUnit.SECONDS) + .atMost(IoTDBSubscriptionITConstant.AWAITILITY_AT_MOST_SECOND, TimeUnit.SECONDS) .untilAsserted( () -> { if (receiverCrashed.get()) { diff --git a/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionTopicIT.java b/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionTopicIT.java index 42fb88adaf49..6379fe66c2b0 100644 --- a/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionTopicIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionTopicIT.java @@ -28,6 +28,7 @@ import org.apache.iotdb.session.subscription.SubscriptionPullConsumer; import org.apache.iotdb.session.subscription.SubscriptionSession; import org.apache.iotdb.session.subscription.SubscriptionSessionDataSets; +import org.apache.iotdb.subscription.it.IoTDBSubscriptionITConstant; import org.apache.iotdb.tsfile.write.record.Tablet; import org.awaitility.Awaitility; @@ -39,7 +40,6 @@ import java.sql.Connection; import java.sql.Statement; -import java.time.Duration; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; @@ -48,6 +48,7 @@ import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.locks.LockSupport; import static org.junit.Assert.fail; @@ -108,16 +109,9 @@ public void testTopicPathSubscription() throws Exception { consumer.open(); consumer.subscribe(topicName); while (!isClosed.get()) { - try { - Thread.sleep(1000); // wait some time - } catch (final InterruptedException e) { - break; - } + LockSupport.parkNanos(IoTDBSubscriptionITConstant.SLEEP_NS); // wait some time final List messages = - consumer.poll(Duration.ofMillis(10000)); - if (messages.isEmpty()) { - continue; - } + consumer.poll(IoTDBSubscriptionITConstant.POLL_TIMEOUT_MS); for (final SubscriptionMessage message : messages) { final SubscriptionSessionDataSets payload = (SubscriptionSessionDataSets) message.getPayload(); @@ -144,9 +138,10 @@ public void testTopicPathSubscription() throws Exception { final Statement statement = connection.createStatement()) { // Keep retrying if there are execution failures Awaitility.await() - .pollDelay(1, TimeUnit.SECONDS) - .pollInterval(1, TimeUnit.SECONDS) - .atMost(120, TimeUnit.SECONDS) + .pollDelay(IoTDBSubscriptionITConstant.AWAITILITY_POLL_DELAY_SECOND, TimeUnit.SECONDS) + .pollInterval( + IoTDBSubscriptionITConstant.AWAITILITY_POLL_INTERVAL_SECOND, TimeUnit.SECONDS) + .atMost(IoTDBSubscriptionITConstant.AWAITILITY_AT_MOST_SECOND, TimeUnit.SECONDS) .untilAsserted( () -> TestUtils.assertSingleResultSetEqual( @@ -215,16 +210,9 @@ public void testTopicTimeSubscription() throws Exception { consumer.open(); consumer.subscribe(topicName); while (!isClosed.get()) { - try { - Thread.sleep(1000); // wait some time - } catch (final InterruptedException e) { - break; - } + LockSupport.parkNanos(IoTDBSubscriptionITConstant.SLEEP_NS); // wait some time final List messages = - consumer.poll(Duration.ofMillis(10000)); - if (messages.isEmpty()) { - continue; - } + consumer.poll(IoTDBSubscriptionITConstant.POLL_TIMEOUT_MS); for (final SubscriptionMessage message : messages) { final SubscriptionSessionDataSets payload = (SubscriptionSessionDataSets) message.getPayload(); @@ -251,9 +239,10 @@ public void testTopicTimeSubscription() throws Exception { final Statement statement = connection.createStatement()) { // Keep retrying if there are execution failures Awaitility.await() - .pollDelay(1, TimeUnit.SECONDS) - .pollInterval(1, TimeUnit.SECONDS) - .atMost(120, TimeUnit.SECONDS) + .pollDelay(IoTDBSubscriptionITConstant.AWAITILITY_POLL_DELAY_SECOND, TimeUnit.SECONDS) + .pollInterval( + IoTDBSubscriptionITConstant.AWAITILITY_POLL_INTERVAL_SECOND, TimeUnit.SECONDS) + .atMost(IoTDBSubscriptionITConstant.AWAITILITY_AT_MOST_SECOND, TimeUnit.SECONDS) .untilAsserted( () -> TestUtils.assertSingleResultSetEqual( @@ -318,16 +307,9 @@ public void testTopicProcessorSubscription() throws Exception { consumer.open(); consumer.subscribe(topicName); while (!isClosed.get()) { - try { - Thread.sleep(1000); // wait some time - } catch (final InterruptedException e) { - break; - } + LockSupport.parkNanos(IoTDBSubscriptionITConstant.SLEEP_NS); // wait some time final List messages = - consumer.poll(Duration.ofMillis(10000)); - if (messages.isEmpty()) { - continue; - } + consumer.poll(IoTDBSubscriptionITConstant.POLL_TIMEOUT_MS); for (final SubscriptionMessage message : messages) { final SubscriptionSessionDataSets payload = (SubscriptionSessionDataSets) message.getPayload(); @@ -358,9 +340,10 @@ public void testTopicProcessorSubscription() throws Exception { final Statement statement = connection.createStatement()) { // Keep retrying if there are execution failures Awaitility.await() - .pollDelay(1, TimeUnit.SECONDS) - .pollInterval(1, TimeUnit.SECONDS) - .atMost(120, TimeUnit.SECONDS) + .pollDelay(IoTDBSubscriptionITConstant.AWAITILITY_POLL_DELAY_SECOND, TimeUnit.SECONDS) + .pollInterval( + IoTDBSubscriptionITConstant.AWAITILITY_POLL_INTERVAL_SECOND, TimeUnit.SECONDS) + .atMost(IoTDBSubscriptionITConstant.AWAITILITY_AT_MOST_SECOND, TimeUnit.SECONDS) .untilAsserted( () -> TestUtils.assertResultSetEqual( @@ -451,16 +434,9 @@ public void testTopicNameWithBackQuote() throws Exception { consumer.open(); consumer.subscribe(topics); while (!isClosed.get()) { - try { - Thread.sleep(1000); // wait some time - } catch (final InterruptedException e) { - break; - } + LockSupport.parkNanos(IoTDBSubscriptionITConstant.SLEEP_NS); // wait some time final List messages = - consumer.poll(topics, Duration.ofMillis(10000)); - if (messages.isEmpty()) { - continue; - } + consumer.poll(IoTDBSubscriptionITConstant.POLL_TIMEOUT_MS); for (final SubscriptionMessage message : messages) { final SubscriptionSessionDataSets payload = (SubscriptionSessionDataSets) message.getPayload(); @@ -487,9 +463,10 @@ public void testTopicNameWithBackQuote() throws Exception { final Statement statement = connection.createStatement()) { // Keep retrying if there are execution failures Awaitility.await() - .pollDelay(1, TimeUnit.SECONDS) - .pollInterval(1, TimeUnit.SECONDS) - .atMost(120, TimeUnit.SECONDS) + .pollDelay(IoTDBSubscriptionITConstant.AWAITILITY_POLL_DELAY_SECOND, TimeUnit.SECONDS) + .pollInterval( + IoTDBSubscriptionITConstant.AWAITILITY_POLL_INTERVAL_SECOND, TimeUnit.SECONDS) + .atMost(IoTDBSubscriptionITConstant.AWAITILITY_AT_MOST_SECOND, TimeUnit.SECONDS) .untilAsserted( () -> TestUtils.assertSingleResultSetEqual( diff --git a/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionTsFileIT.java b/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionTsFileIT.java deleted file mode 100644 index 618c36effd91..000000000000 --- a/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionTsFileIT.java +++ /dev/null @@ -1,24 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iotdb.subscription.it.dual; - -public class IoTDBSubscriptionTsFileIT { - -} diff --git a/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionBasicIT.java b/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionBasicIT.java index 6938763c8f82..81ef46c53bef 100644 --- a/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionBasicIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionBasicIT.java @@ -23,11 +23,18 @@ import org.apache.iotdb.it.env.EnvFactory; import org.apache.iotdb.it.framework.IoTDBTestRunner; import org.apache.iotdb.itbase.category.LocalStandaloneIT; +import org.apache.iotdb.rpc.subscription.config.TopicConstant; import org.apache.iotdb.session.subscription.SubscriptionMessage; import org.apache.iotdb.session.subscription.SubscriptionPullConsumer; import org.apache.iotdb.session.subscription.SubscriptionSession; import org.apache.iotdb.session.subscription.SubscriptionSessionDataSet; import org.apache.iotdb.session.subscription.SubscriptionSessionDataSets; +import org.apache.iotdb.session.subscription.SubscriptionTsFileReader; +import org.apache.iotdb.subscription.it.IoTDBSubscriptionITConstant; +import org.apache.iotdb.tsfile.read.TsFileReader; +import org.apache.iotdb.tsfile.read.common.Path; +import org.apache.iotdb.tsfile.read.expression.QueryExpression; +import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet; import org.awaitility.Awaitility; import org.junit.After; @@ -39,11 +46,13 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.time.Duration; +import java.util.Collections; import java.util.List; +import java.util.Properties; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.locks.LockSupport; import static org.junit.Assert.fail; @@ -64,7 +73,7 @@ public void tearDown() throws Exception { } @Test - public void testBasicSubscription() throws Exception { + public void testBasicSubscribeTablets() throws Exception { // Insert some historical data try (final ISession session = EnvFactory.getEnv().getSessionConnection()) { for (int i = 0; i < 100; ++i) { @@ -106,16 +115,9 @@ public void testBasicSubscription() throws Exception { consumer.open(); consumer.subscribe(topicName); while (!isClosed.get()) { - try { - Thread.sleep(1000); // wait some time - } catch (final InterruptedException e) { - break; - } + LockSupport.parkNanos(IoTDBSubscriptionITConstant.SLEEP_NS); // wait some time final List messages = - consumer.poll(Duration.ofMillis(10000)); - if (messages.isEmpty()) { - continue; - } + consumer.poll(IoTDBSubscriptionITConstant.POLL_TIMEOUT_MS); for (final SubscriptionMessage message : messages) { final SubscriptionSessionDataSets payload = (SubscriptionSessionDataSets) message.getPayload(); @@ -142,9 +144,102 @@ public void testBasicSubscription() throws Exception { try { // Keep retrying if there are execution failures Awaitility.await() - .pollDelay(1, TimeUnit.SECONDS) - .pollInterval(1, TimeUnit.SECONDS) - .atMost(120, TimeUnit.SECONDS) + .pollDelay(IoTDBSubscriptionITConstant.AWAITILITY_POLL_DELAY_SECOND, TimeUnit.SECONDS) + .pollInterval( + IoTDBSubscriptionITConstant.AWAITILITY_POLL_INTERVAL_SECOND, TimeUnit.SECONDS) + .atMost(IoTDBSubscriptionITConstant.AWAITILITY_AT_MOST_SECOND, TimeUnit.SECONDS) + .untilAsserted(() -> Assert.assertEquals(100, rowCount.get())); + } catch (final Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } finally { + isClosed.set(true); + thread.join(); + } + } + + @Test + public void testBasicSubscribeTsFile() throws Exception { + // Insert some historical data + try (final ISession session = EnvFactory.getEnv().getSessionConnection()) { + for (int i = 0; i < 100; ++i) { + session.executeNonQueryStatement( + String.format("insert into root.db.d1(time, s1) values (%s, 1)", i)); + } + session.executeNonQueryStatement("flush"); + } catch (final Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + + // Create topic + final String topicName = "topic2"; + final String host = EnvFactory.getEnv().getIP(); + final int port = Integer.parseInt(EnvFactory.getEnv().getPort()); + try (final SubscriptionSession session = new SubscriptionSession(host, port)) { + session.open(); + final Properties config = new Properties(); + config.put(TopicConstant.FORMAT_KEY, TopicConstant.FORMAT_TS_FILE_READER_VALUE); + session.createTopic(topicName, config); + } catch (final Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + + // Subscription + final AtomicInteger rowCount = new AtomicInteger(); + final AtomicBoolean isClosed = new AtomicBoolean(false); + final Thread thread = + new Thread( + () -> { + try (final SubscriptionPullConsumer consumer = + new SubscriptionPullConsumer.Builder() + .host(host) + .port(port) + .consumerId("c1") + .consumerGroupId("cg1") + .autoCommit(false) + .buildPullConsumer()) { + consumer.open(); + consumer.subscribe(topicName); + while (!isClosed.get()) { + LockSupport.parkNanos(IoTDBSubscriptionITConstant.SLEEP_NS); // wait some time + final List messages = + consumer.poll(IoTDBSubscriptionITConstant.POLL_TIMEOUT_MS); + for (final SubscriptionMessage message : messages) { + final SubscriptionTsFileReader reader = + (SubscriptionTsFileReader) message.getPayload(); + try (final TsFileReader tsFileReader = reader.open()) { + final Path path = new Path("root.db.d1", "s1", true); + final QueryDataSet dataSet = + tsFileReader.query( + QueryExpression.create(Collections.singletonList(path), null)); + while (dataSet.hasNext()) { + dataSet.next(); + rowCount.addAndGet(1); + } + } + } + consumer.commitSync(messages); + } + consumer.unsubscribe(topicName); + } catch (final Exception e) { + e.printStackTrace(); + // Avoid failure + } finally { + LOGGER.info("consumer exiting..."); + } + }); + thread.start(); + + // Check row count + try { + // Keep retrying if there are execution failures + Awaitility.await() + .pollDelay(IoTDBSubscriptionITConstant.AWAITILITY_POLL_DELAY_SECOND, TimeUnit.SECONDS) + .pollInterval( + IoTDBSubscriptionITConstant.AWAITILITY_POLL_INTERVAL_SECOND, TimeUnit.SECONDS) + .atMost(IoTDBSubscriptionITConstant.AWAITILITY_AT_MOST_SECOND, TimeUnit.SECONDS) .untilAsserted(() -> Assert.assertEquals(100, rowCount.get())); } catch (final Exception e) { e.printStackTrace(); diff --git a/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionRestartIT.java b/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionRestartIT.java index c0ff9d67466f..9ebb8e6b5791 100644 --- a/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionRestartIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionRestartIT.java @@ -37,6 +37,7 @@ import org.apache.iotdb.session.subscription.SubscriptionSession; import org.apache.iotdb.session.subscription.SubscriptionSessionDataSet; import org.apache.iotdb.session.subscription.SubscriptionSessionDataSets; +import org.apache.iotdb.subscription.it.IoTDBSubscriptionITConstant; import org.awaitility.Awaitility; import org.junit.After; @@ -48,12 +49,12 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.time.Duration; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.locks.LockSupport; import static org.junit.Assert.fail; @@ -160,16 +161,9 @@ public void testSubscriptionAfterRestartCluster() throws Exception { .buildPullConsumer()) { consumer.open(); while (!isClosed.get()) { - try { - Thread.sleep(1000); // wait some time - } catch (final InterruptedException e) { - break; - } + LockSupport.parkNanos(IoTDBSubscriptionITConstant.SLEEP_NS); // wait some time final List messages = - consumer.poll(Duration.ofMillis(10000)); - if (messages.isEmpty()) { - continue; - } + consumer.poll(IoTDBSubscriptionITConstant.POLL_TIMEOUT_MS); for (final SubscriptionMessage message : messages) { final SubscriptionSessionDataSets payload = (SubscriptionSessionDataSets) message.getPayload(); @@ -196,9 +190,10 @@ public void testSubscriptionAfterRestartCluster() throws Exception { try { // Keep retrying if there are execution failures Awaitility.await() - .pollDelay(1, TimeUnit.SECONDS) - .pollInterval(1, TimeUnit.SECONDS) - .atMost(120, TimeUnit.SECONDS) + .pollDelay(IoTDBSubscriptionITConstant.AWAITILITY_POLL_DELAY_SECOND, TimeUnit.SECONDS) + .pollInterval( + IoTDBSubscriptionITConstant.AWAITILITY_POLL_INTERVAL_SECOND, TimeUnit.SECONDS) + .atMost(IoTDBSubscriptionITConstant.AWAITILITY_AT_MOST_SECOND, TimeUnit.SECONDS) .untilAsserted(() -> Assert.assertEquals(100, timestamps.size())); } catch (final Exception e) { e.printStackTrace(); @@ -235,7 +230,7 @@ public void testSubscriptionAfterRestartDataNode() throws Exception { .consumerId("c1") .consumerGroupId("cg1") .autoCommit(true) - .heartbeatIntervalMs(1000) + .heartbeatIntervalMs(1000) // narrow heartbeat interval .endpointsSyncIntervalMs(5000) // narrow endpoints sync interval .buildPullConsumer(); consumer.open(); @@ -271,14 +266,10 @@ public void testSubscriptionAfterRestartDataNode() throws Exception { () -> { try (final SubscriptionPullConsumer consumerRef = consumer) { while (!isClosed.get()) { - try { - Thread.sleep(1000); // wait some time - } catch (final InterruptedException e) { - break; - } + LockSupport.parkNanos(IoTDBSubscriptionITConstant.SLEEP_NS); // wait some time final List messages; try { - messages = consumerRef.poll(Duration.ofMillis(10000)); + messages = consumer.poll(IoTDBSubscriptionITConstant.POLL_TIMEOUT_MS); } catch (final Exception e) { e.printStackTrace(); // Avoid failure @@ -328,9 +319,10 @@ public void testSubscriptionAfterRestartDataNode() throws Exception { try { // Keep retrying if there are execution failures Awaitility.await() - .pollDelay(1, TimeUnit.SECONDS) - .pollInterval(1, TimeUnit.SECONDS) - .atMost(120, TimeUnit.SECONDS) + .pollDelay(IoTDBSubscriptionITConstant.AWAITILITY_POLL_DELAY_SECOND, TimeUnit.SECONDS) + .pollInterval( + IoTDBSubscriptionITConstant.AWAITILITY_POLL_INTERVAL_SECOND, TimeUnit.SECONDS) + .atMost(IoTDBSubscriptionITConstant.AWAITILITY_AT_MOST_SECOND, TimeUnit.SECONDS) .untilAsserted(() -> Assert.assertEquals(200, timestamps.size())); } catch (final Exception e) { e.printStackTrace(); @@ -367,7 +359,7 @@ public void testSubscriptionWhenConfigNodeLeaderChange() throws Exception { .consumerId("c1") .consumerGroupId("cg1") .autoCommit(true) - .heartbeatIntervalMs(1000) + .heartbeatIntervalMs(1000) // narrow heartbeat interval .endpointsSyncIntervalMs(5000) // narrow endpoints sync interval .buildPullConsumer(); consumer.open(); @@ -398,14 +390,10 @@ public void testSubscriptionWhenConfigNodeLeaderChange() throws Exception { () -> { try (final SubscriptionPullConsumer consumerRef = consumer) { while (!isClosed.get()) { - try { - Thread.sleep(1000); // wait some time - } catch (final InterruptedException e) { - break; - } + LockSupport.parkNanos(IoTDBSubscriptionITConstant.SLEEP_NS); // wait some time final List messages; try { - messages = consumerRef.poll(Duration.ofMillis(10000)); + messages = consumerRef.poll(IoTDBSubscriptionITConstant.POLL_TIMEOUT_MS); } catch (final Exception e) { e.printStackTrace(); // Avoid failure @@ -467,9 +455,10 @@ public void testSubscriptionWhenConfigNodeLeaderChange() throws Exception { try { // Keep retrying if there are execution failures Awaitility.await() - .pollDelay(1, TimeUnit.SECONDS) - .pollInterval(1, TimeUnit.SECONDS) - .atMost(120, TimeUnit.SECONDS) + .pollDelay(IoTDBSubscriptionITConstant.AWAITILITY_POLL_DELAY_SECOND, TimeUnit.SECONDS) + .pollInterval( + IoTDBSubscriptionITConstant.AWAITILITY_POLL_INTERVAL_SECOND, TimeUnit.SECONDS) + .atMost(IoTDBSubscriptionITConstant.AWAITILITY_AT_MOST_SECOND, TimeUnit.SECONDS) .untilAsserted(() -> Assert.assertEquals(200, timestamps.size())); } catch (final Exception e) { e.printStackTrace(); diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionCommitContext.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionCommitContext.java index 37be5aa97a50..964dba331f06 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionCommitContext.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionCommitContext.java @@ -25,6 +25,7 @@ import java.io.DataOutputStream; import java.io.IOException; import java.nio.ByteBuffer; +import java.util.Comparator; import java.util.Objects; public class SubscriptionCommitContext implements Comparable { @@ -139,8 +140,12 @@ public String toString() { } @Override - public int compareTo(SubscriptionCommitContext commitContext) { - // TODO - return 0; + public int compareTo(SubscriptionCommitContext that) { + return Comparator.comparingInt(SubscriptionCommitContext::getDataNodeId) + .thenComparingInt(SubscriptionCommitContext::getRebootTimes) + .thenComparing(SubscriptionCommitContext::getTopicName) + .thenComparing(SubscriptionCommitContext::getConsumerGroupId) + .thenComparingLong(SubscriptionCommitContext::getCommitId) + .compare(this, that); } } diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionMessage.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionMessage.java index 8add0127e31d..a9ab97391430 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionMessage.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionMessage.java @@ -29,15 +29,19 @@ public class SubscriptionMessage implements Comparable { private final SubscriptionCommitContext commitContext; + private final short messageType; + private final SubscriptionMessagePayload payload; public SubscriptionMessage(SubscriptionCommitContext commitContext, List tablets) { this.commitContext = commitContext; + this.messageType = SubscriptionMessageType.SESSION_DATA_SET.getType(); this.payload = new SubscriptionSessionDataSets(tablets); } public SubscriptionMessage(SubscriptionCommitContext commitContext, String filePath) { this.commitContext = commitContext; + this.messageType = SubscriptionMessageType.TS_FILE_READER.getType(); this.payload = new SubscriptionTsFileReader(filePath); } @@ -45,6 +49,10 @@ public SubscriptionCommitContext getCommitContext() { return commitContext; } + public short getMessageType() { + return messageType; + } + public SubscriptionMessagePayload getPayload() { return payload; } @@ -60,13 +68,14 @@ public boolean equals(Object obj) { return false; } SubscriptionMessage that = (SubscriptionMessage) obj; - return Objects.equals(this.payload, that.payload) - && Objects.equals(this.commitContext, that.commitContext); + return Objects.equals(this.commitContext, that.commitContext) + && Objects.equals(this.messageType, that.messageType) + && Objects.equals(this.payload, that.payload); } @Override public int hashCode() { - return Objects.hash(payload, commitContext); + return Objects.hash(commitContext, messageType, payload); } @Override diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionMessageType.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionMessageType.java new file mode 100644 index 000000000000..3152bf79bb25 --- /dev/null +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionMessageType.java @@ -0,0 +1,36 @@ +package org.apache.iotdb.session.subscription; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; + +public enum SubscriptionMessageType { + SESSION_DATA_SET((short) 0), + TS_FILE_READER((short) 1), + ; + + private final short type; + + SubscriptionMessageType(short type) { + this.type = type; + } + + public short getType() { + return type; + } + + private static final Map TYPE_MAP = + Arrays.stream(SubscriptionMessageType.values()) + .collect( + HashMap::new, + (typeMap, messageType) -> typeMap.put(messageType.getType(), messageType), + HashMap::putAll); + + public static boolean isValidatedMessageType(short type) { + return TYPE_MAP.containsKey(type); + } + + public static SubscriptionMessageType valueOf(short type) { + return TYPE_MAP.get(type); + } +} From d6382f69bc81de0fde977aef0377cce8e24a45c8 Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Wed, 17 Apr 2024 12:45:17 +0800 Subject: [PATCH 20/93] fix --- .../it/IoTDBSubscriptionITConstant.java | 19 +++++++++++++++++++ .../subscription/SubscriptionMessageType.java | 19 +++++++++++++++++++ 2 files changed, 38 insertions(+) diff --git a/integration-test/src/test/java/org/apache/iotdb/subscription/it/IoTDBSubscriptionITConstant.java b/integration-test/src/test/java/org/apache/iotdb/subscription/it/IoTDBSubscriptionITConstant.java index 5df67d22b755..d2dc8ee49a5f 100644 --- a/integration-test/src/test/java/org/apache/iotdb/subscription/it/IoTDBSubscriptionITConstant.java +++ b/integration-test/src/test/java/org/apache/iotdb/subscription/it/IoTDBSubscriptionITConstant.java @@ -1,3 +1,22 @@ +/* + * 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.subscription.it; public class IoTDBSubscriptionITConstant { diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionMessageType.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionMessageType.java index 3152bf79bb25..2d69d1c01ff0 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionMessageType.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionMessageType.java @@ -1,3 +1,22 @@ +/* + * 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.session.subscription; import java.util.Arrays; From ddec40668d1a8c216b30793ca1409d93e1d61cd6 Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Wed, 17 Apr 2024 13:28:10 +0800 Subject: [PATCH 21/93] more ITs --- .../IoTDBSubscriptionConsumerGroupIT.java | 205 ++++++++++++++++-- .../it/dual/IoTDBSubscriptionTopicIT.java | 38 ++++ 2 files changed, 223 insertions(+), 20 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionConsumerGroupIT.java b/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionConsumerGroupIT.java index 0865c7c214c6..4be8100ef6dc 100644 --- a/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionConsumerGroupIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionConsumerGroupIT.java @@ -28,12 +28,20 @@ import org.apache.iotdb.it.framework.IoTDBTestRunner; import org.apache.iotdb.itbase.category.MultiClusterIT2Subscription; import org.apache.iotdb.rpc.TSStatusCode; +import org.apache.iotdb.rpc.subscription.config.TopicConstant; import org.apache.iotdb.session.subscription.SubscriptionMessage; +import org.apache.iotdb.session.subscription.SubscriptionMessageType; import org.apache.iotdb.session.subscription.SubscriptionPullConsumer; +import org.apache.iotdb.session.subscription.SubscriptionSession; import org.apache.iotdb.session.subscription.SubscriptionSessionDataSet; import org.apache.iotdb.session.subscription.SubscriptionSessionDataSets; +import org.apache.iotdb.session.subscription.SubscriptionTsFileReader; import org.apache.iotdb.subscription.it.IoTDBSubscriptionITConstant; +import org.apache.iotdb.tsfile.read.TsFileReader; +import org.apache.iotdb.tsfile.read.common.Path; import org.apache.iotdb.tsfile.read.common.RowRecord; +import org.apache.iotdb.tsfile.read.expression.QueryExpression; +import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet; import org.apache.iotdb.tsfile.utils.Pair; import org.awaitility.Awaitility; @@ -54,6 +62,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Properties; import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -84,6 +93,8 @@ public class IoTDBSubscriptionConsumerGroupIT extends AbstractSubscriptionDualIT private static Pair, Map> __3C_1CG_SUBSCRIBE_TWO_TOPIC; private static Pair, Map> __3C_3CG_SUBSCRIBE_TWO_TOPIC; private static Pair, Map> __4C_2CG_SUBSCRIBE_TWO_TOPIC; + private static Pair, Map> + __4C_2CG_SUBSCRIBE_TWO_TOPIC_WITH_TS_FILE; static final class SubscriptionInfo { final String consumerId; @@ -209,6 +220,25 @@ public void setUp() { __4C_2CG_SUBSCRIBE_TWO_TOPIC = new Pair<>(subscriptionInfoList, expectedHeaderWithResult); } + { + final List subscriptionInfoList = new ArrayList<>(); + subscriptionInfoList.add(new SubscriptionInfo("c1", "cg1", Collections.singleton("all"))); + subscriptionInfoList.add( + new SubscriptionInfo("c2", "cg2", new HashSet<>(Arrays.asList("topic1", "topic2")))); + subscriptionInfoList.add(new SubscriptionInfo("c3", "cg1", Collections.singleton("all"))); + subscriptionInfoList.add(new SubscriptionInfo("c4", "cg2", Collections.singleton("topic2"))); + + final Map expectedHeaderWithResult = new HashMap<>(); + expectedHeaderWithResult.put("count(root.cg1.topic1.s)", "100"); + expectedHeaderWithResult.put("count(root.cg1.topic2.s)", "100"); + expectedHeaderWithResult.put("count(root.cg2.topic1.s)", "100"); + expectedHeaderWithResult.put("count(root.cg2.topic2.s)", "100"); + expectedHeaderWithResult.put("count(root.topic1.s)", "100"); + expectedHeaderWithResult.put("count(root.topic2.s)", "100"); + + __4C_2CG_SUBSCRIBE_TWO_TOPIC_WITH_TS_FILE = + new Pair<>(subscriptionInfoList, expectedHeaderWithResult); + } } private void testSubscriptionHistoricalDataTemplate( @@ -221,10 +251,10 @@ private void testSubscriptionHistoricalDataTemplate( // Insert some historical data insertData(currentTime); - // Create topic 'topic1' and 'topic2' + // Create topics createTopics(currentTime); - // Create pipe 'sync_topic1' and 'sync_topic2' with given connector attributes + // Create pipes with given connector attributes createPipes(currentTime, connectorAttributes); // Create subscription and check result @@ -251,10 +281,10 @@ private void testSubscriptionRealtimeDataTemplate( throws Exception { final long currentTime = System.currentTimeMillis(); - // Create topic 'topic1' and 'topic2' + // Create topics createTopics(currentTime); - // Create pipe 'sync_topic1' and 'sync_topic2' with given connector attributes + // Create pipes with given connector attributes createPipes(currentTime, connectorAttributes); // Insert some realtime data @@ -617,15 +647,104 @@ public void test4C2CGSubscribeTwoTopicRealtimeDataWithAirGapConnector() throws E __4C_2CG_SUBSCRIBE_TWO_TOPIC.right); } + // ------------------------------------------------------ // + // 4 consumers, 2 consumer groups, 2 topics (with tsfile) // + // ------------------------------------------------------ // + + @Test + public void test4C2CGSubscribeTwoTopicWithTsFileHistoricalDataWithAsyncConnector() + throws Exception { + testSubscriptionHistoricalDataTemplate( + ASYNC_CONNECTOR_ATTRIBUTES, + __4C_2CG_SUBSCRIBE_TWO_TOPIC_WITH_TS_FILE.left, + __4C_2CG_SUBSCRIBE_TWO_TOPIC_WITH_TS_FILE.right); + } + + @Test + public void test4C2CGSubscribeTwoTopicWithTsFileHistoricalDataWithSyncConnector() + throws Exception { + testSubscriptionHistoricalDataTemplate( + SYNC_CONNECTOR_ATTRIBUTES, + __4C_2CG_SUBSCRIBE_TWO_TOPIC_WITH_TS_FILE.left, + __4C_2CG_SUBSCRIBE_TWO_TOPIC_WITH_TS_FILE.right); + } + + @Test + public void test4C2CGSubscribeTwoTopicWithTsFileHistoricalDataWithLegacyConnector() + throws Exception { + testSubscriptionHistoricalDataTemplate( + LEGACY_CONNECTOR_ATTRIBUTES, + __4C_2CG_SUBSCRIBE_TWO_TOPIC_WITH_TS_FILE.left, + __4C_2CG_SUBSCRIBE_TWO_TOPIC_WITH_TS_FILE.right); + } + + @Test + public void test4C2CGSubscribeTwoTopicWithTsFileHistoricalDataWithAirGapConnector() + throws Exception { + testSubscriptionHistoricalDataTemplate( + AIR_GAP_CONNECTOR_ATTRIBUTES, + __4C_2CG_SUBSCRIBE_TWO_TOPIC_WITH_TS_FILE.left, + __4C_2CG_SUBSCRIBE_TWO_TOPIC_WITH_TS_FILE.right); + } + + @Test + public void test4C2CGSubscribeTwoTopicWithTsFileRealtimeDataWithAsyncConnector() + throws Exception { + testSubscriptionRealtimeDataTemplate( + ASYNC_CONNECTOR_ATTRIBUTES, + __4C_2CG_SUBSCRIBE_TWO_TOPIC_WITH_TS_FILE.left, + __4C_2CG_SUBSCRIBE_TWO_TOPIC_WITH_TS_FILE.right); + } + + @Test + public void test4C2CGSubscribeTwoTopicWithTsFileRealtimeDataWithSyncConnector() throws Exception { + testSubscriptionRealtimeDataTemplate( + SYNC_CONNECTOR_ATTRIBUTES, + __4C_2CG_SUBSCRIBE_TWO_TOPIC_WITH_TS_FILE.left, + __4C_2CG_SUBSCRIBE_TWO_TOPIC_WITH_TS_FILE.right); + } + + @Test + public void test4C2CGSubscribeTwoTopicWithTsFileRealtimeDataWithLegacyConnector() + throws Exception { + testSubscriptionRealtimeDataTemplate( + LEGACY_CONNECTOR_ATTRIBUTES, + __4C_2CG_SUBSCRIBE_TWO_TOPIC_WITH_TS_FILE.left, + __4C_2CG_SUBSCRIBE_TWO_TOPIC_WITH_TS_FILE.right); + } + + @Test + public void test4C2CGSubscribeTwoTopicWithTsFileRealtimeDataWithAirGapConnector() + throws Exception { + testSubscriptionRealtimeDataTemplate( + AIR_GAP_CONNECTOR_ATTRIBUTES, + __4C_2CG_SUBSCRIBE_TWO_TOPIC_WITH_TS_FILE.left, + __4C_2CG_SUBSCRIBE_TWO_TOPIC_WITH_TS_FILE.right); + } + /////////////////////////////// utility /////////////////////////////// private void createTopics(final long currentTime) { // Create topics on sender - try (final ISession session = senderEnv.getSessionConnection()) { - session.executeNonQueryStatement( - String.format("create topic topic1 with ('end-time'='%s')", currentTime - 1)); - session.executeNonQueryStatement( - String.format("create topic topic2 with ('start-time'='%s')", currentTime)); + final String host = senderEnv.getIP(); + final int port = Integer.parseInt(senderEnv.getPort()); + try (final SubscriptionSession session = new SubscriptionSession(host, port)) { + session.open(); + { + final Properties config = new Properties(); + config.put(TopicConstant.END_TIME_KEY, currentTime - 1); + session.createTopic("topic1", config); + } + { + final Properties config = new Properties(); + config.put(TopicConstant.START_TIME_KEY, currentTime); + session.createTopic("topic2", config); + } + { + final Properties config = new Properties(); + config.put(TopicConstant.FORMAT_KEY, TopicConstant.FORMAT_TS_FILE_READER_VALUE); + session.createTopic("all", config); + } } catch (final Exception e) { e.printStackTrace(); fail(e.getMessage()); @@ -728,18 +847,57 @@ private void pollMessagesAndCheck( final List messages = consumer.poll(IoTDBSubscriptionITConstant.POLL_TIMEOUT_MS); for (final SubscriptionMessage message : messages) { - final SubscriptionSessionDataSets payload = - (SubscriptionSessionDataSets) message.getPayload(); - for (final SubscriptionSessionDataSet dataSet : payload) { - final List columnNameList = dataSet.getColumnNames(); - while (dataSet.hasNext()) { - final RowRecord record = dataSet.next(); - if (!insertRowRecordEnrichedByConsumerGroupId( - columnNameList, record, consumerGroupId)) { - receiverCrashed.set(true); - throw new RuntimeException("detect receiver crashed"); - } + final short messageType = message.getMessageType(); + if (SubscriptionMessageType.isValidatedMessageType(messageType)) { + switch (SubscriptionMessageType.valueOf(messageType)) { + case SESSION_DATA_SET: + { + final SubscriptionSessionDataSets payload = + (SubscriptionSessionDataSets) message.getPayload(); + for (final SubscriptionSessionDataSet dataSet : payload) { + final List columnNameList = dataSet.getColumnNames(); + while (dataSet.hasNext()) { + final RowRecord record = dataSet.next(); + if (!insertRowRecordEnrichedByConsumerGroupId( + columnNameList, record, consumerGroupId)) { + receiverCrashed.set(true); + throw new RuntimeException("detect receiver crashed"); + } + } + } + break; + } + case TS_FILE_READER: + { + final SubscriptionTsFileReader reader = + (SubscriptionTsFileReader) message.getPayload(); + try (final TsFileReader tsFileReader = reader.open()) { + final QueryDataSet dataSet = + tsFileReader.query( + QueryExpression.create( + Arrays.asList( + new Path("root.topic1", "s", true), + new Path("root.topic2", "s", true)), + null)); + while (dataSet.hasNext()) { + final RowRecord record = dataSet.next(); + if (!insertRowRecordEnrichedByConsumerGroupId( + dataSet.getPaths().get(0).toString(), + record, + consumerGroupId)) { + receiverCrashed.set(true); + throw new RuntimeException("detect receiver crashed"); + } + } + } + break; + } + default: + LOGGER.warn("unexpected message type: {}", messageType); + break; } + } else { + LOGGER.warn("unexpected message type: {}", messageType); } } consumer.commitSync(messages); @@ -799,6 +957,13 @@ private boolean insertRowRecordEnrichedByConsumerGroupId( throw new Exception("unexpected column name list"); } final String columnName = columnNameList.get(1); + return insertRowRecordEnrichedByConsumerGroupId(columnName, record, consumerGroupId); + } + + /** @return false -> receiver crashed */ + private boolean insertRowRecordEnrichedByConsumerGroupId( + final String columnName, final RowRecord record, final String consumerGroupId) + throws Exception { if ("root.topic1.s".equals(columnName)) { final String sql = String.format( diff --git a/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionTopicIT.java b/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionTopicIT.java index 6379fe66c2b0..c4e67cffd154 100644 --- a/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionTopicIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionTopicIT.java @@ -40,6 +40,7 @@ import java.sql.Connection; import java.sql.Statement; +import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; @@ -485,4 +486,41 @@ public void testTopicNameWithBackQuote() throws Exception { thread.join(); } } + + @Test + public void testInvalidTopicConfig() { + final String host = senderEnv.getIP(); + final int port = Integer.parseInt(senderEnv.getPort()); + + final List configs = new ArrayList<>(); + { + final Properties config = new Properties(); + config.put(TopicConstant.FORMAT_KEY, TopicConstant.FORMAT_TS_FILE_READER_VALUE); + config.put(TopicConstant.PATH_KEY, "root.db.*.s"); + configs.add(config); + } + { + final Properties config = new Properties(); + config.put(TopicConstant.FORMAT_KEY, TopicConstant.FORMAT_TS_FILE_READER_VALUE); + config.put(TopicConstant.START_TIME_KEY, System.currentTimeMillis()); + configs.add(config); + } + { + final Properties config = new Properties(); + config.put(TopicConstant.FORMAT_KEY, TopicConstant.FORMAT_TS_FILE_READER_VALUE); + config.put("processor", "tumbling-time-sampling-processor"); + config.put("processor.tumbling-time.interval-seconds", "1"); + config.put("processor.down-sampling.split-file", "true"); + configs.add(config); + } + + for (final Properties config : configs) { + try (final SubscriptionSession session = new SubscriptionSession(host, port)) { + session.open(); + session.createTopic("foo", config); + fail(); + } catch (final Exception ignored) { + } + } + } } From 5db570fcdbcf2b0ed0731ae067e8bfaacbeafb41 Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Wed, 17 Apr 2024 13:59:09 +0800 Subject: [PATCH 22/93] fix --- .../it/dual/IoTDBSubscriptionConsumerGroupIT.java | 5 ++--- .../session/subscription/SubscriptionConsumer.java | 13 ++++++++++++- .../subscription/SubscriptionPullConsumer.java | 4 ++-- .../broker/SubscriptionPrefetchingQueue.java | 3 +++ .../broker/SubscriptionPrefetchingTsFileQueue.java | 4 ++++ 5 files changed, 23 insertions(+), 6 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionConsumerGroupIT.java b/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionConsumerGroupIT.java index 4be8100ef6dc..0ae6d3afa812 100644 --- a/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionConsumerGroupIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionConsumerGroupIT.java @@ -907,11 +907,10 @@ private void pollMessagesAndCheck( e.printStackTrace(); // Avoid failure } finally { - LOGGER.info( - "consumer {} (consumer group {}) exiting...", consumerId, consumerGroupId); + LOGGER.info("consumer {} exiting...", consumers.get(index)); } }, - String.format("%s_%s", consumerId, consumerGroupId)); + consumers.get(index).toString()); t.start(); threads.add(t); } diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java index e7544c549a52..92d44ccf7242 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java @@ -89,6 +89,17 @@ public String getConsumerGroupId() { return consumerGroupId; } + /////////////////////////////// object /////////////////////////////// + + @Override + public String toString() { + return "SubscriptionConsumer{consumerId=" + + consumerId + + ", consumerGroupId=" + + consumerGroupId + + "}"; + } + /////////////////////////////// tsfile dir /////////////////////////////// protected Path subscribedTsFileBaseDirPath; @@ -100,7 +111,7 @@ public Path getTsFileDir(String topicName) throws IOException { if (Objects.isNull(subscribedTsFileBaseDirPath)) { subscribedTsFileBaseDirPath = Files.createTempDirectory( - String.format("subscribedTsFile_%s_%s", consumerId, consumerGroupId)); + String.format("subscribedTsFile_%s_%s#", consumerId, consumerGroupId)); } final Path dirPath = subscribedTsFileBaseDirPath.resolve(topicName); Files.createDirectories(dirPath); diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java index 146489afbee9..9173d1fff054 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java @@ -246,7 +246,7 @@ private SubscriptionMessage pollTsFile( final RandomAccessFile fileWriter = new RandomAccessFile(file, "rw"); commitContextToTsFile.put(commitContext, new Pair<>(file, fileWriter)); - LOGGER.info("start poll tsfile: {}", file.getAbsolutePath()); + LOGGER.info("{} start poll tsfile: {}", this, file.getAbsolutePath()); long endWritingOffset = 0; while (true) { @@ -299,7 +299,7 @@ private SubscriptionMessage pollTsFile( fileWriter.getFD().sync(); fileWriter.close(); commitContextToTsFile.remove(commitContext); - LOGGER.info("successfully poll tsfile: {}", file.getAbsolutePath()); + LOGGER.info("{} successfully poll tsfile: {}", this, file.getAbsolutePath()); // generate subscription message return new SubscriptionMessage(commitContext, file.getAbsolutePath()); } 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 100b7421c800..dcb974d41681 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 @@ -73,6 +73,9 @@ public void commit(final SubscriptionCommitContext commitContext) { event.decreaseReferenceCount(); event.recordCommittedTimestamp(); uncommittedEvents.remove(commitContext); + if (this instanceof SubscriptionPrefetchingTsFileQueue) { + ((SubscriptionPrefetchingTsFileQueue) this).resetEventRef(); + } } protected SubscriptionCommitContext generateSubscriptionCommitContext() { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java index 0570cf68fd07..191fe484c811 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java @@ -228,4 +228,8 @@ public SubscriptionEvent pollTsFile(String fileName, long endWritingOffset) { public void executePrefetch() { // do nothing now } + + void resetEventRef() { + eventRef.set(null); + } } From a90ecd34e00271774a9bd511661bc641cfe9ed70 Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Wed, 17 Apr 2024 15:03:50 +0800 Subject: [PATCH 23/93] fix --- .../iotdb/SubscriptionSessionExample.java | 12 +- .../IoTDBSubscriptionConsumerGroupIT.java | 680 +++++++++--------- .../subscription/SubscriptionConsumer.java | 7 +- .../SubscriptionPrefetchingTsFileQueue.java | 2 +- 4 files changed, 355 insertions(+), 346 deletions(-) diff --git a/example/session/src/main/java/org/apache/iotdb/SubscriptionSessionExample.java b/example/session/src/main/java/org/apache/iotdb/SubscriptionSessionExample.java index 70f4f2268631..46c4da1c5d83 100644 --- a/example/session/src/main/java/org/apache/iotdb/SubscriptionSessionExample.java +++ b/example/session/src/main/java/org/apache/iotdb/SubscriptionSessionExample.java @@ -114,6 +114,7 @@ private static void createTopics() throws Exception { } private static void subscriptionExample1() throws Exception { + int retryCount = 0; // Subscription: property-style ctor final Properties config = new Properties(); config.put(ConsumerConstant.CONSUMER_ID_KEY, "c1"); @@ -125,7 +126,10 @@ private static void subscriptionExample1() throws Exception { Thread.sleep(1000); // Wait for some time final List messages = consumer1.poll(Duration.ofMillis(10000)); if (messages.isEmpty()) { - break; + retryCount++; + if (retryCount >= 5) { + break; + } } for (final SubscriptionMessage message : messages) { final SubscriptionSessionDataSets dataSets = @@ -153,6 +157,7 @@ private static void subscriptionExample1() throws Exception { } private static void subscriptionExample2() throws Exception { + int retryCount = 0; // Subscription: builder-style ctor try (final SubscriptionPullConsumer consumer2 = new SubscriptionPullConsumer.Builder() @@ -167,7 +172,10 @@ private static void subscriptionExample2() throws Exception { final List messages = consumer2.poll(Collections.singleton(TOPIC_2), Duration.ofMillis(10000)); if (messages.isEmpty()) { - break; + retryCount++; + if (retryCount >= 5) { + break; + } } for (final SubscriptionMessage message : messages) { final SubscriptionTsFileReader reader = (SubscriptionTsFileReader) message.getPayload(); diff --git a/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionConsumerGroupIT.java b/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionConsumerGroupIT.java index 0ae6d3afa812..5cc5c7febc85 100644 --- a/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionConsumerGroupIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionConsumerGroupIT.java @@ -247,6 +247,7 @@ private void testSubscriptionHistoricalDataTemplate( final Map expectedHeaderWithResult) throws Exception { final long currentTime = System.currentTimeMillis(); + LOGGER.info("currentTime: {}", currentTime); // Insert some historical data insertData(currentTime); @@ -280,6 +281,7 @@ private void testSubscriptionRealtimeDataTemplate( final Map expectedHeaderWithResult) throws Exception { final long currentTime = System.currentTimeMillis(); + LOGGER.info("currentTime: {}", currentTime); // Create topics createTopics(currentTime); @@ -307,345 +309,345 @@ private void testSubscriptionRealtimeDataTemplate( expectedHeaderWithResult); } - // -------------------------------------- // - // 3 consumers, 1 consumer group, 1 topic // - // -------------------------------------- // - - @Test - public void test3C1CGSubscribeOneTopicHistoricalDataWithAsyncConnector() throws Exception { - testSubscriptionHistoricalDataTemplate( - ASYNC_CONNECTOR_ATTRIBUTES, - __3C_1CG_SUBSCRIBE_ONE_TOPIC.left, - __3C_1CG_SUBSCRIBE_ONE_TOPIC.right); - } - - @Test - public void test3C1CGSubscribeOneTopicHistoricalDataWithSyncConnector() throws Exception { - testSubscriptionHistoricalDataTemplate( - SYNC_CONNECTOR_ATTRIBUTES, - __3C_1CG_SUBSCRIBE_ONE_TOPIC.left, - __3C_1CG_SUBSCRIBE_ONE_TOPIC.right); - } - - @Test - public void test3C1CGSubscribeOneTopicHistoricalDataWithLegacyConnector() throws Exception { - testSubscriptionHistoricalDataTemplate( - LEGACY_CONNECTOR_ATTRIBUTES, - __3C_1CG_SUBSCRIBE_ONE_TOPIC.left, - __3C_1CG_SUBSCRIBE_ONE_TOPIC.right); - } - - @Test - public void test3C1CGSubscribeOneTopicHistoricalDataWithAirGapConnector() throws Exception { - testSubscriptionHistoricalDataTemplate( - AIR_GAP_CONNECTOR_ATTRIBUTES, - __3C_1CG_SUBSCRIBE_ONE_TOPIC.left, - __3C_1CG_SUBSCRIBE_ONE_TOPIC.right); - } - - @Test - public void test3C1CGSubscribeOneTopicRealtimeDataWithAsyncConnector() throws Exception { - testSubscriptionRealtimeDataTemplate( - ASYNC_CONNECTOR_ATTRIBUTES, - __3C_1CG_SUBSCRIBE_ONE_TOPIC.left, - __3C_1CG_SUBSCRIBE_ONE_TOPIC.right); - } - - @Test - public void test3C1CGSubscribeOneTopicRealtimeDataWithSyncConnector() throws Exception { - testSubscriptionRealtimeDataTemplate( - SYNC_CONNECTOR_ATTRIBUTES, - __3C_1CG_SUBSCRIBE_ONE_TOPIC.left, - __3C_1CG_SUBSCRIBE_ONE_TOPIC.right); - } - - @Test - public void test3C1CGSubscribeOneTopicRealtimeDataWithLegacyConnector() throws Exception { - testSubscriptionRealtimeDataTemplate( - LEGACY_CONNECTOR_ATTRIBUTES, - __3C_1CG_SUBSCRIBE_ONE_TOPIC.left, - __3C_1CG_SUBSCRIBE_ONE_TOPIC.right); - } - - @Test - public void test3C1CGSubscribeOneTopicRealtimeDataWithAirGapConnector() throws Exception { - testSubscriptionRealtimeDataTemplate( - AIR_GAP_CONNECTOR_ATTRIBUTES, - __3C_1CG_SUBSCRIBE_ONE_TOPIC.left, - __3C_1CG_SUBSCRIBE_ONE_TOPIC.right); - } - - // --------------------------------------- // - // 3 consumers, 3 consumer groups, 1 topic // - // --------------------------------------- // - - @Test - public void test3C3CGSubscribeOneTopicHistoricalDataWithAsyncConnector() throws Exception { - testSubscriptionHistoricalDataTemplate( - ASYNC_CONNECTOR_ATTRIBUTES, - __3C_3CG_SUBSCRIBE_ONE_TOPIC.left, - __3C_3CG_SUBSCRIBE_ONE_TOPIC.right); - } - - @Test - public void test3C3CGSubscribeOneTopicHistoricalDataWithSyncConnector() throws Exception { - testSubscriptionHistoricalDataTemplate( - SYNC_CONNECTOR_ATTRIBUTES, - __3C_3CG_SUBSCRIBE_ONE_TOPIC.left, - __3C_3CG_SUBSCRIBE_ONE_TOPIC.right); - } - - @Test - public void test3C3CGSubscribeOneTopicHistoricalDataWithLegacyConnector() throws Exception { - testSubscriptionHistoricalDataTemplate( - LEGACY_CONNECTOR_ATTRIBUTES, - __3C_3CG_SUBSCRIBE_ONE_TOPIC.left, - __3C_3CG_SUBSCRIBE_ONE_TOPIC.right); - } - - @Test - public void test3C3CGSubscribeOneTopicHistoricalDataWithAirGapConnector() throws Exception { - testSubscriptionHistoricalDataTemplate( - AIR_GAP_CONNECTOR_ATTRIBUTES, - __3C_3CG_SUBSCRIBE_ONE_TOPIC.left, - __3C_3CG_SUBSCRIBE_ONE_TOPIC.right); - } - - @Test - public void test3C3CGSubscribeOneTopicRealtimeDataWithAsyncConnector() throws Exception { - testSubscriptionRealtimeDataTemplate( - ASYNC_CONNECTOR_ATTRIBUTES, - __3C_3CG_SUBSCRIBE_ONE_TOPIC.left, - __3C_3CG_SUBSCRIBE_ONE_TOPIC.right); - } - - @Test - public void test3C3CGSubscribeOneTopicRealtimeDataWithSyncConnector() throws Exception { - testSubscriptionRealtimeDataTemplate( - SYNC_CONNECTOR_ATTRIBUTES, - __3C_3CG_SUBSCRIBE_ONE_TOPIC.left, - __3C_3CG_SUBSCRIBE_ONE_TOPIC.right); - } - - @Test - public void test3C3CGSubscribeOneTopicRealtimeDataWithLegacyConnector() throws Exception { - testSubscriptionRealtimeDataTemplate( - LEGACY_CONNECTOR_ATTRIBUTES, - __3C_3CG_SUBSCRIBE_ONE_TOPIC.left, - __3C_3CG_SUBSCRIBE_ONE_TOPIC.right); - } - - @Test - public void test3C3CGSubscribeOneTopicRealtimeDataWithAirGapConnector() throws Exception { - testSubscriptionRealtimeDataTemplate( - AIR_GAP_CONNECTOR_ATTRIBUTES, - __3C_3CG_SUBSCRIBE_ONE_TOPIC.left, - __3C_3CG_SUBSCRIBE_ONE_TOPIC.right); - } - - // --------------------------------------- // - // 3 consumers, 1 consumer group, 2 topics // - // --------------------------------------- // - - @Test - public void test3C1CGSubscribeTwoTopicHistoricalDataWithAsyncConnector() throws Exception { - testSubscriptionHistoricalDataTemplate( - ASYNC_CONNECTOR_ATTRIBUTES, - __3C_1CG_SUBSCRIBE_TWO_TOPIC.left, - __3C_1CG_SUBSCRIBE_TWO_TOPIC.right); - } - - @Test - public void test3C1CGSubscribeTwoTopicHistoricalDataWithSyncConnector() throws Exception { - testSubscriptionHistoricalDataTemplate( - SYNC_CONNECTOR_ATTRIBUTES, - __3C_1CG_SUBSCRIBE_TWO_TOPIC.left, - __3C_1CG_SUBSCRIBE_TWO_TOPIC.right); - } - - @Test - public void test3C1CGSubscribeTwoTopicHistoricalDataWithLegacyConnector() throws Exception { - testSubscriptionHistoricalDataTemplate( - LEGACY_CONNECTOR_ATTRIBUTES, - __3C_1CG_SUBSCRIBE_TWO_TOPIC.left, - __3C_1CG_SUBSCRIBE_TWO_TOPIC.right); - } - - @Test - public void test3C1CGSubscribeTwoTopicHistoricalDataWithAirGapConnector() throws Exception { - testSubscriptionHistoricalDataTemplate( - AIR_GAP_CONNECTOR_ATTRIBUTES, - __3C_1CG_SUBSCRIBE_TWO_TOPIC.left, - __3C_1CG_SUBSCRIBE_TWO_TOPIC.right); - } - - @Test - public void test3C1CGSubscribeTwoTopicRealtimeDataWithAsyncConnector() throws Exception { - testSubscriptionRealtimeDataTemplate( - ASYNC_CONNECTOR_ATTRIBUTES, - __3C_1CG_SUBSCRIBE_TWO_TOPIC.left, - __3C_1CG_SUBSCRIBE_TWO_TOPIC.right); - } - - @Test - public void test3C1CGSubscribeTwoTopicRealtimeDataWithSyncConnector() throws Exception { - testSubscriptionRealtimeDataTemplate( - SYNC_CONNECTOR_ATTRIBUTES, - __3C_1CG_SUBSCRIBE_TWO_TOPIC.left, - __3C_1CG_SUBSCRIBE_TWO_TOPIC.right); - } - - @Test - public void test3C1CGSubscribeTwoTopicRealtimeDataWithLegacyConnector() throws Exception { - testSubscriptionRealtimeDataTemplate( - LEGACY_CONNECTOR_ATTRIBUTES, - __3C_1CG_SUBSCRIBE_TWO_TOPIC.left, - __3C_1CG_SUBSCRIBE_TWO_TOPIC.right); - } - - @Test - public void test3C1CGSubscribeTwoTopicRealtimeDataWithAirGapConnector() throws Exception { - testSubscriptionRealtimeDataTemplate( - AIR_GAP_CONNECTOR_ATTRIBUTES, - __3C_1CG_SUBSCRIBE_TWO_TOPIC.left, - __3C_1CG_SUBSCRIBE_TWO_TOPIC.right); - } - - // ---------------------------------------- // - // 3 consumers, 3 consumer groups, 2 topics // - // ---------------------------------------- // - - @Test - public void test3C3CGSubscribeTwoTopicHistoricalDataWithAsyncConnector() throws Exception { - testSubscriptionHistoricalDataTemplate( - ASYNC_CONNECTOR_ATTRIBUTES, - __3C_3CG_SUBSCRIBE_TWO_TOPIC.left, - __3C_3CG_SUBSCRIBE_TWO_TOPIC.right); - } - - @Test - public void test3C3CGSubscribeTwoTopicHistoricalDataWithSyncConnector() throws Exception { - testSubscriptionHistoricalDataTemplate( - SYNC_CONNECTOR_ATTRIBUTES, - __3C_3CG_SUBSCRIBE_TWO_TOPIC.left, - __3C_3CG_SUBSCRIBE_TWO_TOPIC.right); - } - - @Test - public void test3C3CGSubscribeTwoTopicHistoricalDataWithLegacyConnector() throws Exception { - testSubscriptionHistoricalDataTemplate( - LEGACY_CONNECTOR_ATTRIBUTES, - __3C_3CG_SUBSCRIBE_TWO_TOPIC.left, - __3C_3CG_SUBSCRIBE_TWO_TOPIC.right); - } - - @Test - public void test3C3CGSubscribeTwoTopicHistoricalDataWithAirGapConnector() throws Exception { - testSubscriptionHistoricalDataTemplate( - AIR_GAP_CONNECTOR_ATTRIBUTES, - __3C_3CG_SUBSCRIBE_TWO_TOPIC.left, - __3C_3CG_SUBSCRIBE_TWO_TOPIC.right); - } - - @Test - public void test3C3CGSubscribeTwoTopicRealtimeDataWithAsyncConnector() throws Exception { - testSubscriptionRealtimeDataTemplate( - ASYNC_CONNECTOR_ATTRIBUTES, - __3C_3CG_SUBSCRIBE_TWO_TOPIC.left, - __3C_3CG_SUBSCRIBE_TWO_TOPIC.right); - } - - @Test - public void test3C3CGSubscribeTwoTopicRealtimeDataWithSyncConnector() throws Exception { - testSubscriptionRealtimeDataTemplate( - SYNC_CONNECTOR_ATTRIBUTES, - __3C_3CG_SUBSCRIBE_TWO_TOPIC.left, - __3C_3CG_SUBSCRIBE_TWO_TOPIC.right); - } - - @Test - public void test3C3CGSubscribeTwoTopicRealtimeDataWithLegacyConnector() throws Exception { - testSubscriptionRealtimeDataTemplate( - LEGACY_CONNECTOR_ATTRIBUTES, - __3C_3CG_SUBSCRIBE_TWO_TOPIC.left, - __3C_3CG_SUBSCRIBE_TWO_TOPIC.right); - } - - @Test - public void test3C3CGSubscribeTwoTopicRealtimeDataWithAirGapConnector() throws Exception { - testSubscriptionRealtimeDataTemplate( - AIR_GAP_CONNECTOR_ATTRIBUTES, - __3C_3CG_SUBSCRIBE_TWO_TOPIC.left, - __3C_3CG_SUBSCRIBE_TWO_TOPIC.right); - } - - // ---------------------------------------- // - // 4 consumers, 2 consumer groups, 2 topics // - // ---------------------------------------- // - - @Test - public void test4C2CGSubscribeTwoTopicHistoricalDataWithAsyncConnector() throws Exception { - testSubscriptionHistoricalDataTemplate( - ASYNC_CONNECTOR_ATTRIBUTES, - __4C_2CG_SUBSCRIBE_TWO_TOPIC.left, - __4C_2CG_SUBSCRIBE_TWO_TOPIC.right); - } - - @Test - public void test4C2CGSubscribeTwoTopicHistoricalDataWithSyncConnector() throws Exception { - testSubscriptionHistoricalDataTemplate( - SYNC_CONNECTOR_ATTRIBUTES, - __4C_2CG_SUBSCRIBE_TWO_TOPIC.left, - __4C_2CG_SUBSCRIBE_TWO_TOPIC.right); - } - - @Test - public void test4C2CGSubscribeTwoTopicHistoricalDataWithLegacyConnector() throws Exception { - testSubscriptionHistoricalDataTemplate( - LEGACY_CONNECTOR_ATTRIBUTES, - __4C_2CG_SUBSCRIBE_TWO_TOPIC.left, - __4C_2CG_SUBSCRIBE_TWO_TOPIC.right); - } - - @Test - public void test4C2CGSubscribeTwoTopicHistoricalDataWithAirGapConnector() throws Exception { - testSubscriptionHistoricalDataTemplate( - AIR_GAP_CONNECTOR_ATTRIBUTES, - __4C_2CG_SUBSCRIBE_TWO_TOPIC.left, - __4C_2CG_SUBSCRIBE_TWO_TOPIC.right); - } - - @Test - public void test4C2CGSubscribeTwoTopicRealtimeDataWithAsyncConnector() throws Exception { - testSubscriptionRealtimeDataTemplate( - ASYNC_CONNECTOR_ATTRIBUTES, - __4C_2CG_SUBSCRIBE_TWO_TOPIC.left, - __4C_2CG_SUBSCRIBE_TWO_TOPIC.right); - } - - @Test - public void test4C2CGSubscribeTwoTopicRealtimeDataWithSyncConnector() throws Exception { - testSubscriptionRealtimeDataTemplate( - SYNC_CONNECTOR_ATTRIBUTES, - __4C_2CG_SUBSCRIBE_TWO_TOPIC.left, - __4C_2CG_SUBSCRIBE_TWO_TOPIC.right); - } - - @Test - public void test4C2CGSubscribeTwoTopicRealtimeDataWithLegacyConnector() throws Exception { - testSubscriptionRealtimeDataTemplate( - LEGACY_CONNECTOR_ATTRIBUTES, - __4C_2CG_SUBSCRIBE_TWO_TOPIC.left, - __4C_2CG_SUBSCRIBE_TWO_TOPIC.right); - } - - @Test - public void test4C2CGSubscribeTwoTopicRealtimeDataWithAirGapConnector() throws Exception { - testSubscriptionRealtimeDataTemplate( - AIR_GAP_CONNECTOR_ATTRIBUTES, - __4C_2CG_SUBSCRIBE_TWO_TOPIC.left, - __4C_2CG_SUBSCRIBE_TWO_TOPIC.right); - } + // // -------------------------------------- // + // // 3 consumers, 1 consumer group, 1 topic // + // // -------------------------------------- // + // + // @Test + // public void test3C1CGSubscribeOneTopicHistoricalDataWithAsyncConnector() throws Exception { + // testSubscriptionHistoricalDataTemplate( + // ASYNC_CONNECTOR_ATTRIBUTES, + // __3C_1CG_SUBSCRIBE_ONE_TOPIC.left, + // __3C_1CG_SUBSCRIBE_ONE_TOPIC.right); + // } + // + // @Test + // public void test3C1CGSubscribeOneTopicHistoricalDataWithSyncConnector() throws Exception { + // testSubscriptionHistoricalDataTemplate( + // SYNC_CONNECTOR_ATTRIBUTES, + // __3C_1CG_SUBSCRIBE_ONE_TOPIC.left, + // __3C_1CG_SUBSCRIBE_ONE_TOPIC.right); + // } + // + // @Test + // public void test3C1CGSubscribeOneTopicHistoricalDataWithLegacyConnector() throws Exception { + // testSubscriptionHistoricalDataTemplate( + // LEGACY_CONNECTOR_ATTRIBUTES, + // __3C_1CG_SUBSCRIBE_ONE_TOPIC.left, + // __3C_1CG_SUBSCRIBE_ONE_TOPIC.right); + // } + // + // @Test + // public void test3C1CGSubscribeOneTopicHistoricalDataWithAirGapConnector() throws Exception { + // testSubscriptionHistoricalDataTemplate( + // AIR_GAP_CONNECTOR_ATTRIBUTES, + // __3C_1CG_SUBSCRIBE_ONE_TOPIC.left, + // __3C_1CG_SUBSCRIBE_ONE_TOPIC.right); + // } + // + // @Test + // public void test3C1CGSubscribeOneTopicRealtimeDataWithAsyncConnector() throws Exception { + // testSubscriptionRealtimeDataTemplate( + // ASYNC_CONNECTOR_ATTRIBUTES, + // __3C_1CG_SUBSCRIBE_ONE_TOPIC.left, + // __3C_1CG_SUBSCRIBE_ONE_TOPIC.right); + // } + // + // @Test + // public void test3C1CGSubscribeOneTopicRealtimeDataWithSyncConnector() throws Exception { + // testSubscriptionRealtimeDataTemplate( + // SYNC_CONNECTOR_ATTRIBUTES, + // __3C_1CG_SUBSCRIBE_ONE_TOPIC.left, + // __3C_1CG_SUBSCRIBE_ONE_TOPIC.right); + // } + // + // @Test + // public void test3C1CGSubscribeOneTopicRealtimeDataWithLegacyConnector() throws Exception { + // testSubscriptionRealtimeDataTemplate( + // LEGACY_CONNECTOR_ATTRIBUTES, + // __3C_1CG_SUBSCRIBE_ONE_TOPIC.left, + // __3C_1CG_SUBSCRIBE_ONE_TOPIC.right); + // } + // + // @Test + // public void test3C1CGSubscribeOneTopicRealtimeDataWithAirGapConnector() throws Exception { + // testSubscriptionRealtimeDataTemplate( + // AIR_GAP_CONNECTOR_ATTRIBUTES, + // __3C_1CG_SUBSCRIBE_ONE_TOPIC.left, + // __3C_1CG_SUBSCRIBE_ONE_TOPIC.right); + // } + // + // // --------------------------------------- // + // // 3 consumers, 3 consumer groups, 1 topic // + // // --------------------------------------- // + // + // @Test + // public void test3C3CGSubscribeOneTopicHistoricalDataWithAsyncConnector() throws Exception { + // testSubscriptionHistoricalDataTemplate( + // ASYNC_CONNECTOR_ATTRIBUTES, + // __3C_3CG_SUBSCRIBE_ONE_TOPIC.left, + // __3C_3CG_SUBSCRIBE_ONE_TOPIC.right); + // } + // + // @Test + // public void test3C3CGSubscribeOneTopicHistoricalDataWithSyncConnector() throws Exception { + // testSubscriptionHistoricalDataTemplate( + // SYNC_CONNECTOR_ATTRIBUTES, + // __3C_3CG_SUBSCRIBE_ONE_TOPIC.left, + // __3C_3CG_SUBSCRIBE_ONE_TOPIC.right); + // } + // + // @Test + // public void test3C3CGSubscribeOneTopicHistoricalDataWithLegacyConnector() throws Exception { + // testSubscriptionHistoricalDataTemplate( + // LEGACY_CONNECTOR_ATTRIBUTES, + // __3C_3CG_SUBSCRIBE_ONE_TOPIC.left, + // __3C_3CG_SUBSCRIBE_ONE_TOPIC.right); + // } + // + // @Test + // public void test3C3CGSubscribeOneTopicHistoricalDataWithAirGapConnector() throws Exception { + // testSubscriptionHistoricalDataTemplate( + // AIR_GAP_CONNECTOR_ATTRIBUTES, + // __3C_3CG_SUBSCRIBE_ONE_TOPIC.left, + // __3C_3CG_SUBSCRIBE_ONE_TOPIC.right); + // } + // + // @Test + // public void test3C3CGSubscribeOneTopicRealtimeDataWithAsyncConnector() throws Exception { + // testSubscriptionRealtimeDataTemplate( + // ASYNC_CONNECTOR_ATTRIBUTES, + // __3C_3CG_SUBSCRIBE_ONE_TOPIC.left, + // __3C_3CG_SUBSCRIBE_ONE_TOPIC.right); + // } + // + // @Test + // public void test3C3CGSubscribeOneTopicRealtimeDataWithSyncConnector() throws Exception { + // testSubscriptionRealtimeDataTemplate( + // SYNC_CONNECTOR_ATTRIBUTES, + // __3C_3CG_SUBSCRIBE_ONE_TOPIC.left, + // __3C_3CG_SUBSCRIBE_ONE_TOPIC.right); + // } + // + // @Test + // public void test3C3CGSubscribeOneTopicRealtimeDataWithLegacyConnector() throws Exception { + // testSubscriptionRealtimeDataTemplate( + // LEGACY_CONNECTOR_ATTRIBUTES, + // __3C_3CG_SUBSCRIBE_ONE_TOPIC.left, + // __3C_3CG_SUBSCRIBE_ONE_TOPIC.right); + // } + // + // @Test + // public void test3C3CGSubscribeOneTopicRealtimeDataWithAirGapConnector() throws Exception { + // testSubscriptionRealtimeDataTemplate( + // AIR_GAP_CONNECTOR_ATTRIBUTES, + // __3C_3CG_SUBSCRIBE_ONE_TOPIC.left, + // __3C_3CG_SUBSCRIBE_ONE_TOPIC.right); + // } + // + // // --------------------------------------- // + // // 3 consumers, 1 consumer group, 2 topics // + // // --------------------------------------- // + // + // @Test + // public void test3C1CGSubscribeTwoTopicHistoricalDataWithAsyncConnector() throws Exception { + // testSubscriptionHistoricalDataTemplate( + // ASYNC_CONNECTOR_ATTRIBUTES, + // __3C_1CG_SUBSCRIBE_TWO_TOPIC.left, + // __3C_1CG_SUBSCRIBE_TWO_TOPIC.right); + // } + // + // @Test + // public void test3C1CGSubscribeTwoTopicHistoricalDataWithSyncConnector() throws Exception { + // testSubscriptionHistoricalDataTemplate( + // SYNC_CONNECTOR_ATTRIBUTES, + // __3C_1CG_SUBSCRIBE_TWO_TOPIC.left, + // __3C_1CG_SUBSCRIBE_TWO_TOPIC.right); + // } + // + // @Test + // public void test3C1CGSubscribeTwoTopicHistoricalDataWithLegacyConnector() throws Exception { + // testSubscriptionHistoricalDataTemplate( + // LEGACY_CONNECTOR_ATTRIBUTES, + // __3C_1CG_SUBSCRIBE_TWO_TOPIC.left, + // __3C_1CG_SUBSCRIBE_TWO_TOPIC.right); + // } + // + // @Test + // public void test3C1CGSubscribeTwoTopicHistoricalDataWithAirGapConnector() throws Exception { + // testSubscriptionHistoricalDataTemplate( + // AIR_GAP_CONNECTOR_ATTRIBUTES, + // __3C_1CG_SUBSCRIBE_TWO_TOPIC.left, + // __3C_1CG_SUBSCRIBE_TWO_TOPIC.right); + // } + // + // @Test + // public void test3C1CGSubscribeTwoTopicRealtimeDataWithAsyncConnector() throws Exception { + // testSubscriptionRealtimeDataTemplate( + // ASYNC_CONNECTOR_ATTRIBUTES, + // __3C_1CG_SUBSCRIBE_TWO_TOPIC.left, + // __3C_1CG_SUBSCRIBE_TWO_TOPIC.right); + // } + // + // @Test + // public void test3C1CGSubscribeTwoTopicRealtimeDataWithSyncConnector() throws Exception { + // testSubscriptionRealtimeDataTemplate( + // SYNC_CONNECTOR_ATTRIBUTES, + // __3C_1CG_SUBSCRIBE_TWO_TOPIC.left, + // __3C_1CG_SUBSCRIBE_TWO_TOPIC.right); + // } + // + // @Test + // public void test3C1CGSubscribeTwoTopicRealtimeDataWithLegacyConnector() throws Exception { + // testSubscriptionRealtimeDataTemplate( + // LEGACY_CONNECTOR_ATTRIBUTES, + // __3C_1CG_SUBSCRIBE_TWO_TOPIC.left, + // __3C_1CG_SUBSCRIBE_TWO_TOPIC.right); + // } + // + // @Test + // public void test3C1CGSubscribeTwoTopicRealtimeDataWithAirGapConnector() throws Exception { + // testSubscriptionRealtimeDataTemplate( + // AIR_GAP_CONNECTOR_ATTRIBUTES, + // __3C_1CG_SUBSCRIBE_TWO_TOPIC.left, + // __3C_1CG_SUBSCRIBE_TWO_TOPIC.right); + // } + // + // // ---------------------------------------- // + // // 3 consumers, 3 consumer groups, 2 topics // + // // ---------------------------------------- // + // + // @Test + // public void test3C3CGSubscribeTwoTopicHistoricalDataWithAsyncConnector() throws Exception { + // testSubscriptionHistoricalDataTemplate( + // ASYNC_CONNECTOR_ATTRIBUTES, + // __3C_3CG_SUBSCRIBE_TWO_TOPIC.left, + // __3C_3CG_SUBSCRIBE_TWO_TOPIC.right); + // } + // + // @Test + // public void test3C3CGSubscribeTwoTopicHistoricalDataWithSyncConnector() throws Exception { + // testSubscriptionHistoricalDataTemplate( + // SYNC_CONNECTOR_ATTRIBUTES, + // __3C_3CG_SUBSCRIBE_TWO_TOPIC.left, + // __3C_3CG_SUBSCRIBE_TWO_TOPIC.right); + // } + // + // @Test + // public void test3C3CGSubscribeTwoTopicHistoricalDataWithLegacyConnector() throws Exception { + // testSubscriptionHistoricalDataTemplate( + // LEGACY_CONNECTOR_ATTRIBUTES, + // __3C_3CG_SUBSCRIBE_TWO_TOPIC.left, + // __3C_3CG_SUBSCRIBE_TWO_TOPIC.right); + // } + // + // @Test + // public void test3C3CGSubscribeTwoTopicHistoricalDataWithAirGapConnector() throws Exception { + // testSubscriptionHistoricalDataTemplate( + // AIR_GAP_CONNECTOR_ATTRIBUTES, + // __3C_3CG_SUBSCRIBE_TWO_TOPIC.left, + // __3C_3CG_SUBSCRIBE_TWO_TOPIC.right); + // } + // + // @Test + // public void test3C3CGSubscribeTwoTopicRealtimeDataWithAsyncConnector() throws Exception { + // testSubscriptionRealtimeDataTemplate( + // ASYNC_CONNECTOR_ATTRIBUTES, + // __3C_3CG_SUBSCRIBE_TWO_TOPIC.left, + // __3C_3CG_SUBSCRIBE_TWO_TOPIC.right); + // } + // + // @Test + // public void test3C3CGSubscribeTwoTopicRealtimeDataWithSyncConnector() throws Exception { + // testSubscriptionRealtimeDataTemplate( + // SYNC_CONNECTOR_ATTRIBUTES, + // __3C_3CG_SUBSCRIBE_TWO_TOPIC.left, + // __3C_3CG_SUBSCRIBE_TWO_TOPIC.right); + // } + // + // @Test + // public void test3C3CGSubscribeTwoTopicRealtimeDataWithLegacyConnector() throws Exception { + // testSubscriptionRealtimeDataTemplate( + // LEGACY_CONNECTOR_ATTRIBUTES, + // __3C_3CG_SUBSCRIBE_TWO_TOPIC.left, + // __3C_3CG_SUBSCRIBE_TWO_TOPIC.right); + // } + // + // @Test + // public void test3C3CGSubscribeTwoTopicRealtimeDataWithAirGapConnector() throws Exception { + // testSubscriptionRealtimeDataTemplate( + // AIR_GAP_CONNECTOR_ATTRIBUTES, + // __3C_3CG_SUBSCRIBE_TWO_TOPIC.left, + // __3C_3CG_SUBSCRIBE_TWO_TOPIC.right); + // } + // + // // ---------------------------------------- // + // // 4 consumers, 2 consumer groups, 2 topics // + // // ---------------------------------------- // + // + // @Test + // public void test4C2CGSubscribeTwoTopicHistoricalDataWithAsyncConnector() throws Exception { + // testSubscriptionHistoricalDataTemplate( + // ASYNC_CONNECTOR_ATTRIBUTES, + // __4C_2CG_SUBSCRIBE_TWO_TOPIC.left, + // __4C_2CG_SUBSCRIBE_TWO_TOPIC.right); + // } + // + // @Test + // public void test4C2CGSubscribeTwoTopicHistoricalDataWithSyncConnector() throws Exception { + // testSubscriptionHistoricalDataTemplate( + // SYNC_CONNECTOR_ATTRIBUTES, + // __4C_2CG_SUBSCRIBE_TWO_TOPIC.left, + // __4C_2CG_SUBSCRIBE_TWO_TOPIC.right); + // } + // + // @Test + // public void test4C2CGSubscribeTwoTopicHistoricalDataWithLegacyConnector() throws Exception { + // testSubscriptionHistoricalDataTemplate( + // LEGACY_CONNECTOR_ATTRIBUTES, + // __4C_2CG_SUBSCRIBE_TWO_TOPIC.left, + // __4C_2CG_SUBSCRIBE_TWO_TOPIC.right); + // } + // + // @Test + // public void test4C2CGSubscribeTwoTopicHistoricalDataWithAirGapConnector() throws Exception { + // testSubscriptionHistoricalDataTemplate( + // AIR_GAP_CONNECTOR_ATTRIBUTES, + // __4C_2CG_SUBSCRIBE_TWO_TOPIC.left, + // __4C_2CG_SUBSCRIBE_TWO_TOPIC.right); + // } + // + // @Test + // public void test4C2CGSubscribeTwoTopicRealtimeDataWithAsyncConnector() throws Exception { + // testSubscriptionRealtimeDataTemplate( + // ASYNC_CONNECTOR_ATTRIBUTES, + // __4C_2CG_SUBSCRIBE_TWO_TOPIC.left, + // __4C_2CG_SUBSCRIBE_TWO_TOPIC.right); + // } + // + // @Test + // public void test4C2CGSubscribeTwoTopicRealtimeDataWithSyncConnector() throws Exception { + // testSubscriptionRealtimeDataTemplate( + // SYNC_CONNECTOR_ATTRIBUTES, + // __4C_2CG_SUBSCRIBE_TWO_TOPIC.left, + // __4C_2CG_SUBSCRIBE_TWO_TOPIC.right); + // } + // + // @Test + // public void test4C2CGSubscribeTwoTopicRealtimeDataWithLegacyConnector() throws Exception { + // testSubscriptionRealtimeDataTemplate( + // LEGACY_CONNECTOR_ATTRIBUTES, + // __4C_2CG_SUBSCRIBE_TWO_TOPIC.left, + // __4C_2CG_SUBSCRIBE_TWO_TOPIC.right); + // } + // + // @Test + // public void test4C2CGSubscribeTwoTopicRealtimeDataWithAirGapConnector() throws Exception { + // testSubscriptionRealtimeDataTemplate( + // AIR_GAP_CONNECTOR_ATTRIBUTES, + // __4C_2CG_SUBSCRIBE_TWO_TOPIC.left, + // __4C_2CG_SUBSCRIBE_TWO_TOPIC.right); + // } // ------------------------------------------------------ // // 4 consumers, 2 consumer groups, 2 topics (with tsfile) // diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java index 92d44ccf7242..5efd48b222fc 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java @@ -109,11 +109,10 @@ public String toString() { public Path getTsFileDir(String topicName) throws IOException { if (Objects.isNull(subscribedTsFileBaseDirPath)) { - subscribedTsFileBaseDirPath = - Files.createTempDirectory( - String.format("subscribedTsFile_%s_%s#", consumerId, consumerGroupId)); + subscribedTsFileBaseDirPath = Files.createTempDirectory("subscribedTsFile#"); } - final Path dirPath = subscribedTsFileBaseDirPath.resolve(topicName); + final Path dirPath = + subscribedTsFileBaseDirPath.resolve(consumerGroupId).resolve(consumerId).resolve(topicName); Files.createDirectories(dirPath); return dirPath; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java index 191fe484c811..470d98f1b369 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java @@ -63,7 +63,7 @@ public SubscriptionPrefetchingTsFileQueue( } @Override - public SubscriptionEvent poll(final SubscriptionPollTimer timer) { + public synchronized SubscriptionEvent poll(final SubscriptionPollTimer timer) { if (Objects.nonNull(eventRef.get())) { return null; } From b377e7c82fbadf6217c9e7cc8075a0a49ad2f1a0 Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Wed, 17 Apr 2024 15:04:32 +0800 Subject: [PATCH 24/93] fix --- .../IoTDBSubscriptionConsumerGroupIT.java | 678 +++++++++--------- 1 file changed, 339 insertions(+), 339 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionConsumerGroupIT.java b/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionConsumerGroupIT.java index 5cc5c7febc85..9973e3211c83 100644 --- a/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionConsumerGroupIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionConsumerGroupIT.java @@ -309,345 +309,345 @@ private void testSubscriptionRealtimeDataTemplate( expectedHeaderWithResult); } - // // -------------------------------------- // - // // 3 consumers, 1 consumer group, 1 topic // - // // -------------------------------------- // - // - // @Test - // public void test3C1CGSubscribeOneTopicHistoricalDataWithAsyncConnector() throws Exception { - // testSubscriptionHistoricalDataTemplate( - // ASYNC_CONNECTOR_ATTRIBUTES, - // __3C_1CG_SUBSCRIBE_ONE_TOPIC.left, - // __3C_1CG_SUBSCRIBE_ONE_TOPIC.right); - // } - // - // @Test - // public void test3C1CGSubscribeOneTopicHistoricalDataWithSyncConnector() throws Exception { - // testSubscriptionHistoricalDataTemplate( - // SYNC_CONNECTOR_ATTRIBUTES, - // __3C_1CG_SUBSCRIBE_ONE_TOPIC.left, - // __3C_1CG_SUBSCRIBE_ONE_TOPIC.right); - // } - // - // @Test - // public void test3C1CGSubscribeOneTopicHistoricalDataWithLegacyConnector() throws Exception { - // testSubscriptionHistoricalDataTemplate( - // LEGACY_CONNECTOR_ATTRIBUTES, - // __3C_1CG_SUBSCRIBE_ONE_TOPIC.left, - // __3C_1CG_SUBSCRIBE_ONE_TOPIC.right); - // } - // - // @Test - // public void test3C1CGSubscribeOneTopicHistoricalDataWithAirGapConnector() throws Exception { - // testSubscriptionHistoricalDataTemplate( - // AIR_GAP_CONNECTOR_ATTRIBUTES, - // __3C_1CG_SUBSCRIBE_ONE_TOPIC.left, - // __3C_1CG_SUBSCRIBE_ONE_TOPIC.right); - // } - // - // @Test - // public void test3C1CGSubscribeOneTopicRealtimeDataWithAsyncConnector() throws Exception { - // testSubscriptionRealtimeDataTemplate( - // ASYNC_CONNECTOR_ATTRIBUTES, - // __3C_1CG_SUBSCRIBE_ONE_TOPIC.left, - // __3C_1CG_SUBSCRIBE_ONE_TOPIC.right); - // } - // - // @Test - // public void test3C1CGSubscribeOneTopicRealtimeDataWithSyncConnector() throws Exception { - // testSubscriptionRealtimeDataTemplate( - // SYNC_CONNECTOR_ATTRIBUTES, - // __3C_1CG_SUBSCRIBE_ONE_TOPIC.left, - // __3C_1CG_SUBSCRIBE_ONE_TOPIC.right); - // } - // - // @Test - // public void test3C1CGSubscribeOneTopicRealtimeDataWithLegacyConnector() throws Exception { - // testSubscriptionRealtimeDataTemplate( - // LEGACY_CONNECTOR_ATTRIBUTES, - // __3C_1CG_SUBSCRIBE_ONE_TOPIC.left, - // __3C_1CG_SUBSCRIBE_ONE_TOPIC.right); - // } - // - // @Test - // public void test3C1CGSubscribeOneTopicRealtimeDataWithAirGapConnector() throws Exception { - // testSubscriptionRealtimeDataTemplate( - // AIR_GAP_CONNECTOR_ATTRIBUTES, - // __3C_1CG_SUBSCRIBE_ONE_TOPIC.left, - // __3C_1CG_SUBSCRIBE_ONE_TOPIC.right); - // } - // - // // --------------------------------------- // - // // 3 consumers, 3 consumer groups, 1 topic // - // // --------------------------------------- // - // - // @Test - // public void test3C3CGSubscribeOneTopicHistoricalDataWithAsyncConnector() throws Exception { - // testSubscriptionHistoricalDataTemplate( - // ASYNC_CONNECTOR_ATTRIBUTES, - // __3C_3CG_SUBSCRIBE_ONE_TOPIC.left, - // __3C_3CG_SUBSCRIBE_ONE_TOPIC.right); - // } - // - // @Test - // public void test3C3CGSubscribeOneTopicHistoricalDataWithSyncConnector() throws Exception { - // testSubscriptionHistoricalDataTemplate( - // SYNC_CONNECTOR_ATTRIBUTES, - // __3C_3CG_SUBSCRIBE_ONE_TOPIC.left, - // __3C_3CG_SUBSCRIBE_ONE_TOPIC.right); - // } - // - // @Test - // public void test3C3CGSubscribeOneTopicHistoricalDataWithLegacyConnector() throws Exception { - // testSubscriptionHistoricalDataTemplate( - // LEGACY_CONNECTOR_ATTRIBUTES, - // __3C_3CG_SUBSCRIBE_ONE_TOPIC.left, - // __3C_3CG_SUBSCRIBE_ONE_TOPIC.right); - // } - // - // @Test - // public void test3C3CGSubscribeOneTopicHistoricalDataWithAirGapConnector() throws Exception { - // testSubscriptionHistoricalDataTemplate( - // AIR_GAP_CONNECTOR_ATTRIBUTES, - // __3C_3CG_SUBSCRIBE_ONE_TOPIC.left, - // __3C_3CG_SUBSCRIBE_ONE_TOPIC.right); - // } - // - // @Test - // public void test3C3CGSubscribeOneTopicRealtimeDataWithAsyncConnector() throws Exception { - // testSubscriptionRealtimeDataTemplate( - // ASYNC_CONNECTOR_ATTRIBUTES, - // __3C_3CG_SUBSCRIBE_ONE_TOPIC.left, - // __3C_3CG_SUBSCRIBE_ONE_TOPIC.right); - // } - // - // @Test - // public void test3C3CGSubscribeOneTopicRealtimeDataWithSyncConnector() throws Exception { - // testSubscriptionRealtimeDataTemplate( - // SYNC_CONNECTOR_ATTRIBUTES, - // __3C_3CG_SUBSCRIBE_ONE_TOPIC.left, - // __3C_3CG_SUBSCRIBE_ONE_TOPIC.right); - // } - // - // @Test - // public void test3C3CGSubscribeOneTopicRealtimeDataWithLegacyConnector() throws Exception { - // testSubscriptionRealtimeDataTemplate( - // LEGACY_CONNECTOR_ATTRIBUTES, - // __3C_3CG_SUBSCRIBE_ONE_TOPIC.left, - // __3C_3CG_SUBSCRIBE_ONE_TOPIC.right); - // } - // - // @Test - // public void test3C3CGSubscribeOneTopicRealtimeDataWithAirGapConnector() throws Exception { - // testSubscriptionRealtimeDataTemplate( - // AIR_GAP_CONNECTOR_ATTRIBUTES, - // __3C_3CG_SUBSCRIBE_ONE_TOPIC.left, - // __3C_3CG_SUBSCRIBE_ONE_TOPIC.right); - // } - // - // // --------------------------------------- // - // // 3 consumers, 1 consumer group, 2 topics // - // // --------------------------------------- // - // - // @Test - // public void test3C1CGSubscribeTwoTopicHistoricalDataWithAsyncConnector() throws Exception { - // testSubscriptionHistoricalDataTemplate( - // ASYNC_CONNECTOR_ATTRIBUTES, - // __3C_1CG_SUBSCRIBE_TWO_TOPIC.left, - // __3C_1CG_SUBSCRIBE_TWO_TOPIC.right); - // } - // - // @Test - // public void test3C1CGSubscribeTwoTopicHistoricalDataWithSyncConnector() throws Exception { - // testSubscriptionHistoricalDataTemplate( - // SYNC_CONNECTOR_ATTRIBUTES, - // __3C_1CG_SUBSCRIBE_TWO_TOPIC.left, - // __3C_1CG_SUBSCRIBE_TWO_TOPIC.right); - // } - // - // @Test - // public void test3C1CGSubscribeTwoTopicHistoricalDataWithLegacyConnector() throws Exception { - // testSubscriptionHistoricalDataTemplate( - // LEGACY_CONNECTOR_ATTRIBUTES, - // __3C_1CG_SUBSCRIBE_TWO_TOPIC.left, - // __3C_1CG_SUBSCRIBE_TWO_TOPIC.right); - // } - // - // @Test - // public void test3C1CGSubscribeTwoTopicHistoricalDataWithAirGapConnector() throws Exception { - // testSubscriptionHistoricalDataTemplate( - // AIR_GAP_CONNECTOR_ATTRIBUTES, - // __3C_1CG_SUBSCRIBE_TWO_TOPIC.left, - // __3C_1CG_SUBSCRIBE_TWO_TOPIC.right); - // } - // - // @Test - // public void test3C1CGSubscribeTwoTopicRealtimeDataWithAsyncConnector() throws Exception { - // testSubscriptionRealtimeDataTemplate( - // ASYNC_CONNECTOR_ATTRIBUTES, - // __3C_1CG_SUBSCRIBE_TWO_TOPIC.left, - // __3C_1CG_SUBSCRIBE_TWO_TOPIC.right); - // } - // - // @Test - // public void test3C1CGSubscribeTwoTopicRealtimeDataWithSyncConnector() throws Exception { - // testSubscriptionRealtimeDataTemplate( - // SYNC_CONNECTOR_ATTRIBUTES, - // __3C_1CG_SUBSCRIBE_TWO_TOPIC.left, - // __3C_1CG_SUBSCRIBE_TWO_TOPIC.right); - // } - // - // @Test - // public void test3C1CGSubscribeTwoTopicRealtimeDataWithLegacyConnector() throws Exception { - // testSubscriptionRealtimeDataTemplate( - // LEGACY_CONNECTOR_ATTRIBUTES, - // __3C_1CG_SUBSCRIBE_TWO_TOPIC.left, - // __3C_1CG_SUBSCRIBE_TWO_TOPIC.right); - // } - // - // @Test - // public void test3C1CGSubscribeTwoTopicRealtimeDataWithAirGapConnector() throws Exception { - // testSubscriptionRealtimeDataTemplate( - // AIR_GAP_CONNECTOR_ATTRIBUTES, - // __3C_1CG_SUBSCRIBE_TWO_TOPIC.left, - // __3C_1CG_SUBSCRIBE_TWO_TOPIC.right); - // } - // - // // ---------------------------------------- // - // // 3 consumers, 3 consumer groups, 2 topics // - // // ---------------------------------------- // - // - // @Test - // public void test3C3CGSubscribeTwoTopicHistoricalDataWithAsyncConnector() throws Exception { - // testSubscriptionHistoricalDataTemplate( - // ASYNC_CONNECTOR_ATTRIBUTES, - // __3C_3CG_SUBSCRIBE_TWO_TOPIC.left, - // __3C_3CG_SUBSCRIBE_TWO_TOPIC.right); - // } - // - // @Test - // public void test3C3CGSubscribeTwoTopicHistoricalDataWithSyncConnector() throws Exception { - // testSubscriptionHistoricalDataTemplate( - // SYNC_CONNECTOR_ATTRIBUTES, - // __3C_3CG_SUBSCRIBE_TWO_TOPIC.left, - // __3C_3CG_SUBSCRIBE_TWO_TOPIC.right); - // } - // - // @Test - // public void test3C3CGSubscribeTwoTopicHistoricalDataWithLegacyConnector() throws Exception { - // testSubscriptionHistoricalDataTemplate( - // LEGACY_CONNECTOR_ATTRIBUTES, - // __3C_3CG_SUBSCRIBE_TWO_TOPIC.left, - // __3C_3CG_SUBSCRIBE_TWO_TOPIC.right); - // } - // - // @Test - // public void test3C3CGSubscribeTwoTopicHistoricalDataWithAirGapConnector() throws Exception { - // testSubscriptionHistoricalDataTemplate( - // AIR_GAP_CONNECTOR_ATTRIBUTES, - // __3C_3CG_SUBSCRIBE_TWO_TOPIC.left, - // __3C_3CG_SUBSCRIBE_TWO_TOPIC.right); - // } - // - // @Test - // public void test3C3CGSubscribeTwoTopicRealtimeDataWithAsyncConnector() throws Exception { - // testSubscriptionRealtimeDataTemplate( - // ASYNC_CONNECTOR_ATTRIBUTES, - // __3C_3CG_SUBSCRIBE_TWO_TOPIC.left, - // __3C_3CG_SUBSCRIBE_TWO_TOPIC.right); - // } - // - // @Test - // public void test3C3CGSubscribeTwoTopicRealtimeDataWithSyncConnector() throws Exception { - // testSubscriptionRealtimeDataTemplate( - // SYNC_CONNECTOR_ATTRIBUTES, - // __3C_3CG_SUBSCRIBE_TWO_TOPIC.left, - // __3C_3CG_SUBSCRIBE_TWO_TOPIC.right); - // } - // - // @Test - // public void test3C3CGSubscribeTwoTopicRealtimeDataWithLegacyConnector() throws Exception { - // testSubscriptionRealtimeDataTemplate( - // LEGACY_CONNECTOR_ATTRIBUTES, - // __3C_3CG_SUBSCRIBE_TWO_TOPIC.left, - // __3C_3CG_SUBSCRIBE_TWO_TOPIC.right); - // } - // - // @Test - // public void test3C3CGSubscribeTwoTopicRealtimeDataWithAirGapConnector() throws Exception { - // testSubscriptionRealtimeDataTemplate( - // AIR_GAP_CONNECTOR_ATTRIBUTES, - // __3C_3CG_SUBSCRIBE_TWO_TOPIC.left, - // __3C_3CG_SUBSCRIBE_TWO_TOPIC.right); - // } - // - // // ---------------------------------------- // - // // 4 consumers, 2 consumer groups, 2 topics // - // // ---------------------------------------- // - // - // @Test - // public void test4C2CGSubscribeTwoTopicHistoricalDataWithAsyncConnector() throws Exception { - // testSubscriptionHistoricalDataTemplate( - // ASYNC_CONNECTOR_ATTRIBUTES, - // __4C_2CG_SUBSCRIBE_TWO_TOPIC.left, - // __4C_2CG_SUBSCRIBE_TWO_TOPIC.right); - // } - // - // @Test - // public void test4C2CGSubscribeTwoTopicHistoricalDataWithSyncConnector() throws Exception { - // testSubscriptionHistoricalDataTemplate( - // SYNC_CONNECTOR_ATTRIBUTES, - // __4C_2CG_SUBSCRIBE_TWO_TOPIC.left, - // __4C_2CG_SUBSCRIBE_TWO_TOPIC.right); - // } - // - // @Test - // public void test4C2CGSubscribeTwoTopicHistoricalDataWithLegacyConnector() throws Exception { - // testSubscriptionHistoricalDataTemplate( - // LEGACY_CONNECTOR_ATTRIBUTES, - // __4C_2CG_SUBSCRIBE_TWO_TOPIC.left, - // __4C_2CG_SUBSCRIBE_TWO_TOPIC.right); - // } - // - // @Test - // public void test4C2CGSubscribeTwoTopicHistoricalDataWithAirGapConnector() throws Exception { - // testSubscriptionHistoricalDataTemplate( - // AIR_GAP_CONNECTOR_ATTRIBUTES, - // __4C_2CG_SUBSCRIBE_TWO_TOPIC.left, - // __4C_2CG_SUBSCRIBE_TWO_TOPIC.right); - // } - // - // @Test - // public void test4C2CGSubscribeTwoTopicRealtimeDataWithAsyncConnector() throws Exception { - // testSubscriptionRealtimeDataTemplate( - // ASYNC_CONNECTOR_ATTRIBUTES, - // __4C_2CG_SUBSCRIBE_TWO_TOPIC.left, - // __4C_2CG_SUBSCRIBE_TWO_TOPIC.right); - // } - // - // @Test - // public void test4C2CGSubscribeTwoTopicRealtimeDataWithSyncConnector() throws Exception { - // testSubscriptionRealtimeDataTemplate( - // SYNC_CONNECTOR_ATTRIBUTES, - // __4C_2CG_SUBSCRIBE_TWO_TOPIC.left, - // __4C_2CG_SUBSCRIBE_TWO_TOPIC.right); - // } - // - // @Test - // public void test4C2CGSubscribeTwoTopicRealtimeDataWithLegacyConnector() throws Exception { - // testSubscriptionRealtimeDataTemplate( - // LEGACY_CONNECTOR_ATTRIBUTES, - // __4C_2CG_SUBSCRIBE_TWO_TOPIC.left, - // __4C_2CG_SUBSCRIBE_TWO_TOPIC.right); - // } - // - // @Test - // public void test4C2CGSubscribeTwoTopicRealtimeDataWithAirGapConnector() throws Exception { - // testSubscriptionRealtimeDataTemplate( - // AIR_GAP_CONNECTOR_ATTRIBUTES, - // __4C_2CG_SUBSCRIBE_TWO_TOPIC.left, - // __4C_2CG_SUBSCRIBE_TWO_TOPIC.right); - // } + // -------------------------------------- // + // 3 consumers, 1 consumer group, 1 topic // + // -------------------------------------- // + + @Test + public void test3C1CGSubscribeOneTopicHistoricalDataWithAsyncConnector() throws Exception { + testSubscriptionHistoricalDataTemplate( + ASYNC_CONNECTOR_ATTRIBUTES, + __3C_1CG_SUBSCRIBE_ONE_TOPIC.left, + __3C_1CG_SUBSCRIBE_ONE_TOPIC.right); + } + + @Test + public void test3C1CGSubscribeOneTopicHistoricalDataWithSyncConnector() throws Exception { + testSubscriptionHistoricalDataTemplate( + SYNC_CONNECTOR_ATTRIBUTES, + __3C_1CG_SUBSCRIBE_ONE_TOPIC.left, + __3C_1CG_SUBSCRIBE_ONE_TOPIC.right); + } + + @Test + public void test3C1CGSubscribeOneTopicHistoricalDataWithLegacyConnector() throws Exception { + testSubscriptionHistoricalDataTemplate( + LEGACY_CONNECTOR_ATTRIBUTES, + __3C_1CG_SUBSCRIBE_ONE_TOPIC.left, + __3C_1CG_SUBSCRIBE_ONE_TOPIC.right); + } + + @Test + public void test3C1CGSubscribeOneTopicHistoricalDataWithAirGapConnector() throws Exception { + testSubscriptionHistoricalDataTemplate( + AIR_GAP_CONNECTOR_ATTRIBUTES, + __3C_1CG_SUBSCRIBE_ONE_TOPIC.left, + __3C_1CG_SUBSCRIBE_ONE_TOPIC.right); + } + + @Test + public void test3C1CGSubscribeOneTopicRealtimeDataWithAsyncConnector() throws Exception { + testSubscriptionRealtimeDataTemplate( + ASYNC_CONNECTOR_ATTRIBUTES, + __3C_1CG_SUBSCRIBE_ONE_TOPIC.left, + __3C_1CG_SUBSCRIBE_ONE_TOPIC.right); + } + + @Test + public void test3C1CGSubscribeOneTopicRealtimeDataWithSyncConnector() throws Exception { + testSubscriptionRealtimeDataTemplate( + SYNC_CONNECTOR_ATTRIBUTES, + __3C_1CG_SUBSCRIBE_ONE_TOPIC.left, + __3C_1CG_SUBSCRIBE_ONE_TOPIC.right); + } + + @Test + public void test3C1CGSubscribeOneTopicRealtimeDataWithLegacyConnector() throws Exception { + testSubscriptionRealtimeDataTemplate( + LEGACY_CONNECTOR_ATTRIBUTES, + __3C_1CG_SUBSCRIBE_ONE_TOPIC.left, + __3C_1CG_SUBSCRIBE_ONE_TOPIC.right); + } + + @Test + public void test3C1CGSubscribeOneTopicRealtimeDataWithAirGapConnector() throws Exception { + testSubscriptionRealtimeDataTemplate( + AIR_GAP_CONNECTOR_ATTRIBUTES, + __3C_1CG_SUBSCRIBE_ONE_TOPIC.left, + __3C_1CG_SUBSCRIBE_ONE_TOPIC.right); + } + + // --------------------------------------- // + // 3 consumers, 3 consumer groups, 1 topic // + // --------------------------------------- // + + @Test + public void test3C3CGSubscribeOneTopicHistoricalDataWithAsyncConnector() throws Exception { + testSubscriptionHistoricalDataTemplate( + ASYNC_CONNECTOR_ATTRIBUTES, + __3C_3CG_SUBSCRIBE_ONE_TOPIC.left, + __3C_3CG_SUBSCRIBE_ONE_TOPIC.right); + } + + @Test + public void test3C3CGSubscribeOneTopicHistoricalDataWithSyncConnector() throws Exception { + testSubscriptionHistoricalDataTemplate( + SYNC_CONNECTOR_ATTRIBUTES, + __3C_3CG_SUBSCRIBE_ONE_TOPIC.left, + __3C_3CG_SUBSCRIBE_ONE_TOPIC.right); + } + + @Test + public void test3C3CGSubscribeOneTopicHistoricalDataWithLegacyConnector() throws Exception { + testSubscriptionHistoricalDataTemplate( + LEGACY_CONNECTOR_ATTRIBUTES, + __3C_3CG_SUBSCRIBE_ONE_TOPIC.left, + __3C_3CG_SUBSCRIBE_ONE_TOPIC.right); + } + + @Test + public void test3C3CGSubscribeOneTopicHistoricalDataWithAirGapConnector() throws Exception { + testSubscriptionHistoricalDataTemplate( + AIR_GAP_CONNECTOR_ATTRIBUTES, + __3C_3CG_SUBSCRIBE_ONE_TOPIC.left, + __3C_3CG_SUBSCRIBE_ONE_TOPIC.right); + } + + @Test + public void test3C3CGSubscribeOneTopicRealtimeDataWithAsyncConnector() throws Exception { + testSubscriptionRealtimeDataTemplate( + ASYNC_CONNECTOR_ATTRIBUTES, + __3C_3CG_SUBSCRIBE_ONE_TOPIC.left, + __3C_3CG_SUBSCRIBE_ONE_TOPIC.right); + } + + @Test + public void test3C3CGSubscribeOneTopicRealtimeDataWithSyncConnector() throws Exception { + testSubscriptionRealtimeDataTemplate( + SYNC_CONNECTOR_ATTRIBUTES, + __3C_3CG_SUBSCRIBE_ONE_TOPIC.left, + __3C_3CG_SUBSCRIBE_ONE_TOPIC.right); + } + + @Test + public void test3C3CGSubscribeOneTopicRealtimeDataWithLegacyConnector() throws Exception { + testSubscriptionRealtimeDataTemplate( + LEGACY_CONNECTOR_ATTRIBUTES, + __3C_3CG_SUBSCRIBE_ONE_TOPIC.left, + __3C_3CG_SUBSCRIBE_ONE_TOPIC.right); + } + + @Test + public void test3C3CGSubscribeOneTopicRealtimeDataWithAirGapConnector() throws Exception { + testSubscriptionRealtimeDataTemplate( + AIR_GAP_CONNECTOR_ATTRIBUTES, + __3C_3CG_SUBSCRIBE_ONE_TOPIC.left, + __3C_3CG_SUBSCRIBE_ONE_TOPIC.right); + } + + // --------------------------------------- // + // 3 consumers, 1 consumer group, 2 topics // + // --------------------------------------- // + + @Test + public void test3C1CGSubscribeTwoTopicHistoricalDataWithAsyncConnector() throws Exception { + testSubscriptionHistoricalDataTemplate( + ASYNC_CONNECTOR_ATTRIBUTES, + __3C_1CG_SUBSCRIBE_TWO_TOPIC.left, + __3C_1CG_SUBSCRIBE_TWO_TOPIC.right); + } + + @Test + public void test3C1CGSubscribeTwoTopicHistoricalDataWithSyncConnector() throws Exception { + testSubscriptionHistoricalDataTemplate( + SYNC_CONNECTOR_ATTRIBUTES, + __3C_1CG_SUBSCRIBE_TWO_TOPIC.left, + __3C_1CG_SUBSCRIBE_TWO_TOPIC.right); + } + + @Test + public void test3C1CGSubscribeTwoTopicHistoricalDataWithLegacyConnector() throws Exception { + testSubscriptionHistoricalDataTemplate( + LEGACY_CONNECTOR_ATTRIBUTES, + __3C_1CG_SUBSCRIBE_TWO_TOPIC.left, + __3C_1CG_SUBSCRIBE_TWO_TOPIC.right); + } + + @Test + public void test3C1CGSubscribeTwoTopicHistoricalDataWithAirGapConnector() throws Exception { + testSubscriptionHistoricalDataTemplate( + AIR_GAP_CONNECTOR_ATTRIBUTES, + __3C_1CG_SUBSCRIBE_TWO_TOPIC.left, + __3C_1CG_SUBSCRIBE_TWO_TOPIC.right); + } + + @Test + public void test3C1CGSubscribeTwoTopicRealtimeDataWithAsyncConnector() throws Exception { + testSubscriptionRealtimeDataTemplate( + ASYNC_CONNECTOR_ATTRIBUTES, + __3C_1CG_SUBSCRIBE_TWO_TOPIC.left, + __3C_1CG_SUBSCRIBE_TWO_TOPIC.right); + } + + @Test + public void test3C1CGSubscribeTwoTopicRealtimeDataWithSyncConnector() throws Exception { + testSubscriptionRealtimeDataTemplate( + SYNC_CONNECTOR_ATTRIBUTES, + __3C_1CG_SUBSCRIBE_TWO_TOPIC.left, + __3C_1CG_SUBSCRIBE_TWO_TOPIC.right); + } + + @Test + public void test3C1CGSubscribeTwoTopicRealtimeDataWithLegacyConnector() throws Exception { + testSubscriptionRealtimeDataTemplate( + LEGACY_CONNECTOR_ATTRIBUTES, + __3C_1CG_SUBSCRIBE_TWO_TOPIC.left, + __3C_1CG_SUBSCRIBE_TWO_TOPIC.right); + } + + @Test + public void test3C1CGSubscribeTwoTopicRealtimeDataWithAirGapConnector() throws Exception { + testSubscriptionRealtimeDataTemplate( + AIR_GAP_CONNECTOR_ATTRIBUTES, + __3C_1CG_SUBSCRIBE_TWO_TOPIC.left, + __3C_1CG_SUBSCRIBE_TWO_TOPIC.right); + } + + // ---------------------------------------- // + // 3 consumers, 3 consumer groups, 2 topics // + // ---------------------------------------- // + + @Test + public void test3C3CGSubscribeTwoTopicHistoricalDataWithAsyncConnector() throws Exception { + testSubscriptionHistoricalDataTemplate( + ASYNC_CONNECTOR_ATTRIBUTES, + __3C_3CG_SUBSCRIBE_TWO_TOPIC.left, + __3C_3CG_SUBSCRIBE_TWO_TOPIC.right); + } + + @Test + public void test3C3CGSubscribeTwoTopicHistoricalDataWithSyncConnector() throws Exception { + testSubscriptionHistoricalDataTemplate( + SYNC_CONNECTOR_ATTRIBUTES, + __3C_3CG_SUBSCRIBE_TWO_TOPIC.left, + __3C_3CG_SUBSCRIBE_TWO_TOPIC.right); + } + + @Test + public void test3C3CGSubscribeTwoTopicHistoricalDataWithLegacyConnector() throws Exception { + testSubscriptionHistoricalDataTemplate( + LEGACY_CONNECTOR_ATTRIBUTES, + __3C_3CG_SUBSCRIBE_TWO_TOPIC.left, + __3C_3CG_SUBSCRIBE_TWO_TOPIC.right); + } + + @Test + public void test3C3CGSubscribeTwoTopicHistoricalDataWithAirGapConnector() throws Exception { + testSubscriptionHistoricalDataTemplate( + AIR_GAP_CONNECTOR_ATTRIBUTES, + __3C_3CG_SUBSCRIBE_TWO_TOPIC.left, + __3C_3CG_SUBSCRIBE_TWO_TOPIC.right); + } + + @Test + public void test3C3CGSubscribeTwoTopicRealtimeDataWithAsyncConnector() throws Exception { + testSubscriptionRealtimeDataTemplate( + ASYNC_CONNECTOR_ATTRIBUTES, + __3C_3CG_SUBSCRIBE_TWO_TOPIC.left, + __3C_3CG_SUBSCRIBE_TWO_TOPIC.right); + } + + @Test + public void test3C3CGSubscribeTwoTopicRealtimeDataWithSyncConnector() throws Exception { + testSubscriptionRealtimeDataTemplate( + SYNC_CONNECTOR_ATTRIBUTES, + __3C_3CG_SUBSCRIBE_TWO_TOPIC.left, + __3C_3CG_SUBSCRIBE_TWO_TOPIC.right); + } + + @Test + public void test3C3CGSubscribeTwoTopicRealtimeDataWithLegacyConnector() throws Exception { + testSubscriptionRealtimeDataTemplate( + LEGACY_CONNECTOR_ATTRIBUTES, + __3C_3CG_SUBSCRIBE_TWO_TOPIC.left, + __3C_3CG_SUBSCRIBE_TWO_TOPIC.right); + } + + @Test + public void test3C3CGSubscribeTwoTopicRealtimeDataWithAirGapConnector() throws Exception { + testSubscriptionRealtimeDataTemplate( + AIR_GAP_CONNECTOR_ATTRIBUTES, + __3C_3CG_SUBSCRIBE_TWO_TOPIC.left, + __3C_3CG_SUBSCRIBE_TWO_TOPIC.right); + } + + // ---------------------------------------- // + // 4 consumers, 2 consumer groups, 2 topics // + // ---------------------------------------- // + + @Test + public void test4C2CGSubscribeTwoTopicHistoricalDataWithAsyncConnector() throws Exception { + testSubscriptionHistoricalDataTemplate( + ASYNC_CONNECTOR_ATTRIBUTES, + __4C_2CG_SUBSCRIBE_TWO_TOPIC.left, + __4C_2CG_SUBSCRIBE_TWO_TOPIC.right); + } + + @Test + public void test4C2CGSubscribeTwoTopicHistoricalDataWithSyncConnector() throws Exception { + testSubscriptionHistoricalDataTemplate( + SYNC_CONNECTOR_ATTRIBUTES, + __4C_2CG_SUBSCRIBE_TWO_TOPIC.left, + __4C_2CG_SUBSCRIBE_TWO_TOPIC.right); + } + + @Test + public void test4C2CGSubscribeTwoTopicHistoricalDataWithLegacyConnector() throws Exception { + testSubscriptionHistoricalDataTemplate( + LEGACY_CONNECTOR_ATTRIBUTES, + __4C_2CG_SUBSCRIBE_TWO_TOPIC.left, + __4C_2CG_SUBSCRIBE_TWO_TOPIC.right); + } + + @Test + public void test4C2CGSubscribeTwoTopicHistoricalDataWithAirGapConnector() throws Exception { + testSubscriptionHistoricalDataTemplate( + AIR_GAP_CONNECTOR_ATTRIBUTES, + __4C_2CG_SUBSCRIBE_TWO_TOPIC.left, + __4C_2CG_SUBSCRIBE_TWO_TOPIC.right); + } + + @Test + public void test4C2CGSubscribeTwoTopicRealtimeDataWithAsyncConnector() throws Exception { + testSubscriptionRealtimeDataTemplate( + ASYNC_CONNECTOR_ATTRIBUTES, + __4C_2CG_SUBSCRIBE_TWO_TOPIC.left, + __4C_2CG_SUBSCRIBE_TWO_TOPIC.right); + } + + @Test + public void test4C2CGSubscribeTwoTopicRealtimeDataWithSyncConnector() throws Exception { + testSubscriptionRealtimeDataTemplate( + SYNC_CONNECTOR_ATTRIBUTES, + __4C_2CG_SUBSCRIBE_TWO_TOPIC.left, + __4C_2CG_SUBSCRIBE_TWO_TOPIC.right); + } + + @Test + public void test4C2CGSubscribeTwoTopicRealtimeDataWithLegacyConnector() throws Exception { + testSubscriptionRealtimeDataTemplate( + LEGACY_CONNECTOR_ATTRIBUTES, + __4C_2CG_SUBSCRIBE_TWO_TOPIC.left, + __4C_2CG_SUBSCRIBE_TWO_TOPIC.right); + } + + @Test + public void test4C2CGSubscribeTwoTopicRealtimeDataWithAirGapConnector() throws Exception { + testSubscriptionRealtimeDataTemplate( + AIR_GAP_CONNECTOR_ATTRIBUTES, + __4C_2CG_SUBSCRIBE_TWO_TOPIC.left, + __4C_2CG_SUBSCRIBE_TWO_TOPIC.right); + } // ------------------------------------------------------ // // 4 consumers, 2 consumer groups, 2 topics (with tsfile) // From db157231ab368145ef23d001c8b4bf1541ebd3a4 Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Wed, 17 Apr 2024 18:14:04 +0800 Subject: [PATCH 25/93] backup --- .../common/SubscriptionPollMessage.java | 13 ++ .../common/SubscriptionPolledMessage.java | 20 ++- .../common/SubscriptionPolledMessageType.java | 3 +- .../common/TsFileErrorMessagePayload.java | 89 ++++++++++ ...oad.java => TsFileInitMessagePayload.java} | 10 +- .../subscription/SubscriptionMessage.java | 9 + .../SubscriptionPullConsumer.java | 55 ++++-- .../agent/SubscriptionBrokerAgent.java | 6 +- .../broker/SubscriptionBroker.java | 12 +- .../broker/SubscriptionPrefetchingQueue.java | 21 ++- .../SubscriptionPrefetchingTabletsQueue.java | 3 +- .../SubscriptionPrefetchingTsFileQueue.java | 158 +++++++++++++----- .../subscription/event/SubscriptionEvent.java | 27 +++ 13 files changed, 342 insertions(+), 84 deletions(-) create mode 100644 iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileErrorMessagePayload.java rename iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/{TsFileInfoMessagePayload.java => TsFileInitMessagePayload.java} (86%) diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPollMessage.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPollMessage.java index 39520277d8a5..ddc7ab93913f 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPollMessage.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPollMessage.java @@ -96,4 +96,17 @@ public static SubscriptionPollMessage deserialize(final ByteBuffer buffer) { final long timeoutMs = ReadWriteIOUtils.readLong(buffer); return new SubscriptionPollMessage(messageType, messagePayload, timeoutMs); } + + /////////////////////////////// object /////////////////////////////// + + @Override + public String toString() { + return "SubscriptionPollMessage{messageType=" + + SubscriptionPollMessageType.valueOf(messageType).toString() + + ", messagePayload=" + + messagePayload + + ", timeoutMs=" + + timeoutMs + + "}"; + } } diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledMessage.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledMessage.java index c66b23dec276..cadc15db5d66 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledMessage.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledMessage.java @@ -94,8 +94,8 @@ public static SubscriptionPolledMessage deserialize(final ByteBuffer buffer) { case TABLETS: messagePayload = new TabletsMessagePayload().deserialize(buffer); break; - case TS_FILE_INFO: - messagePayload = new TsFileInfoMessagePayload().deserialize(buffer); + case TS_FILE_INIT: + messagePayload = new TsFileInitMessagePayload().deserialize(buffer); break; case TS_FILE_PIECE: messagePayload = new TsFilePieceMessagePayload().deserialize(buffer); @@ -103,6 +103,9 @@ public static SubscriptionPolledMessage deserialize(final ByteBuffer buffer) { case TS_FILE_SEAL: messagePayload = new TsFileSealMessagePayload().deserialize(buffer); break; + case TS_FILE_ERROR: + messagePayload = new TsFileErrorMessagePayload().deserialize(buffer); + break; default: LOGGER.warn("unexpected message type: {}, message payload will be null", messageType); break; @@ -142,4 +145,17 @@ public void resetByteBuffer() { // maybe friendly for gc byteBuffer = null; } + + /////////////////////////////// object /////////////////////////////// + + @Override + public String toString() { + return "SubscriptionPolledMessage{messageType=" + + SubscriptionPolledMessageType.valueOf(messageType).toString() + + ", messagePayload=" + + messagePayload + + ", commitContext=" + + commitContext + + "}"; + } } diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledMessageType.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledMessageType.java index cc6a1026bec7..3abc2d4c1ae2 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledMessageType.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledMessageType.java @@ -26,9 +26,10 @@ public enum SubscriptionPolledMessageType { TABLETS((short) 0), - TS_FILE_INFO((short) 1), + TS_FILE_INIT((short) 1), TS_FILE_PIECE((short) 2), TS_FILE_SEAL((short) 3), + TS_FILE_ERROR((short) 4), ; private final short type; diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileErrorMessagePayload.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileErrorMessagePayload.java new file mode 100644 index 000000000000..851d20cf4d9c --- /dev/null +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileErrorMessagePayload.java @@ -0,0 +1,89 @@ +/* + * 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.rpc.subscription.payload.common; + +import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; + +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Objects; + +public class TsFileErrorMessagePayload implements SubscriptionMessagePayload { + + private transient String errorMessage; + + private transient boolean retryable; + + public String getErrorMessage() { + return errorMessage; + } + + public boolean isRetryable() { + return retryable; + } + + public TsFileErrorMessagePayload() {} + + public TsFileErrorMessagePayload(String errorMessage, boolean retryable) { + this.errorMessage = errorMessage; + this.retryable = retryable; + } + + @Override + public void serialize(DataOutputStream stream) throws IOException { + ReadWriteIOUtils.write(errorMessage, stream); + ReadWriteIOUtils.write(retryable, stream); + } + + @Override + public SubscriptionMessagePayload deserialize(ByteBuffer buffer) { + this.errorMessage = ReadWriteIOUtils.readString(buffer); + this.retryable = ReadWriteIOUtils.readBool(buffer); + return this; + } + + @Override + public boolean equals(final Object obj) { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + final TsFileErrorMessagePayload that = (TsFileErrorMessagePayload) obj; + return Objects.equals(this.errorMessage, that.errorMessage) + && Objects.equals(this.retryable, that.retryable); + } + + @Override + public int hashCode() { + return Objects.hash(errorMessage, retryable); + } + + @Override + public String toString() { + return "TsFileErrorMessagePayload{errorMessage=" + + errorMessage + + ", retryable=" + + retryable + + "}"; + } +} diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileInfoMessagePayload.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileInitMessagePayload.java similarity index 86% rename from iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileInfoMessagePayload.java rename to iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileInitMessagePayload.java index 4b32914a76a4..4cecfff1f975 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileInfoMessagePayload.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileInitMessagePayload.java @@ -26,7 +26,7 @@ import java.nio.ByteBuffer; import java.util.Objects; -public class TsFileInfoMessagePayload implements SubscriptionMessagePayload { +public class TsFileInitMessagePayload implements SubscriptionMessagePayload { private transient String fileName; @@ -34,9 +34,9 @@ public String getFileName() { return fileName; } - public TsFileInfoMessagePayload() {} + public TsFileInitMessagePayload() {} - public TsFileInfoMessagePayload(String fileName) { + public TsFileInitMessagePayload(String fileName) { this.fileName = fileName; } @@ -59,7 +59,7 @@ public boolean equals(final Object obj) { if (obj == null || getClass() != obj.getClass()) { return false; } - final TsFileInfoMessagePayload that = (TsFileInfoMessagePayload) obj; + final TsFileInitMessagePayload that = (TsFileInitMessagePayload) obj; return Objects.equals(this.fileName, that.fileName); } @@ -70,6 +70,6 @@ public int hashCode() { @Override public String toString() { - return "TsFileInfoMessagePayload{fileName=" + fileName + "}"; + return "TsFileInitMessagePayload{fileName=" + fileName + "}"; } } diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionMessage.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionMessage.java index a9ab97391430..76b0602b64ce 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionMessage.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionMessage.java @@ -82,4 +82,13 @@ public int hashCode() { public int compareTo(SubscriptionMessage that) { return this.commitContext.compareTo(that.commitContext); } + + @Override + public String toString() { + return "SubscriptionMessage{commitContext=" + + commitContext + + ", messageType=" + + SubscriptionMessageType.valueOf(messageType).toString() + + "}"; + } } diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java index 9173d1fff054..c701f7d424e5 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java @@ -26,12 +26,13 @@ import org.apache.iotdb.rpc.subscription.payload.common.PollMessagePayload; import org.apache.iotdb.rpc.subscription.payload.common.PollTsFileMessagePayload; import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionCommitContext; +import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionMessagePayload; import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPollMessage; import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPollMessageType; import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPolledMessage; import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPolledMessageType; import org.apache.iotdb.rpc.subscription.payload.common.TabletsMessagePayload; -import org.apache.iotdb.rpc.subscription.payload.common.TsFileInfoMessagePayload; +import org.apache.iotdb.rpc.subscription.payload.common.TsFileInitMessagePayload; import org.apache.iotdb.rpc.subscription.payload.common.TsFilePieceMessagePayload; import org.apache.iotdb.rpc.subscription.payload.common.TsFileSealMessagePayload; import org.apache.iotdb.tsfile.utils.Pair; @@ -196,12 +197,12 @@ public List poll(Set topicNames, long timeoutMs) polledMessage.getCommitContext(), ((TabletsMessagePayload) polledMessage.getMessagePayload()).getTablets())); break; - case TS_FILE_INFO: + case TS_FILE_INIT: try { final SubscriptionMessage message = pollTsFile( polledMessage.getCommitContext(), - ((TsFileInfoMessagePayload) polledMessage.getMessagePayload()).getFileName(), + ((TsFileInitMessagePayload) polledMessage.getMessagePayload()).getFileName(), timeoutMs); if (Objects.isNull(message)) { throw new Exception("poll empty tsfile, will retry later..."); @@ -252,47 +253,63 @@ private SubscriptionMessage pollTsFile( while (true) { final List polledMessages = pollTsFileInternal(dataNodeId, topicName, fileName, endWritingOffset, timeoutMs); + if (Objects.isNull(polledMessages) || polledMessages.size() != 1) { + LOGGER.warn("unexpected polledMessages: {}, consumer: {}", polledMessages, this); return null; } + final SubscriptionPolledMessage polledMessage = polledMessages.get(0); if (Objects.isNull(polledMessage)) { + LOGGER.warn("unexpected polledMessage: {}, consumer: {}", polledMessage, this); + return null; + } + + final SubscriptionMessagePayload messagePayload = polledMessage.getMessagePayload(); + if (Objects.isNull(messagePayload)) { + LOGGER.warn("unexpected messagePayload: {}, consumer: {}", messagePayload, this); return null; } + + final SubscriptionCommitContext incomingCommitContext = polledMessage.getCommitContext(); + if (Objects.isNull(incomingCommitContext) + || !Objects.equals(commitContext, incomingCommitContext)) { + LOGGER.warn( + "inconsistent commit context, current is {}, incoming is {}, consumer: {}", + messagePayload, + incomingCommitContext, + this); + return null; + } + final short messageType = polledMessage.getMessageType(); if (SubscriptionPolledMessageType.isValidatedMessageType(messageType)) { switch (SubscriptionPolledMessageType.valueOf(messageType)) { case TS_FILE_PIECE: { - final TsFilePieceMessagePayload messagePayload = - (TsFilePieceMessagePayload) polledMessage.getMessagePayload(); - if (Objects.isNull(messagePayload)) { - return null; - } // check file name - if (!fileName.equals(messagePayload.getFileName())) { + if (!fileName.equals(((TsFilePieceMessagePayload) messagePayload).getFileName())) { + return null; } // write file piece - fileWriter.write(messagePayload.getFilePiece()); + fileWriter.write(((TsFilePieceMessagePayload) messagePayload).getFilePiece()); fileWriter.getFD().sync(); // update offset - endWritingOffset = messagePayload.getEndWritingOffset(); + endWritingOffset = ((TsFilePieceMessagePayload) messagePayload).getEndWritingOffset(); break; } case TS_FILE_SEAL: { - final TsFileSealMessagePayload messagePayload = - (TsFileSealMessagePayload) polledMessage.getMessagePayload(); - if (Objects.isNull(messagePayload)) { - return null; - } // check file name - if (!fileName.equals(messagePayload.getFileName())) { + if (!fileName.equals(((TsFileSealMessagePayload) messagePayload).getFileName())) { + return null; } // check file length - if (fileWriter.length() != messagePayload.getFileLength()) { + if (fileWriter.length() + != ((TsFileSealMessagePayload) messagePayload).getFileLength()) { + return null; } // sync and close @@ -303,6 +320,8 @@ private SubscriptionMessage pollTsFile( // generate subscription message return new SubscriptionMessage(commitContext, file.getAbsolutePath()); } + case TS_FILE_ERROR: + return null; default: LOGGER.warn("unexpected message type: {}", messageType); return null; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionBrokerAgent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionBrokerAgent.java index 89b5107aefc9..8993b4414e3a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionBrokerAgent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionBrokerAgent.java @@ -57,7 +57,8 @@ public List poll( return Collections.emptyList(); } // TODO: currently we fetch messages from all topics - return broker.poll(topicNames, timer); + final String consumerId = consumerConfig.getConsumerId(); + return broker.poll(consumerId, topicNames, timer); } public List pollTsFile( @@ -69,7 +70,8 @@ public List pollTsFile( "Subscription: broker bound to consumer group [{}] does not exist", consumerGroupId); return Collections.emptyList(); } - return broker.pollTsFile(topicName, fileName, endWritingOffset); + final String consumerId = consumerConfig.getConsumerId(); + return broker.pollTsFile(consumerId, topicName, fileName, endWritingOffset); } public void commit( 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 08d989ec8200..10455e041b31 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 @@ -57,14 +57,14 @@ public boolean isEmpty() { //////////////////////////// provided for SubscriptionBrokerAgent //////////////////////////// public List poll( - final Set topicNames, final SubscriptionPollTimer timer) { + final String consumerId, final Set topicNames, final SubscriptionPollTimer timer) { final List events = new ArrayList<>(); for (final Map.Entry entry : topicNameToPrefetchingQueue.entrySet()) { final String topicName = entry.getKey(); final SubscriptionPrefetchingQueue prefetchingQueue = entry.getValue(); if (topicNames.contains(topicName)) { - final SubscriptionEvent event = prefetchingQueue.poll(timer); + final SubscriptionEvent event = prefetchingQueue.poll(consumerId, timer); if (Objects.nonNull(event)) { events.add(event); } @@ -78,7 +78,7 @@ public List poll( } public List pollTsFile( - String topicName, String fileName, long endWritingOffset) { + String consumerId, String topicName, String fileName, long endWritingOffset) { SubscriptionPrefetchingQueue prefetchingQueue = topicNameToPrefetchingQueue.get(topicName); if (Objects.isNull(prefetchingQueue)) { return null; @@ -89,10 +89,8 @@ public List pollTsFile( final List events = new ArrayList<>(); final SubscriptionEvent event = ((SubscriptionPrefetchingTsFileQueue) prefetchingQueue) - .pollTsFile(fileName, endWritingOffset); - if (Objects.nonNull(event)) { - events.add(event); - } + .pollTsFile(consumerId, fileName, endWritingOffset); + events.add(event); return events; } 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 dcb974d41681..b58280f6faa7 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 @@ -57,7 +57,7 @@ public SubscriptionPrefetchingQueue( this.uncommittedEvents = new ConcurrentHashMap<>(); } - public abstract SubscriptionEvent poll(SubscriptionPollTimer timer); + public abstract SubscriptionEvent poll(String consumerId, SubscriptionPollTimer timer); public abstract void executePrefetch(); @@ -73,9 +73,6 @@ public void commit(final SubscriptionCommitContext commitContext) { event.decreaseReferenceCount(); event.recordCommittedTimestamp(); uncommittedEvents.remove(commitContext); - if (this instanceof SubscriptionPrefetchingTsFileQueue) { - ((SubscriptionPrefetchingTsFileQueue) this).resetEventRef(); - } } protected SubscriptionCommitContext generateSubscriptionCommitContext() { @@ -88,4 +85,20 @@ protected SubscriptionCommitContext generateSubscriptionCommitContext() { brokerId, subscriptionCommitIdGenerator.getAndIncrement()); } + + protected SubscriptionCommitContext generateInvalidSubscriptionCommitContext() { + return new SubscriptionCommitContext( + IoTDBDescriptor.getInstance().getConfig().getDataNodeId(), + PipeAgent.runtime().getRebootTimes(), + topicName, + brokerId, + -1); + } + + /////////////////////////////// object /////////////////////////////// + + @Override + public String toString() { + return "SubscriptionPrefetchingQueue{brokerId=" + brokerId + ", topicName=" + topicName + "}"; + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTabletsQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTabletsQueue.java index b891d4c62211..e817c5bfc610 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTabletsQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTabletsQueue.java @@ -62,7 +62,7 @@ public SubscriptionPrefetchingTabletsQueue( } @Override - public SubscriptionEvent poll(final SubscriptionPollTimer timer) { + public SubscriptionEvent poll(final String consumerId, final SubscriptionPollTimer timer) { if (prefetchingQueue.isEmpty()) { prefetchOnce(SubscriptionConfig.getInstance().getSubscriptionMaxTabletsPerPrefetching()); // without serializeOnce here @@ -88,6 +88,7 @@ public SubscriptionEvent poll(final SubscriptionPollTimer timer) { if (!currentEvent.pollable()) { continue; } + currentEvent.recordLastPolledConsumerId(consumerId); currentEvent.recordLastPolledTimestamp(); return currentEvent; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java index 470d98f1b369..c216bbe42ffc 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java @@ -31,10 +31,12 @@ import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionMessagePayload; import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPolledMessage; import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPolledMessageType; -import org.apache.iotdb.rpc.subscription.payload.common.TsFileInfoMessagePayload; +import org.apache.iotdb.rpc.subscription.payload.common.TsFileErrorMessagePayload; +import org.apache.iotdb.rpc.subscription.payload.common.TsFileInitMessagePayload; import org.apache.iotdb.rpc.subscription.payload.common.TsFilePieceMessagePayload; import org.apache.iotdb.rpc.subscription.payload.common.TsFileSealMessagePayload; +import org.checkerframework.checker.nullness.qual.NonNull; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -43,15 +45,16 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.Objects; -import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.ConcurrentHashMap; public class SubscriptionPrefetchingTsFileQueue extends SubscriptionPrefetchingQueue { private static final Logger LOGGER = LoggerFactory.getLogger(SubscriptionPrefetchingTsFileQueue.class); - private final AtomicReference eventRef; + private final Map consumerIdToCurrentEvents; public SubscriptionPrefetchingTsFileQueue( final String brokerId, @@ -59,12 +62,18 @@ public SubscriptionPrefetchingTsFileQueue( final BoundedBlockingPendingQueue inputPendingQueue) { super(brokerId, topicName, inputPendingQueue); - this.eventRef = new AtomicReference<>(); + this.consumerIdToCurrentEvents = new ConcurrentHashMap<>(); } @Override - public synchronized SubscriptionEvent poll(final SubscriptionPollTimer timer) { - if (Objects.nonNull(eventRef.get())) { + public SubscriptionEvent poll(final String consumerId, final SubscriptionPollTimer timer) { + final SubscriptionEvent currentEvent = consumerIdToCurrentEvents.get(consumerId); + if (Objects.nonNull(currentEvent)) { + LOGGER.info( + "{} is currently transferring tsfile (with event {}) to consumer {}.", + this, + currentEvent, + consumerId); return null; } @@ -73,7 +82,8 @@ public synchronized SubscriptionEvent poll(final SubscriptionPollTimer timer) { event = UserDefinedEnrichedEvent.maybeOf(inputPendingQueue.waitedPoll()))) { if (!(event instanceof PipeTsFileInsertionEvent)) { LOGGER.warn( - "Subscription: SubscriptionPrefetchingTsFileQueue only support poll PipeTsFileInsertionEvent. Ignore {}.", + "Subscription: SubscriptionPrefetchingTsFileQueue {} only support poll PipeTsFileInsertionEvent. Ignore {}.", + this, event); continue; } @@ -85,94 +95,137 @@ public synchronized SubscriptionEvent poll(final SubscriptionPollTimer timer) { new SubscriptionEvent( Collections.singletonList(tsFileInsertionEvent), new SubscriptionPolledMessage( - SubscriptionPolledMessageType.TS_FILE_INFO.getType(), - new TsFileInfoMessagePayload(tsFileInsertionEvent.getTsFile().getName()), + SubscriptionPolledMessageType.TS_FILE_INIT.getType(), + new TsFileInitMessagePayload(tsFileInsertionEvent.getTsFile().getName()), commitContext)); - eventRef.set(subscriptionEvent); + consumerIdToCurrentEvents.put(consumerId, subscriptionEvent); // don't allow commit now + subscriptionEvent.recordLastPolledConsumerId(consumerId); + subscriptionEvent.recordLastPolledTimestamp(); return subscriptionEvent; } return null; } - public SubscriptionEvent pollTsFile(String fileName, long endWritingOffset) { - final SubscriptionEvent event = eventRef.get(); + public @NonNull SubscriptionEvent pollTsFile( + String consumerId, String fileName, long endWritingOffset) { + // 1. Extract the current event and inspect it + final SubscriptionEvent event = consumerIdToCurrentEvents.get(consumerId); if (Objects.isNull(event)) { - return null; + final String errorMessage = + String.format( + "%s is currently not transferring any tsfile to consumer %s", this, consumerId); + LOGGER.warn(errorMessage); + return generateSubscriptionEventWithTsFileErrorMessage(errorMessage, false); + } + + // check consumer id + if (!consumerId.equals(event.getLastPolledConsumerId())) { + final String errorMessage = + String.format( + "inconsistent polled consumer id, current is %s, incoming is %s, prefetching queue: %s", + event.getLastPolledConsumerId(), consumerId, this); + LOGGER.warn(errorMessage); + return generateSubscriptionEventWithTsFileErrorMessage(errorMessage, false); } final List enrichedEvents = event.getEnrichedEvents(); if (Objects.isNull(enrichedEvents) || enrichedEvents.size() != 1) { - return null; + final String errorMessage = + String.format( + "unexpected enrichedEvents: %s, prefetching queue: %s", enrichedEvents, this); + LOGGER.warn(errorMessage); + return generateSubscriptionEventWithTsFileErrorMessage(errorMessage, false); } final PipeTsFileInsertionEvent tsFileInsertionEvent = (PipeTsFileInsertionEvent) enrichedEvents.get(0); if (Objects.isNull(tsFileInsertionEvent)) { - return null; + final String errorMessage = + String.format( + "unexpected tsFileInsertionEvent: %s, prefetching queue: %s", + tsFileInsertionEvent, this); + LOGGER.warn(errorMessage); + return generateSubscriptionEventWithTsFileErrorMessage(errorMessage, false); } final SubscriptionPolledMessage polledMessage = event.getMessage(); if (Objects.isNull(polledMessage)) { - return null; + final String errorMessage = + String.format("unexpected polledMessage: %s, prefetching queue: %s", polledMessage, this); + LOGGER.warn(errorMessage); + return generateSubscriptionEventWithTsFileErrorMessage(errorMessage, false); } final SubscriptionMessagePayload messagePayload = polledMessage.getMessagePayload(); if (Objects.isNull(messagePayload)) { - return null; + final String errorMessage = + String.format( + "unexpected messagePayload: %s, prefetching queue: %s", messagePayload, this); + LOGGER.warn(errorMessage); + return generateSubscriptionEventWithTsFileErrorMessage(errorMessage, false); } final SubscriptionCommitContext commitContext = polledMessage.getCommitContext(); if (Objects.isNull(commitContext)) { - return null; + final String errorMessage = + String.format("unexpected commitContext: %s, prefetching queue: %s", commitContext, this); + LOGGER.warn(errorMessage); + return generateSubscriptionEventWithTsFileErrorMessage(errorMessage, false); } + // 2. Check message type, file name and offset final short messageType = polledMessage.getMessageType(); if (SubscriptionPolledMessageType.isValidatedMessageType(messageType)) { switch (SubscriptionPolledMessageType.valueOf(messageType)) { - case TS_FILE_INFO: + case TS_FILE_INIT: // check file name - if (!fileName.equals(((TsFileInfoMessagePayload) messagePayload).getFileName())) { - LOGGER.warn( - "inconsistent file name, current is {}, incoming is {}", - fileName, - ((TsFileInfoMessagePayload) messagePayload).getFileName()); - return null; + if (!fileName.equals(((TsFileInitMessagePayload) messagePayload).getFileName())) { + final String errorMessage = + String.format( + "inconsistent file name, current is %s, incoming is %s, prefetching queue: %s", + ((TsFileInitMessagePayload) messagePayload).getFileName(), fileName, this); + LOGGER.warn(errorMessage); + return generateSubscriptionEventWithTsFileErrorMessage(errorMessage, false); } // check offset if (endWritingOffset != 0) { - LOGGER.warn("reset file {} offset to {}", fileName, endWritingOffset); + LOGGER.warn("{} reset file {} offset to {}", this, fileName, endWritingOffset); } break; case TS_FILE_PIECE: // check file name if (!fileName.equals(((TsFilePieceMessagePayload) messagePayload).getFileName())) { - LOGGER.warn( - "inconsistent file name, current is {}, incoming is {}", - fileName, - ((TsFilePieceMessagePayload) messagePayload).getFileName()); - return null; + final String errorMessage = + String.format( + "inconsistent file name, current is %s, incoming is %s, prefetching queue: %s", + ((TsFilePieceMessagePayload) messagePayload).getFileName(), fileName, this); + LOGGER.warn(errorMessage); + return generateSubscriptionEventWithTsFileErrorMessage(errorMessage, false); } // check offset if (endWritingOffset != ((TsFilePieceMessagePayload) messagePayload).getEndWritingOffset()) { - LOGGER.warn("reset file {} offset to {}", fileName, endWritingOffset); + LOGGER.warn("{} reset file {} offset to {}", this, fileName, endWritingOffset); } break; case TS_FILE_SEAL: - LOGGER.warn("reset file {} offset to {}", fileName, endWritingOffset); + LOGGER.warn("{} reset file {} offset to {}", this, fileName, endWritingOffset); uncommittedEvents.remove(commitContext); break; default: - LOGGER.warn("unexpected message type: {}", messageType); - return null; + final String errorMessage = String.format("unexpected message type: %s", messageType); + LOGGER.warn(errorMessage); + return generateSubscriptionEventWithTsFileErrorMessage(errorMessage, false); } } else { - LOGGER.warn("unexpected message type: {}", messageType); - return null; + final String errorMessage = String.format("unexpected message type: %s", messageType); + LOGGER.warn(errorMessage); + return generateSubscriptionEventWithTsFileErrorMessage(errorMessage, false); } + // 3. Poll tsfile piece or tsfile seal final int readFileBufferSize = SubscriptionConfig.getInstance().getSubscriptionReadFileBufferSize(); final byte[] readBuffer = new byte[readFileBufferSize]; @@ -200,7 +253,10 @@ public SubscriptionEvent pollTsFile(String fileName, long endWritingOffset) { fileName, endWritingOffset + readLength, filePiece), commitContext)); - eventRef.set(newEvent); + consumerIdToCurrentEvents.put(consumerId, newEvent); + // don't allow commit now + newEvent.recordLastPolledConsumerId(consumerId); + newEvent.recordLastPolledTimestamp(); return newEvent; } @@ -212,16 +268,22 @@ public SubscriptionEvent pollTsFile(String fileName, long endWritingOffset) { SubscriptionPolledMessageType.TS_FILE_SEAL.getType(), new TsFileSealMessagePayload(fileName, tsFileInsertionEvent.getTsFile().length()), commitContext)); - eventRef.set(newEvent); + consumerIdToCurrentEvents.put(consumerId, newEvent); // allow commit now uncommittedEvents.put(commitContext, newEvent); + newEvent.recordLastPolledConsumerId(consumerId); + newEvent.recordLastPolledTimestamp(); return newEvent; } catch (IOException e) { - LOGGER.warn(e.getMessage()); + final String errorMessage = + String.format( + "IOException errored when %s transferring tsfile (with event %s) to consumer %s: %s", + this, event, consumerId, e.getMessage()); + LOGGER.warn(errorMessage); + // allow retry + return generateSubscriptionEventWithTsFileErrorMessage(errorMessage, true); } - - return null; } @Override @@ -229,7 +291,15 @@ public void executePrefetch() { // do nothing now } - void resetEventRef() { - eventRef.set(null); + /////////////////////////////// utility /////////////////////////////// + + private SubscriptionEvent generateSubscriptionEventWithTsFileErrorMessage( + final String errorMessage, final boolean retryable) { + return new SubscriptionEvent( + Collections.emptyList(), + new SubscriptionPolledMessage( + SubscriptionPolledMessageType.TS_FILE_ERROR.getType(), + new TsFileErrorMessagePayload(errorMessage, retryable), + super.generateInvalidSubscriptionCommitContext())); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionEvent.java index 9e539ce695e0..7536a598c0df 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionEvent.java @@ -24,6 +24,7 @@ import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPolledMessage; import java.util.List; +import java.util.stream.Collectors; public class SubscriptionEvent { @@ -32,6 +33,7 @@ public class SubscriptionEvent { private final List enrichedEvents; private final SubscriptionPolledMessage message; + private String lastPolledConsumerId; private long lastPolledTimestamp; private long committedTimestamp; @@ -83,4 +85,29 @@ public boolean pollable() { return System.currentTimeMillis() - lastPolledTimestamp > SubscriptionConfig.getInstance().getSubscriptionRecycleUncommittedEventIntervalMs(); } + + public void recordLastPolledConsumerId(final String consumerId) { + lastPolledConsumerId = consumerId; + } + + public String getLastPolledConsumerId() { + return lastPolledConsumerId; + } + + /////////////////////////////// object /////////////////////////////// + + @Override + public String toString() { + return "SubscriptionEvent{enrichedEvents=" + + enrichedEvents.stream().map(EnrichedEvent::coreReportMessage).collect(Collectors.toList()) + + ", message=" + + message + + ", lastPolledConsumerId=" + + lastPolledConsumerId + + ", lastPolledTimestamp=" + + lastPolledTimestamp + + ", committedTimestamp=" + + committedTimestamp + + "}"; + } } From b73fb778760385ceb81f9ad90bed6219b7d11e4e Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Wed, 17 Apr 2024 22:01:19 +0800 Subject: [PATCH 26/93] refact --- .../subscription/config/ConsumerConstant.java | 3 + .../subscription/SubscriptionConsumer.java | 99 +++++++- .../SubscriptionPullConsumer.java | 225 +++++++++++++----- .../SubscriptionPrefetchingTsFileQueue.java | 40 +++- 4 files changed, 285 insertions(+), 82 deletions(-) diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/config/ConsumerConstant.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/config/ConsumerConstant.java index 1db77c7e03a0..a3b458591aeb 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/config/ConsumerConstant.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/config/ConsumerConstant.java @@ -42,6 +42,9 @@ public class ConsumerConstant { public static final long ENDPOINTS_SYNC_INTERVAL_MS_DEFAULT_VALUE = 30000; public static final long ENDPOINTS_SYNC_INTERVAL_MS_MIN_VALUE = 5000; + public static final String TS_FILE_BASE_DIR_KEY = "ts-file-base-dir"; + public static final String TS_FILE_BASE_DIR_DEFAULT_VALUE = System.getProperty("java.io.tmpdir"); + /////////////////////////////// pull consumer /////////////////////////////// public static final String AUTO_COMMIT_KEY = "auto-commit"; diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java index 5efd48b222fc..2422a3729c11 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java @@ -26,7 +26,6 @@ import org.apache.iotdb.rpc.subscription.config.ConsumerConstant; import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionCommitContext; import org.apache.iotdb.session.util.SessionUtils; -import org.apache.iotdb.tsfile.utils.Pair; import org.apache.thrift.TException; import org.slf4j.Logger; @@ -37,6 +36,7 @@ import java.io.RandomAccessFile; import java.nio.file.Files; import java.nio.file.Path; +import java.nio.file.Paths; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -81,6 +81,8 @@ public abstract class SubscriptionConsumer implements AutoCloseable { private final AtomicBoolean isClosed = new AtomicBoolean(true); + private final String tsFileBaseDir; + public String getConsumerId() { return consumerId; } @@ -100,19 +102,82 @@ public String toString() { + "}"; } - /////////////////////////////// tsfile dir /////////////////////////////// + /////////////////////////////// tsfile /////////////////////////////// + + protected static class OnTheFlyTsFileInfo { - protected Path subscribedTsFileBaseDirPath; + SubscriptionCommitContext commitContext; + File file; + RandomAccessFile fileWriter; - protected final Map> - commitContextToTsFile = new ConcurrentHashMap<>(); + OnTheFlyTsFileInfo( + SubscriptionCommitContext commitContext, File file, RandomAccessFile fileWriter) { + this.commitContext = commitContext; + this.file = file; + this.fileWriter = fileWriter; + } - public Path getTsFileDir(String topicName) throws IOException { - if (Objects.isNull(subscribedTsFileBaseDirPath)) { - subscribedTsFileBaseDirPath = Files.createTempDirectory("subscribedTsFile#"); + String getTopicName() { + return commitContext.getTopicName(); + } + } + + protected final Map topicNameToOnTheFlyTsFileInfo = + new ConcurrentHashMap<>(); + + protected OnTheFlyTsFileInfo getOnTheFlyTsFileInfo(String topicName) { + final OnTheFlyTsFileInfo info = topicNameToOnTheFlyTsFileInfo.get(topicName); + if (Objects.isNull(info)) { + return null; + } + + if (!info.file.exists()) { + try { + info.fileWriter.close(); + } catch (final IOException e) { + LOGGER.warn(e.getMessage()); + } + topicNameToOnTheFlyTsFileInfo.remove(topicName); + return null; + } + + return info; + } + + protected void removeOnTheFlyTsFileInfo(String topicName) { + final OnTheFlyTsFileInfo info = topicNameToOnTheFlyTsFileInfo.get(topicName); + if (Objects.isNull(info)) { + return; } + + try { + info.fileWriter.close(); + } catch (final IOException e) { + LOGGER.warn(e.getMessage()); + } + topicNameToOnTheFlyTsFileInfo.remove(topicName); + } + + protected OnTheFlyTsFileInfo createOnTheFlyTsFileInfo( + SubscriptionCommitContext commitContext, String fileName) { + try { + final String topicName = commitContext.getTopicName(); + final Path filePath = getTsFileDir(topicName).resolve(fileName); + Files.createFile(filePath); + final File file = filePath.toFile(); + final RandomAccessFile fileWriter = new RandomAccessFile(file, "rw"); + final OnTheFlyTsFileInfo info = new OnTheFlyTsFileInfo(commitContext, file, fileWriter); + topicNameToOnTheFlyTsFileInfo.put(topicName, info); + return info; + } catch (final IOException e) { + LOGGER.warn(e.getMessage()); + return null; + } + } + + public Path getTsFileDir(final String topicName) throws IOException { final Path dirPath = - subscribedTsFileBaseDirPath.resolve(consumerGroupId).resolve(consumerId).resolve(topicName); + Paths.get(tsFileBaseDir).resolve(consumerGroupId).resolve(consumerId).resolve(topicName); Files.createDirectories(dirPath); return dirPath; } @@ -137,6 +202,8 @@ protected SubscriptionConsumer(Builder builder) { this.heartbeatIntervalMs = builder.heartbeatIntervalMs; this.endpointsSyncIntervalMs = builder.endpointsSyncIntervalMs; + + this.tsFileBaseDir = builder.tsFileBaseDir; } protected SubscriptionConsumer(Builder builder, Properties properties) { @@ -168,7 +235,12 @@ protected SubscriptionConsumer(Builder builder, Properties properties) { (Long) properties.getOrDefault( ConsumerConstant.ENDPOINTS_SYNC_INTERVAL_MS_KEY, - ConsumerConstant.ENDPOINTS_SYNC_INTERVAL_MS_DEFAULT_VALUE))); + ConsumerConstant.ENDPOINTS_SYNC_INTERVAL_MS_DEFAULT_VALUE)) + .tsFileBaseDir( + (String) + properties.getOrDefault( + ConsumerConstant.TS_FILE_BASE_DIR_KEY, + ConsumerConstant.TS_FILE_BASE_DIR_DEFAULT_VALUE))); } /////////////////////////////// open & close /////////////////////////////// @@ -535,6 +607,8 @@ public abstract static class Builder { protected long endpointsSyncIntervalMs = ConsumerConstant.ENDPOINTS_SYNC_INTERVAL_MS_DEFAULT_VALUE; + protected String tsFileBaseDir; + public Builder host(String host) { this.host = host; return this; @@ -582,6 +656,11 @@ public Builder endpointsSyncIntervalMs(long endpointsSyncIntervalMs) { return this; } + public Builder tsFileBaseDir(String tsFileBaseDir) { + this.tsFileBaseDir = tsFileBaseDir; + return this; + } + public abstract SubscriptionPullConsumer buildPullConsumer(); public abstract SubscriptionPushConsumer buildPushConsumer(); diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java index c701f7d424e5..4a86b7f1a781 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java @@ -32,6 +32,7 @@ import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPolledMessage; import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPolledMessageType; import org.apache.iotdb.rpc.subscription.payload.common.TabletsMessagePayload; +import org.apache.iotdb.rpc.subscription.payload.common.TsFileErrorMessagePayload; import org.apache.iotdb.rpc.subscription.payload.common.TsFileInitMessagePayload; import org.apache.iotdb.rpc.subscription.payload.common.TsFilePieceMessagePayload; import org.apache.iotdb.rpc.subscription.payload.common.TsFileSealMessagePayload; @@ -44,8 +45,6 @@ import java.io.File; import java.io.IOException; import java.io.RandomAccessFile; -import java.nio.file.Files; -import java.nio.file.Path; import java.time.Duration; import java.util.ArrayList; import java.util.Collections; @@ -53,6 +52,7 @@ import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Optional; import java.util.Properties; import java.util.Set; import java.util.SortedMap; @@ -62,6 +62,7 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.stream.Collectors; public class SubscriptionPullConsumer extends SubscriptionConsumer { @@ -151,42 +152,39 @@ public synchronized void close() throws IoTDBConnectionException { /////////////////////////////// poll & commit /////////////////////////////// public List poll(Duration timeoutMs) - throws TException, IOException, StatementExecutionException { + throws TException, IOException, StatementExecutionException, IoTDBConnectionException { return poll(Collections.emptySet(), timeoutMs.toMillis()); } public List poll(long timeoutMs) - throws TException, IOException, StatementExecutionException { + throws TException, IOException, StatementExecutionException, IoTDBConnectionException { return poll(Collections.emptySet(), timeoutMs); } public List poll(Set topicNames, Duration timeoutMs) - throws TException, IOException, StatementExecutionException { + throws TException, IOException, StatementExecutionException, IoTDBConnectionException { return poll(topicNames, timeoutMs.toMillis()); } public List poll(Set topicNames, long timeoutMs) - throws TException, IOException, StatementExecutionException { - List polledMessages = new ArrayList<>(); + throws TException, IOException, StatementExecutionException, IoTDBConnectionException { + final List messages = new ArrayList<>(); - acquireReadLock(); - try { - for (final SubscriptionProvider provider : getAllAvailableProviders()) { - // TODO: network timeout - polledMessages.addAll( - provider - .getSessionConnection() - .poll( - new SubscriptionPollMessage( - SubscriptionPollMessageType.POLL.getType(), - new PollMessagePayload(topicNames), - timeoutMs))); - } - } finally { - releaseReadLock(); + for (final OnTheFlyTsFileInfo info : + topicNameToOnTheFlyTsFileInfo.values().stream() + .filter( + info -> { + if (topicNames.isEmpty()) { + return true; + } + return topicNames.contains(info.getTopicName()); + }) + .collect(Collectors.toList())) { + pollTsFile(info.commitContext, info.file.getName(), timeoutMs).ifPresent(messages::add); } - final List messages = new ArrayList<>(); + final List polledMessages = pollInternal(topicNames, timeoutMs); + for (final SubscriptionPolledMessage polledMessage : polledMessages) { final short messageType = polledMessage.getMessageType(); if (SubscriptionPolledMessageType.isValidatedMessageType(messageType)) { @@ -198,19 +196,11 @@ public List poll(Set topicNames, long timeoutMs) ((TabletsMessagePayload) polledMessage.getMessagePayload()).getTablets())); break; case TS_FILE_INIT: - try { - final SubscriptionMessage message = - pollTsFile( - polledMessage.getCommitContext(), - ((TsFileInitMessagePayload) polledMessage.getMessagePayload()).getFileName(), - timeoutMs); - if (Objects.isNull(message)) { - throw new Exception("poll empty tsfile, will retry later..."); - } - messages.add(message); - } catch (Exception e) { - LOGGER.warn(e.getMessage()); - } + pollTsFile( + polledMessage.getCommitContext(), + ((TsFileInitMessagePayload) polledMessage.getMessagePayload()).getFileName(), + timeoutMs) + .ifPresent(messages::add); break; default: LOGGER.warn("unexpected message type: {}", messageType); @@ -235,40 +225,82 @@ public List poll(Set topicNames, long timeoutMs) return messages; } - private SubscriptionMessage pollTsFile( + private Optional pollTsFile( + SubscriptionCommitContext commitContext, String fileName, long timeoutMs) { + try { + final Pair messageWithRetryable = + pollTsFileInternal(commitContext, fileName, timeoutMs); + if (Objects.nonNull(messageWithRetryable.getLeft())) { + removeOnTheFlyTsFileInfo(commitContext.getTopicName()); + return Optional.of(messageWithRetryable.getLeft()); + } + if (!messageWithRetryable.getRight()) { + removeOnTheFlyTsFileInfo(commitContext.getTopicName()); + } + } catch (IOException e) { + LOGGER.warn( + "Exception occurred when {} polling TsFile {} with commit context {}: {}", + this, + fileName, + commitContext, + e.getMessage()); + // assume retryable + } catch (TException | IoTDBConnectionException | StatementExecutionException e) { + LOGGER.warn( + "Exception occurred when {} polling TsFile {} with commit context {}: {}", + this, + fileName, + commitContext, + e.getMessage()); + // assume non-retryable + removeOnTheFlyTsFileInfo(commitContext.getTopicName()); + } + return Optional.empty(); + } + + private Pair pollTsFileInternal( SubscriptionCommitContext commitContext, String fileName, long timeoutMs) - throws TException, IOException, StatementExecutionException, IoTDBConnectionException { + throws IOException, TException, IoTDBConnectionException, StatementExecutionException { final int dataNodeId = commitContext.getDataNodeId(); final String topicName = commitContext.getTopicName(); - final Path filePath = getTsFileDir(topicName).resolve(fileName); - Files.createFile(filePath); - final File file = filePath.toFile(); - final RandomAccessFile fileWriter = new RandomAccessFile(file, "rw"); - commitContextToTsFile.put(commitContext, new Pair<>(file, fileWriter)); + OnTheFlyTsFileInfo info = getOnTheFlyTsFileInfo(topicName); + if (Objects.isNull(info)) { + info = createOnTheFlyTsFileInfo(commitContext, fileName); + } + if (Objects.isNull(info)) { + return new Pair<>(null, false); + } + + final File file = info.file; + final RandomAccessFile fileWriter = info.fileWriter; - LOGGER.info("{} start poll tsfile: {}", this, file.getAbsolutePath()); + LOGGER.info( + "{} start to poll TsFile {} with commit context {}", + this, + file.getAbsolutePath(), + commitContext); + long endWritingOffset = fileWriter.length(); - long endWritingOffset = 0; while (true) { final List polledMessages = pollTsFileInternal(dataNodeId, topicName, fileName, endWritingOffset, timeoutMs); if (Objects.isNull(polledMessages) || polledMessages.size() != 1) { LOGGER.warn("unexpected polledMessages: {}, consumer: {}", polledMessages, this); - return null; + return new Pair<>(null, false); } final SubscriptionPolledMessage polledMessage = polledMessages.get(0); if (Objects.isNull(polledMessage)) { LOGGER.warn("unexpected polledMessage: {}, consumer: {}", polledMessage, this); - return null; + return new Pair<>(null, false); } final SubscriptionMessagePayload messagePayload = polledMessage.getMessagePayload(); if (Objects.isNull(messagePayload)) { LOGGER.warn("unexpected messagePayload: {}, consumer: {}", messagePayload, this); - return null; + return new Pair<>(null, false); } final SubscriptionCommitContext incomingCommitContext = polledMessage.getCommitContext(); @@ -276,10 +308,10 @@ private SubscriptionMessage pollTsFile( || !Objects.equals(commitContext, incomingCommitContext)) { LOGGER.warn( "inconsistent commit context, current is {}, incoming is {}, consumer: {}", - messagePayload, + commitContext, incomingCommitContext, this); - return null; + return new Pair<>(null, false); } final short messageType = polledMessage.getMessageType(); @@ -288,13 +320,20 @@ private SubscriptionMessage pollTsFile( case TS_FILE_PIECE: { // check file name - if (!fileName.equals(((TsFilePieceMessagePayload) messagePayload).getFileName())) { - - return null; + if (!Objects.equals( + fileName, ((TsFilePieceMessagePayload) messagePayload).getFileName())) { + LOGGER.warn( + "inconsistent file name, current is {}, incoming is {}, consumer: {}", + fileName, + ((TsFilePieceMessagePayload) messagePayload).getFileName(), + this); + return new Pair<>(null, false); } + // write file piece fileWriter.write(((TsFilePieceMessagePayload) messagePayload).getFilePiece()); fileWriter.getFD().sync(); + // update offset endWritingOffset = ((TsFilePieceMessagePayload) messagePayload).getEndWritingOffset(); break; @@ -302,37 +341,90 @@ private SubscriptionMessage pollTsFile( case TS_FILE_SEAL: { // check file name - if (!fileName.equals(((TsFileSealMessagePayload) messagePayload).getFileName())) { - - return null; + if (!Objects.equals( + fileName, ((TsFileSealMessagePayload) messagePayload).getFileName())) { + LOGGER.warn( + "inconsistent file name, current is {}, incoming is {}, consumer: {}", + fileName, + ((TsFileSealMessagePayload) messagePayload).getFileName(), + this); + return new Pair<>(null, false); } + // check file length if (fileWriter.length() != ((TsFileSealMessagePayload) messagePayload).getFileLength()) { - - return null; + LOGGER.warn( + "inconsistent file length, current is {}, incoming is {}, consumer: {}", + fileWriter.length(), + ((TsFileSealMessagePayload) messagePayload).getFileLength(), + this); + return new Pair<>(null, false); } + // sync and close fileWriter.getFD().sync(); fileWriter.close(); - commitContextToTsFile.remove(commitContext); - LOGGER.info("{} successfully poll tsfile: {}", this, file.getAbsolutePath()); + + LOGGER.info( + "{} successfully poll TsFile {} with commit context {}", + this, + file.getAbsolutePath(), + commitContext); + // generate subscription message - return new SubscriptionMessage(commitContext, file.getAbsolutePath()); + return new Pair<>( + new SubscriptionMessage(commitContext, file.getAbsolutePath()), true); } case TS_FILE_ERROR: - return null; + { + final String errorMessage = + ((TsFileErrorMessagePayload) messagePayload).getErrorMessage(); + final boolean retryable = ((TsFileErrorMessagePayload) messagePayload).isRetryable(); + LOGGER.warn( + "Error occurred when {} polling TsFile {} with commit context {}: {}, retryable: {}", + this, + file.getAbsolutePath(), + commitContext, + errorMessage, + retryable); + return new Pair<>(null, retryable); + } default: LOGGER.warn("unexpected message type: {}", messageType); - return null; + return new Pair<>(null, false); } } else { LOGGER.warn("unexpected message type: {}", messageType); - return null; + return new Pair<>(null, false); } } } + private List pollInternal(Set topicNames, long timeoutMs) + throws TException, IOException, StatementExecutionException { + final List polledMessages = new ArrayList<>(); + + acquireReadLock(); + try { + for (final SubscriptionProvider provider : getAllAvailableProviders()) { + // TODO: network timeout + polledMessages.addAll( + provider + .getSessionConnection() + .poll( + new SubscriptionPollMessage( + SubscriptionPollMessageType.POLL.getType(), + new PollMessagePayload(topicNames), + timeoutMs))); + } + } finally { + releaseReadLock(); + } + + return polledMessages; + } + private List pollTsFileInternal( int dataNodeId, String topicName, String fileName, long endWritingOffset, long timeoutMs) throws TException, IOException, StatementExecutionException, IoTDBConnectionException { @@ -499,6 +591,11 @@ public Builder endpointsSyncIntervalMs(long endpointsSyncIntervalMs) { return this; } + public Builder tsFileBaseDir(String tsFileBaseDir) { + super.tsFileBaseDir(tsFileBaseDir); + return this; + } + public Builder autoCommit(boolean autoCommit) { this.autoCommit = autoCommit; return this; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java index c216bbe42ffc..090dd10bf4c5 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java @@ -110,7 +110,7 @@ public SubscriptionEvent poll(final String consumerId, final SubscriptionPollTim public @NonNull SubscriptionEvent pollTsFile( String consumerId, String fileName, long endWritingOffset) { - // 1. Extract the current event and inspect it + // 1. Extract current event and check it final SubscriptionEvent event = consumerIdToCurrentEvents.get(consumerId); if (Objects.isNull(event)) { final String errorMessage = @@ -121,7 +121,7 @@ public SubscriptionEvent poll(final String consumerId, final SubscriptionPollTim } // check consumer id - if (!consumerId.equals(event.getLastPolledConsumerId())) { + if (!Objects.equals(event.getLastPolledConsumerId(), consumerId)) { final String errorMessage = String.format( "inconsistent polled consumer id, current is %s, incoming is %s, prefetching queue: %s", @@ -150,6 +150,15 @@ public SubscriptionEvent poll(final String consumerId, final SubscriptionPollTim return generateSubscriptionEventWithTsFileErrorMessage(errorMessage, false); } + if (!Objects.equals(tsFileInsertionEvent.getTsFile().getName(), fileName)) { + final String errorMessage = + String.format( + "inconsistent file name, current is %s, incoming is %s, prefetching queue: %s", + tsFileInsertionEvent.getTsFile().getName(), fileName, this); + LOGGER.warn(errorMessage); + return generateSubscriptionEventWithTsFileErrorMessage(errorMessage, false); + } + final SubscriptionPolledMessage polledMessage = event.getMessage(); if (Objects.isNull(polledMessage)) { final String errorMessage = @@ -181,7 +190,8 @@ public SubscriptionEvent poll(final String consumerId, final SubscriptionPollTim switch (SubscriptionPolledMessageType.valueOf(messageType)) { case TS_FILE_INIT: // check file name - if (!fileName.equals(((TsFileInitMessagePayload) messagePayload).getFileName())) { + if (!Objects.equals( + ((TsFileInitMessagePayload) messagePayload).getFileName(), fileName)) { final String errorMessage = String.format( "inconsistent file name, current is %s, incoming is %s, prefetching queue: %s", @@ -196,7 +206,8 @@ public SubscriptionEvent poll(final String consumerId, final SubscriptionPollTim break; case TS_FILE_PIECE: // check file name - if (!fileName.equals(((TsFilePieceMessagePayload) messagePayload).getFileName())) { + if (!Objects.equals( + ((TsFilePieceMessagePayload) messagePayload).getFileName(), fileName)) { final String errorMessage = String.format( "inconsistent file name, current is %s, incoming is %s, prefetching queue: %s", @@ -212,6 +223,7 @@ public SubscriptionEvent poll(final String consumerId, final SubscriptionPollTim break; case TS_FILE_SEAL: LOGGER.warn("{} reset file {} offset to {}", this, fileName, endWritingOffset); + // don't allow commit now uncommittedEvents.remove(commitContext); break; default: @@ -226,6 +238,14 @@ public SubscriptionEvent poll(final String consumerId, final SubscriptionPollTim } // 3. Poll tsfile piece or tsfile seal + return pollTsFile(consumerId, tsFileInsertionEvent, endWritingOffset, commitContext); + } + + private @NonNull SubscriptionEvent pollTsFile( + String consumerId, + PipeTsFileInsertionEvent tsFileInsertionEvent, + long endWritingOffset, + SubscriptionCommitContext commitContext) { final int readFileBufferSize = SubscriptionConfig.getInstance().getSubscriptionReadFileBufferSize(); final byte[] readBuffer = new byte[readFileBufferSize]; @@ -250,7 +270,9 @@ public SubscriptionEvent poll(final String consumerId, final SubscriptionPollTim new SubscriptionPolledMessage( SubscriptionPolledMessageType.TS_FILE_PIECE.getType(), new TsFilePieceMessagePayload( - fileName, endWritingOffset + readLength, filePiece), + tsFileInsertionEvent.getTsFile().getName(), + endWritingOffset + readLength, + filePiece), commitContext)); consumerIdToCurrentEvents.put(consumerId, newEvent); @@ -266,7 +288,9 @@ public SubscriptionEvent poll(final String consumerId, final SubscriptionPollTim Collections.singletonList(tsFileInsertionEvent), new SubscriptionPolledMessage( SubscriptionPolledMessageType.TS_FILE_SEAL.getType(), - new TsFileSealMessagePayload(fileName, tsFileInsertionEvent.getTsFile().length()), + new TsFileSealMessagePayload( + tsFileInsertionEvent.getTsFile().getName(), + tsFileInsertionEvent.getTsFile().length()), commitContext)); consumerIdToCurrentEvents.put(consumerId, newEvent); @@ -278,8 +302,8 @@ public SubscriptionEvent poll(final String consumerId, final SubscriptionPollTim } catch (IOException e) { final String errorMessage = String.format( - "IOException errored when %s transferring tsfile (with event %s) to consumer %s: %s", - this, event, consumerId, e.getMessage()); + "IOException occurred when %s transferring tsfile to consumer %s: %s", + this, consumerId, e.getMessage()); LOGGER.warn(errorMessage); // allow retry return generateSubscriptionEventWithTsFileErrorMessage(errorMessage, true); From 6f5d74269fdf7dac16a4494f465e0c04dff091ca Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Wed, 17 Apr 2024 22:34:40 +0800 Subject: [PATCH 27/93] improve --- .../subscription/SubscriptionConsumer.java | 36 +++++++++ .../SubscriptionPullConsumer.java | 15 +++- .../SubscriptionPrefetchingTsFileQueue.java | 75 +++++++++++++++++-- 3 files changed, 116 insertions(+), 10 deletions(-) diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java index 2422a3729c11..ec00a397c82e 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java @@ -109,17 +109,37 @@ protected static class OnTheFlyTsFileInfo { SubscriptionCommitContext commitContext; File file; RandomAccessFile fileWriter; + int retryCount; OnTheFlyTsFileInfo( SubscriptionCommitContext commitContext, File file, RandomAccessFile fileWriter) { this.commitContext = commitContext; this.file = file; this.fileWriter = fileWriter; + this.retryCount = 0; } String getTopicName() { return commitContext.getTopicName(); } + + /** @return {@code true} if exceed retry limit */ + boolean increaseRetryCountAndCheck() { + retryCount++; + return retryCount > 3; + } + + @Override + public String toString() { + return "OnTheFlyTsFileInfo{" + + "commitContext=" + + commitContext + + ", file=" + + file.getAbsoluteFile() + + ", retryCount=" + + retryCount + + "}"; + } } protected final Map topicNameToOnTheFlyTsFileInfo = @@ -155,19 +175,35 @@ protected void removeOnTheFlyTsFileInfo(String topicName) { } catch (final IOException e) { LOGGER.warn(e.getMessage()); } + + LOGGER.info("consumer {} remove on the fly tsfile info {}", this, info); topicNameToOnTheFlyTsFileInfo.remove(topicName); } + protected void increaseOnTheFlyTsFileInfoRetryCountOrRemove(String topicName) { + final OnTheFlyTsFileInfo info = topicNameToOnTheFlyTsFileInfo.get(topicName); + if (Objects.isNull(info)) { + return; + } + + if (info.increaseRetryCountAndCheck()) { + removeOnTheFlyTsFileInfo(topicName); + } + } + protected OnTheFlyTsFileInfo createOnTheFlyTsFileInfo( SubscriptionCommitContext commitContext, String fileName) { try { final String topicName = commitContext.getTopicName(); final Path filePath = getTsFileDir(topicName).resolve(fileName); + Files.createFile(filePath); final File file = filePath.toFile(); final RandomAccessFile fileWriter = new RandomAccessFile(file, "rw"); + final OnTheFlyTsFileInfo info = new OnTheFlyTsFileInfo(commitContext, file, fileWriter); topicNameToOnTheFlyTsFileInfo.put(topicName, info); + LOGGER.info("consumer {} create on the fly tsfile info {}", this, info); return info; } catch (final IOException e) { LOGGER.warn(e.getMessage()); diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java index 4a86b7f1a781..ca178c4b56a7 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java @@ -170,6 +170,7 @@ public List poll(Set topicNames, long timeoutMs) throws TException, IOException, StatementExecutionException, IoTDBConnectionException { final List messages = new ArrayList<>(); + // poll on the fly tsfile for (final OnTheFlyTsFileInfo info : topicNameToOnTheFlyTsFileInfo.values().stream() .filter( @@ -183,9 +184,8 @@ public List poll(Set topicNames, long timeoutMs) pollTsFile(info.commitContext, info.file.getName(), timeoutMs).ifPresent(messages::add); } - final List polledMessages = pollInternal(topicNames, timeoutMs); - - for (final SubscriptionPolledMessage polledMessage : polledMessages) { + // poll tablets or tsfile + for (final SubscriptionPolledMessage polledMessage : pollInternal(topicNames, timeoutMs)) { final short messageType = polledMessage.getMessageType(); if (SubscriptionPolledMessageType.isValidatedMessageType(messageType)) { switch (SubscriptionPolledMessageType.valueOf(messageType)) { @@ -211,8 +211,9 @@ public List poll(Set topicNames, long timeoutMs) } } + // add to uncommitted messages if (autoCommit) { - long currentTimestamp = System.currentTimeMillis(); + final long currentTimestamp = System.currentTimeMillis(); long index = currentTimestamp / autoCommitIntervalMs; if (currentTimestamp % autoCommitIntervalMs == 0) { index -= 1; @@ -235,7 +236,10 @@ private Optional pollTsFile( return Optional.of(messageWithRetryable.getLeft()); } if (!messageWithRetryable.getRight()) { + // non-retryable removeOnTheFlyTsFileInfo(commitContext.getTopicName()); + } else { + increaseOnTheFlyTsFileInfoRetryCountOrRemove(commitContext.getTopicName()); } } catch (IOException e) { LOGGER.warn( @@ -245,6 +249,7 @@ private Optional pollTsFile( commitContext, e.getMessage()); // assume retryable + increaseOnTheFlyTsFileInfoRetryCountOrRemove(commitContext.getTopicName()); } catch (TException | IoTDBConnectionException | StatementExecutionException e) { LOGGER.warn( "Exception occurred when {} polling TsFile {} with commit context {}: {}", @@ -449,6 +454,8 @@ private List pollTsFileInternal( } } + /////////////////////////////// commit /////////////////////////////// + public void commitSync(SubscriptionMessage message) throws TException, IOException, StatementExecutionException, IoTDBConnectionException { commitSync(Collections.singletonList(message)); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java index 090dd10bf4c5..82392de93038 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java @@ -69,12 +69,40 @@ public SubscriptionPrefetchingTsFileQueue( public SubscriptionEvent poll(final String consumerId, final SubscriptionPollTimer timer) { final SubscriptionEvent currentEvent = consumerIdToCurrentEvents.get(consumerId); if (Objects.nonNull(currentEvent)) { - LOGGER.info( - "{} is currently transferring tsfile (with event {}) to consumer {}.", - this, - currentEvent, - consumerId); - return null; + if (currentEvent.isCommitted()) { + consumerIdToCurrentEvents.remove(consumerId); + } else { + // uncommitted event + if (!currentEvent.pollable()) { + LOGGER.info( + "{} is currently transferring tsfile (with event {}) to consumer {}.", + this, + currentEvent, + consumerId); + } else { + // uncommitted and pollable event + final SubscriptionEvent subscriptionEvent = + new SubscriptionEvent( + Collections.singletonList(currentEvent.getEnrichedEvents().get(0)), + new SubscriptionPolledMessage( + SubscriptionPolledMessageType.TS_FILE_INIT.getType(), + new TsFileInitMessagePayload( + ((PipeTsFileInsertionEvent) currentEvent.getEnrichedEvents().get(0)) + .getTsFile() + .getName()), + currentEvent.getMessage().getCommitContext())); + consumerIdToCurrentEvents.put(consumerId, subscriptionEvent); + // don't allow commit now + subscriptionEvent.recordLastPolledConsumerId(consumerId); + subscriptionEvent.recordLastPolledTimestamp(); + return subscriptionEvent; + } + } + } + + final SubscriptionEvent stealEvent = stealPollableOnTheFlyTsFile(consumerId); + if (Objects.nonNull(stealEvent)) { + return stealEvent; } Event event; @@ -317,6 +345,41 @@ public void executePrefetch() { /////////////////////////////// utility /////////////////////////////// + private SubscriptionEvent stealPollableOnTheFlyTsFile(final String consumerId) { + for (final Map.Entry entry : consumerIdToCurrentEvents.entrySet()) { + if (Objects.equals(consumerId, entry.getKey())) { + continue; + } + final SubscriptionEvent currentEvent = entry.getValue(); + if (currentEvent.isCommitted()) { + consumerIdToCurrentEvents.remove(entry.getKey()); + continue; + } + // uncommitted event + if (currentEvent.pollable()) { + consumerIdToCurrentEvents.remove(entry.getKey()); + + final SubscriptionEvent subscriptionEvent = + new SubscriptionEvent( + Collections.singletonList(currentEvent.getEnrichedEvents().get(0)), + new SubscriptionPolledMessage( + SubscriptionPolledMessageType.TS_FILE_INIT.getType(), + new TsFileInitMessagePayload( + ((PipeTsFileInsertionEvent) currentEvent.getEnrichedEvents().get(0)) + .getTsFile() + .getName()), + currentEvent.getMessage().getCommitContext())); + consumerIdToCurrentEvents.put(consumerId, subscriptionEvent); + // don't allow commit now + subscriptionEvent.recordLastPolledConsumerId(consumerId); + subscriptionEvent.recordLastPolledTimestamp(); + return subscriptionEvent; + } + } + + return null; + } + private SubscriptionEvent generateSubscriptionEventWithTsFileErrorMessage( final String errorMessage, final boolean retryable) { return new SubscriptionEvent( From 43fe24c247a9768f30269e5673527edf68da31fc Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Wed, 17 Apr 2024 22:42:41 +0800 Subject: [PATCH 28/93] improve --- .../SubscriptionPrefetchingTsFileQueue.java | 88 +++++++------------ 1 file changed, 30 insertions(+), 58 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java index 82392de93038..56583cdf7a5a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java @@ -67,42 +67,9 @@ public SubscriptionPrefetchingTsFileQueue( @Override public SubscriptionEvent poll(final String consumerId, final SubscriptionPollTimer timer) { - final SubscriptionEvent currentEvent = consumerIdToCurrentEvents.get(consumerId); - if (Objects.nonNull(currentEvent)) { - if (currentEvent.isCommitted()) { - consumerIdToCurrentEvents.remove(consumerId); - } else { - // uncommitted event - if (!currentEvent.pollable()) { - LOGGER.info( - "{} is currently transferring tsfile (with event {}) to consumer {}.", - this, - currentEvent, - consumerId); - } else { - // uncommitted and pollable event - final SubscriptionEvent subscriptionEvent = - new SubscriptionEvent( - Collections.singletonList(currentEvent.getEnrichedEvents().get(0)), - new SubscriptionPolledMessage( - SubscriptionPolledMessageType.TS_FILE_INIT.getType(), - new TsFileInitMessagePayload( - ((PipeTsFileInsertionEvent) currentEvent.getEnrichedEvents().get(0)) - .getTsFile() - .getName()), - currentEvent.getMessage().getCommitContext())); - consumerIdToCurrentEvents.put(consumerId, subscriptionEvent); - // don't allow commit now - subscriptionEvent.recordLastPolledConsumerId(consumerId); - subscriptionEvent.recordLastPolledTimestamp(); - return subscriptionEvent; - } - } - } - - final SubscriptionEvent stealEvent = stealPollableOnTheFlyTsFile(consumerId); - if (Objects.nonNull(stealEvent)) { - return stealEvent; + final SubscriptionEvent pollableEvent = checkPollableOnTheFlyTsFile(consumerId); + if (Objects.nonNull(pollableEvent)) { + return pollableEvent; } Event event; @@ -345,36 +312,41 @@ public void executePrefetch() { /////////////////////////////// utility /////////////////////////////// - private SubscriptionEvent stealPollableOnTheFlyTsFile(final String consumerId) { + private synchronized SubscriptionEvent checkPollableOnTheFlyTsFile(final String consumerId) { for (final Map.Entry entry : consumerIdToCurrentEvents.entrySet()) { - if (Objects.equals(consumerId, entry.getKey())) { - continue; - } final SubscriptionEvent currentEvent = entry.getValue(); if (currentEvent.isCommitted()) { consumerIdToCurrentEvents.remove(entry.getKey()); continue; } - // uncommitted event - if (currentEvent.pollable()) { - consumerIdToCurrentEvents.remove(entry.getKey()); - final SubscriptionEvent subscriptionEvent = - new SubscriptionEvent( - Collections.singletonList(currentEvent.getEnrichedEvents().get(0)), - new SubscriptionPolledMessage( - SubscriptionPolledMessageType.TS_FILE_INIT.getType(), - new TsFileInitMessagePayload( - ((PipeTsFileInsertionEvent) currentEvent.getEnrichedEvents().get(0)) - .getTsFile() - .getName()), - currentEvent.getMessage().getCommitContext())); - consumerIdToCurrentEvents.put(consumerId, subscriptionEvent); - // don't allow commit now - subscriptionEvent.recordLastPolledConsumerId(consumerId); - subscriptionEvent.recordLastPolledTimestamp(); - return subscriptionEvent; + if (!currentEvent.pollable()) { + LOGGER.info( + "{} is currently transferring tsfile (with event {}) to consumer {}.", + this, + currentEvent, + entry.getKey()); + continue; } + + // uncommitted and pollable event + consumerIdToCurrentEvents.remove(entry.getKey()); + + final SubscriptionEvent subscriptionEvent = + new SubscriptionEvent( + Collections.singletonList(currentEvent.getEnrichedEvents().get(0)), + new SubscriptionPolledMessage( + SubscriptionPolledMessageType.TS_FILE_INIT.getType(), + new TsFileInitMessagePayload( + ((PipeTsFileInsertionEvent) currentEvent.getEnrichedEvents().get(0)) + .getTsFile() + .getName()), + currentEvent.getMessage().getCommitContext())); + consumerIdToCurrentEvents.put(consumerId, subscriptionEvent); + // don't allow commit now + subscriptionEvent.recordLastPolledConsumerId(consumerId); + subscriptionEvent.recordLastPolledTimestamp(); + return subscriptionEvent; } return null; From dece6e19e3a5df4be435fdde99ed6a673cc30e6c Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Wed, 17 Apr 2024 22:49:07 +0800 Subject: [PATCH 29/93] fix --- .../apache/iotdb/session/subscription/SubscriptionConsumer.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java index ec00a397c82e..a341e94bea6b 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java @@ -643,7 +643,7 @@ public abstract static class Builder { protected long endpointsSyncIntervalMs = ConsumerConstant.ENDPOINTS_SYNC_INTERVAL_MS_DEFAULT_VALUE; - protected String tsFileBaseDir; + protected String tsFileBaseDir = ConsumerConstant.TS_FILE_BASE_DIR_DEFAULT_VALUE; public Builder host(String host) { this.host = host; From f3abdd06b9c9dd9b2fb76df6fca498f2317e88b9 Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Thu, 18 Apr 2024 13:42:11 +0800 Subject: [PATCH 30/93] prefetch tsfile --- .../common/PollTsFileMessagePayload.java | 22 +- .../common/TsFilePieceMessagePayload.java | 22 +- .../subscription/SubscriptionConsumer.java | 8 +- .../SubscriptionPullConsumer.java | 25 +- .../agent/SubscriptionBrokerAgent.java | 4 +- .../broker/SubscriptionBroker.java | 17 +- .../SubscriptionPrefetchingTsFileQueue.java | 246 ++++++------------ .../subscription/event/SubscriptionEvent.java | 4 +- .../event/SubscriptionTsFileEvent.java | 217 +++++++++++++++ .../receiver/SubscriptionReceiverV1.java | 2 +- 10 files changed, 348 insertions(+), 219 deletions(-) create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionTsFileEvent.java diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/PollTsFileMessagePayload.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/PollTsFileMessagePayload.java index 658416eaad7c..4993bbfc0ff5 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/PollTsFileMessagePayload.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/PollTsFileMessagePayload.java @@ -32,7 +32,7 @@ public class PollTsFileMessagePayload implements SubscriptionMessagePayload { private transient String fileName; - private transient long endWritingOffset; + private transient long writingOffset; public String getTopicName() { return topicName; @@ -42,30 +42,30 @@ public String getFileName() { return fileName; } - public long getEndWritingOffset() { - return endWritingOffset; + public long getWritingOffset() { + return writingOffset; } public PollTsFileMessagePayload() {} - public PollTsFileMessagePayload(String topicName, String fileName, long endWritingOffset) { + public PollTsFileMessagePayload(String topicName, String fileName, long writingOffset) { this.topicName = topicName; this.fileName = fileName; - this.endWritingOffset = endWritingOffset; + this.writingOffset = writingOffset; } @Override public void serialize(DataOutputStream stream) throws IOException { ReadWriteIOUtils.write(topicName, stream); ReadWriteIOUtils.write(fileName, stream); - ReadWriteIOUtils.write(endWritingOffset, stream); + ReadWriteIOUtils.write(writingOffset, stream); } @Override public SubscriptionMessagePayload deserialize(ByteBuffer buffer) { topicName = ReadWriteIOUtils.readString(buffer); fileName = ReadWriteIOUtils.readString(buffer); - endWritingOffset = ReadWriteIOUtils.readLong(buffer); + writingOffset = ReadWriteIOUtils.readLong(buffer); return this; } @@ -82,12 +82,12 @@ public boolean equals(final Object obj) { final PollTsFileMessagePayload that = (PollTsFileMessagePayload) obj; return Objects.equals(this.topicName, that.topicName) && Objects.equals(this.fileName, that.fileName) - && Objects.equals(this.endWritingOffset, that.endWritingOffset); + && Objects.equals(this.writingOffset, that.writingOffset); } @Override public int hashCode() { - return Objects.hash(topicName, fileName, endWritingOffset); + return Objects.hash(topicName, fileName, writingOffset); } @Override @@ -96,8 +96,8 @@ public String toString() { + topicName + ", fileName=" + fileName - + ", endWritingOffset=" - + endWritingOffset + + ", writingOffset=" + + writingOffset + "}"; } } diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFilePieceMessagePayload.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFilePieceMessagePayload.java index 36660b69334a..3d3c7323ab58 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFilePieceMessagePayload.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFilePieceMessagePayload.java @@ -32,7 +32,7 @@ public class TsFilePieceMessagePayload implements SubscriptionMessagePayload { private transient String fileName; - private transient long endWritingOffset; + private transient long nextWritingOffset; private transient byte[] filePiece; @@ -40,8 +40,8 @@ public String getFileName() { return fileName; } - public long getEndWritingOffset() { - return endWritingOffset; + public long getNextWritingOffset() { + return nextWritingOffset; } public byte[] getFilePiece() { @@ -50,23 +50,23 @@ public byte[] getFilePiece() { public TsFilePieceMessagePayload() {} - public TsFilePieceMessagePayload(String fileName, long endWritingOffset, byte[] filePiece) { + public TsFilePieceMessagePayload(String fileName, long nextWritingOffset, byte[] filePiece) { this.fileName = fileName; - this.endWritingOffset = endWritingOffset; + this.nextWritingOffset = nextWritingOffset; this.filePiece = filePiece; } @Override public void serialize(DataOutputStream stream) throws IOException { ReadWriteIOUtils.write(fileName, stream); - ReadWriteIOUtils.write(endWritingOffset, stream); + ReadWriteIOUtils.write(nextWritingOffset, stream); ReadWriteIOUtils.write(new Binary(filePiece), stream); } @Override public SubscriptionMessagePayload deserialize(ByteBuffer buffer) { this.fileName = ReadWriteIOUtils.readString(buffer); - this.endWritingOffset = ReadWriteIOUtils.readLong(buffer); + this.nextWritingOffset = ReadWriteIOUtils.readLong(buffer); final int size = ReadWriteIOUtils.readInt(buffer); this.filePiece = ReadWriteIOUtils.readBytes(buffer, size); return this; @@ -82,21 +82,21 @@ public boolean equals(final Object obj) { } final TsFilePieceMessagePayload that = (TsFilePieceMessagePayload) obj; return Objects.equals(this.fileName, that.fileName) - && Objects.equals(this.endWritingOffset, that.endWritingOffset) + && Objects.equals(this.nextWritingOffset, that.nextWritingOffset) && Arrays.equals(this.filePiece, that.filePiece); } @Override public int hashCode() { - return Objects.hash(fileName, endWritingOffset, filePiece); + return Objects.hash(fileName, nextWritingOffset, filePiece); } @Override public String toString() { return "TsFilePieceMessagePayload{fileName=" + fileName - + ", endWritingOffset=" - + endWritingOffset + + ", nextWritingOffset=" + + nextWritingOffset + ", filePiece=" + Arrays.toString(filePiece) + "}"; diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java index a341e94bea6b..d1286ca5b275 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java @@ -61,6 +61,8 @@ public abstract class SubscriptionConsumer implements AutoCloseable { private static final IoTDBConnectionException NO_PROVIDERS_EXCEPTION = new IoTDBConnectionException("Cluster has no available subscription providers to connect"); + private static final int ON_THE_FLY_TS_FILE_RETRY_LIMIT = 3; + private final List initialEndpoints; private final String username; @@ -124,9 +126,9 @@ String getTopicName() { } /** @return {@code true} if exceed retry limit */ - boolean increaseRetryCountAndCheck() { + boolean increaseRetryCountAndCheckIfExceedRetryLimit() { retryCount++; - return retryCount > 3; + return retryCount > ON_THE_FLY_TS_FILE_RETRY_LIMIT; } @Override @@ -186,7 +188,7 @@ protected void increaseOnTheFlyTsFileInfoRetryCountOrRemove(String topicName) { return; } - if (info.increaseRetryCountAndCheck()) { + if (info.increaseRetryCountAndCheckIfExceedRetryLimit()) { removeOnTheFlyTsFileInfo(topicName); } } diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java index ca178c4b56a7..aab07b837f44 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java @@ -239,6 +239,7 @@ private Optional pollTsFile( // non-retryable removeOnTheFlyTsFileInfo(commitContext.getTopicName()); } else { + // retryable increaseOnTheFlyTsFileInfoRetryCountOrRemove(commitContext.getTopicName()); } } catch (IOException e) { @@ -285,29 +286,19 @@ private Pair pollTsFileInternal( this, file.getAbsolutePath(), commitContext); - long endWritingOffset = fileWriter.length(); + long writingOffset = fileWriter.length(); while (true) { final List polledMessages = - pollTsFileInternal(dataNodeId, topicName, fileName, endWritingOffset, timeoutMs); + pollTsFileInternal(dataNodeId, topicName, fileName, writingOffset, timeoutMs); - if (Objects.isNull(polledMessages) || polledMessages.size() != 1) { - LOGGER.warn("unexpected polledMessages: {}, consumer: {}", polledMessages, this); + if (polledMessages.isEmpty()) { + LOGGER.warn("poll empty messages, consumer: {}", this); return new Pair<>(null, false); } final SubscriptionPolledMessage polledMessage = polledMessages.get(0); - if (Objects.isNull(polledMessage)) { - LOGGER.warn("unexpected polledMessage: {}, consumer: {}", polledMessage, this); - return new Pair<>(null, false); - } - final SubscriptionMessagePayload messagePayload = polledMessage.getMessagePayload(); - if (Objects.isNull(messagePayload)) { - LOGGER.warn("unexpected messagePayload: {}, consumer: {}", messagePayload, this); - return new Pair<>(null, false); - } - final SubscriptionCommitContext incomingCommitContext = polledMessage.getCommitContext(); if (Objects.isNull(incomingCommitContext) || !Objects.equals(commitContext, incomingCommitContext)) { @@ -340,7 +331,7 @@ private Pair pollTsFileInternal( fileWriter.getFD().sync(); // update offset - endWritingOffset = ((TsFilePieceMessagePayload) messagePayload).getEndWritingOffset(); + writingOffset = ((TsFilePieceMessagePayload) messagePayload).getNextWritingOffset(); break; } case TS_FILE_SEAL: @@ -431,7 +422,7 @@ private List pollInternal(Set topicNames, lon } private List pollTsFileInternal( - int dataNodeId, String topicName, String fileName, long endWritingOffset, long timeoutMs) + int dataNodeId, String topicName, String fileName, long writingOffset, long timeoutMs) throws TException, IOException, StatementExecutionException, IoTDBConnectionException { acquireReadLock(); try { @@ -447,7 +438,7 @@ private List pollTsFileInternal( .poll( new SubscriptionPollMessage( SubscriptionPollMessageType.POLL_TS_FILE.getType(), - new PollTsFileMessagePayload(topicName, fileName, endWritingOffset), + new PollTsFileMessagePayload(topicName, fileName, writingOffset), timeoutMs)); } finally { releaseReadLock(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionBrokerAgent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionBrokerAgent.java index 8993b4414e3a..3e9b6cf8772b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionBrokerAgent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionBrokerAgent.java @@ -62,7 +62,7 @@ public List poll( } public List pollTsFile( - ConsumerConfig consumerConfig, String topicName, String fileName, long endWritingOffset) { + ConsumerConfig consumerConfig, String topicName, String fileName, long writingOffset) { final String consumerGroupId = consumerConfig.getConsumerGroupId(); final SubscriptionBroker broker = consumerGroupIdToSubscriptionBroker.get(consumerGroupId); if (Objects.isNull(broker)) { @@ -71,7 +71,7 @@ public List pollTsFile( return Collections.emptyList(); } final String consumerId = consumerConfig.getConsumerId(); - return broker.pollTsFile(consumerId, topicName, fileName, endWritingOffset); + return broker.pollTsFile(consumerId, topicName, fileName, writingOffset); } public void commit( 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 10455e041b31..c301178fb887 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 @@ -31,6 +31,7 @@ import org.slf4j.LoggerFactory; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Objects; @@ -78,20 +79,18 @@ public List poll( } public List pollTsFile( - String consumerId, String topicName, String fileName, long endWritingOffset) { - SubscriptionPrefetchingQueue prefetchingQueue = topicNameToPrefetchingQueue.get(topicName); + String consumerId, String topicName, String fileName, long writingOffset) { + final SubscriptionPrefetchingQueue prefetchingQueue = + topicNameToPrefetchingQueue.get(topicName); if (Objects.isNull(prefetchingQueue)) { - return null; + return Collections.emptyList(); } if (!(prefetchingQueue instanceof SubscriptionPrefetchingTsFileQueue)) { - return null; + return Collections.emptyList(); } - final List events = new ArrayList<>(); - final SubscriptionEvent event = + return Collections.singletonList( ((SubscriptionPrefetchingTsFileQueue) prefetchingQueue) - .pollTsFile(consumerId, fileName, endWritingOffset); - events.add(event); - return events; + .pollTsFile(consumerId, fileName, writingOffset)); } public void commit(final List commitContexts) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java index 56583cdf7a5a..cf66806b33d0 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java @@ -21,10 +21,9 @@ import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.commons.pipe.task.connection.BoundedBlockingPendingQueue; -import org.apache.iotdb.commons.subscription.config.SubscriptionConfig; import org.apache.iotdb.db.pipe.event.UserDefinedEnrichedEvent; import org.apache.iotdb.db.pipe.event.common.tsfile.PipeTsFileInsertionEvent; -import org.apache.iotdb.db.subscription.event.SubscriptionEvent; +import org.apache.iotdb.db.subscription.event.SubscriptionTsFileEvent; import org.apache.iotdb.db.subscription.timer.SubscriptionPollTimer; import org.apache.iotdb.pipe.api.event.Event; import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionCommitContext; @@ -34,15 +33,13 @@ import org.apache.iotdb.rpc.subscription.payload.common.TsFileErrorMessagePayload; import org.apache.iotdb.rpc.subscription.payload.common.TsFileInitMessagePayload; import org.apache.iotdb.rpc.subscription.payload.common.TsFilePieceMessagePayload; -import org.apache.iotdb.rpc.subscription.payload.common.TsFileSealMessagePayload; +import org.apache.iotdb.tsfile.utils.Pair; import org.checkerframework.checker.nullness.qual.NonNull; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; -import java.io.RandomAccessFile; -import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.Map; @@ -54,7 +51,7 @@ public class SubscriptionPrefetchingTsFileQueue extends SubscriptionPrefetchingQ private static final Logger LOGGER = LoggerFactory.getLogger(SubscriptionPrefetchingTsFileQueue.class); - private final Map consumerIdToCurrentEvents; + private final Map consumerIdToCurrentEventMap; public SubscriptionPrefetchingTsFileQueue( final String brokerId, @@ -62,12 +59,13 @@ public SubscriptionPrefetchingTsFileQueue( final BoundedBlockingPendingQueue inputPendingQueue) { super(brokerId, topicName, inputPendingQueue); - this.consumerIdToCurrentEvents = new ConcurrentHashMap<>(); + this.consumerIdToCurrentEventMap = new ConcurrentHashMap<>(); } @Override - public SubscriptionEvent poll(final String consumerId, final SubscriptionPollTimer timer) { - final SubscriptionEvent pollableEvent = checkPollableOnTheFlyTsFile(consumerId); + public SubscriptionTsFileEvent poll(final String consumerId, final SubscriptionPollTimer timer) { + final SubscriptionTsFileEvent pollableEvent = + getPollableOnTheFlySubscriptionTsFileEvent(consumerId); if (Objects.nonNull(pollableEvent)) { return pollableEvent; } @@ -86,15 +84,12 @@ public SubscriptionEvent poll(final String consumerId, final SubscriptionPollTim final PipeTsFileInsertionEvent tsFileInsertionEvent = (PipeTsFileInsertionEvent) event; final SubscriptionCommitContext commitContext = generateSubscriptionCommitContext(); - final SubscriptionEvent subscriptionEvent = - new SubscriptionEvent( - Collections.singletonList(tsFileInsertionEvent), - new SubscriptionPolledMessage( - SubscriptionPolledMessageType.TS_FILE_INIT.getType(), - new TsFileInitMessagePayload(tsFileInsertionEvent.getTsFile().getName()), - commitContext)); - consumerIdToCurrentEvents.put(consumerId, subscriptionEvent); - // don't allow commit now + // update current event + final SubscriptionTsFileEvent subscriptionEvent = + SubscriptionTsFileEvent.generateSubscriptionTsFileEventWithInitPayload( + tsFileInsertionEvent, commitContext); + consumerIdToCurrentEventMap.put(consumerId, subscriptionEvent); + subscriptionEvent.recordLastPolledConsumerId(consumerId); subscriptionEvent.recordLastPolledTimestamp(); return subscriptionEvent; @@ -103,16 +98,16 @@ public SubscriptionEvent poll(final String consumerId, final SubscriptionPollTim return null; } - public @NonNull SubscriptionEvent pollTsFile( - String consumerId, String fileName, long endWritingOffset) { + public @NonNull SubscriptionTsFileEvent pollTsFile( + String consumerId, String fileName, long writingOffset) { // 1. Extract current event and check it - final SubscriptionEvent event = consumerIdToCurrentEvents.get(consumerId); + final SubscriptionTsFileEvent event = consumerIdToCurrentEventMap.get(consumerId); if (Objects.isNull(event)) { final String errorMessage = String.format( "%s is currently not transferring any tsfile to consumer %s", this, consumerId); LOGGER.warn(errorMessage); - return generateSubscriptionEventWithTsFileErrorMessage(errorMessage, false); + return generateSubscriptionTsFileEventWithErrorMessage(errorMessage, false); } // check consumer id @@ -122,62 +117,25 @@ public SubscriptionEvent poll(final String consumerId, final SubscriptionPollTim "inconsistent polled consumer id, current is %s, incoming is %s, prefetching queue: %s", event.getLastPolledConsumerId(), consumerId, this); LOGGER.warn(errorMessage); - return generateSubscriptionEventWithTsFileErrorMessage(errorMessage, false); + return generateSubscriptionTsFileEventWithErrorMessage(errorMessage, false); } final List enrichedEvents = event.getEnrichedEvents(); - if (Objects.isNull(enrichedEvents) || enrichedEvents.size() != 1) { - final String errorMessage = - String.format( - "unexpected enrichedEvents: %s, prefetching queue: %s", enrichedEvents, this); - LOGGER.warn(errorMessage); - return generateSubscriptionEventWithTsFileErrorMessage(errorMessage, false); - } - final PipeTsFileInsertionEvent tsFileInsertionEvent = (PipeTsFileInsertionEvent) enrichedEvents.get(0); - if (Objects.isNull(tsFileInsertionEvent)) { - final String errorMessage = - String.format( - "unexpected tsFileInsertionEvent: %s, prefetching queue: %s", - tsFileInsertionEvent, this); - LOGGER.warn(errorMessage); - return generateSubscriptionEventWithTsFileErrorMessage(errorMessage, false); - } + // check file name if (!Objects.equals(tsFileInsertionEvent.getTsFile().getName(), fileName)) { final String errorMessage = String.format( "inconsistent file name, current is %s, incoming is %s, prefetching queue: %s", tsFileInsertionEvent.getTsFile().getName(), fileName, this); LOGGER.warn(errorMessage); - return generateSubscriptionEventWithTsFileErrorMessage(errorMessage, false); + return generateSubscriptionTsFileEventWithErrorMessage(errorMessage, false); } final SubscriptionPolledMessage polledMessage = event.getMessage(); - if (Objects.isNull(polledMessage)) { - final String errorMessage = - String.format("unexpected polledMessage: %s, prefetching queue: %s", polledMessage, this); - LOGGER.warn(errorMessage); - return generateSubscriptionEventWithTsFileErrorMessage(errorMessage, false); - } - final SubscriptionMessagePayload messagePayload = polledMessage.getMessagePayload(); - if (Objects.isNull(messagePayload)) { - final String errorMessage = - String.format( - "unexpected messagePayload: %s, prefetching queue: %s", messagePayload, this); - LOGGER.warn(errorMessage); - return generateSubscriptionEventWithTsFileErrorMessage(errorMessage, false); - } - - final SubscriptionCommitContext commitContext = polledMessage.getCommitContext(); - if (Objects.isNull(commitContext)) { - final String errorMessage = - String.format("unexpected commitContext: %s, prefetching queue: %s", commitContext, this); - LOGGER.warn(errorMessage); - return generateSubscriptionEventWithTsFileErrorMessage(errorMessage, false); - } // 2. Check message type, file name and offset final short messageType = polledMessage.getMessageType(); @@ -192,11 +150,11 @@ public SubscriptionEvent poll(final String consumerId, final SubscriptionPollTim "inconsistent file name, current is %s, incoming is %s, prefetching queue: %s", ((TsFileInitMessagePayload) messagePayload).getFileName(), fileName, this); LOGGER.warn(errorMessage); - return generateSubscriptionEventWithTsFileErrorMessage(errorMessage, false); + return generateSubscriptionTsFileEventWithErrorMessage(errorMessage, false); } // check offset - if (endWritingOffset != 0) { - LOGGER.warn("{} reset file {} offset to {}", this, fileName, endWritingOffset); + if (writingOffset != 0) { + LOGGER.warn("{} reset file {} offset to {}", this, fileName, writingOffset); } break; case TS_FILE_PIECE: @@ -208,115 +166,81 @@ public SubscriptionEvent poll(final String consumerId, final SubscriptionPollTim "inconsistent file name, current is %s, incoming is %s, prefetching queue: %s", ((TsFilePieceMessagePayload) messagePayload).getFileName(), fileName, this); LOGGER.warn(errorMessage); - return generateSubscriptionEventWithTsFileErrorMessage(errorMessage, false); + return generateSubscriptionTsFileEventWithErrorMessage(errorMessage, false); } // check offset - if (endWritingOffset - != ((TsFilePieceMessagePayload) messagePayload).getEndWritingOffset()) { - LOGGER.warn("{} reset file {} offset to {}", this, fileName, endWritingOffset); + if (writingOffset + != ((TsFilePieceMessagePayload) messagePayload).getNextWritingOffset()) { + LOGGER.warn("{} reset file {} offset to {}", this, fileName, writingOffset); } break; case TS_FILE_SEAL: - LOGGER.warn("{} reset file {} offset to {}", this, fileName, endWritingOffset); - // don't allow commit now - uncommittedEvents.remove(commitContext); + LOGGER.warn("{} reset file {} offset to {}", this, fileName, writingOffset); break; default: final String errorMessage = String.format("unexpected message type: %s", messageType); LOGGER.warn(errorMessage); - return generateSubscriptionEventWithTsFileErrorMessage(errorMessage, false); + return generateSubscriptionTsFileEventWithErrorMessage(errorMessage, false); } } else { final String errorMessage = String.format("unexpected message type: %s", messageType); LOGGER.warn(errorMessage); - return generateSubscriptionEventWithTsFileErrorMessage(errorMessage, false); + return generateSubscriptionTsFileEventWithErrorMessage(errorMessage, false); } // 3. Poll tsfile piece or tsfile seal - return pollTsFile(consumerId, tsFileInsertionEvent, endWritingOffset, commitContext); + return pollTsFile(consumerId, writingOffset, event); } - private @NonNull SubscriptionEvent pollTsFile( - String consumerId, - PipeTsFileInsertionEvent tsFileInsertionEvent, - long endWritingOffset, - SubscriptionCommitContext commitContext) { - final int readFileBufferSize = - SubscriptionConfig.getInstance().getSubscriptionReadFileBufferSize(); - final byte[] readBuffer = new byte[readFileBufferSize]; - try (final RandomAccessFile reader = - new RandomAccessFile(tsFileInsertionEvent.getTsFile(), "r")) { - while (true) { - reader.seek(endWritingOffset); - final int readLength = reader.read(readBuffer); - if (readLength == -1) { - break; - } - - final byte[] filePiece = - readLength == readFileBufferSize - ? readBuffer - : Arrays.copyOfRange(readBuffer, 0, readLength); - - // poll tsfile piece - final SubscriptionEvent newEvent = - new SubscriptionEvent( - Collections.singletonList(tsFileInsertionEvent), - new SubscriptionPolledMessage( - SubscriptionPolledMessageType.TS_FILE_PIECE.getType(), - new TsFilePieceMessagePayload( - tsFileInsertionEvent.getTsFile().getName(), - endWritingOffset + readLength, - filePiece), - commitContext)); - - consumerIdToCurrentEvents.put(consumerId, newEvent); - // don't allow commit now - newEvent.recordLastPolledConsumerId(consumerId); - newEvent.recordLastPolledTimestamp(); - return newEvent; + private @NonNull SubscriptionTsFileEvent pollTsFile( + String consumerId, long writingOffset, SubscriptionTsFileEvent event) { + Pair newEventWithCommittable = event.matchNext(writingOffset); + if (Objects.isNull(newEventWithCommittable)) { + try { + newEventWithCommittable = + event.generateSubscriptionTsFileEventWithPieceOrSealPayload(writingOffset); + } catch (IOException e) { + final String errorMessage = + String.format( + "IOException occurred when %s transferring tsfile to consumer %s: %s", + this, consumerId, e.getMessage()); + LOGGER.warn(errorMessage); + // assume retryable + return generateSubscriptionTsFileEventWithErrorMessage(errorMessage, true); } + } - // poll tsfile seal - final SubscriptionEvent newEvent = - new SubscriptionEvent( - Collections.singletonList(tsFileInsertionEvent), - new SubscriptionPolledMessage( - SubscriptionPolledMessageType.TS_FILE_SEAL.getType(), - new TsFileSealMessagePayload( - tsFileInsertionEvent.getTsFile().getName(), - tsFileInsertionEvent.getTsFile().length()), - commitContext)); - - consumerIdToCurrentEvents.put(consumerId, newEvent); - // allow commit now - uncommittedEvents.put(commitContext, newEvent); - newEvent.recordLastPolledConsumerId(consumerId); - newEvent.recordLastPolledTimestamp(); - return newEvent; - } catch (IOException e) { - final String errorMessage = - String.format( - "IOException occurred when %s transferring tsfile to consumer %s: %s", - this, consumerId, e.getMessage()); - LOGGER.warn(errorMessage); - // allow retry - return generateSubscriptionEventWithTsFileErrorMessage(errorMessage, true); + // remove outdated event + consumerIdToCurrentEventMap.remove(consumerId); + event.resetNext(); + + // update current event + final SubscriptionTsFileEvent newEvent = newEventWithCommittable.getLeft(); + consumerIdToCurrentEventMap.put(consumerId, newEvent); + if (newEventWithCommittable.getRight()) { + uncommittedEvents.put(newEvent.getMessage().getCommitContext(), newEvent); } + + newEvent.recordLastPolledConsumerId(consumerId); + newEvent.recordLastPolledTimestamp(); + return newEvent; } @Override public void executePrefetch() { - // do nothing now + consumerIdToCurrentEventMap.values().forEach(SubscriptionTsFileEvent::prefetchNext); + consumerIdToCurrentEventMap.values().forEach(SubscriptionTsFileEvent::serializeNext); } /////////////////////////////// utility /////////////////////////////// - private synchronized SubscriptionEvent checkPollableOnTheFlyTsFile(final String consumerId) { - for (final Map.Entry entry : consumerIdToCurrentEvents.entrySet()) { - final SubscriptionEvent currentEvent = entry.getValue(); + private synchronized SubscriptionTsFileEvent getPollableOnTheFlySubscriptionTsFileEvent( + final String consumerId) { + for (final Map.Entry entry : + consumerIdToCurrentEventMap.entrySet()) { + final SubscriptionTsFileEvent currentEvent = entry.getValue(); if (currentEvent.isCommitted()) { - consumerIdToCurrentEvents.remove(entry.getKey()); + consumerIdToCurrentEventMap.remove(entry.getKey()); continue; } @@ -330,31 +254,27 @@ private synchronized SubscriptionEvent checkPollableOnTheFlyTsFile(final String } // uncommitted and pollable event - consumerIdToCurrentEvents.remove(entry.getKey()); - - final SubscriptionEvent subscriptionEvent = - new SubscriptionEvent( - Collections.singletonList(currentEvent.getEnrichedEvents().get(0)), - new SubscriptionPolledMessage( - SubscriptionPolledMessageType.TS_FILE_INIT.getType(), - new TsFileInitMessagePayload( - ((PipeTsFileInsertionEvent) currentEvent.getEnrichedEvents().get(0)) - .getTsFile() - .getName()), - currentEvent.getMessage().getCommitContext())); - consumerIdToCurrentEvents.put(consumerId, subscriptionEvent); - // don't allow commit now - subscriptionEvent.recordLastPolledConsumerId(consumerId); - subscriptionEvent.recordLastPolledTimestamp(); - return subscriptionEvent; + + // remove outdated event + consumerIdToCurrentEventMap.remove(entry.getKey()); + currentEvent.resetNext(); + + // update current event + final SubscriptionTsFileEvent newEvent = + currentEvent.generateSubscriptionTsFileEventWithInitPayload(); + consumerIdToCurrentEventMap.put(consumerId, newEvent); + + newEvent.recordLastPolledConsumerId(consumerId); + newEvent.recordLastPolledTimestamp(); + return newEvent; } return null; } - private SubscriptionEvent generateSubscriptionEventWithTsFileErrorMessage( + private SubscriptionTsFileEvent generateSubscriptionTsFileEventWithErrorMessage( final String errorMessage, final boolean retryable) { - return new SubscriptionEvent( + return new SubscriptionTsFileEvent( Collections.emptyList(), new SubscriptionPolledMessage( SubscriptionPolledMessageType.TS_FILE_ERROR.getType(), diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionEvent.java index 7536a598c0df..53025f8c2286 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionEvent.java @@ -30,8 +30,8 @@ public class SubscriptionEvent { private static final long INVALID_TIMESTAMP = -1; - private final List enrichedEvents; - private final SubscriptionPolledMessage message; + protected final List enrichedEvents; + protected final SubscriptionPolledMessage message; private String lastPolledConsumerId; private long lastPolledTimestamp; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionTsFileEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionTsFileEvent.java new file mode 100644 index 000000000000..bc2c61f63263 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionTsFileEvent.java @@ -0,0 +1,217 @@ +/* + * 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.subscription.event; + +import org.apache.iotdb.commons.pipe.event.EnrichedEvent; +import org.apache.iotdb.commons.subscription.config.SubscriptionConfig; +import org.apache.iotdb.db.pipe.event.common.tsfile.PipeTsFileInsertionEvent; +import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionCommitContext; +import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionMessagePayload; +import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPolledMessage; +import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPolledMessageType; +import org.apache.iotdb.rpc.subscription.payload.common.TsFileInitMessagePayload; +import org.apache.iotdb.rpc.subscription.payload.common.TsFilePieceMessagePayload; +import org.apache.iotdb.rpc.subscription.payload.common.TsFileSealMessagePayload; +import org.apache.iotdb.tsfile.utils.Pair; + +import org.checkerframework.checker.nullness.qual.NonNull; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.RandomAccessFile; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Objects; +import java.util.concurrent.atomic.AtomicReference; + +public class SubscriptionTsFileEvent extends SubscriptionEvent { + + private static final Logger LOGGER = LoggerFactory.getLogger(SubscriptionTsFileEvent.class); + + private final AtomicReference> nextEventWithCommittableRef; + + public SubscriptionTsFileEvent( + List enrichedEvents, SubscriptionPolledMessage message) { + super(enrichedEvents, message); + + this.nextEventWithCommittableRef = new AtomicReference<>(); + } + + public void prefetchNext() { + nextEventWithCommittableRef.getAndUpdate( + (nextEventWithCommittable) -> { + if (Objects.nonNull(nextEventWithCommittable)) { + // do nothing + return nextEventWithCommittable; + } + + final SubscriptionPolledMessage polledMessage = this.getMessage(); + final short messageType = polledMessage.getMessageType(); + final SubscriptionMessagePayload messagePayload = polledMessage.getMessagePayload(); + if (SubscriptionPolledMessageType.isValidatedMessageType(messageType)) { + switch (SubscriptionPolledMessageType.valueOf(messageType)) { + case TS_FILE_INIT: + try { + return generateSubscriptionTsFileEventWithPieceOrSealPayload(0); + } catch (IOException e) { + LOGGER.warn(e.getMessage()); + return null; + } + case TS_FILE_PIECE: + try { + return generateSubscriptionTsFileEventWithPieceOrSealPayload( + ((TsFilePieceMessagePayload) messagePayload).getNextWritingOffset()); + } catch (IOException e) { + LOGGER.warn(e.getMessage()); + return null; + } + case TS_FILE_SEAL: + return null; + default: + LOGGER.warn("unexpected message type: {}", messageType); + return null; + } + } else { + LOGGER.warn("unexpected message type: {}", messageType); + return null; + } + }); + } + + public void serializeNext() { + nextEventWithCommittableRef.getAndUpdate( + (nextEventWithCommittable) -> { + if (Objects.nonNull(nextEventWithCommittable)) { + nextEventWithCommittable.getLeft().getMessage().trySerialize(); + return nextEventWithCommittable; + } + + return null; + }); + } + + public void resetNext() { + nextEventWithCommittableRef.set(null); + } + + public Pair<@NonNull SubscriptionTsFileEvent, Boolean> matchNext(long writingOffset) { + final Pair nextEventWithCommittable = + nextEventWithCommittableRef.get(); + if (Objects.isNull(nextEventWithCommittable)) { + return null; + } + + final SubscriptionPolledMessage polledMessage = this.getMessage(); + final short messageType = polledMessage.getMessageType(); + final SubscriptionMessagePayload messagePayload = polledMessage.getMessagePayload(); + if (SubscriptionPolledMessageType.isValidatedMessageType(messageType)) { + switch (SubscriptionPolledMessageType.valueOf(messageType)) { + case TS_FILE_INIT: + if (Objects.equals(writingOffset, 0)) { + return nextEventWithCommittable; + } + return null; + case TS_FILE_PIECE: + if (Objects.equals( + writingOffset, ((TsFilePieceMessagePayload) messagePayload).getNextWritingOffset())) { + return nextEventWithCommittable; + } + return null; + case TS_FILE_SEAL: + return null; + default: + LOGGER.warn("unexpected message type: {}", messageType); + return null; + } + } else { + LOGGER.warn("unexpected message type: {}", messageType); + return null; + } + } + + public static SubscriptionTsFileEvent generateSubscriptionTsFileEventWithInitPayload( + PipeTsFileInsertionEvent tsFileInsertionEvent, SubscriptionCommitContext commitContext) { + return new SubscriptionTsFileEvent( + Collections.singletonList(tsFileInsertionEvent), + new SubscriptionPolledMessage( + SubscriptionPolledMessageType.TS_FILE_INIT.getType(), + new TsFileInitMessagePayload(tsFileInsertionEvent.getTsFile().getName()), + commitContext)); + } + + public SubscriptionTsFileEvent generateSubscriptionTsFileEventWithInitPayload() { + return generateSubscriptionTsFileEventWithInitPayload( + (PipeTsFileInsertionEvent) this.getEnrichedEvents().get(0), + this.getMessage().getCommitContext()); + } + + public @NonNull Pair<@NonNull SubscriptionTsFileEvent, Boolean> + generateSubscriptionTsFileEventWithPieceOrSealPayload(long writingOffset) throws IOException { + final PipeTsFileInsertionEvent tsFileInsertionEvent = + (PipeTsFileInsertionEvent) this.getEnrichedEvents().get(0); + final SubscriptionCommitContext commitContext = this.getMessage().getCommitContext(); + + final int readFileBufferSize = + SubscriptionConfig.getInstance().getSubscriptionReadFileBufferSize(); + final byte[] readBuffer = new byte[readFileBufferSize]; + try (final RandomAccessFile reader = + new RandomAccessFile(tsFileInsertionEvent.getTsFile(), "r")) { + while (true) { + reader.seek(writingOffset); + final int readLength = reader.read(readBuffer); + if (readLength == -1) { + break; + } + + final byte[] filePiece = + readLength == readFileBufferSize + ? readBuffer + : Arrays.copyOfRange(readBuffer, 0, readLength); + + // generate subscription tsfile event with piece payload + return new Pair<>( + new SubscriptionTsFileEvent( + Collections.singletonList(tsFileInsertionEvent), + new SubscriptionPolledMessage( + SubscriptionPolledMessageType.TS_FILE_PIECE.getType(), + new TsFilePieceMessagePayload( + tsFileInsertionEvent.getTsFile().getName(), + writingOffset + readLength, + filePiece), + commitContext)), + false); + } + + // generate subscription tsfile event with seal payload + return new Pair<>( + new SubscriptionTsFileEvent( + Collections.singletonList(tsFileInsertionEvent), + new SubscriptionPolledMessage( + SubscriptionPolledMessageType.TS_FILE_SEAL.getType(), + new TsFileSealMessagePayload( + tsFileInsertionEvent.getTsFile().getName(), + tsFileInsertionEvent.getTsFile().length()), + commitContext)), + true); + } + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java index 584bfecba58c..fe056469c65b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java @@ -396,7 +396,7 @@ private TPipeSubscribeResp handlePipeSubscribePollTsFileInternal( consumerConfig, messagePayload.getTopicName(), messagePayload.getFileName(), - messagePayload.getEndWritingOffset()); + messagePayload.getWritingOffset()); final List polledMessages = events.stream().map(SubscriptionEvent::getMessage).collect(Collectors.toList()); From 376510e10a1ee8e52b0f819bbab7a6aebb25b66c Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Thu, 18 Apr 2024 13:45:50 +0800 Subject: [PATCH 31/93] fix --- .../subscription/broker/SubscriptionPrefetchingTsFileQueue.java | 1 + 1 file changed, 1 insertion(+) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java index cf66806b33d0..9dd22bd37fed 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java @@ -196,6 +196,7 @@ public SubscriptionTsFileEvent poll(final String consumerId, final SubscriptionP String consumerId, long writingOffset, SubscriptionTsFileEvent event) { Pair newEventWithCommittable = event.matchNext(writingOffset); if (Objects.isNull(newEventWithCommittable)) { + event.resetNext(); try { newEventWithCommittable = event.generateSubscriptionTsFileEventWithPieceOrSealPayload(writingOffset); From 0a8992d961b2c5e748c10cd066a50d1266b69aba Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Thu, 18 Apr 2024 23:30:15 +0800 Subject: [PATCH 32/93] fix --- .../subscription/payload/common/TsFilePieceMessagePayload.java | 2 +- .../subscription/broker/SubscriptionPrefetchingTsFileQueue.java | 1 + .../java/org/apache/iotdb/commons/pipe/event/EnrichedEvent.java | 1 + 3 files changed, 3 insertions(+), 1 deletion(-) diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFilePieceMessagePayload.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFilePieceMessagePayload.java index 3d3c7323ab58..d8197b6f340c 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFilePieceMessagePayload.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFilePieceMessagePayload.java @@ -88,7 +88,7 @@ public boolean equals(final Object obj) { @Override public int hashCode() { - return Objects.hash(fileName, nextWritingOffset, filePiece); + return Objects.hash(fileName, nextWritingOffset, Arrays.hashCode(filePiece)); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java index 9dd22bd37fed..e84e0cedd480 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java @@ -176,6 +176,7 @@ public SubscriptionTsFileEvent poll(final String consumerId, final SubscriptionP break; case TS_FILE_SEAL: LOGGER.warn("{} reset file {} offset to {}", this, fileName, writingOffset); + uncommittedEvents.remove(polledMessage.getCommitContext()); break; default: final String errorMessage = String.format("unexpected message type: %s", messageType); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/event/EnrichedEvent.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/event/EnrichedEvent.java index 7076b9eb5d9e..9d1f0525ee1d 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/event/EnrichedEvent.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/event/EnrichedEvent.java @@ -183,6 +183,7 @@ public boolean clearReferenceCount(final String holderMessage) { boolean isSuccessful = true; synchronized (this) { if (referenceCount.get() >= 1 && !isReleased.get()) { + // We assume that this function will not throw any exceptions. isSuccessful = internallyDecreaseResourceReferenceCount(holderMessage); isReleased.set(true); } From 92e705fb74d136df2362459b4a11f4196a38b23e Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Thu, 18 Apr 2024 23:49:52 +0800 Subject: [PATCH 33/93] improve --- .../SubscriptionPrefetchingTsFileQueue.java | 12 ++-- .../event/SubscriptionTsFileEvent.java | 68 +++++++++---------- 2 files changed, 38 insertions(+), 42 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java index e84e0cedd480..fc2d372a8437 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java @@ -98,7 +98,7 @@ public SubscriptionTsFileEvent poll(final String consumerId, final SubscriptionP return null; } - public @NonNull SubscriptionTsFileEvent pollTsFile( + public synchronized @NonNull SubscriptionTsFileEvent pollTsFile( String consumerId, String fileName, long writingOffset) { // 1. Extract current event and check it final SubscriptionTsFileEvent event = consumerIdToCurrentEventMap.get(consumerId); @@ -193,11 +193,11 @@ public SubscriptionTsFileEvent poll(final String consumerId, final SubscriptionP return pollTsFile(consumerId, writingOffset, event); } - private @NonNull SubscriptionTsFileEvent pollTsFile( + private synchronized @NonNull SubscriptionTsFileEvent pollTsFile( String consumerId, long writingOffset, SubscriptionTsFileEvent event) { - Pair newEventWithCommittable = event.matchNext(writingOffset); + Pair newEventWithCommittable = + event.matchOrResetNext(writingOffset); if (Objects.isNull(newEventWithCommittable)) { - event.resetNext(); try { newEventWithCommittable = event.generateSubscriptionTsFileEventWithPieceOrSealPayload(writingOffset); @@ -214,7 +214,6 @@ public SubscriptionTsFileEvent poll(final String consumerId, final SubscriptionP // remove outdated event consumerIdToCurrentEventMap.remove(consumerId); - event.resetNext(); // update current event final SubscriptionTsFileEvent newEvent = newEventWithCommittable.getLeft(); @@ -229,7 +228,7 @@ public SubscriptionTsFileEvent poll(final String consumerId, final SubscriptionP } @Override - public void executePrefetch() { + public synchronized void executePrefetch() { consumerIdToCurrentEventMap.values().forEach(SubscriptionTsFileEvent::prefetchNext); consumerIdToCurrentEventMap.values().forEach(SubscriptionTsFileEvent::serializeNext); } @@ -259,7 +258,6 @@ private synchronized SubscriptionTsFileEvent getPollableOnTheFlySubscriptionTsFi // remove outdated event consumerIdToCurrentEventMap.remove(entry.getKey()); - currentEvent.resetNext(); // update current event final SubscriptionTsFileEvent newEvent = diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionTsFileEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionTsFileEvent.java index bc2c61f63263..1326ae9b827b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionTsFileEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionTsFileEvent.java @@ -109,43 +109,41 @@ public void serializeNext() { }); } - public void resetNext() { - nextEventWithCommittableRef.set(null); - } - - public Pair<@NonNull SubscriptionTsFileEvent, Boolean> matchNext(long writingOffset) { - final Pair nextEventWithCommittable = - nextEventWithCommittableRef.get(); - if (Objects.isNull(nextEventWithCommittable)) { - return null; - } - - final SubscriptionPolledMessage polledMessage = this.getMessage(); - final short messageType = polledMessage.getMessageType(); - final SubscriptionMessagePayload messagePayload = polledMessage.getMessagePayload(); - if (SubscriptionPolledMessageType.isValidatedMessageType(messageType)) { - switch (SubscriptionPolledMessageType.valueOf(messageType)) { - case TS_FILE_INIT: - if (Objects.equals(writingOffset, 0)) { - return nextEventWithCommittable; + public Pair<@NonNull SubscriptionTsFileEvent, Boolean> matchOrResetNext(long writingOffset) { + return nextEventWithCommittableRef.getAndUpdate( + (nextEventWithCommittable) -> { + if (Objects.isNull(nextEventWithCommittable)) { + return null; } - return null; - case TS_FILE_PIECE: - if (Objects.equals( - writingOffset, ((TsFilePieceMessagePayload) messagePayload).getNextWritingOffset())) { - return nextEventWithCommittable; + + final SubscriptionPolledMessage polledMessage = this.getMessage(); + final short messageType = polledMessage.getMessageType(); + final SubscriptionMessagePayload messagePayload = polledMessage.getMessagePayload(); + if (SubscriptionPolledMessageType.isValidatedMessageType(messageType)) { + switch (SubscriptionPolledMessageType.valueOf(messageType)) { + case TS_FILE_INIT: + if (Objects.equals(writingOffset, 0)) { + return nextEventWithCommittable; + } + return null; + case TS_FILE_PIECE: + if (Objects.equals( + writingOffset, + ((TsFilePieceMessagePayload) messagePayload).getNextWritingOffset())) { + return nextEventWithCommittable; + } + return null; + case TS_FILE_SEAL: + return null; + default: + LOGGER.warn("unexpected message type: {}", messageType); + return null; + } + } else { + LOGGER.warn("unexpected message type: {}", messageType); + return null; } - return null; - case TS_FILE_SEAL: - return null; - default: - LOGGER.warn("unexpected message type: {}", messageType); - return null; - } - } else { - LOGGER.warn("unexpected message type: {}", messageType); - return null; - } + }); } public static SubscriptionTsFileEvent generateSubscriptionTsFileEventWithInitPayload( From 1775bb613a85b3399f3e5bfce79fddef9db8b71e Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Fri, 19 Apr 2024 00:21:03 +0800 Subject: [PATCH 34/93] fix --- .../main/java/org/apache/iotdb/rpc/TSStatusCode.java | 1 - .../subscription/SubscriptionPullConsumer.java | 12 ++++++++++++ 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java index f70c07ab1c51..e434852e870d 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java @@ -239,7 +239,6 @@ public enum TSStatusCode { SUBSCRIPTION_UNSUBSCRIBE_ERROR(1908), SUBSCRIPTION_MISSING_CUSTOMER(1909), SHOW_SUBSCRIPTION_ERROR(1910), - SUBSCRIPTION_POLL_TS_FILE_ERROR(1911), // Topic CREATE_TOPIC_ERROR(2000), diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java index aab07b837f44..6dde48334fc0 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java @@ -330,6 +330,18 @@ private Pair pollTsFileInternal( fileWriter.write(((TsFilePieceMessagePayload) messagePayload).getFilePiece()); fileWriter.getFD().sync(); + // check offset + if (!Objects.equals( + fileWriter.length(), + ((TsFilePieceMessagePayload) messagePayload).getNextWritingOffset())) { + LOGGER.warn( + "inconsistent file offset, current is {}, incoming is {}, consumer: {}", + fileWriter.length(), + ((TsFilePieceMessagePayload) messagePayload).getNextWritingOffset(), + this); + return new Pair<>(null, false); + } + // update offset writingOffset = ((TsFilePieceMessagePayload) messagePayload).getNextWritingOffset(); break; From 0322247f27e559170dd702aba13e870620764b8e Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Fri, 19 Apr 2024 15:01:48 +0800 Subject: [PATCH 35/93] improve --- .../PullConsumerAutoCommitWorker.java | 60 ------- .../subscription/SubscriptionConsumer.java | 167 +++++++++--------- .../SubscriptionPullConsumer.java | 40 +++-- .../SubscriptionPushConsumer.java | 37 ++-- 4 files changed, 142 insertions(+), 162 deletions(-) delete mode 100644 iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/PullConsumerAutoCommitWorker.java diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/PullConsumerAutoCommitWorker.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/PullConsumerAutoCommitWorker.java deleted file mode 100644 index d69be7097ee3..000000000000 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/PullConsumerAutoCommitWorker.java +++ /dev/null @@ -1,60 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iotdb.session.subscription; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.Map; -import java.util.Set; - -public class PullConsumerAutoCommitWorker implements Runnable { - - private static final Logger LOGGER = LoggerFactory.getLogger(PullConsumerAutoCommitWorker.class); - - private final SubscriptionPullConsumer consumer; - - public PullConsumerAutoCommitWorker(SubscriptionPullConsumer consumer) { - this.consumer = consumer; - } - - @Override - public void run() { - if (consumer.isClosed()) { - return; - } - - long currentTimestamp = System.currentTimeMillis(); - long index = currentTimestamp / consumer.getAutoCommitIntervalMs(); - if (currentTimestamp % consumer.getAutoCommitIntervalMs() == 0) { - index -= 1; - } - - for (Map.Entry> entry : - consumer.getUncommittedMessages().headMap(index).entrySet()) { - try { - consumer.commitSync(entry.getValue()); - consumer.getUncommittedMessages().remove(entry.getKey()); - } catch (final Exception e) { - LOGGER.warn("something unexpected happened when auto commit messages...", e); - } - } - } -} diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java index 190ef32450af..fb8fc7f30599 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java @@ -124,6 +124,9 @@ public String toString() { /////////////////////////////// tsfile /////////////////////////////// + protected final Map topicNameToOnTheFlyTsFileInfo = + new ConcurrentHashMap<>(); + protected static class OnTheFlyTsFileInfo { SubscriptionCommitContext commitContext; @@ -162,9 +165,6 @@ public String toString() { } } - protected final Map topicNameToOnTheFlyTsFileInfo = - new ConcurrentHashMap<>(); - protected OnTheFlyTsFileInfo getOnTheFlyTsFileInfo(String topicName) { final OnTheFlyTsFileInfo info = topicNameToOnTheFlyTsFileInfo.get(topicName); if (Objects.isNull(info)) { @@ -334,7 +334,7 @@ public synchronized void close() throws IoTDBConnectionException { // shutdown endpoints syncer shutdownEndpointsSyncer(); - // shutdown workers + // shutdown workers: heartbeat worker and async commit executor shutdownWorkers(); // close subscription providers @@ -539,7 +539,63 @@ void openProviders() throws IoTDBConnectionException { } } - /////////////////////////////// poll & commit /////////////////////////////// + /** Caller should ensure that the method is called in the lock {@link #acquireWriteLock()}. */ + private void closeProviders() throws IoTDBConnectionException { + for (final SubscriptionProvider provider : getAllProviders()) { + provider.close(); + } + subscriptionProviders.clear(); + } + + /** Caller should ensure that the method is called in the lock {@link #acquireWriteLock()}. */ + void addProvider(final int dataNodeId, final SubscriptionProvider provider) { + // the subscription provider is opened + LOGGER.info("add new subscription provider {}", provider); + subscriptionProviders.put(dataNodeId, provider); + } + + /** Caller should ensure that the method is called in the lock {@link #acquireWriteLock()}. */ + void closeAndRemoveProvider(final int dataNodeId) throws IoTDBConnectionException { + if (!containsProvider(dataNodeId)) { + return; + } + final SubscriptionProvider provider = subscriptionProviders.get(dataNodeId); + try { + provider.close(); + } finally { + LOGGER.info("close and remove stale subscription provider {}", provider); + subscriptionProviders.remove(dataNodeId); + } + } + + /** Caller should ensure that the method is called in the lock {@link #acquireReadLock()}. */ + boolean hasNoProviders() { + return subscriptionProviders.isEmpty(); + } + + /** Caller should ensure that the method is called in the lock {@link #acquireReadLock()}. */ + boolean containsProvider(final int dataNodeId) { + return subscriptionProviders.containsKey(dataNodeId); + } + + /** Caller should ensure that the method is called in the lock {@link #acquireReadLock()}. */ + List getAllAvailableProviders() { + return subscriptionProviders.values().stream() + .filter(SubscriptionProvider::isAvailable) + .collect(Collectors.toList()); + } + + /** Caller should ensure that the method is called in the lock {@link #acquireReadLock()}. */ + List getAllProviders() { + return new ArrayList<>(subscriptionProviders.values()); + } + + /** Caller should ensure that the method is called in the lock {@link #acquireReadLock()}. */ + SubscriptionProvider getProvider(final int dataNodeId) { + return containsProvider(dataNodeId) ? subscriptionProviders.get(dataNodeId) : null; + } + + /////////////////////////////// poll /////////////////////////////// protected List poll(Set topicNames, long timeoutMs) throws TException, IOException, StatementExecutionException, IoTDBConnectionException { @@ -820,6 +876,8 @@ private List pollTsFileInternal( } } + /////////////////////////////// commit sync /////////////////////////////// + protected void commitSync(Iterable messages) throws TException, IOException, StatementExecutionException, IoTDBConnectionException { Map> dataNodeIdToSubscriptionCommitContexts = @@ -835,12 +893,31 @@ protected void commitSync(Iterable messages) } } + private void commitSyncInternal( + int dataNodeId, List subscriptionCommitContexts) + throws TException, IOException, StatementExecutionException, IoTDBConnectionException { + acquireReadLock(); + try { + final SubscriptionProvider provider = getProvider(dataNodeId); + if (Objects.isNull(provider) || !provider.isAvailable()) { + throw new IoTDBConnectionException( + String.format( + "something unexpected happened when commit messages to subscription provider with data node id %s, the subscription provider may be unavailable or not existed", + dataNodeId)); + } + provider.getSessionConnection().commitSync(subscriptionCommitContexts); + } finally { + releaseReadLock(); + } + } + + /////////////////////////////// commit async /////////////////////////////// + protected void commitAsync(Iterable messages) { commitAsync(messages, new AsyncCommitCallback() {}); } protected void commitAsync(Iterable messages, AsyncCommitCallback callback) { - // Initiate executor if needed if (asyncCommitExecutor == null) { synchronized (this) { @@ -871,82 +948,6 @@ protected void commitAsync(Iterable messages, AsyncCommitCa asyncCommitExecutor.submit(new AsyncCommitWorker(messages, callback)); } - /////////////////////////////// utility /////////////////////////////// - - private void commitSyncInternal( - int dataNodeId, List subscriptionCommitContexts) - throws TException, IOException, StatementExecutionException, IoTDBConnectionException { - acquireReadLock(); - try { - final SubscriptionProvider provider = getProvider(dataNodeId); - if (Objects.isNull(provider) || !provider.isAvailable()) { - throw new IoTDBConnectionException( - String.format( - "something unexpected happened when commit messages to subscription provider with data node id %s, the subscription provider may be unavailable or not existed", - dataNodeId)); - } - provider.getSessionConnection().commitSync(subscriptionCommitContexts); - } finally { - releaseReadLock(); - } - } - - /** Caller should ensure that the method is called in the lock {@link #acquireWriteLock()}. */ - private void closeProviders() throws IoTDBConnectionException { - for (final SubscriptionProvider provider : getAllProviders()) { - provider.close(); - } - subscriptionProviders.clear(); - } - - /** Caller should ensure that the method is called in the lock {@link #acquireWriteLock()}. */ - void addProvider(final int dataNodeId, final SubscriptionProvider provider) { - // the subscription provider is opened - LOGGER.info("add new subscription provider {}", provider); - subscriptionProviders.put(dataNodeId, provider); - } - - /** Caller should ensure that the method is called in the lock {@link #acquireWriteLock()}. */ - void closeAndRemoveProvider(final int dataNodeId) throws IoTDBConnectionException { - if (!containsProvider(dataNodeId)) { - return; - } - final SubscriptionProvider provider = subscriptionProviders.get(dataNodeId); - try { - provider.close(); - } finally { - LOGGER.info("close and remove stale subscription provider {}", provider); - subscriptionProviders.remove(dataNodeId); - } - } - - /** Caller should ensure that the method is called in the lock {@link #acquireReadLock()}. */ - boolean hasNoProviders() { - return subscriptionProviders.isEmpty(); - } - - /** Caller should ensure that the method is called in the lock {@link #acquireReadLock()}. */ - boolean containsProvider(final int dataNodeId) { - return subscriptionProviders.containsKey(dataNodeId); - } - - /** Caller should ensure that the method is called in the lock {@link #acquireReadLock()}. */ - List getAllAvailableProviders() { - return subscriptionProviders.values().stream() - .filter(SubscriptionProvider::isAvailable) - .collect(Collectors.toList()); - } - - /** Caller should ensure that the method is called in the lock {@link #acquireReadLock()}. */ - List getAllProviders() { - return new ArrayList<>(subscriptionProviders.values()); - } - - /** Caller should ensure that the method is called in the lock {@link #acquireReadLock()}. */ - SubscriptionProvider getProvider(final int dataNodeId) { - return containsProvider(dataNodeId) ? subscriptionProviders.get(dataNodeId) : null; - } - /////////////////////////////// redirection /////////////////////////////// /** Caller should ensure that the method is called in the lock {@link #acquireReadLock()}. */ @@ -1083,6 +1084,8 @@ public Builder tsFileBaseDir(String tsFileBaseDir) { public abstract SubscriptionPushConsumer buildPushConsumer(); } + /////////////////////////////// commit async worker /////////////////////////////// + class AsyncCommitWorker implements Runnable { private final Iterable messages; private final AsyncCommitCallback callback; diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java index 10a7b0863f91..5df03bebc1d1 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java @@ -23,6 +23,7 @@ import org.apache.iotdb.rpc.StatementExecutionException; import org.apache.iotdb.rpc.subscription.config.ConsumerConstant; import org.apache.iotdb.rpc.subscription.exception.SubscriptionException; +import org.apache.iotdb.session.subscription.SubscriptionConsumer.Builder; import org.apache.thrift.TException; import org.slf4j.Logger; @@ -62,7 +63,7 @@ boolean isClosed() { /////////////////////////////// ctor /////////////////////////////// - public SubscriptionPullConsumer(SubscriptionPullConsumer.Builder builder) { + protected SubscriptionPullConsumer(SubscriptionPullConsumer.Builder builder) { super(builder); this.autoCommit = builder.autoCommit; @@ -215,7 +216,7 @@ private void launchAutoCommitWorker() { return t; }); autoCommitWorkerExecutor.scheduleAtFixedRate( - new PullConsumerAutoCommitWorker(this), 0, autoCommitIntervalMs, TimeUnit.MILLISECONDS); + new AutoCommitWorker(), 0, autoCommitIntervalMs, TimeUnit.MILLISECONDS); } private void shutdownAutoCommitWorker() { @@ -234,14 +235,6 @@ private void commitAllUncommittedMessages() { } } - long getAutoCommitIntervalMs() { - return autoCommitIntervalMs; - } - - SortedMap> getUncommittedMessages() { - return uncommittedMessages; - } - /////////////////////////////// builder /////////////////////////////// public static class Builder extends SubscriptionConsumer.Builder { @@ -321,4 +314,31 @@ public SubscriptionPushConsumer buildPushConsumer() { "SubscriptionPullConsumer.Builder do not support build push consumer."); } } + + /////////////////////////////// auto commit worker /////////////////////////////// + + class AutoCommitWorker implements Runnable { + @Override + public void run() { + if (isClosed()) { + return; + } + + long currentTimestamp = System.currentTimeMillis(); + long index = currentTimestamp / autoCommitIntervalMs; + if (currentTimestamp % autoCommitIntervalMs == 0) { + index -= 1; + } + + for (Map.Entry> entry : + uncommittedMessages.headMap(index).entrySet()) { + try { + commitSync(entry.getValue()); + uncommittedMessages.remove(entry.getKey()); + } catch (final Exception e) { + LOGGER.warn("something unexpected happened when auto commit messages...", e); + } + } + } + } } diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPushConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPushConsumer.java index 8355d90a6c14..720065dbcba0 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPushConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPushConsumer.java @@ -44,7 +44,7 @@ public class SubscriptionPushConsumer extends SubscriptionConsumer { private final AckStrategy ackStrategy; private final ConsumeListener consumeListener; - private ScheduledExecutorService workerExecutor; + private ScheduledExecutorService autoPollWorkerExecutor; private final AtomicBoolean isClosed = new AtomicBoolean(true); @@ -96,7 +96,7 @@ public synchronized void close() throws IoTDBConnectionException { } try { - shutdownWorker(); + shutdownAutoPollWorker(); super.close(); } finally { isClosed.set(true); @@ -108,11 +108,11 @@ boolean isClosed() { return isClosed.get(); } - /////////////////////////////// auto poll worker /////////////////////////////// + /////////////////////////////// auto poll /////////////////////////////// @SuppressWarnings("unsafeThreadSchedule") private void launchAutoPollWorker() { - workerExecutor = + autoPollWorkerExecutor = Executors.newSingleThreadScheduledExecutor( r -> { Thread t = @@ -125,16 +125,16 @@ private void launchAutoPollWorker() { } return t; }); - workerExecutor.scheduleAtFixedRate( - new PushConsumerWorker(), + autoPollWorkerExecutor.scheduleAtFixedRate( + new AutoPollWorker(), 0, ConsumerConstant.PUSH_CONSUMER_AUTO_POLL_INTERVAL_MS, TimeUnit.MILLISECONDS); } - private void shutdownWorker() { - workerExecutor.shutdown(); - workerExecutor = null; + private void shutdownAutoPollWorker() { + autoPollWorkerExecutor.shutdown(); + autoPollWorkerExecutor = null; } /////////////////////////////// builder /////////////////////////////// @@ -174,6 +174,21 @@ public SubscriptionPushConsumer.Builder consumerGroupId(String consumerGroupId) return this; } + public SubscriptionPushConsumer.Builder heartbeatIntervalMs(long heartbeatIntervalMs) { + super.heartbeatIntervalMs(heartbeatIntervalMs); + return this; + } + + public SubscriptionPushConsumer.Builder endpointsSyncIntervalMs(long endpointsSyncIntervalMs) { + super.endpointsSyncIntervalMs(endpointsSyncIntervalMs); + return this; + } + + public SubscriptionPushConsumer.Builder tsFileBaseDir(String tsFileBaseDir) { + this.tsFileBaseDir = tsFileBaseDir; + return this; + } + public SubscriptionPushConsumer.Builder ackStrategy(AckStrategy ackStrategy) { this.ackStrategy = ackStrategy; return this; @@ -196,7 +211,9 @@ public SubscriptionPushConsumer buildPushConsumer() { } } - class PushConsumerWorker implements Runnable { + /////////////////////////////// auto poll worker /////////////////////////////// + + class AutoPollWorker implements Runnable { @Override public void run() { if (isClosed()) { From c9724177265a052ac41ac42655ba7d1f2ba18b6d Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Fri, 19 Apr 2024 15:16:38 +0800 Subject: [PATCH 36/93] improve --- .../subscription/SubscriptionConsumer.java | 96 +++++++++---------- 1 file changed, 48 insertions(+), 48 deletions(-) diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java index fb8fc7f30599..5cb626910799 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java @@ -77,8 +77,6 @@ public abstract class SubscriptionConsumer implements AutoCloseable { private static final IoTDBConnectionException NO_PROVIDERS_EXCEPTION = new IoTDBConnectionException("Cluster has no available subscription providers to connect"); - private static final int ON_THE_FLY_TS_FILE_RETRY_LIMIT = 3; - private final List initialEndpoints; private final String username; @@ -111,23 +109,14 @@ public String getConsumerGroupId() { return consumerGroupId; } - /////////////////////////////// object /////////////////////////////// - - @Override - public String toString() { - return "SubscriptionConsumer{consumerId=" - + consumerId - + ", consumerGroupId=" - + consumerGroupId - + "}"; - } - /////////////////////////////// tsfile /////////////////////////////// - protected final Map topicNameToOnTheFlyTsFileInfo = + private static final int ON_THE_FLY_TS_FILE_RETRY_LIMIT = 3; + + private final Map topicNameToOnTheFlyTsFileInfo = new ConcurrentHashMap<>(); - protected static class OnTheFlyTsFileInfo { + private static class OnTheFlyTsFileInfo { SubscriptionCommitContext commitContext; File file; @@ -165,7 +154,34 @@ public String toString() { } } - protected OnTheFlyTsFileInfo getOnTheFlyTsFileInfo(String topicName) { + private Path getTsFileDir(final String topicName) throws IOException { + final Path dirPath = + Paths.get(tsFileBaseDir).resolve(consumerGroupId).resolve(consumerId).resolve(topicName); + Files.createDirectories(dirPath); + return dirPath; + } + + private OnTheFlyTsFileInfo createOnTheFlyTsFileInfo( + SubscriptionCommitContext commitContext, String fileName) { + try { + final String topicName = commitContext.getTopicName(); + final Path filePath = getTsFileDir(topicName).resolve(fileName); + + Files.createFile(filePath); + final File file = filePath.toFile(); + final RandomAccessFile fileWriter = new RandomAccessFile(file, "rw"); + + final OnTheFlyTsFileInfo info = new OnTheFlyTsFileInfo(commitContext, file, fileWriter); + topicNameToOnTheFlyTsFileInfo.put(topicName, info); + LOGGER.info("consumer {} create on the fly tsfile info {}", this, info); + return info; + } catch (final IOException e) { + LOGGER.warn(e.getMessage()); + return null; + } + } + + private OnTheFlyTsFileInfo getOnTheFlyTsFileInfo(String topicName) { final OnTheFlyTsFileInfo info = topicNameToOnTheFlyTsFileInfo.get(topicName); if (Objects.isNull(info)) { return null; @@ -184,7 +200,7 @@ protected OnTheFlyTsFileInfo getOnTheFlyTsFileInfo(String topicName) { return info; } - protected void removeOnTheFlyTsFileInfo(String topicName) { + private void removeOnTheFlyTsFileInfo(String topicName) { final OnTheFlyTsFileInfo info = topicNameToOnTheFlyTsFileInfo.get(topicName); if (Objects.isNull(info)) { return; @@ -200,7 +216,7 @@ protected void removeOnTheFlyTsFileInfo(String topicName) { topicNameToOnTheFlyTsFileInfo.remove(topicName); } - protected void increaseOnTheFlyTsFileInfoRetryCountOrRemove(String topicName) { + private void increaseOnTheFlyTsFileInfoRetryCountOrRemove(String topicName) { final OnTheFlyTsFileInfo info = topicNameToOnTheFlyTsFileInfo.get(topicName); if (Objects.isNull(info)) { return; @@ -211,33 +227,6 @@ protected void increaseOnTheFlyTsFileInfoRetryCountOrRemove(String topicName) { } } - protected OnTheFlyTsFileInfo createOnTheFlyTsFileInfo( - SubscriptionCommitContext commitContext, String fileName) { - try { - final String topicName = commitContext.getTopicName(); - final Path filePath = getTsFileDir(topicName).resolve(fileName); - - Files.createFile(filePath); - final File file = filePath.toFile(); - final RandomAccessFile fileWriter = new RandomAccessFile(file, "rw"); - - final OnTheFlyTsFileInfo info = new OnTheFlyTsFileInfo(commitContext, file, fileWriter); - topicNameToOnTheFlyTsFileInfo.put(topicName, info); - LOGGER.info("consumer {} create on the fly tsfile info {}", this, info); - return info; - } catch (final IOException e) { - LOGGER.warn(e.getMessage()); - return null; - } - } - - public Path getTsFileDir(final String topicName) throws IOException { - final Path dirPath = - Paths.get(tsFileBaseDir).resolve(consumerGroupId).resolve(consumerId).resolve(topicName); - Files.createDirectories(dirPath); - return dirPath; - } - /////////////////////////////// ctor /////////////////////////////// protected SubscriptionConsumer(Builder builder) { @@ -951,7 +940,7 @@ protected void commitAsync(Iterable messages, AsyncCommitCa /////////////////////////////// redirection /////////////////////////////// /** Caller should ensure that the method is called in the lock {@link #acquireReadLock()}. */ - public void subscribeWithRedirection(final Set topicNames) + private void subscribeWithRedirection(final Set topicNames) throws IoTDBConnectionException { for (final SubscriptionProvider provider : getAllAvailableProviders()) { try { @@ -969,7 +958,7 @@ public void subscribeWithRedirection(final Set topicNames) } /** Caller should ensure that the method is called in the lock {@link #acquireReadLock()}. */ - public void unsubscribeWithRedirection(final Set topicNames) + private void unsubscribeWithRedirection(final Set topicNames) throws IoTDBConnectionException { for (final SubscriptionProvider provider : getAllAvailableProviders()) { try { @@ -987,7 +976,7 @@ public void unsubscribeWithRedirection(final Set topicNames) } /** Caller should ensure that the method is called in the lock {@link #acquireReadLock()}. */ - public Map fetchAllEndPointsWithRedirection() + Map fetchAllEndPointsWithRedirection() throws IoTDBConnectionException { Map endPoints = null; for (final SubscriptionProvider provider : getAllAvailableProviders()) { @@ -1109,4 +1098,15 @@ public void run() { } } } + + /////////////////////////////// object /////////////////////////////// + + @Override + public String toString() { + return "SubscriptionConsumer{consumerId=" + + consumerId + + ", consumerGroupId=" + + consumerGroupId + + "}"; + } } From 1614c9b047d52959218f3913aa7055ff520e934e Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Fri, 19 Apr 2024 15:17:49 +0800 Subject: [PATCH 37/93] spotless --- .../iotdb/subscription/it/local/IoTDBSubscriptionBasicIT.java | 2 +- .../iotdb/session/subscription/SubscriptionConsumer.java | 3 +-- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionBasicIT.java b/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionBasicIT.java index 390ade35613d..e9b1d1f07dda 100644 --- a/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionBasicIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionBasicIT.java @@ -19,7 +19,6 @@ package org.apache.iotdb.subscription.it.local; -import java.time.Duration; import org.apache.iotdb.isession.ISession; import org.apache.iotdb.it.env.EnvFactory; import org.apache.iotdb.it.framework.IoTDBTestRunner; @@ -51,6 +50,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.time.Duration; import java.util.Collections; import java.util.List; import java.util.Properties; diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java index 5cb626910799..367c153086a7 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java @@ -976,8 +976,7 @@ private void unsubscribeWithRedirection(final Set topicNames) } /** Caller should ensure that the method is called in the lock {@link #acquireReadLock()}. */ - Map fetchAllEndPointsWithRedirection() - throws IoTDBConnectionException { + Map fetchAllEndPointsWithRedirection() throws IoTDBConnectionException { Map endPoints = null; for (final SubscriptionProvider provider : getAllAvailableProviders()) { try { From 7fd091a1e3f7f2d83ab456897ec16418cd5e920b Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Fri, 19 Apr 2024 15:19:55 +0800 Subject: [PATCH 38/93] improve --- .../it/local/IoTDBSubscriptionBasicIT.java | 35 +++++++++++-------- 1 file changed, 20 insertions(+), 15 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionBasicIT.java b/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionBasicIT.java index e9b1d1f07dda..920bb88cc739 100644 --- a/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionBasicIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionBasicIT.java @@ -352,9 +352,10 @@ public void onFailure(Throwable e) { try { // Keep retrying if there are execution failures Awaitility.await() - .pollDelay(1, TimeUnit.SECONDS) - .pollInterval(1, TimeUnit.SECONDS) - .atMost(120, TimeUnit.SECONDS) + .pollDelay(IoTDBSubscriptionITConstant.AWAITILITY_POLL_DELAY_SECOND, TimeUnit.SECONDS) + .pollInterval( + IoTDBSubscriptionITConstant.AWAITILITY_POLL_INTERVAL_SECOND, TimeUnit.SECONDS) + .atMost(IoTDBSubscriptionITConstant.AWAITILITY_AT_MOST_SECOND, TimeUnit.SECONDS) .untilAsserted(() -> Assert.assertEquals(100, rowCount.get())); Assert.assertTrue(commitSuccessCount.get() > lastCommitSuccessCount.get()); Assert.assertEquals(0, commitFailureCount.get()); @@ -382,9 +383,10 @@ public void onFailure(Throwable e) { try { // Keep retrying if there are execution failures Awaitility.await() - .pollDelay(1, TimeUnit.SECONDS) - .pollInterval(1, TimeUnit.SECONDS) - .atMost(120, TimeUnit.SECONDS) + .pollDelay(IoTDBSubscriptionITConstant.AWAITILITY_POLL_DELAY_SECOND, TimeUnit.SECONDS) + .pollInterval( + IoTDBSubscriptionITConstant.AWAITILITY_POLL_INTERVAL_SECOND, TimeUnit.SECONDS) + .atMost(IoTDBSubscriptionITConstant.AWAITILITY_AT_MOST_SECOND, TimeUnit.SECONDS) .untilAsserted(() -> Assert.assertEquals(200, rowCount.get())); Assert.assertTrue(commitSuccessCount.get() > lastCommitSuccessCount.get()); Assert.assertEquals(0, commitFailureCount.get()); @@ -451,9 +453,10 @@ public void testBasicPushConsumer() { // The push consumer should automatically poll 10 rows of data by 1 onReceive() Awaitility.await() - .pollDelay(1, TimeUnit.SECONDS) - .pollInterval(1, TimeUnit.SECONDS) - .atMost(10, TimeUnit.SECONDS) + .pollDelay(IoTDBSubscriptionITConstant.AWAITILITY_POLL_DELAY_SECOND, TimeUnit.SECONDS) + .pollInterval( + IoTDBSubscriptionITConstant.AWAITILITY_POLL_INTERVAL_SECOND, TimeUnit.SECONDS) + .atMost(IoTDBSubscriptionITConstant.AWAITILITY_AT_MOST_SECOND, TimeUnit.SECONDS) .untilAsserted( () -> { Assert.assertEquals(10, rowCount.get()); @@ -472,9 +475,10 @@ public void testBasicPushConsumer() { } Awaitility.await() - .pollDelay(1, TimeUnit.SECONDS) - .pollInterval(1, TimeUnit.SECONDS) - .atMost(10, TimeUnit.SECONDS) + .pollDelay(IoTDBSubscriptionITConstant.AWAITILITY_POLL_DELAY_SECOND, TimeUnit.SECONDS) + .pollInterval( + IoTDBSubscriptionITConstant.AWAITILITY_POLL_INTERVAL_SECOND, TimeUnit.SECONDS) + .atMost(IoTDBSubscriptionITConstant.AWAITILITY_AT_MOST_SECOND, TimeUnit.SECONDS) .untilAsserted( () -> { Assert.assertEquals(20, rowCount.get()); @@ -492,9 +496,10 @@ public void testBasicPushConsumer() { } Awaitility.await() - .pollDelay(1, TimeUnit.SECONDS) - .pollInterval(1, TimeUnit.SECONDS) - .atMost(10, TimeUnit.SECONDS) + .pollDelay(IoTDBSubscriptionITConstant.AWAITILITY_POLL_DELAY_SECOND, TimeUnit.SECONDS) + .pollInterval( + IoTDBSubscriptionITConstant.AWAITILITY_POLL_INTERVAL_SECOND, TimeUnit.SECONDS) + .atMost(IoTDBSubscriptionITConstant.AWAITILITY_AT_MOST_SECOND, TimeUnit.SECONDS) .untilAsserted( () -> { Assert.assertEquals(30, rowCount.get()); From 974c64c927289c1a78a54fcabff89ec7ecc32081 Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Fri, 19 Apr 2024 15:21:50 +0800 Subject: [PATCH 39/93] spotless --- .../iotdb/session/subscription/SubscriptionPullConsumer.java | 1 - 1 file changed, 1 deletion(-) diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java index 5df03bebc1d1..58e2171352ae 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java @@ -23,7 +23,6 @@ import org.apache.iotdb.rpc.StatementExecutionException; import org.apache.iotdb.rpc.subscription.config.ConsumerConstant; import org.apache.iotdb.rpc.subscription.exception.SubscriptionException; -import org.apache.iotdb.session.subscription.SubscriptionConsumer.Builder; import org.apache.thrift.TException; import org.slf4j.Logger; From 0c3b6d36f5f60eda6166e7a531d6979a6949ea40 Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Fri, 19 Apr 2024 15:31:49 +0800 Subject: [PATCH 40/93] rename --- .../subscription/SubscriptionConsumer.java | 57 ++++++++++--------- 1 file changed, 29 insertions(+), 28 deletions(-) diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java index 367c153086a7..f2d54cc6d0db 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java @@ -113,17 +113,17 @@ public String getConsumerGroupId() { private static final int ON_THE_FLY_TS_FILE_RETRY_LIMIT = 3; - private final Map topicNameToOnTheFlyTsFileInfo = + private final Map topicNameToSubscriptionTsFileInfo = new ConcurrentHashMap<>(); - private static class OnTheFlyTsFileInfo { + private static class SubscriptionTsFileInfo { SubscriptionCommitContext commitContext; File file; RandomAccessFile fileWriter; int retryCount; - OnTheFlyTsFileInfo( + SubscriptionTsFileInfo( SubscriptionCommitContext commitContext, File file, RandomAccessFile fileWriter) { this.commitContext = commitContext; this.file = file; @@ -143,7 +143,7 @@ boolean increaseRetryCountAndCheckIfExceedRetryLimit() { @Override public String toString() { - return "OnTheFlyTsFileInfo{" + return "SubscriptionTsFileInfo{" + "commitContext=" + commitContext + ", file=" @@ -161,7 +161,7 @@ private Path getTsFileDir(final String topicName) throws IOException { return dirPath; } - private OnTheFlyTsFileInfo createOnTheFlyTsFileInfo( + private SubscriptionTsFileInfo createSubscriptionTsFileInfoTsFileInfo( SubscriptionCommitContext commitContext, String fileName) { try { final String topicName = commitContext.getTopicName(); @@ -171,9 +171,10 @@ private OnTheFlyTsFileInfo createOnTheFlyTsFileInfo( final File file = filePath.toFile(); final RandomAccessFile fileWriter = new RandomAccessFile(file, "rw"); - final OnTheFlyTsFileInfo info = new OnTheFlyTsFileInfo(commitContext, file, fileWriter); - topicNameToOnTheFlyTsFileInfo.put(topicName, info); - LOGGER.info("consumer {} create on the fly tsfile info {}", this, info); + final SubscriptionTsFileInfo info = + new SubscriptionTsFileInfo(commitContext, file, fileWriter); + topicNameToSubscriptionTsFileInfo.put(topicName, info); + LOGGER.info("consumer {} create subscription TsFile info {}", this, info); return info; } catch (final IOException e) { LOGGER.warn(e.getMessage()); @@ -181,8 +182,8 @@ private OnTheFlyTsFileInfo createOnTheFlyTsFileInfo( } } - private OnTheFlyTsFileInfo getOnTheFlyTsFileInfo(String topicName) { - final OnTheFlyTsFileInfo info = topicNameToOnTheFlyTsFileInfo.get(topicName); + private SubscriptionTsFileInfo getSubscriptionTsFileInfoTsFileInfo(String topicName) { + final SubscriptionTsFileInfo info = topicNameToSubscriptionTsFileInfo.get(topicName); if (Objects.isNull(info)) { return null; } @@ -193,15 +194,15 @@ private OnTheFlyTsFileInfo getOnTheFlyTsFileInfo(String topicName) { } catch (final IOException e) { LOGGER.warn(e.getMessage()); } - topicNameToOnTheFlyTsFileInfo.remove(topicName); + topicNameToSubscriptionTsFileInfo.remove(topicName); return null; } return info; } - private void removeOnTheFlyTsFileInfo(String topicName) { - final OnTheFlyTsFileInfo info = topicNameToOnTheFlyTsFileInfo.get(topicName); + private void removeSubscriptionTsFileInfo(String topicName) { + final SubscriptionTsFileInfo info = topicNameToSubscriptionTsFileInfo.get(topicName); if (Objects.isNull(info)) { return; } @@ -212,18 +213,18 @@ private void removeOnTheFlyTsFileInfo(String topicName) { LOGGER.warn(e.getMessage()); } - LOGGER.info("consumer {} remove on the fly tsfile info {}", this, info); - topicNameToOnTheFlyTsFileInfo.remove(topicName); + LOGGER.info("consumer {} remove subscription TsFile info {}", this, info); + topicNameToSubscriptionTsFileInfo.remove(topicName); } - private void increaseOnTheFlyTsFileInfoRetryCountOrRemove(String topicName) { - final OnTheFlyTsFileInfo info = topicNameToOnTheFlyTsFileInfo.get(topicName); + private void increaseSubscriptionTsFileInfoRetryCountOrRemove(String topicName) { + final SubscriptionTsFileInfo info = topicNameToSubscriptionTsFileInfo.get(topicName); if (Objects.isNull(info)) { return; } if (info.increaseRetryCountAndCheckIfExceedRetryLimit()) { - removeOnTheFlyTsFileInfo(topicName); + removeSubscriptionTsFileInfo(topicName); } } @@ -591,8 +592,8 @@ protected List poll(Set topicNames, long timeoutMs) final List messages = new ArrayList<>(); // poll on the fly tsfile - for (final OnTheFlyTsFileInfo info : - topicNameToOnTheFlyTsFileInfo.values().stream() + for (final SubscriptionTsFileInfo info : + topicNameToSubscriptionTsFileInfo.values().stream() .filter( info -> { if (topicNames.isEmpty()) { @@ -640,15 +641,15 @@ private Optional pollTsFile( final Pair messageWithRetryable = pollTsFileInternal(commitContext, fileName, timeoutMs); if (Objects.nonNull(messageWithRetryable.getLeft())) { - removeOnTheFlyTsFileInfo(commitContext.getTopicName()); + removeSubscriptionTsFileInfo(commitContext.getTopicName()); return Optional.of(messageWithRetryable.getLeft()); } if (!messageWithRetryable.getRight()) { // non-retryable - removeOnTheFlyTsFileInfo(commitContext.getTopicName()); + removeSubscriptionTsFileInfo(commitContext.getTopicName()); } else { // retryable - increaseOnTheFlyTsFileInfoRetryCountOrRemove(commitContext.getTopicName()); + increaseSubscriptionTsFileInfoRetryCountOrRemove(commitContext.getTopicName()); } } catch (IOException e) { LOGGER.warn( @@ -658,7 +659,7 @@ private Optional pollTsFile( commitContext, e.getMessage()); // assume retryable - increaseOnTheFlyTsFileInfoRetryCountOrRemove(commitContext.getTopicName()); + increaseSubscriptionTsFileInfoRetryCountOrRemove(commitContext.getTopicName()); } catch (TException | IoTDBConnectionException | StatementExecutionException e) { LOGGER.warn( "Exception occurred when {} polling TsFile {} with commit context {}: {}", @@ -667,7 +668,7 @@ private Optional pollTsFile( commitContext, e.getMessage()); // assume non-retryable - removeOnTheFlyTsFileInfo(commitContext.getTopicName()); + removeSubscriptionTsFileInfo(commitContext.getTopicName()); } return Optional.empty(); } @@ -678,9 +679,9 @@ private Pair pollTsFileInternal( final int dataNodeId = commitContext.getDataNodeId(); final String topicName = commitContext.getTopicName(); - OnTheFlyTsFileInfo info = getOnTheFlyTsFileInfo(topicName); + SubscriptionTsFileInfo info = getSubscriptionTsFileInfoTsFileInfo(topicName); if (Objects.isNull(info)) { - info = createOnTheFlyTsFileInfo(commitContext, fileName); + info = createSubscriptionTsFileInfoTsFileInfo(commitContext, fileName); } if (Objects.isNull(info)) { return new Pair<>(null, false); @@ -850,7 +851,7 @@ private List pollTsFileInternal( if (Objects.isNull(provider) || !provider.isAvailable()) { throw new IoTDBConnectionException( String.format( - "something unexpected happened when poll tsfile from subscription provider with data node id %s, the subscription provider may be unavailable or not existed", + "something unexpected happened when poll TsFile from subscription provider with data node id %s, the subscription provider may be unavailable or not existed", dataNodeId)); } return provider From d82af5740e0345939c9d57f1f3035207388b8626 Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Fri, 19 Apr 2024 18:14:28 +0800 Subject: [PATCH 41/93] enhance logs --- .../subscription/SubscriptionConsumer.java | 37 ++++---- .../SubscriptionPrefetchingTsFileQueue.java | 84 +++++++++++++++---- 2 files changed, 83 insertions(+), 38 deletions(-) diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java index f2d54cc6d0db..3b8c9419b239 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java @@ -111,7 +111,7 @@ public String getConsumerGroupId() { /////////////////////////////// tsfile /////////////////////////////// - private static final int ON_THE_FLY_TS_FILE_RETRY_LIMIT = 3; + private static final int POLL_TS_FILE_RETRY_LIMIT = 3; private final Map topicNameToSubscriptionTsFileInfo = new ConcurrentHashMap<>(); @@ -138,7 +138,7 @@ String getTopicName() { /** @return {@code true} if exceed retry limit */ boolean increaseRetryCountAndCheckIfExceedRetryLimit() { retryCount++; - return retryCount > ON_THE_FLY_TS_FILE_RETRY_LIMIT; + return retryCount > POLL_TS_FILE_RETRY_LIMIT; } @Override @@ -162,24 +162,18 @@ private Path getTsFileDir(final String topicName) throws IOException { } private SubscriptionTsFileInfo createSubscriptionTsFileInfoTsFileInfo( - SubscriptionCommitContext commitContext, String fileName) { - try { - final String topicName = commitContext.getTopicName(); - final Path filePath = getTsFileDir(topicName).resolve(fileName); - - Files.createFile(filePath); - final File file = filePath.toFile(); - final RandomAccessFile fileWriter = new RandomAccessFile(file, "rw"); - - final SubscriptionTsFileInfo info = - new SubscriptionTsFileInfo(commitContext, file, fileWriter); - topicNameToSubscriptionTsFileInfo.put(topicName, info); - LOGGER.info("consumer {} create subscription TsFile info {}", this, info); - return info; - } catch (final IOException e) { - LOGGER.warn(e.getMessage()); - return null; - } + SubscriptionCommitContext commitContext, String fileName) throws IOException { + final String topicName = commitContext.getTopicName(); + final Path filePath = getTsFileDir(topicName).resolve(fileName); + + Files.createFile(filePath); + final File file = filePath.toFile(); + final RandomAccessFile fileWriter = new RandomAccessFile(file, "rw"); + + final SubscriptionTsFileInfo info = new SubscriptionTsFileInfo(commitContext, file, fileWriter); + topicNameToSubscriptionTsFileInfo.put(topicName, info); + LOGGER.info("consumer {} create subscription TsFile info {}", this, info); + return info; } private SubscriptionTsFileInfo getSubscriptionTsFileInfoTsFileInfo(String topicName) { @@ -683,9 +677,6 @@ private Pair pollTsFileInternal( if (Objects.isNull(info)) { info = createSubscriptionTsFileInfoTsFileInfo(commitContext, fileName); } - if (Objects.isNull(info)) { - return new Pair<>(null, false); - } final File file = info.file; final RandomAccessFile fileWriter = info.fileWriter; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java index fc2d372a8437..e96b43e9cc3e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java @@ -33,6 +33,7 @@ import org.apache.iotdb.rpc.subscription.payload.common.TsFileErrorMessagePayload; import org.apache.iotdb.rpc.subscription.payload.common.TsFileInitMessagePayload; import org.apache.iotdb.rpc.subscription.payload.common.TsFilePieceMessagePayload; +import org.apache.iotdb.rpc.subscription.payload.common.TsFileSealMessagePayload; import org.apache.iotdb.tsfile.utils.Pair; import org.checkerframework.checker.nullness.qual.NonNull; @@ -105,7 +106,18 @@ public SubscriptionTsFileEvent poll(final String consumerId, final SubscriptionP if (Objects.isNull(event)) { final String errorMessage = String.format( - "%s is currently not transferring any tsfile to consumer %s", this, consumerId); + "SubscriptionPrefetchingTsFileQueue %s is currently not transferring any TsFile to consumer %s, file name: %s, writing offset: %s", + this, consumerId, fileName, writingOffset); + LOGGER.warn(errorMessage); + return generateSubscriptionTsFileEventWithErrorMessage(errorMessage, false); + } + + if (event.isCommitted()) { + consumerIdToCurrentEventMap.remove(consumerId); + final String errorMessage = + String.format( + "SubscriptionEvent %s related to TsFile is committed, consumer: %s, writing offset: %s, prefetching queue: %s", + event, consumerId, writingOffset, this); LOGGER.warn(errorMessage); return generateSubscriptionTsFileEventWithErrorMessage(errorMessage, false); } @@ -114,8 +126,8 @@ public SubscriptionTsFileEvent poll(final String consumerId, final SubscriptionP if (!Objects.equals(event.getLastPolledConsumerId(), consumerId)) { final String errorMessage = String.format( - "inconsistent polled consumer id, current is %s, incoming is %s, prefetching queue: %s", - event.getLastPolledConsumerId(), consumerId, this); + "inconsistent polled consumer id, current: %s, incoming: %s, file name: %s, writing offset: %s, prefetching queue: %s", + event.getLastPolledConsumerId(), consumerId, fileName, writingOffset, this); LOGGER.warn(errorMessage); return generateSubscriptionTsFileEventWithErrorMessage(errorMessage, false); } @@ -128,8 +140,12 @@ public SubscriptionTsFileEvent poll(final String consumerId, final SubscriptionP if (!Objects.equals(tsFileInsertionEvent.getTsFile().getName(), fileName)) { final String errorMessage = String.format( - "inconsistent file name, current is %s, incoming is %s, prefetching queue: %s", - tsFileInsertionEvent.getTsFile().getName(), fileName, this); + "inconsistent file name, current: %s, incoming: %s, consumer: %s, writing offset: %s, prefetching queue: %s", + tsFileInsertionEvent.getTsFile().getName(), + fileName, + consumerId, + writingOffset, + this); LOGGER.warn(errorMessage); return generateSubscriptionTsFileEventWithErrorMessage(errorMessage, false); } @@ -147,14 +163,23 @@ public SubscriptionTsFileEvent poll(final String consumerId, final SubscriptionP ((TsFileInitMessagePayload) messagePayload).getFileName(), fileName)) { final String errorMessage = String.format( - "inconsistent file name, current is %s, incoming is %s, prefetching queue: %s", - ((TsFileInitMessagePayload) messagePayload).getFileName(), fileName, this); + "inconsistent file name, current: %s, incoming: %s, consumer: %s, writing offset: %s, prefetching queue: %s", + ((TsFileInitMessagePayload) messagePayload).getFileName(), + fileName, + consumerId, + writingOffset, + this); LOGGER.warn(errorMessage); return generateSubscriptionTsFileEventWithErrorMessage(errorMessage, false); } // check offset if (writingOffset != 0) { - LOGGER.warn("{} reset file {} offset to {}", this, fileName, writingOffset); + LOGGER.warn( + "SubscriptionPrefetchingTsFileQueue {} set TsFile (with event {}) writing offset to {} for consumer {}", + this, + event, + writingOffset, + consumerId); } break; case TS_FILE_PIECE: @@ -163,19 +188,48 @@ public SubscriptionTsFileEvent poll(final String consumerId, final SubscriptionP ((TsFilePieceMessagePayload) messagePayload).getFileName(), fileName)) { final String errorMessage = String.format( - "inconsistent file name, current is %s, incoming is %s, prefetching queue: %s", - ((TsFilePieceMessagePayload) messagePayload).getFileName(), fileName, this); + "inconsistent file name, current: %s, incoming: %s, consumer: %s, writing offset: %s, prefetching queue: %s", + ((TsFilePieceMessagePayload) messagePayload).getFileName(), + fileName, + consumerId, + writingOffset, + this); LOGGER.warn(errorMessage); return generateSubscriptionTsFileEventWithErrorMessage(errorMessage, false); } // check offset if (writingOffset != ((TsFilePieceMessagePayload) messagePayload).getNextWritingOffset()) { - LOGGER.warn("{} reset file {} offset to {}", this, fileName, writingOffset); + LOGGER.warn( + "SubscriptionPrefetchingTsFileQueue {} set TsFile (with event {}) writing offset to {} for consumer {}", + this, + event, + writingOffset, + consumerId); } break; case TS_FILE_SEAL: - LOGGER.warn("{} reset file {} offset to {}", this, fileName, writingOffset); + // check file name + if (!Objects.equals( + ((TsFileSealMessagePayload) messagePayload).getFileName(), fileName)) { + final String errorMessage = + String.format( + "inconsistent file name, current: %s, incoming: %s, consumer: %s, writing offset: %s, prefetching queue: %s", + ((TsFileSealMessagePayload) messagePayload).getFileName(), + fileName, + consumerId, + writingOffset, + this); + LOGGER.warn(errorMessage); + return generateSubscriptionTsFileEventWithErrorMessage(errorMessage, false); + } + + LOGGER.warn( + "SubscriptionPrefetchingTsFileQueue {} set TsFile (with event {}) writing offset to {} after transferring seal signal to consumer {}", + this, + event, + writingOffset, + consumerId); uncommittedEvents.remove(polledMessage.getCommitContext()); break; default: @@ -204,8 +258,8 @@ public SubscriptionTsFileEvent poll(final String consumerId, final SubscriptionP } catch (IOException e) { final String errorMessage = String.format( - "IOException occurred when %s transferring tsfile to consumer %s: %s", - this, consumerId, e.getMessage()); + "IOException occurred when SubscriptionPrefetchingTsFileQueue %s transferring TsFile (with event %s) to consumer %s: %s", + this, event, consumerId, e.getMessage()); LOGGER.warn(errorMessage); // assume retryable return generateSubscriptionTsFileEventWithErrorMessage(errorMessage, true); @@ -247,7 +301,7 @@ private synchronized SubscriptionTsFileEvent getPollableOnTheFlySubscriptionTsFi if (!currentEvent.pollable()) { LOGGER.info( - "{} is currently transferring tsfile (with event {}) to consumer {}.", + "SubscriptionPrefetchingTsFileQueue {} is currently transferring TsFile (with event {}) to consumer {}", this, currentEvent, entry.getKey()); From c2cb55dd04ac4d1d74f7b57000c0c6e05d3daab7 Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Fri, 19 Apr 2024 18:29:23 +0800 Subject: [PATCH 42/93] improve --- .../subscription/event/SubscriptionTsFileEvent.java | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionTsFileEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionTsFileEvent.java index 1326ae9b827b..92f93dd59508 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionTsFileEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionTsFileEvent.java @@ -73,7 +73,10 @@ public void prefetchNext() { try { return generateSubscriptionTsFileEventWithPieceOrSealPayload(0); } catch (IOException e) { - LOGGER.warn(e.getMessage()); + LOGGER.warn( + "IOException occurred when prefetching next SubscriptionTsFileEvent: {}, current SubscriptionTsFileEvent: {}", + e.getMessage(), + this); return null; } case TS_FILE_PIECE: @@ -81,10 +84,14 @@ public void prefetchNext() { return generateSubscriptionTsFileEventWithPieceOrSealPayload( ((TsFilePieceMessagePayload) messagePayload).getNextWritingOffset()); } catch (IOException e) { - LOGGER.warn(e.getMessage()); + LOGGER.warn( + "IOException occurred when prefetching next SubscriptionTsFileEvent: {}, current SubscriptionTsFileEvent: {}", + e.getMessage(), + this); return null; } case TS_FILE_SEAL: + // not need to prefetch return null; default: LOGGER.warn("unexpected message type: {}", messageType); @@ -125,6 +132,7 @@ public void serializeNext() { if (Objects.equals(writingOffset, 0)) { return nextEventWithCommittable; } + // reset next SubscriptionTsFileEvent return null; case TS_FILE_PIECE: if (Objects.equals( @@ -132,6 +140,7 @@ public void serializeNext() { ((TsFilePieceMessagePayload) messagePayload).getNextWritingOffset())) { return nextEventWithCommittable; } + // reset next SubscriptionTsFileEvent return null; case TS_FILE_SEAL: return null; From e494522a404d510512bc1b2816611d5555a49cf2 Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Fri, 19 Apr 2024 22:57:20 +0800 Subject: [PATCH 43/93] improve exception handling --- .../org/apache/iotdb/rpc/TSStatusCode.java | 1 + .../SubscriptionConnectionException.java | 40 +++ .../exception/SubscriptionException.java | 4 +- .../SubscriptionNonRetryableException.java | 40 +++ ...ubscriptionParameterNotValidException.java | 9 +- .../SubscriptionPollTimeOutException.java | 9 +- .../SubscriptionRetryableException.java | 40 +++ .../response/PipeSubscribeHandshakeResp.java | 3 +- .../response/PipeSubscribePollResp.java | 3 +- .../subscription/SubscriptionConsumer.java | 229 +++++++++--------- .../subscription/SubscriptionProvider.java | 138 +++++++++-- .../SubscriptionPullConsumer.java | 6 +- .../SubscriptionSessionConnection.java | 28 +-- .../agent/SubscriptionBrokerAgent.java | 31 ++- .../broker/SubscriptionBroker.java | 23 +- .../broker/SubscriptionPrefetchingQueue.java | 10 +- .../SubscriptionPrefetchingTabletsQueue.java | 23 +- .../receiver/SubscriptionReceiverV1.java | 22 +- 18 files changed, 459 insertions(+), 200 deletions(-) create mode 100644 iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionConnectionException.java create mode 100644 iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionNonRetryableException.java create mode 100644 iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionRetryableException.java diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java index e434852e870d..78db457a054b 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java @@ -239,6 +239,7 @@ public enum TSStatusCode { SUBSCRIPTION_UNSUBSCRIBE_ERROR(1908), SUBSCRIPTION_MISSING_CUSTOMER(1909), SHOW_SUBSCRIPTION_ERROR(1910), + SUBSCRIPTION_SERIALIZATION_ERROR(1911), // Topic CREATE_TOPIC_ERROR(2000), diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionConnectionException.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionConnectionException.java new file mode 100644 index 000000000000..3ebf522e0b84 --- /dev/null +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionConnectionException.java @@ -0,0 +1,40 @@ +/* + * 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.rpc.subscription.exception; + +import java.util.Objects; + +public class SubscriptionConnectionException extends SubscriptionException { + + public SubscriptionConnectionException(String message) { + super(message); + } + + public SubscriptionConnectionException(String message, Throwable cause) { + super(message, cause); + } + + @Override + public boolean equals(Object obj) { + return obj instanceof SubscriptionConnectionException + && Objects.equals(getMessage(), ((SubscriptionConnectionException) obj).getMessage()) + && Objects.equals(getTimeStamp(), ((SubscriptionConnectionException) obj).getTimeStamp()); + } +} diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionException.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionException.java index eb93e29d8c04..f86c71891c6b 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionException.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionException.java @@ -29,8 +29,8 @@ public SubscriptionException(String message) { super(message); } - protected SubscriptionException(String message, long timeStamp) { - super(message, timeStamp); + public SubscriptionException(String message, Throwable cause) { + super(message, cause); } @Override diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionNonRetryableException.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionNonRetryableException.java new file mode 100644 index 000000000000..1ec85c5c6613 --- /dev/null +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionNonRetryableException.java @@ -0,0 +1,40 @@ +/* + * 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.rpc.subscription.exception; + +import java.util.Objects; + +public class SubscriptionNonRetryableException extends SubscriptionException { + + public SubscriptionNonRetryableException(String message) { + super(message); + } + + public SubscriptionNonRetryableException(String message, Throwable cause) { + super(message, cause); + } + + @Override + public boolean equals(Object obj) { + return obj instanceof SubscriptionNonRetryableException + && Objects.equals(getMessage(), ((SubscriptionNonRetryableException) obj).getMessage()) + && Objects.equals(getTimeStamp(), ((SubscriptionNonRetryableException) obj).getTimeStamp()); + } +} diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionParameterNotValidException.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionParameterNotValidException.java index a9992020ee9d..5c903a94d648 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionParameterNotValidException.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionParameterNotValidException.java @@ -27,8 +27,8 @@ public SubscriptionParameterNotValidException(String message) { super(message); } - protected SubscriptionParameterNotValidException(String message, long timeStamp) { - super(message, timeStamp); + public SubscriptionParameterNotValidException(String message, Throwable cause) { + super(message, cause); } @Override @@ -38,9 +38,4 @@ public boolean equals(Object obj) { && Objects.equals( getTimeStamp(), ((SubscriptionParameterNotValidException) obj).getTimeStamp()); } - - @Override - public int hashCode() { - return Objects.hash(getMessage(), getTimeStamp()); - } } diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionPollTimeOutException.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionPollTimeOutException.java index 6d23e477c636..033d975c7e5d 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionPollTimeOutException.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionPollTimeOutException.java @@ -27,8 +27,8 @@ public SubscriptionPollTimeOutException(String message) { super(message); } - protected SubscriptionPollTimeOutException(String message, long timeStamp) { - super(message, timeStamp); + public SubscriptionPollTimeOutException(String message, Throwable cause) { + super(message, cause); } @Override @@ -37,9 +37,4 @@ public boolean equals(Object obj) { && Objects.equals(getMessage(), ((SubscriptionPollTimeOutException) obj).getMessage()) && Objects.equals(getTimeStamp(), ((SubscriptionPollTimeOutException) obj).getTimeStamp()); } - - @Override - public int hashCode() { - return Objects.hash(getMessage(), getTimeStamp()); - } } diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionRetryableException.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionRetryableException.java new file mode 100644 index 000000000000..d0467133c682 --- /dev/null +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionRetryableException.java @@ -0,0 +1,40 @@ +/* + * 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.rpc.subscription.exception; + +import java.util.Objects; + +public class SubscriptionRetryableException extends SubscriptionException { + + public SubscriptionRetryableException(String message) { + super(message); + } + + public SubscriptionRetryableException(String message, Throwable cause) { + super(message, cause); + } + + @Override + public boolean equals(Object obj) { + return obj instanceof SubscriptionRetryableException + && Objects.equals(getMessage(), ((SubscriptionRetryableException) obj).getMessage()) + && Objects.equals(getTimeStamp(), ((SubscriptionRetryableException) obj).getTimeStamp()); + } +} diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribeHandshakeResp.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribeHandshakeResp.java index fbf95914b4b2..b7558a3d972c 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribeHandshakeResp.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribeHandshakeResp.java @@ -62,7 +62,8 @@ public static PipeSubscribeHandshakeResp toTPipeSubscribeResp(TSStatus status, i Collections.singletonList( ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size())); } catch (IOException e) { - resp.status = RpcUtils.getStatus(TSStatusCode.SUBSCRIPTION_HANDSHAKE_ERROR, e.getMessage()); + resp.status = + RpcUtils.getStatus(TSStatusCode.SUBSCRIPTION_SERIALIZATION_ERROR, e.getMessage()); return resp; } diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollResp.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollResp.java index 335c8f8ca8f9..15154da36eb1 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollResp.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollResp.java @@ -64,7 +64,8 @@ public static PipeSubscribePollResp toTPipeSubscribeResp( } } } catch (IOException e) { - resp.status = RpcUtils.getStatus(TSStatusCode.SUBSCRIPTION_POLL_ERROR, e.getMessage()); + resp.status = + RpcUtils.getStatus(TSStatusCode.SUBSCRIPTION_SERIALIZATION_ERROR, e.getMessage()); } return resp; diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java index 3b8c9419b239..245de436b25e 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java @@ -24,6 +24,10 @@ import org.apache.iotdb.rpc.IoTDBConnectionException; import org.apache.iotdb.rpc.StatementExecutionException; import org.apache.iotdb.rpc.subscription.config.ConsumerConstant; +import org.apache.iotdb.rpc.subscription.exception.SubscriptionConnectionException; +import org.apache.iotdb.rpc.subscription.exception.SubscriptionException; +import org.apache.iotdb.rpc.subscription.exception.SubscriptionNonRetryableException; +import org.apache.iotdb.rpc.subscription.exception.SubscriptionRetryableException; import org.apache.iotdb.rpc.subscription.payload.common.PollMessagePayload; import org.apache.iotdb.rpc.subscription.payload.common.PollTsFileMessagePayload; import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionCommitContext; @@ -38,7 +42,6 @@ import org.apache.iotdb.rpc.subscription.payload.common.TsFilePieceMessagePayload; import org.apache.iotdb.rpc.subscription.payload.common.TsFileSealMessagePayload; import org.apache.iotdb.session.util.SessionUtils; -import org.apache.iotdb.tsfile.utils.Pair; import org.apache.thrift.TException; import org.slf4j.Logger; @@ -161,6 +164,31 @@ private Path getTsFileDir(final String topicName) throws IOException { return dirPath; } + private SubscriptionTsFileInfo getSubscriptionTsFileInfoTsFileInfo( + SubscriptionCommitContext commitContext, String fileName) throws IOException { + final String topicName = commitContext.getTopicName(); + SubscriptionTsFileInfo info = topicNameToSubscriptionTsFileInfo.get(topicName); + if (Objects.isNull(info)) { + info = createSubscriptionTsFileInfoTsFileInfo(commitContext, fileName); + } else { + if (!info.file.exists()) { + try { + info.fileWriter.close(); + } catch (final IOException e) { + LOGGER.warn(e.getMessage()); + } + LOGGER.info( + "consumer {} remove subscription TsFile info {} because the file does not exist", + this, + info); + topicNameToSubscriptionTsFileInfo.remove(topicName); + info = createSubscriptionTsFileInfoTsFileInfo(commitContext, fileName); + } + } + + return info; + } + private SubscriptionTsFileInfo createSubscriptionTsFileInfoTsFileInfo( SubscriptionCommitContext commitContext, String fileName) throws IOException { final String topicName = commitContext.getTopicName(); @@ -176,25 +204,6 @@ private SubscriptionTsFileInfo createSubscriptionTsFileInfoTsFileInfo( return info; } - private SubscriptionTsFileInfo getSubscriptionTsFileInfoTsFileInfo(String topicName) { - final SubscriptionTsFileInfo info = topicNameToSubscriptionTsFileInfo.get(topicName); - if (Objects.isNull(info)) { - return null; - } - - if (!info.file.exists()) { - try { - info.fileWriter.close(); - } catch (final IOException e) { - LOGGER.warn(e.getMessage()); - } - topicNameToSubscriptionTsFileInfo.remove(topicName); - return null; - } - - return info; - } - private void removeSubscriptionTsFileInfo(String topicName) { final SubscriptionTsFileInfo info = topicNameToSubscriptionTsFileInfo.get(topicName); if (Objects.isNull(info)) { @@ -582,7 +591,7 @@ SubscriptionProvider getProvider(final int dataNodeId) { /////////////////////////////// poll /////////////////////////////// protected List poll(Set topicNames, long timeoutMs) - throws TException, IOException, StatementExecutionException, IoTDBConnectionException { + throws SubscriptionException { final List messages = new ArrayList<>(); // poll on the fly tsfile @@ -630,54 +639,43 @@ protected List poll(Set topicNames, long timeoutMs) } private Optional pollTsFile( - SubscriptionCommitContext commitContext, String fileName, long timeoutMs) { + SubscriptionCommitContext commitContext, String fileName, long timeoutMs) + throws SubscriptionException { try { - final Pair messageWithRetryable = - pollTsFileInternal(commitContext, fileName, timeoutMs); - if (Objects.nonNull(messageWithRetryable.getLeft())) { - removeSubscriptionTsFileInfo(commitContext.getTopicName()); - return Optional.of(messageWithRetryable.getLeft()); - } - if (!messageWithRetryable.getRight()) { - // non-retryable - removeSubscriptionTsFileInfo(commitContext.getTopicName()); - } else { - // retryable - increaseSubscriptionTsFileInfoRetryCountOrRemove(commitContext.getTopicName()); - } - } catch (IOException e) { + final SubscriptionMessage message = pollTsFileInternal(commitContext, fileName, timeoutMs); + removeSubscriptionTsFileInfo(commitContext.getTopicName()); + return Optional.of(message); + } catch (IOException | SubscriptionRetryableException e) { LOGGER.warn( - "Exception occurred when {} polling TsFile {} with commit context {}: {}", + "IOException or SubscriptionRetryableException occurred when SubscriptionConsumer {} polling TsFile {} with commit context {}: {}", this, fileName, commitContext, e.getMessage()); // assume retryable increaseSubscriptionTsFileInfoRetryCountOrRemove(commitContext.getTopicName()); - } catch (TException | IoTDBConnectionException | StatementExecutionException e) { + } catch (SubscriptionNonRetryableException e) { LOGGER.warn( - "Exception occurred when {} polling TsFile {} with commit context {}: {}", + "SubscriptionNonRetryableException occurred when SubscriptionConsumer {} polling TsFile {} with commit context {}: {}", this, fileName, commitContext, e.getMessage()); // assume non-retryable removeSubscriptionTsFileInfo(commitContext.getTopicName()); + throw e; } return Optional.empty(); } - private Pair pollTsFileInternal( + private SubscriptionMessage pollTsFileInternal( SubscriptionCommitContext commitContext, String fileName, long timeoutMs) - throws IOException, TException, IoTDBConnectionException, StatementExecutionException { + throws IOException, SubscriptionException { final int dataNodeId = commitContext.getDataNodeId(); final String topicName = commitContext.getTopicName(); - SubscriptionTsFileInfo info = getSubscriptionTsFileInfoTsFileInfo(topicName); - if (Objects.isNull(info)) { - info = createSubscriptionTsFileInfoTsFileInfo(commitContext, fileName); - } - + final SubscriptionTsFileInfo info = + getSubscriptionTsFileInfoTsFileInfo(commitContext, fileName); final File file = info.file; final RandomAccessFile fileWriter = info.fileWriter; @@ -693,8 +691,10 @@ private Pair pollTsFileInternal( pollTsFileInternal(dataNodeId, topicName, fileName, writingOffset, timeoutMs); if (polledMessages.isEmpty()) { - LOGGER.warn("poll empty messages, consumer: {}", this); - return new Pair<>(null, false); + final String errorMessage = + String.format("SubscriptionConsumer %s poll empty tsfile message", this); + LOGGER.warn(errorMessage); + throw new SubscriptionNonRetryableException(errorMessage); } final SubscriptionPolledMessage polledMessage = polledMessages.get(0); @@ -702,12 +702,12 @@ private Pair pollTsFileInternal( final SubscriptionCommitContext incomingCommitContext = polledMessage.getCommitContext(); if (Objects.isNull(incomingCommitContext) || !Objects.equals(commitContext, incomingCommitContext)) { - LOGGER.warn( - "inconsistent commit context, current is {}, incoming is {}, consumer: {}", - commitContext, - incomingCommitContext, - this); - return new Pair<>(null, false); + final String errorMessage = + String.format( + "inconsistent commit context, current is %s, incoming is %s, consumer: %s", + commitContext, incomingCommitContext, this); + LOGGER.warn(errorMessage); + throw new SubscriptionNonRetryableException(errorMessage); } final short messageType = polledMessage.getMessageType(); @@ -718,12 +718,12 @@ private Pair pollTsFileInternal( // check file name if (!Objects.equals( fileName, ((TsFilePieceMessagePayload) messagePayload).getFileName())) { - LOGGER.warn( - "inconsistent file name, current is {}, incoming is {}, consumer: {}", - fileName, - ((TsFilePieceMessagePayload) messagePayload).getFileName(), - this); - return new Pair<>(null, false); + final String errorMessage = + String.format( + "inconsistent file name, current is %s, incoming is %s, consumer: %s", + fileName, ((TsFilePieceMessagePayload) messagePayload).getFileName(), this); + LOGGER.warn(errorMessage); + throw new SubscriptionNonRetryableException(errorMessage); } // write file piece @@ -734,12 +734,14 @@ private Pair pollTsFileInternal( if (!Objects.equals( fileWriter.length(), ((TsFilePieceMessagePayload) messagePayload).getNextWritingOffset())) { - LOGGER.warn( - "inconsistent file offset, current is {}, incoming is {}, consumer: {}", - fileWriter.length(), - ((TsFilePieceMessagePayload) messagePayload).getNextWritingOffset(), - this); - return new Pair<>(null, false); + final String errorMessage = + String.format( + "inconsistent file offset, current is %s, incoming is %s, consumer: %s", + fileWriter.length(), + ((TsFilePieceMessagePayload) messagePayload).getNextWritingOffset(), + this); + LOGGER.warn(errorMessage); + throw new SubscriptionNonRetryableException(errorMessage); } // update offset @@ -751,23 +753,25 @@ private Pair pollTsFileInternal( // check file name if (!Objects.equals( fileName, ((TsFileSealMessagePayload) messagePayload).getFileName())) { - LOGGER.warn( - "inconsistent file name, current is {}, incoming is {}, consumer: {}", - fileName, - ((TsFileSealMessagePayload) messagePayload).getFileName(), - this); - return new Pair<>(null, false); + final String errorMessage = + String.format( + "inconsistent file name, current is %s, incoming is %s, consumer: %s", + fileName, ((TsFileSealMessagePayload) messagePayload).getFileName(), this); + LOGGER.warn(errorMessage); + throw new SubscriptionNonRetryableException(errorMessage); } // check file length if (fileWriter.length() != ((TsFileSealMessagePayload) messagePayload).getFileLength()) { - LOGGER.warn( - "inconsistent file length, current is {}, incoming is {}, consumer: {}", - fileWriter.length(), - ((TsFileSealMessagePayload) messagePayload).getFileLength(), - this); - return new Pair<>(null, false); + final String errorMessage = + String.format( + "inconsistent file length, current is %s, incoming is %s, consumer: %s", + fileWriter.length(), + ((TsFileSealMessagePayload) messagePayload).getFileLength(), + this); + LOGGER.warn(errorMessage); + throw new SubscriptionNonRetryableException(errorMessage); } // sync and close @@ -775,14 +779,13 @@ private Pair pollTsFileInternal( fileWriter.close(); LOGGER.info( - "{} successfully poll TsFile {} with commit context {}", + "SubscriptionConsumer {} successfully poll TsFile {} with commit context {}", this, file.getAbsolutePath(), commitContext); // generate subscription message - return new Pair<>( - new SubscriptionMessage(commitContext, file.getAbsolutePath()), true); + return new SubscriptionMessage(commitContext, file.getAbsolutePath()); } case TS_FILE_ERROR: { @@ -790,41 +793,48 @@ private Pair pollTsFileInternal( ((TsFileErrorMessagePayload) messagePayload).getErrorMessage(); final boolean retryable = ((TsFileErrorMessagePayload) messagePayload).isRetryable(); LOGGER.warn( - "Error occurred when {} polling TsFile {} with commit context {}: {}, retryable: {}", + "Error occurred when SubscriptionConsumer {} polling TsFile {} with commit context {}: {}, retryable: {}", this, file.getAbsolutePath(), commitContext, errorMessage, retryable); - return new Pair<>(null, retryable); + if (retryable) { + throw new SubscriptionRetryableException(errorMessage); + } else { + throw new SubscriptionNonRetryableException(errorMessage); + } } default: - LOGGER.warn("unexpected message type: {}", messageType); - return new Pair<>(null, false); + final String errorMessage = String.format("unexpected message type: %s", messageType); + LOGGER.warn(errorMessage); + throw new SubscriptionNonRetryableException(errorMessage); } } else { - LOGGER.warn("unexpected message type: {}", messageType); - return new Pair<>(null, false); + final String errorMessage = String.format("unexpected message type: %s", messageType); + LOGGER.warn(errorMessage); + throw new SubscriptionNonRetryableException(errorMessage); } } } private List pollInternal(Set topicNames, long timeoutMs) - throws TException, IOException, StatementExecutionException { + throws SubscriptionException { final List polledMessages = new ArrayList<>(); acquireReadLock(); try { for (final SubscriptionProvider provider : getAllAvailableProviders()) { - // TODO: network timeout - polledMessages.addAll( - provider - .getSessionConnection() - .poll( - new SubscriptionPollMessage( - SubscriptionPollMessageType.POLL.getType(), - new PollMessagePayload(topicNames), - timeoutMs))); + try { + polledMessages.addAll( + provider.poll( + new SubscriptionPollMessage( + SubscriptionPollMessageType.POLL.getType(), + new PollMessagePayload(topicNames), + timeoutMs))); + } catch (final Exception e) { + LOGGER.warn(e.getMessage()); + } } } finally { releaseReadLock(); @@ -835,23 +845,21 @@ private List pollInternal(Set topicNames, lon private List pollTsFileInternal( int dataNodeId, String topicName, String fileName, long writingOffset, long timeoutMs) - throws TException, IOException, StatementExecutionException, IoTDBConnectionException { + throws SubscriptionException { acquireReadLock(); try { final SubscriptionProvider provider = getProvider(dataNodeId); if (Objects.isNull(provider) || !provider.isAvailable()) { - throw new IoTDBConnectionException( + throw new SubscriptionConnectionException( String.format( "something unexpected happened when poll TsFile from subscription provider with data node id %s, the subscription provider may be unavailable or not existed", dataNodeId)); } - return provider - .getSessionConnection() - .poll( - new SubscriptionPollMessage( - SubscriptionPollMessageType.POLL_TS_FILE.getType(), - new PollTsFileMessagePayload(topicName, fileName, writingOffset), - timeoutMs)); + return provider.poll( + new SubscriptionPollMessage( + SubscriptionPollMessageType.POLL_TS_FILE.getType(), + new PollTsFileMessagePayload(topicName, fileName, writingOffset), + timeoutMs)); } finally { releaseReadLock(); } @@ -859,8 +867,7 @@ private List pollTsFileInternal( /////////////////////////////// commit sync /////////////////////////////// - protected void commitSync(Iterable messages) - throws TException, IOException, StatementExecutionException, IoTDBConnectionException { + protected void commitSync(Iterable messages) throws SubscriptionException { Map> dataNodeIdToSubscriptionCommitContexts = new HashMap<>(); for (SubscriptionMessage message : messages) { @@ -876,17 +883,17 @@ protected void commitSync(Iterable messages) private void commitSyncInternal( int dataNodeId, List subscriptionCommitContexts) - throws TException, IOException, StatementExecutionException, IoTDBConnectionException { + throws SubscriptionException { acquireReadLock(); try { final SubscriptionProvider provider = getProvider(dataNodeId); if (Objects.isNull(provider) || !provider.isAvailable()) { - throw new IoTDBConnectionException( + throw new SubscriptionConnectionException( String.format( "something unexpected happened when commit messages to subscription provider with data node id %s, the subscription provider may be unavailable or not existed", dataNodeId)); } - provider.getSessionConnection().commitSync(subscriptionCommitContexts); + provider.commitSync(subscriptionCommitContexts); } finally { releaseReadLock(); } diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionProvider.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionProvider.java index 45006f43f642..77b5d1fb5d2c 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionProvider.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionProvider.java @@ -20,20 +20,36 @@ package org.apache.iotdb.session.subscription; import org.apache.iotdb.common.rpc.thrift.TEndPoint; +import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.rpc.IoTDBConnectionException; import org.apache.iotdb.rpc.StatementExecutionException; import org.apache.iotdb.rpc.subscription.config.ConsumerConfig; import org.apache.iotdb.rpc.subscription.config.ConsumerConstant; +import org.apache.iotdb.rpc.subscription.exception.SubscriptionException; +import org.apache.iotdb.rpc.subscription.exception.SubscriptionNonRetryableException; +import org.apache.iotdb.rpc.subscription.exception.SubscriptionRetryableException; +import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionCommitContext; +import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPollMessage; +import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPolledMessage; +import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribeCommitReq; +import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribePollReq; +import org.apache.iotdb.rpc.subscription.payload.response.PipeSubscribePollResp; +import org.apache.iotdb.service.rpc.thrift.TPipeSubscribeResp; import org.apache.thrift.TException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.IOException; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.concurrent.atomic.AtomicBoolean; final class SubscriptionProvider extends SubscriptionSession { + private static final Logger LOGGER = LoggerFactory.getLogger(SubscriptionProvider.class); + private final String consumerId; private final String consumerGroupId; @@ -56,6 +72,28 @@ final class SubscriptionProvider extends SubscriptionSession { this.consumerGroupId = consumerGroupId; } + boolean isAvailable() { + return isAvailable.get(); + } + + void setAvailable() { + isAvailable.set(true); + } + + void setUnavailable() { + isAvailable.set(false); + } + + int getDataNodeId() { + return dataNodeId; + } + + TEndPoint getEndPoint() { + return endPoint; + } + + /////////////////////////////// open & close /////////////////////////////// + synchronized int handshake() throws IoTDBConnectionException, TException, IOException, StatementExecutionException { if (!isClosed.get()) { @@ -96,24 +134,96 @@ SubscriptionSessionConnection getSessionConnection() { return (SubscriptionSessionConnection) defaultSessionConnection; } - boolean isAvailable() { - return isAvailable.get(); - } - - void setAvailable() { - isAvailable.set(true); - } - - void setUnavailable() { - isAvailable.set(false); + List poll(SubscriptionPollMessage pollMessage) + throws SubscriptionException { + final PipeSubscribePollReq req; + try { + req = PipeSubscribePollReq.toTPipeSubscribeReq(pollMessage); + } catch (final IOException e) { + LOGGER.warn( + "IOException occurred when serialize poll request {}: {}", pollMessage, e.getMessage()); + throw new SubscriptionRetryableException(e.getMessage(), e); + } + final TPipeSubscribeResp resp; + try { + resp = getSessionConnection().pipeSubscribe(req); + } catch (final TException e) { + // TODO: Distinguish between TTransportException, TApplicationException, and + // TProtocolException. + LOGGER.warn( + "TException occurred when poll with request {}: {}, set SubscriptionProvider {} unavailable", + pollMessage, + e.getMessage(), + true); + setUnavailable(); + throw new SubscriptionNonRetryableException(e.getMessage(), e); + } + verifyPipeSubscribeSuccess(resp.status); + final PipeSubscribePollResp pollResp = PipeSubscribePollResp.fromTPipeSubscribeResp(resp); + return pollResp.getMessages(); } - int getDataNodeId() { - return dataNodeId; + void commitSync(List subscriptionCommitContexts) + throws SubscriptionException { + final PipeSubscribeCommitReq req; + try { + req = PipeSubscribeCommitReq.toTPipeSubscribeReq(subscriptionCommitContexts); + } catch (final IOException e) { + LOGGER.warn( + "IOException occurred when serialize commit request {}: {}", + subscriptionCommitContexts, + e.getMessage()); + throw new SubscriptionRetryableException(e.getMessage(), e); + } + final TPipeSubscribeResp resp; + try { + resp = getSessionConnection().pipeSubscribe(req); + } catch (final TException e) { + // TODO: Distinguish between TTransportException, TApplicationException, and + // TProtocolException. + LOGGER.warn( + "TException occurred when commit with request {}: {}, set SubscriptionProvider {} unavailable", + subscriptionCommitContexts, + e.getMessage(), + true); + setUnavailable(); + throw new SubscriptionNonRetryableException(e.getMessage(), e); + } + verifyPipeSubscribeSuccess(resp.status); } - TEndPoint getEndPoint() { - return endPoint; + private static void verifyPipeSubscribeSuccess(TSStatus status) throws SubscriptionException { + switch (status.code) { + case 200: // SUCCESS_STATUS + return; + case 1900: // SUBSCRIPTION_VERSION_ERROR + case 1901: // SUBSCRIPTION_TYPE_ERROR + case 1902: // SUBSCRIPTION_HANDSHAKE_ERROR + case 1903: // SUBSCRIPTION_HEARTBEAT_ERROR + case 1904: // SUBSCRIPTION_POLL_ERROR + case 1905: // SUBSCRIPTION_COMMIT_ERROR + case 1906: // SUBSCRIPTION_CLOSE_ERROR + case 1907: // SUBSCRIPTION_SUBSCRIBE_ERROR + case 1908: // SUBSCRIPTION_UNSUBSCRIBE_ERROR + case 1909: // SUBSCRIPTION_MISSING_CUSTOMER + LOGGER.warn( + "Internal error occurred, status code {}, status message {}", + status.code, + status.message); + throw new SubscriptionNonRetryableException(status.message); + case 1911: // SUBSCRIPTION_SERIALIZATION_ERROR + LOGGER.warn( + "Internal error occurred when serialize response, status code {}, status message {}", + status.code, + status.message); + throw new SubscriptionRetryableException(status.message); + default: + LOGGER.warn( + "Internal error occurred, status code {}, status message {}", + status.code, + status.message); + throw new SubscriptionNonRetryableException(status.message); + } } @Override diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java index 58e2171352ae..72220da4176c 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java @@ -166,13 +166,11 @@ public List poll(Set topicNames, long timeoutMs) /////////////////////////////// commit /////////////////////////////// - public void commitSync(SubscriptionMessage message) - throws TException, IOException, StatementExecutionException, IoTDBConnectionException { + public void commitSync(SubscriptionMessage message) throws SubscriptionException { super.commitSync(Collections.singletonList(message)); } - public void commitSync(Iterable messages) - throws TException, IOException, StatementExecutionException, IoTDBConnectionException { + public void commitSync(Iterable messages) throws SubscriptionException { super.commitSync(messages); } diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionSessionConnection.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionSessionConnection.java index e6d957f84b73..8245770afdc7 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionSessionConnection.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionSessionConnection.java @@ -25,18 +25,13 @@ import org.apache.iotdb.rpc.RpcUtils; import org.apache.iotdb.rpc.StatementExecutionException; import org.apache.iotdb.rpc.subscription.config.ConsumerConfig; -import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionCommitContext; -import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPollMessage; -import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPolledMessage; import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribeCloseReq; -import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribeCommitReq; import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribeHandshakeReq; import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribeHeartbeatReq; -import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribePollReq; import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribeSubscribeReq; import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribeUnsubscribeReq; import org.apache.iotdb.rpc.subscription.payload.response.PipeSubscribeHandshakeResp; -import org.apache.iotdb.rpc.subscription.payload.response.PipeSubscribePollResp; +import org.apache.iotdb.service.rpc.thrift.TPipeSubscribeReq; import org.apache.iotdb.service.rpc.thrift.TPipeSubscribeResp; import org.apache.iotdb.session.Session; import org.apache.iotdb.session.SessionConnection; @@ -92,6 +87,10 @@ public Map fetchAllEndPoints() return endPoints; } + public TPipeSubscribeResp pipeSubscribe(TPipeSubscribeReq req) throws TException { + return client.pipeSubscribe(req); + } + public int handshake(ConsumerConfig consumerConfig) throws TException, IOException, StatementExecutionException { TPipeSubscribeResp resp = @@ -125,21 +124,4 @@ public void unsubscribe(Set topicNames) client.pipeSubscribe(PipeSubscribeUnsubscribeReq.toTPipeSubscribeReq(topicNames)); RpcUtils.verifySuccess(resp.status); } - - public List poll(SubscriptionPollMessage pollMessage) - throws TException, IOException, StatementExecutionException { - TPipeSubscribeResp resp = - client.pipeSubscribe(PipeSubscribePollReq.toTPipeSubscribeReq(pollMessage)); - RpcUtils.verifySuccess(resp.status); - PipeSubscribePollResp pollResp = PipeSubscribePollResp.fromTPipeSubscribeResp(resp); - return pollResp.getMessages(); - } - - public void commitSync(List subscriptionCommitContexts) - throws TException, IOException, StatementExecutionException { - TPipeSubscribeResp resp = - client.pipeSubscribe( - PipeSubscribeCommitReq.toTPipeSubscribeReq(subscriptionCommitContexts)); - RpcUtils.verifySuccess(resp.status); - } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionBrokerAgent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionBrokerAgent.java index 3e9b6cf8772b..8bea194d4e96 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionBrokerAgent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionBrokerAgent.java @@ -24,12 +24,12 @@ import org.apache.iotdb.db.subscription.task.subtask.SubscriptionConnectorSubtask; import org.apache.iotdb.db.subscription.timer.SubscriptionPollTimer; import org.apache.iotdb.rpc.subscription.config.ConsumerConfig; +import org.apache.iotdb.rpc.subscription.exception.SubscriptionException; import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionCommitContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Objects; @@ -52,9 +52,11 @@ public List poll( final String consumerGroupId = consumerConfig.getConsumerGroupId(); final SubscriptionBroker broker = consumerGroupIdToSubscriptionBroker.get(consumerGroupId); if (Objects.isNull(broker)) { - LOGGER.warn( - "Subscription: broker bound to consumer group [{}] does not exist", consumerGroupId); - return Collections.emptyList(); + final String errorMessage = + String.format( + "Subscription: broker bound to consumer group [%s] does not exist", consumerGroupId); + LOGGER.warn(errorMessage); + throw new SubscriptionException(errorMessage); } // TODO: currently we fetch messages from all topics final String consumerId = consumerConfig.getConsumerId(); @@ -66,24 +68,29 @@ public List pollTsFile( final String consumerGroupId = consumerConfig.getConsumerGroupId(); final SubscriptionBroker broker = consumerGroupIdToSubscriptionBroker.get(consumerGroupId); if (Objects.isNull(broker)) { - LOGGER.warn( - "Subscription: broker bound to consumer group [{}] does not exist", consumerGroupId); - return Collections.emptyList(); + final String errorMessage = + String.format( + "Subscription: broker bound to consumer group [%s] does not exist", consumerGroupId); + LOGGER.warn(errorMessage); + throw new SubscriptionException(errorMessage); } final String consumerId = consumerConfig.getConsumerId(); return broker.pollTsFile(consumerId, topicName, fileName, writingOffset); } - public void commit( + /** @return list of successful commit contexts */ + public List commit( final ConsumerConfig consumerConfig, final List commitContexts) { final String consumerGroupId = consumerConfig.getConsumerGroupId(); final SubscriptionBroker broker = consumerGroupIdToSubscriptionBroker.get(consumerGroupId); if (Objects.isNull(broker)) { - LOGGER.warn( - "Subscription: broker bound to consumer group [{}] does not exist", consumerGroupId); - return; + final String errorMessage = + String.format( + "Subscription: broker bound to consumer group [%s] does not exist", consumerGroupId); + LOGGER.warn(errorMessage); + throw new SubscriptionException(errorMessage); } - broker.commit(commitContexts); + return broker.commit(commitContexts); } /////////////////////////////// broker /////////////////////////////// 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 c301178fb887..15505b06bee5 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 @@ -25,6 +25,7 @@ import org.apache.iotdb.db.subscription.timer.SubscriptionPollTimer; import org.apache.iotdb.pipe.api.event.Event; import org.apache.iotdb.rpc.subscription.config.TopicConstant; +import org.apache.iotdb.rpc.subscription.exception.SubscriptionException; import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionCommitContext; import org.slf4j.Logger; @@ -83,17 +84,28 @@ public List pollTsFile( final SubscriptionPrefetchingQueue prefetchingQueue = topicNameToPrefetchingQueue.get(topicName); if (Objects.isNull(prefetchingQueue)) { - return Collections.emptyList(); + final String errorMessage = + String.format( + "Subscription: prefetching queue bound to topic [%s] does not exist", topicName); + LOGGER.warn(errorMessage); + throw new SubscriptionException(errorMessage); } if (!(prefetchingQueue instanceof SubscriptionPrefetchingTsFileQueue)) { - return Collections.emptyList(); + final String errorMessage = + String.format( + "Subscription: prefetching queue bound to topic [%s] is invalid", topicName); + LOGGER.warn(errorMessage); + throw new SubscriptionException(errorMessage); } return Collections.singletonList( ((SubscriptionPrefetchingTsFileQueue) prefetchingQueue) .pollTsFile(consumerId, fileName, writingOffset)); } - public void commit(final List commitContexts) { + /** @return list of successful commit contexts */ + public List commit( + final List commitContexts) { + final List successfulCommitContexts = new ArrayList<>(); for (final SubscriptionCommitContext commitContext : commitContexts) { final String topicName = commitContext.getTopicName(); final SubscriptionPrefetchingQueue prefetchingQueue = @@ -103,8 +115,11 @@ public void commit(final List commitContexts) { "Subscription: prefetching queue bound to topic [{}] does not exist", topicName); continue; } - prefetchingQueue.commit(commitContext); + if (prefetchingQueue.commit(commitContext)) { + successfulCommitContexts.add(commitContext); + } } + return successfulCommitContexts; } /////////////////////////////// prefetching queue /////////////////////////////// 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 b58280f6faa7..50a9d2d1ea8f 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 @@ -63,16 +63,20 @@ public SubscriptionPrefetchingQueue( /////////////////////////////// commit /////////////////////////////// - public void commit(final SubscriptionCommitContext commitContext) { + /** @return {@code true} if commit successfully */ + public boolean commit(final SubscriptionCommitContext commitContext) { final SubscriptionEvent event = uncommittedEvents.get(commitContext); if (Objects.isNull(event)) { LOGGER.warn( - "Subscription: subscription commit context [{}] does not exist, it may have been committed or something unexpected happened", - commitContext); + "Subscription: subscription commit context [{}] does not exist, it may have been committed or something unexpected happened, prefetching queue: {}", + commitContext, + this); + return false; } event.decreaseReferenceCount(); event.recordCommittedTimestamp(); uncommittedEvents.remove(commitContext); + return true; } protected SubscriptionCommitContext generateSubscriptionCommitContext() { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTabletsQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTabletsQueue.java index e817c5bfc610..a533463aefb8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTabletsQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTabletsQueue.java @@ -94,7 +94,10 @@ public SubscriptionEvent poll(final String consumerId, final SubscriptionPollTim } } catch (final InterruptedException e) { Thread.currentThread().interrupt(); - LOGGER.warn("Subscription: Interrupted while polling events.", e); + LOGGER.warn( + "Subscription: SubscriptionPrefetchingTabletsQueue {} interrupted while polling events.", + this, + e); } return null; @@ -116,7 +119,10 @@ private void prefetchOnce(final long limit) { while (Objects.nonNull( event = UserDefinedEnrichedEvent.maybeOf(inputPendingQueue.waitedPoll()))) { if (!(event instanceof EnrichedEvent)) { - LOGGER.warn("Subscription: Only support prefetch EnrichedEvent. Ignore {}.", event); + LOGGER.warn( + "Subscription: SubscriptionPrefetchingTabletsQueue {} only support prefetch EnrichedEvent. Ignore {}.", + this, + event); continue; } @@ -148,7 +154,10 @@ private void prefetchOnce(final long limit) { // - PipeHeartbeatEvent: ignored? (may affect pipe metrics) // - UserDefinedEnrichedEvent: ignored? // - Others: events related to meta sync, safe to ignore - LOGGER.warn("Subscription: Ignore EnrichedEvent {} when prefetching.", event); + LOGGER.warn( + "Subscription: SubscriptionPrefetchingTabletsQueue {} ignore EnrichedEvent {} when prefetching.", + this, + event); } } @@ -195,7 +204,10 @@ private void serializeOnce() { } } catch (final InterruptedException e) { Thread.currentThread().interrupt(); - LOGGER.warn("Subscription: Interrupted while serializing events.", e); + LOGGER.warn( + "Subscription: SubscriptionPrefetchingTabletsQueue {} interrupted while serializing events.", + this, + e); } } @@ -209,7 +221,8 @@ private Tablet convertToTablet(final TabletInsertionEvent tabletInsertionEvent) } LOGGER.warn( - "Subscription: Only support convert PipeInsertNodeTabletInsertionEvent or PipeRawTabletInsertionEvent to tablet. Ignore {}.", + "Subscription: SubscriptionPrefetchingTabletsQueue {} only support convert PipeInsertNodeTabletInsertionEvent or PipeRawTabletInsertionEvent to tablet. Ignore {}.", + this, tabletInsertionEvent); return null; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java index fe056469c65b..3f79cde94e06 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java @@ -444,12 +444,22 @@ private TPipeSubscribeResp handlePipeSubscribeCommitInternal(final PipeSubscribe // commit final List commitContexts = req.getCommitContexts(); - SubscriptionAgent.broker().commit(consumerConfig, commitContexts); + final List successfulCommitContexts = + SubscriptionAgent.broker().commit(consumerConfig, commitContexts); + + if (Objects.equals(successfulCommitContexts.size(), commitContexts.size())) { + LOGGER.info( + "Subscription: consumer {} commit successfully, commit contexts: {}", + consumerConfig, + commitContexts); + } else { + LOGGER.warn( + "Subscription: consumer {} commit partially successful, commit contexts: {}, successful commit contexts: {}", + consumerConfig, + commitContexts, + successfulCommitContexts); + } - LOGGER.info( - "Subscription: consumer commit {} successfully, commit contexts: {}", - consumerConfig, - commitContexts); return PipeSubscribeCommitResp.toTPipeSubscribeResp(RpcUtils.SUCCESS_STATUS); } @@ -463,7 +473,7 @@ private TPipeSubscribeResp handlePipeSubscribeClose(final PipeSubscribeCloseReq e.getMessage(), req); LOGGER.warn(exceptionMessage); return PipeSubscribeHandshakeResp.toTPipeSubscribeResp( - RpcUtils.getStatus(TSStatusCode.SUBSCRIPTION_COMMIT_ERROR, exceptionMessage)); + RpcUtils.getStatus(TSStatusCode.SUBSCRIPTION_CLOSE_ERROR, exceptionMessage)); } } From 33a9a3d6d611f38c187ebec9181cb6d25c0c9189 Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Fri, 19 Apr 2024 23:00:18 +0800 Subject: [PATCH 44/93] fix --- .../session/subscription/SubscriptionPullConsumer.java | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java index 72220da4176c..5146a8cb63ff 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java @@ -130,23 +130,21 @@ public synchronized void close() throws IoTDBConnectionException { /////////////////////////////// poll & commit /////////////////////////////// - public List poll(Duration timeoutMs) - throws TException, IOException, StatementExecutionException, IoTDBConnectionException { + public List poll(Duration timeoutMs) throws SubscriptionException { return poll(Collections.emptySet(), timeoutMs.toMillis()); } - public List poll(long timeoutMs) - throws TException, IOException, StatementExecutionException, IoTDBConnectionException { + public List poll(long timeoutMs) throws SubscriptionException { return poll(Collections.emptySet(), timeoutMs); } public List poll(Set topicNames, Duration timeoutMs) - throws TException, IOException, StatementExecutionException, IoTDBConnectionException { + throws SubscriptionException { return poll(topicNames, timeoutMs.toMillis()); } public List poll(Set topicNames, long timeoutMs) - throws TException, IOException, StatementExecutionException, IoTDBConnectionException { + throws SubscriptionException { final List messages = super.poll(topicNames, timeoutMs); // add to uncommitted messages From d7f7b05f79b776c158a781bcf18df1c969adfae6 Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Fri, 19 Apr 2024 23:02:50 +0800 Subject: [PATCH 45/93] fix --- .../iotdb/session/subscription/SubscriptionPushConsumer.java | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPushConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPushConsumer.java index 720065dbcba0..a3e174d5651b 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPushConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPushConsumer.java @@ -240,10 +240,7 @@ public void run() { commitSync(pollResults); } - } catch (TException - | IOException - | StatementExecutionException - | IoTDBConnectionException e) { + } catch (SubscriptionException e) { LOGGER.warn("Exception occurred when auto polling: ", e); } } From b73aa9ae2326271d2d7de898895ccced02626224 Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Fri, 19 Apr 2024 23:40:28 +0800 Subject: [PATCH 46/93] fix --- .../rpc/subscription/payload/common/PollMessagePayload.java | 2 +- .../payload/common/PollTsFileMessagePayload.java | 2 +- .../payload/common/SubscriptionCommitContext.java | 4 ++-- .../subscription/payload/common/SubscriptionPollMessage.java | 5 ++--- .../payload/common/SubscriptionPolledMessage.java | 5 ++--- .../subscription/payload/common/TabletsMessagePayload.java | 4 ++-- .../payload/common/TsFileErrorMessagePayload.java | 2 +- .../payload/common/TsFileInitMessagePayload.java | 2 +- .../payload/common/TsFilePieceMessagePayload.java | 4 ++-- .../payload/common/TsFileSealMessagePayload.java | 2 +- 10 files changed, 15 insertions(+), 17 deletions(-) diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/PollMessagePayload.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/PollMessagePayload.java index 8bd08c4c7d48..2e4ddb3acfe8 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/PollMessagePayload.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/PollMessagePayload.java @@ -19,7 +19,7 @@ package org.apache.iotdb.rpc.subscription.payload.common; -import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; +import org.apache.tsfile.utils.ReadWriteIOUtils; import java.io.DataOutputStream; import java.io.IOException; diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/PollTsFileMessagePayload.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/PollTsFileMessagePayload.java index 4993bbfc0ff5..402650aaff2e 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/PollTsFileMessagePayload.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/PollTsFileMessagePayload.java @@ -19,7 +19,7 @@ package org.apache.iotdb.rpc.subscription.payload.common; -import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; +import org.apache.tsfile.utils.ReadWriteIOUtils; import java.io.DataOutputStream; import java.io.IOException; diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionCommitContext.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionCommitContext.java index 964dba331f06..95016b78b1c8 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionCommitContext.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionCommitContext.java @@ -19,8 +19,8 @@ package org.apache.iotdb.rpc.subscription.payload.common; -import org.apache.iotdb.tsfile.utils.PublicBAOS; -import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; +import org.apache.tsfile.utils.PublicBAOS; +import org.apache.tsfile.utils.ReadWriteIOUtils; import java.io.DataOutputStream; import java.io.IOException; diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPollMessage.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPollMessage.java index ddc7ab93913f..86a6247b1331 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPollMessage.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPollMessage.java @@ -19,9 +19,8 @@ package org.apache.iotdb.rpc.subscription.payload.common; -import org.apache.iotdb.tsfile.utils.PublicBAOS; -import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; - +import org.apache.tsfile.utils.PublicBAOS; +import org.apache.tsfile.utils.ReadWriteIOUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledMessage.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledMessage.java index cadc15db5d66..2a03874cda33 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledMessage.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledMessage.java @@ -19,9 +19,8 @@ package org.apache.iotdb.rpc.subscription.payload.common; -import org.apache.iotdb.tsfile.utils.PublicBAOS; -import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; - +import org.apache.tsfile.utils.PublicBAOS; +import org.apache.tsfile.utils.ReadWriteIOUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TabletsMessagePayload.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TabletsMessagePayload.java index 18258a476458..e83b9f5a0d20 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TabletsMessagePayload.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TabletsMessagePayload.java @@ -19,8 +19,8 @@ package org.apache.iotdb.rpc.subscription.payload.common; -import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; -import org.apache.iotdb.tsfile.write.record.Tablet; +import org.apache.tsfile.utils.ReadWriteIOUtils; +import org.apache.tsfile.write.record.Tablet; import java.io.DataOutputStream; import java.io.IOException; diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileErrorMessagePayload.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileErrorMessagePayload.java index 851d20cf4d9c..15d928c699f0 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileErrorMessagePayload.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileErrorMessagePayload.java @@ -19,7 +19,7 @@ package org.apache.iotdb.rpc.subscription.payload.common; -import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; +import org.apache.tsfile.utils.ReadWriteIOUtils; import java.io.DataOutputStream; import java.io.IOException; diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileInitMessagePayload.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileInitMessagePayload.java index 4cecfff1f975..b2329392acf0 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileInitMessagePayload.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileInitMessagePayload.java @@ -19,7 +19,7 @@ package org.apache.iotdb.rpc.subscription.payload.common; -import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; +import org.apache.tsfile.utils.ReadWriteIOUtils; import java.io.DataOutputStream; import java.io.IOException; diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFilePieceMessagePayload.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFilePieceMessagePayload.java index d8197b6f340c..a3233f2986f5 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFilePieceMessagePayload.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFilePieceMessagePayload.java @@ -19,8 +19,8 @@ package org.apache.iotdb.rpc.subscription.payload.common; -import org.apache.iotdb.tsfile.utils.Binary; -import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; +import org.apache.tsfile.utils.Binary; +import org.apache.tsfile.utils.ReadWriteIOUtils; import java.io.DataOutputStream; import java.io.IOException; diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileSealMessagePayload.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileSealMessagePayload.java index dfe6d491e14e..b899f0b582cc 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileSealMessagePayload.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileSealMessagePayload.java @@ -19,7 +19,7 @@ package org.apache.iotdb.rpc.subscription.payload.common; -import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; +import org.apache.tsfile.utils.ReadWriteIOUtils; import java.io.DataOutputStream; import java.io.IOException; From d4f2e7e0f7b662322f7d1885b3068d4ae4ec860d Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Sat, 20 Apr 2024 00:05:16 +0800 Subject: [PATCH 47/93] fix --- .../org/apache/iotdb/SubscriptionSessionExample.java | 9 +++++---- .../iotdb/session/subscription/SubscriptionMessage.java | 3 ++- .../broker/SubscriptionPrefetchingTabletsQueue.java | 2 +- .../broker/SubscriptionPrefetchingTsFileQueue.java | 2 +- .../db/subscription/event/SubscriptionTsFileEvent.java | 2 +- 5 files changed, 10 insertions(+), 8 deletions(-) diff --git a/example/session/src/main/java/org/apache/iotdb/SubscriptionSessionExample.java b/example/session/src/main/java/org/apache/iotdb/SubscriptionSessionExample.java index 46c4da1c5d83..71ffcd7e60ba 100644 --- a/example/session/src/main/java/org/apache/iotdb/SubscriptionSessionExample.java +++ b/example/session/src/main/java/org/apache/iotdb/SubscriptionSessionExample.java @@ -30,10 +30,11 @@ import org.apache.iotdb.session.subscription.SubscriptionSessionDataSet; import org.apache.iotdb.session.subscription.SubscriptionSessionDataSets; import org.apache.iotdb.session.subscription.SubscriptionTsFileReader; -import org.apache.iotdb.tsfile.read.TsFileReader; -import org.apache.iotdb.tsfile.read.common.Path; -import org.apache.iotdb.tsfile.read.expression.QueryExpression; -import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet; + +import org.apache.tsfile.read.TsFileReader; +import org.apache.tsfile.read.common.Path; +import org.apache.tsfile.read.expression.QueryExpression; +import org.apache.tsfile.read.query.dataset.QueryDataSet; import java.time.Duration; import java.util.Collections; diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionMessage.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionMessage.java index 76b0602b64ce..128d3a41e184 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionMessage.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionMessage.java @@ -20,7 +20,8 @@ package org.apache.iotdb.session.subscription; import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionCommitContext; -import org.apache.iotdb.tsfile.write.record.Tablet; + +import org.apache.tsfile.write.record.Tablet; import java.util.List; import java.util.Objects; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTabletsQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTabletsQueue.java index a533463aefb8..ddfcf17d0f2c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTabletsQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTabletsQueue.java @@ -34,8 +34,8 @@ import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPolledMessage; import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPolledMessageType; import org.apache.iotdb.rpc.subscription.payload.common.TabletsMessagePayload; -import org.apache.iotdb.tsfile.write.record.Tablet; +import org.apache.tsfile.write.record.Tablet; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java index e96b43e9cc3e..c7b63bbd4f84 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java @@ -34,8 +34,8 @@ import org.apache.iotdb.rpc.subscription.payload.common.TsFileInitMessagePayload; import org.apache.iotdb.rpc.subscription.payload.common.TsFilePieceMessagePayload; import org.apache.iotdb.rpc.subscription.payload.common.TsFileSealMessagePayload; -import org.apache.iotdb.tsfile.utils.Pair; +import org.apache.tsfile.utils.Pair; import org.checkerframework.checker.nullness.qual.NonNull; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionTsFileEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionTsFileEvent.java index 92f93dd59508..f088ede74136 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionTsFileEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionTsFileEvent.java @@ -29,8 +29,8 @@ import org.apache.iotdb.rpc.subscription.payload.common.TsFileInitMessagePayload; import org.apache.iotdb.rpc.subscription.payload.common.TsFilePieceMessagePayload; import org.apache.iotdb.rpc.subscription.payload.common.TsFileSealMessagePayload; -import org.apache.iotdb.tsfile.utils.Pair; +import org.apache.tsfile.utils.Pair; import org.checkerframework.checker.nullness.qual.NonNull; import org.slf4j.Logger; import org.slf4j.LoggerFactory; From 8809eda98e2fb7af45145ca9f483f8059e0fab33 Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Sat, 20 Apr 2024 00:28:00 +0800 Subject: [PATCH 48/93] spotless --- .../subscription/it/dual/IoTDBSubscriptionConsumerGroupIT.java | 2 +- .../iotdb/subscription/it/dual/IoTDBSubscriptionTopicIT.java | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionConsumerGroupIT.java b/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionConsumerGroupIT.java index 504f26b80b85..945a3c3ea494 100644 --- a/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionConsumerGroupIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionConsumerGroupIT.java @@ -35,9 +35,9 @@ import org.apache.iotdb.session.subscription.SubscriptionSession; import org.apache.iotdb.session.subscription.SubscriptionSessionDataSet; import org.apache.iotdb.session.subscription.SubscriptionSessionDataSets; - import org.apache.iotdb.session.subscription.SubscriptionTsFileReader; import org.apache.iotdb.subscription.it.IoTDBSubscriptionITConstant; + import org.apache.tsfile.read.TsFileReader; import org.apache.tsfile.read.common.Path; import org.apache.tsfile.read.common.RowRecord; diff --git a/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionTopicIT.java b/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionTopicIT.java index d7fb7c998d3d..4efef40043a8 100644 --- a/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionTopicIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionTopicIT.java @@ -30,7 +30,6 @@ import org.apache.iotdb.session.subscription.SubscriptionSessionDataSets; import org.apache.iotdb.subscription.it.IoTDBSubscriptionITConstant; - import org.apache.tsfile.write.record.Tablet; import org.awaitility.Awaitility; import org.junit.Test; From 82b2d93e3c0a552ba8fabb84daeeb5745e95d02b Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Sat, 20 Apr 2024 00:42:27 +0800 Subject: [PATCH 49/93] fix --- .../subscription/it/local/IoTDBSubscriptionBasicIT.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionBasicIT.java b/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionBasicIT.java index 920bb88cc739..c33848391e7c 100644 --- a/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionBasicIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionBasicIT.java @@ -35,11 +35,11 @@ import org.apache.iotdb.session.subscription.SubscriptionSessionDataSets; import org.apache.iotdb.session.subscription.SubscriptionTsFileReader; import org.apache.iotdb.subscription.it.IoTDBSubscriptionITConstant; -import org.apache.iotdb.tsfile.read.TsFileReader; -import org.apache.iotdb.tsfile.read.common.Path; -import org.apache.iotdb.tsfile.read.expression.QueryExpression; -import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet; +import org.apache.tsfile.read.TsFileReader; +import org.apache.tsfile.read.common.Path; +import org.apache.tsfile.read.expression.QueryExpression; +import org.apache.tsfile.read.query.dataset.QueryDataSet; import org.awaitility.Awaitility; import org.junit.After; import org.junit.Assert; From a39e52a0fe2dbb92b0610a9a68936bd578f27efd Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Sat, 20 Apr 2024 16:29:04 +0800 Subject: [PATCH 50/93] basic tsfile poll timer --- .../agent/SubscriptionBrokerAgent.java | 8 +++-- .../broker/SubscriptionBroker.java | 8 +++-- .../SubscriptionPrefetchingTsFileQueue.java | 30 ++++++++++++++++--- .../event/SubscriptionTsFileEvent.java | 15 ++++++---- .../receiver/SubscriptionReceiverV1.java | 23 +++++++------- 5 files changed, 58 insertions(+), 26 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionBrokerAgent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionBrokerAgent.java index 8bea194d4e96..1c74b1dc6d6a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionBrokerAgent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionBrokerAgent.java @@ -64,7 +64,11 @@ public List poll( } public List pollTsFile( - ConsumerConfig consumerConfig, String topicName, String fileName, long writingOffset) { + final ConsumerConfig consumerConfig, + final String topicName, + final String fileName, + final long writingOffset, + final SubscriptionPollTimer timer) { final String consumerGroupId = consumerConfig.getConsumerGroupId(); final SubscriptionBroker broker = consumerGroupIdToSubscriptionBroker.get(consumerGroupId); if (Objects.isNull(broker)) { @@ -75,7 +79,7 @@ public List pollTsFile( throw new SubscriptionException(errorMessage); } final String consumerId = consumerConfig.getConsumerId(); - return broker.pollTsFile(consumerId, topicName, fileName, writingOffset); + return broker.pollTsFile(consumerId, topicName, fileName, writingOffset, timer); } /** @return list of successful commit contexts */ 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 15505b06bee5..ba59c580d568 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 @@ -80,7 +80,11 @@ public List poll( } public List pollTsFile( - String consumerId, String topicName, String fileName, long writingOffset) { + final String consumerId, + final String topicName, + final String fileName, + final long writingOffset, + final SubscriptionPollTimer timer) { final SubscriptionPrefetchingQueue prefetchingQueue = topicNameToPrefetchingQueue.get(topicName); if (Objects.isNull(prefetchingQueue)) { @@ -99,7 +103,7 @@ public List pollTsFile( } return Collections.singletonList( ((SubscriptionPrefetchingTsFileQueue) prefetchingQueue) - .pollTsFile(consumerId, fileName, writingOffset)); + .pollTsFile(consumerId, fileName, writingOffset, timer)); } /** @return list of successful commit contexts */ diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java index c7b63bbd4f84..2d43a7eaa719 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java @@ -71,6 +71,11 @@ public SubscriptionTsFileEvent poll(final String consumerId, final SubscriptionP return pollableEvent; } + timer.update(); + if (timer.isExpired()) { + return null; + } + Event event; while (Objects.nonNull( event = UserDefinedEnrichedEvent.maybeOf(inputPendingQueue.waitedPoll()))) { @@ -100,7 +105,10 @@ public SubscriptionTsFileEvent poll(final String consumerId, final SubscriptionP } public synchronized @NonNull SubscriptionTsFileEvent pollTsFile( - String consumerId, String fileName, long writingOffset) { + final String consumerId, + final String fileName, + final long writingOffset, + final SubscriptionPollTimer timer) { // 1. Extract current event and check it final SubscriptionTsFileEvent event = consumerIdToCurrentEventMap.get(consumerId); if (Objects.isNull(event)) { @@ -244,18 +252,32 @@ public SubscriptionTsFileEvent poll(final String consumerId, final SubscriptionP } // 3. Poll tsfile piece or tsfile seal - return pollTsFile(consumerId, writingOffset, event); + return pollTsFile(consumerId, writingOffset, event, timer); } private synchronized @NonNull SubscriptionTsFileEvent pollTsFile( - String consumerId, long writingOffset, SubscriptionTsFileEvent event) { + final String consumerId, + final long writingOffset, + final SubscriptionTsFileEvent event, + final SubscriptionPollTimer timer) { + timer.update(); + if (timer.isExpired()) { + final String errorMessage = + String.format( + "Timeout occurred when SubscriptionPrefetchingTsFileQueue %s transferring TsFile (with event %s) to consumer %s", + this, event, consumerId); + LOGGER.warn(errorMessage); + // assume retryable + return generateSubscriptionTsFileEventWithErrorMessage(errorMessage, true); + } + Pair newEventWithCommittable = event.matchOrResetNext(writingOffset); if (Objects.isNull(newEventWithCommittable)) { try { newEventWithCommittable = event.generateSubscriptionTsFileEventWithPieceOrSealPayload(writingOffset); - } catch (IOException e) { + } catch (final IOException e) { final String errorMessage = String.format( "IOException occurred when SubscriptionPrefetchingTsFileQueue %s transferring TsFile (with event %s) to consumer %s: %s", diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionTsFileEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionTsFileEvent.java index f088ede74136..3db03f5f6127 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionTsFileEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionTsFileEvent.java @@ -50,7 +50,7 @@ public class SubscriptionTsFileEvent extends SubscriptionEvent { private final AtomicReference> nextEventWithCommittableRef; public SubscriptionTsFileEvent( - List enrichedEvents, SubscriptionPolledMessage message) { + final List enrichedEvents, final SubscriptionPolledMessage message) { super(enrichedEvents, message); this.nextEventWithCommittableRef = new AtomicReference<>(); @@ -72,7 +72,7 @@ public void prefetchNext() { case TS_FILE_INIT: try { return generateSubscriptionTsFileEventWithPieceOrSealPayload(0); - } catch (IOException e) { + } catch (final IOException e) { LOGGER.warn( "IOException occurred when prefetching next SubscriptionTsFileEvent: {}, current SubscriptionTsFileEvent: {}", e.getMessage(), @@ -83,7 +83,7 @@ public void prefetchNext() { try { return generateSubscriptionTsFileEventWithPieceOrSealPayload( ((TsFilePieceMessagePayload) messagePayload).getNextWritingOffset()); - } catch (IOException e) { + } catch (final IOException e) { LOGGER.warn( "IOException occurred when prefetching next SubscriptionTsFileEvent: {}, current SubscriptionTsFileEvent: {}", e.getMessage(), @@ -116,7 +116,8 @@ public void serializeNext() { }); } - public Pair<@NonNull SubscriptionTsFileEvent, Boolean> matchOrResetNext(long writingOffset) { + public Pair<@NonNull SubscriptionTsFileEvent, Boolean> matchOrResetNext( + final long writingOffset) { return nextEventWithCommittableRef.getAndUpdate( (nextEventWithCommittable) -> { if (Objects.isNull(nextEventWithCommittable)) { @@ -156,7 +157,8 @@ public void serializeNext() { } public static SubscriptionTsFileEvent generateSubscriptionTsFileEventWithInitPayload( - PipeTsFileInsertionEvent tsFileInsertionEvent, SubscriptionCommitContext commitContext) { + final PipeTsFileInsertionEvent tsFileInsertionEvent, + final SubscriptionCommitContext commitContext) { return new SubscriptionTsFileEvent( Collections.singletonList(tsFileInsertionEvent), new SubscriptionPolledMessage( @@ -172,7 +174,8 @@ public SubscriptionTsFileEvent generateSubscriptionTsFileEventWithInitPayload() } public @NonNull Pair<@NonNull SubscriptionTsFileEvent, Boolean> - generateSubscriptionTsFileEventWithPieceOrSealPayload(long writingOffset) throws IOException { + generateSubscriptionTsFileEventWithPieceOrSealPayload(final long writingOffset) + throws IOException { final PipeTsFileInsertionEvent tsFileInsertionEvent = (PipeTsFileInsertionEvent) this.getEnrichedEvents().get(0); final SubscriptionCommitContext commitContext = this.getMessage().getCommitContext(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java index 3f79cde94e06..dc136205e803 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java @@ -301,10 +301,10 @@ private TPipeSubscribeResp handlePipeSubscribePoll(final PipeSubscribePollReq re } try { - SubscriptionPollMessage pollMessage = req.getPollMessage(); - short messageType = pollMessage.getMessageType(); + final SubscriptionPollMessage pollMessage = req.getPollMessage(); + final short messageType = pollMessage.getMessageType(); - long timeoutMs = pollMessage.getTimeoutMs(); + final long timeoutMs = pollMessage.getTimeoutMs(); final SubscriptionPollTimer timer = new SubscriptionPollTimer( System.currentTimeMillis(), @@ -339,9 +339,9 @@ private TPipeSubscribeResp handlePipeSubscribePoll(final PipeSubscribePollReq re } private TPipeSubscribeResp handlePipeSubscribePollInternal( - ConsumerConfig consumerConfig, - PollMessagePayload messagePayload, - SubscriptionPollTimer timer) { + final ConsumerConfig consumerConfig, + final PollMessagePayload messagePayload, + final SubscriptionPollTimer timer) { Set topicNames = messagePayload.getTopicNames(); if (topicNames.isEmpty()) { // poll all subscribed topics @@ -384,11 +384,9 @@ private TPipeSubscribeResp handlePipeSubscribePollInternal( } private TPipeSubscribeResp handlePipeSubscribePollTsFileInternal( - ConsumerConfig consumerConfig, - PollTsFileMessagePayload messagePayload, - SubscriptionPollTimer timer) { - // TODO: timer - + final ConsumerConfig consumerConfig, + final PollTsFileMessagePayload messagePayload, + final SubscriptionPollTimer timer) { // poll final List events = SubscriptionAgent.broker() @@ -396,7 +394,8 @@ private TPipeSubscribeResp handlePipeSubscribePollTsFileInternal( consumerConfig, messagePayload.getTopicName(), messagePayload.getFileName(), - messagePayload.getWritingOffset()); + messagePayload.getWritingOffset(), + timer); final List polledMessages = events.stream().map(SubscriptionEvent::getMessage).collect(Collectors.toList()); From e797f67d557d7f585bb2671e93385f510b61f51c Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Sat, 20 Apr 2024 17:49:05 +0800 Subject: [PATCH 51/93] improve exception handling cont --- .../SubscriptionConnectionException.java | 2 +- .../subscription/ConsumerHeartbeatWorker.java | 2 +- .../subscription/SubscriptionConsumer.java | 154 +++++++++------ .../SubscriptionEndpointsSyncer.java | 2 +- .../subscription/SubscriptionProvider.java | 181 +++++++++++++++--- .../SubscriptionPullConsumer.java | 60 +++--- .../SubscriptionPushConsumer.java | 41 ++-- .../subscription/SubscriptionSession.java | 44 +++-- .../SubscriptionSessionConnection.java | 46 +---- .../SubscriptionSessionDataSet.java | 33 ++-- .../SubscriptionSessionDataSets.java | 2 +- .../SubscriptionTsFileReader.java | 2 +- 12 files changed, 353 insertions(+), 216 deletions(-) diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionConnectionException.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionConnectionException.java index 3ebf522e0b84..fd2ea63cdbc0 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionConnectionException.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionConnectionException.java @@ -21,7 +21,7 @@ import java.util.Objects; -public class SubscriptionConnectionException extends SubscriptionException { +public class SubscriptionConnectionException extends SubscriptionNonRetryableException { public SubscriptionConnectionException(String message) { super(message); diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/ConsumerHeartbeatWorker.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/ConsumerHeartbeatWorker.java index 12354ee7eeac..f803e0d71f8c 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/ConsumerHeartbeatWorker.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/ConsumerHeartbeatWorker.java @@ -49,7 +49,7 @@ public void run() { private void heartbeatInternal() { for (final SubscriptionProvider provider : consumer.getAllProviders()) { try { - provider.getSessionConnection().heartbeat(); + provider.heartbeat(); provider.setAvailable(); } catch (final Exception e) { LOGGER.warn( diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java index 245de436b25e..8d2a14d78de1 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java @@ -127,7 +127,9 @@ private static class SubscriptionTsFileInfo { int retryCount; SubscriptionTsFileInfo( - SubscriptionCommitContext commitContext, File file, RandomAccessFile fileWriter) { + final SubscriptionCommitContext commitContext, + final File file, + final RandomAccessFile fileWriter) { this.commitContext = commitContext; this.file = file; this.fileWriter = fileWriter; @@ -165,23 +167,24 @@ private Path getTsFileDir(final String topicName) throws IOException { } private SubscriptionTsFileInfo getSubscriptionTsFileInfoTsFileInfo( - SubscriptionCommitContext commitContext, String fileName) throws IOException { + final SubscriptionCommitContext commitContext, final String fileName) throws IOException { final String topicName = commitContext.getTopicName(); SubscriptionTsFileInfo info = topicNameToSubscriptionTsFileInfo.get(topicName); if (Objects.isNull(info)) { info = createSubscriptionTsFileInfoTsFileInfo(commitContext, fileName); } else { if (!info.file.exists()) { - try { - info.fileWriter.close(); - } catch (final IOException e) { - LOGGER.warn(e.getMessage()); - } LOGGER.info( - "consumer {} remove subscription TsFile info {} because the file does not exist", - this, - info); - topicNameToSubscriptionTsFileInfo.remove(topicName); + "file {} does not exist, remove corresponding subscription TsFile info...", fileName); + removeSubscriptionTsFileInfo(topicName); + info = createSubscriptionTsFileInfoTsFileInfo(commitContext, fileName); + } + if (!Objects.equals(info.file.getName(), fileName)) { + LOGGER.info( + "inconsistent file name, current is {}, incoming is {}, remove corresponding subscription TsFile info...", + info.file.getName(), + fileName); + removeSubscriptionTsFileInfo(topicName); info = createSubscriptionTsFileInfoTsFileInfo(commitContext, fileName); } } @@ -190,7 +193,7 @@ private SubscriptionTsFileInfo getSubscriptionTsFileInfoTsFileInfo( } private SubscriptionTsFileInfo createSubscriptionTsFileInfoTsFileInfo( - SubscriptionCommitContext commitContext, String fileName) throws IOException { + final SubscriptionCommitContext commitContext, final String fileName) throws IOException { final String topicName = commitContext.getTopicName(); final Path filePath = getTsFileDir(topicName).resolve(fileName); @@ -204,7 +207,7 @@ private SubscriptionTsFileInfo createSubscriptionTsFileInfoTsFileInfo( return info; } - private void removeSubscriptionTsFileInfo(String topicName) { + private void removeSubscriptionTsFileInfo(final String topicName) { final SubscriptionTsFileInfo info = topicNameToSubscriptionTsFileInfo.get(topicName); if (Objects.isNull(info)) { return; @@ -220,20 +223,29 @@ private void removeSubscriptionTsFileInfo(String topicName) { topicNameToSubscriptionTsFileInfo.remove(topicName); } - private void increaseSubscriptionTsFileInfoRetryCountOrRemove(String topicName) { + private void removeAllSubscriptionTsFileInfo() { + for (final String topicName : topicNameToSubscriptionTsFileInfo.keySet()) { + removeSubscriptionTsFileInfo(topicName); + } + } + + private void increaseSubscriptionTsFileInfoRetryCountOrRemove(final String topicName) { final SubscriptionTsFileInfo info = topicNameToSubscriptionTsFileInfo.get(topicName); if (Objects.isNull(info)) { return; } if (info.increaseRetryCountAndCheckIfExceedRetryLimit()) { + LOGGER.info( + "exceed retry limit {}, remove corresponding subscription TsFile info...", + POLL_TS_FILE_RETRY_LIMIT); removeSubscriptionTsFileInfo(topicName); } } /////////////////////////////// ctor /////////////////////////////// - protected SubscriptionConsumer(Builder builder) { + protected SubscriptionConsumer(final Builder builder) { this.initialEndpoints = new ArrayList<>(); // From org.apache.iotdb.session.Session.getNodeUrls // Priority is given to `host:port` over `nodeUrls`. @@ -255,7 +267,7 @@ protected SubscriptionConsumer(Builder builder) { this.tsFileBaseDir = builder.tsFileBaseDir; } - protected SubscriptionConsumer(Builder builder, Properties properties) { + protected SubscriptionConsumer(final Builder builder, final Properties properties) { this( builder .host( @@ -324,6 +336,9 @@ public synchronized void close() throws IoTDBConnectionException { } try { + // remove all subscription TsFile info + removeAllSubscriptionTsFileInfo(); + // shutdown endpoints syncer shutdownEndpointsSyncer(); @@ -366,17 +381,17 @@ void releaseWriteLock() { /////////////////////////////// subscribe & unsubscribe /////////////////////////////// - public void subscribe(String topicName) + public void subscribe(final String topicName) throws TException, IOException, StatementExecutionException, IoTDBConnectionException { subscribe(Collections.singleton(topicName)); } - public void subscribe(String... topicNames) + public void subscribe(final String... topicNames) throws TException, IOException, StatementExecutionException, IoTDBConnectionException { subscribe(new HashSet<>(Arrays.asList(topicNames))); } - public void subscribe(Set topicNames) + public void subscribe(final Set topicNames) throws TException, IOException, StatementExecutionException, IoTDBConnectionException { acquireReadLock(); try { @@ -386,17 +401,17 @@ public void subscribe(Set topicNames) } } - public void unsubscribe(String topicName) + public void unsubscribe(final String topicName) throws TException, IOException, StatementExecutionException, IoTDBConnectionException { unsubscribe(Collections.singleton(topicName)); } - public void unsubscribe(String... topicNames) + public void unsubscribe(final String... topicNames) throws TException, IOException, StatementExecutionException, IoTDBConnectionException { unsubscribe(new HashSet<>(Arrays.asList(topicNames))); } - public void unsubscribe(Set topicNames) + public void unsubscribe(final Set topicNames) throws TException, IOException, StatementExecutionException, IoTDBConnectionException { acquireReadLock(); try { @@ -413,7 +428,7 @@ private void launchHeartbeatWorker() { heartbeatWorkerExecutor = Executors.newSingleThreadScheduledExecutor( r -> { - Thread t = + final Thread t = new Thread( Thread.currentThread().getThreadGroup(), r, "ConsumerHeartbeatWorker", 0); if (!t.isDaemon()) { @@ -449,7 +464,7 @@ private void launchEndpointsSyncer() { endpointsSyncerExecutor = Executors.newSingleThreadScheduledExecutor( r -> { - Thread t = + final Thread t = new Thread( Thread.currentThread().getThreadGroup(), r, "SubscriptionEndpointsSyncer", 0); if (!t.isDaemon()) { @@ -590,7 +605,7 @@ SubscriptionProvider getProvider(final int dataNodeId) { /////////////////////////////// poll /////////////////////////////// - protected List poll(Set topicNames, long timeoutMs) + protected List poll(final Set topicNames, final long timeoutMs) throws SubscriptionException { final List messages = new ArrayList<>(); @@ -639,13 +654,13 @@ protected List poll(Set topicNames, long timeoutMs) } private Optional pollTsFile( - SubscriptionCommitContext commitContext, String fileName, long timeoutMs) + final SubscriptionCommitContext commitContext, final String fileName, final long timeoutMs) throws SubscriptionException { try { final SubscriptionMessage message = pollTsFileInternal(commitContext, fileName, timeoutMs); removeSubscriptionTsFileInfo(commitContext.getTopicName()); return Optional.of(message); - } catch (IOException | SubscriptionRetryableException e) { + } catch (final IOException | SubscriptionRetryableException e) { LOGGER.warn( "IOException or SubscriptionRetryableException occurred when SubscriptionConsumer {} polling TsFile {} with commit context {}: {}", this, @@ -654,7 +669,7 @@ private Optional pollTsFile( e.getMessage()); // assume retryable increaseSubscriptionTsFileInfoRetryCountOrRemove(commitContext.getTopicName()); - } catch (SubscriptionNonRetryableException e) { + } catch (final SubscriptionNonRetryableException e) { LOGGER.warn( "SubscriptionNonRetryableException occurred when SubscriptionConsumer {} polling TsFile {} with commit context {}: {}", this, @@ -663,13 +678,15 @@ private Optional pollTsFile( e.getMessage()); // assume non-retryable removeSubscriptionTsFileInfo(commitContext.getTopicName()); + // TODO: Consider mid-process failures. + // rethrow throw e; } return Optional.empty(); } private SubscriptionMessage pollTsFileInternal( - SubscriptionCommitContext commitContext, String fileName, long timeoutMs) + final SubscriptionCommitContext commitContext, final String fileName, final long timeoutMs) throws IOException, SubscriptionException { final int dataNodeId = commitContext.getDataNodeId(); final String topicName = commitContext.getTopicName(); @@ -690,6 +707,8 @@ private SubscriptionMessage pollTsFileInternal( final List polledMessages = pollTsFileInternal(dataNodeId, topicName, fileName, writingOffset, timeoutMs); + // It's agreed that the server will always return at least one message, even in case of + // failure. if (polledMessages.isEmpty()) { final String errorMessage = String.format("SubscriptionConsumer %s poll empty tsfile message", this); @@ -699,6 +718,8 @@ private SubscriptionMessage pollTsFileInternal( final SubscriptionPolledMessage polledMessage = polledMessages.get(0); final SubscriptionMessagePayload messagePayload = polledMessage.getMessagePayload(); + + // check commit context final SubscriptionCommitContext incomingCommitContext = polledMessage.getCommitContext(); if (Objects.isNull(incomingCommitContext) || !Objects.equals(commitContext, incomingCommitContext)) { @@ -818,8 +839,8 @@ private SubscriptionMessage pollTsFileInternal( } } - private List pollInternal(Set topicNames, long timeoutMs) - throws SubscriptionException { + private List pollInternal( + final Set topicNames, final long timeoutMs) throws SubscriptionException { final List polledMessages = new ArrayList<>(); acquireReadLock(); @@ -832,8 +853,22 @@ private List pollInternal(Set topicNames, lon SubscriptionPollMessageType.POLL.getType(), new PollMessagePayload(topicNames), timeoutMs))); - } catch (final Exception e) { - LOGGER.warn(e.getMessage()); + } catch (final SubscriptionRetryableException e) { + LOGGER.warn( + "SubscriptionRetryableException occurred when SubscriptionConsumer {} polling from SubscriptionProvider {}: {}", + this, + provider, + e.getMessage()); + // ignore + } catch (final SubscriptionNonRetryableException e) { + LOGGER.warn( + "SubscriptionNonRetryableException occurred when SubscriptionConsumer {} polling from SubscriptionProvider {}: {}", + this, + provider, + e.getMessage()); + // TODO: Consider mid-process failures. + // rethrow + throw e; } } } finally { @@ -844,7 +879,11 @@ private List pollInternal(Set topicNames, lon } private List pollTsFileInternal( - int dataNodeId, String topicName, String fileName, long writingOffset, long timeoutMs) + final int dataNodeId, + final String topicName, + final String fileName, + final long writingOffset, + final long timeoutMs) throws SubscriptionException { acquireReadLock(); try { @@ -867,22 +906,23 @@ private List pollTsFileInternal( /////////////////////////////// commit sync /////////////////////////////// - protected void commitSync(Iterable messages) throws SubscriptionException { - Map> dataNodeIdToSubscriptionCommitContexts = + protected void commitSync(final Iterable messages) + throws SubscriptionException { + final Map> dataNodeIdToSubscriptionCommitContexts = new HashMap<>(); - for (SubscriptionMessage message : messages) { + for (final SubscriptionMessage message : messages) { dataNodeIdToSubscriptionCommitContexts .computeIfAbsent(message.getCommitContext().getDataNodeId(), (id) -> new ArrayList<>()) .add(message.getCommitContext()); } - for (Map.Entry> entry : + for (final Map.Entry> entry : dataNodeIdToSubscriptionCommitContexts.entrySet()) { commitSyncInternal(entry.getKey(), entry.getValue()); } } private void commitSyncInternal( - int dataNodeId, List subscriptionCommitContexts) + final int dataNodeId, final List subscriptionCommitContexts) throws SubscriptionException { acquireReadLock(); try { @@ -901,11 +941,12 @@ private void commitSyncInternal( /////////////////////////////// commit async /////////////////////////////// - protected void commitAsync(Iterable messages) { + protected void commitAsync(final Iterable messages) { commitAsync(messages, new AsyncCommitCallback() {}); } - protected void commitAsync(Iterable messages, AsyncCommitCallback callback) { + protected void commitAsync( + final Iterable messages, final AsyncCommitCallback callback) { // Initiate executor if needed if (asyncCommitExecutor == null) { synchronized (this) { @@ -916,7 +957,7 @@ protected void commitAsync(Iterable messages, AsyncCommitCa asyncCommitExecutor = Executors.newSingleThreadExecutor( r -> { - Thread t = + final Thread t = new Thread( Thread.currentThread().getThreadGroup(), r, @@ -943,7 +984,7 @@ private void subscribeWithRedirection(final Set topicNames) throws IoTDBConnectionException { for (final SubscriptionProvider provider : getAllAvailableProviders()) { try { - provider.getSessionConnection().subscribe(topicNames); + provider.subscribe(topicNames); return; } catch (final Exception e) { LOGGER.warn( @@ -961,7 +1002,7 @@ private void unsubscribeWithRedirection(final Set topicNames) throws IoTDBConnectionException { for (final SubscriptionProvider provider : getAllAvailableProviders()) { try { - provider.getSessionConnection().unsubscribe(topicNames); + provider.unsubscribe(topicNames); return; } catch (final Exception e) { LOGGER.warn( @@ -1014,54 +1055,54 @@ public abstract static class Builder { protected String tsFileBaseDir = ConsumerConstant.TS_FILE_BASE_DIR_DEFAULT_VALUE; - public Builder host(String host) { + public Builder host(final String host) { this.host = host; return this; } - public Builder port(int port) { + public Builder port(final int port) { this.port = port; return this; } - public Builder nodeUrls(List nodeUrls) { + public Builder nodeUrls(final List nodeUrls) { this.nodeUrls = nodeUrls; return this; } - public Builder username(String username) { + public Builder username(final String username) { this.username = username; return this; } - public Builder password(String password) { + public Builder password(final String password) { this.password = password; return this; } - public Builder consumerId(String consumerId) { + public Builder consumerId(final String consumerId) { this.consumerId = consumerId; return this; } - public Builder consumerGroupId(String consumerGroupId) { + public Builder consumerGroupId(final String consumerGroupId) { this.consumerGroupId = consumerGroupId; return this; } - public Builder heartbeatIntervalMs(long heartbeatIntervalMs) { + public Builder heartbeatIntervalMs(final long heartbeatIntervalMs) { this.heartbeatIntervalMs = Math.max(heartbeatIntervalMs, ConsumerConstant.HEARTBEAT_INTERVAL_MS_MIN_VALUE); return this; } - public Builder endpointsSyncIntervalMs(long endpointsSyncIntervalMs) { + public Builder endpointsSyncIntervalMs(final long endpointsSyncIntervalMs) { this.endpointsSyncIntervalMs = Math.max(endpointsSyncIntervalMs, ConsumerConstant.ENDPOINTS_SYNC_INTERVAL_MS_MIN_VALUE); return this; } - public Builder tsFileBaseDir(String tsFileBaseDir) { + public Builder tsFileBaseDir(final String tsFileBaseDir) { this.tsFileBaseDir = tsFileBaseDir; return this; } @@ -1077,7 +1118,8 @@ class AsyncCommitWorker implements Runnable { private final Iterable messages; private final AsyncCommitCallback callback; - public AsyncCommitWorker(Iterable messages, AsyncCommitCallback callback) { + public AsyncCommitWorker( + final Iterable messages, final AsyncCommitCallback callback) { this.messages = messages; this.callback = callback; } @@ -1091,7 +1133,7 @@ public void run() { try { commitSync(messages); callback.onComplete(); - } catch (Exception e) { + } catch (final Exception e) { callback.onFailure(e); } } diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionEndpointsSyncer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionEndpointsSyncer.java index d7aea1981d5b..70d8aa9468cc 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionEndpointsSyncer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionEndpointsSyncer.java @@ -90,7 +90,7 @@ private void syncInternal() { } else { // existing provider try { - provider.getSessionConnection().heartbeat(); + provider.heartbeat(); provider.setAvailable(); } catch (final Exception e) { LOGGER.warn( diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionProvider.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionProvider.java index 77b5d1fb5d2c..878c6678203c 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionProvider.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionProvider.java @@ -22,7 +22,6 @@ import org.apache.iotdb.common.rpc.thrift.TEndPoint; import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.rpc.IoTDBConnectionException; -import org.apache.iotdb.rpc.StatementExecutionException; import org.apache.iotdb.rpc.subscription.config.ConsumerConfig; import org.apache.iotdb.rpc.subscription.config.ConsumerConstant; import org.apache.iotdb.rpc.subscription.exception.SubscriptionException; @@ -31,8 +30,14 @@ import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionCommitContext; import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPollMessage; import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPolledMessage; +import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribeCloseReq; import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribeCommitReq; +import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribeHandshakeReq; +import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribeHeartbeatReq; import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribePollReq; +import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribeSubscribeReq; +import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribeUnsubscribeReq; +import org.apache.iotdb.rpc.subscription.payload.response.PipeSubscribeHandshakeResp; import org.apache.iotdb.rpc.subscription.payload.response.PipeSubscribePollResp; import org.apache.iotdb.service.rpc.thrift.TPipeSubscribeResp; @@ -44,6 +49,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; final class SubscriptionProvider extends SubscriptionSession { @@ -60,11 +66,11 @@ final class SubscriptionProvider extends SubscriptionSession { private int dataNodeId; SubscriptionProvider( - TEndPoint endPoint, - String username, - String password, - String consumerId, - String consumerGroupId) { + final TEndPoint endPoint, + final String username, + final String password, + final String consumerId, + final String consumerGroupId) { super(endPoint.ip, endPoint.port, username, password); this.endPoint = endPoint; @@ -94,8 +100,7 @@ TEndPoint getEndPoint() { /////////////////////////////// open & close /////////////////////////////// - synchronized int handshake() - throws IoTDBConnectionException, TException, IOException, StatementExecutionException { + synchronized int handshake() throws SubscriptionException, IoTDBConnectionException { if (!isClosed.get()) { return -1; } @@ -105,7 +110,7 @@ synchronized int handshake() final Map consumerAttributes = new HashMap<>(); consumerAttributes.put(ConsumerConstant.CONSUMER_GROUP_ID_KEY, consumerGroupId); consumerAttributes.put(ConsumerConstant.CONSUMER_ID_KEY, consumerId); - dataNodeId = getSessionConnection().handshake(new ConsumerConfig(consumerAttributes)); + dataNodeId = handshake(new ConsumerConfig(consumerAttributes)); isClosed.set(false); setAvailable(); @@ -113,16 +118,13 @@ synchronized int handshake() } @Override - public synchronized void close() throws IoTDBConnectionException { + public synchronized void close() throws SubscriptionException, IoTDBConnectionException { if (isClosed.get()) { return; } try { - getSessionConnection().closeConsumer(); - } catch (TException | StatementExecutionException e) { - // wrap to IoTDBConnectionException to keep interface consistent - throw new IoTDBConnectionException(e); + closeInternal(); } finally { super.close(); setUnavailable(); @@ -134,14 +136,141 @@ SubscriptionSessionConnection getSessionConnection() { return (SubscriptionSessionConnection) defaultSessionConnection; } - List poll(SubscriptionPollMessage pollMessage) + public int handshake(final ConsumerConfig consumerConfig) throws SubscriptionException { + final PipeSubscribeHandshakeReq req; + try { + req = PipeSubscribeHandshakeReq.toTPipeSubscribeReq(consumerConfig); + } catch (final IOException e) { + LOGGER.warn( + "IOException occurred when SubscriptionProvider {} serialize handshake request {}: {}", + this, + consumerConfig, + e.getMessage()); + throw new SubscriptionRetryableException(e.getMessage(), e); + } + final TPipeSubscribeResp resp; + try { + resp = getSessionConnection().pipeSubscribe(req); + } catch (final TException e) { + // TODO: Distinguish between TTransportException, TApplicationException, and + // TProtocolException. + LOGGER.warn( + "TException occurred when SubscriptionProvider {} handshake with request {}: {}, set SubscriptionProvider unavailable", + this, + consumerConfig, + e.getMessage()); + setUnavailable(); + throw new SubscriptionNonRetryableException(e.getMessage(), e); + } + verifyPipeSubscribeSuccess(resp.status); + final PipeSubscribeHandshakeResp handshakeResp = + PipeSubscribeHandshakeResp.fromTPipeSubscribeResp(resp); + return handshakeResp.getDataNodeId(); + } + + public void closeInternal() throws SubscriptionException { + final TPipeSubscribeResp resp; + try { + resp = getSessionConnection().pipeSubscribe(PipeSubscribeCloseReq.toTPipeSubscribeReq()); + } catch (final TException e) { + // TODO: Distinguish between TTransportException, TApplicationException, and + // TProtocolException. + LOGGER.warn( + "TException occurred when SubscriptionProvider {} close: {}, set SubscriptionProvider unavailable", + this, + e.getMessage()); + setUnavailable(); + throw new SubscriptionNonRetryableException(e.getMessage(), e); + } + verifyPipeSubscribeSuccess(resp.status); + } + + public void heartbeat() throws SubscriptionException { + final TPipeSubscribeResp resp; + try { + resp = getSessionConnection().pipeSubscribe(PipeSubscribeHeartbeatReq.toTPipeSubscribeReq()); + } catch (final TException e) { + // TODO: Distinguish between TTransportException, TApplicationException, and + // TProtocolException. + LOGGER.warn( + "TException occurred when SubscriptionProvider {} heartbeat: {}, set SubscriptionProvider unavailable", + this, + e.getMessage()); + setUnavailable(); + throw new SubscriptionNonRetryableException(e.getMessage(), e); + } + verifyPipeSubscribeSuccess(resp.status); + } + + public void subscribe(final Set topicNames) throws SubscriptionException { + final PipeSubscribeSubscribeReq req; + try { + req = PipeSubscribeSubscribeReq.toTPipeSubscribeReq(topicNames); + } catch (final IOException e) { + LOGGER.warn( + "IOException occurred when SubscriptionProvider {} serialize subscribe request {}: {}", + this, + topicNames, + e.getMessage()); + throw new SubscriptionRetryableException(e.getMessage(), e); + } + final TPipeSubscribeResp resp; + try { + resp = getSessionConnection().pipeSubscribe(req); + } catch (final TException e) { + // TODO: Distinguish between TTransportException, TApplicationException, and + // TProtocolException. + LOGGER.warn( + "TException occurred when SubscriptionProvider {} subscribe with request {}: {}, set SubscriptionProvider unavailable", + this, + topicNames, + e.getMessage()); + setUnavailable(); + throw new SubscriptionNonRetryableException(e.getMessage(), e); + } + verifyPipeSubscribeSuccess(resp.status); + } + + public void unsubscribe(final Set topicNames) throws SubscriptionException { + final PipeSubscribeUnsubscribeReq req; + try { + req = PipeSubscribeUnsubscribeReq.toTPipeSubscribeReq(topicNames); + } catch (final IOException e) { + LOGGER.warn( + "IOException occurred when SubscriptionProvider {} serialize unsubscribe request {}: {}", + this, + topicNames, + e.getMessage()); + throw new SubscriptionRetryableException(e.getMessage(), e); + } + final TPipeSubscribeResp resp; + try { + resp = getSessionConnection().pipeSubscribe(req); + } catch (final TException e) { + // TODO: Distinguish between TTransportException, TApplicationException, and + // TProtocolException. + LOGGER.warn( + "TException occurred when SubscriptionProvider {} unsubscribe with request {}: {}, set SubscriptionProvider unavailable", + this, + topicNames, + e.getMessage()); + setUnavailable(); + throw new SubscriptionNonRetryableException(e.getMessage(), e); + } + verifyPipeSubscribeSuccess(resp.status); + } + + List poll(final SubscriptionPollMessage pollMessage) throws SubscriptionException { final PipeSubscribePollReq req; try { req = PipeSubscribePollReq.toTPipeSubscribeReq(pollMessage); } catch (final IOException e) { LOGGER.warn( - "IOException occurred when serialize poll request {}: {}", pollMessage, e.getMessage()); + "IOException occurred when SubscriptionProvider {} serialize poll request {}: {}", + this, + pollMessage, + e.getMessage()); throw new SubscriptionRetryableException(e.getMessage(), e); } final TPipeSubscribeResp resp; @@ -151,10 +280,10 @@ List poll(SubscriptionPollMessage pollMessage) // TODO: Distinguish between TTransportException, TApplicationException, and // TProtocolException. LOGGER.warn( - "TException occurred when poll with request {}: {}, set SubscriptionProvider {} unavailable", + "TException occurred when SubscriptionProvider {} poll with request {}: {}, set SubscriptionProvider unavailable", + this, pollMessage, - e.getMessage(), - true); + e.getMessage()); setUnavailable(); throw new SubscriptionNonRetryableException(e.getMessage(), e); } @@ -163,14 +292,15 @@ List poll(SubscriptionPollMessage pollMessage) return pollResp.getMessages(); } - void commitSync(List subscriptionCommitContexts) + void commitSync(final List subscriptionCommitContexts) throws SubscriptionException { final PipeSubscribeCommitReq req; try { req = PipeSubscribeCommitReq.toTPipeSubscribeReq(subscriptionCommitContexts); } catch (final IOException e) { LOGGER.warn( - "IOException occurred when serialize commit request {}: {}", + "IOException occurred when SubscriptionProvider {} serialize commit request {}: {}", + this, subscriptionCommitContexts, e.getMessage()); throw new SubscriptionRetryableException(e.getMessage(), e); @@ -182,17 +312,18 @@ void commitSync(List subscriptionCommitContexts) // TODO: Distinguish between TTransportException, TApplicationException, and // TProtocolException. LOGGER.warn( - "TException occurred when commit with request {}: {}, set SubscriptionProvider {} unavailable", + "TException occurred when SubscriptionProvider {} commit with request {}: {}, set SubscriptionProvider unavailable", + this, subscriptionCommitContexts, - e.getMessage(), - true); + e.getMessage()); setUnavailable(); throw new SubscriptionNonRetryableException(e.getMessage(), e); } verifyPipeSubscribeSuccess(resp.status); } - private static void verifyPipeSubscribeSuccess(TSStatus status) throws SubscriptionException { + private static void verifyPipeSubscribeSuccess(final TSStatus status) + throws SubscriptionException { switch (status.code) { case 200: // SUCCESS_STATUS return; diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java index 5146a8cb63ff..dd2ab632b3c8 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java @@ -62,14 +62,14 @@ boolean isClosed() { /////////////////////////////// ctor /////////////////////////////// - protected SubscriptionPullConsumer(SubscriptionPullConsumer.Builder builder) { + protected SubscriptionPullConsumer(final SubscriptionPullConsumer.Builder builder) { super(builder); this.autoCommit = builder.autoCommit; this.autoCommitIntervalMs = builder.autoCommitIntervalMs; } - public SubscriptionPullConsumer(Properties properties) { + public SubscriptionPullConsumer(final Properties properties) { this( properties, (Boolean) @@ -82,7 +82,7 @@ public SubscriptionPullConsumer(Properties properties) { } private SubscriptionPullConsumer( - Properties properties, boolean autoCommit, long autoCommitIntervalMs) { + final Properties properties, final boolean autoCommit, final long autoCommitIntervalMs) { super( new Builder().autoCommit(autoCommit).autoCommitIntervalMs(autoCommitIntervalMs), properties); @@ -130,20 +130,20 @@ public synchronized void close() throws IoTDBConnectionException { /////////////////////////////// poll & commit /////////////////////////////// - public List poll(Duration timeoutMs) throws SubscriptionException { + public List poll(final Duration timeoutMs) throws SubscriptionException { return poll(Collections.emptySet(), timeoutMs.toMillis()); } - public List poll(long timeoutMs) throws SubscriptionException { + public List poll(final long timeoutMs) throws SubscriptionException { return poll(Collections.emptySet(), timeoutMs); } - public List poll(Set topicNames, Duration timeoutMs) + public List poll(final Set topicNames, final Duration timeoutMs) throws SubscriptionException { return poll(topicNames, timeoutMs.toMillis()); } - public List poll(Set topicNames, long timeoutMs) + public List poll(final Set topicNames, final long timeoutMs) throws SubscriptionException { final List messages = super.poll(topicNames, timeoutMs); @@ -164,27 +164,29 @@ public List poll(Set topicNames, long timeoutMs) /////////////////////////////// commit /////////////////////////////// - public void commitSync(SubscriptionMessage message) throws SubscriptionException { + public void commitSync(final SubscriptionMessage message) throws SubscriptionException { super.commitSync(Collections.singletonList(message)); } - public void commitSync(Iterable messages) throws SubscriptionException { + public void commitSync(final Iterable messages) + throws SubscriptionException { super.commitSync(messages); } - public void commitAsync(SubscriptionMessage message) { + public void commitAsync(final SubscriptionMessage message) { super.commitAsync(Collections.singletonList(message)); } - public void commitAsync(Iterable messages) { + public void commitAsync(final Iterable messages) { super.commitAsync(messages); } - public void commitAsync(SubscriptionMessage message, AsyncCommitCallback callback) { + public void commitAsync(final SubscriptionMessage message, final AsyncCommitCallback callback) { super.commitAsync(Collections.singletonList(message), callback); } - public void commitAsync(Iterable messages, AsyncCommitCallback callback) { + public void commitAsync( + final Iterable messages, final AsyncCommitCallback callback) { super.commitAsync(messages, callback); } @@ -196,7 +198,7 @@ private void launchAutoCommitWorker() { autoCommitWorkerExecutor = Executors.newSingleThreadScheduledExecutor( r -> { - Thread t = + final Thread t = new Thread( Thread.currentThread().getThreadGroup(), r, @@ -220,7 +222,7 @@ private void shutdownAutoCommitWorker() { } private void commitAllUncommittedMessages() { - for (Map.Entry> entry : uncommittedMessages.entrySet()) { + for (final Map.Entry> entry : uncommittedMessages.entrySet()) { try { commitSync(entry.getValue()); uncommittedMessages.remove(entry.getKey()); @@ -237,62 +239,62 @@ public static class Builder extends SubscriptionConsumer.Builder { private boolean autoCommit = ConsumerConstant.AUTO_COMMIT_DEFAULT_VALUE; private long autoCommitIntervalMs = ConsumerConstant.AUTO_COMMIT_INTERVAL_MS_DEFAULT_VALUE; - public Builder host(String host) { + public Builder host(final String host) { super.host(host); return this; } - public Builder port(int port) { + public Builder port(final int port) { super.port(port); return this; } - public Builder nodeUrls(List nodeUrls) { + public Builder nodeUrls(final List nodeUrls) { super.nodeUrls(nodeUrls); return this; } - public Builder username(String username) { + public Builder username(final String username) { super.username(username); return this; } - public Builder password(String password) { + public Builder password(final String password) { super.password(password); return this; } - public Builder consumerId(String consumerId) { + public Builder consumerId(final String consumerId) { super.consumerId(consumerId); return this; } - public Builder consumerGroupId(String consumerGroupId) { + public Builder consumerGroupId(final String consumerGroupId) { super.consumerGroupId(consumerGroupId); return this; } - public Builder heartbeatIntervalMs(long heartbeatIntervalMs) { + public Builder heartbeatIntervalMs(final long heartbeatIntervalMs) { super.heartbeatIntervalMs(heartbeatIntervalMs); return this; } - public Builder endpointsSyncIntervalMs(long endpointsSyncIntervalMs) { + public Builder endpointsSyncIntervalMs(final long endpointsSyncIntervalMs) { super.endpointsSyncIntervalMs(endpointsSyncIntervalMs); return this; } - public Builder tsFileBaseDir(String tsFileBaseDir) { + public Builder tsFileBaseDir(final String tsFileBaseDir) { super.tsFileBaseDir(tsFileBaseDir); return this; } - public Builder autoCommit(boolean autoCommit) { + public Builder autoCommit(final boolean autoCommit) { this.autoCommit = autoCommit; return this; } - public Builder autoCommitIntervalMs(long autoCommitIntervalMs) { + public Builder autoCommitIntervalMs(final long autoCommitIntervalMs) { this.autoCommitIntervalMs = Math.max(autoCommitIntervalMs, ConsumerConstant.AUTO_COMMIT_INTERVAL_MS_MIN_VALUE); return this; @@ -319,13 +321,13 @@ public void run() { return; } - long currentTimestamp = System.currentTimeMillis(); + final long currentTimestamp = System.currentTimeMillis(); long index = currentTimestamp / autoCommitIntervalMs; if (currentTimestamp % autoCommitIntervalMs == 0) { index -= 1; } - for (Map.Entry> entry : + for (final Map.Entry> entry : uncommittedMessages.headMap(index).entrySet()) { try { commitSync(entry.getValue()); diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPushConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPushConsumer.java index a3e174d5651b..aaf7e111860b 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPushConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPushConsumer.java @@ -48,14 +48,14 @@ public class SubscriptionPushConsumer extends SubscriptionConsumer { private final AtomicBoolean isClosed = new AtomicBoolean(true); - protected SubscriptionPushConsumer(Builder builder) { + protected SubscriptionPushConsumer(final Builder builder) { super(builder); this.ackStrategy = builder.ackStrategy; this.consumeListener = builder.consumeListener; } - public SubscriptionPushConsumer(Properties config) { + public SubscriptionPushConsumer(final Properties config) { this( config, (AckStrategy) @@ -67,7 +67,9 @@ public SubscriptionPushConsumer(Properties config) { } private SubscriptionPushConsumer( - Properties config, AckStrategy ackStrategy, ConsumeListener consumeListener) { + final Properties config, + final AckStrategy ackStrategy, + final ConsumeListener consumeListener) { super(new Builder().ackStrategy(ackStrategy), config); this.ackStrategy = ackStrategy; @@ -115,7 +117,7 @@ private void launchAutoPollWorker() { autoPollWorkerExecutor = Executors.newSingleThreadScheduledExecutor( r -> { - Thread t = + final Thread t = new Thread(Thread.currentThread().getThreadGroup(), r, "PushConsumerWorker", 0); if (!t.isDaemon()) { t.setDaemon(true); @@ -144,57 +146,58 @@ public static class Builder extends SubscriptionConsumer.Builder { private AckStrategy ackStrategy = AckStrategy.defaultValue(); private ConsumeListener consumeListener = message -> ConsumeResult.SUCCESS; - public SubscriptionPushConsumer.Builder host(String host) { + public SubscriptionPushConsumer.Builder host(final String host) { super.host(host); return this; } - public SubscriptionPushConsumer.Builder port(int port) { + public SubscriptionPushConsumer.Builder port(final int port) { super.port(port); return this; } - public SubscriptionPushConsumer.Builder username(String username) { + public SubscriptionPushConsumer.Builder username(final String username) { super.username(username); return this; } - public SubscriptionPushConsumer.Builder password(String password) { + public SubscriptionPushConsumer.Builder password(final String password) { super.password(password); return this; } - public SubscriptionPushConsumer.Builder consumerId(String consumerId) { + public SubscriptionPushConsumer.Builder consumerId(final String consumerId) { super.consumerId(consumerId); return this; } - public SubscriptionPushConsumer.Builder consumerGroupId(String consumerGroupId) { + public SubscriptionPushConsumer.Builder consumerGroupId(final String consumerGroupId) { super.consumerGroupId(consumerGroupId); return this; } - public SubscriptionPushConsumer.Builder heartbeatIntervalMs(long heartbeatIntervalMs) { + public SubscriptionPushConsumer.Builder heartbeatIntervalMs(final long heartbeatIntervalMs) { super.heartbeatIntervalMs(heartbeatIntervalMs); return this; } - public SubscriptionPushConsumer.Builder endpointsSyncIntervalMs(long endpointsSyncIntervalMs) { + public SubscriptionPushConsumer.Builder endpointsSyncIntervalMs( + final long endpointsSyncIntervalMs) { super.endpointsSyncIntervalMs(endpointsSyncIntervalMs); return this; } - public SubscriptionPushConsumer.Builder tsFileBaseDir(String tsFileBaseDir) { + public SubscriptionPushConsumer.Builder tsFileBaseDir(final String tsFileBaseDir) { this.tsFileBaseDir = tsFileBaseDir; return this; } - public SubscriptionPushConsumer.Builder ackStrategy(AckStrategy ackStrategy) { + public SubscriptionPushConsumer.Builder ackStrategy(final AckStrategy ackStrategy) { this.ackStrategy = ackStrategy; return this; } - public SubscriptionPushConsumer.Builder consumeListener(ConsumeListener consumeListener) { + public SubscriptionPushConsumer.Builder consumeListener(final ConsumeListener consumeListener) { this.consumeListener = consumeListener; return this; } @@ -222,15 +225,15 @@ public void run() { try { // Poll all subscribed topics by passing an empty set - List pollResults = + final List pollResults = poll(Collections.emptySet(), ConsumerConstant.PUSH_CONSUMER_AUTO_POLL_TIME_OUT_MS); if (ackStrategy.equals(AckStrategy.BEFORE_CONSUME)) { commitSync(pollResults); } - for (SubscriptionMessage pollResult : pollResults) { - ConsumeResult consumeResult = consumeListener.onReceive(pollResult); + for (final SubscriptionMessage pollResult : pollResults) { + final ConsumeResult consumeResult = consumeListener.onReceive(pollResult); if (consumeResult.equals(ConsumeResult.FAILURE)) { LOGGER.warn("consumeListener failed when processing message: {}", pollResult); } @@ -240,7 +243,7 @@ public void run() { commitSync(pollResults); } - } catch (SubscriptionException e) { + } catch (final SubscriptionException e) { LOGGER.warn("Exception occurred when auto polling: ", e); } } diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionSession.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionSession.java index cb7c913b4405..41da7a3528f9 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionSession.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionSession.java @@ -45,11 +45,12 @@ public class SubscriptionSession extends Session { - public SubscriptionSession(String host, int port) { + public SubscriptionSession(final String host, final int port) { this(host, port, SessionConfig.DEFAULT_USER, SessionConfig.DEFAULT_PASSWORD); } - public SubscriptionSession(String host, int port, String username, String password) { + public SubscriptionSession( + final String host, final int port, final String username, final String password) { // TODO: more configs control super( new Session.Builder() @@ -65,7 +66,8 @@ public SubscriptionSession(String host, int port, String username, String passwo @Override public SessionConnection constructSessionConnection( - Session session, TEndPoint endpoint, ZoneId zoneId) throws IoTDBConnectionException { + final Session session, final TEndPoint endpoint, final ZoneId zoneId) + throws IoTDBConnectionException { if (Objects.isNull(endpoint)) { throw new SubscriptionParameterNotValidException( "Subscription session must be configured with an endpoint."); @@ -76,13 +78,13 @@ public SessionConnection constructSessionConnection( /////////////////////////////// topic /////////////////////////////// - public void createTopic(String topicName) + public void createTopic(final String topicName) throws IoTDBConnectionException, StatementExecutionException { final String sql = String.format("CREATE TOPIC %s", topicName); executeNonQueryStatement(sql); } - public void createTopic(String topicName, Properties properties) + public void createTopic(final String topicName, final Properties properties) throws IoTDBConnectionException, StatementExecutionException { if (properties.isEmpty()) { createTopic(topicName); @@ -105,7 +107,7 @@ public void createTopic(String topicName, Properties properties) executeNonQueryStatement(sql); } - public void dropTopic(String topicName) + public void dropTopic(final String topicName) throws IoTDBConnectionException, StatementExecutionException { final String sql = String.format("DROP TOPIC %s", topicName); executeNonQueryStatement(sql); @@ -113,16 +115,16 @@ public void dropTopic(String topicName) public Set getTopics() throws IoTDBConnectionException, StatementExecutionException { final String sql = "SHOW TOPICS"; - try (SessionDataSet dataSet = executeQueryStatement(sql)) { + try (final SessionDataSet dataSet = executeQueryStatement(sql)) { return convertDataSetToTopics(dataSet); } } - public Optional getTopic(String topicName) + public Optional getTopic(final String topicName) throws IoTDBConnectionException, StatementExecutionException { final String sql = String.format("SHOW TOPIC %s", topicName); - try (SessionDataSet dataSet = executeQueryStatement(sql)) { - Set topics = convertDataSetToTopics(dataSet); + try (final SessionDataSet dataSet = executeQueryStatement(sql)) { + final Set topics = convertDataSetToTopics(dataSet); if (topics.isEmpty()) { return Optional.empty(); } @@ -135,27 +137,27 @@ public Optional getTopic(String topicName) public Set getSubscriptions() throws IoTDBConnectionException, StatementExecutionException { final String sql = "SHOW SUBSCRIPTIONS"; - try (SessionDataSet dataSet = executeQueryStatement(sql)) { + try (final SessionDataSet dataSet = executeQueryStatement(sql)) { return convertDataSetToSubscriptions(dataSet); } } - public Set getSubscriptions(String topicName) + public Set getSubscriptions(final String topicName) throws IoTDBConnectionException, StatementExecutionException { final String sql = String.format("SHOW SUBSCRIPTIONS ON %s", topicName); - try (SessionDataSet dataSet = executeQueryStatement(sql)) { + try (final SessionDataSet dataSet = executeQueryStatement(sql)) { return convertDataSetToSubscriptions(dataSet); } } /////////////////////////////// utility /////////////////////////////// - public Set convertDataSetToTopics(SessionDataSet dataSet) + public Set convertDataSetToTopics(final SessionDataSet dataSet) throws IoTDBConnectionException, StatementExecutionException { - Set topics = new HashSet<>(); + final Set topics = new HashSet<>(); while (dataSet.hasNext()) { - RowRecord record = dataSet.next(); - List fields = record.getFields(); + final RowRecord record = dataSet.next(); + final List fields = record.getFields(); if (fields.size() != 2) { throw new SubscriptionException( String.format( @@ -167,12 +169,12 @@ public Set convertDataSetToTopics(SessionDataSet dataSet) return topics; } - public Set convertDataSetToSubscriptions(SessionDataSet dataSet) + public Set convertDataSetToSubscriptions(final SessionDataSet dataSet) throws IoTDBConnectionException, StatementExecutionException { - Set subscriptions = new HashSet<>(); + final Set subscriptions = new HashSet<>(); while (dataSet.hasNext()) { - RowRecord record = dataSet.next(); - List fields = record.getFields(); + final RowRecord record = dataSet.next(); + final List fields = record.getFields(); if (fields.size() != 3) { throw new SubscriptionException( String.format( diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionSessionConnection.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionSessionConnection.java index 8245770afdc7..135e1055f47c 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionSessionConnection.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionSessionConnection.java @@ -22,15 +22,7 @@ import org.apache.iotdb.common.rpc.thrift.TEndPoint; import org.apache.iotdb.isession.SessionDataSet; import org.apache.iotdb.rpc.IoTDBConnectionException; -import org.apache.iotdb.rpc.RpcUtils; import org.apache.iotdb.rpc.StatementExecutionException; -import org.apache.iotdb.rpc.subscription.config.ConsumerConfig; -import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribeCloseReq; -import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribeHandshakeReq; -import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribeHeartbeatReq; -import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribeSubscribeReq; -import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribeUnsubscribeReq; -import org.apache.iotdb.rpc.subscription.payload.response.PipeSubscribeHandshakeResp; import org.apache.iotdb.service.rpc.thrift.TPipeSubscribeReq; import org.apache.iotdb.service.rpc.thrift.TPipeSubscribeResp; import org.apache.iotdb.session.Session; @@ -38,12 +30,10 @@ import org.apache.thrift.TException; -import java.io.IOException; import java.time.ZoneId; import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Set; import java.util.function.Supplier; public class SubscriptionSessionConnection extends SessionConnection { @@ -87,41 +77,7 @@ public Map fetchAllEndPoints() return endPoints; } - public TPipeSubscribeResp pipeSubscribe(TPipeSubscribeReq req) throws TException { + public TPipeSubscribeResp pipeSubscribe(final TPipeSubscribeReq req) throws TException { return client.pipeSubscribe(req); } - - public int handshake(ConsumerConfig consumerConfig) - throws TException, IOException, StatementExecutionException { - TPipeSubscribeResp resp = - client.pipeSubscribe(PipeSubscribeHandshakeReq.toTPipeSubscribeReq(consumerConfig)); - RpcUtils.verifySuccess(resp.status); - PipeSubscribeHandshakeResp handshakeResp = - PipeSubscribeHandshakeResp.fromTPipeSubscribeResp(resp); - return handshakeResp.getDataNodeId(); - } - - public void heartbeat() throws TException, StatementExecutionException { - TPipeSubscribeResp resp = client.pipeSubscribe(PipeSubscribeHeartbeatReq.toTPipeSubscribeReq()); - RpcUtils.verifySuccess(resp.status); - } - - public void closeConsumer() throws TException, StatementExecutionException { - TPipeSubscribeResp resp = client.pipeSubscribe(PipeSubscribeCloseReq.toTPipeSubscribeReq()); - RpcUtils.verifySuccess(resp.status); - } - - public void subscribe(Set topicNames) - throws TException, IOException, StatementExecutionException { - TPipeSubscribeResp resp = - client.pipeSubscribe(PipeSubscribeSubscribeReq.toTPipeSubscribeReq(topicNames)); - RpcUtils.verifySuccess(resp.status); - } - - public void unsubscribe(Set topicNames) - throws TException, IOException, StatementExecutionException { - TPipeSubscribeResp resp = - client.pipeSubscribe(PipeSubscribeUnsubscribeReq.toTPipeSubscribeReq(topicNames)); - RpcUtils.verifySuccess(resp.status); - } } diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionSessionDataSet.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionSessionDataSet.java index ff73fa4139b2..803936c6c3a0 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionSessionDataSet.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionSessionDataSet.java @@ -45,7 +45,7 @@ public Tablet getTablet() { return tablet; } - public SubscriptionSessionDataSet(Tablet tablet) { + public SubscriptionSessionDataSet(final Tablet tablet) { this.tablet = tablet; generateRowIterator(); } @@ -64,8 +64,8 @@ public List getColumnNames() { columnNameList = new ArrayList<>(); columnNameList.add("Time"); - String deviceId = tablet.deviceId; - List schemas = tablet.getSchemas(); + final String deviceId = tablet.deviceId; + final List schemas = tablet.getSchemas(); columnNameList.addAll( schemas.stream() .map((schema) -> deviceId + "." + schema.getMeasurementId()) @@ -82,7 +82,7 @@ public List getColumnTypes() { columnTypeList = new ArrayList<>(); columnTypeList.add(TSDataType.INT64.toString()); - List schemas = tablet.getSchemas(); + final List schemas = tablet.getSchemas(); columnTypeList.addAll( schemas.stream().map((schema) -> schema.getType().toString()).collect(Collectors.toList())); return columnTypeList; @@ -94,7 +94,7 @@ public boolean hasNext() { @Override public RowRecord next() { - Map.Entry entry = this.rowIterator.next(); + final Map.Entry entry = this.rowIterator.next(); final int columnSize = getColumnSize(); final List fields = new ArrayList<>(); @@ -106,7 +106,7 @@ public RowRecord next() { if (tablet.bitMaps[columnIndex].isMarked(rowIndex)) { field = new Field(null); } else { - TSDataType dataType = tablet.getSchemas().get(columnIndex).getType(); + final TSDataType dataType = tablet.getSchemas().get(columnIndex).getType(); field = generateFieldFromTabletValue(dataType, tablet.values[columnIndex], rowIndex); } fields.add(field); @@ -130,41 +130,42 @@ private int getColumnSize() { private void generateRowIterator() { // timestamp -> row index - long[] timestamps = tablet.timestamps; - TreeMap timestampToRowIndex = new TreeMap<>(); + final long[] timestamps = tablet.timestamps; + final TreeMap timestampToRowIndex = new TreeMap<>(); final int rowSize = timestamps.length; for (int rowIndex = 0; rowIndex < rowSize; ++rowIndex) { - Long timestamp = timestamps[rowIndex]; + final Long timestamp = timestamps[rowIndex]; timestampToRowIndex.put(timestamp, rowIndex); } this.rowIterator = timestampToRowIndex.entrySet().iterator(); } - private static Field generateFieldFromTabletValue(TSDataType dataType, Object value, int index) { + private static Field generateFieldFromTabletValue( + final TSDataType dataType, final Object value, final int index) { final Field field = new Field(dataType); switch (dataType) { case BOOLEAN: - boolean booleanValue = ((boolean[]) value)[index]; + final boolean booleanValue = ((boolean[]) value)[index]; field.setBoolV(booleanValue); break; case INT32: - int intValue = ((int[]) value)[index]; + final int intValue = ((int[]) value)[index]; field.setIntV(intValue); break; case INT64: - long longValue = ((long[]) value)[index]; + final long longValue = ((long[]) value)[index]; field.setLongV(longValue); break; case FLOAT: - float floatValue = ((float[]) value)[index]; + final float floatValue = ((float[]) value)[index]; field.setFloatV(floatValue); break; case DOUBLE: - double doubleValue = ((double[]) value)[index]; + final double doubleValue = ((double[]) value)[index]; field.setDoubleV(doubleValue); break; case TEXT: - Binary binaryValue = new Binary((((Binary[]) value)[index]).getValues()); + final Binary binaryValue = new Binary((((Binary[]) value)[index]).getValues()); field.setBinaryV(binaryValue); break; default: diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionSessionDataSets.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionSessionDataSets.java index 13abcc42f87a..ea88bd8e6fe9 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionSessionDataSets.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionSessionDataSets.java @@ -32,7 +32,7 @@ public class SubscriptionSessionDataSets private final List tablets; - public SubscriptionSessionDataSets(List tablets) { + public SubscriptionSessionDataSets(final List tablets) { this.dataSetList = new ArrayList<>(); this.tablets = tablets; tablets.forEach((tablet -> this.dataSetList.add(new SubscriptionSessionDataSet(tablet)))); diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionTsFileReader.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionTsFileReader.java index f03e52b73c48..44ac79bc5885 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionTsFileReader.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionTsFileReader.java @@ -28,7 +28,7 @@ public class SubscriptionTsFileReader implements SubscriptionMessagePayload { private final String filePath; - public SubscriptionTsFileReader(String filePath) { + public SubscriptionTsFileReader(final String filePath) { this.filePath = filePath; } From 1e9c3b7c6a746c08aaba37ea999b8afeb073e316 Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Mon, 22 Apr 2024 23:30:14 +0800 Subject: [PATCH 52/93] improve --- .../subscription/SubscriptionProvider.java | 50 ++++++++++--------- 1 file changed, 26 insertions(+), 24 deletions(-) diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionProvider.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionProvider.java index 878c6678203c..6cefcff664dc 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionProvider.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionProvider.java @@ -98,6 +98,10 @@ TEndPoint getEndPoint() { return endPoint; } + SubscriptionSessionConnection getSessionConnection() { + return (SubscriptionSessionConnection) defaultSessionConnection; + } + /////////////////////////////// open & close /////////////////////////////// synchronized int handshake() throws SubscriptionException, IoTDBConnectionException { @@ -117,26 +121,7 @@ synchronized int handshake() throws SubscriptionException, IoTDBConnectionExcept return dataNodeId; } - @Override - public synchronized void close() throws SubscriptionException, IoTDBConnectionException { - if (isClosed.get()) { - return; - } - - try { - closeInternal(); - } finally { - super.close(); - setUnavailable(); - isClosed.set(true); - } - } - - SubscriptionSessionConnection getSessionConnection() { - return (SubscriptionSessionConnection) defaultSessionConnection; - } - - public int handshake(final ConsumerConfig consumerConfig) throws SubscriptionException { + int handshake(final ConsumerConfig consumerConfig) throws SubscriptionException { final PipeSubscribeHandshakeReq req; try { req = PipeSubscribeHandshakeReq.toTPipeSubscribeReq(consumerConfig); @@ -168,7 +153,22 @@ public int handshake(final ConsumerConfig consumerConfig) throws SubscriptionExc return handshakeResp.getDataNodeId(); } - public void closeInternal() throws SubscriptionException { + @Override + public synchronized void close() throws SubscriptionException, IoTDBConnectionException { + if (isClosed.get()) { + return; + } + + try { + closeInternal(); + } finally { + super.close(); + setUnavailable(); + isClosed.set(true); + } + } + + void closeInternal() throws SubscriptionException { final TPipeSubscribeResp resp; try { resp = getSessionConnection().pipeSubscribe(PipeSubscribeCloseReq.toTPipeSubscribeReq()); @@ -185,7 +185,9 @@ public void closeInternal() throws SubscriptionException { verifyPipeSubscribeSuccess(resp.status); } - public void heartbeat() throws SubscriptionException { + /////////////////////////////// subscription APIs /////////////////////////////// + + void heartbeat() throws SubscriptionException { final TPipeSubscribeResp resp; try { resp = getSessionConnection().pipeSubscribe(PipeSubscribeHeartbeatReq.toTPipeSubscribeReq()); @@ -202,7 +204,7 @@ public void heartbeat() throws SubscriptionException { verifyPipeSubscribeSuccess(resp.status); } - public void subscribe(final Set topicNames) throws SubscriptionException { + void subscribe(final Set topicNames) throws SubscriptionException { final PipeSubscribeSubscribeReq req; try { req = PipeSubscribeSubscribeReq.toTPipeSubscribeReq(topicNames); @@ -231,7 +233,7 @@ public void subscribe(final Set topicNames) throws SubscriptionException verifyPipeSubscribeSuccess(resp.status); } - public void unsubscribe(final Set topicNames) throws SubscriptionException { + void unsubscribe(final Set topicNames) throws SubscriptionException { final PipeSubscribeUnsubscribeReq req; try { req = PipeSubscribeUnsubscribeReq.toTPipeSubscribeReq(topicNames); From 844decdf3093e0aaac887cf45806b3b94be34857 Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Tue, 23 Apr 2024 12:07:00 +0800 Subject: [PATCH 53/93] simplify on the fly tsfile & consider duplication --- .../subscription/ConsumerHeartbeatWorker.java | 4 +- .../subscription/SubscriptionConsumer.java | 260 +++++------------- .../SubscriptionEndpointsSyncer.java | 19 +- .../subscription/SubscriptionProvider.java | 48 ++-- .../SubscriptionTsFileReader.java | 5 + .../SubscriptionPrefetchingTsFileQueue.java | 38 ++- .../receiver/SubscriptionReceiverV1.java | 12 +- .../iotdb/commons/conf/CommonConfig.java | 2 +- 8 files changed, 156 insertions(+), 232 deletions(-) diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/ConsumerHeartbeatWorker.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/ConsumerHeartbeatWorker.java index f803e0d71f8c..b792b1752b25 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/ConsumerHeartbeatWorker.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/ConsumerHeartbeatWorker.java @@ -53,9 +53,9 @@ private void heartbeatInternal() { provider.setAvailable(); } catch (final Exception e) { LOGGER.warn( - "something unexpected happened when sending heartbeat to subscription provider {}, exception: {}, set subscription provider unavailable", + "something unexpected happened when sending heartbeat to subscription provider {}, set subscription provider unavailable", provider, - e.getMessage()); + e); provider.setUnavailable(); } } diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java index 8d2a14d78de1..94386b4741fa 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java @@ -50,6 +50,7 @@ import java.io.File; import java.io.IOException; import java.io.RandomAccessFile; +import java.nio.file.FileAlreadyExistsException; import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; @@ -104,61 +105,6 @@ public abstract class SubscriptionConsumer implements AutoCloseable { private final String tsFileBaseDir; - public String getConsumerId() { - return consumerId; - } - - public String getConsumerGroupId() { - return consumerGroupId; - } - - /////////////////////////////// tsfile /////////////////////////////// - - private static final int POLL_TS_FILE_RETRY_LIMIT = 3; - - private final Map topicNameToSubscriptionTsFileInfo = - new ConcurrentHashMap<>(); - - private static class SubscriptionTsFileInfo { - - SubscriptionCommitContext commitContext; - File file; - RandomAccessFile fileWriter; - int retryCount; - - SubscriptionTsFileInfo( - final SubscriptionCommitContext commitContext, - final File file, - final RandomAccessFile fileWriter) { - this.commitContext = commitContext; - this.file = file; - this.fileWriter = fileWriter; - this.retryCount = 0; - } - - String getTopicName() { - return commitContext.getTopicName(); - } - - /** @return {@code true} if exceed retry limit */ - boolean increaseRetryCountAndCheckIfExceedRetryLimit() { - retryCount++; - return retryCount > POLL_TS_FILE_RETRY_LIMIT; - } - - @Override - public String toString() { - return "SubscriptionTsFileInfo{" - + "commitContext=" - + commitContext - + ", file=" - + file.getAbsoluteFile() - + ", retryCount=" - + retryCount - + "}"; - } - } - private Path getTsFileDir(final String topicName) throws IOException { final Path dirPath = Paths.get(tsFileBaseDir).resolve(consumerGroupId).resolve(consumerId).resolve(topicName); @@ -166,81 +112,12 @@ private Path getTsFileDir(final String topicName) throws IOException { return dirPath; } - private SubscriptionTsFileInfo getSubscriptionTsFileInfoTsFileInfo( - final SubscriptionCommitContext commitContext, final String fileName) throws IOException { - final String topicName = commitContext.getTopicName(); - SubscriptionTsFileInfo info = topicNameToSubscriptionTsFileInfo.get(topicName); - if (Objects.isNull(info)) { - info = createSubscriptionTsFileInfoTsFileInfo(commitContext, fileName); - } else { - if (!info.file.exists()) { - LOGGER.info( - "file {} does not exist, remove corresponding subscription TsFile info...", fileName); - removeSubscriptionTsFileInfo(topicName); - info = createSubscriptionTsFileInfoTsFileInfo(commitContext, fileName); - } - if (!Objects.equals(info.file.getName(), fileName)) { - LOGGER.info( - "inconsistent file name, current is {}, incoming is {}, remove corresponding subscription TsFile info...", - info.file.getName(), - fileName); - removeSubscriptionTsFileInfo(topicName); - info = createSubscriptionTsFileInfoTsFileInfo(commitContext, fileName); - } - } - - return info; - } - - private SubscriptionTsFileInfo createSubscriptionTsFileInfoTsFileInfo( - final SubscriptionCommitContext commitContext, final String fileName) throws IOException { - final String topicName = commitContext.getTopicName(); - final Path filePath = getTsFileDir(topicName).resolve(fileName); - - Files.createFile(filePath); - final File file = filePath.toFile(); - final RandomAccessFile fileWriter = new RandomAccessFile(file, "rw"); - - final SubscriptionTsFileInfo info = new SubscriptionTsFileInfo(commitContext, file, fileWriter); - topicNameToSubscriptionTsFileInfo.put(topicName, info); - LOGGER.info("consumer {} create subscription TsFile info {}", this, info); - return info; - } - - private void removeSubscriptionTsFileInfo(final String topicName) { - final SubscriptionTsFileInfo info = topicNameToSubscriptionTsFileInfo.get(topicName); - if (Objects.isNull(info)) { - return; - } - - try { - info.fileWriter.close(); - } catch (final IOException e) { - LOGGER.warn(e.getMessage()); - } - - LOGGER.info("consumer {} remove subscription TsFile info {}", this, info); - topicNameToSubscriptionTsFileInfo.remove(topicName); - } - - private void removeAllSubscriptionTsFileInfo() { - for (final String topicName : topicNameToSubscriptionTsFileInfo.keySet()) { - removeSubscriptionTsFileInfo(topicName); - } + public String getConsumerId() { + return consumerId; } - private void increaseSubscriptionTsFileInfoRetryCountOrRemove(final String topicName) { - final SubscriptionTsFileInfo info = topicNameToSubscriptionTsFileInfo.get(topicName); - if (Objects.isNull(info)) { - return; - } - - if (info.increaseRetryCountAndCheckIfExceedRetryLimit()) { - LOGGER.info( - "exceed retry limit {}, remove corresponding subscription TsFile info...", - POLL_TS_FILE_RETRY_LIMIT); - removeSubscriptionTsFileInfo(topicName); - } + public String getConsumerGroupId() { + return consumerGroupId; } /////////////////////////////// ctor /////////////////////////////// @@ -336,9 +213,6 @@ public synchronized void close() throws IoTDBConnectionException { } try { - // remove all subscription TsFile info - removeAllSubscriptionTsFileInfo(); - // shutdown endpoints syncer shutdownEndpointsSyncer(); @@ -504,7 +378,7 @@ void openProviders() throws IoTDBConnectionException { defaultProvider = constructProvider(endPoint); defaultDataNodeId = defaultProvider.handshake(); } catch (final Exception e) { - LOGGER.warn("Failed to create connection with {}, exception: {}", endPoint, e.getMessage()); + LOGGER.warn("Failed to create connection with {}", endPoint, e); continue; // try next endpoint } addProvider(defaultDataNodeId, defaultProvider); @@ -513,10 +387,7 @@ void openProviders() throws IoTDBConnectionException { try { allEndPoints = defaultProvider.getSessionConnection().fetchAllEndPoints(); } catch (final Exception e) { - LOGGER.warn( - "Failed to fetch all endpoints from {}, exception: {}, will retry later...", - endPoint, - e.getMessage()); + LOGGER.warn("Failed to fetch all endpoints from {}, will retry later...", endPoint, e); break; // retry later } @@ -531,9 +402,7 @@ void openProviders() throws IoTDBConnectionException { provider.handshake(); } catch (final Exception e) { LOGGER.warn( - "Failed to create connection with {}, exception: {}, will retry later...", - entry.getValue(), - e.getMessage()); + "Failed to create connection with {}, will retry later...", entry.getValue(), e); continue; // retry later } addProvider(entry.getKey(), provider); @@ -609,20 +478,6 @@ protected List poll(final Set topicNames, final lon throws SubscriptionException { final List messages = new ArrayList<>(); - // poll on the fly tsfile - for (final SubscriptionTsFileInfo info : - topicNameToSubscriptionTsFileInfo.values().stream() - .filter( - info -> { - if (topicNames.isEmpty()) { - return true; - } - return topicNames.contains(info.getTopicName()); - }) - .collect(Collectors.toList())) { - pollTsFile(info.commitContext, info.file.getName(), timeoutMs).ifPresent(messages::add); - } - // poll tablets or tsfile for (final SubscriptionPolledMessage polledMessage : pollInternal(topicNames, timeoutMs)) { final short messageType = polledMessage.getMessageType(); @@ -654,47 +509,81 @@ protected List poll(final Set topicNames, final lon } private Optional pollTsFile( - final SubscriptionCommitContext commitContext, final String fileName, final long timeoutMs) + final SubscriptionCommitContext commitContext, String fileName, final long timeoutMs) throws SubscriptionException { + final String topicName = commitContext.getTopicName(); + Path filePath; + try { - final SubscriptionMessage message = pollTsFileInternal(commitContext, fileName, timeoutMs); - removeSubscriptionTsFileInfo(commitContext.getTopicName()); - return Optional.of(message); + filePath = getTsFileDir(topicName).resolve(fileName); + Files.createFile(filePath); + } catch (final FileAlreadyExistsException fileAlreadyExistsException) { + LOGGER.info( + "FileAlreadyExistsException occurred when SubscriptionConsumer {} polling TsFile {} with commit context {}, append \".1\" to file name", + this, + fileName, + commitContext, + fileAlreadyExistsException); + fileName += ".1"; + try { + filePath = getTsFileDir(topicName).resolve(fileName); + Files.createFile(filePath); + } catch (final IOException e) { + LOGGER.warn( + "IOException occurred when SubscriptionConsumer {} polling TsFile {} with commit context {}", + this, + fileName, + commitContext, + e); + // TODO: Consider mid-process failures. + // rethrow + throw new SubscriptionNonRetryableException(e.getMessage(), e); + } + } catch (final IOException e) { + LOGGER.warn( + "IOException occurred when SubscriptionConsumer {} polling TsFile {} with commit context {}", + this, + fileName, + commitContext, + e); + // TODO: Consider mid-process failures. + // rethrow + throw new SubscriptionNonRetryableException(e.getMessage(), e); + } + + final File file = filePath.toFile(); + try (final RandomAccessFile fileWriter = new RandomAccessFile(file, "rw")) { + return Optional.of(pollTsFileInternal(commitContext, file, fileWriter, timeoutMs)); } catch (final IOException | SubscriptionRetryableException e) { LOGGER.warn( - "IOException or SubscriptionRetryableException occurred when SubscriptionConsumer {} polling TsFile {} with commit context {}: {}", + "IOException or SubscriptionRetryableException occurred when SubscriptionConsumer {} polling TsFile {} with commit context {}", this, fileName, commitContext, - e.getMessage()); - // assume retryable - increaseSubscriptionTsFileInfoRetryCountOrRemove(commitContext.getTopicName()); + e); + return Optional.empty(); } catch (final SubscriptionNonRetryableException e) { LOGGER.warn( - "SubscriptionNonRetryableException occurred when SubscriptionConsumer {} polling TsFile {} with commit context {}: {}", + "SubscriptionNonRetryableException occurred when SubscriptionConsumer {} polling TsFile {} with commit context {}", this, fileName, commitContext, - e.getMessage()); - // assume non-retryable - removeSubscriptionTsFileInfo(commitContext.getTopicName()); + e); // TODO: Consider mid-process failures. // rethrow throw e; } - return Optional.empty(); } private SubscriptionMessage pollTsFileInternal( - final SubscriptionCommitContext commitContext, final String fileName, final long timeoutMs) + final SubscriptionCommitContext commitContext, + final File file, + final RandomAccessFile fileWriter, + final long timeoutMs) throws IOException, SubscriptionException { final int dataNodeId = commitContext.getDataNodeId(); final String topicName = commitContext.getTopicName(); - - final SubscriptionTsFileInfo info = - getSubscriptionTsFileInfoTsFileInfo(commitContext, fileName); - final File file = info.file; - final RandomAccessFile fileWriter = info.fileWriter; + final String fileName = file.getName(); LOGGER.info( "{} start to poll TsFile {} with commit context {}", @@ -737,8 +626,8 @@ private SubscriptionMessage pollTsFileInternal( case TS_FILE_PIECE: { // check file name - if (!Objects.equals( - fileName, ((TsFilePieceMessagePayload) messagePayload).getFileName())) { + if (!fileName.startsWith( + ((TsFilePieceMessagePayload) messagePayload).getFileName())) { final String errorMessage = String.format( "inconsistent file name, current is %s, incoming is %s, consumer: %s", @@ -772,8 +661,7 @@ private SubscriptionMessage pollTsFileInternal( case TS_FILE_SEAL: { // check file name - if (!Objects.equals( - fileName, ((TsFileSealMessagePayload) messagePayload).getFileName())) { + if (!fileName.startsWith(((TsFileSealMessagePayload) messagePayload).getFileName())) { final String errorMessage = String.format( "inconsistent file name, current is %s, incoming is %s, consumer: %s", @@ -855,17 +743,17 @@ private List pollInternal( timeoutMs))); } catch (final SubscriptionRetryableException e) { LOGGER.warn( - "SubscriptionRetryableException occurred when SubscriptionConsumer {} polling from SubscriptionProvider {}: {}", + "SubscriptionRetryableException occurred when SubscriptionConsumer {} polling from SubscriptionProvider {}", this, provider, - e.getMessage()); + e); // ignore } catch (final SubscriptionNonRetryableException e) { LOGGER.warn( - "SubscriptionNonRetryableException occurred when SubscriptionConsumer {} polling from SubscriptionProvider {}: {}", + "SubscriptionNonRetryableException occurred when SubscriptionConsumer {} polling from SubscriptionProvider {}", this, provider, - e.getMessage()); + e); // TODO: Consider mid-process failures. // rethrow throw e; @@ -988,10 +876,10 @@ private void subscribeWithRedirection(final Set topicNames) return; } catch (final Exception e) { LOGGER.warn( - "Failed to subscribe topics {} from subscription provider {}, exception: {}, try next subscription provider...", + "Failed to subscribe topics {} from subscription provider {}, try next subscription provider...", topicNames, provider, - e.getMessage()); + e); } } throw NO_PROVIDERS_EXCEPTION; @@ -1006,10 +894,10 @@ private void unsubscribeWithRedirection(final Set topicNames) return; } catch (final Exception e) { LOGGER.warn( - "Failed to unsubscribe topics {} from subscription provider {}, exception: {}, try next subscription provider...", + "Failed to unsubscribe topics {} from subscription provider {}, try next subscription provider...", topicNames, provider, - e.getMessage()); + e); } } throw NO_PROVIDERS_EXCEPTION; @@ -1024,9 +912,9 @@ Map fetchAllEndPointsWithRedirection() throws IoTDBConnectio break; } catch (final Exception e) { LOGGER.warn( - "Failed to fetch all endpoints from subscription provider {}, exception: {}, try next subscription provider...", + "Failed to fetch all endpoints from subscription provider {}, try next subscription provider...", provider, - e.getMessage()); + e); } } if (Objects.isNull(endPoints)) { diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionEndpointsSyncer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionEndpointsSyncer.java index 70d8aa9468cc..dc7acc94d79e 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionEndpointsSyncer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionEndpointsSyncer.java @@ -66,8 +66,7 @@ private void syncInternal() { try { allEndPoints = consumer.fetchAllEndPointsWithRedirection(); } catch (final Exception e) { - LOGGER.warn( - "Failed to fetch all endpoints, exception: {}, will retry later...", e.getMessage()); + LOGGER.warn("Failed to fetch all endpoints, will retry later...", e); return; // retry later } @@ -81,9 +80,9 @@ private void syncInternal() { newProvider.handshake(); } catch (final Exception e) { LOGGER.warn( - "Failed to create connection with subscription provider {}, exception: {}, will retry later...", + "Failed to create connection with subscription provider {}, will retry later...", newProvider, - e.getMessage()); + e); continue; // retry later } consumer.addProvider(entry.getKey(), newProvider); @@ -94,9 +93,9 @@ private void syncInternal() { provider.setAvailable(); } catch (final Exception e) { LOGGER.warn( - "something unexpected happened when sending heartbeat to subscription provider {}, exception: {}, set subscription provider unavailable", + "something unexpected happened when sending heartbeat to subscription provider {}, set subscription provider unavailable", provider, - e.getMessage()); + e); provider.setUnavailable(); } // close and remove unavailable provider (reset the connection as much as possible) @@ -105,9 +104,9 @@ private void syncInternal() { consumer.closeAndRemoveProvider(entry.getKey()); } catch (final IoTDBConnectionException e) { LOGGER.warn( - "Exception occurred when closing and removing subscription provider with data node id {}: {}", + "Exception occurred when closing and removing subscription provider with data node id {}", entry.getKey(), - e.getMessage()); + e); } } } @@ -121,9 +120,9 @@ private void syncInternal() { consumer.closeAndRemoveProvider(dataNodeId); } catch (final IoTDBConnectionException e) { LOGGER.warn( - "Exception occurred when closing and removing subscription provider with data node id {}: {}", + "Exception occurred when closing and removing subscription provider with data node id {}", dataNodeId, - e.getMessage()); + e); } } } diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionProvider.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionProvider.java index 6cefcff664dc..2b7d1a982d9d 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionProvider.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionProvider.java @@ -127,10 +127,10 @@ int handshake(final ConsumerConfig consumerConfig) throws SubscriptionException req = PipeSubscribeHandshakeReq.toTPipeSubscribeReq(consumerConfig); } catch (final IOException e) { LOGGER.warn( - "IOException occurred when SubscriptionProvider {} serialize handshake request {}: {}", + "IOException occurred when SubscriptionProvider {} serialize handshake request {}", this, consumerConfig, - e.getMessage()); + e); throw new SubscriptionRetryableException(e.getMessage(), e); } final TPipeSubscribeResp resp; @@ -140,10 +140,10 @@ int handshake(final ConsumerConfig consumerConfig) throws SubscriptionException // TODO: Distinguish between TTransportException, TApplicationException, and // TProtocolException. LOGGER.warn( - "TException occurred when SubscriptionProvider {} handshake with request {}: {}, set SubscriptionProvider unavailable", + "TException occurred when SubscriptionProvider {} handshake with request {}, set SubscriptionProvider unavailable", this, consumerConfig, - e.getMessage()); + e); setUnavailable(); throw new SubscriptionNonRetryableException(e.getMessage(), e); } @@ -176,9 +176,9 @@ void closeInternal() throws SubscriptionException { // TODO: Distinguish between TTransportException, TApplicationException, and // TProtocolException. LOGGER.warn( - "TException occurred when SubscriptionProvider {} close: {}, set SubscriptionProvider unavailable", + "TException occurred when SubscriptionProvider {} close, set SubscriptionProvider unavailable", this, - e.getMessage()); + e); setUnavailable(); throw new SubscriptionNonRetryableException(e.getMessage(), e); } @@ -195,9 +195,9 @@ void heartbeat() throws SubscriptionException { // TODO: Distinguish between TTransportException, TApplicationException, and // TProtocolException. LOGGER.warn( - "TException occurred when SubscriptionProvider {} heartbeat: {}, set SubscriptionProvider unavailable", + "TException occurred when SubscriptionProvider {} heartbeat, set SubscriptionProvider unavailable", this, - e.getMessage()); + e); setUnavailable(); throw new SubscriptionNonRetryableException(e.getMessage(), e); } @@ -210,10 +210,10 @@ void subscribe(final Set topicNames) throws SubscriptionException { req = PipeSubscribeSubscribeReq.toTPipeSubscribeReq(topicNames); } catch (final IOException e) { LOGGER.warn( - "IOException occurred when SubscriptionProvider {} serialize subscribe request {}: {}", + "IOException occurred when SubscriptionProvider {} serialize subscribe request {}", this, topicNames, - e.getMessage()); + e); throw new SubscriptionRetryableException(e.getMessage(), e); } final TPipeSubscribeResp resp; @@ -223,10 +223,10 @@ void subscribe(final Set topicNames) throws SubscriptionException { // TODO: Distinguish between TTransportException, TApplicationException, and // TProtocolException. LOGGER.warn( - "TException occurred when SubscriptionProvider {} subscribe with request {}: {}, set SubscriptionProvider unavailable", + "TException occurred when SubscriptionProvider {} subscribe with request {}, set SubscriptionProvider unavailable", this, topicNames, - e.getMessage()); + e); setUnavailable(); throw new SubscriptionNonRetryableException(e.getMessage(), e); } @@ -239,10 +239,10 @@ void unsubscribe(final Set topicNames) throws SubscriptionException { req = PipeSubscribeUnsubscribeReq.toTPipeSubscribeReq(topicNames); } catch (final IOException e) { LOGGER.warn( - "IOException occurred when SubscriptionProvider {} serialize unsubscribe request {}: {}", + "IOException occurred when SubscriptionProvider {} serialize unsubscribe request {}", this, topicNames, - e.getMessage()); + e); throw new SubscriptionRetryableException(e.getMessage(), e); } final TPipeSubscribeResp resp; @@ -252,10 +252,10 @@ void unsubscribe(final Set topicNames) throws SubscriptionException { // TODO: Distinguish between TTransportException, TApplicationException, and // TProtocolException. LOGGER.warn( - "TException occurred when SubscriptionProvider {} unsubscribe with request {}: {}, set SubscriptionProvider unavailable", + "TException occurred when SubscriptionProvider {} unsubscribe with request {}, set SubscriptionProvider unavailable", this, topicNames, - e.getMessage()); + e); setUnavailable(); throw new SubscriptionNonRetryableException(e.getMessage(), e); } @@ -269,10 +269,10 @@ List poll(final SubscriptionPollMessage pollMessage) req = PipeSubscribePollReq.toTPipeSubscribeReq(pollMessage); } catch (final IOException e) { LOGGER.warn( - "IOException occurred when SubscriptionProvider {} serialize poll request {}: {}", + "IOException occurred when SubscriptionProvider {} serialize poll request {}", this, pollMessage, - e.getMessage()); + e); throw new SubscriptionRetryableException(e.getMessage(), e); } final TPipeSubscribeResp resp; @@ -282,10 +282,10 @@ List poll(final SubscriptionPollMessage pollMessage) // TODO: Distinguish between TTransportException, TApplicationException, and // TProtocolException. LOGGER.warn( - "TException occurred when SubscriptionProvider {} poll with request {}: {}, set SubscriptionProvider unavailable", + "TException occurred when SubscriptionProvider {} poll with request {}, set SubscriptionProvider unavailable", this, pollMessage, - e.getMessage()); + e); setUnavailable(); throw new SubscriptionNonRetryableException(e.getMessage(), e); } @@ -301,10 +301,10 @@ void commitSync(final List subscriptionCommitContexts req = PipeSubscribeCommitReq.toTPipeSubscribeReq(subscriptionCommitContexts); } catch (final IOException e) { LOGGER.warn( - "IOException occurred when SubscriptionProvider {} serialize commit request {}: {}", + "IOException occurred when SubscriptionProvider {} serialize commit request {}", this, subscriptionCommitContexts, - e.getMessage()); + e); throw new SubscriptionRetryableException(e.getMessage(), e); } final TPipeSubscribeResp resp; @@ -314,10 +314,10 @@ void commitSync(final List subscriptionCommitContexts // TODO: Distinguish between TTransportException, TApplicationException, and // TProtocolException. LOGGER.warn( - "TException occurred when SubscriptionProvider {} commit with request {}: {}, set SubscriptionProvider unavailable", + "TException occurred when SubscriptionProvider {} commit with request {}, set SubscriptionProvider unavailable", this, subscriptionCommitContexts, - e.getMessage()); + e); setUnavailable(); throw new SubscriptionNonRetryableException(e.getMessage(), e); } diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionTsFileReader.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionTsFileReader.java index 44ac79bc5885..554b193ac13d 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionTsFileReader.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionTsFileReader.java @@ -35,4 +35,9 @@ public SubscriptionTsFileReader(final String filePath) { public TsFileReader open() throws IOException { return new TsFileReader(new TsFileSequenceReader(filePath)); } + + @Override + public String toString() { + return "SubscriptionTsFileReader{" + "filePath='" + filePath + '\'' + '}'; + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java index 2d43a7eaa719..6a1fb8677bbf 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java @@ -65,6 +65,10 @@ public SubscriptionPrefetchingTsFileQueue( @Override public SubscriptionTsFileEvent poll(final String consumerId, final SubscriptionPollTimer timer) { + if (hasPollableOnTheFlySubscriptionTsFileEvent(consumerId)) { + return null; + } + final SubscriptionTsFileEvent pollableEvent = getPollableOnTheFlySubscriptionTsFileEvent(consumerId); if (Objects.nonNull(pollableEvent)) { @@ -145,7 +149,7 @@ public SubscriptionTsFileEvent poll(final String consumerId, final SubscriptionP (PipeTsFileInsertionEvent) enrichedEvents.get(0); // check file name - if (!Objects.equals(tsFileInsertionEvent.getTsFile().getName(), fileName)) { + if (!fileName.startsWith(tsFileInsertionEvent.getTsFile().getName())) { final String errorMessage = String.format( "inconsistent file name, current: %s, incoming: %s, consumer: %s, writing offset: %s, prefetching queue: %s", @@ -167,8 +171,7 @@ public SubscriptionTsFileEvent poll(final String consumerId, final SubscriptionP switch (SubscriptionPolledMessageType.valueOf(messageType)) { case TS_FILE_INIT: // check file name - if (!Objects.equals( - ((TsFileInitMessagePayload) messagePayload).getFileName(), fileName)) { + if (!fileName.startsWith(((TsFileInitMessagePayload) messagePayload).getFileName())) { final String errorMessage = String.format( "inconsistent file name, current: %s, incoming: %s, consumer: %s, writing offset: %s, prefetching queue: %s", @@ -192,8 +195,7 @@ public SubscriptionTsFileEvent poll(final String consumerId, final SubscriptionP break; case TS_FILE_PIECE: // check file name - if (!Objects.equals( - ((TsFilePieceMessagePayload) messagePayload).getFileName(), fileName)) { + if (!fileName.startsWith(((TsFilePieceMessagePayload) messagePayload).getFileName())) { final String errorMessage = String.format( "inconsistent file name, current: %s, incoming: %s, consumer: %s, writing offset: %s, prefetching queue: %s", @@ -218,8 +220,7 @@ public SubscriptionTsFileEvent poll(final String consumerId, final SubscriptionP break; case TS_FILE_SEAL: // check file name - if (!Objects.equals( - ((TsFileSealMessagePayload) messagePayload).getFileName(), fileName)) { + if (!fileName.startsWith(((TsFileSealMessagePayload) messagePayload).getFileName())) { final String errorMessage = String.format( "inconsistent file name, current: %s, incoming: %s, consumer: %s, writing offset: %s, prefetching queue: %s", @@ -311,6 +312,29 @@ public synchronized void executePrefetch() { /////////////////////////////// utility /////////////////////////////// + private synchronized boolean hasPollableOnTheFlySubscriptionTsFileEvent(final String consumerId) { + final SubscriptionTsFileEvent event = consumerIdToCurrentEventMap.get(consumerId); + if (Objects.isNull(event)) { + return false; + } + + if (event.isCommitted()) { + consumerIdToCurrentEventMap.remove(consumerId); + return false; + } + + if (!event.pollable()) { + LOGGER.info( + "SubscriptionPrefetchingTsFileQueue {} is currently transferring TsFile (with event {}) to consumer {}", + this, + event, + consumerId); + return true; + } + + return false; + } + private synchronized SubscriptionTsFileEvent getPollableOnTheFlySubscriptionTsFileEvent( final String consumerId) { for (final Map.Entry entry : diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java index dc136205e803..9f0b8d7ba352 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java @@ -407,12 +407,20 @@ private TPipeSubscribeResp handlePipeSubscribePollTsFileInternal( // check timer if (timer.isExpired()) { - LOGGER.warn("Subscription: timeout happened when consumer {} poll tsfile", consumerConfig); + LOGGER.warn( + "Subscription: timeout happened when consumer {} poll TsFile (topic name: {}, file name: {}, writing offset: {})", + consumerConfig, + messagePayload.getTopicName(), + messagePayload.getFileName(), + messagePayload.getWritingOffset()); } LOGGER.info( - "Subscription: consumer {} poll tsfile, commit contexts: {}", + "Subscription: consumer {} poll TsFile (topic name: {}, file name: {}, writing offset: {}) successfully, commit contexts: {}", consumerConfig, + messagePayload.getTopicName(), + messagePayload.getFileName(), + messagePayload.getWritingOffset(), commitContexts); return PipeSubscribePollResp.toTPipeSubscribeResp(RpcUtils.SUCCESS_STATUS, polledMessages); 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 aaddde1ba339..1d0617a65cab 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 @@ -234,7 +234,7 @@ public class CommonConfig { private int subscriptionPollMaxBlockingTimeMs = 500; private int subscriptionSerializeMaxBlockingTimeMs = 100; private long subscriptionLaunchRetryIntervalMs = 1000; - private int subscriptionRecycleUncommittedEventIntervalMs = 240000; // 240s + private int subscriptionRecycleUncommittedEventIntervalMs = 120000; // 120s private long subscriptionDefaultPollTimeoutMs = 30000; private long subscriptionMinPollTimeoutMs = 500; private int subscriptionReadFileBufferSize = 8388608; From 72b26842d1c6346b1fc5a49361120e66ffa40edc Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Tue, 23 Apr 2024 13:39:37 +0800 Subject: [PATCH 54/93] timer control on client side --- .../SubscriptionPollTimeOutException.java | 40 --------- .../common/SubscriptionPollMessage.java | 2 +- .../subscription/SubscriptionConsumer.java | 82 ++++++++++--------- .../subscription}/SubscriptionPollTimer.java | 2 +- .../agent/SubscriptionBrokerAgent.java | 12 +-- .../broker/SubscriptionBroker.java | 15 +--- .../broker/SubscriptionPrefetchingQueue.java | 3 +- .../SubscriptionPrefetchingTabletsQueue.java | 12 +-- .../SubscriptionPrefetchingTsFileQueue.java | 31 +------ .../receiver/SubscriptionReceiverV1.java | 47 ++--------- .../iotdb/commons/conf/CommonConfig.java | 20 +---- .../iotdb/commons/conf/CommonDescriptor.java | 10 --- .../config/SubscriptionConfig.java | 10 --- 13 files changed, 71 insertions(+), 215 deletions(-) delete mode 100644 iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionPollTimeOutException.java rename {iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/timer => iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription}/SubscriptionPollTimer.java (97%) diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionPollTimeOutException.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionPollTimeOutException.java deleted file mode 100644 index 033d975c7e5d..000000000000 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionPollTimeOutException.java +++ /dev/null @@ -1,40 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iotdb.rpc.subscription.exception; - -import java.util.Objects; - -public class SubscriptionPollTimeOutException extends SubscriptionException { - - public SubscriptionPollTimeOutException(String message) { - super(message); - } - - public SubscriptionPollTimeOutException(String message, Throwable cause) { - super(message, cause); - } - - @Override - public boolean equals(Object obj) { - return obj instanceof SubscriptionPollTimeOutException - && Objects.equals(getMessage(), ((SubscriptionPollTimeOutException) obj).getMessage()) - && Objects.equals(getTimeStamp(), ((SubscriptionPollTimeOutException) obj).getTimeStamp()); - } -} diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPollMessage.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPollMessage.java index 86a6247b1331..3ceed2f6450a 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPollMessage.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPollMessage.java @@ -36,7 +36,7 @@ public class SubscriptionPollMessage { private final transient SubscriptionMessagePayload messagePayload; - private final transient long timeoutMs; + private final transient long timeoutMs; // unused now public SubscriptionPollMessage( short messageType, SubscriptionMessagePayload messagePayload, long timeoutMs) { diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java index 94386b4741fa..5b68e4759c82 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java @@ -477,40 +477,47 @@ SubscriptionProvider getProvider(final int dataNodeId) { protected List poll(final Set topicNames, final long timeoutMs) throws SubscriptionException { final List messages = new ArrayList<>(); - - // poll tablets or tsfile - for (final SubscriptionPolledMessage polledMessage : pollInternal(topicNames, timeoutMs)) { - final short messageType = polledMessage.getMessageType(); - if (SubscriptionPolledMessageType.isValidatedMessageType(messageType)) { - switch (SubscriptionPolledMessageType.valueOf(messageType)) { - case TABLETS: - messages.add( - new SubscriptionMessage( - polledMessage.getCommitContext(), - ((TabletsMessagePayload) polledMessage.getMessagePayload()).getTablets())); - break; - case TS_FILE_INIT: - pollTsFile( - polledMessage.getCommitContext(), - ((TsFileInitMessagePayload) polledMessage.getMessagePayload()).getFileName(), - timeoutMs) - .ifPresent(messages::add); - break; - default: - LOGGER.warn("unexpected message type: {}", messageType); - break; + final SubscriptionPollTimer timer = + new SubscriptionPollTimer(System.currentTimeMillis(), timeoutMs); + + do { + // update timer + timer.update(); + // poll tablets or tsfile + for (final SubscriptionPolledMessage polledMessage : pollInternal(topicNames)) { + final short messageType = polledMessage.getMessageType(); + if (SubscriptionPolledMessageType.isValidatedMessageType(messageType)) { + switch (SubscriptionPolledMessageType.valueOf(messageType)) { + case TABLETS: + messages.add( + new SubscriptionMessage( + polledMessage.getCommitContext(), + ((TabletsMessagePayload) polledMessage.getMessagePayload()).getTablets())); + break; + case TS_FILE_INIT: + pollTsFile( + polledMessage.getCommitContext(), + ((TsFileInitMessagePayload) polledMessage.getMessagePayload()).getFileName()) + .ifPresent(messages::add); + break; + default: + LOGGER.warn("unexpected message type: {}", messageType); + break; + } + } else { + LOGGER.warn("unexpected message type: {}", messageType); } - } else { - LOGGER.warn("unexpected message type: {}", messageType); } - } + if (!messages.isEmpty()) { + return messages; + } + } while (timer.notExpired()); return messages; } private Optional pollTsFile( - final SubscriptionCommitContext commitContext, String fileName, final long timeoutMs) - throws SubscriptionException { + final SubscriptionCommitContext commitContext, String fileName) throws SubscriptionException { final String topicName = commitContext.getTopicName(); Path filePath; @@ -553,7 +560,7 @@ private Optional pollTsFile( final File file = filePath.toFile(); try (final RandomAccessFile fileWriter = new RandomAccessFile(file, "rw")) { - return Optional.of(pollTsFileInternal(commitContext, file, fileWriter, timeoutMs)); + return Optional.of(pollTsFileInternal(commitContext, file, fileWriter)); } catch (final IOException | SubscriptionRetryableException e) { LOGGER.warn( "IOException or SubscriptionRetryableException occurred when SubscriptionConsumer {} polling TsFile {} with commit context {}", @@ -578,8 +585,7 @@ private Optional pollTsFile( private SubscriptionMessage pollTsFileInternal( final SubscriptionCommitContext commitContext, final File file, - final RandomAccessFile fileWriter, - final long timeoutMs) + final RandomAccessFile fileWriter) throws IOException, SubscriptionException { final int dataNodeId = commitContext.getDataNodeId(); final String topicName = commitContext.getTopicName(); @@ -594,7 +600,7 @@ private SubscriptionMessage pollTsFileInternal( long writingOffset = fileWriter.length(); while (true) { final List polledMessages = - pollTsFileInternal(dataNodeId, topicName, fileName, writingOffset, timeoutMs); + pollTsFileInternal(dataNodeId, topicName, fileName, writingOffset); // It's agreed that the server will always return at least one message, even in case of // failure. @@ -727,8 +733,8 @@ private SubscriptionMessage pollTsFileInternal( } } - private List pollInternal( - final Set topicNames, final long timeoutMs) throws SubscriptionException { + private List pollInternal(final Set topicNames) + throws SubscriptionException { final List polledMessages = new ArrayList<>(); acquireReadLock(); @@ -740,7 +746,7 @@ private List pollInternal( new SubscriptionPollMessage( SubscriptionPollMessageType.POLL.getType(), new PollMessagePayload(topicNames), - timeoutMs))); + 0L))); } catch (final SubscriptionRetryableException e) { LOGGER.warn( "SubscriptionRetryableException occurred when SubscriptionConsumer {} polling from SubscriptionProvider {}", @@ -767,11 +773,7 @@ private List pollInternal( } private List pollTsFileInternal( - final int dataNodeId, - final String topicName, - final String fileName, - final long writingOffset, - final long timeoutMs) + final int dataNodeId, final String topicName, final String fileName, final long writingOffset) throws SubscriptionException { acquireReadLock(); try { @@ -786,7 +788,7 @@ private List pollTsFileInternal( new SubscriptionPollMessage( SubscriptionPollMessageType.POLL_TS_FILE.getType(), new PollTsFileMessagePayload(topicName, fileName, writingOffset), - timeoutMs)); + 0L)); } finally { releaseReadLock(); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/timer/SubscriptionPollTimer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPollTimer.java similarity index 97% rename from iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/timer/SubscriptionPollTimer.java rename to iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPollTimer.java index 71957320e316..9756da1f7c6f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/timer/SubscriptionPollTimer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPollTimer.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.iotdb.db.subscription.timer; +package org.apache.iotdb.session.subscription; public class SubscriptionPollTimer { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionBrokerAgent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionBrokerAgent.java index 1c74b1dc6d6a..15fac4578858 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionBrokerAgent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionBrokerAgent.java @@ -22,7 +22,6 @@ import org.apache.iotdb.db.subscription.broker.SubscriptionBroker; import org.apache.iotdb.db.subscription.event.SubscriptionEvent; import org.apache.iotdb.db.subscription.task.subtask.SubscriptionConnectorSubtask; -import org.apache.iotdb.db.subscription.timer.SubscriptionPollTimer; import org.apache.iotdb.rpc.subscription.config.ConsumerConfig; import org.apache.iotdb.rpc.subscription.exception.SubscriptionException; import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionCommitContext; @@ -46,9 +45,7 @@ public class SubscriptionBrokerAgent { //////////////////////////// provided for subscription agent //////////////////////////// public List poll( - final ConsumerConfig consumerConfig, - final Set topicNames, - final SubscriptionPollTimer timer) { + final ConsumerConfig consumerConfig, final Set topicNames) { final String consumerGroupId = consumerConfig.getConsumerGroupId(); final SubscriptionBroker broker = consumerGroupIdToSubscriptionBroker.get(consumerGroupId); if (Objects.isNull(broker)) { @@ -60,15 +57,14 @@ public List poll( } // TODO: currently we fetch messages from all topics final String consumerId = consumerConfig.getConsumerId(); - return broker.poll(consumerId, topicNames, timer); + return broker.poll(consumerId, topicNames); } public List pollTsFile( final ConsumerConfig consumerConfig, final String topicName, final String fileName, - final long writingOffset, - final SubscriptionPollTimer timer) { + final long writingOffset) { final String consumerGroupId = consumerConfig.getConsumerGroupId(); final SubscriptionBroker broker = consumerGroupIdToSubscriptionBroker.get(consumerGroupId); if (Objects.isNull(broker)) { @@ -79,7 +75,7 @@ public List pollTsFile( throw new SubscriptionException(errorMessage); } final String consumerId = consumerConfig.getConsumerId(); - return broker.pollTsFile(consumerId, topicName, fileName, writingOffset, timer); + return broker.pollTsFile(consumerId, topicName, fileName, writingOffset); } /** @return list of successful commit contexts */ 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 ba59c580d568..d0a7fc7d23c4 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 @@ -22,7 +22,6 @@ import org.apache.iotdb.commons.pipe.task.connection.BoundedBlockingPendingQueue; import org.apache.iotdb.db.subscription.agent.SubscriptionAgent; import org.apache.iotdb.db.subscription.event.SubscriptionEvent; -import org.apache.iotdb.db.subscription.timer.SubscriptionPollTimer; import org.apache.iotdb.pipe.api.event.Event; import org.apache.iotdb.rpc.subscription.config.TopicConstant; import org.apache.iotdb.rpc.subscription.exception.SubscriptionException; @@ -58,22 +57,17 @@ public boolean isEmpty() { //////////////////////////// provided for SubscriptionBrokerAgent //////////////////////////// - public List poll( - final String consumerId, final Set topicNames, final SubscriptionPollTimer timer) { + public List poll(final String consumerId, final Set topicNames) { final List events = new ArrayList<>(); for (final Map.Entry entry : topicNameToPrefetchingQueue.entrySet()) { final String topicName = entry.getKey(); final SubscriptionPrefetchingQueue prefetchingQueue = entry.getValue(); if (topicNames.contains(topicName)) { - final SubscriptionEvent event = prefetchingQueue.poll(consumerId, timer); + final SubscriptionEvent event = prefetchingQueue.poll(consumerId); if (Objects.nonNull(event)) { events.add(event); } - timer.update(); - if (timer.isExpired()) { - break; - } } } return events; @@ -83,8 +77,7 @@ public List pollTsFile( final String consumerId, final String topicName, final String fileName, - final long writingOffset, - final SubscriptionPollTimer timer) { + final long writingOffset) { final SubscriptionPrefetchingQueue prefetchingQueue = topicNameToPrefetchingQueue.get(topicName); if (Objects.isNull(prefetchingQueue)) { @@ -103,7 +96,7 @@ public List pollTsFile( } return Collections.singletonList( ((SubscriptionPrefetchingTsFileQueue) prefetchingQueue) - .pollTsFile(consumerId, fileName, writingOffset, timer)); + .pollTsFile(consumerId, fileName, writingOffset)); } /** @return list of successful commit contexts */ 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 50a9d2d1ea8f..c46f83cf54a0 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 @@ -23,7 +23,6 @@ import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.pipe.agent.PipeAgent; import org.apache.iotdb.db.subscription.event.SubscriptionEvent; -import org.apache.iotdb.db.subscription.timer.SubscriptionPollTimer; import org.apache.iotdb.pipe.api.event.Event; import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionCommitContext; @@ -57,7 +56,7 @@ public SubscriptionPrefetchingQueue( this.uncommittedEvents = new ConcurrentHashMap<>(); } - public abstract SubscriptionEvent poll(String consumerId, SubscriptionPollTimer timer); + public abstract SubscriptionEvent poll(String consumerId); public abstract void executePrefetch(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTabletsQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTabletsQueue.java index ddfcf17d0f2c..9a4734f2f21c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTabletsQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTabletsQueue.java @@ -27,7 +27,6 @@ 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.subscription.event.SubscriptionEvent; -import org.apache.iotdb.db.subscription.timer.SubscriptionPollTimer; import org.apache.iotdb.pipe.api.event.Event; import org.apache.iotdb.pipe.api.event.dml.insertion.TabletInsertionEvent; import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionCommitContext; @@ -62,12 +61,15 @@ public SubscriptionPrefetchingTabletsQueue( } @Override - public SubscriptionEvent poll(final String consumerId, final SubscriptionPollTimer timer) { + public SubscriptionEvent poll(final String consumerId) { if (prefetchingQueue.isEmpty()) { prefetchOnce(SubscriptionConfig.getInstance().getSubscriptionMaxTabletsPerPrefetching()); // without serializeOnce here } + final long size = prefetchingQueue.size(); + long count = 0; + SubscriptionEvent currentEvent; try { while (Objects.nonNull( @@ -80,11 +82,11 @@ public SubscriptionEvent poll(final String consumerId, final SubscriptionPollTim } // Re-enqueue the uncommitted event at the end of the queue. prefetchingQueue.add(currentEvent); - // timeout control - timer.update(); - if (timer.isExpired()) { + // limit control + if (count >= size) { break; } + count++; if (!currentEvent.pollable()) { continue; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java index 6a1fb8677bbf..0808ed2d73f8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java @@ -24,7 +24,6 @@ import org.apache.iotdb.db.pipe.event.UserDefinedEnrichedEvent; import org.apache.iotdb.db.pipe.event.common.tsfile.PipeTsFileInsertionEvent; import org.apache.iotdb.db.subscription.event.SubscriptionTsFileEvent; -import org.apache.iotdb.db.subscription.timer.SubscriptionPollTimer; import org.apache.iotdb.pipe.api.event.Event; import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionCommitContext; import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionMessagePayload; @@ -64,7 +63,7 @@ public SubscriptionPrefetchingTsFileQueue( } @Override - public SubscriptionTsFileEvent poll(final String consumerId, final SubscriptionPollTimer timer) { + public SubscriptionTsFileEvent poll(final String consumerId) { if (hasPollableOnTheFlySubscriptionTsFileEvent(consumerId)) { return null; } @@ -75,11 +74,6 @@ public SubscriptionTsFileEvent poll(final String consumerId, final SubscriptionP return pollableEvent; } - timer.update(); - if (timer.isExpired()) { - return null; - } - Event event; while (Objects.nonNull( event = UserDefinedEnrichedEvent.maybeOf(inputPendingQueue.waitedPoll()))) { @@ -109,10 +103,7 @@ public SubscriptionTsFileEvent poll(final String consumerId, final SubscriptionP } public synchronized @NonNull SubscriptionTsFileEvent pollTsFile( - final String consumerId, - final String fileName, - final long writingOffset, - final SubscriptionPollTimer timer) { + final String consumerId, final String fileName, final long writingOffset) { // 1. Extract current event and check it final SubscriptionTsFileEvent event = consumerIdToCurrentEventMap.get(consumerId); if (Objects.isNull(event)) { @@ -253,25 +244,11 @@ public SubscriptionTsFileEvent poll(final String consumerId, final SubscriptionP } // 3. Poll tsfile piece or tsfile seal - return pollTsFile(consumerId, writingOffset, event, timer); + return pollTsFile(consumerId, writingOffset, event); } private synchronized @NonNull SubscriptionTsFileEvent pollTsFile( - final String consumerId, - final long writingOffset, - final SubscriptionTsFileEvent event, - final SubscriptionPollTimer timer) { - timer.update(); - if (timer.isExpired()) { - final String errorMessage = - String.format( - "Timeout occurred when SubscriptionPrefetchingTsFileQueue %s transferring TsFile (with event %s) to consumer %s", - this, event, consumerId); - LOGGER.warn(errorMessage); - // assume retryable - return generateSubscriptionTsFileEventWithErrorMessage(errorMessage, true); - } - + final String consumerId, final long writingOffset, final SubscriptionTsFileEvent event) { Pair newEventWithCommittable = event.matchOrResetNext(writingOffset); if (Objects.isNull(newEventWithCommittable)) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java index 9f0b8d7ba352..b0dd0bc5b5e6 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java @@ -23,7 +23,6 @@ import org.apache.iotdb.commons.client.IClientManager; import org.apache.iotdb.commons.client.exception.ClientManagerException; import org.apache.iotdb.commons.consensus.ConfigRegionId; -import org.apache.iotdb.commons.subscription.config.SubscriptionConfig; import org.apache.iotdb.confignode.rpc.thrift.TCloseConsumerReq; import org.apache.iotdb.confignode.rpc.thrift.TCreateConsumerReq; import org.apache.iotdb.confignode.rpc.thrift.TSubscribeReq; @@ -35,7 +34,6 @@ import org.apache.iotdb.db.queryengine.plan.parser.ASTVisitor; import org.apache.iotdb.db.subscription.agent.SubscriptionAgent; import org.apache.iotdb.db.subscription.event.SubscriptionEvent; -import org.apache.iotdb.db.subscription.timer.SubscriptionPollTimer; import org.apache.iotdb.rpc.RpcUtils; import org.apache.iotdb.rpc.TSStatusCode; import org.apache.iotdb.rpc.subscription.config.ConsumerConfig; @@ -304,24 +302,14 @@ private TPipeSubscribeResp handlePipeSubscribePoll(final PipeSubscribePollReq re final SubscriptionPollMessage pollMessage = req.getPollMessage(); final short messageType = pollMessage.getMessageType(); - final long timeoutMs = pollMessage.getTimeoutMs(); - final SubscriptionPollTimer timer = - new SubscriptionPollTimer( - System.currentTimeMillis(), - timeoutMs == 0 - ? SubscriptionConfig.getInstance().getSubscriptionDefaultPollTimeoutMs() - : Math.max( - timeoutMs, - SubscriptionConfig.getInstance().getSubscriptionMinPollTimeoutMs())); - if (SubscriptionPollMessageType.isValidatedMessageType(messageType)) { switch (SubscriptionPollMessageType.valueOf(messageType)) { case POLL: return handlePipeSubscribePollInternal( - consumerConfig, (PollMessagePayload) pollMessage.getMessagePayload(), timer); + consumerConfig, (PollMessagePayload) pollMessage.getMessagePayload()); case POLL_TS_FILE: return handlePipeSubscribePollTsFileInternal( - consumerConfig, (PollTsFileMessagePayload) pollMessage.getMessagePayload(), timer); + consumerConfig, (PollTsFileMessagePayload) pollMessage.getMessagePayload()); default: break; } @@ -339,9 +327,7 @@ private TPipeSubscribeResp handlePipeSubscribePoll(final PipeSubscribePollReq re } private TPipeSubscribeResp handlePipeSubscribePollInternal( - final ConsumerConfig consumerConfig, - final PollMessagePayload messagePayload, - final SubscriptionPollTimer timer) { + final ConsumerConfig consumerConfig, final PollMessagePayload messagePayload) { Set topicNames = messagePayload.getTopicNames(); if (topicNames.isEmpty()) { // poll all subscribed topics @@ -355,7 +341,7 @@ private TPipeSubscribeResp handlePipeSubscribePollInternal( // poll final List events = - SubscriptionAgent.broker().poll(consumerConfig, topicNames, timer); + SubscriptionAgent.broker().poll(consumerConfig, topicNames); final List polledMessages = events.stream().map(SubscriptionEvent::getMessage).collect(Collectors.toList()); @@ -365,14 +351,6 @@ private TPipeSubscribeResp handlePipeSubscribePollInternal( .map(SubscriptionPolledMessage::getCommitContext) .collect(Collectors.toList()); - // check timer - if (timer.isExpired()) { - LOGGER.warn( - "Subscription: timeout happened when consumer {} poll topics {}", - consumerConfig, - topicNames); - } - LOGGER.info( "Subscription: consumer {} poll topics {} successfully, commit contexts: {}", consumerConfig, @@ -384,9 +362,7 @@ private TPipeSubscribeResp handlePipeSubscribePollInternal( } private TPipeSubscribeResp handlePipeSubscribePollTsFileInternal( - final ConsumerConfig consumerConfig, - final PollTsFileMessagePayload messagePayload, - final SubscriptionPollTimer timer) { + final ConsumerConfig consumerConfig, final PollTsFileMessagePayload messagePayload) { // poll final List events = SubscriptionAgent.broker() @@ -394,8 +370,7 @@ private TPipeSubscribeResp handlePipeSubscribePollTsFileInternal( consumerConfig, messagePayload.getTopicName(), messagePayload.getFileName(), - messagePayload.getWritingOffset(), - timer); + messagePayload.getWritingOffset()); final List polledMessages = events.stream().map(SubscriptionEvent::getMessage).collect(Collectors.toList()); @@ -405,16 +380,6 @@ private TPipeSubscribeResp handlePipeSubscribePollTsFileInternal( .map(SubscriptionPolledMessage::getCommitContext) .collect(Collectors.toList()); - // check timer - if (timer.isExpired()) { - LOGGER.warn( - "Subscription: timeout happened when consumer {} poll TsFile (topic name: {}, file name: {}, writing offset: {})", - consumerConfig, - messagePayload.getTopicName(), - messagePayload.getFileName(), - messagePayload.getWritingOffset()); - } - LOGGER.info( "Subscription: consumer {} poll TsFile (topic name: {}, file name: {}, writing offset: {}) successfully, commit contexts: {}", consumerConfig, 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 1d0617a65cab..7573cc733a11 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 @@ -234,9 +234,7 @@ public class CommonConfig { private int subscriptionPollMaxBlockingTimeMs = 500; private int subscriptionSerializeMaxBlockingTimeMs = 100; private long subscriptionLaunchRetryIntervalMs = 1000; - private int subscriptionRecycleUncommittedEventIntervalMs = 120000; // 120s - private long subscriptionDefaultPollTimeoutMs = 30000; - private long subscriptionMinPollTimeoutMs = 500; + private int subscriptionRecycleUncommittedEventIntervalMs = 60000; // 60s private int subscriptionReadFileBufferSize = 8388608; /** Whether to use persistent schema mode. */ @@ -1017,22 +1015,6 @@ public void setSubscriptionRecycleUncommittedEventIntervalMs( subscriptionRecycleUncommittedEventIntervalMs; } - public long getSubscriptionDefaultPollTimeoutMs() { - return subscriptionDefaultPollTimeoutMs; - } - - public void setSubscriptionDefaultPollTimeoutMs(long subscriptionDefaultPollTimeoutMs) { - this.subscriptionDefaultPollTimeoutMs = subscriptionDefaultPollTimeoutMs; - } - - public long getSubscriptionMinPollTimeoutMs() { - return subscriptionMinPollTimeoutMs; - } - - public void setSubscriptionMinPollTimeoutMs(long subscriptionMinPollTimeoutMs) { - this.subscriptionMinPollTimeoutMs = subscriptionMinPollTimeoutMs; - } - public int getSubscriptionReadFileBufferSize() { return subscriptionReadFileBufferSize; } 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 c9cd97640414..d5139ecd94e9 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 @@ -552,16 +552,6 @@ private void loadSubscriptionProps(Properties properties) { properties.getProperty( "subscription_recycle_uncommitted_event_interval_ms", String.valueOf(config.getSubscriptionRecycleUncommittedEventIntervalMs())))); - config.setSubscriptionDefaultPollTimeoutMs( - Integer.parseInt( - properties.getProperty( - "subscription_default_poll_timeout_ms", - String.valueOf(config.getSubscriptionDefaultPollTimeoutMs())))); - config.setSubscriptionMinPollTimeoutMs( - Integer.parseInt( - properties.getProperty( - "subscription_min_poll_timeout_ms", - String.valueOf(config.getSubscriptionMinPollTimeoutMs())))); config.setSubscriptionReadFileBufferSize( Integer.parseInt( properties.getProperty( diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/subscription/config/SubscriptionConfig.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/subscription/config/SubscriptionConfig.java index afa4346d2622..e5c97db393fe 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/subscription/config/SubscriptionConfig.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/subscription/config/SubscriptionConfig.java @@ -55,14 +55,6 @@ public int getSubscriptionRecycleUncommittedEventIntervalMs() { return COMMON_CONFIG.getSubscriptionRecycleUncommittedEventIntervalMs(); } - public long getSubscriptionDefaultPollTimeoutMs() { - return COMMON_CONFIG.getSubscriptionDefaultPollTimeoutMs(); - } - - public long getSubscriptionMinPollTimeoutMs() { - return COMMON_CONFIG.getSubscriptionMinPollTimeoutMs(); - } - public int getSubscriptionReadFileBufferSize() { return COMMON_CONFIG.getSubscriptionReadFileBufferSize(); } @@ -84,8 +76,6 @@ public void printAllConfigs() { LOGGER.info( "SubscriptionRecycleUncommittedEventIntervalMs: {}", getSubscriptionRecycleUncommittedEventIntervalMs()); - LOGGER.info("SubscriptionDefaultPollTimeoutMs: {}", getSubscriptionDefaultPollTimeoutMs()); - LOGGER.info("SubscriptionMinPollTimeoutMs: {}", getSubscriptionMinPollTimeoutMs()); LOGGER.info("SubscriptionReadFileBufferSize: {}", getSubscriptionReadFileBufferSize()); } From 62c4a2c85753982903fe8c75bb93693388356167 Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Tue, 23 Apr 2024 22:20:41 +0800 Subject: [PATCH 55/93] minor improve --- .../iotdb/SubscriptionSessionExample.java | 114 ++++++++++-------- .../common/SubscriptionPolledMessage.java | 4 +- .../common/TsFilePieceMessagePayload.java | 2 +- .../subscription/SubscriptionConsumer.java | 79 ++++++------ .../SubscriptionPullConsumer.java | 6 +- .../SubscriptionPushConsumer.java | 6 +- .../agent/SubscriptionConsumerAgent.java | 7 +- .../agent/SubscriptionTopicAgent.java | 7 +- .../SubscriptionPrefetchingTsFileQueue.java | 2 +- .../event/SubscriptionTsFileEvent.java | 12 +- .../receiver/SubscriptionReceiverV1.java | 28 ++--- 11 files changed, 135 insertions(+), 132 deletions(-) diff --git a/example/session/src/main/java/org/apache/iotdb/SubscriptionSessionExample.java b/example/session/src/main/java/org/apache/iotdb/SubscriptionSessionExample.java index 71ffcd7e60ba..80f219dfae47 100644 --- a/example/session/src/main/java/org/apache/iotdb/SubscriptionSessionExample.java +++ b/example/session/src/main/java/org/apache/iotdb/SubscriptionSessionExample.java @@ -21,6 +21,7 @@ import org.apache.iotdb.isession.SessionDataSet; import org.apache.iotdb.isession.util.Version; +import org.apache.iotdb.rpc.IoTDBConnectionException; import org.apache.iotdb.rpc.subscription.config.ConsumerConstant; import org.apache.iotdb.rpc.subscription.config.TopicConstant; import org.apache.iotdb.session.Session; @@ -31,15 +32,11 @@ import org.apache.iotdb.session.subscription.SubscriptionSessionDataSets; import org.apache.iotdb.session.subscription.SubscriptionTsFileReader; -import org.apache.tsfile.read.TsFileReader; -import org.apache.tsfile.read.common.Path; -import org.apache.tsfile.read.expression.QueryExpression; -import org.apache.tsfile.read.query.dataset.QueryDataSet; - -import java.time.Duration; +import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.Properties; +import java.util.concurrent.locks.LockSupport; public class SubscriptionSessionExample { @@ -51,6 +48,10 @@ public class SubscriptionSessionExample { private static final String TOPIC_1 = "topic1"; private static final String TOPIC_2 = "`topic2`"; + public static final long SLEEP_NS = 1_000_000_000L; + public static final long POLL_TIMEOUT_MS = 10_000L; + private static final int MAX_RETRY_TIMES = 3; + private static void prepareData() throws Exception { // Open session session = @@ -65,7 +66,7 @@ private static void prepareData() throws Exception { // Insert some historical data final long currentTime = System.currentTimeMillis(); - for (int i = 0; i < 100; ++i) { + for (int i = 0; i < 10000; ++i) { session.executeNonQueryStatement( String.format("insert into root.db.d1(time, s1, s2) values (%s, 1, 2)", i)); session.executeNonQueryStatement( @@ -103,18 +104,13 @@ private static void dataQuery() throws Exception { session = null; } - private static void createTopics() throws Exception { + private static void subscriptionExample1() throws Exception { // Create topics try (final SubscriptionSession subscriptionSession = new SubscriptionSession(HOST, PORT)) { subscriptionSession.open(); subscriptionSession.createTopic(TOPIC_1); - final Properties config = new Properties(); - config.put(TopicConstant.FORMAT_KEY, TopicConstant.FORMAT_TS_FILE_READER_VALUE); - subscriptionSession.createTopic(TOPIC_2, config); } - } - private static void subscriptionExample1() throws Exception { int retryCount = 0; // Subscription: property-style ctor final Properties config = new Properties(); @@ -124,11 +120,11 @@ private static void subscriptionExample1() throws Exception { consumer1.open(); consumer1.subscribe(TOPIC_1); while (true) { - Thread.sleep(1000); // Wait for some time - final List messages = consumer1.poll(Duration.ofMillis(10000)); + LockSupport.parkNanos(SLEEP_NS); // wait some time + final List messages = consumer1.poll(POLL_TIMEOUT_MS); if (messages.isEmpty()) { retryCount++; - if (retryCount >= 5) { + if (retryCount >= MAX_RETRY_TIMES) { break; } } @@ -158,47 +154,63 @@ private static void subscriptionExample1() throws Exception { } private static void subscriptionExample2() throws Exception { - int retryCount = 0; - // Subscription: builder-style ctor - try (final SubscriptionPullConsumer consumer2 = - new SubscriptionPullConsumer.Builder() - .consumerId("c2") - .consumerGroupId("cg2") - .autoCommit(false) - .buildPullConsumer()) { - consumer2.open(); - consumer2.subscribe(TOPIC_2); - while (true) { - Thread.sleep(1000); // wait some time - final List messages = - consumer2.poll(Collections.singleton(TOPIC_2), Duration.ofMillis(10000)); - if (messages.isEmpty()) { - retryCount++; - if (retryCount >= 5) { - break; - } - } - for (final SubscriptionMessage message : messages) { - final SubscriptionTsFileReader reader = (SubscriptionTsFileReader) message.getPayload(); - try (final TsFileReader tsFileReader = reader.open()) { - final Path path = new Path("root.db.d1", "s1", true); - final QueryDataSet dataSet = - tsFileReader.query(QueryExpression.create(Collections.singletonList(path), null)); - while (dataSet.hasNext()) { - System.out.println(dataSet.next()); - } - } - } - consumer2.commitSync(messages); - } - consumer2.unsubscribe(TOPIC_2); + try (final SubscriptionSession subscriptionSession = new SubscriptionSession(HOST, PORT)) { + subscriptionSession.open(); + final Properties config = new Properties(); + config.put(TopicConstant.FORMAT_KEY, TopicConstant.FORMAT_TS_FILE_READER_VALUE); + subscriptionSession.createTopic(TOPIC_2, config); + } + + final List threads = new ArrayList<>(); + for (int i = 0; i < 8; ++i) { + final int idx = i; + final Thread thread = + new Thread( + () -> { + int retryCount = 0; + // Subscription: builder-style ctor + try (final SubscriptionPullConsumer consumer2 = + new SubscriptionPullConsumer.Builder() + .consumerId("c" + idx) + .consumerGroupId("cg2") + .autoCommit(false) + .buildPullConsumer()) { + consumer2.open(); + consumer2.subscribe(TOPIC_2); + while (true) { + LockSupport.parkNanos(SLEEP_NS); // wait some time + final List messages = + consumer2.poll(Collections.singleton(TOPIC_2), POLL_TIMEOUT_MS); + if (messages.isEmpty()) { + retryCount++; + if (retryCount >= MAX_RETRY_TIMES) { + break; + } + } + for (final SubscriptionMessage message : messages) { + final SubscriptionTsFileReader reader = + (SubscriptionTsFileReader) message.getPayload(); + System.out.println(reader.toString()); + } + consumer2.commitSync(messages); + } + consumer2.unsubscribe(TOPIC_2); + } catch (IoTDBConnectionException e) { + throw new RuntimeException(e); + } + }); + thread.start(); + threads.add(thread); + } + + for (final Thread thread : threads) { + thread.join(); } } public static void main(final String[] args) throws Exception { prepareData(); dataQuery(); - createTopics(); subscriptionExample1(); subscriptionExample2(); } diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledMessage.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledMessage.java index 2a03874cda33..02bd0f0bef01 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledMessage.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledMessage.java @@ -132,8 +132,8 @@ public boolean trySerialize() { return true; } catch (final IOException e) { LOGGER.warn( - "Subscription: something unexpected happened when serializing SubscriptionRawMessage, exception is {}", - e.getMessage()); + "Subscription: something unexpected happened when serializing SubscriptionRawMessage", + e); } return false; } diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFilePieceMessagePayload.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFilePieceMessagePayload.java index a3233f2986f5..ed557d784560 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFilePieceMessagePayload.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFilePieceMessagePayload.java @@ -98,7 +98,7 @@ public String toString() { + ", nextWritingOffset=" + nextWritingOffset + ", filePiece=" - + Arrays.toString(filePiece) + + Arrays.toString(filePiece).substring(0, 256) + "}"; } } diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java index 5b68e4759c82..7d0a5ed8a41d 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java @@ -22,7 +22,6 @@ import org.apache.iotdb.common.rpc.thrift.TEndPoint; import org.apache.iotdb.isession.SessionConfig; import org.apache.iotdb.rpc.IoTDBConnectionException; -import org.apache.iotdb.rpc.StatementExecutionException; import org.apache.iotdb.rpc.subscription.config.ConsumerConstant; import org.apache.iotdb.rpc.subscription.exception.SubscriptionConnectionException; import org.apache.iotdb.rpc.subscription.exception.SubscriptionException; @@ -43,7 +42,6 @@ import org.apache.iotdb.rpc.subscription.payload.common.TsFileSealMessagePayload; import org.apache.iotdb.session.util.SessionUtils; -import org.apache.thrift.TException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -78,8 +76,9 @@ public abstract class SubscriptionConsumer implements AutoCloseable { private static final Logger LOGGER = LoggerFactory.getLogger(SubscriptionConsumer.class); - private static final IoTDBConnectionException NO_PROVIDERS_EXCEPTION = - new IoTDBConnectionException("Cluster has no available subscription providers to connect"); + private static final SubscriptionConnectionException NO_PROVIDERS_EXCEPTION = + new SubscriptionConnectionException( + "Cluster has no available subscription providers to connect"); private final List initialEndpoints; @@ -183,8 +182,7 @@ protected SubscriptionConsumer(final Builder builder, final Properties propertie /////////////////////////////// open & close /////////////////////////////// - public synchronized void open() - throws TException, IoTDBConnectionException, IOException, StatementExecutionException { + public synchronized void open() throws IoTDBConnectionException { if (!isClosed.get()) { return; } @@ -255,18 +253,15 @@ void releaseWriteLock() { /////////////////////////////// subscribe & unsubscribe /////////////////////////////// - public void subscribe(final String topicName) - throws TException, IOException, StatementExecutionException, IoTDBConnectionException { + public void subscribe(final String topicName) throws SubscriptionException { subscribe(Collections.singleton(topicName)); } - public void subscribe(final String... topicNames) - throws TException, IOException, StatementExecutionException, IoTDBConnectionException { + public void subscribe(final String... topicNames) throws SubscriptionException { subscribe(new HashSet<>(Arrays.asList(topicNames))); } - public void subscribe(final Set topicNames) - throws TException, IOException, StatementExecutionException, IoTDBConnectionException { + public void subscribe(final Set topicNames) throws SubscriptionException { acquireReadLock(); try { subscribeWithRedirection(topicNames); @@ -275,18 +270,15 @@ public void subscribe(final Set topicNames) } } - public void unsubscribe(final String topicName) - throws TException, IOException, StatementExecutionException, IoTDBConnectionException { + public void unsubscribe(final String topicName) throws SubscriptionException { unsubscribe(Collections.singleton(topicName)); } - public void unsubscribe(final String... topicNames) - throws TException, IOException, StatementExecutionException, IoTDBConnectionException { + public void unsubscribe(final String... topicNames) throws SubscriptionException { unsubscribe(new HashSet<>(Arrays.asList(topicNames))); } - public void unsubscribe(final Set topicNames) - throws TException, IOException, StatementExecutionException, IoTDBConnectionException { + public void unsubscribe(final Set topicNames) throws SubscriptionException { acquireReadLock(); try { unsubscribeWithRedirection(topicNames); @@ -481,8 +473,6 @@ protected List poll(final Set topicNames, final lon new SubscriptionPollTimer(System.currentTimeMillis(), timeoutMs); do { - // update timer - timer.update(); // poll tablets or tsfile for (final SubscriptionPolledMessage polledMessage : pollInternal(topicNames)) { final short messageType = polledMessage.getMessageType(); @@ -511,8 +501,12 @@ protected List poll(final Set topicNames, final lon if (!messages.isEmpty()) { return messages; } + // update timer + timer.update(); } while (timer.notExpired()); + LOGGER.info( + "SubscriptionConsumer {} poll empty message after {} millisecond(s)", this, timeoutMs); return messages; } @@ -614,23 +608,24 @@ private SubscriptionMessage pollTsFileInternal( final SubscriptionPolledMessage polledMessage = polledMessages.get(0); final SubscriptionMessagePayload messagePayload = polledMessage.getMessagePayload(); - // check commit context - final SubscriptionCommitContext incomingCommitContext = polledMessage.getCommitContext(); - if (Objects.isNull(incomingCommitContext) - || !Objects.equals(commitContext, incomingCommitContext)) { - final String errorMessage = - String.format( - "inconsistent commit context, current is %s, incoming is %s, consumer: %s", - commitContext, incomingCommitContext, this); - LOGGER.warn(errorMessage); - throw new SubscriptionNonRetryableException(errorMessage); - } - final short messageType = polledMessage.getMessageType(); if (SubscriptionPolledMessageType.isValidatedMessageType(messageType)) { switch (SubscriptionPolledMessageType.valueOf(messageType)) { case TS_FILE_PIECE: { + // check commit context + final SubscriptionCommitContext incomingCommitContext = + polledMessage.getCommitContext(); + if (Objects.isNull(incomingCommitContext) + || !Objects.equals(commitContext, incomingCommitContext)) { + final String errorMessage = + String.format( + "inconsistent commit context, current is %s, incoming is %s, consumer: %s", + commitContext, incomingCommitContext, this); + LOGGER.warn(errorMessage); + throw new SubscriptionNonRetryableException(errorMessage); + } + // check file name if (!fileName.startsWith( ((TsFilePieceMessagePayload) messagePayload).getFileName())) { @@ -666,6 +661,19 @@ private SubscriptionMessage pollTsFileInternal( } case TS_FILE_SEAL: { + // check commit context + final SubscriptionCommitContext incomingCommitContext = + polledMessage.getCommitContext(); + if (Objects.isNull(incomingCommitContext) + || !Objects.equals(commitContext, incomingCommitContext)) { + final String errorMessage = + String.format( + "inconsistent commit context, current is %s, incoming is %s, consumer: %s", + commitContext, incomingCommitContext, this); + LOGGER.warn(errorMessage); + throw new SubscriptionNonRetryableException(errorMessage); + } + // check file name if (!fileName.startsWith(((TsFileSealMessagePayload) messagePayload).getFileName())) { final String errorMessage = @@ -870,8 +878,7 @@ protected void commitAsync( /////////////////////////////// redirection /////////////////////////////// /** Caller should ensure that the method is called in the lock {@link #acquireReadLock()}. */ - private void subscribeWithRedirection(final Set topicNames) - throws IoTDBConnectionException { + private void subscribeWithRedirection(final Set topicNames) throws SubscriptionException { for (final SubscriptionProvider provider : getAllAvailableProviders()) { try { provider.subscribe(topicNames); @@ -889,7 +896,7 @@ private void subscribeWithRedirection(final Set topicNames) /** Caller should ensure that the method is called in the lock {@link #acquireReadLock()}. */ private void unsubscribeWithRedirection(final Set topicNames) - throws IoTDBConnectionException { + throws SubscriptionException { for (final SubscriptionProvider provider : getAllAvailableProviders()) { try { provider.unsubscribe(topicNames); @@ -906,7 +913,7 @@ private void unsubscribeWithRedirection(final Set topicNames) } /** Caller should ensure that the method is called in the lock {@link #acquireReadLock()}. */ - Map fetchAllEndPointsWithRedirection() throws IoTDBConnectionException { + Map fetchAllEndPointsWithRedirection() throws SubscriptionException { Map endPoints = null; for (final SubscriptionProvider provider : getAllAvailableProviders()) { try { diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java index dd2ab632b3c8..b5db556460f6 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java @@ -20,15 +20,12 @@ package org.apache.iotdb.session.subscription; import org.apache.iotdb.rpc.IoTDBConnectionException; -import org.apache.iotdb.rpc.StatementExecutionException; import org.apache.iotdb.rpc.subscription.config.ConsumerConstant; import org.apache.iotdb.rpc.subscription.exception.SubscriptionException; -import org.apache.thrift.TException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; import java.time.Duration; import java.util.Collections; import java.util.List; @@ -93,8 +90,7 @@ private SubscriptionPullConsumer( /////////////////////////////// open & close /////////////////////////////// - public synchronized void open() - throws TException, IoTDBConnectionException, IOException, StatementExecutionException { + public synchronized void open() throws IoTDBConnectionException { if (!isClosed.get()) { return; } diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPushConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPushConsumer.java index aaf7e111860b..ffab53b7d678 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPushConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPushConsumer.java @@ -20,15 +20,12 @@ package org.apache.iotdb.session.subscription; import org.apache.iotdb.rpc.IoTDBConnectionException; -import org.apache.iotdb.rpc.StatementExecutionException; import org.apache.iotdb.rpc.subscription.config.ConsumerConstant; import org.apache.iotdb.rpc.subscription.exception.SubscriptionException; -import org.apache.thrift.TException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; import java.util.Collections; import java.util.List; import java.util.Properties; @@ -78,8 +75,7 @@ private SubscriptionPushConsumer( /////////////////////////////// open & close /////////////////////////////// - public synchronized void open() - throws TException, IoTDBConnectionException, IOException, StatementExecutionException { + public synchronized void open() throws IoTDBConnectionException { if (!isClosed.get()) { return; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionConsumerAgent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionConsumerAgent.java index d8cd497ff4bb..d6572923635c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionConsumerAgent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionConsumerAgent.java @@ -76,7 +76,7 @@ public TPushConsumerGroupMetaRespExceptionMessage handleSingleConsumerGroupMetaC final String exceptionMessage = String.format( "Subscription: Failed to handle single consumer group meta changes for consumer group %s, because %s", - consumerGroupId, e.getMessage()); + consumerGroupId, e); LOGGER.warn(exceptionMessage); return new TPushConsumerGroupMetaRespExceptionMessage( consumerGroupId, exceptionMessage, System.currentTimeMillis()); @@ -133,7 +133,7 @@ public TPushConsumerGroupMetaRespExceptionMessage handleConsumerGroupMetaChanges final String exceptionMessage = String.format( "Subscription: Failed to handle single consumer group meta changes for consumer group %s, because %s", - consumerGroupId, e.getMessage()); + consumerGroupId, e); LOGGER.warn(exceptionMessage); return new TPushConsumerGroupMetaRespExceptionMessage( consumerGroupId, exceptionMessage, System.currentTimeMillis()); @@ -154,8 +154,7 @@ public TPushConsumerGroupMetaRespExceptionMessage handleDropConsumerGroup( } catch (Exception e) { final String exceptionMessage = String.format( - "Subscription: Failed to drop consumer group %s, because %s", - consumerGroupId, e.getMessage()); + "Subscription: Failed to drop consumer group %s, because %s", consumerGroupId, e); LOGGER.warn(exceptionMessage); return new TPushConsumerGroupMetaRespExceptionMessage( consumerGroupId, exceptionMessage, System.currentTimeMillis()); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionTopicAgent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionTopicAgent.java index 5190f58550bf..fb27e8825b8e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionTopicAgent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionTopicAgent.java @@ -70,7 +70,7 @@ public TPushTopicMetaRespExceptionMessage handleSingleTopicMetaChanges( final String exceptionMessage = String.format( "Subscription: Failed to handle single topic meta changes for topic %s, because %s", - topicName, e.getMessage()); + topicName, e); LOGGER.warn(exceptionMessage); return new TPushTopicMetaRespExceptionMessage( topicName, exceptionMessage, System.currentTimeMillis()); @@ -97,7 +97,7 @@ public TPushTopicMetaRespExceptionMessage handleTopicMetaChanges( final String exceptionMessage = String.format( "Subscription: Failed to handle single topic meta changes for topic %s, because %s", - topicName, e.getMessage()); + topicName, e); LOGGER.warn(exceptionMessage); return new TPushTopicMetaRespExceptionMessage( topicName, exceptionMessage, System.currentTimeMillis()); @@ -116,8 +116,7 @@ public TPushTopicMetaRespExceptionMessage handleDropTopic(String topicName) { return null; } catch (Exception e) { final String exceptionMessage = - String.format( - "Subscription: Failed to drop topic %s, because %s", topicName, e.getMessage()); + String.format("Subscription: Failed to drop topic %s, because %s", topicName, e); LOGGER.warn(exceptionMessage); return new TPushTopicMetaRespExceptionMessage( topicName, exceptionMessage, System.currentTimeMillis()); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java index 0808ed2d73f8..64bfd77f2bea 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java @@ -259,7 +259,7 @@ public SubscriptionTsFileEvent poll(final String consumerId) { final String errorMessage = String.format( "IOException occurred when SubscriptionPrefetchingTsFileQueue %s transferring TsFile (with event %s) to consumer %s: %s", - this, event, consumerId, e.getMessage()); + this, event, consumerId, e); LOGGER.warn(errorMessage); // assume retryable return generateSubscriptionTsFileEventWithErrorMessage(errorMessage, true); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionTsFileEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionTsFileEvent.java index 3db03f5f6127..937a365da8d3 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionTsFileEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionTsFileEvent.java @@ -74,9 +74,9 @@ public void prefetchNext() { return generateSubscriptionTsFileEventWithPieceOrSealPayload(0); } catch (final IOException e) { LOGGER.warn( - "IOException occurred when prefetching next SubscriptionTsFileEvent: {}, current SubscriptionTsFileEvent: {}", - e.getMessage(), - this); + "IOException occurred when prefetching next SubscriptionTsFileEvent, current SubscriptionTsFileEvent: {}", + this, + e); return null; } case TS_FILE_PIECE: @@ -85,9 +85,9 @@ public void prefetchNext() { ((TsFilePieceMessagePayload) messagePayload).getNextWritingOffset()); } catch (final IOException e) { LOGGER.warn( - "IOException occurred when prefetching next SubscriptionTsFileEvent: {}, current SubscriptionTsFileEvent: {}", - e.getMessage(), - this); + "IOException occurred when prefetching next SubscriptionTsFileEvent, current SubscriptionTsFileEvent: {}", + this, + e); return null; } case TS_FILE_SEAL: diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java index b0dd0bc5b5e6..395656671814 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java @@ -146,8 +146,7 @@ private TPipeSubscribeResp handlePipeSubscribeHandshake(final PipeSubscribeHands } catch (final SubscriptionException e) { final String exceptionMessage = String.format( - "Subscription: something unexpected happened when handshaking: %s, req: %s", - e.getMessage(), req); + "Subscription: something unexpected happened when handshaking: %s, req: %s", e, req); LOGGER.warn(exceptionMessage); return PipeSubscribeHandshakeResp.toTPipeSubscribeResp( RpcUtils.getStatus(TSStatusCode.SUBSCRIPTION_HANDSHAKE_ERROR, exceptionMessage)); @@ -198,8 +197,7 @@ private TPipeSubscribeResp handlePipeSubscribeHeartbeat(final PipeSubscribeHeart } catch (final SubscriptionException e) { final String exceptionMessage = String.format( - "Subscription: something unexpected happened when heartbeat: %s, req: %s", - e.getMessage(), req); + "Subscription: something unexpected happened when heartbeat: %s, req: %s", e, req); LOGGER.warn(exceptionMessage); return PipeSubscribeHandshakeResp.toTPipeSubscribeResp( RpcUtils.getStatus(TSStatusCode.SUBSCRIPTION_HEARTBEAT_ERROR, exceptionMessage)); @@ -228,8 +226,7 @@ private TPipeSubscribeResp handlePipeSubscribeSubscribe(final PipeSubscribeSubsc } catch (final SubscriptionException e) { final String exceptionMessage = String.format( - "Subscription: something unexpected happened when subscribing: %s, req: %s", - e.getMessage(), req); + "Subscription: something unexpected happened when subscribing: %s, req: %s", e, req); LOGGER.warn(exceptionMessage); return PipeSubscribeHandshakeResp.toTPipeSubscribeResp( RpcUtils.getStatus(TSStatusCode.SUBSCRIPTION_SUBSCRIBE_ERROR, exceptionMessage)); @@ -262,7 +259,7 @@ private TPipeSubscribeResp handlePipeSubscribeUnsubscribe(final PipeSubscribeUns final String exceptionMessage = String.format( "Subscription: something unexpected happened when unsubscribing: %s, req: %s", - e.getMessage(), req); + e, req); LOGGER.warn(exceptionMessage); return PipeSubscribeHandshakeResp.toTPipeSubscribeResp( RpcUtils.getStatus(TSStatusCode.SUBSCRIPTION_UNSUBSCRIBE_ERROR, exceptionMessage)); @@ -318,8 +315,7 @@ private TPipeSubscribeResp handlePipeSubscribePoll(final PipeSubscribePollReq re } catch (final SubscriptionException e) { final String exceptionMessage = String.format( - "Subscription: something unexpected happened when polling: %s, req: %s", - e.getMessage(), req); + "Subscription: something unexpected happened when polling: %s, req: %s", e, req); LOGGER.warn(exceptionMessage); return PipeSubscribeHandshakeResp.toTPipeSubscribeResp( RpcUtils.getStatus(TSStatusCode.SUBSCRIPTION_POLL_ERROR, exceptionMessage)); @@ -397,8 +393,7 @@ private TPipeSubscribeResp handlePipeSubscribeCommit(final PipeSubscribeCommitRe } catch (final SubscriptionException e) { final String exceptionMessage = String.format( - "Subscription: something unexpected happened when committing: %s, req: %s", - e.getMessage(), req); + "Subscription: something unexpected happened when committing: %s, req: %s", e, req); LOGGER.warn(exceptionMessage); return PipeSubscribeHandshakeResp.toTPipeSubscribeResp( RpcUtils.getStatus(TSStatusCode.SUBSCRIPTION_COMMIT_ERROR, exceptionMessage)); @@ -441,8 +436,7 @@ private TPipeSubscribeResp handlePipeSubscribeClose(final PipeSubscribeCloseReq } catch (final SubscriptionException e) { final String exceptionMessage = String.format( - "Subscription: something unexpected happened when closing: %s, req: %s", - e.getMessage(), req); + "Subscription: something unexpected happened when closing: %s, req: %s", e, req); LOGGER.warn(exceptionMessage); return PipeSubscribeHandshakeResp.toTPipeSubscribeResp( RpcUtils.getStatus(TSStatusCode.SUBSCRIPTION_CLOSE_ERROR, exceptionMessage)); @@ -508,7 +502,7 @@ private void createConsumer(final ConsumerConfig consumerConfig) throws Subscrip final String exceptionMessage = String.format( "Subscription: Failed to create consumer %s in config node, exception is %s.", - consumerConfig, e.getMessage()); + consumerConfig, e); LOGGER.warn(exceptionMessage); throw new SubscriptionException(exceptionMessage); } @@ -534,7 +528,7 @@ private void dropConsumer(final ConsumerConfig consumerConfig) throws Subscripti final String exceptionMessage = String.format( "Subscription: Failed to close consumer %s in config node, exception is %s.", - consumerConfig, e.getMessage()); + consumerConfig, e); LOGGER.warn(exceptionMessage); throw new SubscriptionException(exceptionMessage); } @@ -564,7 +558,7 @@ private void subscribe(final ConsumerConfig consumerConfig, final Set to final String exceptionMessage = String.format( "Subscription: Failed to subscribe topics %s for consumer %s in config node, exception is %s.", - topicNames, consumerConfig, e.getMessage()); + topicNames, consumerConfig, e); LOGGER.warn(exceptionMessage); throw new SubscriptionException(exceptionMessage); } @@ -592,7 +586,7 @@ private void unsubscribe(final ConsumerConfig consumerConfig, final Set final String exceptionMessage = String.format( "Subscription: Failed to unsubscribe topics %s for consumer %s in config node, exception is %s.", - topicNames, consumerConfig, e.getMessage()); + topicNames, consumerConfig, e); LOGGER.warn(exceptionMessage); throw new SubscriptionException(exceptionMessage); } From 0b9b459e9d182c725f2425e289f17dfa6833326c Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Tue, 23 Apr 2024 22:28:04 +0800 Subject: [PATCH 56/93] add comment --- .../apache/iotdb/session/subscription/SubscriptionConsumer.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java index 7d0a5ed8a41d..eb93298100b7 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java @@ -712,6 +712,8 @@ private SubscriptionMessage pollTsFileInternal( } case TS_FILE_ERROR: { + // no need to check commit context + final String errorMessage = ((TsFileErrorMessagePayload) messagePayload).getErrorMessage(); final boolean retryable = ((TsFileErrorMessagePayload) messagePayload).isRetryable(); From 8f81d506c9b86340ccee96158d9ba3a13ce05c27 Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Wed, 24 Apr 2024 13:44:00 +0800 Subject: [PATCH 57/93] improve --- .../exception/SubscriptionParameterNotValidException.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionParameterNotValidException.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionParameterNotValidException.java index 5c903a94d648..322a65776f0e 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionParameterNotValidException.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionParameterNotValidException.java @@ -21,7 +21,7 @@ import java.util.Objects; -public class SubscriptionParameterNotValidException extends SubscriptionException { +public class SubscriptionParameterNotValidException extends SubscriptionNonRetryableException { public SubscriptionParameterNotValidException(String message) { super(message); From eb450d18401fca8417e26722333e202d9d5cafeb Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Wed, 24 Apr 2024 13:53:33 +0800 Subject: [PATCH 58/93] rename --- .../broker/SubscriptionPrefetchingTsFileQueue.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java index 64bfd77f2bea..a50e5ecb3838 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java @@ -64,7 +64,7 @@ public SubscriptionPrefetchingTsFileQueue( @Override public SubscriptionTsFileEvent poll(final String consumerId) { - if (hasPollableOnTheFlySubscriptionTsFileEvent(consumerId)) { + if (hasUnPollableOnTheFlySubscriptionTsFileEvent(consumerId)) { return null; } @@ -289,7 +289,8 @@ public synchronized void executePrefetch() { /////////////////////////////// utility /////////////////////////////// - private synchronized boolean hasPollableOnTheFlySubscriptionTsFileEvent(final String consumerId) { + private synchronized boolean hasUnPollableOnTheFlySubscriptionTsFileEvent( + final String consumerId) { final SubscriptionTsFileEvent event = consumerIdToCurrentEventMap.get(consumerId); if (Objects.isNull(event)) { return false; From 9da4e337bfe0fc49dc58db2b0d4be52e5ce29591 Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Mon, 6 May 2024 11:41:12 +0800 Subject: [PATCH 59/93] spotless --- .../it/dual/IoTDBSubscriptionConsumerGroupIT.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionConsumerGroupIT.java b/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionConsumerGroupIT.java index 3ab7f857b713..fecd7e8168df 100644 --- a/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionConsumerGroupIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionConsumerGroupIT.java @@ -963,7 +963,9 @@ private boolean insertRowRecordEnrichedByConsumerGroupId( return insertRowRecordEnrichedByConsumerGroupId(columnName, record, consumerGroupId); } - /** @return false -> receiver crashed */ + /** + * @return false -> receiver crashed + */ private boolean insertRowRecordEnrichedByConsumerGroupId( final String columnName, final RowRecord record, final String consumerGroupId) throws Exception { From 9de9b81a47b2e304dde47627ad3ab9aa7397d086 Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Mon, 6 May 2024 11:51:13 +0800 Subject: [PATCH 60/93] fix --- .../SubscriptionPrefetchingTabletsQueue.java | 22 ++++++++++--------- 1 file changed, 12 insertions(+), 10 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTabletsQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTabletsQueue.java index 9a4734f2f21c..061da48a2d74 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTabletsQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTabletsQueue.java @@ -39,6 +39,7 @@ import org.slf4j.LoggerFactory; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.Objects; import java.util.concurrent.LinkedBlockingQueue; @@ -129,11 +130,11 @@ private void prefetchOnce(final long limit) { } if (event instanceof TabletInsertionEvent) { - final Tablet tablet = convertToTablet((TabletInsertionEvent) event); - if (Objects.isNull(tablet)) { + final List currentTablets = convertToTablets((TabletInsertionEvent) event); + if (currentTablets.isEmpty()) { continue; } - tablets.add(tablet); + tablets.addAll(currentTablets); enrichedEvents.add((EnrichedEvent) event); if (tablets.size() >= limit) { break; @@ -141,11 +142,11 @@ private void prefetchOnce(final long limit) { } else if (event instanceof PipeTsFileInsertionEvent) { for (final TabletInsertionEvent tabletInsertionEvent : ((PipeTsFileInsertionEvent) event).toTabletInsertionEvents()) { - final Tablet tablet = convertToTablet(tabletInsertionEvent); - if (Objects.isNull(tablet)) { + final List currentTablets = convertToTablets(tabletInsertionEvent); + if (Objects.isNull(currentTablets)) { continue; } - tablets.add(tablet); + tablets.addAll(currentTablets); } enrichedEvents.add((EnrichedEvent) event); if (tablets.size() >= limit) { @@ -215,17 +216,18 @@ private void serializeOnce() { /////////////////////////////// utility /////////////////////////////// - private Tablet convertToTablet(final TabletInsertionEvent tabletInsertionEvent) { + private List convertToTablets(final TabletInsertionEvent tabletInsertionEvent) { if (tabletInsertionEvent instanceof PipeInsertNodeTabletInsertionEvent) { - return ((PipeInsertNodeTabletInsertionEvent) tabletInsertionEvent).convertToTablet(); + return ((PipeInsertNodeTabletInsertionEvent) tabletInsertionEvent).convertToTablets(); } else if (tabletInsertionEvent instanceof PipeRawTabletInsertionEvent) { - return ((PipeRawTabletInsertionEvent) tabletInsertionEvent).convertToTablet(); + return Collections.singletonList( + ((PipeRawTabletInsertionEvent) tabletInsertionEvent).convertToTablet()); } LOGGER.warn( "Subscription: SubscriptionPrefetchingTabletsQueue {} only support convert PipeInsertNodeTabletInsertionEvent or PipeRawTabletInsertionEvent to tablet. Ignore {}.", this, tabletInsertionEvent); - return null; + return Collections.emptyList(); } } From f4fa0afb3b9def3657ccb2419e40a0ebb35ea65d Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Mon, 6 May 2024 20:52:26 +0800 Subject: [PATCH 61/93] fixup --- .../iotdb/db/subscription/receiver/SubscriptionReceiverV1.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java index 87e90e6fbc1f..b6ead8b9b4da 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java @@ -456,7 +456,7 @@ private TPipeSubscribeResp handlePipeSubscribeClose(final PipeSubscribeCloseReq "Subscription: something unexpected happened when closing: %s, req: %s", e, req); LOGGER.warn(exceptionMessage); return PipeSubscribeCloseResp.toTPipeSubscribeResp( - RpcUtils.getStatus(TSStatusCode.SUBSCRIPTION_COMMIT_ERROR, exceptionMessage)); + RpcUtils.getStatus(TSStatusCode.SUBSCRIPTION_CLOSE_ERROR, exceptionMessage)); } } From 3f5cbc8da679e17f0b0b6bfbd2aaa8a4c131263c Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Wed, 8 May 2024 18:27:38 +0800 Subject: [PATCH 62/93] minor improve --- .../it/dual/IoTDBSubscriptionTopicIT.java | 4 +-- .../common/SubscriptionPolledMessage.java | 20 +++++++++----- ...SubscriptionPolledMessageBinaryCache.java} | 26 +++++++++---------- 3 files changed, 28 insertions(+), 22 deletions(-) rename iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/{EnrichedTabletsBinaryCache.java => SubscriptionPolledMessageBinaryCache.java} (68%) diff --git a/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionTopicIT.java b/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionTopicIT.java index f6bc9e809602..dd7bbc00a6ac 100644 --- a/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionTopicIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionTopicIT.java @@ -517,7 +517,7 @@ public void testTopicInvalidConfig() throws Exception { final Properties properties = new Properties(); properties.put(TopicConstant.START_TIME_KEY, "2001.01.01T08:00:00"); properties.put(TopicConstant.END_TIME_KEY, "2000.01.01T08:00:00"); - session.createTopic("topic1", properties); + session.createTopic("topic2", properties); fail(); } catch (final Exception ignored) { } @@ -549,7 +549,7 @@ public void testTopicInvalidConfig() throws Exception { for (final Properties config : configs) { try (final SubscriptionSession session = new SubscriptionSession(host, port)) { session.open(); - session.createTopic("foo", config); + session.createTopic("topic3", config); fail(); } catch (final Exception ignored) { } diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledMessage.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledMessage.java index cafecd4a7daa..89effe55d25a 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledMessage.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledMessage.java @@ -125,16 +125,11 @@ public static SubscriptionPolledMessage deserialize(final ByteBuffer buffer) { public boolean trySerialize() { if (Objects.isNull(byteBuffer)) { try { - try (final PublicBAOS byteArrayOutputStream = new PublicBAOS(); - final DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream)) { - serialize(outputStream); - byteBuffer = - ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size()); - } + serialize(); return true; } catch (final IOException e) { LOGGER.warn( - "Subscription: something unexpected happened when serializing SubscriptionRawMessage", + "Subscription: something unexpected happened when serializing SubscriptionPolledMessage", e); } return false; @@ -142,6 +137,17 @@ public boolean trySerialize() { return true; } + public void serialize() throws IOException { + if (Objects.isNull(byteBuffer)) { + try (final PublicBAOS byteArrayOutputStream = new PublicBAOS(); + final DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream)) { + serialize(outputStream); + byteBuffer = + ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size()); + } + } + } + public void resetByteBuffer() { // maybe friendly for gc byteBuffer = null; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/EnrichedTabletsBinaryCache.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPolledMessageBinaryCache.java similarity index 68% rename from iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/EnrichedTabletsBinaryCache.java rename to iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPolledMessageBinaryCache.java index dc7e9ec47c17..ecc890bc9b14 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/EnrichedTabletsBinaryCache.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPolledMessageBinaryCache.java @@ -19,43 +19,43 @@ package org.apache.iotdb.db.subscription.broker; -import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.db.pipe.resource.PipeResourceManager; import org.apache.iotdb.db.pipe.resource.memory.PipeMemoryBlock; +import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPolledMessage; import com.github.benmanes.caffeine.cache.CacheLoader; import com.github.benmanes.caffeine.cache.Caffeine; import com.github.benmanes.caffeine.cache.LoadingCache; import com.github.benmanes.caffeine.cache.Weigher; -import org.apache.tsfile.utils.Pair; import org.checkerframework.checker.nullness.qual.NonNull; import org.checkerframework.checker.nullness.qual.Nullable; +import java.io.IOException; import java.nio.ByteBuffer; -// TODO -public class EnrichedTabletsBinaryCache { +public class SubscriptionPolledMessageBinaryCache { private final PipeMemoryBlock allocatedMemoryBlock; - private final LoadingCache> cache; + private final LoadingCache cache; - public EnrichedTabletsBinaryCache() { - // TODO: config + public SubscriptionPolledMessageBinaryCache() { this.allocatedMemoryBlock = PipeResourceManager.memory().tryAllocate(Runtime.getRuntime().maxMemory() / 50); this.cache = Caffeine.newBuilder() .maximumWeight(this.allocatedMemoryBlock.getMemoryUsageInBytes()) .weigher( - (Weigher>) - (id, enrichedTablets) -> enrichedTablets.left.limit()) + (Weigher) + (message, buffer) -> buffer.limit()) .build( - new CacheLoader>() { + new CacheLoader() { @Override - public @Nullable Pair load(@NonNull Long aLong) - throws Exception { - return null; + public @Nullable ByteBuffer load( + @NonNull final SubscriptionPolledMessage subscriptionPolledMessage) + throws IOException { + subscriptionPolledMessage.serialize(); + return subscriptionPolledMessage.getByteBuffer(); } }); } From d204b2a93ac4db76957bf2f1dd19430c45a02486 Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Fri, 10 May 2024 16:44:01 +0800 Subject: [PATCH 63/93] intergrate with metrics framework --- .../subscription/SubscriptionSession.java | 4 +- .../metrics/DataNodeMetricsHelper.java | 4 + .../broker/SubscriptionBroker.java | 33 ++- .../broker/SubscriptionPrefetchingQueue.java | 14 ++ .../metric/SubscriptionMetrics.java | 57 +++++ .../SubscriptionPrefetchingQueueMetrics.java | 197 ++++++++++++++++++ .../commons/service/metric/enums/Metric.java | 4 + 7 files changed, 304 insertions(+), 9 deletions(-) create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/metric/SubscriptionMetrics.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/metric/SubscriptionPrefetchingQueueMetrics.java diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionSession.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionSession.java index 41da7a3528f9..7c5589e64b79 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionSession.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionSession.java @@ -61,7 +61,9 @@ public SubscriptionSession( // disable auto fetch .enableAutoFetch(false) // disable redirection - .enableRedirection(false)); + .enableRedirection(false) + // TODO: config + .thriftMaxFrameSize(2147483647)); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/metrics/DataNodeMetricsHelper.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/metrics/DataNodeMetricsHelper.java index 680585d99642..ade06cb875f5 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/metrics/DataNodeMetricsHelper.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/metrics/DataNodeMetricsHelper.java @@ -40,6 +40,7 @@ import org.apache.iotdb.db.queryengine.metric.QueryRelatedResourceMetricSet; import org.apache.iotdb.db.queryengine.metric.QueryResourceMetricSet; import org.apache.iotdb.db.queryengine.metric.SeriesScanCostMetricSet; +import org.apache.iotdb.db.subscription.metric.SubscriptionMetrics; import org.apache.iotdb.metrics.metricsets.UpTimeMetrics; import org.apache.iotdb.metrics.metricsets.disk.DiskMetrics; import org.apache.iotdb.metrics.metricsets.jvm.JvmMetrics; @@ -89,6 +90,9 @@ public static void bind() { // bind load tsfile memory related metrics MetricService.getInstance().addMetricSet(LoadTsFileMemMetricSet.getInstance()); + + // bind subscription related metrics + MetricService.getInstance().addMetricSet(SubscriptionMetrics.getInstance()); } private static void initSystemMetrics() { 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 1ed7267831fe..c7184bb7f823 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 @@ -22,6 +22,7 @@ import org.apache.iotdb.commons.pipe.task.connection.BoundedBlockingPendingQueue; import org.apache.iotdb.db.subscription.agent.SubscriptionAgent; import org.apache.iotdb.db.subscription.event.SubscriptionEvent; +import org.apache.iotdb.db.subscription.metric.SubscriptionPrefetchingQueueMetrics; import org.apache.iotdb.pipe.api.event.Event; import org.apache.iotdb.rpc.subscription.config.TopicConstant; import org.apache.iotdb.rpc.subscription.exception.SubscriptionException; @@ -66,6 +67,12 @@ public List poll(final String consumerId, final Set t if (topicNames.contains(topicName)) { final SubscriptionEvent event = prefetchingQueue.poll(consumerId); if (Objects.nonNull(event)) { + if (event.getMessage().trySerialize()) { + SubscriptionPrefetchingQueueMetrics.getInstance() + .mark( + prefetchingQueue.getPrefetchingQueueId(), + event.getMessage().getByteBuffer().limit()); + } events.add(event); } } @@ -94,9 +101,15 @@ public List pollTsFile( LOGGER.warn(errorMessage); throw new SubscriptionException(errorMessage); } - return Collections.singletonList( + final SubscriptionEvent event = ((SubscriptionPrefetchingTsFileQueue) prefetchingQueue) - .pollTsFile(consumerId, fileName, writingOffset)); + .pollTsFile(consumerId, fileName, writingOffset); + if (event.getMessage().trySerialize()) { + SubscriptionPrefetchingQueueMetrics.getInstance() + .mark( + prefetchingQueue.getPrefetchingQueueId(), event.getMessage().getByteBuffer().limit()); + } + return Collections.singletonList(event); } /** @@ -134,13 +147,15 @@ public void bindPrefetchingQueue( } final String topicFormat = SubscriptionAgent.topic().getTopicFormat(topicName); if (TopicConstant.FORMAT_TS_FILE_READER_VALUE.equals(topicFormat)) { - topicNameToPrefetchingQueue.put( - topicName, - new SubscriptionPrefetchingTsFileQueue(brokerId, topicName, inputPendingQueue)); + final SubscriptionPrefetchingQueue queue = + new SubscriptionPrefetchingTsFileQueue(brokerId, topicName, inputPendingQueue); + SubscriptionPrefetchingQueueMetrics.getInstance().register(queue); + topicNameToPrefetchingQueue.put(topicName, queue); } else { - topicNameToPrefetchingQueue.put( - topicName, - new SubscriptionPrefetchingTabletsQueue(brokerId, topicName, inputPendingQueue)); + final SubscriptionPrefetchingQueue queue = + new SubscriptionPrefetchingTabletsQueue(brokerId, topicName, inputPendingQueue); + SubscriptionPrefetchingQueueMetrics.getInstance().register(queue); + topicNameToPrefetchingQueue.put(topicName, queue); } } @@ -153,6 +168,8 @@ public void unbindPrefetchingQueue(final String topicName) { } // TODO: do something for events on-the-fly topicNameToPrefetchingQueue.remove(topicName); + SubscriptionPrefetchingQueueMetrics.getInstance() + .deregister(prefetchingQueue.getPrefetchingQueueId()); } public void executePrefetch(final String topicName) { 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 84b55bd83718..40b8d540a6fb 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 @@ -106,4 +106,18 @@ protected SubscriptionCommitContext generateInvalidSubscriptionCommitContext() { public String toString() { return "SubscriptionPrefetchingQueue{brokerId=" + brokerId + ", topicName=" + topicName + "}"; } + + //////////////////////////// APIs provided for metric framework //////////////////////////// + + public String getPrefetchingQueueId() { + return brokerId + "_" + topicName; + } + + public long getUncommittedEventCount() { + return uncommittedEvents.size(); + } + + public long getCurrentCommitId() { + return subscriptionCommitIdGenerator.get(); + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/metric/SubscriptionMetrics.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/metric/SubscriptionMetrics.java new file mode 100644 index 000000000000..48a6dc50e6d4 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/metric/SubscriptionMetrics.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.db.subscription.metric; + +import org.apache.iotdb.metrics.AbstractMetricService; +import org.apache.iotdb.metrics.metricsets.IMetricSet; + +public class SubscriptionMetrics implements IMetricSet { + + //////////////////////////// bindTo & unbindFrom (metric framework) //////////////////////////// + + @Override + public void bindTo(final AbstractMetricService metricService) { + SubscriptionPrefetchingQueueMetrics.getInstance().bindTo(metricService); + } + + @Override + public void unbindFrom(final AbstractMetricService metricService) { + SubscriptionPrefetchingQueueMetrics.getInstance().unbindFrom(metricService); + } + + //////////////////////////// singleton //////////////////////////// + + private static class SubscriptionMetricsHolder { + + private static final SubscriptionMetrics INSTANCE = new SubscriptionMetrics(); + + private SubscriptionMetricsHolder() { + // empty constructor + } + } + + public static SubscriptionMetrics getInstance() { + return SubscriptionMetrics.SubscriptionMetricsHolder.INSTANCE; + } + + private SubscriptionMetrics() { + // empty constructor + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/metric/SubscriptionPrefetchingQueueMetrics.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/metric/SubscriptionPrefetchingQueueMetrics.java new file mode 100644 index 000000000000..d38de0d709e1 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/metric/SubscriptionPrefetchingQueueMetrics.java @@ -0,0 +1,197 @@ +/* + * 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.subscription.metric; + +import org.apache.iotdb.commons.service.metric.enums.Metric; +import org.apache.iotdb.commons.service.metric.enums.Tag; +import org.apache.iotdb.db.subscription.broker.SubscriptionPrefetchingQueue; +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.Map; +import java.util.Objects; +import java.util.concurrent.ConcurrentHashMap; + +public class SubscriptionPrefetchingQueueMetrics implements IMetricSet { + + private static final Logger LOGGER = + LoggerFactory.getLogger(SubscriptionPrefetchingQueueMetrics.class); + + private volatile AbstractMetricService metricService; + + private final Map prefetchingQueueMap = + new ConcurrentHashMap<>(); + + private final Map rateMap = new ConcurrentHashMap<>(); + + @Override + public void bindTo(final AbstractMetricService metricService) { + this.metricService = metricService; + final ImmutableSet ids = ImmutableSet.copyOf(prefetchingQueueMap.keySet()); + for (final String id : ids) { + createMetrics(id); + } + } + + @Override + public void unbindFrom(final AbstractMetricService metricService) { + final ImmutableSet ids = ImmutableSet.copyOf(prefetchingQueueMap.keySet()); + for (final String id : ids) { + deregister(id); + } + if (!prefetchingQueueMap.isEmpty()) { + LOGGER.warn( + "Failed to unbind from subscription prefetching queue metrics, prefetching queue map not empty"); + } + } + + //////////////////////////// register & deregister //////////////////////////// + + public void register(@NonNull final SubscriptionPrefetchingQueue prefetchingQueue) { + final String id = prefetchingQueue.getPrefetchingQueueId(); + prefetchingQueueMap.putIfAbsent(id, prefetchingQueue); + if (Objects.nonNull(metricService)) { + createMetrics(id); + } + } + + private void createMetrics(final String id) { + createAutoGauge(id); + createRate(id); + } + + private void createAutoGauge(final String id) { + final SubscriptionPrefetchingQueue queue = prefetchingQueueMap.get(id); + // uncommited event count + metricService.createAutoGauge( + Metric.SUBSCRIPTION_UNCOMMITTED_EVENT_COUNT.toString(), + MetricLevel.IMPORTANT, + queue, + SubscriptionPrefetchingQueue::getUncommittedEventCount, + Tag.NAME.toString(), + queue.getPrefetchingQueueId()); + // current commit id + metricService.createAutoGauge( + Metric.SUBSCRIPTION_CURRENT_COMMIT_ID.toString(), + MetricLevel.IMPORTANT, + queue, + SubscriptionPrefetchingQueue::getCurrentCommitId, + Tag.NAME.toString(), + queue.getPrefetchingQueueId()); + } + + private void createRate(final String id) { + final SubscriptionPrefetchingQueue queue = prefetchingQueueMap.get(id); + // transfer event rate + rateMap.put( + id, + metricService.getOrCreateRate( + Metric.SUBSCRIPTION_EVENT_TRANSFER.toString(), + MetricLevel.IMPORTANT, + Tag.NAME.toString(), + queue.getPrefetchingQueueId())); + } + + public void deregister(final String id) { + if (!prefetchingQueueMap.containsKey(id)) { + LOGGER.warn( + "Failed to deregister subscription prefetching queue metrics, SubscriptionPrefetchingQueue({}) does not exist", + id); + return; + } + if (Objects.nonNull(metricService)) { + removeMetrics(id); + } + prefetchingQueueMap.remove(id); + } + + private void removeMetrics(final String id) { + removeAutoGauge(id); + removeRate(id); + } + + private void removeAutoGauge(final String id) { + final SubscriptionPrefetchingQueue queue = prefetchingQueueMap.get(id); + // uncommited event count + metricService.remove( + MetricType.AUTO_GAUGE, + Metric.SUBSCRIPTION_UNCOMMITTED_EVENT_COUNT.toString(), + Tag.NAME.toString(), + queue.getPrefetchingQueueId()); + // current commit id + metricService.remove( + MetricType.AUTO_GAUGE, + Metric.SUBSCRIPTION_CURRENT_COMMIT_ID.toString(), + Tag.NAME.toString(), + queue.getPrefetchingQueueId()); + } + + private void removeRate(final String id) { + final SubscriptionPrefetchingQueue queue = prefetchingQueueMap.get(id); + // transfer event rate + metricService.remove( + MetricType.RATE, + Metric.SUBSCRIPTION_EVENT_TRANSFER.toString(), + Tag.NAME.toString(), + queue.getPrefetchingQueueId()); + } + + public void mark(final String id, final long size) { + if (Objects.isNull(metricService)) { + return; + } + final Rate rate = rateMap.get(id); + if (rate == null) { + LOGGER.warn( + "Failed to mark transfer event rate, SubscriptionPrefetchingQueue({}) does not exist", + id); + return; + } + rate.mark(size); + } + + //////////////////////////// singleton //////////////////////////// + + private static class SubscriptionPrefetchingQueueMetricsHolder { + + private static final SubscriptionPrefetchingQueueMetrics INSTANCE = + new SubscriptionPrefetchingQueueMetrics(); + + private SubscriptionPrefetchingQueueMetricsHolder() { + // empty constructor + } + } + + public static SubscriptionPrefetchingQueueMetrics getInstance() { + return SubscriptionPrefetchingQueueMetrics.SubscriptionPrefetchingQueueMetricsHolder.INSTANCE; + } + + private SubscriptionPrefetchingQueueMetrics() { + // empty constructor + } +} 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..994ba724f385 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,10 @@ public enum Metric { PIPE_EVENT_COMMIT_QUEUE_SIZE("pipe_event_commit_queue_size"), PIPE_PROCEDURE("pipe_procedure"), PIPE_TASK_STATUS("pipe_task_status"), + // subscription related + SUBSCRIPTION_UNCOMMITTED_EVENT_COUNT("subscription_uncommitted_event_count"), + SUBSCRIPTION_CURRENT_COMMIT_ID("subscription_current_commit_id"), + SUBSCRIPTION_EVENT_TRANSFER("subscription_event_transfer"), // load related LOAD_MEM("load_mem"); From 75ad4a18179319b0d13f0ebda75f091ff759870d Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Mon, 13 May 2024 16:59:03 +0800 Subject: [PATCH 64/93] improve ex msg for sub client --- .../subscription/SubscriptionConsumer.java | 87 +++++++++++++------ .../SubscriptionEndpointsSyncer.java | 7 +- .../subscription/SubscriptionProvider.java | 9 +- 3 files changed, 68 insertions(+), 35 deletions(-) diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java index 9d1085114a63..984805c46eed 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java @@ -76,10 +76,6 @@ public abstract class SubscriptionConsumer implements AutoCloseable { private static final Logger LOGGER = LoggerFactory.getLogger(SubscriptionConsumer.class); - private static final SubscriptionConnectionException NO_PROVIDERS_EXCEPTION = - new SubscriptionConnectionException( - "Cluster has no available subscription providers to connect"); - private final List initialEndpoints; private final String username; @@ -182,7 +178,7 @@ protected SubscriptionConsumer(final Builder builder, final Properties propertie /////////////////////////////// open & close /////////////////////////////// - public synchronized void open() throws IoTDBConnectionException { + public synchronized void open() throws SubscriptionException, IoTDBConnectionException { if (!isClosed.get()) { return; } @@ -190,7 +186,7 @@ public synchronized void open() throws IoTDBConnectionException { // open subscription providers acquireWriteLock(); try { - openProviders(); // throw IoTDBConnectionException + openProviders(); // throw SubscriptionException or IoTDBConnectionException } finally { releaseWriteLock(); } @@ -205,7 +201,7 @@ public synchronized void open() throws IoTDBConnectionException { } @Override - public synchronized void close() throws IoTDBConnectionException { + public synchronized void close() throws SubscriptionException, IoTDBConnectionException { if (isClosed.get()) { return; } @@ -220,7 +216,7 @@ public synchronized void close() throws IoTDBConnectionException { // close subscription providers acquireWriteLock(); try { - closeProviders(); + closeProviders(); // throw SubscriptionException or IoTDBConnectionException } finally { releaseWriteLock(); } @@ -371,7 +367,7 @@ SubscriptionProvider constructProviderAndHandshake(final TEndPoint endPoint) } /** Caller should ensure that the method is called in the lock {@link #acquireWriteLock()}. */ - void openProviders() throws IoTDBConnectionException { + void openProviders() throws SubscriptionException, IoTDBConnectionException { // close stale providers closeProviders(); @@ -416,12 +412,15 @@ void openProviders() throws IoTDBConnectionException { } if (hasNoProviders()) { - throw NO_PROVIDERS_EXCEPTION; + throw new SubscriptionConnectionException( + String.format( + "Cluster has no available subscription providers to connect with initial endpoints %s", + initialEndpoints)); } } /** Caller should ensure that the method is called in the lock {@link #acquireWriteLock()}. */ - private void closeProviders() throws IoTDBConnectionException { + private void closeProviders() throws SubscriptionException, IoTDBConnectionException { for (final SubscriptionProvider provider : getAllProviders()) { provider.close(); } @@ -436,7 +435,8 @@ void addProvider(final int dataNodeId, final SubscriptionProvider provider) { } /** Caller should ensure that the method is called in the lock {@link #acquireWriteLock()}. */ - void closeAndRemoveProvider(final int dataNodeId) throws IoTDBConnectionException { + void closeAndRemoveProvider(final int dataNodeId) + throws SubscriptionException, IoTDBConnectionException { if (!containsProvider(dataNodeId)) { return; } @@ -893,57 +893,90 @@ protected void commitAsync( /** Caller should ensure that the method is called in the lock {@link #acquireReadLock()}. */ private void subscribeWithRedirection(final Set topicNames) throws SubscriptionException { - for (final SubscriptionProvider provider : getAllAvailableProviders()) { + final List providers = getAllAvailableProviders(); + if (providers.isEmpty()) { + throw new SubscriptionConnectionException( + String.format( + "Cluster has no available subscription providers when %s subscribe topic %s", + this, topicNames)); + } + for (final SubscriptionProvider provider : providers) { try { provider.subscribe(topicNames); return; } catch (final Exception e) { LOGGER.warn( - "Failed to subscribe topics {} from subscription provider {}, try next subscription provider...", + "{} failed to subscribe topics {} from subscription provider {}, try next subscription provider...", + this, topicNames, provider, e); } } - throw NO_PROVIDERS_EXCEPTION; + final String errorMessage = + String.format( + "%s failed to subscribe topics %s from all available subscription providers %s", + this, topicNames, providers); + LOGGER.warn(errorMessage); + throw new SubscriptionNonRetryableException(errorMessage); } /** Caller should ensure that the method is called in the lock {@link #acquireReadLock()}. */ private void unsubscribeWithRedirection(final Set topicNames) throws SubscriptionException { - for (final SubscriptionProvider provider : getAllAvailableProviders()) { + final List providers = getAllAvailableProviders(); + if (providers.isEmpty()) { + throw new SubscriptionConnectionException( + String.format( + "Cluster has no available subscription providers when %s unsubscribe topic %s", + this, topicNames)); + } + for (final SubscriptionProvider provider : providers) { try { provider.unsubscribe(topicNames); return; } catch (final Exception e) { LOGGER.warn( - "Failed to unsubscribe topics {} from subscription provider {}, try next subscription provider...", + "{} failed to unsubscribe topics {} from subscription provider {}, try next subscription provider...", + this, topicNames, provider, e); } } - throw NO_PROVIDERS_EXCEPTION; + final String errorMessage = + String.format( + "%s failed to unsubscribe topics %s from all available subscription providers %s", + this, topicNames, providers); + LOGGER.warn(errorMessage); + throw new SubscriptionNonRetryableException(errorMessage); } /** Caller should ensure that the method is called in the lock {@link #acquireReadLock()}. */ Map fetchAllEndPointsWithRedirection() throws SubscriptionException { - Map endPoints = null; - for (final SubscriptionProvider provider : getAllAvailableProviders()) { + final List providers = getAllAvailableProviders(); + if (providers.isEmpty()) { + throw new SubscriptionConnectionException( + String.format( + "Cluster has no available subscription providers when %s fetch all endpoints", this)); + } + for (final SubscriptionProvider provider : providers) { try { - endPoints = provider.getSessionConnection().fetchAllEndPoints(); - break; + return provider.getSessionConnection().fetchAllEndPoints(); } catch (final Exception e) { LOGGER.warn( - "Failed to fetch all endpoints from subscription provider {}, try next subscription provider...", + "{} failed to fetch all endpoints from subscription provider {}, try next subscription provider...", + this, provider, e); } } - if (Objects.isNull(endPoints)) { - throw NO_PROVIDERS_EXCEPTION; - } - return endPoints; + final String errorMessage = + String.format( + "%s failed to fetch all endpoints from all available subscription providers %s", + this, providers); + LOGGER.warn(errorMessage); + throw new SubscriptionNonRetryableException(errorMessage); } /////////////////////////////// builder /////////////////////////////// diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionEndpointsSyncer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionEndpointsSyncer.java index c0afb99010e9..fe04341045f4 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionEndpointsSyncer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionEndpointsSyncer.java @@ -20,7 +20,6 @@ package org.apache.iotdb.session.subscription; import org.apache.iotdb.common.rpc.thrift.TEndPoint; -import org.apache.iotdb.rpc.IoTDBConnectionException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -56,7 +55,7 @@ private void syncInternal() { if (consumer.hasNoProviders()) { try { consumer.openProviders(); - } catch (final IoTDBConnectionException e) { + } catch (final Exception e) { LOGGER.warn("something unexpected happened when syncing subscription endpoints...", e); return; } @@ -101,7 +100,7 @@ private void syncInternal() { if (!provider.isAvailable()) { try { consumer.closeAndRemoveProvider(entry.getKey()); - } catch (final IoTDBConnectionException e) { + } catch (final Exception e) { LOGGER.warn( "Exception occurred when closing and removing subscription provider with data node id {}", entry.getKey(), @@ -117,7 +116,7 @@ private void syncInternal() { if (!allEndPoints.containsKey(dataNodeId)) { try { consumer.closeAndRemoveProvider(dataNodeId); - } catch (final IoTDBConnectionException e) { + } catch (final Exception e) { LOGGER.warn( "Exception occurred when closing and removing subscription provider with data node id {}", dataNodeId, diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionProvider.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionProvider.java index 8cc42a2fe40c..ca8ffef3e269 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionProvider.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionProvider.java @@ -117,13 +117,14 @@ synchronized void handshake() throws SubscriptionException, IoTDBConnectionExcep return; } - super.open(); + super.open(); // throw IoTDBConnectionException final Map consumerAttributes = new HashMap<>(); consumerAttributes.put(ConsumerConstant.CONSUMER_GROUP_ID_KEY, consumerGroupId); consumerAttributes.put(ConsumerConstant.CONSUMER_ID_KEY, consumerId); - final PipeSubscribeHandshakeResp resp = handshake(new ConsumerConfig(consumerAttributes)); + final PipeSubscribeHandshakeResp resp = + handshake(new ConsumerConfig(consumerAttributes)); // throw SubscriptionException dataNodeId = resp.getDataNodeId(); consumerId = resp.getConsumerId(); consumerGroupId = resp.getConsumerGroupId(); @@ -170,9 +171,9 @@ public synchronized void close() throws SubscriptionException, IoTDBConnectionEx } try { - closeInternal(); + closeInternal(); // throw SubscriptionException } finally { - super.close(); + super.close(); // throw IoTDBConnectionException setUnavailable(); isClosed.set(true); } From 5bbcbe334453c314c3a271c0715d7d0b7e17973f Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Mon, 13 May 2024 18:40:14 +0800 Subject: [PATCH 65/93] round-robin poll endpoints --- .../subscription/SubscriptionConsumer.java | 110 ++++++++++++------ 1 file changed, 73 insertions(+), 37 deletions(-) diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java index 984805c46eed..f445c658b3b0 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java @@ -63,12 +63,14 @@ import java.util.Optional; import java.util.Properties; import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; +import java.util.SortedMap; +import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.locks.LockSupport; import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.stream.Collectors; @@ -76,6 +78,8 @@ public abstract class SubscriptionConsumer implements AutoCloseable { private static final Logger LOGGER = LoggerFactory.getLogger(SubscriptionConsumer.class); + public static final long SLEEP_NS = 1_000_000_000L; + private final List initialEndpoints; private final String username; @@ -87,9 +91,10 @@ public abstract class SubscriptionConsumer implements AutoCloseable { private final long heartbeatIntervalMs; private final long endpointsSyncIntervalMs; - private final Map subscriptionProviders = - new ConcurrentHashMap<>(); + private final SortedMap subscriptionProviders = + new ConcurrentSkipListMap<>(); private final ReentrantReadWriteLock subscriptionProvidersLock = new ReentrantReadWriteLock(true); + private int nextDataNodeId = -1; private ScheduledExecutorService heartbeatWorkerExecutor; private ScheduledExecutorService endpointsSyncerExecutor; @@ -417,6 +422,8 @@ void openProviders() throws SubscriptionException, IoTDBConnectionException { "Cluster has no available subscription providers to connect with initial endpoints %s", initialEndpoints)); } + + nextDataNodeId = subscriptionProviders.firstKey(); } /** Caller should ensure that the method is called in the lock {@link #acquireWriteLock()}. */ @@ -455,25 +462,55 @@ boolean hasNoProviders() { } /** Caller should ensure that the method is called in the lock {@link #acquireReadLock()}. */ - boolean containsProvider(final int dataNodeId) { + List getAllProviders() { + return new ArrayList<>(subscriptionProviders.values()); + } + + /** Caller should ensure that the method is called in the lock {@link #acquireReadLock()}. */ + SubscriptionProvider getProvider(final int dataNodeId) { + return containsProvider(dataNodeId) ? subscriptionProviders.get(dataNodeId) : null; + } + + /** Caller should ensure that the method is called in the lock {@link #acquireReadLock()}. */ + private boolean hasNoAvailableProviders() { + return subscriptionProviders.values().stream().noneMatch(SubscriptionProvider::isAvailable); + } + + /** Caller should ensure that the method is called in the lock {@link #acquireReadLock()}. */ + private boolean containsProvider(final int dataNodeId) { return subscriptionProviders.containsKey(dataNodeId); } /** Caller should ensure that the method is called in the lock {@link #acquireReadLock()}. */ - List getAllAvailableProviders() { + private List getAllAvailableProviders() { return subscriptionProviders.values().stream() .filter(SubscriptionProvider::isAvailable) .collect(Collectors.toList()); } /** Caller should ensure that the method is called in the lock {@link #acquireReadLock()}. */ - List getAllProviders() { - return new ArrayList<>(subscriptionProviders.values()); + private void updateNextDataNodeId() { + final SortedMap subProviders = + subscriptionProviders.tailMap(nextDataNodeId + 1); + nextDataNodeId = + subProviders.isEmpty() ? subscriptionProviders.firstKey() : subProviders.firstKey(); } /** Caller should ensure that the method is called in the lock {@link #acquireReadLock()}. */ - SubscriptionProvider getProvider(final int dataNodeId) { - return containsProvider(dataNodeId) ? subscriptionProviders.get(dataNodeId) : null; + private SubscriptionProvider getNextAvailableProvider() { + if (hasNoAvailableProviders()) { + return null; + } + + SubscriptionProvider provider; + provider = getProvider(nextDataNodeId); + while (Objects.isNull(provider) || !provider.isAvailable()) { + updateNextDataNodeId(); + provider = getProvider(nextDataNodeId); + } + updateNextDataNodeId(); + + return provider; } /////////////////////////////// poll /////////////////////////////// @@ -515,6 +552,7 @@ protected List poll(final Set topicNames, final lon } // update timer timer.update(); + LockSupport.parkNanos(SLEEP_NS); // wait some time } while (timer.notExpired()); LOGGER.info( @@ -757,41 +795,39 @@ private SubscriptionMessage pollTsFileInternal( private List pollInternal(final Set topicNames) throws SubscriptionException { - final List polledMessages = new ArrayList<>(); - acquireReadLock(); try { - for (final SubscriptionProvider provider : getAllAvailableProviders()) { - try { - polledMessages.addAll( - provider.poll( - new SubscriptionPollMessage( - SubscriptionPollMessageType.POLL.getType(), - new PollMessagePayload(topicNames), - 0L))); - } catch (final SubscriptionRetryableException e) { - LOGGER.warn( - "SubscriptionRetryableException occurred when SubscriptionConsumer {} polling from SubscriptionProvider {}", - this, - provider, - e); - // ignore - } catch (final SubscriptionNonRetryableException e) { - LOGGER.warn( - "SubscriptionNonRetryableException occurred when SubscriptionConsumer {} polling from SubscriptionProvider {}", - this, - provider, - e); - // TODO: Consider mid-process failures. - // rethrow - throw e; - } + final SubscriptionProvider provider = getNextAvailableProvider(); + if (Objects.isNull(provider)) { + return Collections.emptyList(); + } + try { + return provider.poll( + new SubscriptionPollMessage( + SubscriptionPollMessageType.POLL.getType(), + new PollMessagePayload(topicNames), + 0L)); + } catch (final SubscriptionRetryableException e) { + LOGGER.warn( + "SubscriptionRetryableException occurred when SubscriptionConsumer {} polling from SubscriptionProvider {}", + this, + provider, + e); + // ignore + } catch (final SubscriptionNonRetryableException e) { + LOGGER.warn( + "SubscriptionNonRetryableException occurred when SubscriptionConsumer {} polling from SubscriptionProvider {}", + this, + provider, + e); + // rethrow + throw e; } } finally { releaseReadLock(); } - return polledMessages; + return Collections.emptyList(); } private List pollTsFileInternal( From b830ec707d02450c6219a02a227e3dc885b1a01a Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Tue, 14 May 2024 12:47:19 +0800 Subject: [PATCH 66/93] reset byte buffer after polling msg --- .../payload/common/SubscriptionPolledMessage.java | 8 ++++---- .../payload/response/PipeSubscribePollResp.java | 1 + 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledMessage.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledMessage.java index 89effe55d25a..c4b1810b45b1 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledMessage.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledMessage.java @@ -62,10 +62,6 @@ public SubscriptionCommitContext getCommitContext() { return commitContext; } - public ByteBuffer getByteBuffer() { - return byteBuffer; - } - /////////////////////////////// de/ser /////////////////////////////// public static ByteBuffer serialize(SubscriptionPolledMessage message) throws IOException { @@ -148,6 +144,10 @@ public void serialize() throws IOException { } } + public ByteBuffer getByteBuffer() { + return byteBuffer; + } + public void resetByteBuffer() { // maybe friendly for gc byteBuffer = null; diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollResp.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollResp.java index 15154da36eb1..1d7aacbb2056 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollResp.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollResp.java @@ -59,6 +59,7 @@ public static PipeSubscribePollResp toTPipeSubscribeResp( for (final SubscriptionPolledMessage message : messages) { if (Objects.nonNull(message.getByteBuffer())) { resp.body.add(message.getByteBuffer()); + message.resetByteBuffer(); // maybe friendly for gc } else { resp.body.add(SubscriptionPolledMessage.serialize(message)); } From fc7555f8b4d1014e47d76ab4d466e9117c2cac9f Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Tue, 14 May 2024 19:47:29 +0800 Subject: [PATCH 67/93] refactor subscription providers --- .../iotdb/SubscriptionSessionExample.java | 8 +- .../IoTDBSubscriptionConsumerGroupIT.java | 10 +- .../it/dual/IoTDBSubscriptionTopicIT.java | 4 +- .../it/local/IoTDBSubscriptionBasicIT.java | 8 +- .../it/local/IoTDBSubscriptionRestartIT.java | 6 +- .../common/SubscriptionPolledMessage.java | 66 ++-- .../response/PipeSubscribePollResp.java | 12 +- .../session/subscription/ConsumeListener.java | 2 + .../subscription/ConsumerHeartbeatWorker.java | 63 ---- .../subscription/SubscriptionConsumer.java | 229 ++---------- .../SubscriptionEndpointsSyncer.java | 128 ------- .../subscription/SubscriptionProvider.java | 36 +- .../subscription/SubscriptionProviders.java | 331 ++++++++++++++++++ .../SubscriptionPullConsumer.java | 1 + .../SubscriptionPushConsumer.java | 1 + .../subscription/SubscriptionSession.java | 2 +- .../{ => payload}/SubscriptionMessage.java | 2 +- .../SubscriptionMessagePayload.java | 2 +- .../SubscriptionMessageType.java | 2 +- .../SubscriptionSessionDataSet.java | 2 +- .../SubscriptionSessionDataSets.java | 2 +- .../SubscriptionTsFileReader.java | 2 +- .../{ => util}/SubscriptionPollTimer.java | 2 +- .../SubscriptionPolledMessageBinaryCache.java | 2 +- 24 files changed, 439 insertions(+), 484 deletions(-) delete mode 100644 iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/ConsumerHeartbeatWorker.java delete mode 100644 iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionEndpointsSyncer.java create mode 100644 iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionProviders.java rename iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/{ => payload}/SubscriptionMessage.java (98%) rename iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/{ => payload}/SubscriptionMessagePayload.java (93%) rename iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/{ => payload}/SubscriptionMessageType.java (96%) rename iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/{ => payload}/SubscriptionSessionDataSet.java (99%) rename iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/{ => payload}/SubscriptionSessionDataSets.java (96%) rename iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/{ => payload}/SubscriptionTsFileReader.java (96%) rename iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/{ => util}/SubscriptionPollTimer.java (97%) diff --git a/example/session/src/main/java/org/apache/iotdb/SubscriptionSessionExample.java b/example/session/src/main/java/org/apache/iotdb/SubscriptionSessionExample.java index 80f219dfae47..df9e45212b21 100644 --- a/example/session/src/main/java/org/apache/iotdb/SubscriptionSessionExample.java +++ b/example/session/src/main/java/org/apache/iotdb/SubscriptionSessionExample.java @@ -25,12 +25,12 @@ import org.apache.iotdb.rpc.subscription.config.ConsumerConstant; import org.apache.iotdb.rpc.subscription.config.TopicConstant; import org.apache.iotdb.session.Session; -import org.apache.iotdb.session.subscription.SubscriptionMessage; import org.apache.iotdb.session.subscription.SubscriptionPullConsumer; import org.apache.iotdb.session.subscription.SubscriptionSession; -import org.apache.iotdb.session.subscription.SubscriptionSessionDataSet; -import org.apache.iotdb.session.subscription.SubscriptionSessionDataSets; -import org.apache.iotdb.session.subscription.SubscriptionTsFileReader; +import org.apache.iotdb.session.subscription.payload.SubscriptionMessage; +import org.apache.iotdb.session.subscription.payload.SubscriptionSessionDataSet; +import org.apache.iotdb.session.subscription.payload.SubscriptionSessionDataSets; +import org.apache.iotdb.session.subscription.payload.SubscriptionTsFileReader; import java.util.ArrayList; import java.util.Collections; diff --git a/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionConsumerGroupIT.java b/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionConsumerGroupIT.java index fecd7e8168df..c2e1b9e1a6b6 100644 --- a/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionConsumerGroupIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionConsumerGroupIT.java @@ -29,13 +29,13 @@ import org.apache.iotdb.itbase.category.MultiClusterIT2Subscription; import org.apache.iotdb.rpc.TSStatusCode; import org.apache.iotdb.rpc.subscription.config.TopicConstant; -import org.apache.iotdb.session.subscription.SubscriptionMessage; -import org.apache.iotdb.session.subscription.SubscriptionMessageType; +import org.apache.iotdb.session.subscription.payload.SubscriptionMessage; +import org.apache.iotdb.session.subscription.payload.SubscriptionMessageType; import org.apache.iotdb.session.subscription.SubscriptionPullConsumer; import org.apache.iotdb.session.subscription.SubscriptionSession; -import org.apache.iotdb.session.subscription.SubscriptionSessionDataSet; -import org.apache.iotdb.session.subscription.SubscriptionSessionDataSets; -import org.apache.iotdb.session.subscription.SubscriptionTsFileReader; +import org.apache.iotdb.session.subscription.payload.SubscriptionSessionDataSet; +import org.apache.iotdb.session.subscription.payload.SubscriptionSessionDataSets; +import org.apache.iotdb.session.subscription.payload.SubscriptionTsFileReader; import org.apache.iotdb.subscription.it.IoTDBSubscriptionITConstant; import org.apache.tsfile.read.TsFileReader; diff --git a/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionTopicIT.java b/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionTopicIT.java index dd7bbc00a6ac..3e530d18e9de 100644 --- a/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionTopicIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionTopicIT.java @@ -27,10 +27,10 @@ import org.apache.iotdb.it.framework.IoTDBTestRunner; import org.apache.iotdb.itbase.category.MultiClusterIT2Subscription; import org.apache.iotdb.rpc.subscription.config.TopicConstant; -import org.apache.iotdb.session.subscription.SubscriptionMessage; +import org.apache.iotdb.session.subscription.payload.SubscriptionMessage; import org.apache.iotdb.session.subscription.SubscriptionPullConsumer; import org.apache.iotdb.session.subscription.SubscriptionSession; -import org.apache.iotdb.session.subscription.SubscriptionSessionDataSets; +import org.apache.iotdb.session.subscription.payload.SubscriptionSessionDataSets; import org.apache.iotdb.subscription.it.IoTDBSubscriptionITConstant; import org.apache.tsfile.write.record.Tablet; diff --git a/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionBasicIT.java b/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionBasicIT.java index c33848391e7c..270f7accf30b 100644 --- a/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionBasicIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionBasicIT.java @@ -27,13 +27,13 @@ import org.apache.iotdb.session.subscription.AckStrategy; import org.apache.iotdb.session.subscription.AsyncCommitCallback; import org.apache.iotdb.session.subscription.ConsumeResult; -import org.apache.iotdb.session.subscription.SubscriptionMessage; +import org.apache.iotdb.session.subscription.payload.SubscriptionMessage; import org.apache.iotdb.session.subscription.SubscriptionPullConsumer; import org.apache.iotdb.session.subscription.SubscriptionPushConsumer; import org.apache.iotdb.session.subscription.SubscriptionSession; -import org.apache.iotdb.session.subscription.SubscriptionSessionDataSet; -import org.apache.iotdb.session.subscription.SubscriptionSessionDataSets; -import org.apache.iotdb.session.subscription.SubscriptionTsFileReader; +import org.apache.iotdb.session.subscription.payload.SubscriptionSessionDataSet; +import org.apache.iotdb.session.subscription.payload.SubscriptionSessionDataSets; +import org.apache.iotdb.session.subscription.payload.SubscriptionTsFileReader; import org.apache.iotdb.subscription.it.IoTDBSubscriptionITConstant; import org.apache.tsfile.read.TsFileReader; diff --git a/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionRestartIT.java b/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionRestartIT.java index 9ebb8e6b5791..c959fee88b17 100644 --- a/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionRestartIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionRestartIT.java @@ -32,11 +32,11 @@ import org.apache.iotdb.it.framework.IoTDBTestRunner; import org.apache.iotdb.itbase.category.ClusterIT; import org.apache.iotdb.rpc.RpcUtils; -import org.apache.iotdb.session.subscription.SubscriptionMessage; +import org.apache.iotdb.session.subscription.payload.SubscriptionMessage; import org.apache.iotdb.session.subscription.SubscriptionPullConsumer; import org.apache.iotdb.session.subscription.SubscriptionSession; -import org.apache.iotdb.session.subscription.SubscriptionSessionDataSet; -import org.apache.iotdb.session.subscription.SubscriptionSessionDataSets; +import org.apache.iotdb.session.subscription.payload.SubscriptionSessionDataSet; +import org.apache.iotdb.session.subscription.payload.SubscriptionSessionDataSets; import org.apache.iotdb.subscription.it.IoTDBSubscriptionITConstant; import org.awaitility.Awaitility; diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledMessage.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledMessage.java index c4b1810b45b1..f679fa68e4e9 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledMessage.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledMessage.java @@ -71,7 +71,8 @@ public static ByteBuffer serialize(SubscriptionPolledMessage message) throws IOE try (final PublicBAOS byteArrayOutputStream = new PublicBAOS(); final DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream)) { message.serialize(outputStream); - return ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size()); + return message.byteBuffer = + ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size()); } } @@ -81,6 +82,29 @@ private void serialize(final DataOutputStream stream) throws IOException { commitContext.serialize(stream); } + /** + * @return true -> byte buffer is not null + */ + public boolean trySerialize() { + try { + SubscriptionPolledMessage.serialize(this); + } catch (final IOException e) { + LOGGER.warn( + "Subscription: something unexpected happened when serializing SubscriptionPolledMessage", + e); + } + return Objects.nonNull(byteBuffer); + } + + public ByteBuffer getByteBuffer() { + return byteBuffer; + } + + public void resetByteBuffer() { + // maybe friendly for gc + byteBuffer = null; + } + public static SubscriptionPolledMessage deserialize(final ByteBuffer buffer) { final short messageType = ReadWriteIOUtils.readShort(buffer); SubscriptionMessagePayload messagePayload = null; @@ -113,46 +137,6 @@ public static SubscriptionPolledMessage deserialize(final ByteBuffer buffer) { return new SubscriptionPolledMessage(messageType, messagePayload, commitContext); } - //////////////////////////// serialization //////////////////////////// - - /** - * @return true -> byte buffer is not null - */ - public boolean trySerialize() { - if (Objects.isNull(byteBuffer)) { - try { - serialize(); - return true; - } catch (final IOException e) { - LOGGER.warn( - "Subscription: something unexpected happened when serializing SubscriptionPolledMessage", - e); - } - return false; - } - return true; - } - - public void serialize() throws IOException { - if (Objects.isNull(byteBuffer)) { - try (final PublicBAOS byteArrayOutputStream = new PublicBAOS(); - final DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream)) { - serialize(outputStream); - byteBuffer = - ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size()); - } - } - } - - public ByteBuffer getByteBuffer() { - return byteBuffer; - } - - public void resetByteBuffer() { - // maybe friendly for gc - byteBuffer = null; - } - /////////////////////////////// object /////////////////////////////// @Override diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollResp.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollResp.java index 1d7aacbb2056..3cb0ded76d51 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollResp.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollResp.java @@ -57,12 +57,12 @@ public static PipeSubscribePollResp toTPipeSubscribeResp( try { resp.body = new ArrayList<>(); for (final SubscriptionPolledMessage message : messages) { - if (Objects.nonNull(message.getByteBuffer())) { - resp.body.add(message.getByteBuffer()); - message.resetByteBuffer(); // maybe friendly for gc - } else { - resp.body.add(SubscriptionPolledMessage.serialize(message)); - } + final ByteBuffer byteBuffer = + Objects.nonNull(message.getByteBuffer()) + ? message.getByteBuffer() + : SubscriptionPolledMessage.serialize(message); + resp.body.add(byteBuffer); + message.resetByteBuffer(); // maybe friendly for gc } } catch (IOException e) { resp.status = diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/ConsumeListener.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/ConsumeListener.java index f266c60b9268..e26d7986ae73 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/ConsumeListener.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/ConsumeListener.java @@ -19,6 +19,8 @@ package org.apache.iotdb.session.subscription; +import org.apache.iotdb.session.subscription.payload.SubscriptionMessage; + @FunctionalInterface public interface ConsumeListener { ConsumeResult onReceive(SubscriptionMessage message); diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/ConsumerHeartbeatWorker.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/ConsumerHeartbeatWorker.java deleted file mode 100644 index b792b1752b25..000000000000 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/ConsumerHeartbeatWorker.java +++ /dev/null @@ -1,63 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iotdb.session.subscription; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class ConsumerHeartbeatWorker implements Runnable { - - private static final Logger LOGGER = LoggerFactory.getLogger(ConsumerHeartbeatWorker.class); - - private final SubscriptionConsumer consumer; - - public ConsumerHeartbeatWorker(SubscriptionConsumer consumer) { - this.consumer = consumer; - } - - @Override - public void run() { - if (consumer.isClosed()) { - return; - } - - consumer.acquireWriteLock(); - try { - heartbeatInternal(); - } finally { - consumer.releaseWriteLock(); - } - } - - private void heartbeatInternal() { - for (final SubscriptionProvider provider : consumer.getAllProviders()) { - try { - provider.heartbeat(); - provider.setAvailable(); - } catch (final Exception e) { - LOGGER.warn( - "something unexpected happened when sending heartbeat to subscription provider {}, set subscription provider unavailable", - provider, - e); - provider.setUnavailable(); - } - } - } -} diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java index f445c658b3b0..6fef6d1f5e6c 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java @@ -40,6 +40,8 @@ import org.apache.iotdb.rpc.subscription.payload.common.TsFileInitMessagePayload; import org.apache.iotdb.rpc.subscription.payload.common.TsFilePieceMessagePayload; import org.apache.iotdb.rpc.subscription.payload.common.TsFileSealMessagePayload; +import org.apache.iotdb.session.subscription.payload.SubscriptionMessage; +import org.apache.iotdb.session.subscription.util.SubscriptionPollTimer; import org.apache.iotdb.session.util.SessionUtils; import org.slf4j.Logger; @@ -63,16 +65,12 @@ import java.util.Optional; import java.util.Properties; import java.util.Set; -import java.util.SortedMap; -import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.locks.LockSupport; -import java.util.concurrent.locks.ReentrantReadWriteLock; -import java.util.stream.Collectors; public abstract class SubscriptionConsumer implements AutoCloseable { @@ -80,8 +78,6 @@ public abstract class SubscriptionConsumer implements AutoCloseable { public static final long SLEEP_NS = 1_000_000_000L; - private final List initialEndpoints; - private final String username; private final String password; @@ -91,10 +87,7 @@ public abstract class SubscriptionConsumer implements AutoCloseable { private final long heartbeatIntervalMs; private final long endpointsSyncIntervalMs; - private final SortedMap subscriptionProviders = - new ConcurrentSkipListMap<>(); - private final ReentrantReadWriteLock subscriptionProvidersLock = new ReentrantReadWriteLock(true); - private int nextDataNodeId = -1; + private SubscriptionProviders subscriptionProviders; private ScheduledExecutorService heartbeatWorkerExecutor; private ScheduledExecutorService endpointsSyncerExecutor; @@ -123,7 +116,7 @@ public String getConsumerGroupId() { /////////////////////////////// ctor /////////////////////////////// protected SubscriptionConsumer(final Builder builder) { - this.initialEndpoints = new ArrayList<>(); + final List initialEndpoints = new ArrayList<>(); // From org.apache.iotdb.session.Session.getNodeUrls // Priority is given to `host:port` over `nodeUrls`. if (Objects.nonNull(builder.host)) { @@ -131,6 +124,7 @@ protected SubscriptionConsumer(final Builder builder) { } else { initialEndpoints.addAll(SessionUtils.parseSeedNodeUrls(builder.nodeUrls)); } + this.subscriptionProviders = new SubscriptionProviders(initialEndpoints); this.username = builder.username; this.password = builder.password; @@ -189,11 +183,12 @@ public synchronized void open() throws SubscriptionException, IoTDBConnectionExc } // open subscription providers - acquireWriteLock(); + subscriptionProviders.acquireWriteLock(); try { - openProviders(); // throw SubscriptionException or IoTDBConnectionException + subscriptionProviders.openProviders( + this); // throw SubscriptionException or IoTDBConnectionException } finally { - releaseWriteLock(); + subscriptionProviders.releaseWriteLock(); } // launch heartbeat worker @@ -219,11 +214,12 @@ public synchronized void close() throws SubscriptionException, IoTDBConnectionEx shutdownWorkers(); // close subscription providers - acquireWriteLock(); + subscriptionProviders.acquireWriteLock(); try { - closeProviders(); // throw SubscriptionException or IoTDBConnectionException + subscriptionProviders + .closeProviders(); // throw SubscriptionException or IoTDBConnectionException } finally { - releaseWriteLock(); + subscriptionProviders.releaseWriteLock(); } } finally { isClosed.set(true); @@ -234,24 +230,6 @@ boolean isClosed() { return isClosed.get(); } - /////////////////////////////// lock /////////////////////////////// - - void acquireReadLock() { - subscriptionProvidersLock.readLock().lock(); - } - - void releaseReadLock() { - subscriptionProvidersLock.readLock().unlock(); - } - - void acquireWriteLock() { - subscriptionProvidersLock.writeLock().lock(); - } - - void releaseWriteLock() { - subscriptionProvidersLock.writeLock().unlock(); - } - /////////////////////////////// subscribe & unsubscribe /////////////////////////////// public void subscribe(final String topicName) throws SubscriptionException { @@ -263,11 +241,11 @@ public void subscribe(final String... topicNames) throws SubscriptionException { } public void subscribe(final Set topicNames) throws SubscriptionException { - acquireReadLock(); + subscriptionProviders.acquireReadLock(); try { subscribeWithRedirection(topicNames); } finally { - releaseReadLock(); + subscriptionProviders.releaseReadLock(); } } @@ -280,11 +258,11 @@ public void unsubscribe(final String... topicNames) throws SubscriptionException } public void unsubscribe(final Set topicNames) throws SubscriptionException { - acquireReadLock(); + subscriptionProviders.acquireReadLock(); try { unsubscribeWithRedirection(topicNames); } finally { - releaseReadLock(); + subscriptionProviders.releaseReadLock(); } } @@ -307,7 +285,7 @@ private void launchHeartbeatWorker() { return t; }); heartbeatWorkerExecutor.scheduleAtFixedRate( - new ConsumerHeartbeatWorker(this), 0, heartbeatIntervalMs, TimeUnit.MILLISECONDS); + () -> subscriptionProviders.heartbeat(this), 0, heartbeatIntervalMs, TimeUnit.MILLISECONDS); } /** @@ -343,7 +321,7 @@ private void launchEndpointsSyncer() { return t; }); endpointsSyncerExecutor.scheduleAtFixedRate( - new SubscriptionEndpointsSyncer(this), 0, endpointsSyncIntervalMs, TimeUnit.MILLISECONDS); + () -> subscriptionProviders.sync(this), 0, endpointsSyncIntervalMs, TimeUnit.MILLISECONDS); } private void shutdownEndpointsSyncer() { @@ -371,148 +349,6 @@ SubscriptionProvider constructProviderAndHandshake(final TEndPoint endPoint) return provider; } - /** Caller should ensure that the method is called in the lock {@link #acquireWriteLock()}. */ - void openProviders() throws SubscriptionException, IoTDBConnectionException { - // close stale providers - closeProviders(); - - for (final TEndPoint endPoint : initialEndpoints) { - final SubscriptionProvider defaultProvider; - final int defaultDataNodeId; - - try { - defaultProvider = constructProviderAndHandshake(endPoint); - } catch (final Exception e) { - LOGGER.warn("Failed to create connection with {}", endPoint, e); - continue; // try next endpoint - } - defaultDataNodeId = defaultProvider.getDataNodeId(); - addProvider(defaultDataNodeId, defaultProvider); - - final Map allEndPoints; - try { - allEndPoints = defaultProvider.getSessionConnection().fetchAllEndPoints(); - } catch (final Exception e) { - LOGGER.warn("Failed to fetch all endpoints from {}, will retry later...", endPoint, e); - break; // retry later - } - - for (final Map.Entry entry : allEndPoints.entrySet()) { - if (defaultDataNodeId == entry.getKey()) { - continue; - } - - final SubscriptionProvider provider; - try { - provider = constructProviderAndHandshake(entry.getValue()); - } catch (final Exception e) { - LOGGER.warn( - "Failed to create connection with {}, will retry later...", entry.getValue(), e); - continue; // retry later - } - addProvider(entry.getKey(), provider); - } - - break; - } - - if (hasNoProviders()) { - throw new SubscriptionConnectionException( - String.format( - "Cluster has no available subscription providers to connect with initial endpoints %s", - initialEndpoints)); - } - - nextDataNodeId = subscriptionProviders.firstKey(); - } - - /** Caller should ensure that the method is called in the lock {@link #acquireWriteLock()}. */ - private void closeProviders() throws SubscriptionException, IoTDBConnectionException { - for (final SubscriptionProvider provider : getAllProviders()) { - provider.close(); - } - subscriptionProviders.clear(); - } - - /** Caller should ensure that the method is called in the lock {@link #acquireWriteLock()}. */ - void addProvider(final int dataNodeId, final SubscriptionProvider provider) { - // the subscription provider is opened - LOGGER.info("add new subscription provider {}", provider); - subscriptionProviders.put(dataNodeId, provider); - } - - /** Caller should ensure that the method is called in the lock {@link #acquireWriteLock()}. */ - void closeAndRemoveProvider(final int dataNodeId) - throws SubscriptionException, IoTDBConnectionException { - if (!containsProvider(dataNodeId)) { - return; - } - final SubscriptionProvider provider = subscriptionProviders.get(dataNodeId); - try { - provider.close(); - } finally { - LOGGER.info("close and remove stale subscription provider {}", provider); - subscriptionProviders.remove(dataNodeId); - } - } - - /** Caller should ensure that the method is called in the lock {@link #acquireReadLock()}. */ - boolean hasNoProviders() { - return subscriptionProviders.isEmpty(); - } - - /** Caller should ensure that the method is called in the lock {@link #acquireReadLock()}. */ - List getAllProviders() { - return new ArrayList<>(subscriptionProviders.values()); - } - - /** Caller should ensure that the method is called in the lock {@link #acquireReadLock()}. */ - SubscriptionProvider getProvider(final int dataNodeId) { - return containsProvider(dataNodeId) ? subscriptionProviders.get(dataNodeId) : null; - } - - /** Caller should ensure that the method is called in the lock {@link #acquireReadLock()}. */ - private boolean hasNoAvailableProviders() { - return subscriptionProviders.values().stream().noneMatch(SubscriptionProvider::isAvailable); - } - - /** Caller should ensure that the method is called in the lock {@link #acquireReadLock()}. */ - private boolean containsProvider(final int dataNodeId) { - return subscriptionProviders.containsKey(dataNodeId); - } - - /** Caller should ensure that the method is called in the lock {@link #acquireReadLock()}. */ - private List getAllAvailableProviders() { - return subscriptionProviders.values().stream() - .filter(SubscriptionProvider::isAvailable) - .collect(Collectors.toList()); - } - - /** Caller should ensure that the method is called in the lock {@link #acquireReadLock()}. */ - private void updateNextDataNodeId() { - final SortedMap subProviders = - subscriptionProviders.tailMap(nextDataNodeId + 1); - nextDataNodeId = - subProviders.isEmpty() ? subscriptionProviders.firstKey() : subProviders.firstKey(); - } - - /** Caller should ensure that the method is called in the lock {@link #acquireReadLock()}. */ - private SubscriptionProvider getNextAvailableProvider() { - if (hasNoAvailableProviders()) { - return null; - } - - SubscriptionProvider provider; - provider = getProvider(nextDataNodeId); - while (Objects.isNull(provider) || !provider.isAvailable()) { - updateNextDataNodeId(); - provider = getProvider(nextDataNodeId); - } - updateNextDataNodeId(); - - return provider; - } - /////////////////////////////// poll /////////////////////////////// protected List poll(final Set topicNames, final long timeoutMs) @@ -795,9 +631,9 @@ private SubscriptionMessage pollTsFileInternal( private List pollInternal(final Set topicNames) throws SubscriptionException { - acquireReadLock(); + subscriptionProviders.acquireReadLock(); try { - final SubscriptionProvider provider = getNextAvailableProvider(); + final SubscriptionProvider provider = subscriptionProviders.getNextAvailableProvider(); if (Objects.isNull(provider)) { return Collections.emptyList(); } @@ -824,7 +660,7 @@ private List pollInternal(final Set topicName throw e; } } finally { - releaseReadLock(); + subscriptionProviders.releaseReadLock(); } return Collections.emptyList(); @@ -833,9 +669,9 @@ private List pollInternal(final Set topicName private List pollTsFileInternal( final int dataNodeId, final String topicName, final String fileName, final long writingOffset) throws SubscriptionException { - acquireReadLock(); + subscriptionProviders.acquireReadLock(); try { - final SubscriptionProvider provider = getProvider(dataNodeId); + final SubscriptionProvider provider = subscriptionProviders.getProvider(dataNodeId); if (Objects.isNull(provider) || !provider.isAvailable()) { throw new SubscriptionConnectionException( String.format( @@ -848,7 +684,7 @@ private List pollTsFileInternal( new PollTsFileMessagePayload(topicName, fileName, writingOffset), 0L)); } finally { - releaseReadLock(); + subscriptionProviders.releaseReadLock(); } } @@ -872,9 +708,9 @@ protected void commitSync(final Iterable messages) private void commitSyncInternal( final int dataNodeId, final List subscriptionCommitContexts) throws SubscriptionException { - acquireReadLock(); + subscriptionProviders.acquireReadLock(); try { - final SubscriptionProvider provider = getProvider(dataNodeId); + final SubscriptionProvider provider = subscriptionProviders.getProvider(dataNodeId); if (Objects.isNull(provider) || !provider.isAvailable()) { throw new SubscriptionConnectionException( String.format( @@ -883,7 +719,7 @@ private void commitSyncInternal( } provider.commitSync(subscriptionCommitContexts); } finally { - releaseReadLock(); + subscriptionProviders.releaseReadLock(); } } @@ -927,9 +763,8 @@ protected void commitAsync( /////////////////////////////// redirection /////////////////////////////// - /** Caller should ensure that the method is called in the lock {@link #acquireReadLock()}. */ private void subscribeWithRedirection(final Set topicNames) throws SubscriptionException { - final List providers = getAllAvailableProviders(); + final List providers = subscriptionProviders.getAllAvailableProviders(); if (providers.isEmpty()) { throw new SubscriptionConnectionException( String.format( @@ -957,10 +792,9 @@ private void subscribeWithRedirection(final Set topicNames) throws Subsc throw new SubscriptionNonRetryableException(errorMessage); } - /** Caller should ensure that the method is called in the lock {@link #acquireReadLock()}. */ private void unsubscribeWithRedirection(final Set topicNames) throws SubscriptionException { - final List providers = getAllAvailableProviders(); + final List providers = subscriptionProviders.getAllAvailableProviders(); if (providers.isEmpty()) { throw new SubscriptionConnectionException( String.format( @@ -988,9 +822,8 @@ private void unsubscribeWithRedirection(final Set topicNames) throw new SubscriptionNonRetryableException(errorMessage); } - /** Caller should ensure that the method is called in the lock {@link #acquireReadLock()}. */ Map fetchAllEndPointsWithRedirection() throws SubscriptionException { - final List providers = getAllAvailableProviders(); + final List providers = subscriptionProviders.getAllAvailableProviders(); if (providers.isEmpty()) { throw new SubscriptionConnectionException( String.format( diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionEndpointsSyncer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionEndpointsSyncer.java deleted file mode 100644 index fe04341045f4..000000000000 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionEndpointsSyncer.java +++ /dev/null @@ -1,128 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iotdb.session.subscription; - -import org.apache.iotdb.common.rpc.thrift.TEndPoint; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.Map; -import java.util.Objects; - -public class SubscriptionEndpointsSyncer implements Runnable { - - private static final Logger LOGGER = LoggerFactory.getLogger(SubscriptionEndpointsSyncer.class); - - private final SubscriptionConsumer consumer; - - public SubscriptionEndpointsSyncer(SubscriptionConsumer consumer) { - this.consumer = consumer; - } - - @Override - public void run() { - if (consumer.isClosed()) { - return; - } - - consumer.acquireWriteLock(); - try { - syncInternal(); - } finally { - consumer.releaseWriteLock(); - } - } - - private void syncInternal() { - if (consumer.hasNoProviders()) { - try { - consumer.openProviders(); - } catch (final Exception e) { - LOGGER.warn("something unexpected happened when syncing subscription endpoints...", e); - return; - } - } - - final Map allEndPoints; - try { - allEndPoints = consumer.fetchAllEndPointsWithRedirection(); - } catch (final Exception e) { - LOGGER.warn("Failed to fetch all endpoints, will retry later...", e); - return; // retry later - } - - // add new providers or handshake existing providers - for (final Map.Entry entry : allEndPoints.entrySet()) { - final SubscriptionProvider provider = consumer.getProvider(entry.getKey()); - if (Objects.isNull(provider)) { - // new provider - final TEndPoint endPoint = entry.getValue(); - final SubscriptionProvider newProvider; - try { - newProvider = consumer.constructProviderAndHandshake(endPoint); - } catch (final Exception e) { - LOGGER.warn( - "Failed to create connection with endpoint {}, will retry later...", endPoint, e); - continue; // retry later - } - consumer.addProvider(entry.getKey(), newProvider); - } else { - // existing provider - try { - provider.heartbeat(); - provider.setAvailable(); - } catch (final Exception e) { - LOGGER.warn( - "something unexpected happened when sending heartbeat to subscription provider {}, set subscription provider unavailable", - provider, - e); - provider.setUnavailable(); - } - // close and remove unavailable provider (reset the connection as much as possible) - if (!provider.isAvailable()) { - try { - consumer.closeAndRemoveProvider(entry.getKey()); - } catch (final Exception e) { - LOGGER.warn( - "Exception occurred when closing and removing subscription provider with data node id {}", - entry.getKey(), - e); - } - } - } - } - - // close and remove stale providers - for (final SubscriptionProvider provider : consumer.getAllProviders()) { - final int dataNodeId = provider.getDataNodeId(); - if (!allEndPoints.containsKey(dataNodeId)) { - try { - consumer.closeAndRemoveProvider(dataNodeId); - } catch (final Exception e) { - LOGGER.warn( - "Exception occurred when closing and removing subscription provider with data node id {}", - dataNodeId, - e); - } - } - } - } -} diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionProvider.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionProvider.java index ca8ffef3e269..a098b07f63ad 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionProvider.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionProvider.java @@ -24,6 +24,7 @@ import org.apache.iotdb.rpc.IoTDBConnectionException; import org.apache.iotdb.rpc.subscription.config.ConsumerConfig; import org.apache.iotdb.rpc.subscription.config.ConsumerConstant; +import org.apache.iotdb.rpc.subscription.exception.SubscriptionConnectionException; import org.apache.iotdb.rpc.subscription.exception.SubscriptionException; import org.apache.iotdb.rpc.subscription.exception.SubscriptionNonRetryableException; import org.apache.iotdb.rpc.subscription.exception.SubscriptionRetryableException; @@ -150,15 +151,14 @@ PipeSubscribeHandshakeResp handshake(final ConsumerConfig consumerConfig) try { resp = getSessionConnection().pipeSubscribe(req); } catch (final TException e) { - // TODO: Distinguish between TTransportException, TApplicationException, and - // TProtocolException. + // Assume provider unavailable LOGGER.warn( "TException occurred when SubscriptionProvider {} handshake with request {}, set SubscriptionProvider unavailable", this, consumerConfig, e); setUnavailable(); - throw new SubscriptionNonRetryableException(e.getMessage(), e); + throw new SubscriptionConnectionException(e.getMessage(), e); } verifyPipeSubscribeSuccess(resp.status); return PipeSubscribeHandshakeResp.fromTPipeSubscribeResp(resp); @@ -184,14 +184,13 @@ void closeInternal() throws SubscriptionException { try { resp = getSessionConnection().pipeSubscribe(PipeSubscribeCloseReq.toTPipeSubscribeReq()); } catch (final TException e) { - // TODO: Distinguish between TTransportException, TApplicationException, and - // TProtocolException. + // Assume provider unavailable LOGGER.warn( "TException occurred when SubscriptionProvider {} close, set SubscriptionProvider unavailable", this, e); setUnavailable(); - throw new SubscriptionNonRetryableException(e.getMessage(), e); + throw new SubscriptionConnectionException(e.getMessage(), e); } verifyPipeSubscribeSuccess(resp.status); } @@ -203,14 +202,13 @@ void heartbeat() throws SubscriptionException { try { resp = getSessionConnection().pipeSubscribe(PipeSubscribeHeartbeatReq.toTPipeSubscribeReq()); } catch (final TException e) { - // TODO: Distinguish between TTransportException, TApplicationException, and - // TProtocolException. + // Assume provider unavailable LOGGER.warn( "TException occurred when SubscriptionProvider {} heartbeat, set SubscriptionProvider unavailable", this, e); setUnavailable(); - throw new SubscriptionNonRetryableException(e.getMessage(), e); + throw new SubscriptionConnectionException(e.getMessage(), e); } verifyPipeSubscribeSuccess(resp.status); } @@ -231,15 +229,14 @@ void subscribe(final Set topicNames) throws SubscriptionException { try { resp = getSessionConnection().pipeSubscribe(req); } catch (final TException e) { - // TODO: Distinguish between TTransportException, TApplicationException, and - // TProtocolException. + // Assume provider unavailable LOGGER.warn( "TException occurred when SubscriptionProvider {} subscribe with request {}, set SubscriptionProvider unavailable", this, topicNames, e); setUnavailable(); - throw new SubscriptionNonRetryableException(e.getMessage(), e); + throw new SubscriptionConnectionException(e.getMessage(), e); } verifyPipeSubscribeSuccess(resp.status); } @@ -260,15 +257,14 @@ void unsubscribe(final Set topicNames) throws SubscriptionException { try { resp = getSessionConnection().pipeSubscribe(req); } catch (final TException e) { - // TODO: Distinguish between TTransportException, TApplicationException, and - // TProtocolException. + // Assume provider unavailable LOGGER.warn( "TException occurred when SubscriptionProvider {} unsubscribe with request {}, set SubscriptionProvider unavailable", this, topicNames, e); setUnavailable(); - throw new SubscriptionNonRetryableException(e.getMessage(), e); + throw new SubscriptionConnectionException(e.getMessage(), e); } verifyPipeSubscribeSuccess(resp.status); } @@ -290,15 +286,14 @@ List poll(final SubscriptionPollMessage pollMessage) try { resp = getSessionConnection().pipeSubscribe(req); } catch (final TException e) { - // TODO: Distinguish between TTransportException, TApplicationException, and - // TProtocolException. + // Assume provider unavailable LOGGER.warn( "TException occurred when SubscriptionProvider {} poll with request {}, set SubscriptionProvider unavailable", this, pollMessage, e); setUnavailable(); - throw new SubscriptionNonRetryableException(e.getMessage(), e); + throw new SubscriptionConnectionException(e.getMessage(), e); } verifyPipeSubscribeSuccess(resp.status); final PipeSubscribePollResp pollResp = PipeSubscribePollResp.fromTPipeSubscribeResp(resp); @@ -322,15 +317,14 @@ void commitSync(final List subscriptionCommitContexts try { resp = getSessionConnection().pipeSubscribe(req); } catch (final TException e) { - // TODO: Distinguish between TTransportException, TApplicationException, and - // TProtocolException. + // Assume provider unavailable LOGGER.warn( "TException occurred when SubscriptionProvider {} commit with request {}, set SubscriptionProvider unavailable", this, subscriptionCommitContexts, e); setUnavailable(); - throw new SubscriptionNonRetryableException(e.getMessage(), e); + throw new SubscriptionConnectionException(e.getMessage(), e); } verifyPipeSubscribeSuccess(resp.status); } diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionProviders.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionProviders.java new file mode 100644 index 000000000000..4db277212d77 --- /dev/null +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionProviders.java @@ -0,0 +1,331 @@ +/* + * 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.session.subscription; + +import org.apache.iotdb.common.rpc.thrift.TEndPoint; +import org.apache.iotdb.rpc.IoTDBConnectionException; +import org.apache.iotdb.rpc.subscription.exception.SubscriptionConnectionException; +import org.apache.iotdb.rpc.subscription.exception.SubscriptionException; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.SortedMap; +import java.util.concurrent.ConcurrentSkipListMap; +import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.stream.Collectors; + +final class SubscriptionProviders { + + private static final Logger LOGGER = LoggerFactory.getLogger(SubscriptionProviders.class); + + private final SortedMap subscriptionProviders = + new ConcurrentSkipListMap<>(); + private int nextDataNodeId = -1; + + private final ReentrantReadWriteLock subscriptionProvidersLock = new ReentrantReadWriteLock(true); + + private final List initialEndpoints; + + SubscriptionProviders(final List initialEndpoints) { + this.initialEndpoints = initialEndpoints; + } + + /////////////////////////////// lock /////////////////////////////// + + void acquireReadLock() { + subscriptionProvidersLock.readLock().lock(); + } + + void releaseReadLock() { + subscriptionProvidersLock.readLock().unlock(); + } + + void acquireWriteLock() { + subscriptionProvidersLock.writeLock().lock(); + } + + void releaseWriteLock() { + subscriptionProvidersLock.writeLock().unlock(); + } + + /** Caller should ensure that the method is called in the lock {@link #acquireWriteLock()}. */ + void openProviders(final SubscriptionConsumer consumer) + throws SubscriptionException, IoTDBConnectionException { + // close stale providers + closeProviders(); + + for (final TEndPoint endPoint : initialEndpoints) { + final SubscriptionProvider defaultProvider; + final int defaultDataNodeId; + + try { + defaultProvider = consumer.constructProviderAndHandshake(endPoint); + } catch (final Exception e) { + LOGGER.warn("Failed to create connection with {}", endPoint, e); + continue; // try next endpoint + } + defaultDataNodeId = defaultProvider.getDataNodeId(); + addProvider(defaultDataNodeId, defaultProvider); + + final Map allEndPoints; + try { + allEndPoints = defaultProvider.getSessionConnection().fetchAllEndPoints(); + } catch (final Exception e) { + LOGGER.warn("Failed to fetch all endpoints from {}, will retry later...", endPoint, e); + break; // retry later + } + + for (final Map.Entry entry : allEndPoints.entrySet()) { + if (defaultDataNodeId == entry.getKey()) { + continue; + } + + final SubscriptionProvider provider; + try { + provider = consumer.constructProviderAndHandshake(entry.getValue()); + } catch (final Exception e) { + LOGGER.warn( + "Failed to create connection with {}, will retry later...", entry.getValue(), e); + continue; // retry later + } + addProvider(entry.getKey(), provider); + } + + break; + } + + if (hasNoProviders()) { + throw new SubscriptionConnectionException( + String.format( + "Cluster has no available subscription providers to connect with initial endpoints %s", + initialEndpoints)); + } + + nextDataNodeId = subscriptionProviders.firstKey(); + } + + /** Caller should ensure that the method is called in the lock {@link #acquireWriteLock()}. */ + void closeProviders() throws SubscriptionException, IoTDBConnectionException { + for (final SubscriptionProvider provider : getAllProviders()) { + provider.close(); + } + subscriptionProviders.clear(); + } + + /** Caller should ensure that the method is called in the lock {@link #acquireWriteLock()}. */ + void addProvider(final int dataNodeId, final SubscriptionProvider provider) { + // the subscription provider is opened + LOGGER.info("add new subscription provider {}", provider); + subscriptionProviders.put(dataNodeId, provider); + } + + /** Caller should ensure that the method is called in the lock {@link #acquireWriteLock()}. */ + void closeAndRemoveProvider(final int dataNodeId) + throws SubscriptionException, IoTDBConnectionException { + if (!containsProvider(dataNodeId)) { + return; + } + final SubscriptionProvider provider = subscriptionProviders.get(dataNodeId); + try { + provider.close(); + } finally { + LOGGER.info("close and remove stale subscription provider {}", provider); + subscriptionProviders.remove(dataNodeId); + } + } + + /** Caller should ensure that the method is called in the lock {@link #acquireReadLock()}. */ + boolean hasNoProviders() { + return subscriptionProviders.isEmpty(); + } + + /** Caller should ensure that the method is called in the lock {@link #acquireReadLock()}. */ + List getAllProviders() { + return new ArrayList<>(subscriptionProviders.values()); + } + + /** Caller should ensure that the method is called in the lock {@link #acquireReadLock()}. */ + SubscriptionProvider getProvider(final int dataNodeId) { + return containsProvider(dataNodeId) ? subscriptionProviders.get(dataNodeId) : null; + } + + /** Caller should ensure that the method is called in the lock {@link #acquireReadLock()}. */ + boolean hasNoAvailableProviders() { + return subscriptionProviders.values().stream().noneMatch(SubscriptionProvider::isAvailable); + } + + /** Caller should ensure that the method is called in the lock {@link #acquireReadLock()}. */ + boolean containsProvider(final int dataNodeId) { + return subscriptionProviders.containsKey(dataNodeId); + } + + /** Caller should ensure that the method is called in the lock {@link #acquireReadLock()}. */ + List getAllAvailableProviders() { + return subscriptionProviders.values().stream() + .filter(SubscriptionProvider::isAvailable) + .collect(Collectors.toList()); + } + + /** Caller should ensure that the method is called in the lock {@link #acquireReadLock()}. */ + void updateNextDataNodeId() { + final SortedMap subProviders = + subscriptionProviders.tailMap(nextDataNodeId + 1); + nextDataNodeId = + subProviders.isEmpty() ? subscriptionProviders.firstKey() : subProviders.firstKey(); + } + + /** Caller should ensure that the method is called in the lock {@link #acquireReadLock()}. */ + SubscriptionProvider getNextAvailableProvider() { + if (hasNoAvailableProviders()) { + return null; + } + + SubscriptionProvider provider; + provider = getProvider(nextDataNodeId); + while (Objects.isNull(provider) || !provider.isAvailable()) { + updateNextDataNodeId(); + provider = getProvider(nextDataNodeId); + } + updateNextDataNodeId(); + + return provider; + } + + void heartbeat(final SubscriptionConsumer consumer) { + if (consumer.isClosed()) { + return; + } + + acquireWriteLock(); + try { + heartbeatInternal(); + } finally { + releaseWriteLock(); + } + } + + private void heartbeatInternal() { + for (final SubscriptionProvider provider : getAllProviders()) { + try { + provider.heartbeat(); + provider.setAvailable(); + } catch (final Exception e) { + LOGGER.warn( + "something unexpected happened when sending heartbeat to subscription provider {}, set subscription provider unavailable", + provider, + e); + provider.setUnavailable(); + } + } + } + + void sync(final SubscriptionConsumer consumer) { + if (consumer.isClosed()) { + return; + } + + acquireWriteLock(); + try { + syncInternal(consumer); + } finally { + releaseWriteLock(); + } + } + + private void syncInternal(final SubscriptionConsumer consumer) { + if (hasNoProviders()) { + try { + openProviders(consumer); + } catch (final Exception e) { + LOGGER.warn("something unexpected happened when syncing subscription endpoints...", e); + return; + } + } + + final Map allEndPoints; + try { + allEndPoints = consumer.fetchAllEndPointsWithRedirection(); + } catch (final Exception e) { + LOGGER.warn("Failed to fetch all endpoints, will retry later...", e); + return; // retry later + } + + // add new providers or handshake existing providers + for (final Map.Entry entry : allEndPoints.entrySet()) { + final SubscriptionProvider provider = getProvider(entry.getKey()); + if (Objects.isNull(provider)) { + // new provider + final TEndPoint endPoint = entry.getValue(); + final SubscriptionProvider newProvider; + try { + newProvider = consumer.constructProviderAndHandshake(endPoint); + } catch (final Exception e) { + LOGGER.warn( + "Failed to create connection with endpoint {}, will retry later...", endPoint, e); + continue; // retry later + } + addProvider(entry.getKey(), newProvider); + } else { + // existing provider + try { + provider.heartbeat(); + provider.setAvailable(); + } catch (final Exception e) { + LOGGER.warn( + "something unexpected happened when sending heartbeat to subscription provider {}, set subscription provider unavailable", + provider, + e); + provider.setUnavailable(); + } + // close and remove unavailable provider (reset the connection as much as possible) + if (!provider.isAvailable()) { + try { + closeAndRemoveProvider(entry.getKey()); + } catch (final Exception e) { + LOGGER.warn( + "Exception occurred when closing and removing subscription provider with data node id {}", + entry.getKey(), + e); + } + } + } + } + + // close and remove stale providers + for (final SubscriptionProvider provider : getAllProviders()) { + final int dataNodeId = provider.getDataNodeId(); + if (!allEndPoints.containsKey(dataNodeId)) { + try { + closeAndRemoveProvider(dataNodeId); + } catch (final Exception e) { + LOGGER.warn( + "Exception occurred when closing and removing subscription provider with data node id {}", + dataNodeId, + e); + } + } + } + } +} diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java index b5db556460f6..6a8e8d065961 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java @@ -22,6 +22,7 @@ import org.apache.iotdb.rpc.IoTDBConnectionException; import org.apache.iotdb.rpc.subscription.config.ConsumerConstant; import org.apache.iotdb.rpc.subscription.exception.SubscriptionException; +import org.apache.iotdb.session.subscription.payload.SubscriptionMessage; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPushConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPushConsumer.java index ffab53b7d678..0a1e6d9f524a 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPushConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPushConsumer.java @@ -22,6 +22,7 @@ import org.apache.iotdb.rpc.IoTDBConnectionException; import org.apache.iotdb.rpc.subscription.config.ConsumerConstant; import org.apache.iotdb.rpc.subscription.exception.SubscriptionException; +import org.apache.iotdb.session.subscription.payload.SubscriptionMessage; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionSession.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionSession.java index 7c5589e64b79..ea51667a1daf 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionSession.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionSession.java @@ -63,7 +63,7 @@ public SubscriptionSession( // disable redirection .enableRedirection(false) // TODO: config - .thriftMaxFrameSize(2147483647)); + .thriftMaxFrameSize(Integer.MAX_VALUE)); } @Override diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionMessage.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionMessage.java similarity index 98% rename from iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionMessage.java rename to iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionMessage.java index 128d3a41e184..95ce271176f5 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionMessage.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionMessage.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.iotdb.session.subscription; +package org.apache.iotdb.session.subscription.payload; import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionCommitContext; diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionMessagePayload.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionMessagePayload.java similarity index 93% rename from iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionMessagePayload.java rename to iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionMessagePayload.java index 0408c9da45a9..0178afa2182a 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionMessagePayload.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionMessagePayload.java @@ -17,6 +17,6 @@ * under the License. */ -package org.apache.iotdb.session.subscription; +package org.apache.iotdb.session.subscription.payload; public interface SubscriptionMessagePayload {} diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionMessageType.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionMessageType.java similarity index 96% rename from iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionMessageType.java rename to iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionMessageType.java index 2d69d1c01ff0..c418476a15dc 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionMessageType.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionMessageType.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.iotdb.session.subscription; +package org.apache.iotdb.session.subscription.payload; import java.util.Arrays; import java.util.HashMap; diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionSessionDataSet.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionSessionDataSet.java similarity index 99% rename from iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionSessionDataSet.java rename to iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionSessionDataSet.java index 803936c6c3a0..2ba8657354d8 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionSessionDataSet.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionSessionDataSet.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.iotdb.session.subscription; +package org.apache.iotdb.session.subscription.payload; import org.apache.iotdb.isession.ISessionDataSet; diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionSessionDataSets.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionSessionDataSets.java similarity index 96% rename from iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionSessionDataSets.java rename to iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionSessionDataSets.java index ea88bd8e6fe9..35bbbf372d78 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionSessionDataSets.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionSessionDataSets.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.iotdb.session.subscription; +package org.apache.iotdb.session.subscription.payload; import org.apache.tsfile.write.record.Tablet; diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionTsFileReader.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionTsFileReader.java similarity index 96% rename from iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionTsFileReader.java rename to iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionTsFileReader.java index 554b193ac13d..11c17e71c198 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionTsFileReader.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionTsFileReader.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.iotdb.session.subscription; +package org.apache.iotdb.session.subscription.payload; import org.apache.tsfile.read.TsFileReader; import org.apache.tsfile.read.TsFileSequenceReader; diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPollTimer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/util/SubscriptionPollTimer.java similarity index 97% rename from iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPollTimer.java rename to iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/util/SubscriptionPollTimer.java index 9756da1f7c6f..756b1fd678f4 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPollTimer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/util/SubscriptionPollTimer.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.iotdb.session.subscription; +package org.apache.iotdb.session.subscription.util; public class SubscriptionPollTimer { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPolledMessageBinaryCache.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPolledMessageBinaryCache.java index ecc890bc9b14..3f65d3bf822f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPolledMessageBinaryCache.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPolledMessageBinaryCache.java @@ -54,7 +54,7 @@ public SubscriptionPolledMessageBinaryCache() { public @Nullable ByteBuffer load( @NonNull final SubscriptionPolledMessage subscriptionPolledMessage) throws IOException { - subscriptionPolledMessage.serialize(); + SubscriptionPolledMessage.serialize(subscriptionPolledMessage); return subscriptionPolledMessage.getByteBuffer(); } }); From c2e52a6105dda5dd4e71299f746ccda89ef3b75c Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Tue, 14 May 2024 19:48:38 +0800 Subject: [PATCH 68/93] spotless --- .../it/dual/IoTDBSubscriptionConsumerGroupIT.java | 4 ++-- .../iotdb/subscription/it/dual/IoTDBSubscriptionTopicIT.java | 2 +- .../iotdb/subscription/it/local/IoTDBSubscriptionBasicIT.java | 2 +- .../subscription/it/local/IoTDBSubscriptionRestartIT.java | 2 +- 4 files changed, 5 insertions(+), 5 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionConsumerGroupIT.java b/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionConsumerGroupIT.java index c2e1b9e1a6b6..d99f27bf769d 100644 --- a/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionConsumerGroupIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionConsumerGroupIT.java @@ -29,10 +29,10 @@ import org.apache.iotdb.itbase.category.MultiClusterIT2Subscription; import org.apache.iotdb.rpc.TSStatusCode; import org.apache.iotdb.rpc.subscription.config.TopicConstant; -import org.apache.iotdb.session.subscription.payload.SubscriptionMessage; -import org.apache.iotdb.session.subscription.payload.SubscriptionMessageType; import org.apache.iotdb.session.subscription.SubscriptionPullConsumer; import org.apache.iotdb.session.subscription.SubscriptionSession; +import org.apache.iotdb.session.subscription.payload.SubscriptionMessage; +import org.apache.iotdb.session.subscription.payload.SubscriptionMessageType; import org.apache.iotdb.session.subscription.payload.SubscriptionSessionDataSet; import org.apache.iotdb.session.subscription.payload.SubscriptionSessionDataSets; import org.apache.iotdb.session.subscription.payload.SubscriptionTsFileReader; diff --git a/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionTopicIT.java b/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionTopicIT.java index 3e530d18e9de..265bbf597be6 100644 --- a/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionTopicIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionTopicIT.java @@ -27,9 +27,9 @@ import org.apache.iotdb.it.framework.IoTDBTestRunner; import org.apache.iotdb.itbase.category.MultiClusterIT2Subscription; import org.apache.iotdb.rpc.subscription.config.TopicConstant; -import org.apache.iotdb.session.subscription.payload.SubscriptionMessage; import org.apache.iotdb.session.subscription.SubscriptionPullConsumer; import org.apache.iotdb.session.subscription.SubscriptionSession; +import org.apache.iotdb.session.subscription.payload.SubscriptionMessage; import org.apache.iotdb.session.subscription.payload.SubscriptionSessionDataSets; import org.apache.iotdb.subscription.it.IoTDBSubscriptionITConstant; diff --git a/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionBasicIT.java b/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionBasicIT.java index 270f7accf30b..d0c1f5329fec 100644 --- a/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionBasicIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionBasicIT.java @@ -27,10 +27,10 @@ import org.apache.iotdb.session.subscription.AckStrategy; import org.apache.iotdb.session.subscription.AsyncCommitCallback; import org.apache.iotdb.session.subscription.ConsumeResult; -import org.apache.iotdb.session.subscription.payload.SubscriptionMessage; import org.apache.iotdb.session.subscription.SubscriptionPullConsumer; import org.apache.iotdb.session.subscription.SubscriptionPushConsumer; import org.apache.iotdb.session.subscription.SubscriptionSession; +import org.apache.iotdb.session.subscription.payload.SubscriptionMessage; import org.apache.iotdb.session.subscription.payload.SubscriptionSessionDataSet; import org.apache.iotdb.session.subscription.payload.SubscriptionSessionDataSets; import org.apache.iotdb.session.subscription.payload.SubscriptionTsFileReader; diff --git a/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionRestartIT.java b/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionRestartIT.java index c959fee88b17..1e89a8a2e6c0 100644 --- a/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionRestartIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionRestartIT.java @@ -32,9 +32,9 @@ import org.apache.iotdb.it.framework.IoTDBTestRunner; import org.apache.iotdb.itbase.category.ClusterIT; import org.apache.iotdb.rpc.RpcUtils; -import org.apache.iotdb.session.subscription.payload.SubscriptionMessage; import org.apache.iotdb.session.subscription.SubscriptionPullConsumer; import org.apache.iotdb.session.subscription.SubscriptionSession; +import org.apache.iotdb.session.subscription.payload.SubscriptionMessage; import org.apache.iotdb.session.subscription.payload.SubscriptionSessionDataSet; import org.apache.iotdb.session.subscription.payload.SubscriptionSessionDataSets; import org.apache.iotdb.subscription.it.IoTDBSubscriptionITConstant; From 0ef11fa167835211043aa8c1ef7c05a3852f2b1a Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Tue, 14 May 2024 20:16:27 +0800 Subject: [PATCH 69/93] minor improve --- .../iotdb/session/subscription/SubscriptionConsumer.java | 4 ++-- .../iotdb/session/subscription/SubscriptionProviders.java | 6 ++++++ 2 files changed, 8 insertions(+), 2 deletions(-) diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java index 6fef6d1f5e6c..e862e740f2b5 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java @@ -76,7 +76,7 @@ public abstract class SubscriptionConsumer implements AutoCloseable { private static final Logger LOGGER = LoggerFactory.getLogger(SubscriptionConsumer.class); - public static final long SLEEP_NS = 1_000_000_000L; + private static final long SLEEP_NS = 1_000_000_000L; private final String username; private final String password; @@ -302,7 +302,7 @@ private void shutdownWorkers() { } } - /////////////////////////////// endpoints syncer /////////////////////////////// + /////////////////////////////// sync endpoints /////////////////////////////// @SuppressWarnings("unsafeThreadSchedule") private void launchEndpointsSyncer() { diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionProviders.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionProviders.java index 4db277212d77..3e39211f7d5b 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionProviders.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionProviders.java @@ -70,6 +70,8 @@ void releaseWriteLock() { subscriptionProvidersLock.writeLock().unlock(); } + /////////////////////////////// CRUD /////////////////////////////// + /** Caller should ensure that the method is called in the lock {@link #acquireWriteLock()}. */ void openProviders(final SubscriptionConsumer consumer) throws SubscriptionException, IoTDBConnectionException { @@ -213,6 +215,8 @@ SubscriptionProvider getNextAvailableProvider() { return provider; } + /////////////////////////////// heartbeat /////////////////////////////// + void heartbeat(final SubscriptionConsumer consumer) { if (consumer.isClosed()) { return; @@ -241,6 +245,8 @@ private void heartbeatInternal() { } } + /////////////////////////////// sync endpoints /////////////////////////////// + void sync(final SubscriptionConsumer consumer) { if (consumer.isClosed()) { return; From 7eac7baf3a41badeabb403b61c0c68995eec5482 Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Wed, 15 May 2024 22:01:33 +0800 Subject: [PATCH 70/93] improve push consumer --- .../subscription/config/ConsumerConstant.java | 2 +- .../SubscriptionPushConsumer.java | 28 +++++++++++++------ 2 files changed, 20 insertions(+), 10 deletions(-) diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/config/ConsumerConstant.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/config/ConsumerConstant.java index 315129f42c70..eb2107ce6fa6 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/config/ConsumerConstant.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/config/ConsumerConstant.java @@ -43,7 +43,7 @@ public class ConsumerConstant { public static final long ENDPOINTS_SYNC_INTERVAL_MS_MIN_VALUE = 5000; public static final String TS_FILE_BASE_DIR_KEY = "ts-file-base-dir"; - public static final String TS_FILE_BASE_DIR_DEFAULT_VALUE = System.getProperty("java.io.tmpdir"); + public static final String TS_FILE_BASE_DIR_DEFAULT_VALUE = System.getProperty("user.dir"); /////////////////////////////// pull consumer /////////////////////////////// diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPushConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPushConsumer.java index 0a1e6d9f524a..dd7d0418d97a 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPushConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPushConsumer.java @@ -27,6 +27,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.Properties; @@ -222,26 +223,35 @@ public void run() { try { // Poll all subscribed topics by passing an empty set - final List pollResults = + final List messages = poll(Collections.emptySet(), ConsumerConstant.PUSH_CONSUMER_AUTO_POLL_TIME_OUT_MS); if (ackStrategy.equals(AckStrategy.BEFORE_CONSUME)) { - commitSync(pollResults); + commitSync(messages); } - for (final SubscriptionMessage pollResult : pollResults) { - final ConsumeResult consumeResult = consumeListener.onReceive(pollResult); - if (consumeResult.equals(ConsumeResult.FAILURE)) { - LOGGER.warn("consumeListener failed when processing message: {}", pollResult); + final List ackedMessages = new ArrayList<>(); + for (final SubscriptionMessage message : messages) { + final ConsumeResult consumeResult; + try { + consumeResult = consumeListener.onReceive(message); + if (consumeResult.equals(ConsumeResult.SUCCESS)) { + ackedMessages.add(message); + } else { + LOGGER.warn("Consumer listener result failure when consuming message: {}", message); + } + } catch (final Throwable t) { + LOGGER.warn("Consumer listener raised an exception while consuming messages", t); } } if (ackStrategy.equals(AckStrategy.AFTER_CONSUME)) { - commitSync(pollResults); + // TODO: NACK + commitSync(ackedMessages); } - } catch (final SubscriptionException e) { - LOGGER.warn("Exception occurred when auto polling: ", e); + } catch (final Exception e) { + LOGGER.warn("something unexpected happened when auto poll messages...", e); } } } From 239f115bbcf0f4fde6d081b0a55a0f3ae9c27643 Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Thu, 16 May 2024 00:22:17 +0800 Subject: [PATCH 71/93] minor improve --- .../it/local/IoTDBSubscriptionBasicIT.java | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionBasicIT.java b/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionBasicIT.java index d0c1f5329fec..a0129d824c7c 100644 --- a/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionBasicIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionBasicIT.java @@ -19,6 +19,7 @@ package org.apache.iotdb.subscription.it.local; +import java.util.stream.Collectors; import org.apache.iotdb.isession.ISession; import org.apache.iotdb.it.env.EnvFactory; import org.apache.iotdb.it.framework.IoTDBTestRunner; @@ -300,11 +301,7 @@ public void testBasicPullConsumerWithCommitAsync() throws Exception { consumer.open(); consumer.subscribe("topic1"); while (!isClosed.get()) { - try { - Thread.sleep(1000); // wait some time - } catch (final InterruptedException e) { - break; - } + LockSupport.parkNanos(IoTDBSubscriptionITConstant.SLEEP_NS); // wait some time final List messages = consumer.poll(Duration.ofMillis(10000)); if (messages.isEmpty()) { @@ -321,7 +318,7 @@ public void testBasicPullConsumerWithCommitAsync() throws Exception { rowCountInOneMessage++; } } - LOGGER.info(rowCountInOneMessage + " rows in message"); + LOGGER.info("{} rows in message", rowCountInOneMessage); } consumer.commitAsync( messages, @@ -329,12 +326,15 @@ public void testBasicPullConsumerWithCommitAsync() throws Exception { @Override public void onComplete() { commitSuccessCount.incrementAndGet(); - LOGGER.info("commit success, messages size: {}", messages.size()); + LOGGER.info("async commit success, commit contexts: {}", + messages.stream().map(SubscriptionMessage::getCommitContext).collect(Collectors.toList())); } @Override - public void onFailure(Throwable e) { + public void onFailure(final Throwable e) { commitFailureCount.incrementAndGet(); + LOGGER.info("async commit failed, commit contexts: {}", + messages.stream().map(SubscriptionMessage::getCommitContext).collect(Collectors.toList()), e); } }); } From e57c636ed478c7e9cab22381bed764f1ce798586 Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Thu, 16 May 2024 01:04:43 +0800 Subject: [PATCH 72/93] intro nack --- .../request/PipeSubscribeCommitReq.java | 14 +++++++-- .../subscription/SubscriptionConsumer.java | 31 ++++++++++++++----- .../subscription/SubscriptionProvider.java | 4 +-- .../SubscriptionPullConsumer.java | 8 ++--- .../SubscriptionPushConsumer.java | 12 ++++--- .../agent/SubscriptionBrokerAgent.java | 6 ++-- .../broker/SubscriptionBroker.java | 12 +++++-- .../broker/SubscriptionPrefetchingQueue.java | 20 ++++++++++-- .../subscription/event/SubscriptionEvent.java | 4 +++ .../receiver/SubscriptionReceiverV1.java | 15 +++++---- 10 files changed, 92 insertions(+), 34 deletions(-) diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribeCommitReq.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribeCommitReq.java index 13a46c7bf8dd..d7fa43e769fe 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribeCommitReq.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribeCommitReq.java @@ -36,10 +36,16 @@ public class PipeSubscribeCommitReq extends TPipeSubscribeReq { private transient List commitContexts = new ArrayList<>(); + private transient boolean nack; + public List getCommitContexts() { return commitContexts; } + public boolean isNack() { + return nack; + } + /////////////////////////////// Thrift /////////////////////////////// /** @@ -47,10 +53,11 @@ public List getCommitContexts() { * client. */ public static PipeSubscribeCommitReq toTPipeSubscribeReq( - List commitContexts) throws IOException { + List commitContexts, boolean nack) throws IOException { final PipeSubscribeCommitReq req = new PipeSubscribeCommitReq(); req.commitContexts = commitContexts; + req.nack = nack; req.version = PipeSubscribeRequestVersion.VERSION_1.getVersion(); req.type = PipeSubscribeRequestType.COMMIT.getType(); @@ -60,6 +67,7 @@ public static PipeSubscribeCommitReq toTPipeSubscribeReq( for (final SubscriptionCommitContext commitContext : commitContexts) { commitContext.serialize(outputStream); } + ReadWriteIOUtils.write(nack, outputStream); req.body = ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size()); } @@ -75,6 +83,7 @@ public static PipeSubscribeCommitReq fromTPipeSubscribeReq(TPipeSubscribeReq com for (int i = 0; i < size; ++i) { req.commitContexts.add(SubscriptionCommitContext.deserialize(commitReq.body)); } + req.nack = ReadWriteIOUtils.readBool(commitReq.body); } req.version = commitReq.version; @@ -96,6 +105,7 @@ public boolean equals(Object obj) { } PipeSubscribeCommitReq that = (PipeSubscribeCommitReq) obj; return Objects.equals(this.commitContexts, that.commitContexts) + && Objects.equals(this.nack, that.nack) && this.version == that.version && this.type == that.type && Objects.equals(this.body, that.body); @@ -103,6 +113,6 @@ public boolean equals(Object obj) { @Override public int hashCode() { - return Objects.hash(commitContexts, version, type, body); + return Objects.hash(commitContexts, nack, version, type, body); } } diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java index e862e740f2b5..91d590affb80 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java @@ -688,10 +688,23 @@ private List pollTsFileInternal( } } - /////////////////////////////// commit sync /////////////////////////////// + /////////////////////////////// commit sync (ack & nack) /////////////////////////////// - protected void commitSync(final Iterable messages) - throws SubscriptionException { + protected void ack(final Iterable messages) throws SubscriptionException { + final Map> dataNodeIdToSubscriptionCommitContexts = + new HashMap<>(); + for (final SubscriptionMessage message : messages) { + dataNodeIdToSubscriptionCommitContexts + .computeIfAbsent(message.getCommitContext().getDataNodeId(), (id) -> new ArrayList<>()) + .add(message.getCommitContext()); + } + for (final Map.Entry> entry : + dataNodeIdToSubscriptionCommitContexts.entrySet()) { + commitInternal(entry.getKey(), entry.getValue(), false); + } + } + + protected void nack(final Iterable messages) { final Map> dataNodeIdToSubscriptionCommitContexts = new HashMap<>(); for (final SubscriptionMessage message : messages) { @@ -701,12 +714,14 @@ protected void commitSync(final Iterable messages) } for (final Map.Entry> entry : dataNodeIdToSubscriptionCommitContexts.entrySet()) { - commitSyncInternal(entry.getKey(), entry.getValue()); + commitInternal(entry.getKey(), entry.getValue(), true); } } - private void commitSyncInternal( - final int dataNodeId, final List subscriptionCommitContexts) + private void commitInternal( + final int dataNodeId, + final List subscriptionCommitContexts, + final boolean nack) throws SubscriptionException { subscriptionProviders.acquireReadLock(); try { @@ -717,7 +732,7 @@ private void commitSyncInternal( "something unexpected happened when commit messages to subscription provider with data node id %s, the subscription provider may be unavailable or not existed", dataNodeId)); } - provider.commitSync(subscriptionCommitContexts); + provider.commit(subscriptionCommitContexts, nack); } finally { subscriptionProviders.releaseReadLock(); } @@ -944,7 +959,7 @@ public void run() { } try { - commitSync(messages); + ack(messages); callback.onComplete(); } catch (final Exception e) { callback.onFailure(e); diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionProvider.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionProvider.java index a098b07f63ad..727bbdfbe246 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionProvider.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionProvider.java @@ -300,11 +300,11 @@ List poll(final SubscriptionPollMessage pollMessage) return pollResp.getMessages(); } - void commitSync(final List subscriptionCommitContexts) + void commit(final List subscriptionCommitContexts, final boolean nack) throws SubscriptionException { final PipeSubscribeCommitReq req; try { - req = PipeSubscribeCommitReq.toTPipeSubscribeReq(subscriptionCommitContexts); + req = PipeSubscribeCommitReq.toTPipeSubscribeReq(subscriptionCommitContexts, nack); } catch (final IOException e) { LOGGER.warn( "IOException occurred when SubscriptionProvider {} serialize commit request {}", diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java index 6a8e8d065961..bdf2a0ba40a3 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java @@ -162,12 +162,12 @@ public List poll(final Set topicNames, final long t /////////////////////////////// commit /////////////////////////////// public void commitSync(final SubscriptionMessage message) throws SubscriptionException { - super.commitSync(Collections.singletonList(message)); + super.ack(Collections.singletonList(message)); } public void commitSync(final Iterable messages) throws SubscriptionException { - super.commitSync(messages); + super.ack(messages); } public void commitAsync(final SubscriptionMessage message) { @@ -221,7 +221,7 @@ private void shutdownAutoCommitWorker() { private void commitAllUncommittedMessages() { for (final Map.Entry> entry : uncommittedMessages.entrySet()) { try { - commitSync(entry.getValue()); + ack(entry.getValue()); uncommittedMessages.remove(entry.getKey()); } catch (final Exception e) { LOGGER.warn("something unexpected happened when commit messages during close", e); @@ -327,7 +327,7 @@ public void run() { for (final Map.Entry> entry : uncommittedMessages.headMap(index).entrySet()) { try { - commitSync(entry.getValue()); + ack(entry.getValue()); uncommittedMessages.remove(entry.getKey()); } catch (final Exception e) { LOGGER.warn("something unexpected happened when auto commit messages...", e); diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPushConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPushConsumer.java index dd7d0418d97a..311c53db48d8 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPushConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPushConsumer.java @@ -227,17 +227,19 @@ public void run() { poll(Collections.emptySet(), ConsumerConstant.PUSH_CONSUMER_AUTO_POLL_TIME_OUT_MS); if (ackStrategy.equals(AckStrategy.BEFORE_CONSUME)) { - commitSync(messages); + ack(messages); } - final List ackedMessages = new ArrayList<>(); + final List messagesToAck = new ArrayList<>(); + final List messagesToNack = new ArrayList<>(); for (final SubscriptionMessage message : messages) { final ConsumeResult consumeResult; try { consumeResult = consumeListener.onReceive(message); if (consumeResult.equals(ConsumeResult.SUCCESS)) { - ackedMessages.add(message); + messagesToAck.add(message); } else { + messagesToNack.add(message); LOGGER.warn("Consumer listener result failure when consuming message: {}", message); } } catch (final Throwable t) { @@ -246,8 +248,8 @@ public void run() { } if (ackStrategy.equals(AckStrategy.AFTER_CONSUME)) { - // TODO: NACK - commitSync(ackedMessages); + ack(messagesToAck); + nack(messagesToNack); } } catch (final Exception e) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionBrokerAgent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionBrokerAgent.java index 344b9ba0bb06..0b95f361ae3d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionBrokerAgent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionBrokerAgent.java @@ -82,7 +82,9 @@ public List pollTsFile( * @return list of successful commit contexts */ public List commit( - final ConsumerConfig consumerConfig, final List commitContexts) { + final ConsumerConfig consumerConfig, + final List commitContexts, + final boolean nack) { final String consumerGroupId = consumerConfig.getConsumerGroupId(); final SubscriptionBroker broker = consumerGroupIdToSubscriptionBroker.get(consumerGroupId); if (Objects.isNull(broker)) { @@ -92,7 +94,7 @@ public List commit( LOGGER.warn(errorMessage); throw new SubscriptionException(errorMessage); } - return broker.commit(commitContexts); + return broker.commit(commitContexts, nack); } /////////////////////////////// broker /////////////////////////////// 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 c7184bb7f823..61978921d5b7 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 @@ -116,7 +116,7 @@ public List pollTsFile( * @return list of successful commit contexts */ public List commit( - final List commitContexts) { + final List commitContexts, final boolean nack) { final List successfulCommitContexts = new ArrayList<>(); for (final SubscriptionCommitContext commitContext : commitContexts) { final String topicName = commitContext.getTopicName(); @@ -127,8 +127,14 @@ public List commit( "Subscription: prefetching queue bound to topic [{}] does not exist", topicName); continue; } - if (prefetchingQueue.commit(commitContext)) { - successfulCommitContexts.add(commitContext); + if (!nack) { + if (prefetchingQueue.ack(commitContext)) { + successfulCommitContexts.add(commitContext); + } + } else { + if (prefetchingQueue.nack(commitContext)) { + successfulCommitContexts.add(commitContext); + } } } return successfulCommitContexts; 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 40b8d540a6fb..2e7ca8481901 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 @@ -63,9 +63,9 @@ public SubscriptionPrefetchingQueue( /////////////////////////////// commit /////////////////////////////// /** - * @return {@code true} if commit successfully + * @return {@code true} if ack successfully */ - public boolean commit(final SubscriptionCommitContext commitContext) { + public boolean ack(final SubscriptionCommitContext commitContext) { final SubscriptionEvent event = uncommittedEvents.get(commitContext); if (Objects.isNull(event)) { LOGGER.warn( @@ -80,6 +80,22 @@ public boolean commit(final SubscriptionCommitContext commitContext) { return true; } + /** + * @return {@code true} if nack successfully + */ + public boolean nack(final SubscriptionCommitContext commitContext) { + final SubscriptionEvent event = uncommittedEvents.get(commitContext); + if (Objects.isNull(event)) { + LOGGER.warn( + "Subscription: subscription commit context [{}] does not exist, it may have been committed or something unexpected happened, prefetching queue: {}", + commitContext, + this); + return false; + } + event.nack(); + return true; + } + protected SubscriptionCommitContext generateSubscriptionCommitContext() { // Recording data node ID and reboot times to address potential stale commit IDs caused by // leader transfers or restarts. diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionEvent.java index 53025f8c2286..d0584f831e37 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionEvent.java @@ -86,6 +86,10 @@ public boolean pollable() { > SubscriptionConfig.getInstance().getSubscriptionRecycleUncommittedEventIntervalMs(); } + public void nack() { + lastPolledTimestamp = INVALID_TIMESTAMP; + } + public void recordLastPolledConsumerId(final String consumerId) { lastPolledConsumerId = consumerId; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java index b6ead8b9b4da..0abe60c8658e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java @@ -426,22 +426,25 @@ private TPipeSubscribeResp handlePipeSubscribeCommitInternal(final PipeSubscribe return SUBSCRIPTION_MISSING_CUSTOMER_RESP; } - // commit + // commit (ack or nack) final List commitContexts = req.getCommitContexts(); + final boolean nack = req.isNack(); final List successfulCommitContexts = - SubscriptionAgent.broker().commit(consumerConfig, commitContexts); + SubscriptionAgent.broker().commit(consumerConfig, commitContexts, nack); if (Objects.equals(successfulCommitContexts.size(), commitContexts.size())) { LOGGER.info( - "Subscription: consumer {} commit successfully, commit contexts: {}", + "Subscription: consumer {} commit successfully, commit contexts: {}, nack: {}", consumerConfig, - commitContexts); + commitContexts, + nack); } else { LOGGER.warn( - "Subscription: consumer {} commit partially successful, commit contexts: {}, successful commit contexts: {}", + "Subscription: consumer {} commit partially successful, commit contexts: {}, successful commit contexts: {}, nack: {}", consumerConfig, commitContexts, - successfulCommitContexts); + successfulCommitContexts, + nack); } return PipeSubscribeCommitResp.toTPipeSubscribeResp(RpcUtils.SUCCESS_STATUS); From 087c5d5ff6a04f0a384fdcfdf2031ba603f9510c Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Thu, 16 May 2024 01:18:03 +0800 Subject: [PATCH 73/93] add more tests for sub tsfile --- .../IoTDBSubscriptionConsumerGroupIT.java | 88 +++++++++++++++++++ .../it/local/IoTDBSubscriptionBasicIT.java | 17 ++-- .../SubscriptionPushConsumer.java | 1 - 3 files changed, 100 insertions(+), 6 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionConsumerGroupIT.java b/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionConsumerGroupIT.java index d99f27bf769d..3b3b510e7729 100644 --- a/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionConsumerGroupIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionConsumerGroupIT.java @@ -95,6 +95,7 @@ public class IoTDBSubscriptionConsumerGroupIT extends AbstractSubscriptionDualIT private static Pair, Map> __4C_2CG_SUBSCRIBE_TWO_TOPIC; private static Pair, Map> __4C_2CG_SUBSCRIBE_TWO_TOPIC_WITH_TS_FILE; + private static Pair, Map> __6C_2CG_SUBSCRIBE_TS_FILE_TOPIC; static final class SubscriptionInfo { final String consumerId; @@ -239,6 +240,25 @@ public void setUp() { __4C_2CG_SUBSCRIBE_TWO_TOPIC_WITH_TS_FILE = new Pair<>(subscriptionInfoList, expectedHeaderWithResult); } + { + final List subscriptionInfoList = new ArrayList<>(); + subscriptionInfoList.add(new SubscriptionInfo("c1", "cg1", Collections.singleton("all"))); + subscriptionInfoList.add(new SubscriptionInfo("c2", "cg1", Collections.singleton("all"))); + subscriptionInfoList.add(new SubscriptionInfo("c3", "cg1", Collections.singleton("all"))); + subscriptionInfoList.add(new SubscriptionInfo("c4", "cg1", Collections.singleton("all"))); + subscriptionInfoList.add(new SubscriptionInfo("c5", "cg2", Collections.singleton("all"))); + subscriptionInfoList.add(new SubscriptionInfo("c6", "cg2", Collections.singleton("all"))); + + final Map expectedHeaderWithResult = new HashMap<>(); + expectedHeaderWithResult.put("count(root.cg1.topic1.s)", "100"); + expectedHeaderWithResult.put("count(root.cg1.topic2.s)", "100"); + expectedHeaderWithResult.put("count(root.cg2.topic1.s)", "100"); + expectedHeaderWithResult.put("count(root.cg2.topic2.s)", "100"); + expectedHeaderWithResult.put("count(root.topic1.s)", "100"); + expectedHeaderWithResult.put("count(root.topic2.s)", "100"); + + __6C_2CG_SUBSCRIBE_TS_FILE_TOPIC = new Pair<>(subscriptionInfoList, expectedHeaderWithResult); + } } private void testSubscriptionHistoricalDataTemplate( @@ -724,6 +744,74 @@ public void test4C2CGSubscribeTwoTopicWithTsFileRealtimeDataWithAirGapConnector( __4C_2CG_SUBSCRIBE_TWO_TOPIC_WITH_TS_FILE.right); } + // ------------------------------------------------ // + // 6 consumers, 2 consumer groups, 1 topic (tsfile) // + // ------------------------------------------------ // + + @Test + public void test6C2CGSubscribeOneTsFileTopicHistoricalDataWithAsyncConnector() throws Exception { + testSubscriptionHistoricalDataTemplate( + ASYNC_CONNECTOR_ATTRIBUTES, + __6C_2CG_SUBSCRIBE_TS_FILE_TOPIC.left, + __6C_2CG_SUBSCRIBE_TS_FILE_TOPIC.right); + } + + @Test + public void test6C2CGSubscribeOneTsFileTopicHistoricalDataWithSyncConnector() throws Exception { + testSubscriptionHistoricalDataTemplate( + SYNC_CONNECTOR_ATTRIBUTES, + __6C_2CG_SUBSCRIBE_TS_FILE_TOPIC.left, + __6C_2CG_SUBSCRIBE_TS_FILE_TOPIC.right); + } + + @Test + public void test6C2CGSubscribeOneTsFileTopicHistoricalDataWithLegacyConnector() throws Exception { + testSubscriptionHistoricalDataTemplate( + LEGACY_CONNECTOR_ATTRIBUTES, + __6C_2CG_SUBSCRIBE_TS_FILE_TOPIC.left, + __6C_2CG_SUBSCRIBE_TS_FILE_TOPIC.right); + } + + @Test + public void test6C2CGSubscribeOneTsFileTopicHistoricalDataWithAirGapConnector() throws Exception { + testSubscriptionHistoricalDataTemplate( + AIR_GAP_CONNECTOR_ATTRIBUTES, + __6C_2CG_SUBSCRIBE_TS_FILE_TOPIC.left, + __6C_2CG_SUBSCRIBE_TS_FILE_TOPIC.right); + } + + @Test + public void test6C2CGSubscribeOneTsFileTopicRealtimeDataWithAsyncConnector() throws Exception { + testSubscriptionRealtimeDataTemplate( + ASYNC_CONNECTOR_ATTRIBUTES, + __6C_2CG_SUBSCRIBE_TS_FILE_TOPIC.left, + __6C_2CG_SUBSCRIBE_TS_FILE_TOPIC.right); + } + + @Test + public void test6C2CGSubscribeOneTsFileTopicRealtimeDataWithSyncConnector() throws Exception { + testSubscriptionRealtimeDataTemplate( + SYNC_CONNECTOR_ATTRIBUTES, + __6C_2CG_SUBSCRIBE_TS_FILE_TOPIC.left, + __6C_2CG_SUBSCRIBE_TS_FILE_TOPIC.right); + } + + @Test + public void test6C2CGSubscribeOneTsFileTopicRealtimeDataWithLegacyConnector() throws Exception { + testSubscriptionRealtimeDataTemplate( + LEGACY_CONNECTOR_ATTRIBUTES, + __6C_2CG_SUBSCRIBE_TS_FILE_TOPIC.left, + __6C_2CG_SUBSCRIBE_TS_FILE_TOPIC.right); + } + + @Test + public void test6C2CGSubscribeOneTsFileTopicRealtimeDataWithAirGapConnector() throws Exception { + testSubscriptionRealtimeDataTemplate( + AIR_GAP_CONNECTOR_ATTRIBUTES, + __6C_2CG_SUBSCRIBE_TS_FILE_TOPIC.left, + __6C_2CG_SUBSCRIBE_TS_FILE_TOPIC.right); + } + /////////////////////////////// utility /////////////////////////////// private void createTopics(final long currentTime) { diff --git a/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionBasicIT.java b/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionBasicIT.java index a0129d824c7c..ac697d8d5773 100644 --- a/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionBasicIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionBasicIT.java @@ -19,7 +19,6 @@ package org.apache.iotdb.subscription.it.local; -import java.util.stream.Collectors; import org.apache.iotdb.isession.ISession; import org.apache.iotdb.it.env.EnvFactory; import org.apache.iotdb.it.framework.IoTDBTestRunner; @@ -59,6 +58,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.locks.LockSupport; +import java.util.stream.Collectors; import static org.junit.Assert.fail; @@ -326,15 +326,22 @@ public void testBasicPullConsumerWithCommitAsync() throws Exception { @Override public void onComplete() { commitSuccessCount.incrementAndGet(); - LOGGER.info("async commit success, commit contexts: {}", - messages.stream().map(SubscriptionMessage::getCommitContext).collect(Collectors.toList())); + LOGGER.info( + "async commit success, commit contexts: {}", + messages.stream() + .map(SubscriptionMessage::getCommitContext) + .collect(Collectors.toList())); } @Override public void onFailure(final Throwable e) { commitFailureCount.incrementAndGet(); - LOGGER.info("async commit failed, commit contexts: {}", - messages.stream().map(SubscriptionMessage::getCommitContext).collect(Collectors.toList()), e); + LOGGER.info( + "async commit failed, commit contexts: {}", + messages.stream() + .map(SubscriptionMessage::getCommitContext) + .collect(Collectors.toList()), + e); } }); } diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPushConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPushConsumer.java index 311c53db48d8..e11af2a0945a 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPushConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPushConsumer.java @@ -251,7 +251,6 @@ public void run() { ack(messagesToAck); nack(messagesToNack); } - } catch (final Exception e) { LOGGER.warn("something unexpected happened when auto poll messages...", e); } From bb0bde789c4ed39c6b8067bf9c6679ca0bd052df Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Thu, 16 May 2024 16:48:54 +0800 Subject: [PATCH 74/93] minor improve --- .../session/subscription/SubscriptionPushConsumer.java | 6 ++++-- .../config/executor/ClusterConfigTaskExecutor.java | 2 +- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPushConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPushConsumer.java index e11af2a0945a..5003f7c46c00 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPushConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPushConsumer.java @@ -239,11 +239,13 @@ public void run() { if (consumeResult.equals(ConsumeResult.SUCCESS)) { messagesToAck.add(message); } else { - messagesToNack.add(message); LOGGER.warn("Consumer listener result failure when consuming message: {}", message); + messagesToNack.add(message); } } catch (final Throwable t) { - LOGGER.warn("Consumer listener raised an exception while consuming messages", t); + LOGGER.warn( + "Consumer listener raised an exception while consuming message: {}", message, t); + messagesToNack.add(message); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java index e1267cbb7ae9..c8546f8d2634 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java @@ -1954,7 +1954,7 @@ public SettableFuture createTopic(CreateTopicStatement createT "Failed to create topic %s, topic config %s is invalid.", topicName, topicConfig); LOGGER.warn(exceptionMessage); future.setException( - new IoTDBException(exceptionMessage, TSStatusCode.PIPE_ERROR.getStatusCode())); + new IoTDBException(exceptionMessage, TSStatusCode.CREATE_TOPIC_ERROR.getStatusCode())); return future; } From 4a8b42c7e1dae969a2a52b2f74a76b92f8e16335 Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Thu, 16 May 2024 18:27:22 +0800 Subject: [PATCH 75/93] intro basic memory control --- .../common/SubscriptionPolledMessage.java | 14 -- .../response/PipeSubscribePollResp.java | 22 +-- .../SubscriptionPushConsumer.java | 4 +- .../broker/SubscriptionBroker.java | 11 -- .../SubscriptionPolledMessageBinaryCache.java | 100 ++++++++++++- .../broker/SubscriptionPrefetchingQueue.java | 7 +- .../SubscriptionPrefetchingTabletsQueue.java | 3 +- .../event/SubscriptionTsFileEvent.java | 4 +- .../receiver/SubscriptionReceiverV1.java | 134 ++++++++++-------- .../iotdb/commons/conf/CommonConfig.java | 11 ++ .../iotdb/commons/conf/CommonDescriptor.java | 6 + .../iotdb/commons/pipe/config/PipeConfig.java | 9 ++ 12 files changed, 217 insertions(+), 108 deletions(-) diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledMessage.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledMessage.java index f679fa68e4e9..10982ee9b750 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledMessage.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledMessage.java @@ -82,20 +82,6 @@ private void serialize(final DataOutputStream stream) throws IOException { commitContext.serialize(stream); } - /** - * @return true -> byte buffer is not null - */ - public boolean trySerialize() { - try { - SubscriptionPolledMessage.serialize(this); - } catch (final IOException e) { - LOGGER.warn( - "Subscription: something unexpected happened when serializing SubscriptionPolledMessage", - e); - } - return Objects.nonNull(byteBuffer); - } - public ByteBuffer getByteBuffer() { return byteBuffer; } diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollResp.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollResp.java index 3cb0ded76d51..6692d41aa90b 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollResp.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollResp.java @@ -20,12 +20,9 @@ package org.apache.iotdb.rpc.subscription.payload.response; import org.apache.iotdb.common.rpc.thrift.TSStatus; -import org.apache.iotdb.rpc.RpcUtils; -import org.apache.iotdb.rpc.TSStatusCode; import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPolledMessage; import org.apache.iotdb.service.rpc.thrift.TPipeSubscribeResp; -import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.List; @@ -46,28 +43,15 @@ public List getMessages() { * server. */ public static PipeSubscribePollResp toTPipeSubscribeResp( - TSStatus status, List messages) { + TSStatus status, List byteBuffers) { final PipeSubscribePollResp resp = new PipeSubscribePollResp(); - resp.messages = messages; + // resp.messages = messages; resp.status = status; resp.version = PipeSubscribeResponseVersion.VERSION_1.getVersion(); resp.type = PipeSubscribeResponseType.ACK.getType(); - try { - resp.body = new ArrayList<>(); - for (final SubscriptionPolledMessage message : messages) { - final ByteBuffer byteBuffer = - Objects.nonNull(message.getByteBuffer()) - ? message.getByteBuffer() - : SubscriptionPolledMessage.serialize(message); - resp.body.add(byteBuffer); - message.resetByteBuffer(); // maybe friendly for gc - } - } catch (IOException e) { - resp.status = - RpcUtils.getStatus(TSStatusCode.SUBSCRIPTION_SERIALIZATION_ERROR, e.getMessage()); - } + resp.body = byteBuffers; return resp; } diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPushConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPushConsumer.java index 5003f7c46c00..143935cc76ad 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPushConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPushConsumer.java @@ -242,9 +242,9 @@ public void run() { LOGGER.warn("Consumer listener result failure when consuming message: {}", message); messagesToNack.add(message); } - } catch (final Throwable t) { + } catch (final Exception e) { LOGGER.warn( - "Consumer listener raised an exception while consuming message: {}", message, t); + "Consumer listener raised an exception while consuming message: {}", message, e); messagesToNack.add(message); } } 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 61978921d5b7..d9afd96c3a01 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 @@ -67,12 +67,6 @@ public List poll(final String consumerId, final Set t if (topicNames.contains(topicName)) { final SubscriptionEvent event = prefetchingQueue.poll(consumerId); if (Objects.nonNull(event)) { - if (event.getMessage().trySerialize()) { - SubscriptionPrefetchingQueueMetrics.getInstance() - .mark( - prefetchingQueue.getPrefetchingQueueId(), - event.getMessage().getByteBuffer().limit()); - } events.add(event); } } @@ -104,11 +98,6 @@ public List pollTsFile( final SubscriptionEvent event = ((SubscriptionPrefetchingTsFileQueue) prefetchingQueue) .pollTsFile(consumerId, fileName, writingOffset); - if (event.getMessage().trySerialize()) { - SubscriptionPrefetchingQueueMetrics.getInstance() - .mark( - prefetchingQueue.getPrefetchingQueueId(), event.getMessage().getByteBuffer().limit()); - } return Collections.singletonList(event); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPolledMessageBinaryCache.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPolledMessageBinaryCache.java index 3f65d3bf822f..56a33904869e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPolledMessageBinaryCache.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPolledMessageBinaryCache.java @@ -19,6 +19,7 @@ package org.apache.iotdb.db.subscription.broker; +import org.apache.iotdb.commons.pipe.config.PipeConfig; import org.apache.iotdb.db.pipe.resource.PipeResourceManager; import org.apache.iotdb.db.pipe.resource.memory.PipeMemoryBlock; import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPolledMessage; @@ -27,27 +28,120 @@ import com.github.benmanes.caffeine.cache.Caffeine; import com.github.benmanes.caffeine.cache.LoadingCache; import com.github.benmanes.caffeine.cache.Weigher; +import com.google.common.util.concurrent.AtomicDouble; import org.checkerframework.checker.nullness.qual.NonNull; import org.checkerframework.checker.nullness.qual.Nullable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.IOException; import java.nio.ByteBuffer; public class SubscriptionPolledMessageBinaryCache { + private static final Logger LOGGER = + LoggerFactory.getLogger(SubscriptionPolledMessageBinaryCache.class); + private final PipeMemoryBlock allocatedMemoryBlock; + private final AtomicDouble memoryUsageCheatFactor = new AtomicDouble(1); + private final LoadingCache cache; - public SubscriptionPolledMessageBinaryCache() { + public ByteBuffer serialize(final SubscriptionPolledMessage message) throws IOException { + try { + return this.cache.get(message); + } catch (final Exception e) { + LOGGER.warn( + "SubscriptionPolledMessageBinaryCache raised an exception while serializing message: {}", + message, + e); + throw new IOException(e); + } + } + + /** + * @return true -> byte buffer is not null + */ + public boolean trySerialize(final SubscriptionPolledMessage message) { + try { + serialize(message); + return true; + } catch (final IOException e) { + LOGGER.warn( + "Subscription: something unexpected happened when serializing SubscriptionPolledMessage", + e); + return false; + } + } + + public void resetByteBuffer(final SubscriptionPolledMessage message) { + message.resetByteBuffer(); + this.cache.invalidate(message); + } + + public void clearCache() { + this.cache.invalidateAll(); + } + + //////////////////////////// singleton //////////////////////////// + + private static class SubscriptionPolledMessageBinaryCacheHolder { + + private static final SubscriptionPolledMessageBinaryCache INSTANCE = + new SubscriptionPolledMessageBinaryCache(); + + private SubscriptionPolledMessageBinaryCacheHolder() { + // empty constructor + } + } + + public static SubscriptionPolledMessageBinaryCache getInstance() { + return SubscriptionPolledMessageBinaryCache.SubscriptionPolledMessageBinaryCacheHolder.INSTANCE; + } + + private SubscriptionPolledMessageBinaryCache() { + final long initMemorySizeInBytes = + PipeResourceManager.memory().getTotalMemorySizeInBytes() / 10; + final long maxMemorySizeInBytes = + (long) + (PipeResourceManager.memory().getTotalMemorySizeInBytes() + * PipeConfig.getInstance().getSubscriptionCacheMemoryUsagePercentage()); + + // properties required by pipe memory control framework this.allocatedMemoryBlock = - PipeResourceManager.memory().tryAllocate(Runtime.getRuntime().maxMemory() / 50); + PipeResourceManager.memory() + .tryAllocate(initMemorySizeInBytes) + .setShrinkMethod(oldMemory -> Math.max(oldMemory / 2, 1)) + .setShrinkCallback( + (oldMemory, newMemory) -> { + memoryUsageCheatFactor.set( + memoryUsageCheatFactor.get() * ((double) oldMemory / newMemory)); + LOGGER.info( + "SubscriptionPolledMessageBinaryCache.allocatedMemoryBlock has shrunk from {} to {}.", + oldMemory, + newMemory); + }) + .setExpandMethod( + oldMemory -> Math.min(Math.max(oldMemory, 1) * 2, maxMemorySizeInBytes)) + .setExpandCallback( + (oldMemory, newMemory) -> { + memoryUsageCheatFactor.set( + memoryUsageCheatFactor.get() / ((double) newMemory / oldMemory)); + LOGGER.info( + "SubscriptionPolledMessageBinaryCache.allocatedMemoryBlock has expanded from {} to {}.", + oldMemory, + newMemory); + }); + this.cache = Caffeine.newBuilder() .maximumWeight(this.allocatedMemoryBlock.getMemoryUsageInBytes()) .weigher( (Weigher) - (message, buffer) -> buffer.limit()) + (message, buffer) -> { + return (int) (buffer.limit() * memoryUsageCheatFactor.get()); + }) .build( new CacheLoader() { @Override 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 2e7ca8481901..baeb4c4ed157 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 @@ -126,7 +126,12 @@ public String toString() { //////////////////////////// APIs provided for metric framework //////////////////////////// public String getPrefetchingQueueId() { - return brokerId + "_" + topicName; + return generatePrefetchingQueueId(brokerId, topicName); + } + + public static String generatePrefetchingQueueId( + final String consumerGroupId, final String topicName) { + return consumerGroupId + "_" + topicName; } public long getUncommittedEventCount() { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTabletsQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTabletsQueue.java index 061da48a2d74..6f1dc9e5b1f3 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTabletsQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTabletsQueue.java @@ -202,7 +202,8 @@ private void serializeOnce() { // Serialize the uncommitted and pollable event. if (currentEvent.pollable()) { // No need to concern whether serialization is successful. - currentEvent.getMessage().trySerialize(); + SubscriptionPolledMessageBinaryCache.getInstance() + .trySerialize(currentEvent.getMessage()); } } } catch (final InterruptedException e) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionTsFileEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionTsFileEvent.java index 937a365da8d3..aec1b2c4c2db 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionTsFileEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionTsFileEvent.java @@ -22,6 +22,7 @@ import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.commons.subscription.config.SubscriptionConfig; import org.apache.iotdb.db.pipe.event.common.tsfile.PipeTsFileInsertionEvent; +import org.apache.iotdb.db.subscription.broker.SubscriptionPolledMessageBinaryCache; import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionCommitContext; import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionMessagePayload; import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPolledMessage; @@ -108,7 +109,8 @@ public void serializeNext() { nextEventWithCommittableRef.getAndUpdate( (nextEventWithCommittable) -> { if (Objects.nonNull(nextEventWithCommittable)) { - nextEventWithCommittable.getLeft().getMessage().trySerialize(); + SubscriptionPolledMessageBinaryCache.getInstance() + .trySerialize(nextEventWithCommittable.getLeft().getMessage()); return nextEventWithCommittable; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java index 0abe60c8658e..a5e2be43ec51 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java @@ -33,7 +33,10 @@ import org.apache.iotdb.db.protocol.client.ConfigNodeInfo; import org.apache.iotdb.db.queryengine.plan.parser.ASTVisitor; import org.apache.iotdb.db.subscription.agent.SubscriptionAgent; +import org.apache.iotdb.db.subscription.broker.SubscriptionPolledMessageBinaryCache; +import org.apache.iotdb.db.subscription.broker.SubscriptionPrefetchingQueue; import org.apache.iotdb.db.subscription.event.SubscriptionEvent; +import org.apache.iotdb.db.subscription.metric.SubscriptionPrefetchingQueueMetrics; import org.apache.iotdb.rpc.RpcUtils; import org.apache.iotdb.rpc.TSStatusCode; import org.apache.iotdb.rpc.subscription.config.ConsumerConfig; @@ -69,6 +72,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.nio.ByteBuffer; import java.util.Collections; import java.util.List; import java.util.Objects; @@ -311,21 +315,70 @@ private TPipeSubscribeResp handlePipeSubscribePoll(final PipeSubscribePollReq re return SUBSCRIPTION_MISSING_CUSTOMER_RESP; } + final List messages; try { final SubscriptionPollMessage pollMessage = req.getPollMessage(); final short messageType = pollMessage.getMessageType(); - if (SubscriptionPollMessageType.isValidatedMessageType(messageType)) { switch (SubscriptionPollMessageType.valueOf(messageType)) { case POLL: - return handlePipeSubscribePollInternal( - consumerConfig, (PollMessagePayload) pollMessage.getMessagePayload()); + messages = + handlePipeSubscribePollInternal( + consumerConfig, (PollMessagePayload) pollMessage.getMessagePayload()); + break; case POLL_TS_FILE: - return handlePipeSubscribePollTsFileInternal( - consumerConfig, (PollTsFileMessagePayload) pollMessage.getMessagePayload()); + messages = + handlePipeSubscribePollTsFileInternal( + consumerConfig, (PollTsFileMessagePayload) pollMessage.getMessagePayload()); + break; default: + messages = null; break; } + } else { + messages = null; + } + if (Objects.nonNull(messages)) { + // generate response + return PipeSubscribePollResp.toTPipeSubscribeResp( + RpcUtils.SUCCESS_STATUS, + messages.parallelStream() + .map( + (message) -> { + final SubscriptionCommitContext commitContext = message.getCommitContext(); + try { + final ByteBuffer byteBuffer = + SubscriptionPolledMessageBinaryCache.getInstance().serialize(message); + SubscriptionPrefetchingQueueMetrics.getInstance() + .mark( + SubscriptionPrefetchingQueue.generatePrefetchingQueueId( + commitContext.getConsumerGroupId(), + commitContext.getTopicName()), + byteBuffer.limit()); + SubscriptionPolledMessageBinaryCache.getInstance().resetByteBuffer(message); + LOGGER.info( + "Subscription: consumer {} poll message successfully with commit context: {}, req message: {}", + consumerConfig, + commitContext, + req.getPollMessage()); + return byteBuffer; + } catch (final Exception e) { + LOGGER.warn( + "Subscription: consumer {} poll message failed with commit context: {}, req message: {}", + consumerConfig, + commitContext, + req.getPollMessage(), + e); + SubscriptionAgent.broker() + .commit( + consumerConfig, + Collections.singletonList(message.getCommitContext()), + true); + return null; + } + }) + .filter(Objects::nonNull) + .collect(Collectors.toList())); } throw new SubscriptionException(String.format("unexpected message type: %s", messageType)); } catch (final SubscriptionException e) { @@ -339,69 +392,38 @@ private TPipeSubscribeResp handlePipeSubscribePoll(final PipeSubscribePollReq re } } - private TPipeSubscribeResp handlePipeSubscribePollInternal( + private List handlePipeSubscribePollInternal( final ConsumerConfig consumerConfig, final PollMessagePayload messagePayload) { - Set topicNames = messagePayload.getTopicNames(); - if (topicNames.isEmpty()) { + final Set topicNames; + if (messagePayload.getTopicNames().isEmpty()) { // poll all subscribed topics topicNames = SubscriptionAgent.consumer() .getTopicsSubscribedByConsumer( consumerConfig.getConsumerGroupId(), consumerConfig.getConsumerId()); } else { - topicNames = topicNames.stream().map(ASTVisitor::parseIdentifier).collect(Collectors.toSet()); + topicNames = + messagePayload.getTopicNames().stream() + .map(ASTVisitor::parseIdentifier) + .collect(Collectors.toSet()); } - // poll - final List events = - SubscriptionAgent.broker().poll(consumerConfig, topicNames); - - final List polledMessages = - events.stream().map(SubscriptionEvent::getMessage).collect(Collectors.toList()); - - final List commitContexts = - polledMessages.stream() - .map(SubscriptionPolledMessage::getCommitContext) - .collect(Collectors.toList()); - - LOGGER.info( - "Subscription: consumer {} poll topics {} successfully, commit contexts: {}", - consumerConfig, - topicNames, - commitContexts); - - // generate response - return PipeSubscribePollResp.toTPipeSubscribeResp(RpcUtils.SUCCESS_STATUS, polledMessages); + return SubscriptionAgent.broker().poll(consumerConfig, topicNames).stream() + .map(SubscriptionEvent::getMessage) + .collect(Collectors.toList()); } - private TPipeSubscribeResp handlePipeSubscribePollTsFileInternal( + private List handlePipeSubscribePollTsFileInternal( final ConsumerConfig consumerConfig, final PollTsFileMessagePayload messagePayload) { - // poll - final List events = - SubscriptionAgent.broker() - .pollTsFile( - consumerConfig, - messagePayload.getTopicName(), - messagePayload.getFileName(), - messagePayload.getWritingOffset()); - - final List polledMessages = - events.stream().map(SubscriptionEvent::getMessage).collect(Collectors.toList()); - - final List commitContexts = - polledMessages.stream() - .map(SubscriptionPolledMessage::getCommitContext) - .collect(Collectors.toList()); - - LOGGER.info( - "Subscription: consumer {} poll TsFile (topic name: {}, file name: {}, writing offset: {}) successfully, commit contexts: {}", - consumerConfig, - messagePayload.getTopicName(), - messagePayload.getFileName(), - messagePayload.getWritingOffset(), - commitContexts); - - return PipeSubscribePollResp.toTPipeSubscribeResp(RpcUtils.SUCCESS_STATUS, polledMessages); + return SubscriptionAgent.broker() + .pollTsFile( + consumerConfig, + messagePayload.getTopicName(), + messagePayload.getFileName(), + messagePayload.getWritingOffset()) + .stream() + .map(SubscriptionEvent::getMessage) + .collect(Collectors.toList()); } private TPipeSubscribeResp handlePipeSubscribeCommit(final PipeSubscribeCommitReq req) { 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 7868f7370c56..ac54e39a91c7 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 @@ -235,6 +235,8 @@ public class CommonConfig { private long twoStageAggregateDataRegionInfoCacheTimeInMs = 3 * 60 * 1000; // 3 minutes private long twoStageAggregateSenderEndPointsCacheInMs = 3 * 60 * 1000; // 3 minutes + private float subscriptionCacheMemoryUsagePercentage = 0.1F; + private int subscriptionSubtaskExecutorMaxThreadNum = Math.min(5, Math.max(1, Runtime.getRuntime().availableProcessors() / 2)); private int subscriptionMaxTabletsPerPrefetching = 16; @@ -1035,6 +1037,15 @@ public void setTwoStageAggregateSenderEndPointsCacheInMs( this.twoStageAggregateSenderEndPointsCacheInMs = twoStageAggregateSenderEndPointsCacheInMs; } + public float getSubscriptionCacheMemoryUsagePercentage() { + return subscriptionCacheMemoryUsagePercentage; + } + + public void setSubscriptionCacheMemoryUsagePercentage( + float subscriptionCacheMemoryUsagePercentage) { + this.subscriptionCacheMemoryUsagePercentage = subscriptionCacheMemoryUsagePercentage; + } + public int getSubscriptionSubtaskExecutorMaxThreadNum() { return subscriptionSubtaskExecutorMaxThreadNum; } 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 3dd7c7fd1e45..0e3fef93ba3a 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 @@ -549,6 +549,12 @@ private void loadPipeProps(Properties properties) { properties.getProperty( "two_stage_aggregate_sender_end_points_cache_in_ms", String.valueOf(config.getTwoStageAggregateSenderEndPointsCacheInMs())))); + + config.setSubscriptionCacheMemoryUsagePercentage( + Float.parseFloat( + properties.getProperty( + "subscription_cache_memory_usage_percentage", + String.valueOf(config.getSubscriptionCacheMemoryUsagePercentage())))); } private void loadSubscriptionProps(Properties properties) { 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..5180925ddd8a 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 @@ -267,6 +267,12 @@ public long getTwoStageAggregateSenderEndPointsCacheInMs() { return COMMON_CONFIG.getTwoStageAggregateSenderEndPointsCacheInMs(); } + /////////////////////////////// Subscription /////////////////////////////// + + public float getSubscriptionCacheMemoryUsagePercentage() { + return COMMON_CONFIG.getSubscriptionCacheMemoryUsagePercentage(); + } + /////////////////////////////// Utils /////////////////////////////// private static final Logger LOGGER = LoggerFactory.getLogger(PipeConfig.class); @@ -376,6 +382,9 @@ public void printAllConfigs() { LOGGER.info( "TwoStageAggregateSenderEndPointsCacheInMs: {}", getTwoStageAggregateSenderEndPointsCacheInMs()); + + LOGGER.info( + "SubscriptionCacheMemoryUsagePercentage: {}", getSubscriptionCacheMemoryUsagePercentage()); } /////////////////////////////// Singleton /////////////////////////////// From 2ac75db1581463309d91fae02a11d20033ea2251 Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Thu, 16 May 2024 21:24:16 +0800 Subject: [PATCH 76/93] improve prefetch and cache --- .../common/SubscriptionPolledMessage.java | 18 +----- .../broker/SubscriptionBroker.java | 3 +- .../broker/SubscriptionPrefetchingQueue.java | 8 +++ .../SubscriptionPrefetchingTabletsQueue.java | 24 +++----- .../subscription/event/SubscriptionEvent.java | 28 ++++++++- .../SubscriptionEventBinaryCache.java} | 61 +++++++------------ .../event/SubscriptionTsFileEvent.java | 21 ++++++- .../receiver/SubscriptionReceiverV1.java | 36 +++++------ 8 files changed, 101 insertions(+), 98 deletions(-) rename iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/{broker/SubscriptionPolledMessageBinaryCache.java => event/SubscriptionEventBinaryCache.java} (61%) diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledMessage.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledMessage.java index 10982ee9b750..03c4ddf1afe9 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledMessage.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledMessage.java @@ -27,7 +27,6 @@ import java.io.DataOutputStream; import java.io.IOException; import java.nio.ByteBuffer; -import java.util.Objects; public class SubscriptionPolledMessage { @@ -39,8 +38,6 @@ public class SubscriptionPolledMessage { private final transient SubscriptionCommitContext commitContext; - private ByteBuffer byteBuffer; - public SubscriptionPolledMessage( short messageType, SubscriptionMessagePayload messagePayload, @@ -65,14 +62,10 @@ public SubscriptionCommitContext getCommitContext() { /////////////////////////////// de/ser /////////////////////////////// public static ByteBuffer serialize(SubscriptionPolledMessage message) throws IOException { - if (Objects.nonNull(message.byteBuffer)) { - return message.byteBuffer; - } try (final PublicBAOS byteArrayOutputStream = new PublicBAOS(); final DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream)) { message.serialize(outputStream); - return message.byteBuffer = - ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size()); + return ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size()); } } @@ -82,15 +75,6 @@ private void serialize(final DataOutputStream stream) throws IOException { commitContext.serialize(stream); } - public ByteBuffer getByteBuffer() { - return byteBuffer; - } - - public void resetByteBuffer() { - // maybe friendly for gc - byteBuffer = null; - } - public static SubscriptionPolledMessage deserialize(final ByteBuffer buffer) { final short messageType = ReadWriteIOUtils.readShort(buffer); SubscriptionMessagePayload messagePayload = null; 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 d9afd96c3a01..5c0a5d25826d 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 @@ -161,7 +161,8 @@ public void unbindPrefetchingQueue(final String topicName) { LOGGER.warn("Subscription: prefetching queue bound to topic [{}] does not exist", topicName); return; } - // TODO: do something for events on-the-fly + // clean up uncommitted events + prefetchingQueue.cleanup(); topicNameToPrefetchingQueue.remove(topicName); SubscriptionPrefetchingQueueMetrics.getInstance() .deregister(prefetchingQueue.getPrefetchingQueueId()); 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 baeb4c4ed157..4b05864a3713 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 @@ -23,6 +23,7 @@ import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.pipe.agent.PipeAgent; import org.apache.iotdb.db.subscription.event.SubscriptionEvent; +import org.apache.iotdb.db.subscription.event.SubscriptionEventBinaryCache; import org.apache.iotdb.pipe.api.event.Event; import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionCommitContext; @@ -60,6 +61,13 @@ public SubscriptionPrefetchingQueue( public abstract void executePrefetch(); + /** clean up uncommitted events */ + public void cleanup() { + for (final SubscriptionEvent event : uncommittedEvents.values()) { + SubscriptionEventBinaryCache.getInstance().resetByteBuffer(event, true); + } + } + /////////////////////////////// commit /////////////////////////////// /** diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTabletsQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTabletsQueue.java index 6f1dc9e5b1f3..2c0ce2ff8fa1 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTabletsQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTabletsQueue.java @@ -27,6 +27,7 @@ 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.subscription.event.SubscriptionEvent; +import org.apache.iotdb.db.subscription.event.SubscriptionEventBinaryCache; import org.apache.iotdb.pipe.api.event.Event; import org.apache.iotdb.pipe.api.event.dml.insertion.TabletInsertionEvent; import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionCommitContext; @@ -63,11 +64,6 @@ public SubscriptionPrefetchingTabletsQueue( @Override public SubscriptionEvent poll(final String consumerId) { - if (prefetchingQueue.isEmpty()) { - prefetchOnce(SubscriptionConfig.getInstance().getSubscriptionMaxTabletsPerPrefetching()); - // without serializeOnce here - } - final long size = prefetchingQueue.size(); long count = 0; @@ -108,13 +104,12 @@ public SubscriptionEvent poll(final String consumerId) { @Override public void executePrefetch() { - prefetchOnce(SubscriptionConfig.getInstance().getSubscriptionMaxTabletsPerPrefetching()); + prefetchOnce(); serializeOnce(); } // TODO: use org.apache.iotdb.db.pipe.resource.memory.PipeMemoryManager.calculateTabletSizeInBytes - // for limit control - private void prefetchOnce(final long limit) { + private void prefetchOnce() { final List tablets = new ArrayList<>(); final List enrichedEvents = new ArrayList<>(); @@ -136,9 +131,6 @@ private void prefetchOnce(final long limit) { } tablets.addAll(currentTablets); enrichedEvents.add((EnrichedEvent) event); - if (tablets.size() >= limit) { - break; - } } else if (event instanceof PipeTsFileInsertionEvent) { for (final TabletInsertionEvent tabletInsertionEvent : ((PipeTsFileInsertionEvent) event).toTabletInsertionEvents()) { @@ -149,9 +141,6 @@ private void prefetchOnce(final long limit) { tablets.addAll(currentTablets); } enrichedEvents.add((EnrichedEvent) event); - if (tablets.size() >= limit) { - break; - } } else { // TODO: // - PipeHeartbeatEvent: ignored? (may affect pipe metrics) @@ -162,6 +151,10 @@ private void prefetchOnce(final long limit) { this, event); } + + if (!tablets.isEmpty()) { + break; + } } if (!tablets.isEmpty()) { @@ -202,8 +195,7 @@ private void serializeOnce() { // Serialize the uncommitted and pollable event. if (currentEvent.pollable()) { // No need to concern whether serialization is successful. - SubscriptionPolledMessageBinaryCache.getInstance() - .trySerialize(currentEvent.getMessage()); + SubscriptionEventBinaryCache.getInstance().trySerialize(currentEvent); } } } catch (final InterruptedException e) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionEvent.java index d0584f831e37..a1901e38157b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionEvent.java @@ -23,7 +23,10 @@ import org.apache.iotdb.commons.subscription.config.SubscriptionConfig; import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPolledMessage; +import java.io.IOException; +import java.nio.ByteBuffer; import java.util.List; +import java.util.Objects; import java.util.stream.Collectors; public class SubscriptionEvent { @@ -37,6 +40,8 @@ public class SubscriptionEvent { private long lastPolledTimestamp; private long committedTimestamp; + protected ByteBuffer byteBuffer; // serialized SubscriptionPolledMessage + public SubscriptionEvent( final List enrichedEvents, final SubscriptionPolledMessage message) { this.enrichedEvents = enrichedEvents; @@ -98,6 +103,24 @@ public String getLastPolledConsumerId() { return lastPolledConsumerId; } + //////////////////////////// byte buffer //////////////////////////// + + public ByteBuffer serialize() throws IOException { + if (Objects.nonNull(byteBuffer)) { + return byteBuffer; + } + return SubscriptionPolledMessage.serialize(message); + } + + public ByteBuffer getByteBuffer() { + return byteBuffer; + } + + public void resetByteBuffer(final boolean recursive) { + // maybe friendly for gc + byteBuffer = null; + } + /////////////////////////////// object /////////////////////////////// @Override @@ -112,6 +135,9 @@ public String toString() { + lastPolledTimestamp + ", committedTimestamp=" + committedTimestamp - + "}"; + + "}" + + "(message byte buffer size: " + + (Objects.nonNull(byteBuffer) ? byteBuffer.limit() : "") + + ")"; } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPolledMessageBinaryCache.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionEventBinaryCache.java similarity index 61% rename from iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPolledMessageBinaryCache.java rename to iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionEventBinaryCache.java index 56a33904869e..35806f2d9381 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPolledMessageBinaryCache.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionEventBinaryCache.java @@ -17,43 +17,38 @@ * under the License. */ -package org.apache.iotdb.db.subscription.broker; +package org.apache.iotdb.db.subscription.event; import org.apache.iotdb.commons.pipe.config.PipeConfig; import org.apache.iotdb.db.pipe.resource.PipeResourceManager; import org.apache.iotdb.db.pipe.resource.memory.PipeMemoryBlock; -import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPolledMessage; -import com.github.benmanes.caffeine.cache.CacheLoader; import com.github.benmanes.caffeine.cache.Caffeine; import com.github.benmanes.caffeine.cache.LoadingCache; import com.github.benmanes.caffeine.cache.Weigher; import com.google.common.util.concurrent.AtomicDouble; -import org.checkerframework.checker.nullness.qual.NonNull; -import org.checkerframework.checker.nullness.qual.Nullable; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; import java.nio.ByteBuffer; -public class SubscriptionPolledMessageBinaryCache { +public class SubscriptionEventBinaryCache { - private static final Logger LOGGER = - LoggerFactory.getLogger(SubscriptionPolledMessageBinaryCache.class); + private static final Logger LOGGER = LoggerFactory.getLogger(SubscriptionEventBinaryCache.class); private final PipeMemoryBlock allocatedMemoryBlock; private final AtomicDouble memoryUsageCheatFactor = new AtomicDouble(1); - private final LoadingCache cache; + private final LoadingCache cache; - public ByteBuffer serialize(final SubscriptionPolledMessage message) throws IOException { + public ByteBuffer serialize(final SubscriptionEvent message) throws IOException { try { return this.cache.get(message); } catch (final Exception e) { LOGGER.warn( - "SubscriptionPolledMessageBinaryCache raised an exception while serializing message: {}", + "SubscriptionMessageBinaryCache raised an exception while serializing message: {}", message, e); throw new IOException(e); @@ -63,44 +58,38 @@ public ByteBuffer serialize(final SubscriptionPolledMessage message) throws IOEx /** * @return true -> byte buffer is not null */ - public boolean trySerialize(final SubscriptionPolledMessage message) { + public boolean trySerialize(final SubscriptionEvent message) { try { serialize(message); return true; } catch (final IOException e) { LOGGER.warn( - "Subscription: something unexpected happened when serializing SubscriptionPolledMessage", - e); + "Subscription: something unexpected happened when serializing SubscriptionEvent", e); return false; } } - public void resetByteBuffer(final SubscriptionPolledMessage message) { - message.resetByteBuffer(); + public void resetByteBuffer(final SubscriptionEvent message, final boolean recursive) { + message.resetByteBuffer(recursive); this.cache.invalidate(message); } - public void clearCache() { - this.cache.invalidateAll(); - } - //////////////////////////// singleton //////////////////////////// - private static class SubscriptionPolledMessageBinaryCacheHolder { + private static class SubscriptionMessageBinaryCacheHolder { - private static final SubscriptionPolledMessageBinaryCache INSTANCE = - new SubscriptionPolledMessageBinaryCache(); + private static final SubscriptionEventBinaryCache INSTANCE = new SubscriptionEventBinaryCache(); - private SubscriptionPolledMessageBinaryCacheHolder() { + private SubscriptionMessageBinaryCacheHolder() { // empty constructor } } - public static SubscriptionPolledMessageBinaryCache getInstance() { - return SubscriptionPolledMessageBinaryCache.SubscriptionPolledMessageBinaryCacheHolder.INSTANCE; + public static SubscriptionEventBinaryCache getInstance() { + return SubscriptionEventBinaryCache.SubscriptionMessageBinaryCacheHolder.INSTANCE; } - private SubscriptionPolledMessageBinaryCache() { + private SubscriptionEventBinaryCache() { final long initMemorySizeInBytes = PipeResourceManager.memory().getTotalMemorySizeInBytes() / 10; final long maxMemorySizeInBytes = @@ -118,7 +107,7 @@ private SubscriptionPolledMessageBinaryCache() { memoryUsageCheatFactor.set( memoryUsageCheatFactor.get() * ((double) oldMemory / newMemory)); LOGGER.info( - "SubscriptionPolledMessageBinaryCache.allocatedMemoryBlock has shrunk from {} to {}.", + "SubscriptionMessageBinaryCache.allocatedMemoryBlock has shrunk from {} to {}.", oldMemory, newMemory); }) @@ -129,7 +118,7 @@ private SubscriptionPolledMessageBinaryCache() { memoryUsageCheatFactor.set( memoryUsageCheatFactor.get() / ((double) newMemory / oldMemory)); LOGGER.info( - "SubscriptionPolledMessageBinaryCache.allocatedMemoryBlock has expanded from {} to {}.", + "SubscriptionMessageBinaryCache.allocatedMemoryBlock has expanded from {} to {}.", oldMemory, newMemory); }); @@ -138,19 +127,11 @@ private SubscriptionPolledMessageBinaryCache() { Caffeine.newBuilder() .maximumWeight(this.allocatedMemoryBlock.getMemoryUsageInBytes()) .weigher( - (Weigher) + (Weigher) (message, buffer) -> { + // TODO: overflow return (int) (buffer.limit() * memoryUsageCheatFactor.get()); }) - .build( - new CacheLoader() { - @Override - public @Nullable ByteBuffer load( - @NonNull final SubscriptionPolledMessage subscriptionPolledMessage) - throws IOException { - SubscriptionPolledMessage.serialize(subscriptionPolledMessage); - return subscriptionPolledMessage.getByteBuffer(); - } - }); + .build(SubscriptionEvent::serialize); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionTsFileEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionTsFileEvent.java index aec1b2c4c2db..e30ecd5af31a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionTsFileEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionTsFileEvent.java @@ -22,7 +22,6 @@ import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.commons.subscription.config.SubscriptionConfig; import org.apache.iotdb.db.pipe.event.common.tsfile.PipeTsFileInsertionEvent; -import org.apache.iotdb.db.subscription.broker.SubscriptionPolledMessageBinaryCache; import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionCommitContext; import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionMessagePayload; import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPolledMessage; @@ -109,8 +108,8 @@ public void serializeNext() { nextEventWithCommittableRef.getAndUpdate( (nextEventWithCommittable) -> { if (Objects.nonNull(nextEventWithCommittable)) { - SubscriptionPolledMessageBinaryCache.getInstance() - .trySerialize(nextEventWithCommittable.getLeft().getMessage()); + SubscriptionEventBinaryCache.getInstance() + .trySerialize(nextEventWithCommittable.getLeft()); return nextEventWithCommittable; } @@ -226,4 +225,20 @@ public SubscriptionTsFileEvent generateSubscriptionTsFileEventWithInitPayload() true); } } + + @Override + public void resetByteBuffer(final boolean recursive) { + super.resetByteBuffer(recursive); + if (recursive) { + nextEventWithCommittableRef.getAndUpdate( + (nextEventWithCommittable) -> { + if (Objects.isNull(nextEventWithCommittable)) { + return null; + } + SubscriptionEventBinaryCache.getInstance() + .resetByteBuffer(nextEventWithCommittable.getLeft(), true); + return nextEventWithCommittable; + }); + } + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java index a5e2be43ec51..5f1754edffab 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java @@ -33,9 +33,9 @@ import org.apache.iotdb.db.protocol.client.ConfigNodeInfo; import org.apache.iotdb.db.queryengine.plan.parser.ASTVisitor; import org.apache.iotdb.db.subscription.agent.SubscriptionAgent; -import org.apache.iotdb.db.subscription.broker.SubscriptionPolledMessageBinaryCache; import org.apache.iotdb.db.subscription.broker.SubscriptionPrefetchingQueue; import org.apache.iotdb.db.subscription.event.SubscriptionEvent; +import org.apache.iotdb.db.subscription.event.SubscriptionEventBinaryCache; import org.apache.iotdb.db.subscription.metric.SubscriptionPrefetchingQueueMetrics; import org.apache.iotdb.rpc.RpcUtils; import org.apache.iotdb.rpc.TSStatusCode; @@ -315,47 +315,48 @@ private TPipeSubscribeResp handlePipeSubscribePoll(final PipeSubscribePollReq re return SUBSCRIPTION_MISSING_CUSTOMER_RESP; } - final List messages; + final List events; try { final SubscriptionPollMessage pollMessage = req.getPollMessage(); final short messageType = pollMessage.getMessageType(); if (SubscriptionPollMessageType.isValidatedMessageType(messageType)) { switch (SubscriptionPollMessageType.valueOf(messageType)) { case POLL: - messages = + events = handlePipeSubscribePollInternal( consumerConfig, (PollMessagePayload) pollMessage.getMessagePayload()); break; case POLL_TS_FILE: - messages = + events = handlePipeSubscribePollTsFileInternal( consumerConfig, (PollTsFileMessagePayload) pollMessage.getMessagePayload()); break; default: - messages = null; + events = null; break; } } else { - messages = null; + events = null; } - if (Objects.nonNull(messages)) { + if (Objects.nonNull(events)) { // generate response return PipeSubscribePollResp.toTPipeSubscribeResp( RpcUtils.SUCCESS_STATUS, - messages.parallelStream() + events.parallelStream() .map( - (message) -> { + (event) -> { + final SubscriptionPolledMessage message = event.getMessage(); final SubscriptionCommitContext commitContext = message.getCommitContext(); try { final ByteBuffer byteBuffer = - SubscriptionPolledMessageBinaryCache.getInstance().serialize(message); + SubscriptionEventBinaryCache.getInstance().serialize(event); SubscriptionPrefetchingQueueMetrics.getInstance() .mark( SubscriptionPrefetchingQueue.generatePrefetchingQueueId( commitContext.getConsumerGroupId(), commitContext.getTopicName()), byteBuffer.limit()); - SubscriptionPolledMessageBinaryCache.getInstance().resetByteBuffer(message); + SubscriptionEventBinaryCache.getInstance().resetByteBuffer(event, false); LOGGER.info( "Subscription: consumer {} poll message successfully with commit context: {}, req message: {}", consumerConfig, @@ -392,7 +393,7 @@ private TPipeSubscribeResp handlePipeSubscribePoll(final PipeSubscribePollReq re } } - private List handlePipeSubscribePollInternal( + private List handlePipeSubscribePollInternal( final ConsumerConfig consumerConfig, final PollMessagePayload messagePayload) { final Set topicNames; if (messagePayload.getTopicNames().isEmpty()) { @@ -408,22 +409,17 @@ private List handlePipeSubscribePollInternal( .collect(Collectors.toSet()); } - return SubscriptionAgent.broker().poll(consumerConfig, topicNames).stream() - .map(SubscriptionEvent::getMessage) - .collect(Collectors.toList()); + return SubscriptionAgent.broker().poll(consumerConfig, topicNames); } - private List handlePipeSubscribePollTsFileInternal( + private List handlePipeSubscribePollTsFileInternal( final ConsumerConfig consumerConfig, final PollTsFileMessagePayload messagePayload) { return SubscriptionAgent.broker() .pollTsFile( consumerConfig, messagePayload.getTopicName(), messagePayload.getFileName(), - messagePayload.getWritingOffset()) - .stream() - .map(SubscriptionEvent::getMessage) - .collect(Collectors.toList()); + messagePayload.getWritingOffset()); } private TPipeSubscribeResp handlePipeSubscribeCommit(final PipeSubscribeCommitReq req) { From 01b75f8676f6b69cd1f8d23ba460a5bd7e1a9abf Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Fri, 17 May 2024 00:12:59 +0800 Subject: [PATCH 77/93] random initial delay & prefetch tsfile recursively & close consumer when exit --- .../subscription/config/ConsumerConstant.java | 16 +++--- .../payload/common/TabletsMessagePayload.java | 10 +++- .../common/TsFilePieceMessagePayload.java | 20 ++++++- .../subscription/SubscriptionConsumer.java | 21 +++++-- .../SubscriptionPullConsumer.java | 7 ++- .../SubscriptionPushConsumer.java | 4 +- .../broker/SubscriptionPrefetchingQueue.java | 3 +- .../SubscriptionPrefetchingTabletsQueue.java | 57 ++++++++++--------- .../SubscriptionPrefetchingTsFileQueue.java | 2 + .../subscription/event/SubscriptionEvent.java | 6 ++ .../event/SubscriptionEventBinaryCache.java | 1 + .../event/SubscriptionTsFileEvent.java | 6 +- .../receiver/SubscriptionReceiverV1.java | 27 ++++++--- 13 files changed, 124 insertions(+), 56 deletions(-) diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/config/ConsumerConstant.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/config/ConsumerConstant.java index eb2107ce6fa6..43c975b1d1b6 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/config/ConsumerConstant.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/config/ConsumerConstant.java @@ -34,13 +34,13 @@ public class ConsumerConstant { public static final String CONSUMER_GROUP_ID_KEY = "group-id"; public static final String HEARTBEAT_INTERVAL_MS_KEY = "heartbeat-interval-ms"; // unit: ms - public static final long HEARTBEAT_INTERVAL_MS_DEFAULT_VALUE = 5000; - public static final long HEARTBEAT_INTERVAL_MS_MIN_VALUE = 1000; + public static final long HEARTBEAT_INTERVAL_MS_DEFAULT_VALUE = 30_000; + public static final long HEARTBEAT_INTERVAL_MS_MIN_VALUE = 1_000; public static final String ENDPOINTS_SYNC_INTERVAL_MS_KEY = "endpoints-sync-interval-ms"; // unit: ms - public static final long ENDPOINTS_SYNC_INTERVAL_MS_DEFAULT_VALUE = 30000; - public static final long ENDPOINTS_SYNC_INTERVAL_MS_MIN_VALUE = 5000; + public static final long ENDPOINTS_SYNC_INTERVAL_MS_DEFAULT_VALUE = 120_000; + public static final long ENDPOINTS_SYNC_INTERVAL_MS_MIN_VALUE = 5_000; public static final String TS_FILE_BASE_DIR_KEY = "ts-file-base-dir"; public static final String TS_FILE_BASE_DIR_DEFAULT_VALUE = System.getProperty("user.dir"); @@ -51,7 +51,7 @@ public class ConsumerConstant { public static final boolean AUTO_COMMIT_DEFAULT_VALUE = true; public static final String AUTO_COMMIT_INTERVAL_MS_KEY = "auto-commit-interval-ms"; // unit: ms - public static final long AUTO_COMMIT_INTERVAL_MS_DEFAULT_VALUE = 5000; + public static final long AUTO_COMMIT_INTERVAL_MS_DEFAULT_VALUE = 5_000; public static final long AUTO_COMMIT_INTERVAL_MS_MIN_VALUE = 500; /////////////////////////////// push consumer /////////////////////////////// @@ -59,9 +59,9 @@ public class ConsumerConstant { public static final String ACK_STRATEGY_KEY = "ack-strategy"; public static final String CONSUME_LISTENER_KEY = "consume-listener"; - // TODO: configure this parameter - public static final int PUSH_CONSUMER_AUTO_POLL_INTERVAL_MS = 1000; - public static final int PUSH_CONSUMER_AUTO_POLL_TIME_OUT_MS = 2000; + // TODO: configure those parameters + public static final int PUSH_CONSUMER_AUTO_POLL_INTERVAL_MS = 5_000; + public static final int PUSH_CONSUMER_AUTO_POLL_TIME_OUT_MS = 10_000; private ConsumerConstant() { throw new IllegalStateException("Utility class"); diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TabletsMessagePayload.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TabletsMessagePayload.java index e83b9f5a0d20..011654edaf44 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TabletsMessagePayload.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TabletsMessagePayload.java @@ -33,10 +33,14 @@ public class TabletsMessagePayload implements SubscriptionMessagePayload { protected transient List tablets = new ArrayList<>(); + // Pure in-memory object, not involved in serialization and deserialization. + protected transient long calculatedTabletSizeInBytes; + public TabletsMessagePayload() {} - public TabletsMessagePayload(List tablets) { + public TabletsMessagePayload(final List tablets, final long calculatedTabletSizeInBytes) { this.tablets = tablets; + this.calculatedTabletSizeInBytes = calculatedTabletSizeInBytes; } public List getTablets() { @@ -81,6 +85,8 @@ public int hashCode() { @Override public String toString() { - return "TabletsMessagePayload{tablets=" + tablets + "}"; + return "TabletsMessagePayload{calculatedTabletSizeInBytes=" + + (calculatedTabletSizeInBytes == 0 ? "" : calculatedTabletSizeInBytes) + + "}"; } } diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFilePieceMessagePayload.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFilePieceMessagePayload.java index ed557d784560..9885f3f4124f 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFilePieceMessagePayload.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFilePieceMessagePayload.java @@ -98,7 +98,25 @@ public String toString() { + ", nextWritingOffset=" + nextWritingOffset + ", filePiece=" - + Arrays.toString(filePiece).substring(0, 256) + + formatByteArray(filePiece, 64) + "}"; } + + private static String formatByteArray(final byte[] filePiece, final int maxLength) { + final int length = filePiece.length; + final int displayLength = Math.min(length, maxLength); + final StringBuilder sb = new StringBuilder(); + sb.append("["); + for (int i = 0; i < displayLength; i++) { + sb.append(String.format("%02x", filePiece[i])); + if (i < displayLength - 1) { + sb.append(", "); + } + } + if (length > maxLength) { + sb.append("..."); + } + sb.append("]"); + return sb.toString(); + } } diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java index 91d590affb80..57a28e7207e5 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java @@ -284,8 +284,11 @@ private void launchHeartbeatWorker() { } return t; }); - heartbeatWorkerExecutor.scheduleAtFixedRate( - () -> subscriptionProviders.heartbeat(this), 0, heartbeatIntervalMs, TimeUnit.MILLISECONDS); + heartbeatWorkerExecutor.scheduleWithFixedDelay( + () -> subscriptionProviders.heartbeat(this), + generateRandomInitialDelayMs(heartbeatIntervalMs), + heartbeatIntervalMs, + TimeUnit.MILLISECONDS); } /** @@ -320,8 +323,11 @@ private void launchEndpointsSyncer() { } return t; }); - endpointsSyncerExecutor.scheduleAtFixedRate( - () -> subscriptionProviders.sync(this), 0, endpointsSyncIntervalMs, TimeUnit.MILLISECONDS); + endpointsSyncerExecutor.scheduleWithFixedDelay( + () -> subscriptionProviders.sync(this), + generateRandomInitialDelayMs(endpointsSyncIntervalMs), + endpointsSyncIntervalMs, + TimeUnit.MILLISECONDS); } private void shutdownEndpointsSyncer() { @@ -388,6 +394,7 @@ protected List poll(final Set topicNames, final lon } // update timer timer.update(); + // TODO: associated with timeoutMs instead of hardcode LockSupport.parkNanos(SLEEP_NS); // wait some time } while (timer.notExpired()); @@ -977,4 +984,10 @@ public String toString() { + consumerGroupId + "}"; } + + /////////////////////////////// utility /////////////////////////////// + + protected long generateRandomInitialDelayMs(final long maxMs) { + return (long) (Math.random() * maxMs); + } } diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java index bdf2a0ba40a3..a9a8972423bd 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java @@ -209,8 +209,11 @@ private void launchAutoCommitWorker() { } return t; }); - autoCommitWorkerExecutor.scheduleAtFixedRate( - new AutoCommitWorker(), 0, autoCommitIntervalMs, TimeUnit.MILLISECONDS); + autoCommitWorkerExecutor.scheduleWithFixedDelay( + new AutoCommitWorker(), + generateRandomInitialDelayMs(autoCommitIntervalMs), + autoCommitIntervalMs, + TimeUnit.MILLISECONDS); } private void shutdownAutoCommitWorker() { diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPushConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPushConsumer.java index 143935cc76ad..a94353706aeb 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPushConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPushConsumer.java @@ -125,9 +125,9 @@ private void launchAutoPollWorker() { } return t; }); - autoPollWorkerExecutor.scheduleAtFixedRate( + autoPollWorkerExecutor.scheduleWithFixedDelay( new AutoPollWorker(), - 0, + generateRandomInitialDelayMs(ConsumerConstant.PUSH_CONSUMER_AUTO_POLL_INTERVAL_MS), ConsumerConstant.PUSH_CONSUMER_AUTO_POLL_INTERVAL_MS, TimeUnit.MILLISECONDS); } 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 4b05864a3713..a7808c24f3a0 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 @@ -57,7 +57,7 @@ public SubscriptionPrefetchingQueue( this.uncommittedEvents = new ConcurrentHashMap<>(); } - public abstract SubscriptionEvent poll(String consumerId); + public abstract SubscriptionEvent poll(final String consumerId); public abstract void executePrefetch(); @@ -84,6 +84,7 @@ public boolean ack(final SubscriptionCommitContext commitContext) { } event.decreaseReferenceCount(); event.recordCommittedTimestamp(); + SubscriptionEventBinaryCache.getInstance().resetByteBuffer(event, true); uncommittedEvents.remove(commitContext); return true; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTabletsQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTabletsQueue.java index 2c0ce2ff8fa1..e3fc21f55753 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTabletsQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTabletsQueue.java @@ -26,6 +26,7 @@ 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.resource.memory.PipeMemoryManager; import org.apache.iotdb.db.subscription.event.SubscriptionEvent; import org.apache.iotdb.db.subscription.event.SubscriptionEventBinaryCache; import org.apache.iotdb.pipe.api.event.Event; @@ -64,31 +65,35 @@ public SubscriptionPrefetchingTabletsQueue( @Override public SubscriptionEvent poll(final String consumerId) { + if (prefetchingQueue.isEmpty()) { + prefetchOnce(); + } + final long size = prefetchingQueue.size(); long count = 0; SubscriptionEvent currentEvent; try { - while (Objects.nonNull( - currentEvent = - prefetchingQueue.poll( - SubscriptionConfig.getInstance().getSubscriptionPollMaxBlockingTimeMs(), - TimeUnit.MILLISECONDS))) { + while (count++ < size // limit control + && Objects.nonNull( + currentEvent = + prefetchingQueue.poll( + SubscriptionConfig.getInstance().getSubscriptionPollMaxBlockingTimeMs(), + TimeUnit.MILLISECONDS))) { if (currentEvent.isCommitted()) { continue; } - // Re-enqueue the uncommitted event at the end of the queue. - prefetchingQueue.add(currentEvent); - // limit control - if (count >= size) { - break; - } - count++; if (!currentEvent.pollable()) { + // Re-enqueue the uncommitted event at the end of the queue. + prefetchingQueue.add(currentEvent); continue; } currentEvent.recordLastPolledConsumerId(consumerId); currentEvent.recordLastPolledTimestamp(); + // Re-enqueue the uncommitted event at the end of the queue. + // This operation should be performed after recordLastPolledTimestamp to prevent multiple + // consumers from consuming the same event. + prefetchingQueue.add(currentEvent); return currentEvent; } } catch (final InterruptedException e) { @@ -108,7 +113,6 @@ public void executePrefetch() { serializeOnce(); } - // TODO: use org.apache.iotdb.db.pipe.resource.memory.PipeMemoryManager.calculateTabletSizeInBytes private void prefetchOnce() { final List tablets = new ArrayList<>(); final List enrichedEvents = new ArrayList<>(); @@ -164,7 +168,12 @@ private void prefetchOnce() { enrichedEvents, new SubscriptionPolledMessage( SubscriptionPolledMessageType.TABLETS.getType(), - new TabletsMessagePayload(tablets), + new TabletsMessagePayload( + tablets, + tablets.stream() + .map((PipeMemoryManager::calculateTabletSizeInBytes)) + .reduce(Long::sum) + .orElse(0L)), commitContext)); uncommittedEvents.put(commitContext, subscriptionEvent); // before enqueuing the event prefetchingQueue.add(subscriptionEvent); @@ -177,26 +186,22 @@ private void serializeOnce() { SubscriptionEvent currentEvent; try { - while (Objects.nonNull( - currentEvent = - prefetchingQueue.poll( - SubscriptionConfig.getInstance().getSubscriptionSerializeMaxBlockingTimeMs(), - TimeUnit.MILLISECONDS))) { + while (count++ < size // limit control + && Objects.nonNull( + currentEvent = + prefetchingQueue.poll( + SubscriptionConfig.getInstance().getSubscriptionSerializeMaxBlockingTimeMs(), + TimeUnit.MILLISECONDS))) { if (currentEvent.isCommitted()) { continue; } - // Re-enqueue the uncommitted event at the end of the queue. - prefetchingQueue.add(currentEvent); - // limit control - if (count >= size) { - break; - } - count++; // Serialize the uncommitted and pollable event. if (currentEvent.pollable()) { // No need to concern whether serialization is successful. SubscriptionEventBinaryCache.getInstance().trySerialize(currentEvent); } + // Re-enqueue the uncommitted event at the end of the queue. + prefetchingQueue.add(currentEvent); } } catch (final InterruptedException e) { Thread.currentThread().interrupt(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java index a50e5ecb3838..dfd8a87ab2cc 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java @@ -230,6 +230,7 @@ public SubscriptionTsFileEvent poll(final String consumerId) { event, writingOffset, consumerId); + // mark uncommittable uncommittedEvents.remove(polledMessage.getCommitContext()); break; default: @@ -273,6 +274,7 @@ public SubscriptionTsFileEvent poll(final String consumerId) { final SubscriptionTsFileEvent newEvent = newEventWithCommittable.getLeft(); consumerIdToCurrentEventMap.put(consumerId, newEvent); if (newEventWithCommittable.getRight()) { + // mark committable uncommittedEvents.put(newEvent.getMessage().getCommitContext(), newEvent); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionEvent.java index a1901e38157b..70fa6a28ffdc 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionEvent.java @@ -33,6 +33,7 @@ public class SubscriptionEvent { private static final long INVALID_TIMESTAMP = -1; + // the current length of enrichedEvents is always 1... protected final List enrichedEvents; protected final SubscriptionPolledMessage message; @@ -47,6 +48,7 @@ public SubscriptionEvent( this.enrichedEvents = enrichedEvents; this.message = message; + this.lastPolledConsumerId = null; this.lastPolledTimestamp = INVALID_TIMESTAMP; this.committedTimestamp = INVALID_TIMESTAMP; } @@ -82,6 +84,9 @@ public void recordLastPolledTimestamp() { } public boolean pollable() { + if (isCommitted()) { + return false; + } if (lastPolledTimestamp == INVALID_TIMESTAMP) { return true; } @@ -92,6 +97,7 @@ public boolean pollable() { } public void nack() { + lastPolledConsumerId = null; lastPolledTimestamp = INVALID_TIMESTAMP; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionEventBinaryCache.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionEventBinaryCache.java index 35806f2d9381..478b5516b08b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionEventBinaryCache.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionEventBinaryCache.java @@ -132,6 +132,7 @@ private SubscriptionEventBinaryCache() { // TODO: overflow return (int) (buffer.limit() * memoryUsageCheatFactor.get()); }) + .recordStats() // TODO: metrics .build(SubscriptionEvent::serialize); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionTsFileEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionTsFileEvent.java index e30ecd5af31a..4428723b9087 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionTsFileEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionTsFileEvent.java @@ -60,7 +60,8 @@ public void prefetchNext() { nextEventWithCommittableRef.getAndUpdate( (nextEventWithCommittable) -> { if (Objects.nonNull(nextEventWithCommittable)) { - // do nothing + // prefetch recursively + nextEventWithCommittable.getLeft().prefetchNext(); return nextEventWithCommittable; } @@ -110,6 +111,8 @@ public void serializeNext() { if (Objects.nonNull(nextEventWithCommittable)) { SubscriptionEventBinaryCache.getInstance() .trySerialize(nextEventWithCommittable.getLeft()); + // serialize recursively + nextEventWithCommittable.getLeft().serializeNext(); return nextEventWithCommittable; } @@ -235,6 +238,7 @@ public void resetByteBuffer(final boolean recursive) { if (Objects.isNull(nextEventWithCommittable)) { return null; } + // reset recursively SubscriptionEventBinaryCache.getInstance() .resetByteBuffer(nextEventWithCommittable.getLeft(), true); return nextEventWithCommittable; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java index 5f1754edffab..1960b9cbc334 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java @@ -104,10 +104,14 @@ public PipeSubscribeRequestVersion getVersion() { @Override public void handleExit() { - LOGGER.info( - "Subscription: remove consumer config {} when handling exit", - consumerConfigThreadLocal.get()); - consumerConfigThreadLocal.remove(); + final ConsumerConfig consumerConfig = consumerConfigThreadLocal.get(); + if (Objects.nonNull(consumerConfig)) { + LOGGER.info( + "Subscription: close and remove consumer config {} when handling exit", + consumerConfigThreadLocal.get()); + closeConsumer(consumerConfig); + consumerConfigThreadLocal.remove(); + } } @Override @@ -358,18 +362,19 @@ private TPipeSubscribeResp handlePipeSubscribePoll(final PipeSubscribePollReq re byteBuffer.limit()); SubscriptionEventBinaryCache.getInstance().resetByteBuffer(event, false); LOGGER.info( - "Subscription: consumer {} poll message successfully with commit context: {}, req message: {}", + "Subscription: consumer {} poll message {} successfully with req message: {}", consumerConfig, - commitContext, + message, req.getPollMessage()); return byteBuffer; } catch (final Exception e) { LOGGER.warn( - "Subscription: consumer {} poll message failed with commit context: {}, req message: {}", + "Subscription: consumer {} poll message {} failed with req message: {}", consumerConfig, - commitContext, + message, req.getPollMessage(), e); + // nack SubscriptionAgent.broker() .commit( consumerConfig, @@ -490,6 +495,11 @@ private TPipeSubscribeResp handlePipeSubscribeCloseInternal(final PipeSubscribeC return SUBSCRIPTION_MISSING_CUSTOMER_RESP; } + closeConsumer(consumerConfig); + return PipeSubscribeCloseResp.toTPipeSubscribeResp(RpcUtils.SUCCESS_STATUS); + } + + private void closeConsumer(final ConsumerConfig consumerConfig) { // unsubscribe all subscribed topics final Set topics = SubscriptionAgent.consumer() @@ -514,7 +524,6 @@ private TPipeSubscribeResp handlePipeSubscribeCloseInternal(final PipeSubscribeC } LOGGER.info("Subscription: consumer {} close successfully", consumerConfig); - return PipeSubscribeCloseResp.toTPipeSubscribeResp(RpcUtils.SUCCESS_STATUS); } //////////////////////////// consumer operations //////////////////////////// From e38f567dc4cdf30dbfb43ea07d8776226d65ba83 Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Fri, 17 May 2024 11:41:46 +0800 Subject: [PATCH 78/93] fix CI --- .../payload/common/TabletsMessagePayload.java | 11 ++++---- .../subscription/SubscriptionProviders.java | 2 +- .../SubscriptionPrefetchingTabletsQueue.java | 27 ++++++++++++++----- .../subscription/event/SubscriptionEvent.java | 1 - .../iotdb/commons/conf/CommonConfig.java | 15 +++++++++-- .../iotdb/commons/conf/CommonDescriptor.java | 5 ++++ .../config/SubscriptionConfig.java | 7 +++++ 7 files changed, 52 insertions(+), 16 deletions(-) diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TabletsMessagePayload.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TabletsMessagePayload.java index 011654edaf44..7d80855c14ff 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TabletsMessagePayload.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TabletsMessagePayload.java @@ -34,13 +34,14 @@ public class TabletsMessagePayload implements SubscriptionMessagePayload { protected transient List tablets = new ArrayList<>(); // Pure in-memory object, not involved in serialization and deserialization. - protected transient long calculatedTabletSizeInBytes; + protected transient long calculatedTabletsSizeInBytes; public TabletsMessagePayload() {} - public TabletsMessagePayload(final List tablets, final long calculatedTabletSizeInBytes) { + public TabletsMessagePayload( + final List tablets, final long calculatedTabletsSizeInBytes) { this.tablets = tablets; - this.calculatedTabletSizeInBytes = calculatedTabletSizeInBytes; + this.calculatedTabletsSizeInBytes = calculatedTabletsSizeInBytes; } public List getTablets() { @@ -85,8 +86,8 @@ public int hashCode() { @Override public String toString() { - return "TabletsMessagePayload{calculatedTabletSizeInBytes=" - + (calculatedTabletSizeInBytes == 0 ? "" : calculatedTabletSizeInBytes) + return "TabletsMessagePayload{calculatedTabletsSizeInBytes=" + + (calculatedTabletsSizeInBytes == 0 ? "" : calculatedTabletsSizeInBytes) + "}"; } } diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionProviders.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionProviders.java index 3e39211f7d5b..2c4346d6f4a2 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionProviders.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionProviders.java @@ -261,7 +261,7 @@ void sync(final SubscriptionConsumer consumer) { } private void syncInternal(final SubscriptionConsumer consumer) { - if (hasNoProviders()) { + if (hasNoAvailableProviders()) { try { openProviders(consumer); } catch (final Exception e) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTabletsQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTabletsQueue.java index e3fc21f55753..6dd48eefc96a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTabletsQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTabletsQueue.java @@ -116,6 +116,7 @@ public void executePrefetch() { private void prefetchOnce() { final List tablets = new ArrayList<>(); final List enrichedEvents = new ArrayList<>(); + long calculatedTabletsSizeInBytes = 0; Event event; while (Objects.nonNull( @@ -134,6 +135,11 @@ private void prefetchOnce() { continue; } tablets.addAll(currentTablets); + calculatedTabletsSizeInBytes += + currentTablets.stream() + .map((PipeMemoryManager::calculateTabletSizeInBytes)) + .reduce(Long::sum) + .orElse(0L); enrichedEvents.add((EnrichedEvent) event); } else if (event instanceof PipeTsFileInsertionEvent) { for (final TabletInsertionEvent tabletInsertionEvent : @@ -143,6 +149,11 @@ private void prefetchOnce() { continue; } tablets.addAll(currentTablets); + calculatedTabletsSizeInBytes += + currentTablets.stream() + .map((PipeMemoryManager::calculateTabletSizeInBytes)) + .reduce(Long::sum) + .orElse(0L); } enrichedEvents.add((EnrichedEvent) event); } else { @@ -156,7 +167,14 @@ private void prefetchOnce() { event); } - if (!tablets.isEmpty()) { + if (tablets.size() + >= SubscriptionConfig.getInstance().getSubscriptionMaxTabletsPerPrefetching()) { + break; + } + + if (calculatedTabletsSizeInBytes + >= SubscriptionConfig.getInstance() + .getSubscriptionMaxTabletsSizeInBytesPerPrefetching()) { break; } } @@ -168,12 +186,7 @@ private void prefetchOnce() { enrichedEvents, new SubscriptionPolledMessage( SubscriptionPolledMessageType.TABLETS.getType(), - new TabletsMessagePayload( - tablets, - tablets.stream() - .map((PipeMemoryManager::calculateTabletSizeInBytes)) - .reduce(Long::sum) - .orElse(0L)), + new TabletsMessagePayload(tablets, calculatedTabletsSizeInBytes), commitContext)); uncommittedEvents.put(commitContext, subscriptionEvent); // before enqueuing the event prefetchingQueue.add(subscriptionEvent); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionEvent.java index 70fa6a28ffdc..19705470f651 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionEvent.java @@ -33,7 +33,6 @@ public class SubscriptionEvent { private static final long INVALID_TIMESTAMP = -1; - // the current length of enrichedEvents is always 1... protected final List enrichedEvents; protected final SubscriptionPolledMessage message; 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 ac54e39a91c7..242d624a55f5 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 @@ -239,12 +239,13 @@ public class CommonConfig { private int subscriptionSubtaskExecutorMaxThreadNum = Math.min(5, Math.max(1, Runtime.getRuntime().availableProcessors() / 2)); - private int subscriptionMaxTabletsPerPrefetching = 16; + private int subscriptionMaxTabletsPerPrefetching = 64; + private int subscriptionMaxTabletsSizeInBytesPerPrefetching = 8388608; // 8MB private int subscriptionPollMaxBlockingTimeMs = 500; private int subscriptionSerializeMaxBlockingTimeMs = 100; private long subscriptionLaunchRetryIntervalMs = 1000; private int subscriptionRecycleUncommittedEventIntervalMs = 60000; // 60s - private int subscriptionReadFileBufferSize = 8388608; + private int subscriptionReadFileBufferSize = 8388608; // 8MB /** Whether to use persistent schema mode. */ private String schemaEngineMode = "Memory"; @@ -1066,6 +1067,16 @@ public void setSubscriptionMaxTabletsPerPrefetching(int subscriptionMaxTabletsPe this.subscriptionMaxTabletsPerPrefetching = subscriptionMaxTabletsPerPrefetching; } + public int getSubscriptionMaxTabletsSizeInBytesPerPrefetching() { + return subscriptionMaxTabletsSizeInBytesPerPrefetching; + } + + public void setSubscriptionMaxTabletsSizeInBytesPerPrefetching( + int subscriptionMaxTabletsSizeInBytesPerPrefetching) { + this.subscriptionMaxTabletsSizeInBytesPerPrefetching = + subscriptionMaxTabletsSizeInBytesPerPrefetching; + } + public int getSubscriptionPollMaxBlockingTimeMs() { return subscriptionPollMaxBlockingTimeMs; } 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 0e3fef93ba3a..92d28f31ae16 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 @@ -571,6 +571,11 @@ private void loadSubscriptionProps(Properties properties) { properties.getProperty( "subscription_max_tablets_per_prefetching", String.valueOf(config.getSubscriptionMaxTabletsPerPrefetching())))); + config.setSubscriptionMaxTabletsSizeInBytesPerPrefetching( + Integer.parseInt( + properties.getProperty( + "subscription_max_tablets_size_in_bytes_per_prefetching", + String.valueOf(config.getSubscriptionMaxTabletsSizeInBytesPerPrefetching())))); config.setSubscriptionPollMaxBlockingTimeMs( Integer.parseInt( properties.getProperty( diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/subscription/config/SubscriptionConfig.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/subscription/config/SubscriptionConfig.java index e5c97db393fe..3a269b505501 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/subscription/config/SubscriptionConfig.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/subscription/config/SubscriptionConfig.java @@ -39,6 +39,10 @@ public int getSubscriptionMaxTabletsPerPrefetching() { return COMMON_CONFIG.getSubscriptionMaxTabletsPerPrefetching(); } + public int getSubscriptionMaxTabletsSizeInBytesPerPrefetching() { + return COMMON_CONFIG.getSubscriptionMaxTabletsSizeInBytesPerPrefetching(); + } + public int getSubscriptionPollMaxBlockingTimeMs() { return COMMON_CONFIG.getSubscriptionPollMaxBlockingTimeMs(); } @@ -69,6 +73,9 @@ public void printAllConfigs() { getSubscriptionSubtaskExecutorMaxThreadNum()); LOGGER.info( "SubscriptionMaxTabletsPerPrefetching: {}", getSubscriptionMaxTabletsPerPrefetching()); + LOGGER.info( + "SubscriptionMaxTabletsSizeInBytesPerPrefetching: {}", + getSubscriptionMaxTabletsSizeInBytesPerPrefetching()); LOGGER.info("SubscriptionPollMaxBlockingTimeMs: {}", getSubscriptionPollMaxBlockingTimeMs()); LOGGER.info( "SubscriptionSerializeMaxBlockingTimeMs: {}", getSubscriptionSerializeMaxBlockingTimeMs()); From 46181c29a253727fb41aa6b547875cf2634cd437 Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Fri, 17 May 2024 12:01:20 +0800 Subject: [PATCH 79/93] improve restart CI --- .../it/local/IoTDBSubscriptionRestartIT.java | 36 ++++++++++--------- 1 file changed, 19 insertions(+), 17 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionRestartIT.java b/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionRestartIT.java index 1e89a8a2e6c0..5c2257f0634c 100644 --- a/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionRestartIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionRestartIT.java @@ -99,20 +99,24 @@ public void testSubscriptionAfterRestartCluster() throws Exception { } // Subscription + final SubscriptionPullConsumer consumer; try { - final SubscriptionPullConsumer consumer = + consumer = new SubscriptionPullConsumer.Builder() .host(host) .port(port) .consumerId("c1") .consumerGroupId("cg1") - .autoCommit(false) + .autoCommit(true) + .heartbeatIntervalMs(1000) // narrow heartbeat interval + .endpointsSyncIntervalMs(5000) // narrow endpoints sync interval .buildPullConsumer(); consumer.open(); consumer.subscribe(topicName); } catch (final Exception e) { e.printStackTrace(); fail(e.getMessage()); + return; } // Restart cluster @@ -151,19 +155,17 @@ public void testSubscriptionAfterRestartCluster() throws Exception { final Thread thread = new Thread( () -> { - try (final SubscriptionPullConsumer consumer = - new SubscriptionPullConsumer.Builder() - .host(host) - .port(port) - .consumerId("c1") - .consumerGroupId("cg1") - .autoCommit(false) - .buildPullConsumer()) { - consumer.open(); + try (final SubscriptionPullConsumer consumerRef = consumer) { while (!isClosed.get()) { LockSupport.parkNanos(IoTDBSubscriptionITConstant.SLEEP_NS); // wait some time - final List messages = - consumer.poll(IoTDBSubscriptionITConstant.POLL_TIMEOUT_MS); + final List messages; + try { + messages = consumer.poll(IoTDBSubscriptionITConstant.POLL_TIMEOUT_MS); + } catch (final Exception e) { + e.printStackTrace(); + // Avoid failure + continue; + } for (final SubscriptionMessage message : messages) { final SubscriptionSessionDataSets payload = (SubscriptionSessionDataSets) message.getPayload(); @@ -174,9 +176,9 @@ public void testSubscriptionAfterRestartCluster() throws Exception { } } } - consumer.commitSync(messages); + // Auto commit } - consumer.unsubscribe(topicName); + consumerRef.unsubscribe(topicName); } catch (final Exception e) { e.printStackTrace(); // Avoid failure @@ -284,8 +286,8 @@ public void testSubscriptionAfterRestartDataNode() throws Exception { timestamps.put(timestamp, timestamp); } } - // Auto commit } + // Auto commit } consumerRef.unsubscribe(topicName); } catch (final Exception e) { @@ -408,8 +410,8 @@ public void testSubscriptionWhenConfigNodeLeaderChange() throws Exception { timestamps.put(timestamp, timestamp); } } - // Auto commit } + // Auto commit } consumerRef.unsubscribe(topicName); } catch (final Exception e) { From 61f8c337b8f4681671b29f72a71c3a83789eaeae Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Fri, 17 May 2024 13:46:13 +0800 Subject: [PATCH 80/93] fix CI --- .../iotdb/db/subscription/receiver/SubscriptionReceiverV1.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java index 1960b9cbc334..c7dc63bb717e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java @@ -109,7 +109,7 @@ public void handleExit() { LOGGER.info( "Subscription: close and remove consumer config {} when handling exit", consumerConfigThreadLocal.get()); - closeConsumer(consumerConfig); + // closeConsumer(consumerConfig); consumerConfigThreadLocal.remove(); } } From 4aaa152fd97a4c7255270baae7daf82f46132f1e Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Sun, 19 May 2024 18:36:33 +0800 Subject: [PATCH 81/93] improve poll auto retry --- .../org/apache/iotdb/rpc/TSStatusCode.java | 1 - .../SubscriptionConnectionException.java | 2 +- ...ubscriptionParameterNotValidException.java | 2 +- ...SubscriptionRuntimeCriticalException.java} | 13 +- ...java => SubscriptionRuntimeException.java} | 12 +- ...bscriptionRuntimeNonCriticalException.java | 42 ++ .../common/TsFileErrorMessagePayload.java | 22 +- .../response/PipeSubscribeHandshakeResp.java | 3 +- .../subscription/SubscriptionConsumer.java | 426 +++++++++--------- .../subscription/SubscriptionProvider.java | 30 +- .../subscription/SubscriptionProviders.java | 2 +- .../util/RandomStringGenerator.java | 37 ++ .../broker/SubscriptionBroker.java | 1 + .../SubscriptionPrefetchingTsFileQueue.java | 174 +++---- .../event/SubscriptionTsFileEvent.java | 108 ++--- .../receiver/SubscriptionReceiverV1.java | 88 ++-- 16 files changed, 513 insertions(+), 450 deletions(-) rename iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/{SubscriptionNonRetryableException.java => SubscriptionRuntimeCriticalException.java} (66%) rename iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/{SubscriptionRetryableException.java => SubscriptionRuntimeException.java} (68%) create mode 100644 iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionRuntimeNonCriticalException.java create mode 100644 iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/util/RandomStringGenerator.java diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java index 78db457a054b..e434852e870d 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java @@ -239,7 +239,6 @@ public enum TSStatusCode { SUBSCRIPTION_UNSUBSCRIBE_ERROR(1908), SUBSCRIPTION_MISSING_CUSTOMER(1909), SHOW_SUBSCRIPTION_ERROR(1910), - SUBSCRIPTION_SERIALIZATION_ERROR(1911), // Topic CREATE_TOPIC_ERROR(2000), diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionConnectionException.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionConnectionException.java index fd2ea63cdbc0..55314ac410b8 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionConnectionException.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionConnectionException.java @@ -21,7 +21,7 @@ import java.util.Objects; -public class SubscriptionConnectionException extends SubscriptionNonRetryableException { +public class SubscriptionConnectionException extends SubscriptionRuntimeCriticalException { public SubscriptionConnectionException(String message) { super(message); diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionParameterNotValidException.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionParameterNotValidException.java index 322a65776f0e..5c903a94d648 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionParameterNotValidException.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionParameterNotValidException.java @@ -21,7 +21,7 @@ import java.util.Objects; -public class SubscriptionParameterNotValidException extends SubscriptionNonRetryableException { +public class SubscriptionParameterNotValidException extends SubscriptionException { public SubscriptionParameterNotValidException(String message) { super(message); diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionNonRetryableException.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionRuntimeCriticalException.java similarity index 66% rename from iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionNonRetryableException.java rename to iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionRuntimeCriticalException.java index 1ec85c5c6613..12236960ec1a 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionNonRetryableException.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionRuntimeCriticalException.java @@ -21,20 +21,21 @@ import java.util.Objects; -public class SubscriptionNonRetryableException extends SubscriptionException { +public class SubscriptionRuntimeCriticalException extends SubscriptionException { - public SubscriptionNonRetryableException(String message) { + public SubscriptionRuntimeCriticalException(String message) { super(message); } - public SubscriptionNonRetryableException(String message, Throwable cause) { + public SubscriptionRuntimeCriticalException(String message, Throwable cause) { super(message, cause); } @Override public boolean equals(Object obj) { - return obj instanceof SubscriptionNonRetryableException - && Objects.equals(getMessage(), ((SubscriptionNonRetryableException) obj).getMessage()) - && Objects.equals(getTimeStamp(), ((SubscriptionNonRetryableException) obj).getTimeStamp()); + return obj instanceof SubscriptionRuntimeCriticalException + && Objects.equals(getMessage(), ((SubscriptionRuntimeCriticalException) obj).getMessage()) + && Objects.equals( + getTimeStamp(), ((SubscriptionRuntimeCriticalException) obj).getTimeStamp()); } } diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionRetryableException.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionRuntimeException.java similarity index 68% rename from iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionRetryableException.java rename to iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionRuntimeException.java index d0467133c682..d27e8a1f26d1 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionRetryableException.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionRuntimeException.java @@ -21,20 +21,20 @@ import java.util.Objects; -public class SubscriptionRetryableException extends SubscriptionException { +public class SubscriptionRuntimeException extends SubscriptionException { - public SubscriptionRetryableException(String message) { + public SubscriptionRuntimeException(String message) { super(message); } - public SubscriptionRetryableException(String message, Throwable cause) { + public SubscriptionRuntimeException(String message, Throwable cause) { super(message, cause); } @Override public boolean equals(Object obj) { - return obj instanceof SubscriptionRetryableException - && Objects.equals(getMessage(), ((SubscriptionRetryableException) obj).getMessage()) - && Objects.equals(getTimeStamp(), ((SubscriptionRetryableException) obj).getTimeStamp()); + return obj instanceof SubscriptionRuntimeException + && Objects.equals(getMessage(), ((SubscriptionRuntimeException) obj).getMessage()) + && Objects.equals(getTimeStamp(), ((SubscriptionRuntimeException) obj).getTimeStamp()); } } diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionRuntimeNonCriticalException.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionRuntimeNonCriticalException.java new file mode 100644 index 000000000000..d23145c7b6bf --- /dev/null +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionRuntimeNonCriticalException.java @@ -0,0 +1,42 @@ +/* + * 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.rpc.subscription.exception; + +import java.util.Objects; + +public class SubscriptionRuntimeNonCriticalException extends SubscriptionRuntimeException { + + public SubscriptionRuntimeNonCriticalException(String message) { + super(message); + } + + public SubscriptionRuntimeNonCriticalException(String message, Throwable cause) { + super(message, cause); + } + + @Override + public boolean equals(Object obj) { + return obj instanceof SubscriptionRuntimeNonCriticalException + && Objects.equals( + getMessage(), ((SubscriptionRuntimeNonCriticalException) obj).getMessage()) + && Objects.equals( + getTimeStamp(), ((SubscriptionRuntimeNonCriticalException) obj).getTimeStamp()); + } +} diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileErrorMessagePayload.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileErrorMessagePayload.java index 15d928c699f0..aac62d5b74ae 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileErrorMessagePayload.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileErrorMessagePayload.java @@ -30,33 +30,33 @@ public class TsFileErrorMessagePayload implements SubscriptionMessagePayload { private transient String errorMessage; - private transient boolean retryable; + private transient boolean critical; public String getErrorMessage() { return errorMessage; } - public boolean isRetryable() { - return retryable; + public boolean isCritical() { + return critical; } public TsFileErrorMessagePayload() {} - public TsFileErrorMessagePayload(String errorMessage, boolean retryable) { + public TsFileErrorMessagePayload(String errorMessage, boolean critical) { this.errorMessage = errorMessage; - this.retryable = retryable; + this.critical = critical; } @Override public void serialize(DataOutputStream stream) throws IOException { ReadWriteIOUtils.write(errorMessage, stream); - ReadWriteIOUtils.write(retryable, stream); + ReadWriteIOUtils.write(critical, stream); } @Override public SubscriptionMessagePayload deserialize(ByteBuffer buffer) { this.errorMessage = ReadWriteIOUtils.readString(buffer); - this.retryable = ReadWriteIOUtils.readBool(buffer); + this.critical = ReadWriteIOUtils.readBool(buffer); return this; } @@ -70,20 +70,20 @@ public boolean equals(final Object obj) { } final TsFileErrorMessagePayload that = (TsFileErrorMessagePayload) obj; return Objects.equals(this.errorMessage, that.errorMessage) - && Objects.equals(this.retryable, that.retryable); + && Objects.equals(this.critical, that.critical); } @Override public int hashCode() { - return Objects.hash(errorMessage, retryable); + return Objects.hash(errorMessage, critical); } @Override public String toString() { return "TsFileErrorMessagePayload{errorMessage=" + errorMessage - + ", retryable=" - + retryable + + ", critical=" + + critical + "}"; } } diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribeHandshakeResp.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribeHandshakeResp.java index c131dcdbd963..d069dd2b4665 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribeHandshakeResp.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribeHandshakeResp.java @@ -80,8 +80,7 @@ public static PipeSubscribeHandshakeResp toTPipeSubscribeResp( Collections.singletonList( ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size())); } catch (IOException e) { - resp.status = - RpcUtils.getStatus(TSStatusCode.SUBSCRIPTION_SERIALIZATION_ERROR, e.getMessage()); + resp.status = RpcUtils.getStatus(TSStatusCode.SUBSCRIPTION_HANDSHAKE_ERROR, e.getMessage()); return resp; } diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java index 57a28e7207e5..3310bf008392 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java @@ -25,8 +25,8 @@ import org.apache.iotdb.rpc.subscription.config.ConsumerConstant; import org.apache.iotdb.rpc.subscription.exception.SubscriptionConnectionException; import org.apache.iotdb.rpc.subscription.exception.SubscriptionException; -import org.apache.iotdb.rpc.subscription.exception.SubscriptionNonRetryableException; -import org.apache.iotdb.rpc.subscription.exception.SubscriptionRetryableException; +import org.apache.iotdb.rpc.subscription.exception.SubscriptionRuntimeCriticalException; +import org.apache.iotdb.rpc.subscription.exception.SubscriptionRuntimeNonCriticalException; import org.apache.iotdb.rpc.subscription.payload.common.PollMessagePayload; import org.apache.iotdb.rpc.subscription.payload.common.PollTsFileMessagePayload; import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionCommitContext; @@ -41,6 +41,7 @@ import org.apache.iotdb.rpc.subscription.payload.common.TsFilePieceMessagePayload; import org.apache.iotdb.rpc.subscription.payload.common.TsFileSealMessagePayload; import org.apache.iotdb.session.subscription.payload.SubscriptionMessage; +import org.apache.iotdb.session.subscription.util.RandomStringGenerator; import org.apache.iotdb.session.subscription.util.SubscriptionPollTimer; import org.apache.iotdb.session.util.SessionUtils; @@ -105,6 +106,25 @@ private Path getTsFileDir(final String topicName) throws IOException { return dirPath; } + private Path getTsFilePath(final String topicName, String fileName) throws SubscriptionException { + Path filePath; + try { + filePath = getTsFileDir(topicName).resolve(fileName); + Files.createFile(filePath); + } catch (final FileAlreadyExistsException fileAlreadyExistsException) { + fileName += "." + RandomStringGenerator.generate(16); + try { + filePath = getTsFileDir(topicName).resolve(fileName); + Files.createFile(filePath); + } catch (final IOException e) { + throw new SubscriptionRuntimeNonCriticalException(e.getMessage(), e); + } + } catch (final IOException e) { + throw new SubscriptionRuntimeNonCriticalException(e.getMessage(), e); + } + return filePath; + } + public String getConsumerId() { return consumerId; } @@ -364,10 +384,14 @@ protected List poll(final Set topicNames, final lon new SubscriptionPollTimer(System.currentTimeMillis(), timeoutMs); do { - // poll tablets or tsfile - for (final SubscriptionPolledMessage polledMessage : pollInternal(topicNames)) { - final short messageType = polledMessage.getMessageType(); - if (SubscriptionPolledMessageType.isValidatedMessageType(messageType)) { + try { + // poll tablets or tsfile + for (final SubscriptionPolledMessage polledMessage : pollInternal(topicNames)) { + final short messageType = polledMessage.getMessageType(); + if (!SubscriptionPolledMessageType.isValidatedMessageType(messageType)) { + LOGGER.warn("unexpected message type: {}", messageType); + continue; + } switch (SubscriptionPolledMessageType.valueOf(messageType)) { case TABLETS: messages.add( @@ -385,16 +409,40 @@ protected List poll(final Set topicNames, final lon LOGGER.warn("unexpected message type: {}", messageType); break; } - } else { - LOGGER.warn("unexpected message type: {}", messageType); } + } catch (final SubscriptionRuntimeNonCriticalException e) { + LOGGER.warn( + "SubscriptionRuntimeNonCriticalException occurred when SubscriptionConsumer {} polling topics {}", + this, + topicNames, + e); + // nack and clear messages + try { + nack(messages); + messages.clear(); + } catch (final Exception ignored) { + } + } catch (final SubscriptionRuntimeCriticalException e) { + LOGGER.warn( + "SubscriptionRuntimeCriticalException occurred when SubscriptionConsumer {} polling topics {}", + this, + topicNames, + e); + // nack and clear messages + try { + nack(messages); + messages.clear(); + } catch (final Exception ignored) { + } + // rethrow + throw e; } if (!messages.isEmpty()) { return messages; } // update timer timer.update(); - // TODO: associated with timeoutMs instead of hardcode + // TODO: associated with timeoutMs instead of hardcoding LockSupport.parkNanos(SLEEP_NS); // wait some time } while (timer.notExpired()); @@ -404,68 +452,15 @@ protected List poll(final Set topicNames, final lon } private Optional pollTsFile( - final SubscriptionCommitContext commitContext, String fileName) throws SubscriptionException { + final SubscriptionCommitContext commitContext, final String fileName) + throws SubscriptionException { final String topicName = commitContext.getTopicName(); - Path filePath; - - try { - filePath = getTsFileDir(topicName).resolve(fileName); - Files.createFile(filePath); - } catch (final FileAlreadyExistsException fileAlreadyExistsException) { - LOGGER.info( - "FileAlreadyExistsException occurred when SubscriptionConsumer {} polling TsFile {} with commit context {}, append \".1\" to file name", - this, - fileName, - commitContext, - fileAlreadyExistsException); - fileName += ".1"; - try { - filePath = getTsFileDir(topicName).resolve(fileName); - Files.createFile(filePath); - } catch (final IOException e) { - LOGGER.warn( - "IOException occurred when SubscriptionConsumer {} polling TsFile {} with commit context {}", - this, - fileName, - commitContext, - e); - // TODO: Consider mid-process failures. - // rethrow - throw new SubscriptionNonRetryableException(e.getMessage(), e); - } - } catch (final IOException e) { - LOGGER.warn( - "IOException occurred when SubscriptionConsumer {} polling TsFile {} with commit context {}", - this, - fileName, - commitContext, - e); - // TODO: Consider mid-process failures. - // rethrow - throw new SubscriptionNonRetryableException(e.getMessage(), e); - } - + final Path filePath = getTsFilePath(topicName, fileName); final File file = filePath.toFile(); try (final RandomAccessFile fileWriter = new RandomAccessFile(file, "rw")) { return Optional.of(pollTsFileInternal(commitContext, file, fileWriter)); - } catch (final IOException | SubscriptionRetryableException e) { - LOGGER.warn( - "IOException or SubscriptionRetryableException occurred when SubscriptionConsumer {} polling TsFile {} with commit context {}", - this, - fileName, - commitContext, - e); - return Optional.empty(); - } catch (final SubscriptionNonRetryableException e) { - LOGGER.warn( - "SubscriptionNonRetryableException occurred when SubscriptionConsumer {} polling TsFile {} with commit context {}", - this, - fileName, - commitContext, - e); - // TODO: Consider mid-process failures. - // rethrow - throw e; + } catch (final IOException e) { + throw new SubscriptionRuntimeNonCriticalException(e.getMessage(), e); } } @@ -495,143 +490,142 @@ private SubscriptionMessage pollTsFileInternal( final String errorMessage = String.format("SubscriptionConsumer %s poll empty tsfile message", this); LOGGER.warn(errorMessage); - throw new SubscriptionNonRetryableException(errorMessage); + throw new SubscriptionRuntimeNonCriticalException(errorMessage); } + // Only one SubscriptionEvent polled currently... final SubscriptionPolledMessage polledMessage = polledMessages.get(0); final SubscriptionMessagePayload messagePayload = polledMessage.getMessagePayload(); - final short messageType = polledMessage.getMessageType(); - if (SubscriptionPolledMessageType.isValidatedMessageType(messageType)) { - switch (SubscriptionPolledMessageType.valueOf(messageType)) { - case TS_FILE_PIECE: - { - // check commit context - final SubscriptionCommitContext incomingCommitContext = - polledMessage.getCommitContext(); - if (Objects.isNull(incomingCommitContext) - || !Objects.equals(commitContext, incomingCommitContext)) { - final String errorMessage = - String.format( - "inconsistent commit context, current is %s, incoming is %s, consumer: %s", - commitContext, incomingCommitContext, this); - LOGGER.warn(errorMessage); - throw new SubscriptionNonRetryableException(errorMessage); - } - - // check file name - if (!fileName.startsWith( - ((TsFilePieceMessagePayload) messagePayload).getFileName())) { - final String errorMessage = - String.format( - "inconsistent file name, current is %s, incoming is %s, consumer: %s", - fileName, ((TsFilePieceMessagePayload) messagePayload).getFileName(), this); - LOGGER.warn(errorMessage); - throw new SubscriptionNonRetryableException(errorMessage); - } - - // write file piece - fileWriter.write(((TsFilePieceMessagePayload) messagePayload).getFilePiece()); - fileWriter.getFD().sync(); - - // check offset - if (!Objects.equals( - fileWriter.length(), - ((TsFilePieceMessagePayload) messagePayload).getNextWritingOffset())) { - final String errorMessage = - String.format( - "inconsistent file offset, current is %s, incoming is %s, consumer: %s", - fileWriter.length(), - ((TsFilePieceMessagePayload) messagePayload).getNextWritingOffset(), - this); - LOGGER.warn(errorMessage); - throw new SubscriptionNonRetryableException(errorMessage); - } + if (!SubscriptionPolledMessageType.isValidatedMessageType(messageType)) { + final String errorMessage = String.format("unexpected message type: %s", messageType); + LOGGER.warn(errorMessage); + throw new SubscriptionRuntimeNonCriticalException(errorMessage); + } - // update offset - writingOffset = ((TsFilePieceMessagePayload) messagePayload).getNextWritingOffset(); - break; + switch (SubscriptionPolledMessageType.valueOf(messageType)) { + case TS_FILE_PIECE: + { + // check commit context + final SubscriptionCommitContext incomingCommitContext = + polledMessage.getCommitContext(); + if (Objects.isNull(incomingCommitContext) + || !Objects.equals(commitContext, incomingCommitContext)) { + final String errorMessage = + String.format( + "inconsistent commit context, current is %s, incoming is %s, consumer: %s", + commitContext, incomingCommitContext, this); + LOGGER.warn(errorMessage); + throw new SubscriptionRuntimeNonCriticalException(errorMessage); } - case TS_FILE_SEAL: - { - // check commit context - final SubscriptionCommitContext incomingCommitContext = - polledMessage.getCommitContext(); - if (Objects.isNull(incomingCommitContext) - || !Objects.equals(commitContext, incomingCommitContext)) { - final String errorMessage = - String.format( - "inconsistent commit context, current is %s, incoming is %s, consumer: %s", - commitContext, incomingCommitContext, this); - LOGGER.warn(errorMessage); - throw new SubscriptionNonRetryableException(errorMessage); - } - // check file name - if (!fileName.startsWith(((TsFileSealMessagePayload) messagePayload).getFileName())) { - final String errorMessage = - String.format( - "inconsistent file name, current is %s, incoming is %s, consumer: %s", - fileName, ((TsFileSealMessagePayload) messagePayload).getFileName(), this); - LOGGER.warn(errorMessage); - throw new SubscriptionNonRetryableException(errorMessage); - } + // check file name + if (!fileName.startsWith(((TsFilePieceMessagePayload) messagePayload).getFileName())) { + final String errorMessage = + String.format( + "inconsistent file name, current is %s, incoming is %s, consumer: %s", + fileName, ((TsFilePieceMessagePayload) messagePayload).getFileName(), this); + LOGGER.warn(errorMessage); + throw new SubscriptionRuntimeNonCriticalException(errorMessage); + } - // check file length - if (fileWriter.length() - != ((TsFileSealMessagePayload) messagePayload).getFileLength()) { - final String errorMessage = - String.format( - "inconsistent file length, current is %s, incoming is %s, consumer: %s", - fileWriter.length(), - ((TsFileSealMessagePayload) messagePayload).getFileLength(), - this); - LOGGER.warn(errorMessage); - throw new SubscriptionNonRetryableException(errorMessage); - } + // write file piece + fileWriter.write(((TsFilePieceMessagePayload) messagePayload).getFilePiece()); + fileWriter.getFD().sync(); - // sync and close - fileWriter.getFD().sync(); - fileWriter.close(); + // check offset + if (!Objects.equals( + fileWriter.length(), + ((TsFilePieceMessagePayload) messagePayload).getNextWritingOffset())) { + final String errorMessage = + String.format( + "inconsistent file offset, current is %s, incoming is %s, consumer: %s", + fileWriter.length(), + ((TsFilePieceMessagePayload) messagePayload).getNextWritingOffset(), + this); + LOGGER.warn(errorMessage); + throw new SubscriptionRuntimeNonCriticalException(errorMessage); + } - LOGGER.info( - "SubscriptionConsumer {} successfully poll TsFile {} with commit context {}", - this, - file.getAbsolutePath(), - commitContext); + // update offset + writingOffset = ((TsFilePieceMessagePayload) messagePayload).getNextWritingOffset(); + break; + } + case TS_FILE_SEAL: + { + // check commit context + final SubscriptionCommitContext incomingCommitContext = + polledMessage.getCommitContext(); + if (Objects.isNull(incomingCommitContext) + || !Objects.equals(commitContext, incomingCommitContext)) { + final String errorMessage = + String.format( + "inconsistent commit context, current is %s, incoming is %s, consumer: %s", + commitContext, incomingCommitContext, this); + LOGGER.warn(errorMessage); + throw new SubscriptionRuntimeNonCriticalException(errorMessage); + } - // generate subscription message - return new SubscriptionMessage(commitContext, file.getAbsolutePath()); + // check file name + if (!fileName.startsWith(((TsFileSealMessagePayload) messagePayload).getFileName())) { + final String errorMessage = + String.format( + "inconsistent file name, current is %s, incoming is %s, consumer: %s", + fileName, ((TsFileSealMessagePayload) messagePayload).getFileName(), this); + LOGGER.warn(errorMessage); + throw new SubscriptionRuntimeNonCriticalException(errorMessage); } - case TS_FILE_ERROR: - { - // no need to check commit context + // check file length + if (fileWriter.length() + != ((TsFileSealMessagePayload) messagePayload).getFileLength()) { final String errorMessage = - ((TsFileErrorMessagePayload) messagePayload).getErrorMessage(); - final boolean retryable = ((TsFileErrorMessagePayload) messagePayload).isRetryable(); - LOGGER.warn( - "Error occurred when SubscriptionConsumer {} polling TsFile {} with commit context {}: {}, retryable: {}", - this, - file.getAbsolutePath(), - commitContext, - errorMessage, - retryable); - if (retryable) { - throw new SubscriptionRetryableException(errorMessage); - } else { - throw new SubscriptionNonRetryableException(errorMessage); - } + String.format( + "inconsistent file length, current is %s, incoming is %s, consumer: %s", + fileWriter.length(), + ((TsFileSealMessagePayload) messagePayload).getFileLength(), + this); + LOGGER.warn(errorMessage); + throw new SubscriptionRuntimeNonCriticalException(errorMessage); } - default: - final String errorMessage = String.format("unexpected message type: %s", messageType); - LOGGER.warn(errorMessage); - throw new SubscriptionNonRetryableException(errorMessage); - } - } else { - final String errorMessage = String.format("unexpected message type: %s", messageType); - LOGGER.warn(errorMessage); - throw new SubscriptionNonRetryableException(errorMessage); + + // sync and close + fileWriter.getFD().sync(); + fileWriter.close(); + + LOGGER.info( + "SubscriptionConsumer {} successfully poll TsFile {} with commit context {}", + this, + file.getAbsolutePath(), + commitContext); + + // generate subscription message + return new SubscriptionMessage(commitContext, file.getAbsolutePath()); + } + case TS_FILE_ERROR: + { + // no need to check commit context + + final String errorMessage = + ((TsFileErrorMessagePayload) messagePayload).getErrorMessage(); + final boolean critical = ((TsFileErrorMessagePayload) messagePayload).isCritical(); + LOGGER.warn( + "Error occurred when SubscriptionConsumer {} polling TsFile {} with commit context {}: {}, critical: {}", + this, + file.getAbsolutePath(), + commitContext, + errorMessage, + critical); + if (critical) { + throw new SubscriptionRuntimeCriticalException(errorMessage); + } else { + throw new SubscriptionRuntimeNonCriticalException(errorMessage); + } + } + default: + final String errorMessage = String.format("unexpected message type: %s", messageType); + LOGGER.warn(errorMessage); + throw new SubscriptionRuntimeNonCriticalException(errorMessage); } } } @@ -641,8 +635,11 @@ private List pollInternal(final Set topicName subscriptionProviders.acquireReadLock(); try { final SubscriptionProvider provider = subscriptionProviders.getNextAvailableProvider(); - if (Objects.isNull(provider)) { - return Collections.emptyList(); + if (Objects.isNull(provider) || !provider.isAvailable()) { + throw new SubscriptionConnectionException( + String.format( + "Cluster has no available subscription providers when %s poll topic %s", + this, topicNames)); } try { return provider.poll( @@ -650,27 +647,12 @@ private List pollInternal(final Set topicName SubscriptionPollMessageType.POLL.getType(), new PollMessagePayload(topicNames), 0L)); - } catch (final SubscriptionRetryableException e) { - LOGGER.warn( - "SubscriptionRetryableException occurred when SubscriptionConsumer {} polling from SubscriptionProvider {}", - this, - provider, - e); - // ignore - } catch (final SubscriptionNonRetryableException e) { - LOGGER.warn( - "SubscriptionNonRetryableException occurred when SubscriptionConsumer {} polling from SubscriptionProvider {}", - this, - provider, - e); - // rethrow - throw e; + } catch (final SubscriptionConnectionException ignored) { + return Collections.emptyList(); } } finally { subscriptionProviders.releaseReadLock(); } - - return Collections.emptyList(); } private List pollTsFileInternal( @@ -682,14 +664,18 @@ private List pollTsFileInternal( if (Objects.isNull(provider) || !provider.isAvailable()) { throw new SubscriptionConnectionException( String.format( - "something unexpected happened when poll TsFile from subscription provider with data node id %s, the subscription provider may be unavailable or not existed", - dataNodeId)); + "something unexpected happened when %s poll TsFile from subscription provider with data node id %s, the subscription provider may be unavailable or not existed", + this, dataNodeId)); + } + try { + return provider.poll( + new SubscriptionPollMessage( + SubscriptionPollMessageType.POLL_TS_FILE.getType(), + new PollTsFileMessagePayload(topicName, fileName, writingOffset), + 0L)); + } catch (final SubscriptionConnectionException ignored) { + return Collections.emptyList(); } - return provider.poll( - new SubscriptionPollMessage( - SubscriptionPollMessageType.POLL_TS_FILE.getType(), - new PollTsFileMessagePayload(topicName, fileName, writingOffset), - 0L)); } finally { subscriptionProviders.releaseReadLock(); } @@ -711,7 +697,7 @@ protected void ack(final Iterable messages) throws Subscrip } } - protected void nack(final Iterable messages) { + protected void nack(final Iterable messages) throws SubscriptionException { final Map> dataNodeIdToSubscriptionCommitContexts = new HashMap<>(); for (final SubscriptionMessage message : messages) { @@ -736,8 +722,8 @@ private void commitInternal( if (Objects.isNull(provider) || !provider.isAvailable()) { throw new SubscriptionConnectionException( String.format( - "something unexpected happened when commit messages to subscription provider with data node id %s, the subscription provider may be unavailable or not existed", - dataNodeId)); + "something unexpected happened when %s commit (nack: %s) messages to subscription provider with data node id %s, the subscription provider may be unavailable or not existed", + this, nack, dataNodeId)); } provider.commit(subscriptionCommitContexts, nack); } finally { @@ -811,7 +797,7 @@ private void subscribeWithRedirection(final Set topicNames) throws Subsc "%s failed to subscribe topics %s from all available subscription providers %s", this, topicNames, providers); LOGGER.warn(errorMessage); - throw new SubscriptionNonRetryableException(errorMessage); + throw new SubscriptionRuntimeCriticalException(errorMessage); } private void unsubscribeWithRedirection(final Set topicNames) @@ -841,7 +827,7 @@ private void unsubscribeWithRedirection(final Set topicNames) "%s failed to unsubscribe topics %s from all available subscription providers %s", this, topicNames, providers); LOGGER.warn(errorMessage); - throw new SubscriptionNonRetryableException(errorMessage); + throw new SubscriptionRuntimeCriticalException(errorMessage); } Map fetchAllEndPointsWithRedirection() throws SubscriptionException { @@ -867,7 +853,7 @@ Map fetchAllEndPointsWithRedirection() throws SubscriptionEx "%s failed to fetch all endpoints from all available subscription providers %s", this, providers); LOGGER.warn(errorMessage); - throw new SubscriptionNonRetryableException(errorMessage); + throw new SubscriptionRuntimeCriticalException(errorMessage); } /////////////////////////////// builder /////////////////////////////// diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionProvider.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionProvider.java index 727bbdfbe246..786a21bf0df0 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionProvider.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionProvider.java @@ -26,8 +26,8 @@ import org.apache.iotdb.rpc.subscription.config.ConsumerConstant; import org.apache.iotdb.rpc.subscription.exception.SubscriptionConnectionException; import org.apache.iotdb.rpc.subscription.exception.SubscriptionException; -import org.apache.iotdb.rpc.subscription.exception.SubscriptionNonRetryableException; -import org.apache.iotdb.rpc.subscription.exception.SubscriptionRetryableException; +import org.apache.iotdb.rpc.subscription.exception.SubscriptionRuntimeCriticalException; +import org.apache.iotdb.rpc.subscription.exception.SubscriptionRuntimeNonCriticalException; import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionCommitContext; import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPollMessage; import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPolledMessage; @@ -145,7 +145,7 @@ PipeSubscribeHandshakeResp handshake(final ConsumerConfig consumerConfig) this, consumerConfig, e); - throw new SubscriptionRetryableException(e.getMessage(), e); + throw new SubscriptionRuntimeNonCriticalException(e.getMessage(), e); } final TPipeSubscribeResp resp; try { @@ -223,7 +223,7 @@ void subscribe(final Set topicNames) throws SubscriptionException { this, topicNames, e); - throw new SubscriptionRetryableException(e.getMessage(), e); + throw new SubscriptionRuntimeNonCriticalException(e.getMessage(), e); } final TPipeSubscribeResp resp; try { @@ -251,7 +251,7 @@ void unsubscribe(final Set topicNames) throws SubscriptionException { this, topicNames, e); - throw new SubscriptionRetryableException(e.getMessage(), e); + throw new SubscriptionRuntimeNonCriticalException(e.getMessage(), e); } final TPipeSubscribeResp resp; try { @@ -280,7 +280,7 @@ List poll(final SubscriptionPollMessage pollMessage) this, pollMessage, e); - throw new SubscriptionRetryableException(e.getMessage(), e); + throw new SubscriptionRuntimeNonCriticalException(e.getMessage(), e); } final TPipeSubscribeResp resp; try { @@ -311,7 +311,7 @@ void commit(final List subscriptionCommitContexts, fi this, subscriptionCommitContexts, e); - throw new SubscriptionRetryableException(e.getMessage(), e); + throw new SubscriptionRuntimeNonCriticalException(e.getMessage(), e); } final TPipeSubscribeResp resp; try { @@ -334,8 +334,6 @@ private static void verifyPipeSubscribeSuccess(final TSStatus status) switch (status.code) { case 200: // SUCCESS_STATUS return; - case 1900: // SUBSCRIPTION_VERSION_ERROR - case 1901: // SUBSCRIPTION_TYPE_ERROR case 1902: // SUBSCRIPTION_HANDSHAKE_ERROR case 1903: // SUBSCRIPTION_HEARTBEAT_ERROR case 1904: // SUBSCRIPTION_POLL_ERROR @@ -343,24 +341,20 @@ private static void verifyPipeSubscribeSuccess(final TSStatus status) case 1906: // SUBSCRIPTION_CLOSE_ERROR case 1907: // SUBSCRIPTION_SUBSCRIBE_ERROR case 1908: // SUBSCRIPTION_UNSUBSCRIBE_ERROR - case 1909: // SUBSCRIPTION_MISSING_CUSTOMER LOGGER.warn( "Internal error occurred, status code {}, status message {}", status.code, status.message); - throw new SubscriptionNonRetryableException(status.message); - case 1911: // SUBSCRIPTION_SERIALIZATION_ERROR - LOGGER.warn( - "Internal error occurred when serialize response, status code {}, status message {}", - status.code, - status.message); - throw new SubscriptionRetryableException(status.message); + throw new SubscriptionRuntimeNonCriticalException(status.message); + case 1900: // SUBSCRIPTION_VERSION_ERROR + case 1901: // SUBSCRIPTION_TYPE_ERROR + case 1909: // SUBSCRIPTION_MISSING_CUSTOMER default: LOGGER.warn( "Internal error occurred, status code {}, status message {}", status.code, status.message); - throw new SubscriptionNonRetryableException(status.message); + throw new SubscriptionRuntimeCriticalException(status.message); } } diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionProviders.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionProviders.java index 2c4346d6f4a2..092a36f34397 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionProviders.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionProviders.java @@ -118,7 +118,7 @@ void openProviders(final SubscriptionConsumer consumer) break; } - if (hasNoProviders()) { + if (hasNoAvailableProviders()) { throw new SubscriptionConnectionException( String.format( "Cluster has no available subscription providers to connect with initial endpoints %s", diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/util/RandomStringGenerator.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/util/RandomStringGenerator.java new file mode 100644 index 000000000000..6b15c23edf5c --- /dev/null +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/util/RandomStringGenerator.java @@ -0,0 +1,37 @@ +/* + * 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.session.subscription.util; + +import java.security.SecureRandom; + +public class RandomStringGenerator { + + private static final String CHARACTERS = + "ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789"; + private static final SecureRandom RANDOM = new SecureRandom(); + + public static String generate(final int length) { + final StringBuilder sb = new StringBuilder(length); + for (int i = 0; i < length; i++) { + sb.append(CHARACTERS.charAt(RANDOM.nextInt(CHARACTERS.length()))); + } + return sb.toString(); + } +} 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 5c0a5d25826d..bc2cc69581f4 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 @@ -98,6 +98,7 @@ public List pollTsFile( final SubscriptionEvent event = ((SubscriptionPrefetchingTsFileQueue) prefetchingQueue) .pollTsFile(consumerId, fileName, writingOffset); + // Only one SubscriptionEvent polled currently... return Collections.singletonList(event); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java index dfd8a87ab2cc..fd63f0f838ce 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java @@ -112,7 +112,7 @@ public SubscriptionTsFileEvent poll(final String consumerId) { "SubscriptionPrefetchingTsFileQueue %s is currently not transferring any TsFile to consumer %s, file name: %s, writing offset: %s", this, consumerId, fileName, writingOffset); LOGGER.warn(errorMessage); - return generateSubscriptionTsFileEventWithErrorMessage(errorMessage, false); + return generateSubscriptionTsFileEventWithErrorMessage(errorMessage); } if (event.isCommitted()) { @@ -122,7 +122,7 @@ public SubscriptionTsFileEvent poll(final String consumerId) { "SubscriptionEvent %s related to TsFile is committed, consumer: %s, writing offset: %s, prefetching queue: %s", event, consumerId, writingOffset, this); LOGGER.warn(errorMessage); - return generateSubscriptionTsFileEventWithErrorMessage(errorMessage, false); + return generateSubscriptionTsFileEventWithErrorMessage(errorMessage); } // check consumer id @@ -132,7 +132,7 @@ public SubscriptionTsFileEvent poll(final String consumerId) { "inconsistent polled consumer id, current: %s, incoming: %s, file name: %s, writing offset: %s, prefetching queue: %s", event.getLastPolledConsumerId(), consumerId, fileName, writingOffset, this); LOGGER.warn(errorMessage); - return generateSubscriptionTsFileEventWithErrorMessage(errorMessage, false); + return generateSubscriptionTsFileEventWithErrorMessage(errorMessage); } final List enrichedEvents = event.getEnrichedEvents(); @@ -150,7 +150,7 @@ public SubscriptionTsFileEvent poll(final String consumerId) { writingOffset, this); LOGGER.warn(errorMessage); - return generateSubscriptionTsFileEventWithErrorMessage(errorMessage, false); + return generateSubscriptionTsFileEventWithErrorMessage(errorMessage); } final SubscriptionPolledMessage polledMessage = event.getMessage(); @@ -158,90 +158,89 @@ public SubscriptionTsFileEvent poll(final String consumerId) { // 2. Check message type, file name and offset final short messageType = polledMessage.getMessageType(); - if (SubscriptionPolledMessageType.isValidatedMessageType(messageType)) { - switch (SubscriptionPolledMessageType.valueOf(messageType)) { - case TS_FILE_INIT: - // check file name - if (!fileName.startsWith(((TsFileInitMessagePayload) messagePayload).getFileName())) { - final String errorMessage = - String.format( - "inconsistent file name, current: %s, incoming: %s, consumer: %s, writing offset: %s, prefetching queue: %s", - ((TsFileInitMessagePayload) messagePayload).getFileName(), - fileName, - consumerId, - writingOffset, - this); - LOGGER.warn(errorMessage); - return generateSubscriptionTsFileEventWithErrorMessage(errorMessage, false); - } - // check offset - if (writingOffset != 0) { - LOGGER.warn( - "SubscriptionPrefetchingTsFileQueue {} set TsFile (with event {}) writing offset to {} for consumer {}", - this, - event, - writingOffset, - consumerId); - } - break; - case TS_FILE_PIECE: - // check file name - if (!fileName.startsWith(((TsFilePieceMessagePayload) messagePayload).getFileName())) { - final String errorMessage = - String.format( - "inconsistent file name, current: %s, incoming: %s, consumer: %s, writing offset: %s, prefetching queue: %s", - ((TsFilePieceMessagePayload) messagePayload).getFileName(), - fileName, - consumerId, - writingOffset, - this); - LOGGER.warn(errorMessage); - return generateSubscriptionTsFileEventWithErrorMessage(errorMessage, false); - } - // check offset - if (writingOffset - != ((TsFilePieceMessagePayload) messagePayload).getNextWritingOffset()) { - LOGGER.warn( - "SubscriptionPrefetchingTsFileQueue {} set TsFile (with event {}) writing offset to {} for consumer {}", - this, - event, - writingOffset, - consumerId); - } - break; - case TS_FILE_SEAL: - // check file name - if (!fileName.startsWith(((TsFileSealMessagePayload) messagePayload).getFileName())) { - final String errorMessage = - String.format( - "inconsistent file name, current: %s, incoming: %s, consumer: %s, writing offset: %s, prefetching queue: %s", - ((TsFileSealMessagePayload) messagePayload).getFileName(), - fileName, - consumerId, - writingOffset, - this); - LOGGER.warn(errorMessage); - return generateSubscriptionTsFileEventWithErrorMessage(errorMessage, false); - } + if (!SubscriptionPolledMessageType.isValidatedMessageType(messageType)) { + final String errorMessage = String.format("unexpected message type: %s", messageType); + LOGGER.warn(errorMessage); + return generateSubscriptionTsFileEventWithErrorMessage(errorMessage); + } + switch (SubscriptionPolledMessageType.valueOf(messageType)) { + case TS_FILE_INIT: + // check file name + if (!fileName.startsWith(((TsFileInitMessagePayload) messagePayload).getFileName())) { + final String errorMessage = + String.format( + "inconsistent file name, current: %s, incoming: %s, consumer: %s, writing offset: %s, prefetching queue: %s", + ((TsFileInitMessagePayload) messagePayload).getFileName(), + fileName, + consumerId, + writingOffset, + this); + LOGGER.warn(errorMessage); + return generateSubscriptionTsFileEventWithErrorMessage(errorMessage); + } + // check offset + if (writingOffset != 0) { LOGGER.warn( - "SubscriptionPrefetchingTsFileQueue {} set TsFile (with event {}) writing offset to {} after transferring seal signal to consumer {}", + "SubscriptionPrefetchingTsFileQueue {} set TsFile (with event {}) writing offset to {} for consumer {}", this, event, writingOffset, consumerId); - // mark uncommittable - uncommittedEvents.remove(polledMessage.getCommitContext()); - break; - default: - final String errorMessage = String.format("unexpected message type: %s", messageType); + } + break; + case TS_FILE_PIECE: + // check file name + if (!fileName.startsWith(((TsFilePieceMessagePayload) messagePayload).getFileName())) { + final String errorMessage = + String.format( + "inconsistent file name, current: %s, incoming: %s, consumer: %s, writing offset: %s, prefetching queue: %s", + ((TsFilePieceMessagePayload) messagePayload).getFileName(), + fileName, + consumerId, + writingOffset, + this); LOGGER.warn(errorMessage); - return generateSubscriptionTsFileEventWithErrorMessage(errorMessage, false); - } - } else { - final String errorMessage = String.format("unexpected message type: %s", messageType); - LOGGER.warn(errorMessage); - return generateSubscriptionTsFileEventWithErrorMessage(errorMessage, false); + return generateSubscriptionTsFileEventWithErrorMessage(errorMessage); + } + // check offset + if (writingOffset != ((TsFilePieceMessagePayload) messagePayload).getNextWritingOffset()) { + LOGGER.warn( + "SubscriptionPrefetchingTsFileQueue {} set TsFile (with event {}) writing offset to {} for consumer {}", + this, + event, + writingOffset, + consumerId); + } + break; + case TS_FILE_SEAL: + // check file name + if (!fileName.startsWith(((TsFileSealMessagePayload) messagePayload).getFileName())) { + final String errorMessage = + String.format( + "inconsistent file name, current: %s, incoming: %s, consumer: %s, writing offset: %s, prefetching queue: %s", + ((TsFileSealMessagePayload) messagePayload).getFileName(), + fileName, + consumerId, + writingOffset, + this); + LOGGER.warn(errorMessage); + return generateSubscriptionTsFileEventWithErrorMessage(errorMessage); + } + + LOGGER.warn( + "SubscriptionPrefetchingTsFileQueue {} set TsFile (with event {}) writing offset to {} after transferring seal signal to consumer {}", + this, + event, + writingOffset, + consumerId); + // mark uncommittable + uncommittedEvents.remove(polledMessage.getCommitContext()); + break; + default: + final String errorMessage = String.format("unexpected message type: %s", messageType); + LOGGER.warn(errorMessage); + return generateSubscriptionTsFileEventWithErrorMessage(errorMessage); } // 3. Poll tsfile piece or tsfile seal @@ -262,8 +261,7 @@ public SubscriptionTsFileEvent poll(final String consumerId) { "IOException occurred when SubscriptionPrefetchingTsFileQueue %s transferring TsFile (with event %s) to consumer %s: %s", this, event, consumerId, e); LOGGER.warn(errorMessage); - // assume retryable - return generateSubscriptionTsFileEventWithErrorMessage(errorMessage, true); + return generateSubscriptionTsFileEventWithErrorMessage(errorMessage); } } @@ -353,12 +351,18 @@ private synchronized SubscriptionTsFileEvent getPollableOnTheFlySubscriptionTsFi } private SubscriptionTsFileEvent generateSubscriptionTsFileEventWithErrorMessage( - final String errorMessage, final boolean retryable) { + final String errorMessage, final boolean critical) { return new SubscriptionTsFileEvent( Collections.emptyList(), new SubscriptionPolledMessage( SubscriptionPolledMessageType.TS_FILE_ERROR.getType(), - new TsFileErrorMessagePayload(errorMessage, retryable), + new TsFileErrorMessagePayload(errorMessage, critical), super.generateInvalidSubscriptionCommitContext())); } + + private SubscriptionTsFileEvent generateSubscriptionTsFileEventWithErrorMessage( + final String errorMessage) { + // consider non-critical by default, meaning the client can retry + return generateSubscriptionTsFileEventWithErrorMessage(errorMessage, false); + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionTsFileEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionTsFileEvent.java index 4428723b9087..3c5d93649b86 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionTsFileEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionTsFileEvent.java @@ -68,40 +68,40 @@ public void prefetchNext() { final SubscriptionPolledMessage polledMessage = this.getMessage(); final short messageType = polledMessage.getMessageType(); final SubscriptionMessagePayload messagePayload = polledMessage.getMessagePayload(); - if (SubscriptionPolledMessageType.isValidatedMessageType(messageType)) { - switch (SubscriptionPolledMessageType.valueOf(messageType)) { - case TS_FILE_INIT: - try { - return generateSubscriptionTsFileEventWithPieceOrSealPayload(0); - } catch (final IOException e) { - LOGGER.warn( - "IOException occurred when prefetching next SubscriptionTsFileEvent, current SubscriptionTsFileEvent: {}", - this, - e); - return null; - } - case TS_FILE_PIECE: - try { - return generateSubscriptionTsFileEventWithPieceOrSealPayload( - ((TsFilePieceMessagePayload) messagePayload).getNextWritingOffset()); - } catch (final IOException e) { - LOGGER.warn( - "IOException occurred when prefetching next SubscriptionTsFileEvent, current SubscriptionTsFileEvent: {}", - this, - e); - return null; - } - case TS_FILE_SEAL: - // not need to prefetch - return null; - default: - LOGGER.warn("unexpected message type: {}", messageType); - return null; - } - } else { + if (!SubscriptionPolledMessageType.isValidatedMessageType(messageType)) { LOGGER.warn("unexpected message type: {}", messageType); return null; } + + switch (SubscriptionPolledMessageType.valueOf(messageType)) { + case TS_FILE_INIT: + try { + return generateSubscriptionTsFileEventWithPieceOrSealPayload(0); + } catch (final IOException e) { + LOGGER.warn( + "IOException occurred when prefetching next SubscriptionTsFileEvent, current SubscriptionTsFileEvent: {}", + this, + e); + return null; + } + case TS_FILE_PIECE: + try { + return generateSubscriptionTsFileEventWithPieceOrSealPayload( + ((TsFilePieceMessagePayload) messagePayload).getNextWritingOffset()); + } catch (final IOException e) { + LOGGER.warn( + "IOException occurred when prefetching next SubscriptionTsFileEvent, current SubscriptionTsFileEvent: {}", + this, + e); + return null; + } + case TS_FILE_SEAL: + // not need to prefetch + return null; + default: + LOGGER.warn("unexpected message type: {}", messageType); + return null; + } }); } @@ -131,32 +131,32 @@ public void serializeNext() { final SubscriptionPolledMessage polledMessage = this.getMessage(); final short messageType = polledMessage.getMessageType(); final SubscriptionMessagePayload messagePayload = polledMessage.getMessagePayload(); - if (SubscriptionPolledMessageType.isValidatedMessageType(messageType)) { - switch (SubscriptionPolledMessageType.valueOf(messageType)) { - case TS_FILE_INIT: - if (Objects.equals(writingOffset, 0)) { - return nextEventWithCommittable; - } - // reset next SubscriptionTsFileEvent - return null; - case TS_FILE_PIECE: - if (Objects.equals( - writingOffset, - ((TsFilePieceMessagePayload) messagePayload).getNextWritingOffset())) { - return nextEventWithCommittable; - } - // reset next SubscriptionTsFileEvent - return null; - case TS_FILE_SEAL: - return null; - default: - LOGGER.warn("unexpected message type: {}", messageType); - return null; - } - } else { + if (!SubscriptionPolledMessageType.isValidatedMessageType(messageType)) { LOGGER.warn("unexpected message type: {}", messageType); return null; } + + switch (SubscriptionPolledMessageType.valueOf(messageType)) { + case TS_FILE_INIT: + if (Objects.equals(writingOffset, 0)) { + return nextEventWithCommittable; + } + // reset next SubscriptionTsFileEvent + return null; + case TS_FILE_PIECE: + if (Objects.equals( + writingOffset, + ((TsFilePieceMessagePayload) messagePayload).getNextWritingOffset())) { + return nextEventWithCommittable; + } + // reset next SubscriptionTsFileEvent + return null; + case TS_FILE_SEAL: + return null; + default: + LOGGER.warn("unexpected message type: {}", messageType); + return null; + } }); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java index c7dc63bb717e..8b63000dda91 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java @@ -342,51 +342,51 @@ private TPipeSubscribeResp handlePipeSubscribePoll(final PipeSubscribePollReq re } else { events = null; } - if (Objects.nonNull(events)) { - // generate response - return PipeSubscribePollResp.toTPipeSubscribeResp( - RpcUtils.SUCCESS_STATUS, - events.parallelStream() - .map( - (event) -> { - final SubscriptionPolledMessage message = event.getMessage(); - final SubscriptionCommitContext commitContext = message.getCommitContext(); - try { - final ByteBuffer byteBuffer = - SubscriptionEventBinaryCache.getInstance().serialize(event); - SubscriptionPrefetchingQueueMetrics.getInstance() - .mark( - SubscriptionPrefetchingQueue.generatePrefetchingQueueId( - commitContext.getConsumerGroupId(), - commitContext.getTopicName()), - byteBuffer.limit()); - SubscriptionEventBinaryCache.getInstance().resetByteBuffer(event, false); - LOGGER.info( - "Subscription: consumer {} poll message {} successfully with req message: {}", - consumerConfig, - message, - req.getPollMessage()); - return byteBuffer; - } catch (final Exception e) { - LOGGER.warn( - "Subscription: consumer {} poll message {} failed with req message: {}", - consumerConfig, - message, - req.getPollMessage(), - e); - // nack - SubscriptionAgent.broker() - .commit( - consumerConfig, - Collections.singletonList(message.getCommitContext()), - true); - return null; - } - }) - .filter(Objects::nonNull) - .collect(Collectors.toList())); + if (Objects.isNull(events)) { + throw new SubscriptionException(String.format("unexpected message type: %s", messageType)); } - throw new SubscriptionException(String.format("unexpected message type: %s", messageType)); + + // generate response + return PipeSubscribePollResp.toTPipeSubscribeResp( + RpcUtils.SUCCESS_STATUS, + events.parallelStream() + .map( + (event) -> { + final SubscriptionPolledMessage message = event.getMessage(); + final SubscriptionCommitContext commitContext = message.getCommitContext(); + try { + final ByteBuffer byteBuffer = + SubscriptionEventBinaryCache.getInstance().serialize(event); + SubscriptionPrefetchingQueueMetrics.getInstance() + .mark( + SubscriptionPrefetchingQueue.generatePrefetchingQueueId( + commitContext.getConsumerGroupId(), commitContext.getTopicName()), + byteBuffer.limit()); + SubscriptionEventBinaryCache.getInstance().resetByteBuffer(event, false); + LOGGER.info( + "Subscription: consumer {} poll message {} successfully with req message: {}", + consumerConfig, + message, + req.getPollMessage()); + return byteBuffer; + } catch (final Exception e) { + LOGGER.warn( + "Subscription: consumer {} poll message {} failed with req message: {}", + consumerConfig, + message, + req.getPollMessage(), + e); + // nack + SubscriptionAgent.broker() + .commit( + consumerConfig, + Collections.singletonList(message.getCommitContext()), + true); + return null; + } + }) + .filter(Objects::nonNull) + .collect(Collectors.toList())); } catch (final SubscriptionException e) { final String exceptionMessage = String.format( From 9b2b684725231fec6bb3398565d5dc02bdb49f7d Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Mon, 20 May 2024 18:45:53 +0800 Subject: [PATCH 82/93] refactor --- .../iotdb/SubscriptionSessionExample.java | 17 +- .../IoTDBSubscriptionConsumerGroupIT.java | 16 +- .../it/dual/IoTDBSubscriptionTopicIT.java | 25 ++- .../it/local/IoTDBSubscriptionBasicIT.java | 22 +-- .../it/local/IoTDBSubscriptionRestartIT.java | 16 +- .../subscription/config/ConsumerConstant.java | 4 +- .../SubscriptionConnectionException.java | 6 +- .../exception/SubscriptionException.java | 6 +- ...scriptionIncompatibleHandlerException.java | 42 +++++ ...ubscriptionParameterNotValidException.java | 6 +- .../SubscriptionRuntimeCriticalException.java | 6 +- .../SubscriptionRuntimeException.java | 6 +- ...bscriptionRuntimeNonCriticalException.java | 6 +- .../ErrorPayload.java} | 20 +- .../FileInitPayload.java} | 16 +- .../FilePiecePayload.java} | 17 +- .../FileSealPayload.java} | 16 +- .../PollFilePayload.java} | 16 +- .../PollPayload.java} | 16 +- .../SubscriptionCommitContext.java | 7 +- .../SubscriptionPollPayload.java} | 6 +- .../SubscriptionPollRequest.java} | 64 +++---- .../SubscriptionPollRequestType.java} | 16 +- .../SubscriptionPollResponse.java} | 80 ++++---- .../SubscriptionPollResponseType.java} | 25 +-- .../TabletsPayload.java} | 17 +- .../request/PipeSubscribeCloseReq.java | 6 +- .../request/PipeSubscribeCommitReq.java | 12 +- .../request/PipeSubscribeHandshakeReq.java | 9 +- .../request/PipeSubscribeHeartbeatReq.java | 7 +- .../payload/request/PipeSubscribePollReq.java | 26 +-- .../request/PipeSubscribeRequestType.java | 6 +- .../request/PipeSubscribeRequestVersion.java | 2 +- .../request/PipeSubscribeSubscribeReq.java | 9 +- .../request/PipeSubscribeUnsubscribeReq.java | 8 +- .../response/PipeSubscribeCloseResp.java | 8 +- .../response/PipeSubscribeCommitResp.java | 9 +- .../response/PipeSubscribeHandshakeResp.java | 13 +- .../response/PipeSubscribeHeartbeatResp.java | 8 +- .../response/PipeSubscribePollResp.java | 24 +-- .../response/PipeSubscribeResponseType.java | 6 +- .../PipeSubscribeResponseVersion.java | 2 +- .../response/PipeSubscribeSubscribeResp.java | 8 +- .../PipeSubscribeUnsubscribeResp.java | 8 +- .../subscription/SubscriptionConsumer.java | 177 +++++++++--------- .../subscription/SubscriptionProvider.java | 10 +- .../SubscriptionPullConsumer.java | 4 +- .../SubscriptionPushConsumer.java | 4 +- .../payload/SubscriptionFileHandler.java | 37 ++++ .../payload/SubscriptionMessage.java | 46 +++-- ...d.java => SubscriptionMessageHandler.java} | 7 +- .../payload/SubscriptionMessageType.java | 10 +- ...> SubscriptionSessionDataSetsHandler.java} | 27 ++- ...er.java => SubscriptionTsFileHandler.java} | 14 +- .../agent/SubscriptionBrokerAgent.java | 2 +- .../broker/SubscriptionBroker.java | 2 +- .../broker/SubscriptionPrefetchingQueue.java | 2 +- .../SubscriptionPrefetchingTabletsQueue.java | 14 +- .../SubscriptionPrefetchingTsFileQueue.java | 89 +++++---- .../subscription/event/SubscriptionEvent.java | 22 +-- .../event/SubscriptionEventBinaryCache.java | 26 +-- .../event/SubscriptionTsFileEvent.java | 85 +++++---- .../receiver/SubscriptionReceiverV1.java | 51 +++-- 63 files changed, 690 insertions(+), 604 deletions(-) create mode 100644 iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionIncompatibleHandlerException.java rename iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/{common/TsFileErrorMessagePayload.java => poll/ErrorPayload.java} (76%) rename iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/{common/TsFileInitMessagePayload.java => poll/FileInitPayload.java} (76%) rename iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/{common/TsFilePieceMessagePayload.java => poll/FilePiecePayload.java} (85%) rename iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/{common/TsFileSealMessagePayload.java => poll/FileSealPayload.java} (77%) rename iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/{common/PollTsFileMessagePayload.java => poll/PollFilePayload.java} (82%) rename iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/{common/PollMessagePayload.java => poll/PollPayload.java} (79%) rename iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/{common => poll}/SubscriptionCommitContext.java (95%) rename iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/{common/SubscriptionMessagePayload.java => poll/SubscriptionPollPayload.java} (85%) rename iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/{common/SubscriptionPollMessage.java => poll/SubscriptionPollRequest.java} (55%) rename iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/{common/SubscriptionPollMessageType.java => poll/SubscriptionPollRequestType.java} (74%) rename iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/{common/SubscriptionPolledMessage.java => poll/SubscriptionPollResponse.java} (51%) rename iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/{common/SubscriptionPolledMessageType.java => poll/SubscriptionPollResponseType.java} (69%) rename iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/{common/TabletsMessagePayload.java => poll/TabletsPayload.java} (80%) create mode 100644 iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionFileHandler.java rename iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/{SubscriptionMessagePayload.java => SubscriptionMessageHandler.java} (84%) rename iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/{SubscriptionSessionDataSets.java => SubscriptionSessionDataSetsHandler.java} (62%) rename iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/{SubscriptionTsFileReader.java => SubscriptionTsFileHandler.java} (75%) diff --git a/example/session/src/main/java/org/apache/iotdb/SubscriptionSessionExample.java b/example/session/src/main/java/org/apache/iotdb/SubscriptionSessionExample.java index df9e45212b21..bde6014e0c46 100644 --- a/example/session/src/main/java/org/apache/iotdb/SubscriptionSessionExample.java +++ b/example/session/src/main/java/org/apache/iotdb/SubscriptionSessionExample.java @@ -29,9 +29,10 @@ import org.apache.iotdb.session.subscription.SubscriptionSession; import org.apache.iotdb.session.subscription.payload.SubscriptionMessage; import org.apache.iotdb.session.subscription.payload.SubscriptionSessionDataSet; -import org.apache.iotdb.session.subscription.payload.SubscriptionSessionDataSets; -import org.apache.iotdb.session.subscription.payload.SubscriptionTsFileReader; +import org.apache.tsfile.read.TsFileReader; + +import java.io.IOException; import java.util.ArrayList; import java.util.Collections; import java.util.List; @@ -129,9 +130,7 @@ private static void subscriptionExample1() throws Exception { } } for (final SubscriptionMessage message : messages) { - final SubscriptionSessionDataSets dataSets = - (SubscriptionSessionDataSets) message.getPayload(); - for (final SubscriptionSessionDataSet dataSet : dataSets) { + for (final SubscriptionSessionDataSet dataSet : message.getSessionDataSetsHandler()) { System.out.println(dataSet.getColumnNames()); System.out.println(dataSet.getColumnTypes()); while (dataSet.hasNext()) { @@ -188,14 +187,14 @@ private static void subscriptionExample2() throws Exception { } } for (final SubscriptionMessage message : messages) { - final SubscriptionTsFileReader reader = - (SubscriptionTsFileReader) message.getPayload(); - System.out.println(reader.toString()); + try (final TsFileReader reader = message.getTsFileHandler().openReader()) { + // do something... + } } consumer2.commitSync(messages); } consumer2.unsubscribe(TOPIC_2); - } catch (IoTDBConnectionException e) { + } catch (IOException | IoTDBConnectionException e) { throw new RuntimeException(e); } }); diff --git a/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionConsumerGroupIT.java b/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionConsumerGroupIT.java index 3b3b510e7729..b472628798f1 100644 --- a/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionConsumerGroupIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionConsumerGroupIT.java @@ -34,8 +34,6 @@ import org.apache.iotdb.session.subscription.payload.SubscriptionMessage; import org.apache.iotdb.session.subscription.payload.SubscriptionMessageType; import org.apache.iotdb.session.subscription.payload.SubscriptionSessionDataSet; -import org.apache.iotdb.session.subscription.payload.SubscriptionSessionDataSets; -import org.apache.iotdb.session.subscription.payload.SubscriptionTsFileReader; import org.apache.iotdb.subscription.it.IoTDBSubscriptionITConstant; import org.apache.tsfile.read.TsFileReader; @@ -940,11 +938,10 @@ private void pollMessagesAndCheck( final short messageType = message.getMessageType(); if (SubscriptionMessageType.isValidatedMessageType(messageType)) { switch (SubscriptionMessageType.valueOf(messageType)) { - case SESSION_DATA_SET: + case SUBSCRIPTION_SESSION_DATA_SETS_HANDLER: { - final SubscriptionSessionDataSets payload = - (SubscriptionSessionDataSets) message.getPayload(); - for (final SubscriptionSessionDataSet dataSet : payload) { + for (final SubscriptionSessionDataSet dataSet : + message.getSessionDataSetsHandler()) { final List columnNameList = dataSet.getColumnNames(); while (dataSet.hasNext()) { final RowRecord record = dataSet.next(); @@ -957,11 +954,10 @@ private void pollMessagesAndCheck( } break; } - case TS_FILE_READER: + case SUBSCRIPTION_TS_FILE_HANDLER: { - final SubscriptionTsFileReader reader = - (SubscriptionTsFileReader) message.getPayload(); - try (final TsFileReader tsFileReader = reader.open()) { + try (final TsFileReader tsFileReader = + message.getTsFileHandler().openReader()) { final QueryDataSet dataSet = tsFileReader.query( QueryExpression.create( diff --git a/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionTopicIT.java b/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionTopicIT.java index 265bbf597be6..edf36030840b 100644 --- a/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionTopicIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionTopicIT.java @@ -30,7 +30,6 @@ import org.apache.iotdb.session.subscription.SubscriptionPullConsumer; import org.apache.iotdb.session.subscription.SubscriptionSession; import org.apache.iotdb.session.subscription.payload.SubscriptionMessage; -import org.apache.iotdb.session.subscription.payload.SubscriptionSessionDataSets; import org.apache.iotdb.subscription.it.IoTDBSubscriptionITConstant; import org.apache.tsfile.write.record.Tablet; @@ -119,9 +118,9 @@ public void testTopicPathSubscription() throws Exception { final List messages = consumer.poll(IoTDBSubscriptionITConstant.POLL_TIMEOUT_MS); for (final SubscriptionMessage message : messages) { - final SubscriptionSessionDataSets payload = - (SubscriptionSessionDataSets) message.getPayload(); - for (final Iterator it = payload.tabletIterator(); it.hasNext(); ) { + for (final Iterator it = + message.getSessionDataSetsHandler().tabletIterator(); + it.hasNext(); ) { final Tablet tablet = it.next(); session.insertTablet(tablet); } @@ -221,9 +220,9 @@ public void testTopicTimeSubscription() throws Exception { final List messages = consumer.poll(IoTDBSubscriptionITConstant.POLL_TIMEOUT_MS); for (final SubscriptionMessage message : messages) { - final SubscriptionSessionDataSets payload = - (SubscriptionSessionDataSets) message.getPayload(); - for (final Iterator it = payload.tabletIterator(); it.hasNext(); ) { + for (final Iterator it = + message.getSessionDataSetsHandler().tabletIterator(); + it.hasNext(); ) { final Tablet tablet = it.next(); session.insertTablet(tablet); } @@ -319,9 +318,9 @@ public void testTopicProcessorSubscription() throws Exception { final List messages = consumer.poll(IoTDBSubscriptionITConstant.POLL_TIMEOUT_MS); for (final SubscriptionMessage message : messages) { - final SubscriptionSessionDataSets payload = - (SubscriptionSessionDataSets) message.getPayload(); - for (final Iterator it = payload.tabletIterator(); it.hasNext(); ) { + for (final Iterator it = + message.getSessionDataSetsHandler().tabletIterator(); + it.hasNext(); ) { final Tablet tablet = it.next(); session.insertTablet(tablet); } @@ -447,9 +446,9 @@ public void testTopicNameWithBackQuote() throws Exception { final List messages = consumer.poll(IoTDBSubscriptionITConstant.POLL_TIMEOUT_MS); for (final SubscriptionMessage message : messages) { - final SubscriptionSessionDataSets payload = - (SubscriptionSessionDataSets) message.getPayload(); - for (final Iterator it = payload.tabletIterator(); it.hasNext(); ) { + for (final Iterator it = + message.getSessionDataSetsHandler().tabletIterator(); + it.hasNext(); ) { final Tablet tablet = it.next(); session.insertTablet(tablet); } diff --git a/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionBasicIT.java b/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionBasicIT.java index ac697d8d5773..8e4c873b4933 100644 --- a/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionBasicIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionBasicIT.java @@ -32,8 +32,6 @@ import org.apache.iotdb.session.subscription.SubscriptionSession; import org.apache.iotdb.session.subscription.payload.SubscriptionMessage; import org.apache.iotdb.session.subscription.payload.SubscriptionSessionDataSet; -import org.apache.iotdb.session.subscription.payload.SubscriptionSessionDataSets; -import org.apache.iotdb.session.subscription.payload.SubscriptionTsFileReader; import org.apache.iotdb.subscription.it.IoTDBSubscriptionITConstant; import org.apache.tsfile.read.TsFileReader; @@ -125,9 +123,8 @@ public void testBasicSubscribeTablets() throws Exception { final List messages = consumer.poll(IoTDBSubscriptionITConstant.POLL_TIMEOUT_MS); for (final SubscriptionMessage message : messages) { - final SubscriptionSessionDataSets payload = - (SubscriptionSessionDataSets) message.getPayload(); - for (final SubscriptionSessionDataSet dataSet : payload) { + for (final SubscriptionSessionDataSet dataSet : + message.getSessionDataSetsHandler()) { while (dataSet.hasNext()) { dataSet.next(); rowCount.addAndGet(1); @@ -213,9 +210,8 @@ public void testBasicSubscribeTsFile() throws Exception { final List messages = consumer.poll(IoTDBSubscriptionITConstant.POLL_TIMEOUT_MS); for (final SubscriptionMessage message : messages) { - final SubscriptionTsFileReader reader = - (SubscriptionTsFileReader) message.getPayload(); - try (final TsFileReader tsFileReader = reader.open()) { + try (final TsFileReader tsFileReader = + message.getTsFileHandler().openReader()) { final Path path = new Path("root.db.d1", "s1", true); final QueryDataSet dataSet = tsFileReader.query( @@ -308,10 +304,9 @@ public void testBasicPullConsumerWithCommitAsync() throws Exception { continue; } for (final SubscriptionMessage message : messages) { - final SubscriptionSessionDataSets payload = - (SubscriptionSessionDataSets) message.getPayload(); int rowCountInOneMessage = 0; - for (final SubscriptionSessionDataSet dataSet : payload) { + for (final SubscriptionSessionDataSet dataSet : + message.getSessionDataSetsHandler()) { while (dataSet.hasNext()) { dataSet.next(); rowCount.addAndGet(1); @@ -446,9 +441,8 @@ public void testBasicPushConsumer() { .consumeListener( message -> { onReceiveCount.getAndIncrement(); - SubscriptionSessionDataSets dataSets = - (SubscriptionSessionDataSets) message.getPayload(); - dataSets + message + .getSessionDataSetsHandler() .tabletIterator() .forEachRemaining(tablet -> rowCount.addAndGet(tablet.rowSize)); return ConsumeResult.SUCCESS; diff --git a/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionRestartIT.java b/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionRestartIT.java index 5c2257f0634c..2b13acdc6d08 100644 --- a/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionRestartIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionRestartIT.java @@ -36,7 +36,6 @@ import org.apache.iotdb.session.subscription.SubscriptionSession; import org.apache.iotdb.session.subscription.payload.SubscriptionMessage; import org.apache.iotdb.session.subscription.payload.SubscriptionSessionDataSet; -import org.apache.iotdb.session.subscription.payload.SubscriptionSessionDataSets; import org.apache.iotdb.subscription.it.IoTDBSubscriptionITConstant; import org.awaitility.Awaitility; @@ -167,9 +166,8 @@ public void testSubscriptionAfterRestartCluster() throws Exception { continue; } for (final SubscriptionMessage message : messages) { - final SubscriptionSessionDataSets payload = - (SubscriptionSessionDataSets) message.getPayload(); - for (final SubscriptionSessionDataSet dataSet : payload) { + for (final SubscriptionSessionDataSet dataSet : + message.getSessionDataSetsHandler()) { while (dataSet.hasNext()) { final long timestamp = dataSet.next().getTimestamp(); timestamps.put(timestamp, timestamp); @@ -278,9 +276,8 @@ public void testSubscriptionAfterRestartDataNode() throws Exception { continue; } for (final SubscriptionMessage message : messages) { - final SubscriptionSessionDataSets payload = - (SubscriptionSessionDataSets) message.getPayload(); - for (final SubscriptionSessionDataSet dataSet : payload) { + for (final SubscriptionSessionDataSet dataSet : + message.getSessionDataSetsHandler()) { while (dataSet.hasNext()) { final long timestamp = dataSet.next().getTimestamp(); timestamps.put(timestamp, timestamp); @@ -402,9 +399,8 @@ public void testSubscriptionWhenConfigNodeLeaderChange() throws Exception { continue; } for (final SubscriptionMessage message : messages) { - final SubscriptionSessionDataSets payload = - (SubscriptionSessionDataSets) message.getPayload(); - for (final SubscriptionSessionDataSet dataSet : payload) { + for (final SubscriptionSessionDataSet dataSet : + message.getSessionDataSetsHandler()) { while (dataSet.hasNext()) { final long timestamp = dataSet.next().getTimestamp(); timestamps.put(timestamp, timestamp); diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/config/ConsumerConstant.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/config/ConsumerConstant.java index 43c975b1d1b6..2b8784c1cda4 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/config/ConsumerConstant.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/config/ConsumerConstant.java @@ -42,8 +42,8 @@ public class ConsumerConstant { public static final long ENDPOINTS_SYNC_INTERVAL_MS_DEFAULT_VALUE = 120_000; public static final long ENDPOINTS_SYNC_INTERVAL_MS_MIN_VALUE = 5_000; - public static final String TS_FILE_BASE_DIR_KEY = "ts-file-base-dir"; - public static final String TS_FILE_BASE_DIR_DEFAULT_VALUE = System.getProperty("user.dir"); + public static final String FILE_SAVE_DIR_KEY = "file-save-dir"; + public static final String FILE_SAVE_DIR_DEFAULT_VALUE = System.getProperty("user.dir"); /////////////////////////////// pull consumer /////////////////////////////// diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionConnectionException.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionConnectionException.java index 55314ac410b8..7bc4f75d59e6 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionConnectionException.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionConnectionException.java @@ -23,16 +23,16 @@ public class SubscriptionConnectionException extends SubscriptionRuntimeCriticalException { - public SubscriptionConnectionException(String message) { + public SubscriptionConnectionException(final String message) { super(message); } - public SubscriptionConnectionException(String message, Throwable cause) { + public SubscriptionConnectionException(final String message, final Throwable cause) { super(message, cause); } @Override - public boolean equals(Object obj) { + public boolean equals(final Object obj) { return obj instanceof SubscriptionConnectionException && Objects.equals(getMessage(), ((SubscriptionConnectionException) obj).getMessage()) && Objects.equals(getTimeStamp(), ((SubscriptionConnectionException) obj).getTimeStamp()); diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionException.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionException.java index f86c71891c6b..1b01dbbc5b11 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionException.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionException.java @@ -25,16 +25,16 @@ public class SubscriptionException extends PipeException { - public SubscriptionException(String message) { + public SubscriptionException(final String message) { super(message); } - public SubscriptionException(String message, Throwable cause) { + public SubscriptionException(final String message, final Throwable cause) { super(message, cause); } @Override - public boolean equals(Object obj) { + public boolean equals(final Object obj) { return obj instanceof SubscriptionException && Objects.equals(getMessage(), ((SubscriptionException) obj).getMessage()) && Objects.equals(getTimeStamp(), ((SubscriptionException) obj).getTimeStamp()); diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionIncompatibleHandlerException.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionIncompatibleHandlerException.java new file mode 100644 index 000000000000..710b8db8d70b --- /dev/null +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionIncompatibleHandlerException.java @@ -0,0 +1,42 @@ +/* + * 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.rpc.subscription.exception; + +import java.util.Objects; + +public class SubscriptionIncompatibleHandlerException extends SubscriptionException { + + public SubscriptionIncompatibleHandlerException(final String message) { + super(message); + } + + public SubscriptionIncompatibleHandlerException(final String message, final Throwable cause) { + super(message, cause); + } + + @Override + public boolean equals(final Object obj) { + return obj instanceof SubscriptionIncompatibleHandlerException + && Objects.equals( + getMessage(), ((SubscriptionIncompatibleHandlerException) obj).getMessage()) + && Objects.equals( + getTimeStamp(), ((SubscriptionIncompatibleHandlerException) obj).getTimeStamp()); + } +} diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionParameterNotValidException.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionParameterNotValidException.java index 5c903a94d648..8062c5829369 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionParameterNotValidException.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionParameterNotValidException.java @@ -23,16 +23,16 @@ public class SubscriptionParameterNotValidException extends SubscriptionException { - public SubscriptionParameterNotValidException(String message) { + public SubscriptionParameterNotValidException(final String message) { super(message); } - public SubscriptionParameterNotValidException(String message, Throwable cause) { + public SubscriptionParameterNotValidException(final String message, final Throwable cause) { super(message, cause); } @Override - public boolean equals(Object obj) { + public boolean equals(final Object obj) { return obj instanceof SubscriptionParameterNotValidException && Objects.equals(getMessage(), ((SubscriptionParameterNotValidException) obj).getMessage()) && Objects.equals( diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionRuntimeCriticalException.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionRuntimeCriticalException.java index 12236960ec1a..1267ea9085c9 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionRuntimeCriticalException.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionRuntimeCriticalException.java @@ -23,16 +23,16 @@ public class SubscriptionRuntimeCriticalException extends SubscriptionException { - public SubscriptionRuntimeCriticalException(String message) { + public SubscriptionRuntimeCriticalException(final String message) { super(message); } - public SubscriptionRuntimeCriticalException(String message, Throwable cause) { + public SubscriptionRuntimeCriticalException(final String message, final Throwable cause) { super(message, cause); } @Override - public boolean equals(Object obj) { + public boolean equals(final Object obj) { return obj instanceof SubscriptionRuntimeCriticalException && Objects.equals(getMessage(), ((SubscriptionRuntimeCriticalException) obj).getMessage()) && Objects.equals( diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionRuntimeException.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionRuntimeException.java index d27e8a1f26d1..aa7abb4eb02b 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionRuntimeException.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionRuntimeException.java @@ -23,16 +23,16 @@ public class SubscriptionRuntimeException extends SubscriptionException { - public SubscriptionRuntimeException(String message) { + public SubscriptionRuntimeException(final String message) { super(message); } - public SubscriptionRuntimeException(String message, Throwable cause) { + public SubscriptionRuntimeException(final String message, final Throwable cause) { super(message, cause); } @Override - public boolean equals(Object obj) { + public boolean equals(final Object obj) { return obj instanceof SubscriptionRuntimeException && Objects.equals(getMessage(), ((SubscriptionRuntimeException) obj).getMessage()) && Objects.equals(getTimeStamp(), ((SubscriptionRuntimeException) obj).getTimeStamp()); diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionRuntimeNonCriticalException.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionRuntimeNonCriticalException.java index d23145c7b6bf..6dab191551d7 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionRuntimeNonCriticalException.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/exception/SubscriptionRuntimeNonCriticalException.java @@ -23,16 +23,16 @@ public class SubscriptionRuntimeNonCriticalException extends SubscriptionRuntimeException { - public SubscriptionRuntimeNonCriticalException(String message) { + public SubscriptionRuntimeNonCriticalException(final String message) { super(message); } - public SubscriptionRuntimeNonCriticalException(String message, Throwable cause) { + public SubscriptionRuntimeNonCriticalException(final String message, final Throwable cause) { super(message, cause); } @Override - public boolean equals(Object obj) { + public boolean equals(final Object obj) { return obj instanceof SubscriptionRuntimeNonCriticalException && Objects.equals( getMessage(), ((SubscriptionRuntimeNonCriticalException) obj).getMessage()) diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileErrorMessagePayload.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/poll/ErrorPayload.java similarity index 76% rename from iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileErrorMessagePayload.java rename to iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/poll/ErrorPayload.java index aac62d5b74ae..7c2a763f1b95 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileErrorMessagePayload.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/poll/ErrorPayload.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.iotdb.rpc.subscription.payload.common; +package org.apache.iotdb.rpc.subscription.payload.poll; import org.apache.tsfile.utils.ReadWriteIOUtils; @@ -26,7 +26,7 @@ import java.nio.ByteBuffer; import java.util.Objects; -public class TsFileErrorMessagePayload implements SubscriptionMessagePayload { +public class ErrorPayload implements SubscriptionPollPayload { private transient String errorMessage; @@ -40,21 +40,21 @@ public boolean isCritical() { return critical; } - public TsFileErrorMessagePayload() {} + public ErrorPayload() {} - public TsFileErrorMessagePayload(String errorMessage, boolean critical) { + public ErrorPayload(final String errorMessage, final boolean critical) { this.errorMessage = errorMessage; this.critical = critical; } @Override - public void serialize(DataOutputStream stream) throws IOException { + public void serialize(final DataOutputStream stream) throws IOException { ReadWriteIOUtils.write(errorMessage, stream); ReadWriteIOUtils.write(critical, stream); } @Override - public SubscriptionMessagePayload deserialize(ByteBuffer buffer) { + public SubscriptionPollPayload deserialize(final ByteBuffer buffer) { this.errorMessage = ReadWriteIOUtils.readString(buffer); this.critical = ReadWriteIOUtils.readBool(buffer); return this; @@ -68,7 +68,7 @@ public boolean equals(final Object obj) { if (obj == null || getClass() != obj.getClass()) { return false; } - final TsFileErrorMessagePayload that = (TsFileErrorMessagePayload) obj; + final ErrorPayload that = (ErrorPayload) obj; return Objects.equals(this.errorMessage, that.errorMessage) && Objects.equals(this.critical, that.critical); } @@ -80,10 +80,6 @@ public int hashCode() { @Override public String toString() { - return "TsFileErrorMessagePayload{errorMessage=" - + errorMessage - + ", critical=" - + critical - + "}"; + return "ErrorPayload{errorMessage=" + errorMessage + ", critical=" + critical + "}"; } } diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileInitMessagePayload.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/poll/FileInitPayload.java similarity index 76% rename from iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileInitMessagePayload.java rename to iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/poll/FileInitPayload.java index b2329392acf0..04ebd8a89ab2 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileInitMessagePayload.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/poll/FileInitPayload.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.iotdb.rpc.subscription.payload.common; +package org.apache.iotdb.rpc.subscription.payload.poll; import org.apache.tsfile.utils.ReadWriteIOUtils; @@ -26,7 +26,7 @@ import java.nio.ByteBuffer; import java.util.Objects; -public class TsFileInitMessagePayload implements SubscriptionMessagePayload { +public class FileInitPayload implements SubscriptionPollPayload { private transient String fileName; @@ -34,19 +34,19 @@ public String getFileName() { return fileName; } - public TsFileInitMessagePayload() {} + public FileInitPayload() {} - public TsFileInitMessagePayload(String fileName) { + public FileInitPayload(final String fileName) { this.fileName = fileName; } @Override - public void serialize(DataOutputStream stream) throws IOException { + public void serialize(final DataOutputStream stream) throws IOException { ReadWriteIOUtils.write(fileName, stream); } @Override - public SubscriptionMessagePayload deserialize(ByteBuffer buffer) { + public SubscriptionPollPayload deserialize(final ByteBuffer buffer) { this.fileName = ReadWriteIOUtils.readString(buffer); return this; } @@ -59,7 +59,7 @@ public boolean equals(final Object obj) { if (obj == null || getClass() != obj.getClass()) { return false; } - final TsFileInitMessagePayload that = (TsFileInitMessagePayload) obj; + final FileInitPayload that = (FileInitPayload) obj; return Objects.equals(this.fileName, that.fileName); } @@ -70,6 +70,6 @@ public int hashCode() { @Override public String toString() { - return "TsFileInitMessagePayload{fileName=" + fileName + "}"; + return "FileInitPayload{fileName=" + fileName + "}"; } } diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFilePieceMessagePayload.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/poll/FilePiecePayload.java similarity index 85% rename from iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFilePieceMessagePayload.java rename to iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/poll/FilePiecePayload.java index 9885f3f4124f..5c915d83c684 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFilePieceMessagePayload.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/poll/FilePiecePayload.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.iotdb.rpc.subscription.payload.common; +package org.apache.iotdb.rpc.subscription.payload.poll; import org.apache.tsfile.utils.Binary; import org.apache.tsfile.utils.ReadWriteIOUtils; @@ -28,7 +28,7 @@ import java.util.Arrays; import java.util.Objects; -public class TsFilePieceMessagePayload implements SubscriptionMessagePayload { +public class FilePiecePayload implements SubscriptionPollPayload { private transient String fileName; @@ -48,23 +48,24 @@ public byte[] getFilePiece() { return filePiece; } - public TsFilePieceMessagePayload() {} + public FilePiecePayload() {} - public TsFilePieceMessagePayload(String fileName, long nextWritingOffset, byte[] filePiece) { + public FilePiecePayload( + final String fileName, final long nextWritingOffset, final byte[] filePiece) { this.fileName = fileName; this.nextWritingOffset = nextWritingOffset; this.filePiece = filePiece; } @Override - public void serialize(DataOutputStream stream) throws IOException { + public void serialize(final DataOutputStream stream) throws IOException { ReadWriteIOUtils.write(fileName, stream); ReadWriteIOUtils.write(nextWritingOffset, stream); ReadWriteIOUtils.write(new Binary(filePiece), stream); } @Override - public SubscriptionMessagePayload deserialize(ByteBuffer buffer) { + public SubscriptionPollPayload deserialize(final ByteBuffer buffer) { this.fileName = ReadWriteIOUtils.readString(buffer); this.nextWritingOffset = ReadWriteIOUtils.readLong(buffer); final int size = ReadWriteIOUtils.readInt(buffer); @@ -80,7 +81,7 @@ public boolean equals(final Object obj) { if (obj == null || getClass() != obj.getClass()) { return false; } - final TsFilePieceMessagePayload that = (TsFilePieceMessagePayload) obj; + final FilePiecePayload that = (FilePiecePayload) obj; return Objects.equals(this.fileName, that.fileName) && Objects.equals(this.nextWritingOffset, that.nextWritingOffset) && Arrays.equals(this.filePiece, that.filePiece); @@ -93,7 +94,7 @@ public int hashCode() { @Override public String toString() { - return "TsFilePieceMessagePayload{fileName=" + return "FilePiecePayload{fileName=" + fileName + ", nextWritingOffset=" + nextWritingOffset diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileSealMessagePayload.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/poll/FileSealPayload.java similarity index 77% rename from iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileSealMessagePayload.java rename to iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/poll/FileSealPayload.java index b899f0b582cc..bec792c2c9dc 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TsFileSealMessagePayload.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/poll/FileSealPayload.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.iotdb.rpc.subscription.payload.common; +package org.apache.iotdb.rpc.subscription.payload.poll; import org.apache.tsfile.utils.ReadWriteIOUtils; @@ -26,7 +26,7 @@ import java.nio.ByteBuffer; import java.util.Objects; -public class TsFileSealMessagePayload implements SubscriptionMessagePayload { +public class FileSealPayload implements SubscriptionPollPayload { private transient String fileName; @@ -40,21 +40,21 @@ public long getFileLength() { return fileLength; } - public TsFileSealMessagePayload() {} + public FileSealPayload() {} - public TsFileSealMessagePayload(String fileName, long fileLength) { + public FileSealPayload(final String fileName, final long fileLength) { this.fileName = fileName; this.fileLength = fileLength; } @Override - public void serialize(DataOutputStream stream) throws IOException { + public void serialize(final DataOutputStream stream) throws IOException { ReadWriteIOUtils.write(fileName, stream); ReadWriteIOUtils.write(fileLength, stream); } @Override - public SubscriptionMessagePayload deserialize(ByteBuffer buffer) { + public SubscriptionPollPayload deserialize(final ByteBuffer buffer) { this.fileName = ReadWriteIOUtils.readString(buffer); this.fileLength = ReadWriteIOUtils.readLong(buffer); return this; @@ -68,7 +68,7 @@ public boolean equals(final Object obj) { if (obj == null || getClass() != obj.getClass()) { return false; } - final TsFileSealMessagePayload that = (TsFileSealMessagePayload) obj; + final FileSealPayload that = (FileSealPayload) obj; return Objects.equals(this.fileName, that.fileName) && Objects.equals(this.fileLength, that.fileLength); } @@ -80,6 +80,6 @@ public int hashCode() { @Override public String toString() { - return "TsFileSealMessagePayload{fileName=" + fileName + ", fileLength=" + fileLength + "}"; + return "FileSealPayload{fileName=" + fileName + ", fileLength=" + fileLength + "}"; } } diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/PollTsFileMessagePayload.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/poll/PollFilePayload.java similarity index 82% rename from iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/PollTsFileMessagePayload.java rename to iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/poll/PollFilePayload.java index 402650aaff2e..06128c70237a 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/PollTsFileMessagePayload.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/poll/PollFilePayload.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.iotdb.rpc.subscription.payload.common; +package org.apache.iotdb.rpc.subscription.payload.poll; import org.apache.tsfile.utils.ReadWriteIOUtils; @@ -26,7 +26,7 @@ import java.nio.ByteBuffer; import java.util.Objects; -public class PollTsFileMessagePayload implements SubscriptionMessagePayload { +public class PollFilePayload implements SubscriptionPollPayload { private transient String topicName; @@ -46,23 +46,23 @@ public long getWritingOffset() { return writingOffset; } - public PollTsFileMessagePayload() {} + public PollFilePayload() {} - public PollTsFileMessagePayload(String topicName, String fileName, long writingOffset) { + public PollFilePayload(final String topicName, final String fileName, final long writingOffset) { this.topicName = topicName; this.fileName = fileName; this.writingOffset = writingOffset; } @Override - public void serialize(DataOutputStream stream) throws IOException { + public void serialize(final DataOutputStream stream) throws IOException { ReadWriteIOUtils.write(topicName, stream); ReadWriteIOUtils.write(fileName, stream); ReadWriteIOUtils.write(writingOffset, stream); } @Override - public SubscriptionMessagePayload deserialize(ByteBuffer buffer) { + public SubscriptionPollPayload deserialize(final ByteBuffer buffer) { topicName = ReadWriteIOUtils.readString(buffer); fileName = ReadWriteIOUtils.readString(buffer); writingOffset = ReadWriteIOUtils.readLong(buffer); @@ -79,7 +79,7 @@ public boolean equals(final Object obj) { if (obj == null || getClass() != obj.getClass()) { return false; } - final PollTsFileMessagePayload that = (PollTsFileMessagePayload) obj; + final PollFilePayload that = (PollFilePayload) obj; return Objects.equals(this.topicName, that.topicName) && Objects.equals(this.fileName, that.fileName) && Objects.equals(this.writingOffset, that.writingOffset); @@ -92,7 +92,7 @@ public int hashCode() { @Override public String toString() { - return "PollTsFileMessagePayload{topicName=" + return "PollFilePayload{topicName=" + topicName + ", fileName=" + fileName diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/PollMessagePayload.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/poll/PollPayload.java similarity index 79% rename from iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/PollMessagePayload.java rename to iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/poll/PollPayload.java index 2e4ddb3acfe8..77f6747bac8e 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/PollMessagePayload.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/poll/PollPayload.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.iotdb.rpc.subscription.payload.common; +package org.apache.iotdb.rpc.subscription.payload.poll; import org.apache.tsfile.utils.ReadWriteIOUtils; @@ -28,13 +28,13 @@ import java.util.Objects; import java.util.Set; -public class PollMessagePayload implements SubscriptionMessagePayload { +public class PollPayload implements SubscriptionPollPayload { private transient Set topicNames = new HashSet<>(); - public PollMessagePayload() {} + public PollPayload() {} - public PollMessagePayload(Set topicNames) { + public PollPayload(final Set topicNames) { this.topicNames = topicNames; } @@ -43,12 +43,12 @@ public Set getTopicNames() { } @Override - public void serialize(DataOutputStream stream) throws IOException { + public void serialize(final DataOutputStream stream) throws IOException { ReadWriteIOUtils.writeObjectSet(topicNames, stream); } @Override - public SubscriptionMessagePayload deserialize(ByteBuffer buffer) { + public SubscriptionPollPayload deserialize(final ByteBuffer buffer) { topicNames = ReadWriteIOUtils.readObjectSet(buffer); return this; } @@ -63,7 +63,7 @@ public boolean equals(final Object obj) { if (obj == null || getClass() != obj.getClass()) { return false; } - final PollMessagePayload that = (PollMessagePayload) obj; + final PollPayload that = (PollPayload) obj; return Objects.equals(this.topicNames, that.topicNames); } @@ -74,6 +74,6 @@ public int hashCode() { @Override public String toString() { - return "PollMessagePayload{topicNames=" + topicNames + "}"; + return "PollPayload{topicNames=" + topicNames + "}"; } } diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionCommitContext.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/poll/SubscriptionCommitContext.java similarity index 95% rename from iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionCommitContext.java rename to iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/poll/SubscriptionCommitContext.java index 95016b78b1c8..72292bf66b7c 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionCommitContext.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/poll/SubscriptionCommitContext.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.iotdb.rpc.subscription.payload.common; +package org.apache.iotdb.rpc.subscription.payload.poll; import org.apache.tsfile.utils.PublicBAOS; import org.apache.tsfile.utils.ReadWriteIOUtils; @@ -75,7 +75,8 @@ public long getCommitId() { /////////////////////////////// de/ser /////////////////////////////// - public static ByteBuffer serialize(SubscriptionCommitContext commitContext) throws IOException { + public static ByteBuffer serialize(final SubscriptionCommitContext commitContext) + throws IOException { try (final PublicBAOS byteArrayOutputStream = new PublicBAOS(); final DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream)) { commitContext.serialize(outputStream); @@ -140,7 +141,7 @@ public String toString() { } @Override - public int compareTo(SubscriptionCommitContext that) { + public int compareTo(final SubscriptionCommitContext that) { return Comparator.comparingInt(SubscriptionCommitContext::getDataNodeId) .thenComparingInt(SubscriptionCommitContext::getRebootTimes) .thenComparing(SubscriptionCommitContext::getTopicName) diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionMessagePayload.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/poll/SubscriptionPollPayload.java similarity index 85% rename from iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionMessagePayload.java rename to iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/poll/SubscriptionPollPayload.java index cfcde495fe88..96908fa3ca2c 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionMessagePayload.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/poll/SubscriptionPollPayload.java @@ -17,15 +17,15 @@ * under the License. */ -package org.apache.iotdb.rpc.subscription.payload.common; +package org.apache.iotdb.rpc.subscription.payload.poll; import java.io.DataOutputStream; import java.io.IOException; import java.nio.ByteBuffer; -public interface SubscriptionMessagePayload { +public interface SubscriptionPollPayload { void serialize(final DataOutputStream stream) throws IOException; - SubscriptionMessagePayload deserialize(final ByteBuffer buffer); + SubscriptionPollPayload deserialize(final ByteBuffer buffer); } diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPollMessage.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/poll/SubscriptionPollRequest.java similarity index 55% rename from iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPollMessage.java rename to iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/poll/SubscriptionPollRequest.java index 3ceed2f6450a..fce474fff0d7 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPollMessage.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/poll/SubscriptionPollRequest.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.iotdb.rpc.subscription.payload.common; +package org.apache.iotdb.rpc.subscription.payload.poll; import org.apache.tsfile.utils.PublicBAOS; import org.apache.tsfile.utils.ReadWriteIOUtils; @@ -28,29 +28,29 @@ import java.io.IOException; import java.nio.ByteBuffer; -public class SubscriptionPollMessage { +public class SubscriptionPollRequest { - private static final Logger LOGGER = LoggerFactory.getLogger(SubscriptionPolledMessage.class); + private static final Logger LOGGER = LoggerFactory.getLogger(SubscriptionPollResponse.class); - private final transient short messageType; + private final transient short requestType; - private final transient SubscriptionMessagePayload messagePayload; + private final transient SubscriptionPollPayload payload; private final transient long timeoutMs; // unused now - public SubscriptionPollMessage( - short messageType, SubscriptionMessagePayload messagePayload, long timeoutMs) { - this.messageType = messageType; - this.messagePayload = messagePayload; + public SubscriptionPollRequest( + final short requestType, final SubscriptionPollPayload payload, final long timeoutMs) { + this.requestType = requestType; + this.payload = payload; this.timeoutMs = timeoutMs; } - public short getMessageType() { - return messageType; + public short getRequestType() { + return requestType; } - public SubscriptionMessagePayload getMessagePayload() { - return messagePayload; + public SubscriptionPollPayload getPayload() { + return payload; } public long getTimeoutMs() { @@ -59,51 +59,51 @@ public long getTimeoutMs() { //////////////////////////// serialization //////////////////////////// - public static ByteBuffer serialize(SubscriptionPollMessage message) throws IOException { + public static ByteBuffer serialize(final SubscriptionPollRequest request) throws IOException { try (final PublicBAOS byteArrayOutputStream = new PublicBAOS(); final DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream)) { - message.serialize(outputStream); + request.serialize(outputStream); return ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size()); } } private void serialize(final DataOutputStream stream) throws IOException { - ReadWriteIOUtils.write(messageType, stream); - messagePayload.serialize(stream); + ReadWriteIOUtils.write(requestType, stream); + payload.serialize(stream); ReadWriteIOUtils.write(timeoutMs, stream); } - public static SubscriptionPollMessage deserialize(final ByteBuffer buffer) { - final short messageType = ReadWriteIOUtils.readShort(buffer); - SubscriptionMessagePayload messagePayload = null; - if (SubscriptionPollMessageType.isValidatedMessageType(messageType)) { - switch (SubscriptionPollMessageType.valueOf(messageType)) { + public static SubscriptionPollRequest deserialize(final ByteBuffer buffer) { + final short requestType = ReadWriteIOUtils.readShort(buffer); + SubscriptionPollPayload payload = null; + if (SubscriptionPollRequestType.isValidatedRequestType(requestType)) { + switch (SubscriptionPollRequestType.valueOf(requestType)) { case POLL: - messagePayload = new PollMessagePayload().deserialize(buffer); + payload = new PollPayload().deserialize(buffer); break; - case POLL_TS_FILE: - messagePayload = new PollTsFileMessagePayload().deserialize(buffer); + case POLL_FILE: + payload = new PollFilePayload().deserialize(buffer); break; default: - LOGGER.warn("unexpected message type: {}, message payload will be null", messageType); + LOGGER.warn("unexpected request type: {}, payload will be null", requestType); break; } } else { - LOGGER.warn("unexpected message type: {}, message payload will be null", messageType); + LOGGER.warn("unexpected request type: {}, payload will be null", requestType); } final long timeoutMs = ReadWriteIOUtils.readLong(buffer); - return new SubscriptionPollMessage(messageType, messagePayload, timeoutMs); + return new SubscriptionPollRequest(requestType, payload, timeoutMs); } /////////////////////////////// object /////////////////////////////// @Override public String toString() { - return "SubscriptionPollMessage{messageType=" - + SubscriptionPollMessageType.valueOf(messageType).toString() - + ", messagePayload=" - + messagePayload + return "SubscriptionPollRequest{requestType=" + + SubscriptionPollRequestType.valueOf(requestType).toString() + + ", payload=" + + payload + ", timeoutMs=" + timeoutMs + "}"; diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPollMessageType.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/poll/SubscriptionPollRequestType.java similarity index 74% rename from iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPollMessageType.java rename to iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/poll/SubscriptionPollRequestType.java index bfdefa58e864..c034d4dfbc68 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPollMessageType.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/poll/SubscriptionPollRequestType.java @@ -17,20 +17,20 @@ * under the License. */ -package org.apache.iotdb.rpc.subscription.payload.common; +package org.apache.iotdb.rpc.subscription.payload.poll; import java.util.Arrays; import java.util.HashMap; import java.util.Map; -public enum SubscriptionPollMessageType { +public enum SubscriptionPollRequestType { POLL((short) 0), - POLL_TS_FILE((short) 1), + POLL_FILE((short) 1), ; private final short type; - SubscriptionPollMessageType(short type) { + SubscriptionPollRequestType(final short type) { this.type = type; } @@ -38,18 +38,18 @@ public short getType() { return type; } - private static final Map TYPE_MAP = - Arrays.stream(SubscriptionPollMessageType.values()) + private static final Map TYPE_MAP = + Arrays.stream(SubscriptionPollRequestType.values()) .collect( HashMap::new, (typeMap, messageType) -> typeMap.put(messageType.getType(), messageType), HashMap::putAll); - public static boolean isValidatedMessageType(short type) { + public static boolean isValidatedRequestType(final short type) { return TYPE_MAP.containsKey(type); } - public static SubscriptionPollMessageType valueOf(short type) { + public static SubscriptionPollRequestType valueOf(final short type) { return TYPE_MAP.get(type); } } diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledMessage.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/poll/SubscriptionPollResponse.java similarity index 51% rename from iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledMessage.java rename to iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/poll/SubscriptionPollResponse.java index 03c4ddf1afe9..d386903f124d 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledMessage.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/poll/SubscriptionPollResponse.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.iotdb.rpc.subscription.payload.common; +package org.apache.iotdb.rpc.subscription.payload.poll; import org.apache.tsfile.utils.PublicBAOS; import org.apache.tsfile.utils.ReadWriteIOUtils; @@ -28,31 +28,31 @@ import java.io.IOException; import java.nio.ByteBuffer; -public class SubscriptionPolledMessage { +public class SubscriptionPollResponse { - private static final Logger LOGGER = LoggerFactory.getLogger(SubscriptionPolledMessage.class); + private static final Logger LOGGER = LoggerFactory.getLogger(SubscriptionPollResponse.class); - private final transient short messageType; + private final transient short responseType; - private final transient SubscriptionMessagePayload messagePayload; + private final transient SubscriptionPollPayload payload; private final transient SubscriptionCommitContext commitContext; - public SubscriptionPolledMessage( - short messageType, - SubscriptionMessagePayload messagePayload, - SubscriptionCommitContext commitContext) { - this.messageType = messageType; - this.messagePayload = messagePayload; + public SubscriptionPollResponse( + final short responseType, + final SubscriptionPollPayload payload, + final SubscriptionCommitContext commitContext) { + this.responseType = responseType; + this.payload = payload; this.commitContext = commitContext; } - public short getMessageType() { - return messageType; + public short getResponseType() { + return responseType; } - public SubscriptionMessagePayload getMessagePayload() { - return messagePayload; + public SubscriptionPollPayload getPayload() { + return payload; } public SubscriptionCommitContext getCommitContext() { @@ -61,60 +61,60 @@ public SubscriptionCommitContext getCommitContext() { /////////////////////////////// de/ser /////////////////////////////// - public static ByteBuffer serialize(SubscriptionPolledMessage message) throws IOException { + public static ByteBuffer serialize(final SubscriptionPollResponse response) throws IOException { try (final PublicBAOS byteArrayOutputStream = new PublicBAOS(); final DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream)) { - message.serialize(outputStream); + response.serialize(outputStream); return ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size()); } } private void serialize(final DataOutputStream stream) throws IOException { - ReadWriteIOUtils.write(messageType, stream); - messagePayload.serialize(stream); + ReadWriteIOUtils.write(responseType, stream); + payload.serialize(stream); commitContext.serialize(stream); } - public static SubscriptionPolledMessage deserialize(final ByteBuffer buffer) { - final short messageType = ReadWriteIOUtils.readShort(buffer); - SubscriptionMessagePayload messagePayload = null; - if (SubscriptionPolledMessageType.isValidatedMessageType(messageType)) { - switch (SubscriptionPolledMessageType.valueOf(messageType)) { + public static SubscriptionPollResponse deserialize(final ByteBuffer buffer) { + final short responseType = ReadWriteIOUtils.readShort(buffer); + SubscriptionPollPayload payload = null; + if (SubscriptionPollResponseType.isValidatedResponseType(responseType)) { + switch (SubscriptionPollResponseType.valueOf(responseType)) { case TABLETS: - messagePayload = new TabletsMessagePayload().deserialize(buffer); + payload = new TabletsPayload().deserialize(buffer); break; - case TS_FILE_INIT: - messagePayload = new TsFileInitMessagePayload().deserialize(buffer); + case FILE_INIT: + payload = new FileInitPayload().deserialize(buffer); break; - case TS_FILE_PIECE: - messagePayload = new TsFilePieceMessagePayload().deserialize(buffer); + case FILE_PIECE: + payload = new FilePiecePayload().deserialize(buffer); break; - case TS_FILE_SEAL: - messagePayload = new TsFileSealMessagePayload().deserialize(buffer); + case FILE_SEAL: + payload = new FileSealPayload().deserialize(buffer); break; - case TS_FILE_ERROR: - messagePayload = new TsFileErrorMessagePayload().deserialize(buffer); + case ERROR: + payload = new ErrorPayload().deserialize(buffer); break; default: - LOGGER.warn("unexpected message type: {}, message payload will be null", messageType); + LOGGER.warn("unexpected response type: {}, payload will be null", responseType); break; } } else { - LOGGER.warn("unexpected message type: {}, message payload will be null", messageType); + LOGGER.warn("unexpected response type: {}, payload will be null", responseType); } final SubscriptionCommitContext commitContext = SubscriptionCommitContext.deserialize(buffer); - return new SubscriptionPolledMessage(messageType, messagePayload, commitContext); + return new SubscriptionPollResponse(responseType, payload, commitContext); } /////////////////////////////// object /////////////////////////////// @Override public String toString() { - return "SubscriptionPolledMessage{messageType=" - + SubscriptionPolledMessageType.valueOf(messageType).toString() - + ", messagePayload=" - + messagePayload + return "SubscriptionPollResponse{responseType=" + + SubscriptionPollResponseType.valueOf(responseType).toString() + + ", payload=" + + payload + ", commitContext=" + commitContext + "}"; diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledMessageType.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/poll/SubscriptionPollResponseType.java similarity index 69% rename from iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledMessageType.java rename to iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/poll/SubscriptionPollResponseType.java index 3abc2d4c1ae2..a22c9590c65f 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/SubscriptionPolledMessageType.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/poll/SubscriptionPollResponseType.java @@ -17,24 +17,25 @@ * under the License. */ -package org.apache.iotdb.rpc.subscription.payload.common; +package org.apache.iotdb.rpc.subscription.payload.poll; import java.util.Arrays; import java.util.HashMap; import java.util.Map; -public enum SubscriptionPolledMessageType { - TABLETS((short) 0), +public enum SubscriptionPollResponseType { + ERROR((short) 0), - TS_FILE_INIT((short) 1), - TS_FILE_PIECE((short) 2), - TS_FILE_SEAL((short) 3), - TS_FILE_ERROR((short) 4), + TABLETS((short) 1), + + FILE_INIT((short) 2), + FILE_PIECE((short) 3), + FILE_SEAL((short) 4), ; private final short type; - SubscriptionPolledMessageType(short type) { + SubscriptionPollResponseType(final short type) { this.type = type; } @@ -42,18 +43,18 @@ public short getType() { return type; } - private static final Map TYPE_MAP = - Arrays.stream(SubscriptionPolledMessageType.values()) + private static final Map TYPE_MAP = + Arrays.stream(SubscriptionPollResponseType.values()) .collect( HashMap::new, (typeMap, messageType) -> typeMap.put(messageType.getType(), messageType), HashMap::putAll); - public static boolean isValidatedMessageType(short type) { + public static boolean isValidatedResponseType(final short type) { return TYPE_MAP.containsKey(type); } - public static SubscriptionPolledMessageType valueOf(short type) { + public static SubscriptionPollResponseType valueOf(final short type) { return TYPE_MAP.get(type); } } diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TabletsMessagePayload.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/poll/TabletsPayload.java similarity index 80% rename from iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TabletsMessagePayload.java rename to iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/poll/TabletsPayload.java index 7d80855c14ff..c0087fd36797 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/common/TabletsMessagePayload.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/poll/TabletsPayload.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.iotdb.rpc.subscription.payload.common; +package org.apache.iotdb.rpc.subscription.payload.poll; import org.apache.tsfile.utils.ReadWriteIOUtils; import org.apache.tsfile.write.record.Tablet; @@ -29,17 +29,16 @@ import java.util.List; import java.util.Objects; -public class TabletsMessagePayload implements SubscriptionMessagePayload { +public class TabletsPayload implements SubscriptionPollPayload { protected transient List tablets = new ArrayList<>(); // Pure in-memory object, not involved in serialization and deserialization. protected transient long calculatedTabletsSizeInBytes; - public TabletsMessagePayload() {} + public TabletsPayload() {} - public TabletsMessagePayload( - final List tablets, final long calculatedTabletsSizeInBytes) { + public TabletsPayload(final List tablets, final long calculatedTabletsSizeInBytes) { this.tablets = tablets; this.calculatedTabletsSizeInBytes = calculatedTabletsSizeInBytes; } @@ -49,7 +48,7 @@ public List getTablets() { } @Override - public void serialize(DataOutputStream stream) throws IOException { + public void serialize(final DataOutputStream stream) throws IOException { ReadWriteIOUtils.write(tablets.size(), stream); for (final Tablet tablet : tablets) { tablet.serialize(stream); @@ -57,7 +56,7 @@ public void serialize(DataOutputStream stream) throws IOException { } @Override - public SubscriptionMessagePayload deserialize(ByteBuffer buffer) { + public SubscriptionPollPayload deserialize(final ByteBuffer buffer) { final List tablets = new ArrayList<>(); final int size = ReadWriteIOUtils.readInt(buffer); for (int i = 0; i < size; ++i) { @@ -75,7 +74,7 @@ public boolean equals(final Object obj) { if (obj == null || getClass() != obj.getClass()) { return false; } - final TabletsMessagePayload that = (TabletsMessagePayload) obj; + final TabletsPayload that = (TabletsPayload) obj; return Objects.equals(this.tablets, that.tablets); } @@ -86,7 +85,7 @@ public int hashCode() { @Override public String toString() { - return "TabletsMessagePayload{calculatedTabletsSizeInBytes=" + return "TabletsPayload{calculatedTabletsSizeInBytes=" + (calculatedTabletsSizeInBytes == 0 ? "" : calculatedTabletsSizeInBytes) + "}"; } diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribeCloseReq.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribeCloseReq.java index 724f0248943d..d1eb1fa874eb 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribeCloseReq.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribeCloseReq.java @@ -41,7 +41,7 @@ public static PipeSubscribeCloseReq toTPipeSubscribeReq() { } /** Deserialize `TPipeSubscribeReq` to obtain parameters, called by the subscription server. */ - public static PipeSubscribeCloseReq fromTPipeSubscribeReq(TPipeSubscribeReq closeReq) { + public static PipeSubscribeCloseReq fromTPipeSubscribeReq(final TPipeSubscribeReq closeReq) { final PipeSubscribeCloseReq req = new PipeSubscribeCloseReq(); req.version = closeReq.version; @@ -54,14 +54,14 @@ public static PipeSubscribeCloseReq fromTPipeSubscribeReq(TPipeSubscribeReq clos /////////////////////////////// Object /////////////////////////////// @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; } - PipeSubscribeCloseReq that = (PipeSubscribeCloseReq) obj; + final PipeSubscribeCloseReq that = (PipeSubscribeCloseReq) obj; return this.version == that.version && this.type == that.type && Objects.equals(this.body, that.body); diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribeCommitReq.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribeCommitReq.java index d7fa43e769fe..f11c47d01a02 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribeCommitReq.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribeCommitReq.java @@ -19,7 +19,7 @@ package org.apache.iotdb.rpc.subscription.payload.request; -import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionCommitContext; +import org.apache.iotdb.rpc.subscription.payload.poll.SubscriptionCommitContext; import org.apache.iotdb.service.rpc.thrift.TPipeSubscribeReq; import org.apache.tsfile.utils.PublicBAOS; @@ -53,7 +53,7 @@ public boolean isNack() { * client. */ public static PipeSubscribeCommitReq toTPipeSubscribeReq( - List commitContexts, boolean nack) throws IOException { + final List commitContexts, final boolean nack) throws IOException { final PipeSubscribeCommitReq req = new PipeSubscribeCommitReq(); req.commitContexts = commitContexts; @@ -75,11 +75,11 @@ public static PipeSubscribeCommitReq toTPipeSubscribeReq( } /** Deserialize `TPipeSubscribeReq` to obtain parameters, called by the subscription server. */ - public static PipeSubscribeCommitReq fromTPipeSubscribeReq(TPipeSubscribeReq commitReq) { + public static PipeSubscribeCommitReq fromTPipeSubscribeReq(final TPipeSubscribeReq commitReq) { final PipeSubscribeCommitReq req = new PipeSubscribeCommitReq(); if (Objects.nonNull(commitReq.body) && commitReq.body.hasRemaining()) { - int size = ReadWriteIOUtils.readInt(commitReq.body); + final int size = ReadWriteIOUtils.readInt(commitReq.body); for (int i = 0; i < size; ++i) { req.commitContexts.add(SubscriptionCommitContext.deserialize(commitReq.body)); } @@ -96,14 +96,14 @@ public static PipeSubscribeCommitReq fromTPipeSubscribeReq(TPipeSubscribeReq com /////////////////////////////// Object /////////////////////////////// @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; } - PipeSubscribeCommitReq that = (PipeSubscribeCommitReq) obj; + final PipeSubscribeCommitReq that = (PipeSubscribeCommitReq) obj; return Objects.equals(this.commitContexts, that.commitContexts) && Objects.equals(this.nack, that.nack) && this.version == that.version diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribeHandshakeReq.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribeHandshakeReq.java index 55c236bd9d88..afcac878dc1f 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribeHandshakeReq.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribeHandshakeReq.java @@ -39,7 +39,7 @@ public ConsumerConfig getConsumerConfig() { * Serialize the incoming parameters into `PipeSubscribeHandshakeReq`, called by the subscription * client. */ - public static PipeSubscribeHandshakeReq toTPipeSubscribeReq(ConsumerConfig consumerConfig) + public static PipeSubscribeHandshakeReq toTPipeSubscribeReq(final ConsumerConfig consumerConfig) throws IOException { final PipeSubscribeHandshakeReq req = new PipeSubscribeHandshakeReq(); @@ -53,7 +53,8 @@ public static PipeSubscribeHandshakeReq toTPipeSubscribeReq(ConsumerConfig consu } /** Deserialize `TPipeSubscribeReq` to obtain parameters, called by the subscription server. */ - public static PipeSubscribeHandshakeReq fromTPipeSubscribeReq(TPipeSubscribeReq handshakeReq) { + public static PipeSubscribeHandshakeReq fromTPipeSubscribeReq( + final TPipeSubscribeReq handshakeReq) { final PipeSubscribeHandshakeReq req = new PipeSubscribeHandshakeReq(); if (Objects.nonNull(handshakeReq.body) && handshakeReq.body.hasRemaining()) { @@ -70,14 +71,14 @@ public static PipeSubscribeHandshakeReq fromTPipeSubscribeReq(TPipeSubscribeReq /////////////////////////////// Object /////////////////////////////// @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; } - PipeSubscribeHandshakeReq that = (PipeSubscribeHandshakeReq) obj; + final PipeSubscribeHandshakeReq that = (PipeSubscribeHandshakeReq) obj; return Objects.equals(this.consumerConfig, that.consumerConfig) && this.version == that.version && this.type == that.type diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribeHeartbeatReq.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribeHeartbeatReq.java index e4945bcca9d9..c278df9786a2 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribeHeartbeatReq.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribeHeartbeatReq.java @@ -41,7 +41,8 @@ public static PipeSubscribeHeartbeatReq toTPipeSubscribeReq() { } /** Deserialize `TPipeSubscribeReq` to obtain parameters, called by the subscription server. */ - public static PipeSubscribeHeartbeatReq fromTPipeSubscribeReq(TPipeSubscribeReq heartbeatReq) { + public static PipeSubscribeHeartbeatReq fromTPipeSubscribeReq( + final TPipeSubscribeReq heartbeatReq) { final PipeSubscribeHeartbeatReq req = new PipeSubscribeHeartbeatReq(); req.version = heartbeatReq.version; @@ -54,14 +55,14 @@ public static PipeSubscribeHeartbeatReq fromTPipeSubscribeReq(TPipeSubscribeReq /////////////////////////////// Object /////////////////////////////// @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; } - PipeSubscribeHeartbeatReq that = (PipeSubscribeHeartbeatReq) obj; + final PipeSubscribeHeartbeatReq that = (PipeSubscribeHeartbeatReq) obj; return this.version == that.version && this.type == that.type && Objects.equals(this.body, that.body); diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribePollReq.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribePollReq.java index 696bd1a26b9e..6ed89311f300 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribePollReq.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribePollReq.java @@ -19,7 +19,7 @@ package org.apache.iotdb.rpc.subscription.payload.request; -import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPollMessage; +import org.apache.iotdb.rpc.subscription.payload.poll.SubscriptionPollRequest; import org.apache.iotdb.service.rpc.thrift.TPipeSubscribeReq; import java.io.IOException; @@ -27,10 +27,10 @@ public class PipeSubscribePollReq extends TPipeSubscribeReq { - private transient SubscriptionPollMessage pollMessage; + private transient SubscriptionPollRequest request; - public SubscriptionPollMessage getPollMessage() { - return pollMessage; + public SubscriptionPollRequest getRequest() { + return request; } /////////////////////////////// Thrift /////////////////////////////// @@ -39,25 +39,25 @@ public SubscriptionPollMessage getPollMessage() { * Serialize the incoming parameters into `PipeSubscribePollReq`, called by the subscription * client. */ - public static PipeSubscribePollReq toTPipeSubscribeReq(SubscriptionPollMessage pollMessage) + public static PipeSubscribePollReq toTPipeSubscribeReq(final SubscriptionPollRequest request) throws IOException { final PipeSubscribePollReq req = new PipeSubscribePollReq(); - req.pollMessage = pollMessage; + req.request = request; req.version = PipeSubscribeRequestVersion.VERSION_1.getVersion(); req.type = PipeSubscribeRequestType.POLL.getType(); - req.body = SubscriptionPollMessage.serialize(pollMessage); + req.body = SubscriptionPollRequest.serialize(request); return req; } /** Deserialize `TPipeSubscribeReq` to obtain parameters, called by the subscription server. */ - public static PipeSubscribePollReq fromTPipeSubscribeReq(TPipeSubscribeReq pollReq) { + public static PipeSubscribePollReq fromTPipeSubscribeReq(final TPipeSubscribeReq pollReq) { final PipeSubscribePollReq req = new PipeSubscribePollReq(); if (Objects.nonNull(pollReq.body) && pollReq.body.hasRemaining()) { - req.pollMessage = SubscriptionPollMessage.deserialize(pollReq.body); + req.request = SubscriptionPollRequest.deserialize(pollReq.body); } req.version = pollReq.version; @@ -70,15 +70,15 @@ public static PipeSubscribePollReq fromTPipeSubscribeReq(TPipeSubscribeReq pollR /////////////////////////////// Object /////////////////////////////// @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; } - PipeSubscribePollReq that = (PipeSubscribePollReq) obj; - return Objects.equals(this.pollMessage, that.pollMessage) + final PipeSubscribePollReq that = (PipeSubscribePollReq) obj; + return Objects.equals(this.request, that.request) && this.version == that.version && this.type == that.type && Objects.equals(this.body, that.body); @@ -86,6 +86,6 @@ public boolean equals(Object obj) { @Override public int hashCode() { - return Objects.hash(pollMessage, pollMessage, version, type, body); + return Objects.hash(request, request, version, type, body); } } diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribeRequestType.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribeRequestType.java index 037481127a14..d649aa567ade 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribeRequestType.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribeRequestType.java @@ -35,7 +35,7 @@ public enum PipeSubscribeRequestType { private final short type; - PipeSubscribeRequestType(short type) { + PipeSubscribeRequestType(final short type) { this.type = type; } @@ -50,11 +50,11 @@ public short getType() { (typeMap, requestType) -> typeMap.put(requestType.getType(), requestType), HashMap::putAll); - public static boolean isValidatedRequestType(short type) { + public static boolean isValidatedRequestType(final short type) { return TYPE_MAP.containsKey(type); } - public static PipeSubscribeRequestType valueOf(short type) { + public static PipeSubscribeRequestType valueOf(final short type) { return TYPE_MAP.get(type); } } diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribeRequestVersion.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribeRequestVersion.java index 8eb58db04a1a..cf60d045a052 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribeRequestVersion.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribeRequestVersion.java @@ -25,7 +25,7 @@ public enum PipeSubscribeRequestVersion { private final byte version; - PipeSubscribeRequestVersion(byte type) { + PipeSubscribeRequestVersion(final byte type) { this.version = type; } diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribeSubscribeReq.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribeSubscribeReq.java index eb3b90ec12db..e8892acd58f2 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribeSubscribeReq.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribeSubscribeReq.java @@ -45,7 +45,7 @@ public Set getTopicNames() { * Serialize the incoming parameters into `PipeSubscribeSubscribeReq`, called by the subscription * client. */ - public static PipeSubscribeSubscribeReq toTPipeSubscribeReq(Set topicNames) + public static PipeSubscribeSubscribeReq toTPipeSubscribeReq(final Set topicNames) throws IOException { final PipeSubscribeSubscribeReq req = new PipeSubscribeSubscribeReq(); @@ -63,7 +63,8 @@ public static PipeSubscribeSubscribeReq toTPipeSubscribeReq(Set topicNam } /** Deserialize `TPipeSubscribeReq` to obtain parameters, called by the subscription server. */ - public static PipeSubscribeSubscribeReq fromTPipeSubscribeReq(TPipeSubscribeReq subscribeReq) { + public static PipeSubscribeSubscribeReq fromTPipeSubscribeReq( + final TPipeSubscribeReq subscribeReq) { final PipeSubscribeSubscribeReq req = new PipeSubscribeSubscribeReq(); if (Objects.nonNull(subscribeReq.body) && subscribeReq.body.hasRemaining()) { @@ -80,14 +81,14 @@ public static PipeSubscribeSubscribeReq fromTPipeSubscribeReq(TPipeSubscribeReq /////////////////////////////// Object /////////////////////////////// @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; } - PipeSubscribeSubscribeReq that = (PipeSubscribeSubscribeReq) obj; + final PipeSubscribeSubscribeReq that = (PipeSubscribeSubscribeReq) obj; return Objects.equals(this.topicNames, that.topicNames) && this.version == that.version && this.type == that.type diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribeUnsubscribeReq.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribeUnsubscribeReq.java index 2de93a56b757..edb4f6678f81 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribeUnsubscribeReq.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/request/PipeSubscribeUnsubscribeReq.java @@ -45,7 +45,7 @@ public Set getTopicNames() { * Serialize the incoming parameters into `PipeSubscribeUnsubscribeReq`, called by the * subscription client. */ - public static PipeSubscribeUnsubscribeReq toTPipeSubscribeReq(Set topicNames) + public static PipeSubscribeUnsubscribeReq toTPipeSubscribeReq(final Set topicNames) throws IOException { final PipeSubscribeUnsubscribeReq req = new PipeSubscribeUnsubscribeReq(); @@ -64,7 +64,7 @@ public static PipeSubscribeUnsubscribeReq toTPipeSubscribeReq(Set topicN /** Deserialize `TPipeSubscribeReq` to obtain parameters, called by the subscription server. */ public static PipeSubscribeUnsubscribeReq fromTPipeSubscribeReq( - TPipeSubscribeReq unsubscribeReq) { + final TPipeSubscribeReq unsubscribeReq) { final PipeSubscribeUnsubscribeReq req = new PipeSubscribeUnsubscribeReq(); if (Objects.nonNull(unsubscribeReq.body) && unsubscribeReq.body.hasRemaining()) { @@ -81,14 +81,14 @@ public static PipeSubscribeUnsubscribeReq fromTPipeSubscribeReq( /////////////////////////////// Object /////////////////////////////// @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; } - PipeSubscribeUnsubscribeReq that = (PipeSubscribeUnsubscribeReq) obj; + final PipeSubscribeUnsubscribeReq that = (PipeSubscribeUnsubscribeReq) obj; return Objects.equals(this.topicNames, that.topicNames) && this.version == that.version && this.type == that.type diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribeCloseResp.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribeCloseResp.java index 1fdbcbc9a69d..9998d63750ae 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribeCloseResp.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribeCloseResp.java @@ -32,7 +32,7 @@ public class PipeSubscribeCloseResp extends TPipeSubscribeResp { * Serialize the incoming parameters into `PipeSubscribeCloseResp`, called by the subscription * server. */ - public static PipeSubscribeCloseResp toTPipeSubscribeResp(TSStatus status) { + public static PipeSubscribeCloseResp toTPipeSubscribeResp(final TSStatus status) { final PipeSubscribeCloseResp resp = new PipeSubscribeCloseResp(); resp.status = status; @@ -43,7 +43,7 @@ public static PipeSubscribeCloseResp toTPipeSubscribeResp(TSStatus status) { } /** Deserialize `TPipeSubscribeResp` to obtain parameters, called by the subscription client. */ - public static PipeSubscribeCloseResp fromTPipeSubscribeResp(TPipeSubscribeResp closeResp) { + public static PipeSubscribeCloseResp fromTPipeSubscribeResp(final TPipeSubscribeResp closeResp) { final PipeSubscribeCloseResp resp = new PipeSubscribeCloseResp(); resp.status = closeResp.status; @@ -57,14 +57,14 @@ public static PipeSubscribeCloseResp fromTPipeSubscribeResp(TPipeSubscribeResp c /////////////////////////////// Object /////////////////////////////// @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; } - PipeSubscribeCloseResp that = (PipeSubscribeCloseResp) obj; + final PipeSubscribeCloseResp that = (PipeSubscribeCloseResp) obj; return Objects.equals(this.status, that.status) && this.version == that.version && this.type == that.type diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribeCommitResp.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribeCommitResp.java index 59407433370f..9478c319a8dd 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribeCommitResp.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribeCommitResp.java @@ -32,7 +32,7 @@ public class PipeSubscribeCommitResp extends TPipeSubscribeResp { * Serialize the incoming parameters into `PipeSubscribeCommitResp`, called by the subscription * server. */ - public static PipeSubscribeCommitResp toTPipeSubscribeResp(TSStatus status) { + public static PipeSubscribeCommitResp toTPipeSubscribeResp(final TSStatus status) { final PipeSubscribeCommitResp resp = new PipeSubscribeCommitResp(); resp.status = status; @@ -43,7 +43,8 @@ public static PipeSubscribeCommitResp toTPipeSubscribeResp(TSStatus status) { } /** Deserialize `TPipeSubscribeResp` to obtain parameters, called by the subscription client. */ - public static PipeSubscribeCommitResp fromTPipeSubscribeResp(TPipeSubscribeResp commitResp) { + public static PipeSubscribeCommitResp fromTPipeSubscribeResp( + final TPipeSubscribeResp commitResp) { final PipeSubscribeCommitResp resp = new PipeSubscribeCommitResp(); resp.status = commitResp.status; @@ -57,14 +58,14 @@ public static PipeSubscribeCommitResp fromTPipeSubscribeResp(TPipeSubscribeResp /////////////////////////////// Object /////////////////////////////// @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; } - PipeSubscribeCommitResp that = (PipeSubscribeCommitResp) obj; + final PipeSubscribeCommitResp that = (PipeSubscribeCommitResp) obj; return Objects.equals(this.status, that.status) && this.version == that.version && this.type == that.type diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribeHandshakeResp.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribeHandshakeResp.java index d069dd2b4665..4d72fd3d9afe 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribeHandshakeResp.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribeHandshakeResp.java @@ -60,7 +60,10 @@ public String getConsumerGroupId() { * server. */ public static PipeSubscribeHandshakeResp toTPipeSubscribeResp( - TSStatus status, int dataNodeId, String consumerId, String consumerGroupId) { + final TSStatus status, + final int dataNodeId, + final String consumerId, + final String consumerGroupId) { final PipeSubscribeHandshakeResp resp = new PipeSubscribeHandshakeResp(); resp.dataNodeId = dataNodeId; @@ -79,7 +82,7 @@ public static PipeSubscribeHandshakeResp toTPipeSubscribeResp( resp.body = Collections.singletonList( ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size())); - } catch (IOException e) { + } catch (final IOException e) { resp.status = RpcUtils.getStatus(TSStatusCode.SUBSCRIPTION_HANDSHAKE_ERROR, e.getMessage()); return resp; } @@ -89,7 +92,7 @@ public static PipeSubscribeHandshakeResp toTPipeSubscribeResp( /** Deserialize `TPipeSubscribeResp` to obtain parameters, called by the subscription client. */ public static PipeSubscribeHandshakeResp fromTPipeSubscribeResp( - TPipeSubscribeResp handshakeResp) { + final TPipeSubscribeResp handshakeResp) { final PipeSubscribeHandshakeResp resp = new PipeSubscribeHandshakeResp(); if (Objects.nonNull(handshakeResp.body)) { @@ -114,14 +117,14 @@ public static PipeSubscribeHandshakeResp fromTPipeSubscribeResp( /////////////////////////////// Object /////////////////////////////// @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; } - PipeSubscribeHandshakeResp that = (PipeSubscribeHandshakeResp) obj; + final PipeSubscribeHandshakeResp that = (PipeSubscribeHandshakeResp) obj; return Objects.equals(this.dataNodeId, that.dataNodeId) && Objects.equals(this.status, that.status) && this.version == that.version diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribeHeartbeatResp.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribeHeartbeatResp.java index 42836a3a5a8d..0b0f61b91583 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribeHeartbeatResp.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribeHeartbeatResp.java @@ -32,7 +32,7 @@ public class PipeSubscribeHeartbeatResp extends TPipeSubscribeResp { * Serialize the incoming parameters into `PipeSubscribeHeartbeatResp`, called by the subscription * server. */ - public static PipeSubscribeHeartbeatResp toTPipeSubscribeResp(TSStatus status) { + public static PipeSubscribeHeartbeatResp toTPipeSubscribeResp(final TSStatus status) { final PipeSubscribeHeartbeatResp resp = new PipeSubscribeHeartbeatResp(); resp.status = status; @@ -44,7 +44,7 @@ public static PipeSubscribeHeartbeatResp toTPipeSubscribeResp(TSStatus status) { /** Deserialize `TPipeSubscribeResp` to obtain parameters, called by the subscription client. */ public static PipeSubscribeHeartbeatResp fromTPipeSubscribeResp( - TPipeSubscribeResp heartbeatResp) { + final TPipeSubscribeResp heartbeatResp) { final PipeSubscribeHeartbeatResp resp = new PipeSubscribeHeartbeatResp(); resp.status = heartbeatResp.status; @@ -58,14 +58,14 @@ public static PipeSubscribeHeartbeatResp fromTPipeSubscribeResp( /////////////////////////////// Object /////////////////////////////// @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; } - PipeSubscribeHeartbeatResp that = (PipeSubscribeHeartbeatResp) obj; + final PipeSubscribeHeartbeatResp that = (PipeSubscribeHeartbeatResp) obj; return Objects.equals(this.status, that.status) && this.version == that.version && this.type == that.type diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollResp.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollResp.java index 6692d41aa90b..f9fb8a076d82 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollResp.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribePollResp.java @@ -20,7 +20,7 @@ package org.apache.iotdb.rpc.subscription.payload.response; import org.apache.iotdb.common.rpc.thrift.TSStatus; -import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPolledMessage; +import org.apache.iotdb.rpc.subscription.payload.poll.SubscriptionPollResponse; import org.apache.iotdb.service.rpc.thrift.TPipeSubscribeResp; import java.nio.ByteBuffer; @@ -30,10 +30,10 @@ public class PipeSubscribePollResp extends TPipeSubscribeResp { - private transient List messages = new ArrayList<>(); + private final transient List responses = new ArrayList<>(); - public List getMessages() { - return messages; + public List getResponses() { + return responses; } /////////////////////////////// Thrift /////////////////////////////// @@ -43,10 +43,10 @@ public List getMessages() { * server. */ public static PipeSubscribePollResp toTPipeSubscribeResp( - TSStatus status, List byteBuffers) { + final TSStatus status, final List byteBuffers) { final PipeSubscribePollResp resp = new PipeSubscribePollResp(); - // resp.messages = messages; + // resp.events = events; resp.status = status; resp.version = PipeSubscribeResponseVersion.VERSION_1.getVersion(); @@ -57,13 +57,13 @@ public static PipeSubscribePollResp toTPipeSubscribeResp( } /** Deserialize `TPipeSubscribeResp` to obtain parameters, called by the subscription client. */ - public static PipeSubscribePollResp fromTPipeSubscribeResp(TPipeSubscribeResp pollResp) { + public static PipeSubscribePollResp fromTPipeSubscribeResp(final TPipeSubscribeResp pollResp) { final PipeSubscribePollResp resp = new PipeSubscribePollResp(); if (Objects.nonNull(pollResp.body)) { for (final ByteBuffer byteBuffer : pollResp.body) { if (Objects.nonNull(byteBuffer) && byteBuffer.hasRemaining()) { - resp.messages.add(SubscriptionPolledMessage.deserialize(byteBuffer)); + resp.responses.add(SubscriptionPollResponse.deserialize(byteBuffer)); } } } @@ -79,15 +79,15 @@ public static PipeSubscribePollResp fromTPipeSubscribeResp(TPipeSubscribeResp po /////////////////////////////// Object /////////////////////////////// @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; } - PipeSubscribePollResp that = (PipeSubscribePollResp) obj; - return Objects.equals(this.messages, that.messages) + final PipeSubscribePollResp that = (PipeSubscribePollResp) obj; + return Objects.equals(this.responses, that.responses) && Objects.equals(this.status, that.status) && this.version == that.version && this.type == that.type @@ -96,6 +96,6 @@ public boolean equals(Object obj) { @Override public int hashCode() { - return Objects.hash(messages, status, version, type, body); + return Objects.hash(responses, status, version, type, body); } } diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribeResponseType.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribeResponseType.java index 6b9f4bfe9713..981b3f789b5e 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribeResponseType.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribeResponseType.java @@ -29,7 +29,7 @@ public enum PipeSubscribeResponseType { private final short type; - PipeSubscribeResponseType(short type) { + PipeSubscribeResponseType(final short type) { this.type = type; } @@ -44,11 +44,11 @@ public short getType() { (typeMap, responseType) -> typeMap.put(responseType.getType(), responseType), HashMap::putAll); - public static boolean isValidatedResponseType(short type) { + public static boolean isValidatedResponseType(final short type) { return TYPE_MAP.containsKey(type); } - public static PipeSubscribeResponseType valueOf(short type) { + public static PipeSubscribeResponseType valueOf(final short type) { return TYPE_MAP.get(type); } } diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribeResponseVersion.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribeResponseVersion.java index d2eeacd6e346..7348b0be2ff1 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribeResponseVersion.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribeResponseVersion.java @@ -25,7 +25,7 @@ public enum PipeSubscribeResponseVersion { private final byte version; - PipeSubscribeResponseVersion(byte type) { + PipeSubscribeResponseVersion(final byte type) { this.version = type; } diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribeSubscribeResp.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribeSubscribeResp.java index 643377419074..a5540150898d 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribeSubscribeResp.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribeSubscribeResp.java @@ -32,7 +32,7 @@ public class PipeSubscribeSubscribeResp extends TPipeSubscribeResp { * Serialize the incoming parameters into `PipeSubscribeSubscribeResp`, called by the subscription * server. */ - public static PipeSubscribeSubscribeResp toTPipeSubscribeResp(TSStatus status) { + public static PipeSubscribeSubscribeResp toTPipeSubscribeResp(final TSStatus status) { final PipeSubscribeSubscribeResp resp = new PipeSubscribeSubscribeResp(); resp.status = status; @@ -44,7 +44,7 @@ public static PipeSubscribeSubscribeResp toTPipeSubscribeResp(TSStatus status) { /** Deserialize `TPipeSubscribeResp` to obtain parameters, called by the subscription client. */ public static PipeSubscribeSubscribeResp fromTPipeSubscribeResp( - TPipeSubscribeResp subscribeResp) { + final TPipeSubscribeResp subscribeResp) { final PipeSubscribeSubscribeResp resp = new PipeSubscribeSubscribeResp(); resp.status = subscribeResp.status; @@ -58,14 +58,14 @@ public static PipeSubscribeSubscribeResp fromTPipeSubscribeResp( /////////////////////////////// Object /////////////////////////////// @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; } - PipeSubscribeSubscribeResp that = (PipeSubscribeSubscribeResp) obj; + final PipeSubscribeSubscribeResp that = (PipeSubscribeSubscribeResp) obj; return Objects.equals(this.status, that.status) && this.version == that.version && this.type == that.type diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribeUnsubscribeResp.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribeUnsubscribeResp.java index 6594a1a79195..a93e38013acc 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribeUnsubscribeResp.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/payload/response/PipeSubscribeUnsubscribeResp.java @@ -32,7 +32,7 @@ public class PipeSubscribeUnsubscribeResp extends TPipeSubscribeResp { * Serialize the incoming parameters into `PipeSubscribeUnsubscribeResp`, called by the * subscription server. */ - public static PipeSubscribeUnsubscribeResp toTPipeSubscribeResp(TSStatus status) { + public static PipeSubscribeUnsubscribeResp toTPipeSubscribeResp(final TSStatus status) { final PipeSubscribeUnsubscribeResp resp = new PipeSubscribeUnsubscribeResp(); resp.status = status; @@ -44,7 +44,7 @@ public static PipeSubscribeUnsubscribeResp toTPipeSubscribeResp(TSStatus status) /** Deserialize `TPipeSubscribeResp` to obtain parameters, called by the subscription client. */ public static PipeSubscribeUnsubscribeResp fromTPipeSubscribeResp( - TPipeSubscribeResp unsubscribeResp) { + final TPipeSubscribeResp unsubscribeResp) { final PipeSubscribeUnsubscribeResp resp = new PipeSubscribeUnsubscribeResp(); resp.status = unsubscribeResp.status; @@ -58,14 +58,14 @@ public static PipeSubscribeUnsubscribeResp fromTPipeSubscribeResp( /////////////////////////////// Object /////////////////////////////// @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; } - PipeSubscribeUnsubscribeResp that = (PipeSubscribeUnsubscribeResp) obj; + final PipeSubscribeUnsubscribeResp that = (PipeSubscribeUnsubscribeResp) obj; return Objects.equals(this.status, that.status) && this.version == that.version && this.type == that.type diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java index 3310bf008392..5463522327f1 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java @@ -27,19 +27,19 @@ import org.apache.iotdb.rpc.subscription.exception.SubscriptionException; import org.apache.iotdb.rpc.subscription.exception.SubscriptionRuntimeCriticalException; import org.apache.iotdb.rpc.subscription.exception.SubscriptionRuntimeNonCriticalException; -import org.apache.iotdb.rpc.subscription.payload.common.PollMessagePayload; -import org.apache.iotdb.rpc.subscription.payload.common.PollTsFileMessagePayload; -import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionCommitContext; -import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionMessagePayload; -import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPollMessage; -import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPollMessageType; -import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPolledMessage; -import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPolledMessageType; -import org.apache.iotdb.rpc.subscription.payload.common.TabletsMessagePayload; -import org.apache.iotdb.rpc.subscription.payload.common.TsFileErrorMessagePayload; -import org.apache.iotdb.rpc.subscription.payload.common.TsFileInitMessagePayload; -import org.apache.iotdb.rpc.subscription.payload.common.TsFilePieceMessagePayload; -import org.apache.iotdb.rpc.subscription.payload.common.TsFileSealMessagePayload; +import org.apache.iotdb.rpc.subscription.payload.poll.ErrorPayload; +import org.apache.iotdb.rpc.subscription.payload.poll.FileInitPayload; +import org.apache.iotdb.rpc.subscription.payload.poll.FilePiecePayload; +import org.apache.iotdb.rpc.subscription.payload.poll.FileSealPayload; +import org.apache.iotdb.rpc.subscription.payload.poll.PollFilePayload; +import org.apache.iotdb.rpc.subscription.payload.poll.PollPayload; +import org.apache.iotdb.rpc.subscription.payload.poll.SubscriptionCommitContext; +import org.apache.iotdb.rpc.subscription.payload.poll.SubscriptionPollPayload; +import org.apache.iotdb.rpc.subscription.payload.poll.SubscriptionPollRequest; +import org.apache.iotdb.rpc.subscription.payload.poll.SubscriptionPollRequestType; +import org.apache.iotdb.rpc.subscription.payload.poll.SubscriptionPollResponse; +import org.apache.iotdb.rpc.subscription.payload.poll.SubscriptionPollResponseType; +import org.apache.iotdb.rpc.subscription.payload.poll.TabletsPayload; import org.apache.iotdb.session.subscription.payload.SubscriptionMessage; import org.apache.iotdb.session.subscription.util.RandomStringGenerator; import org.apache.iotdb.session.subscription.util.SubscriptionPollTimer; @@ -97,24 +97,24 @@ public abstract class SubscriptionConsumer implements AutoCloseable { private final AtomicBoolean isClosed = new AtomicBoolean(true); - private final String tsFileBaseDir; + private final String fileSaveDir; - private Path getTsFileDir(final String topicName) throws IOException { + private Path getFileDir(final String topicName) throws IOException { final Path dirPath = - Paths.get(tsFileBaseDir).resolve(consumerGroupId).resolve(consumerId).resolve(topicName); + Paths.get(fileSaveDir).resolve(consumerGroupId).resolve(consumerId).resolve(topicName); Files.createDirectories(dirPath); return dirPath; } - private Path getTsFilePath(final String topicName, String fileName) throws SubscriptionException { + private Path getFilePath(final String topicName, String fileName) throws SubscriptionException { Path filePath; try { - filePath = getTsFileDir(topicName).resolve(fileName); + filePath = getFileDir(topicName).resolve(fileName); Files.createFile(filePath); } catch (final FileAlreadyExistsException fileAlreadyExistsException) { fileName += "." + RandomStringGenerator.generate(16); try { - filePath = getTsFileDir(topicName).resolve(fileName); + filePath = getFileDir(topicName).resolve(fileName); Files.createFile(filePath); } catch (final IOException e) { throw new SubscriptionRuntimeNonCriticalException(e.getMessage(), e); @@ -155,7 +155,7 @@ protected SubscriptionConsumer(final Builder builder) { this.heartbeatIntervalMs = builder.heartbeatIntervalMs; this.endpointsSyncIntervalMs = builder.endpointsSyncIntervalMs; - this.tsFileBaseDir = builder.tsFileBaseDir; + this.fileSaveDir = builder.fileSaveDir; } protected SubscriptionConsumer(final Builder builder, final Properties properties) { @@ -188,11 +188,11 @@ protected SubscriptionConsumer(final Builder builder, final Properties propertie properties.getOrDefault( ConsumerConstant.ENDPOINTS_SYNC_INTERVAL_MS_KEY, ConsumerConstant.ENDPOINTS_SYNC_INTERVAL_MS_DEFAULT_VALUE)) - .tsFileBaseDir( + .fileSaveDir( (String) properties.getOrDefault( - ConsumerConstant.TS_FILE_BASE_DIR_KEY, - ConsumerConstant.TS_FILE_BASE_DIR_DEFAULT_VALUE))); + ConsumerConstant.FILE_SAVE_DIR_KEY, + ConsumerConstant.FILE_SAVE_DIR_DEFAULT_VALUE))); } /////////////////////////////// open & close /////////////////////////////// @@ -385,28 +385,28 @@ protected List poll(final Set topicNames, final lon do { try { - // poll tablets or tsfile - for (final SubscriptionPolledMessage polledMessage : pollInternal(topicNames)) { - final short messageType = polledMessage.getMessageType(); - if (!SubscriptionPolledMessageType.isValidatedMessageType(messageType)) { - LOGGER.warn("unexpected message type: {}", messageType); + // poll tablets or file + for (final SubscriptionPollResponse pollResponse : pollInternal(topicNames)) { + final short responseType = pollResponse.getResponseType(); + if (!SubscriptionPollResponseType.isValidatedResponseType(responseType)) { + LOGGER.warn("unexpected response type: {}", responseType); continue; } - switch (SubscriptionPolledMessageType.valueOf(messageType)) { + switch (SubscriptionPollResponseType.valueOf(responseType)) { case TABLETS: messages.add( new SubscriptionMessage( - polledMessage.getCommitContext(), - ((TabletsMessagePayload) polledMessage.getMessagePayload()).getTablets())); + pollResponse.getCommitContext(), + ((TabletsPayload) pollResponse.getPayload()).getTablets())); break; - case TS_FILE_INIT: - pollTsFile( - polledMessage.getCommitContext(), - ((TsFileInitMessagePayload) polledMessage.getMessagePayload()).getFileName()) + case FILE_INIT: + pollFile( + pollResponse.getCommitContext(), + ((FileInitPayload) pollResponse.getPayload()).getFileName()) .ifPresent(messages::add); break; default: - LOGGER.warn("unexpected message type: {}", messageType); + LOGGER.warn("unexpected response type: {}", responseType); break; } } @@ -451,20 +451,20 @@ protected List poll(final Set topicNames, final lon return messages; } - private Optional pollTsFile( + private Optional pollFile( final SubscriptionCommitContext commitContext, final String fileName) throws SubscriptionException { final String topicName = commitContext.getTopicName(); - final Path filePath = getTsFilePath(topicName, fileName); + final Path filePath = getFilePath(topicName, fileName); final File file = filePath.toFile(); try (final RandomAccessFile fileWriter = new RandomAccessFile(file, "rw")) { - return Optional.of(pollTsFileInternal(commitContext, file, fileWriter)); + return Optional.of(pollFileInternal(commitContext, file, fileWriter)); } catch (final IOException e) { throw new SubscriptionRuntimeNonCriticalException(e.getMessage(), e); } } - private SubscriptionMessage pollTsFileInternal( + private SubscriptionMessage pollFileInternal( final SubscriptionCommitContext commitContext, final File file, final RandomAccessFile fileWriter) @@ -474,41 +474,40 @@ private SubscriptionMessage pollTsFileInternal( final String fileName = file.getName(); LOGGER.info( - "{} start to poll TsFile {} with commit context {}", + "{} start to poll file {} with commit context {}", this, file.getAbsolutePath(), commitContext); long writingOffset = fileWriter.length(); while (true) { - final List polledMessages = - pollTsFileInternal(dataNodeId, topicName, fileName, writingOffset); + final List responses = + pollFileInternal(dataNodeId, topicName, fileName, writingOffset); - // It's agreed that the server will always return at least one message, even in case of + // It's agreed that the server will always return at least one response, even in case of // failure. - if (polledMessages.isEmpty()) { + if (responses.isEmpty()) { final String errorMessage = - String.format("SubscriptionConsumer %s poll empty tsfile message", this); + String.format("SubscriptionConsumer %s poll empty response", this); LOGGER.warn(errorMessage); throw new SubscriptionRuntimeNonCriticalException(errorMessage); } // Only one SubscriptionEvent polled currently... - final SubscriptionPolledMessage polledMessage = polledMessages.get(0); - final SubscriptionMessagePayload messagePayload = polledMessage.getMessagePayload(); - final short messageType = polledMessage.getMessageType(); - if (!SubscriptionPolledMessageType.isValidatedMessageType(messageType)) { - final String errorMessage = String.format("unexpected message type: %s", messageType); + final SubscriptionPollResponse response = responses.get(0); + final SubscriptionPollPayload payload = response.getPayload(); + final short responseType = response.getResponseType(); + if (!SubscriptionPollResponseType.isValidatedResponseType(responseType)) { + final String errorMessage = String.format("unexpected response type: %s", responseType); LOGGER.warn(errorMessage); throw new SubscriptionRuntimeNonCriticalException(errorMessage); } - switch (SubscriptionPolledMessageType.valueOf(messageType)) { - case TS_FILE_PIECE: + switch (SubscriptionPollResponseType.valueOf(responseType)) { + case FILE_PIECE: { // check commit context - final SubscriptionCommitContext incomingCommitContext = - polledMessage.getCommitContext(); + final SubscriptionCommitContext incomingCommitContext = response.getCommitContext(); if (Objects.isNull(incomingCommitContext) || !Objects.equals(commitContext, incomingCommitContext)) { final String errorMessage = @@ -520,42 +519,40 @@ private SubscriptionMessage pollTsFileInternal( } // check file name - if (!fileName.startsWith(((TsFilePieceMessagePayload) messagePayload).getFileName())) { + if (!fileName.startsWith(((FilePiecePayload) payload).getFileName())) { final String errorMessage = String.format( "inconsistent file name, current is %s, incoming is %s, consumer: %s", - fileName, ((TsFilePieceMessagePayload) messagePayload).getFileName(), this); + fileName, ((FilePiecePayload) payload).getFileName(), this); LOGGER.warn(errorMessage); throw new SubscriptionRuntimeNonCriticalException(errorMessage); } // write file piece - fileWriter.write(((TsFilePieceMessagePayload) messagePayload).getFilePiece()); + fileWriter.write(((FilePiecePayload) payload).getFilePiece()); fileWriter.getFD().sync(); // check offset if (!Objects.equals( - fileWriter.length(), - ((TsFilePieceMessagePayload) messagePayload).getNextWritingOffset())) { + fileWriter.length(), ((FilePiecePayload) payload).getNextWritingOffset())) { final String errorMessage = String.format( "inconsistent file offset, current is %s, incoming is %s, consumer: %s", fileWriter.length(), - ((TsFilePieceMessagePayload) messagePayload).getNextWritingOffset(), + ((FilePiecePayload) payload).getNextWritingOffset(), this); LOGGER.warn(errorMessage); throw new SubscriptionRuntimeNonCriticalException(errorMessage); } // update offset - writingOffset = ((TsFilePieceMessagePayload) messagePayload).getNextWritingOffset(); + writingOffset = ((FilePiecePayload) payload).getNextWritingOffset(); break; } - case TS_FILE_SEAL: + case FILE_SEAL: { // check commit context - final SubscriptionCommitContext incomingCommitContext = - polledMessage.getCommitContext(); + final SubscriptionCommitContext incomingCommitContext = response.getCommitContext(); if (Objects.isNull(incomingCommitContext) || !Objects.equals(commitContext, incomingCommitContext)) { final String errorMessage = @@ -567,24 +564,21 @@ private SubscriptionMessage pollTsFileInternal( } // check file name - if (!fileName.startsWith(((TsFileSealMessagePayload) messagePayload).getFileName())) { + if (!fileName.startsWith(((FileSealPayload) payload).getFileName())) { final String errorMessage = String.format( "inconsistent file name, current is %s, incoming is %s, consumer: %s", - fileName, ((TsFileSealMessagePayload) messagePayload).getFileName(), this); + fileName, ((FileSealPayload) payload).getFileName(), this); LOGGER.warn(errorMessage); throw new SubscriptionRuntimeNonCriticalException(errorMessage); } // check file length - if (fileWriter.length() - != ((TsFileSealMessagePayload) messagePayload).getFileLength()) { + if (fileWriter.length() != ((FileSealPayload) payload).getFileLength()) { final String errorMessage = String.format( "inconsistent file length, current is %s, incoming is %s, consumer: %s", - fileWriter.length(), - ((TsFileSealMessagePayload) messagePayload).getFileLength(), - this); + fileWriter.length(), ((FileSealPayload) payload).getFileLength(), this); LOGGER.warn(errorMessage); throw new SubscriptionRuntimeNonCriticalException(errorMessage); } @@ -594,7 +588,7 @@ private SubscriptionMessage pollTsFileInternal( fileWriter.close(); LOGGER.info( - "SubscriptionConsumer {} successfully poll TsFile {} with commit context {}", + "SubscriptionConsumer {} successfully poll file {} with commit context {}", this, file.getAbsolutePath(), commitContext); @@ -602,15 +596,14 @@ private SubscriptionMessage pollTsFileInternal( // generate subscription message return new SubscriptionMessage(commitContext, file.getAbsolutePath()); } - case TS_FILE_ERROR: + case ERROR: { // no need to check commit context - final String errorMessage = - ((TsFileErrorMessagePayload) messagePayload).getErrorMessage(); - final boolean critical = ((TsFileErrorMessagePayload) messagePayload).isCritical(); + final String errorMessage = ((ErrorPayload) payload).getErrorMessage(); + final boolean critical = ((ErrorPayload) payload).isCritical(); LOGGER.warn( - "Error occurred when SubscriptionConsumer {} polling TsFile {} with commit context {}: {}, critical: {}", + "Error occurred when SubscriptionConsumer {} polling file {} with commit context {}: {}, critical: {}", this, file.getAbsolutePath(), commitContext, @@ -623,14 +616,14 @@ private SubscriptionMessage pollTsFileInternal( } } default: - final String errorMessage = String.format("unexpected message type: %s", messageType); + final String errorMessage = String.format("unexpected response type: %s", responseType); LOGGER.warn(errorMessage); throw new SubscriptionRuntimeNonCriticalException(errorMessage); } } } - private List pollInternal(final Set topicNames) + private List pollInternal(final Set topicNames) throws SubscriptionException { subscriptionProviders.acquireReadLock(); try { @@ -641,12 +634,11 @@ private List pollInternal(final Set topicName "Cluster has no available subscription providers when %s poll topic %s", this, topicNames)); } + // ignore SubscriptionConnectionException to improve poll auto retry try { return provider.poll( - new SubscriptionPollMessage( - SubscriptionPollMessageType.POLL.getType(), - new PollMessagePayload(topicNames), - 0L)); + new SubscriptionPollRequest( + SubscriptionPollRequestType.POLL.getType(), new PollPayload(topicNames), 0L)); } catch (final SubscriptionConnectionException ignored) { return Collections.emptyList(); } @@ -655,7 +647,7 @@ private List pollInternal(final Set topicName } } - private List pollTsFileInternal( + private List pollFileInternal( final int dataNodeId, final String topicName, final String fileName, final long writingOffset) throws SubscriptionException { subscriptionProviders.acquireReadLock(); @@ -664,14 +656,15 @@ private List pollTsFileInternal( if (Objects.isNull(provider) || !provider.isAvailable()) { throw new SubscriptionConnectionException( String.format( - "something unexpected happened when %s poll TsFile from subscription provider with data node id %s, the subscription provider may be unavailable or not existed", + "something unexpected happened when %s poll file from subscription provider with data node id %s, the subscription provider may be unavailable or not existed", this, dataNodeId)); } + // ignore SubscriptionConnectionException to improve poll auto retry try { return provider.poll( - new SubscriptionPollMessage( - SubscriptionPollMessageType.POLL_TS_FILE.getType(), - new PollTsFileMessagePayload(topicName, fileName, writingOffset), + new SubscriptionPollRequest( + SubscriptionPollRequestType.POLL_FILE.getType(), + new PollFilePayload(topicName, fileName, writingOffset), 0L)); } catch (final SubscriptionConnectionException ignored) { return Collections.emptyList(); @@ -874,7 +867,7 @@ public abstract static class Builder { protected long endpointsSyncIntervalMs = ConsumerConstant.ENDPOINTS_SYNC_INTERVAL_MS_DEFAULT_VALUE; - protected String tsFileBaseDir = ConsumerConstant.TS_FILE_BASE_DIR_DEFAULT_VALUE; + protected String fileSaveDir = ConsumerConstant.FILE_SAVE_DIR_DEFAULT_VALUE; public Builder host(final String host) { this.host = host; @@ -923,8 +916,8 @@ public Builder endpointsSyncIntervalMs(final long endpointsSyncIntervalMs) { return this; } - public Builder tsFileBaseDir(final String tsFileBaseDir) { - this.tsFileBaseDir = tsFileBaseDir; + public Builder fileSaveDir(final String fileSaveDir) { + this.fileSaveDir = fileSaveDir; return this; } diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionProvider.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionProvider.java index 786a21bf0df0..cccc7cdb389c 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionProvider.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionProvider.java @@ -28,9 +28,9 @@ import org.apache.iotdb.rpc.subscription.exception.SubscriptionException; import org.apache.iotdb.rpc.subscription.exception.SubscriptionRuntimeCriticalException; import org.apache.iotdb.rpc.subscription.exception.SubscriptionRuntimeNonCriticalException; -import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionCommitContext; -import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPollMessage; -import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPolledMessage; +import org.apache.iotdb.rpc.subscription.payload.poll.SubscriptionCommitContext; +import org.apache.iotdb.rpc.subscription.payload.poll.SubscriptionPollRequest; +import org.apache.iotdb.rpc.subscription.payload.poll.SubscriptionPollResponse; import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribeCloseReq; import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribeCommitReq; import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribeHandshakeReq; @@ -269,7 +269,7 @@ void unsubscribe(final Set topicNames) throws SubscriptionException { verifyPipeSubscribeSuccess(resp.status); } - List poll(final SubscriptionPollMessage pollMessage) + List poll(final SubscriptionPollRequest pollMessage) throws SubscriptionException { final PipeSubscribePollReq req; try { @@ -297,7 +297,7 @@ List poll(final SubscriptionPollMessage pollMessage) } verifyPipeSubscribeSuccess(resp.status); final PipeSubscribePollResp pollResp = PipeSubscribePollResp.fromTPipeSubscribeResp(resp); - return pollResp.getMessages(); + return pollResp.getResponses(); } void commit(final List subscriptionCommitContexts, final boolean nack) diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java index a9a8972423bd..a14e67c820b9 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java @@ -284,8 +284,8 @@ public Builder endpointsSyncIntervalMs(final long endpointsSyncIntervalMs) { return this; } - public Builder tsFileBaseDir(final String tsFileBaseDir) { - super.tsFileBaseDir(tsFileBaseDir); + public Builder fileSaveDir(final String fileSaveDir) { + super.fileSaveDir(fileSaveDir); return this; } diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPushConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPushConsumer.java index a94353706aeb..25e0e66aaf9e 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPushConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPushConsumer.java @@ -185,8 +185,8 @@ public SubscriptionPushConsumer.Builder endpointsSyncIntervalMs( return this; } - public SubscriptionPushConsumer.Builder tsFileBaseDir(final String tsFileBaseDir) { - this.tsFileBaseDir = tsFileBaseDir; + public SubscriptionPushConsumer.Builder fileSaveDir(final String fileSaveDir) { + this.fileSaveDir = fileSaveDir; return this; } diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionFileHandler.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionFileHandler.java new file mode 100644 index 000000000000..0e8237f53376 --- /dev/null +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionFileHandler.java @@ -0,0 +1,37 @@ +/* + * 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.session.subscription.payload; + +import org.apache.iotdb.rpc.subscription.exception.SubscriptionIncompatibleHandlerException; + +public abstract class SubscriptionFileHandler implements SubscriptionMessageHandler { + + protected final String filePath; + + public SubscriptionFileHandler(final String filePath) { + this.filePath = filePath; + } + + @Override + public SubscriptionSessionDataSetsHandler getSessionDataSetsHandler() { + throw new SubscriptionIncompatibleHandlerException( + "SubscriptionFileHandler do not support getSessionDataSetsHandler()."); + } +} diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionMessage.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionMessage.java index 95ce271176f5..becb88e90f42 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionMessage.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionMessage.java @@ -19,31 +19,33 @@ package org.apache.iotdb.session.subscription.payload; -import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionCommitContext; +import org.apache.iotdb.rpc.subscription.payload.poll.SubscriptionCommitContext; import org.apache.tsfile.write.record.Tablet; import java.util.List; import java.util.Objects; -public class SubscriptionMessage implements Comparable { +public class SubscriptionMessage + implements Comparable, SubscriptionMessageHandler { private final SubscriptionCommitContext commitContext; private final short messageType; - private final SubscriptionMessagePayload payload; + private final SubscriptionMessageHandler handler; - public SubscriptionMessage(SubscriptionCommitContext commitContext, List tablets) { + public SubscriptionMessage( + final SubscriptionCommitContext commitContext, final List tablets) { this.commitContext = commitContext; - this.messageType = SubscriptionMessageType.SESSION_DATA_SET.getType(); - this.payload = new SubscriptionSessionDataSets(tablets); + this.messageType = SubscriptionMessageType.SUBSCRIPTION_SESSION_DATA_SETS_HANDLER.getType(); + this.handler = new SubscriptionSessionDataSetsHandler(tablets); } - public SubscriptionMessage(SubscriptionCommitContext commitContext, String filePath) { + public SubscriptionMessage(final SubscriptionCommitContext commitContext, final String filePath) { this.commitContext = commitContext; - this.messageType = SubscriptionMessageType.TS_FILE_READER.getType(); - this.payload = new SubscriptionTsFileReader(filePath); + this.messageType = SubscriptionMessageType.SUBSCRIPTION_TS_FILE_HANDLER.getType(); + this.handler = new SubscriptionTsFileHandler(filePath); } public SubscriptionCommitContext getCommitContext() { @@ -54,33 +56,29 @@ public short getMessageType() { return messageType; } - public SubscriptionMessagePayload getPayload() { - return payload; - } - /////////////////////////////// override /////////////////////////////// @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; } - SubscriptionMessage that = (SubscriptionMessage) obj; + final SubscriptionMessage that = (SubscriptionMessage) obj; return Objects.equals(this.commitContext, that.commitContext) && Objects.equals(this.messageType, that.messageType) - && Objects.equals(this.payload, that.payload); + && Objects.equals(this.handler, that.handler); } @Override public int hashCode() { - return Objects.hash(commitContext, messageType, payload); + return Objects.hash(commitContext, messageType, handler); } @Override - public int compareTo(SubscriptionMessage that) { + public int compareTo(final SubscriptionMessage that) { return this.commitContext.compareTo(that.commitContext); } @@ -92,4 +90,16 @@ public String toString() { + SubscriptionMessageType.valueOf(messageType).toString() + "}"; } + + /////////////////////////////// handlers /////////////////////////////// + + @Override + public SubscriptionSessionDataSetsHandler getSessionDataSetsHandler() { + return handler.getSessionDataSetsHandler(); + } + + @Override + public SubscriptionTsFileHandler getTsFileHandler() { + return handler.getTsFileHandler(); + } } diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionMessagePayload.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionMessageHandler.java similarity index 84% rename from iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionMessagePayload.java rename to iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionMessageHandler.java index 0178afa2182a..9aa5527ea6e3 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionMessagePayload.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionMessageHandler.java @@ -19,4 +19,9 @@ package org.apache.iotdb.session.subscription.payload; -public interface SubscriptionMessagePayload {} +public interface SubscriptionMessageHandler { + + SubscriptionSessionDataSetsHandler getSessionDataSetsHandler(); + + SubscriptionTsFileHandler getTsFileHandler(); +} diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionMessageType.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionMessageType.java index c418476a15dc..ef142b3ec7cf 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionMessageType.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionMessageType.java @@ -24,13 +24,13 @@ import java.util.Map; public enum SubscriptionMessageType { - SESSION_DATA_SET((short) 0), - TS_FILE_READER((short) 1), + SUBSCRIPTION_SESSION_DATA_SETS_HANDLER((short) 0), + SUBSCRIPTION_TS_FILE_HANDLER((short) 1), ; private final short type; - SubscriptionMessageType(short type) { + SubscriptionMessageType(final short type) { this.type = type; } @@ -45,11 +45,11 @@ public short getType() { (typeMap, messageType) -> typeMap.put(messageType.getType(), messageType), HashMap::putAll); - public static boolean isValidatedMessageType(short type) { + public static boolean isValidatedMessageType(final short type) { return TYPE_MAP.containsKey(type); } - public static SubscriptionMessageType valueOf(short type) { + public static SubscriptionMessageType valueOf(final short type) { return TYPE_MAP.get(type); } } diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionSessionDataSets.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionSessionDataSetsHandler.java similarity index 62% rename from iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionSessionDataSets.java rename to iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionSessionDataSetsHandler.java index 35bbbf372d78..9da0aa3805a9 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionSessionDataSets.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionSessionDataSetsHandler.java @@ -19,31 +19,44 @@ package org.apache.iotdb.session.subscription.payload; +import org.apache.iotdb.rpc.subscription.exception.SubscriptionIncompatibleHandlerException; + import org.apache.tsfile.write.record.Tablet; import java.util.ArrayList; import java.util.Iterator; import java.util.List; -public class SubscriptionSessionDataSets - implements Iterable, SubscriptionMessagePayload { +public class SubscriptionSessionDataSetsHandler + implements Iterable, SubscriptionMessageHandler { - private final List dataSetList; + private final List dataSets; private final List tablets; - public SubscriptionSessionDataSets(final List tablets) { - this.dataSetList = new ArrayList<>(); + public SubscriptionSessionDataSetsHandler(final List tablets) { + this.dataSets = new ArrayList<>(); this.tablets = tablets; - tablets.forEach((tablet -> this.dataSetList.add(new SubscriptionSessionDataSet(tablet)))); + tablets.forEach((tablet -> this.dataSets.add(new SubscriptionSessionDataSet(tablet)))); } @Override public Iterator iterator() { - return dataSetList.iterator(); + return dataSets.iterator(); } public Iterator tabletIterator() { return tablets.iterator(); } + + @Override + public SubscriptionSessionDataSetsHandler getSessionDataSetsHandler() { + return this; + } + + @Override + public SubscriptionTsFileHandler getTsFileHandler() { + throw new SubscriptionIncompatibleHandlerException( + "SubscriptionSessionDataSetsHandler do not support getSessionDataSetsHandler()."); + } } diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionTsFileReader.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionTsFileHandler.java similarity index 75% rename from iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionTsFileReader.java rename to iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionTsFileHandler.java index 11c17e71c198..136a4204a5b3 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionTsFileReader.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionTsFileHandler.java @@ -24,20 +24,18 @@ import java.io.IOException; -public class SubscriptionTsFileReader implements SubscriptionMessagePayload { +public class SubscriptionTsFileHandler extends SubscriptionFileHandler { - private final String filePath; - - public SubscriptionTsFileReader(final String filePath) { - this.filePath = filePath; + public SubscriptionTsFileHandler(final String filePath) { + super(filePath); } - public TsFileReader open() throws IOException { + public TsFileReader openReader() throws IOException { return new TsFileReader(new TsFileSequenceReader(filePath)); } @Override - public String toString() { - return "SubscriptionTsFileReader{" + "filePath='" + filePath + '\'' + '}'; + public SubscriptionTsFileHandler getTsFileHandler() { + return this; } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionBrokerAgent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionBrokerAgent.java index 0b95f361ae3d..aff4893c61a0 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionBrokerAgent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionBrokerAgent.java @@ -24,7 +24,7 @@ import org.apache.iotdb.db.subscription.task.subtask.SubscriptionConnectorSubtask; import org.apache.iotdb.rpc.subscription.config.ConsumerConfig; import org.apache.iotdb.rpc.subscription.exception.SubscriptionException; -import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionCommitContext; +import org.apache.iotdb.rpc.subscription.payload.poll.SubscriptionCommitContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; 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 bc2cc69581f4..f7fc08af5c7e 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 @@ -26,7 +26,7 @@ import org.apache.iotdb.pipe.api.event.Event; import org.apache.iotdb.rpc.subscription.config.TopicConstant; import org.apache.iotdb.rpc.subscription.exception.SubscriptionException; -import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionCommitContext; +import org.apache.iotdb.rpc.subscription.payload.poll.SubscriptionCommitContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; 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 a7808c24f3a0..1818880f6210 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 @@ -25,7 +25,7 @@ import org.apache.iotdb.db.subscription.event.SubscriptionEvent; import org.apache.iotdb.db.subscription.event.SubscriptionEventBinaryCache; import org.apache.iotdb.pipe.api.event.Event; -import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionCommitContext; +import org.apache.iotdb.rpc.subscription.payload.poll.SubscriptionCommitContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTabletsQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTabletsQueue.java index 6dd48eefc96a..0b15133a36da 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTabletsQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTabletsQueue.java @@ -31,10 +31,10 @@ import org.apache.iotdb.db.subscription.event.SubscriptionEventBinaryCache; import org.apache.iotdb.pipe.api.event.Event; import org.apache.iotdb.pipe.api.event.dml.insertion.TabletInsertionEvent; -import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionCommitContext; -import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPolledMessage; -import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPolledMessageType; -import org.apache.iotdb.rpc.subscription.payload.common.TabletsMessagePayload; +import org.apache.iotdb.rpc.subscription.payload.poll.SubscriptionCommitContext; +import org.apache.iotdb.rpc.subscription.payload.poll.SubscriptionPollResponse; +import org.apache.iotdb.rpc.subscription.payload.poll.SubscriptionPollResponseType; +import org.apache.iotdb.rpc.subscription.payload.poll.TabletsPayload; import org.apache.tsfile.write.record.Tablet; import org.slf4j.Logger; @@ -184,9 +184,9 @@ private void prefetchOnce() { final SubscriptionEvent subscriptionEvent = new SubscriptionEvent( enrichedEvents, - new SubscriptionPolledMessage( - SubscriptionPolledMessageType.TABLETS.getType(), - new TabletsMessagePayload(tablets, calculatedTabletsSizeInBytes), + new SubscriptionPollResponse( + SubscriptionPollResponseType.TABLETS.getType(), + new TabletsPayload(tablets, calculatedTabletsSizeInBytes), commitContext)); uncommittedEvents.put(commitContext, subscriptionEvent); // before enqueuing the event prefetchingQueue.add(subscriptionEvent); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java index fd63f0f838ce..80b834a93466 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java @@ -25,14 +25,14 @@ import org.apache.iotdb.db.pipe.event.common.tsfile.PipeTsFileInsertionEvent; import org.apache.iotdb.db.subscription.event.SubscriptionTsFileEvent; import org.apache.iotdb.pipe.api.event.Event; -import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionCommitContext; -import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionMessagePayload; -import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPolledMessage; -import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPolledMessageType; -import org.apache.iotdb.rpc.subscription.payload.common.TsFileErrorMessagePayload; -import org.apache.iotdb.rpc.subscription.payload.common.TsFileInitMessagePayload; -import org.apache.iotdb.rpc.subscription.payload.common.TsFilePieceMessagePayload; -import org.apache.iotdb.rpc.subscription.payload.common.TsFileSealMessagePayload; +import org.apache.iotdb.rpc.subscription.payload.poll.ErrorPayload; +import org.apache.iotdb.rpc.subscription.payload.poll.FileInitPayload; +import org.apache.iotdb.rpc.subscription.payload.poll.FilePiecePayload; +import org.apache.iotdb.rpc.subscription.payload.poll.FileSealPayload; +import org.apache.iotdb.rpc.subscription.payload.poll.SubscriptionCommitContext; +import org.apache.iotdb.rpc.subscription.payload.poll.SubscriptionPollPayload; +import org.apache.iotdb.rpc.subscription.payload.poll.SubscriptionPollResponse; +import org.apache.iotdb.rpc.subscription.payload.poll.SubscriptionPollResponseType; import org.apache.tsfile.utils.Pair; import org.checkerframework.checker.nullness.qual.NonNull; @@ -112,7 +112,7 @@ public SubscriptionTsFileEvent poll(final String consumerId) { "SubscriptionPrefetchingTsFileQueue %s is currently not transferring any TsFile to consumer %s, file name: %s, writing offset: %s", this, consumerId, fileName, writingOffset); LOGGER.warn(errorMessage); - return generateSubscriptionTsFileEventWithErrorMessage(errorMessage); + return generateSubscriptionPollErrorResponse(errorMessage); } if (event.isCommitted()) { @@ -122,7 +122,7 @@ public SubscriptionTsFileEvent poll(final String consumerId) { "SubscriptionEvent %s related to TsFile is committed, consumer: %s, writing offset: %s, prefetching queue: %s", event, consumerId, writingOffset, this); LOGGER.warn(errorMessage); - return generateSubscriptionTsFileEventWithErrorMessage(errorMessage); + return generateSubscriptionPollErrorResponse(errorMessage); } // check consumer id @@ -132,7 +132,7 @@ public SubscriptionTsFileEvent poll(final String consumerId) { "inconsistent polled consumer id, current: %s, incoming: %s, file name: %s, writing offset: %s, prefetching queue: %s", event.getLastPolledConsumerId(), consumerId, fileName, writingOffset, this); LOGGER.warn(errorMessage); - return generateSubscriptionTsFileEventWithErrorMessage(errorMessage); + return generateSubscriptionPollErrorResponse(errorMessage); } final List enrichedEvents = event.getEnrichedEvents(); @@ -150,34 +150,34 @@ public SubscriptionTsFileEvent poll(final String consumerId) { writingOffset, this); LOGGER.warn(errorMessage); - return generateSubscriptionTsFileEventWithErrorMessage(errorMessage); + return generateSubscriptionPollErrorResponse(errorMessage); } - final SubscriptionPolledMessage polledMessage = event.getMessage(); - final SubscriptionMessagePayload messagePayload = polledMessage.getMessagePayload(); + final SubscriptionPollResponse response = event.getResponse(); + final SubscriptionPollPayload payload = response.getPayload(); - // 2. Check message type, file name and offset - final short messageType = polledMessage.getMessageType(); - if (!SubscriptionPolledMessageType.isValidatedMessageType(messageType)) { - final String errorMessage = String.format("unexpected message type: %s", messageType); + // 2. Check previous response type, file name and offset + final short responseType = response.getResponseType(); + if (!SubscriptionPollResponseType.isValidatedResponseType(responseType)) { + final String errorMessage = String.format("unexpected response type: %s", responseType); LOGGER.warn(errorMessage); - return generateSubscriptionTsFileEventWithErrorMessage(errorMessage); + return generateSubscriptionPollErrorResponse(errorMessage); } - switch (SubscriptionPolledMessageType.valueOf(messageType)) { - case TS_FILE_INIT: + switch (SubscriptionPollResponseType.valueOf(responseType)) { + case FILE_INIT: // check file name - if (!fileName.startsWith(((TsFileInitMessagePayload) messagePayload).getFileName())) { + if (!fileName.startsWith(((FileInitPayload) payload).getFileName())) { final String errorMessage = String.format( "inconsistent file name, current: %s, incoming: %s, consumer: %s, writing offset: %s, prefetching queue: %s", - ((TsFileInitMessagePayload) messagePayload).getFileName(), + ((FileInitPayload) payload).getFileName(), fileName, consumerId, writingOffset, this); LOGGER.warn(errorMessage); - return generateSubscriptionTsFileEventWithErrorMessage(errorMessage); + return generateSubscriptionPollErrorResponse(errorMessage); } // check offset if (writingOffset != 0) { @@ -189,22 +189,22 @@ public SubscriptionTsFileEvent poll(final String consumerId) { consumerId); } break; - case TS_FILE_PIECE: + case FILE_PIECE: // check file name - if (!fileName.startsWith(((TsFilePieceMessagePayload) messagePayload).getFileName())) { + if (!fileName.startsWith(((FilePiecePayload) payload).getFileName())) { final String errorMessage = String.format( "inconsistent file name, current: %s, incoming: %s, consumer: %s, writing offset: %s, prefetching queue: %s", - ((TsFilePieceMessagePayload) messagePayload).getFileName(), + ((FilePiecePayload) payload).getFileName(), fileName, consumerId, writingOffset, this); LOGGER.warn(errorMessage); - return generateSubscriptionTsFileEventWithErrorMessage(errorMessage); + return generateSubscriptionPollErrorResponse(errorMessage); } // check offset - if (writingOffset != ((TsFilePieceMessagePayload) messagePayload).getNextWritingOffset()) { + if (writingOffset != ((FilePiecePayload) payload).getNextWritingOffset()) { LOGGER.warn( "SubscriptionPrefetchingTsFileQueue {} set TsFile (with event {}) writing offset to {} for consumer {}", this, @@ -213,19 +213,19 @@ public SubscriptionTsFileEvent poll(final String consumerId) { consumerId); } break; - case TS_FILE_SEAL: + case FILE_SEAL: // check file name - if (!fileName.startsWith(((TsFileSealMessagePayload) messagePayload).getFileName())) { + if (!fileName.startsWith(((FileSealPayload) payload).getFileName())) { final String errorMessage = String.format( "inconsistent file name, current: %s, incoming: %s, consumer: %s, writing offset: %s, prefetching queue: %s", - ((TsFileSealMessagePayload) messagePayload).getFileName(), + ((FileSealPayload) payload).getFileName(), fileName, consumerId, writingOffset, this); LOGGER.warn(errorMessage); - return generateSubscriptionTsFileEventWithErrorMessage(errorMessage); + return generateSubscriptionPollErrorResponse(errorMessage); } LOGGER.warn( @@ -235,12 +235,12 @@ public SubscriptionTsFileEvent poll(final String consumerId) { writingOffset, consumerId); // mark uncommittable - uncommittedEvents.remove(polledMessage.getCommitContext()); + uncommittedEvents.remove(response.getCommitContext()); break; default: - final String errorMessage = String.format("unexpected message type: %s", messageType); + final String errorMessage = String.format("unexpected response type: %s", responseType); LOGGER.warn(errorMessage); - return generateSubscriptionTsFileEventWithErrorMessage(errorMessage); + return generateSubscriptionPollErrorResponse(errorMessage); } // 3. Poll tsfile piece or tsfile seal @@ -261,7 +261,7 @@ public SubscriptionTsFileEvent poll(final String consumerId) { "IOException occurred when SubscriptionPrefetchingTsFileQueue %s transferring TsFile (with event %s) to consumer %s: %s", this, event, consumerId, e); LOGGER.warn(errorMessage); - return generateSubscriptionTsFileEventWithErrorMessage(errorMessage); + return generateSubscriptionPollErrorResponse(errorMessage); } } @@ -273,7 +273,7 @@ public SubscriptionTsFileEvent poll(final String consumerId) { consumerIdToCurrentEventMap.put(consumerId, newEvent); if (newEventWithCommittable.getRight()) { // mark committable - uncommittedEvents.put(newEvent.getMessage().getCommitContext(), newEvent); + uncommittedEvents.put(newEvent.getResponse().getCommitContext(), newEvent); } newEvent.recordLastPolledConsumerId(consumerId); @@ -350,19 +350,18 @@ private synchronized SubscriptionTsFileEvent getPollableOnTheFlySubscriptionTsFi return null; } - private SubscriptionTsFileEvent generateSubscriptionTsFileEventWithErrorMessage( + private SubscriptionTsFileEvent generateSubscriptionPollErrorResponse( final String errorMessage, final boolean critical) { return new SubscriptionTsFileEvent( Collections.emptyList(), - new SubscriptionPolledMessage( - SubscriptionPolledMessageType.TS_FILE_ERROR.getType(), - new TsFileErrorMessagePayload(errorMessage, critical), + new SubscriptionPollResponse( + SubscriptionPollResponseType.ERROR.getType(), + new ErrorPayload(errorMessage, critical), super.generateInvalidSubscriptionCommitContext())); } - private SubscriptionTsFileEvent generateSubscriptionTsFileEventWithErrorMessage( - final String errorMessage) { + private SubscriptionTsFileEvent generateSubscriptionPollErrorResponse(final String errorMessage) { // consider non-critical by default, meaning the client can retry - return generateSubscriptionTsFileEventWithErrorMessage(errorMessage, false); + return generateSubscriptionPollErrorResponse(errorMessage, false); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionEvent.java index 19705470f651..c327b7e058d7 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionEvent.java @@ -21,7 +21,7 @@ import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.commons.subscription.config.SubscriptionConfig; -import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPolledMessage; +import org.apache.iotdb.rpc.subscription.payload.poll.SubscriptionPollResponse; import java.io.IOException; import java.nio.ByteBuffer; @@ -34,18 +34,18 @@ public class SubscriptionEvent { private static final long INVALID_TIMESTAMP = -1; protected final List enrichedEvents; - protected final SubscriptionPolledMessage message; + protected final SubscriptionPollResponse response; private String lastPolledConsumerId; private long lastPolledTimestamp; private long committedTimestamp; - protected ByteBuffer byteBuffer; // serialized SubscriptionPolledMessage + protected ByteBuffer byteBuffer; // serialized SubscriptionPollResponse public SubscriptionEvent( - final List enrichedEvents, final SubscriptionPolledMessage message) { + final List enrichedEvents, final SubscriptionPollResponse response) { this.enrichedEvents = enrichedEvents; - this.message = message; + this.response = response; this.lastPolledConsumerId = null; this.lastPolledTimestamp = INVALID_TIMESTAMP; @@ -56,8 +56,8 @@ public List getEnrichedEvents() { return enrichedEvents; } - public SubscriptionPolledMessage getMessage() { - return message; + public SubscriptionPollResponse getResponse() { + return response; } //////////////////////////// commit //////////////////////////// @@ -114,7 +114,7 @@ public ByteBuffer serialize() throws IOException { if (Objects.nonNull(byteBuffer)) { return byteBuffer; } - return SubscriptionPolledMessage.serialize(message); + return SubscriptionPollResponse.serialize(response); } public ByteBuffer getByteBuffer() { @@ -132,8 +132,8 @@ public void resetByteBuffer(final boolean recursive) { public String toString() { return "SubscriptionEvent{enrichedEvents=" + enrichedEvents.stream().map(EnrichedEvent::coreReportMessage).collect(Collectors.toList()) - + ", message=" - + message + + ", response=" + + response + ", lastPolledConsumerId=" + lastPolledConsumerId + ", lastPolledTimestamp=" @@ -141,7 +141,7 @@ public String toString() { + ", committedTimestamp=" + committedTimestamp + "}" - + "(message byte buffer size: " + + "(response event byte buffer size: " + (Objects.nonNull(byteBuffer) ? byteBuffer.limit() : "") + ")"; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionEventBinaryCache.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionEventBinaryCache.java index 478b5516b08b..c5a12942718c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionEventBinaryCache.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionEventBinaryCache.java @@ -43,13 +43,13 @@ public class SubscriptionEventBinaryCache { private final LoadingCache cache; - public ByteBuffer serialize(final SubscriptionEvent message) throws IOException { + public ByteBuffer serialize(final SubscriptionEvent event) throws IOException { try { - return this.cache.get(message); + return this.cache.get(event); } catch (final Exception e) { LOGGER.warn( - "SubscriptionMessageBinaryCache raised an exception while serializing message: {}", - message, + "SubscriptionEventBinaryCache raised an exception while serializing SubscriptionEvent: {}", + event, e); throw new IOException(e); } @@ -58,13 +58,15 @@ public ByteBuffer serialize(final SubscriptionEvent message) throws IOException /** * @return true -> byte buffer is not null */ - public boolean trySerialize(final SubscriptionEvent message) { + public boolean trySerialize(final SubscriptionEvent event) { try { - serialize(message); + serialize(event); return true; } catch (final IOException e) { LOGGER.warn( - "Subscription: something unexpected happened when serializing SubscriptionEvent", e); + "Subscription: something unexpected happened when serializing SubscriptionEvent: {}", + event, + e); return false; } } @@ -76,17 +78,17 @@ public void resetByteBuffer(final SubscriptionEvent message, final boolean recur //////////////////////////// singleton //////////////////////////// - private static class SubscriptionMessageBinaryCacheHolder { + private static class SubscriptionEventBinaryCacheHolder { private static final SubscriptionEventBinaryCache INSTANCE = new SubscriptionEventBinaryCache(); - private SubscriptionMessageBinaryCacheHolder() { + private SubscriptionEventBinaryCacheHolder() { // empty constructor } } public static SubscriptionEventBinaryCache getInstance() { - return SubscriptionEventBinaryCache.SubscriptionMessageBinaryCacheHolder.INSTANCE; + return SubscriptionEventBinaryCache.SubscriptionEventBinaryCacheHolder.INSTANCE; } private SubscriptionEventBinaryCache() { @@ -107,7 +109,7 @@ private SubscriptionEventBinaryCache() { memoryUsageCheatFactor.set( memoryUsageCheatFactor.get() * ((double) oldMemory / newMemory)); LOGGER.info( - "SubscriptionMessageBinaryCache.allocatedMemoryBlock has shrunk from {} to {}.", + "SubscriptionEventBinaryCache.allocatedMemoryBlock has shrunk from {} to {}.", oldMemory, newMemory); }) @@ -118,7 +120,7 @@ private SubscriptionEventBinaryCache() { memoryUsageCheatFactor.set( memoryUsageCheatFactor.get() / ((double) newMemory / oldMemory)); LOGGER.info( - "SubscriptionMessageBinaryCache.allocatedMemoryBlock has expanded from {} to {}.", + "SubscriptionEventBinaryCache.allocatedMemoryBlock has expanded from {} to {}.", oldMemory, newMemory); }); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionTsFileEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionTsFileEvent.java index 3c5d93649b86..222231fee043 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionTsFileEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/event/SubscriptionTsFileEvent.java @@ -22,13 +22,13 @@ import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.commons.subscription.config.SubscriptionConfig; import org.apache.iotdb.db.pipe.event.common.tsfile.PipeTsFileInsertionEvent; -import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionCommitContext; -import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionMessagePayload; -import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPolledMessage; -import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPolledMessageType; -import org.apache.iotdb.rpc.subscription.payload.common.TsFileInitMessagePayload; -import org.apache.iotdb.rpc.subscription.payload.common.TsFilePieceMessagePayload; -import org.apache.iotdb.rpc.subscription.payload.common.TsFileSealMessagePayload; +import org.apache.iotdb.rpc.subscription.payload.poll.FileInitPayload; +import org.apache.iotdb.rpc.subscription.payload.poll.FilePiecePayload; +import org.apache.iotdb.rpc.subscription.payload.poll.FileSealPayload; +import org.apache.iotdb.rpc.subscription.payload.poll.SubscriptionCommitContext; +import org.apache.iotdb.rpc.subscription.payload.poll.SubscriptionPollPayload; +import org.apache.iotdb.rpc.subscription.payload.poll.SubscriptionPollResponse; +import org.apache.iotdb.rpc.subscription.payload.poll.SubscriptionPollResponseType; import org.apache.tsfile.utils.Pair; import org.checkerframework.checker.nullness.qual.NonNull; @@ -50,8 +50,8 @@ public class SubscriptionTsFileEvent extends SubscriptionEvent { private final AtomicReference> nextEventWithCommittableRef; public SubscriptionTsFileEvent( - final List enrichedEvents, final SubscriptionPolledMessage message) { - super(enrichedEvents, message); + final List enrichedEvents, final SubscriptionPollResponse response) { + super(enrichedEvents, response); this.nextEventWithCommittableRef = new AtomicReference<>(); } @@ -65,16 +65,16 @@ public void prefetchNext() { return nextEventWithCommittable; } - final SubscriptionPolledMessage polledMessage = this.getMessage(); - final short messageType = polledMessage.getMessageType(); - final SubscriptionMessagePayload messagePayload = polledMessage.getMessagePayload(); - if (!SubscriptionPolledMessageType.isValidatedMessageType(messageType)) { - LOGGER.warn("unexpected message type: {}", messageType); + final SubscriptionPollResponse polledMessage = this.getResponse(); + final short responseType = polledMessage.getResponseType(); + final SubscriptionPollPayload eventPayload = polledMessage.getPayload(); + if (!SubscriptionPollResponseType.isValidatedResponseType(responseType)) { + LOGGER.warn("unexpected response type: {}", responseType); return null; } - switch (SubscriptionPolledMessageType.valueOf(messageType)) { - case TS_FILE_INIT: + switch (SubscriptionPollResponseType.valueOf(responseType)) { + case FILE_INIT: try { return generateSubscriptionTsFileEventWithPieceOrSealPayload(0); } catch (final IOException e) { @@ -84,10 +84,10 @@ public void prefetchNext() { e); return null; } - case TS_FILE_PIECE: + case FILE_PIECE: try { return generateSubscriptionTsFileEventWithPieceOrSealPayload( - ((TsFilePieceMessagePayload) messagePayload).getNextWritingOffset()); + ((FilePiecePayload) eventPayload).getNextWritingOffset()); } catch (final IOException e) { LOGGER.warn( "IOException occurred when prefetching next SubscriptionTsFileEvent, current SubscriptionTsFileEvent: {}", @@ -95,11 +95,11 @@ public void prefetchNext() { e); return null; } - case TS_FILE_SEAL: + case FILE_SEAL: // not need to prefetch return null; default: - LOGGER.warn("unexpected message type: {}", messageType); + LOGGER.warn("unexpected message type: {}", responseType); return null; } }); @@ -128,33 +128,32 @@ public void serializeNext() { return null; } - final SubscriptionPolledMessage polledMessage = this.getMessage(); - final short messageType = polledMessage.getMessageType(); - final SubscriptionMessagePayload messagePayload = polledMessage.getMessagePayload(); - if (!SubscriptionPolledMessageType.isValidatedMessageType(messageType)) { - LOGGER.warn("unexpected message type: {}", messageType); + final SubscriptionPollResponse polledMessage = this.getResponse(); + final short responseType = polledMessage.getResponseType(); + final SubscriptionPollPayload eventPayload = polledMessage.getPayload(); + if (!SubscriptionPollResponseType.isValidatedResponseType(responseType)) { + LOGGER.warn("unexpected response type: {}", responseType); return null; } - switch (SubscriptionPolledMessageType.valueOf(messageType)) { - case TS_FILE_INIT: + switch (SubscriptionPollResponseType.valueOf(responseType)) { + case FILE_INIT: if (Objects.equals(writingOffset, 0)) { return nextEventWithCommittable; } // reset next SubscriptionTsFileEvent return null; - case TS_FILE_PIECE: + case FILE_PIECE: if (Objects.equals( - writingOffset, - ((TsFilePieceMessagePayload) messagePayload).getNextWritingOffset())) { + writingOffset, ((FilePiecePayload) eventPayload).getNextWritingOffset())) { return nextEventWithCommittable; } // reset next SubscriptionTsFileEvent return null; - case TS_FILE_SEAL: + case FILE_SEAL: return null; default: - LOGGER.warn("unexpected message type: {}", messageType); + LOGGER.warn("unexpected message type: {}", responseType); return null; } }); @@ -165,16 +164,16 @@ public static SubscriptionTsFileEvent generateSubscriptionTsFileEventWithInitPay final SubscriptionCommitContext commitContext) { return new SubscriptionTsFileEvent( Collections.singletonList(tsFileInsertionEvent), - new SubscriptionPolledMessage( - SubscriptionPolledMessageType.TS_FILE_INIT.getType(), - new TsFileInitMessagePayload(tsFileInsertionEvent.getTsFile().getName()), + new SubscriptionPollResponse( + SubscriptionPollResponseType.FILE_INIT.getType(), + new FileInitPayload(tsFileInsertionEvent.getTsFile().getName()), commitContext)); } public SubscriptionTsFileEvent generateSubscriptionTsFileEventWithInitPayload() { return generateSubscriptionTsFileEventWithInitPayload( (PipeTsFileInsertionEvent) this.getEnrichedEvents().get(0), - this.getMessage().getCommitContext()); + this.getResponse().getCommitContext()); } public @NonNull Pair<@NonNull SubscriptionTsFileEvent, Boolean> @@ -182,7 +181,7 @@ public SubscriptionTsFileEvent generateSubscriptionTsFileEventWithInitPayload() throws IOException { final PipeTsFileInsertionEvent tsFileInsertionEvent = (PipeTsFileInsertionEvent) this.getEnrichedEvents().get(0); - final SubscriptionCommitContext commitContext = this.getMessage().getCommitContext(); + final SubscriptionCommitContext commitContext = this.getResponse().getCommitContext(); final int readFileBufferSize = SubscriptionConfig.getInstance().getSubscriptionReadFileBufferSize(); @@ -205,9 +204,9 @@ public SubscriptionTsFileEvent generateSubscriptionTsFileEventWithInitPayload() return new Pair<>( new SubscriptionTsFileEvent( Collections.singletonList(tsFileInsertionEvent), - new SubscriptionPolledMessage( - SubscriptionPolledMessageType.TS_FILE_PIECE.getType(), - new TsFilePieceMessagePayload( + new SubscriptionPollResponse( + SubscriptionPollResponseType.FILE_PIECE.getType(), + new FilePiecePayload( tsFileInsertionEvent.getTsFile().getName(), writingOffset + readLength, filePiece), @@ -219,9 +218,9 @@ public SubscriptionTsFileEvent generateSubscriptionTsFileEventWithInitPayload() return new Pair<>( new SubscriptionTsFileEvent( Collections.singletonList(tsFileInsertionEvent), - new SubscriptionPolledMessage( - SubscriptionPolledMessageType.TS_FILE_SEAL.getType(), - new TsFileSealMessagePayload( + new SubscriptionPollResponse( + SubscriptionPollResponseType.FILE_SEAL.getType(), + new FileSealPayload( tsFileInsertionEvent.getTsFile().getName(), tsFileInsertionEvent.getTsFile().length()), commitContext)), diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java index 8b63000dda91..16d157e18479 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/receiver/SubscriptionReceiverV1.java @@ -41,12 +41,12 @@ import org.apache.iotdb.rpc.TSStatusCode; import org.apache.iotdb.rpc.subscription.config.ConsumerConfig; import org.apache.iotdb.rpc.subscription.exception.SubscriptionException; -import org.apache.iotdb.rpc.subscription.payload.common.PollMessagePayload; -import org.apache.iotdb.rpc.subscription.payload.common.PollTsFileMessagePayload; -import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionCommitContext; -import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPollMessage; -import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPollMessageType; -import org.apache.iotdb.rpc.subscription.payload.common.SubscriptionPolledMessage; +import org.apache.iotdb.rpc.subscription.payload.poll.PollFilePayload; +import org.apache.iotdb.rpc.subscription.payload.poll.PollPayload; +import org.apache.iotdb.rpc.subscription.payload.poll.SubscriptionCommitContext; +import org.apache.iotdb.rpc.subscription.payload.poll.SubscriptionPollRequest; +import org.apache.iotdb.rpc.subscription.payload.poll.SubscriptionPollRequestType; +import org.apache.iotdb.rpc.subscription.payload.poll.SubscriptionPollResponse; import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribeCloseReq; import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribeCommitReq; import org.apache.iotdb.rpc.subscription.payload.request.PipeSubscribeHandshakeReq; @@ -321,19 +321,18 @@ private TPipeSubscribeResp handlePipeSubscribePoll(final PipeSubscribePollReq re final List events; try { - final SubscriptionPollMessage pollMessage = req.getPollMessage(); - final short messageType = pollMessage.getMessageType(); - if (SubscriptionPollMessageType.isValidatedMessageType(messageType)) { - switch (SubscriptionPollMessageType.valueOf(messageType)) { + final SubscriptionPollRequest request = req.getRequest(); + final short requestType = request.getRequestType(); + if (SubscriptionPollRequestType.isValidatedRequestType(requestType)) { + switch (SubscriptionPollRequestType.valueOf(requestType)) { case POLL: events = - handlePipeSubscribePollInternal( - consumerConfig, (PollMessagePayload) pollMessage.getMessagePayload()); + handlePipeSubscribePollInternal(consumerConfig, (PollPayload) request.getPayload()); break; - case POLL_TS_FILE: + case POLL_FILE: events = handlePipeSubscribePollTsFileInternal( - consumerConfig, (PollTsFileMessagePayload) pollMessage.getMessagePayload()); + consumerConfig, (PollFilePayload) request.getPayload()); break; default: events = null; @@ -343,7 +342,7 @@ private TPipeSubscribeResp handlePipeSubscribePoll(final PipeSubscribePollReq re events = null; } if (Objects.isNull(events)) { - throw new SubscriptionException(String.format("unexpected message type: %s", messageType)); + throw new SubscriptionException(String.format("unexpected request type: %s", requestType)); } // generate response @@ -352,8 +351,8 @@ private TPipeSubscribeResp handlePipeSubscribePoll(final PipeSubscribePollReq re events.parallelStream() .map( (event) -> { - final SubscriptionPolledMessage message = event.getMessage(); - final SubscriptionCommitContext commitContext = message.getCommitContext(); + final SubscriptionPollResponse response = event.getResponse(); + final SubscriptionCommitContext commitContext = response.getCommitContext(); try { final ByteBuffer byteBuffer = SubscriptionEventBinaryCache.getInstance().serialize(event); @@ -364,23 +363,23 @@ private TPipeSubscribeResp handlePipeSubscribePoll(final PipeSubscribePollReq re byteBuffer.limit()); SubscriptionEventBinaryCache.getInstance().resetByteBuffer(event, false); LOGGER.info( - "Subscription: consumer {} poll message {} successfully with req message: {}", + "Subscription: consumer {} poll {} successfully with request: {}", consumerConfig, - message, - req.getPollMessage()); + response, + req.getRequest()); return byteBuffer; } catch (final Exception e) { LOGGER.warn( - "Subscription: consumer {} poll message {} failed with req message: {}", + "Subscription: consumer {} poll {} failed with request: {}", consumerConfig, - message, - req.getPollMessage(), + response, + req.getRequest(), e); // nack SubscriptionAgent.broker() .commit( consumerConfig, - Collections.singletonList(message.getCommitContext()), + Collections.singletonList(response.getCommitContext()), true); return null; } @@ -399,7 +398,7 @@ private TPipeSubscribeResp handlePipeSubscribePoll(final PipeSubscribePollReq re } private List handlePipeSubscribePollInternal( - final ConsumerConfig consumerConfig, final PollMessagePayload messagePayload) { + final ConsumerConfig consumerConfig, final PollPayload messagePayload) { final Set topicNames; if (messagePayload.getTopicNames().isEmpty()) { // poll all subscribed topics @@ -418,7 +417,7 @@ private List handlePipeSubscribePollInternal( } private List handlePipeSubscribePollTsFileInternal( - final ConsumerConfig consumerConfig, final PollTsFileMessagePayload messagePayload) { + final ConsumerConfig consumerConfig, final PollFilePayload messagePayload) { return SubscriptionAgent.broker() .pollTsFile( consumerConfig, From 62a1739c78c980a8740f931510aa41c5e2147091 Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Mon, 20 May 2024 23:03:03 +0800 Subject: [PATCH 83/93] close SubscriptionProvider when constructProviderAndHandshake throw exception --- .../session/subscription/SubscriptionConsumer.java | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java index 5463522327f1..054eb36ec432 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java @@ -358,11 +358,20 @@ private void shutdownEndpointsSyncer() { /////////////////////////////// subscription provider /////////////////////////////// SubscriptionProvider constructProviderAndHandshake(final TEndPoint endPoint) - throws SubscriptionException, IoTDBConnectionException { + throws SubscriptionException { final SubscriptionProvider provider = new SubscriptionProvider( endPoint, this.username, this.password, this.consumerId, this.consumerGroupId); - provider.handshake(); + try { + provider.handshake(); + } catch (final Exception e) { + try { + provider.close(); + } catch (final Exception ignored) { + } + throw new SubscriptionConnectionException( + String.format("Failed to handshake with subscription provider %s", provider)); + } // update consumer id and consumer group id if not exist if (Objects.isNull(this.consumerId)) { From c424b19f120bb178e607649ee187f766c2d565a6 Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Mon, 20 May 2024 23:57:13 +0800 Subject: [PATCH 84/93] add file ops for SubscriptionFileHandler --- .../it/local/IoTDBSubscriptionBasicIT.java | 33 ++++++++- .../payload/SubscriptionFileHandler.java | 72 +++++++++++++++++++ 2 files changed, 103 insertions(+), 2 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionBasicIT.java b/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionBasicIT.java index 8e4c873b4933..9911e4b53cc9 100644 --- a/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionBasicIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionBasicIT.java @@ -30,8 +30,10 @@ import org.apache.iotdb.session.subscription.SubscriptionPullConsumer; import org.apache.iotdb.session.subscription.SubscriptionPushConsumer; import org.apache.iotdb.session.subscription.SubscriptionSession; +import org.apache.iotdb.session.subscription.payload.SubscriptionFileHandler; import org.apache.iotdb.session.subscription.payload.SubscriptionMessage; import org.apache.iotdb.session.subscription.payload.SubscriptionSessionDataSet; +import org.apache.iotdb.session.subscription.payload.SubscriptionTsFileHandler; import org.apache.iotdb.subscription.it.IoTDBSubscriptionITConstant; import org.apache.tsfile.read.TsFileReader; @@ -48,7 +50,10 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.nio.file.Files; +import java.nio.file.NoSuchFileException; import java.time.Duration; +import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.Properties; @@ -189,6 +194,9 @@ public void testBasicSubscribeTsFile() throws Exception { fail(e.getMessage()); } + // Record file handlers + final List fileHandlers = new ArrayList<>(); + // Subscription final AtomicInteger rowCount = new AtomicInteger(); final AtomicBoolean isClosed = new AtomicBoolean(false); @@ -210,8 +218,9 @@ public void testBasicSubscribeTsFile() throws Exception { final List messages = consumer.poll(IoTDBSubscriptionITConstant.POLL_TIMEOUT_MS); for (final SubscriptionMessage message : messages) { - try (final TsFileReader tsFileReader = - message.getTsFileHandler().openReader()) { + final SubscriptionTsFileHandler tsFileHandler = message.getTsFileHandler(); + fileHandlers.add(tsFileHandler); + try (final TsFileReader tsFileReader = tsFileHandler.openReader()) { final Path path = new Path("root.db.d1", "s1", true); final QueryDataSet dataSet = tsFileReader.query( @@ -250,6 +259,26 @@ public void testBasicSubscribeTsFile() throws Exception { isClosed.set(true); thread.join(); } + + // Do something for file handlers + Assert.assertFalse(fileHandlers.isEmpty()); + final SubscriptionFileHandler fileHandler = fileHandlers.get(0); + final java.nio.file.Path filePath = fileHandler.getPath(); + Assert.assertTrue(Files.exists(filePath)); + + // Copy file + java.nio.file.Path tmpFilePath; + tmpFilePath = fileHandler.copyFile(Files.createTempFile(null, null).toAbsolutePath()); + Assert.assertTrue(Files.exists(filePath)); + Assert.assertTrue(Files.exists(tmpFilePath)); + + // Move file + tmpFilePath = fileHandler.moveFile(Files.createTempFile(null, null).toAbsolutePath()); + Assert.assertFalse(Files.exists(filePath)); + Assert.assertTrue(Files.exists(tmpFilePath)); + + // Delete file + Assert.assertThrows(NoSuchFileException.class, fileHandler::deleteFile); } @Test diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionFileHandler.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionFileHandler.java index 0e8237f53376..3b8c7993367e 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionFileHandler.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionFileHandler.java @@ -21,6 +21,13 @@ import org.apache.iotdb.rpc.subscription.exception.SubscriptionIncompatibleHandlerException; +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.nio.file.StandardCopyOption; + public abstract class SubscriptionFileHandler implements SubscriptionMessageHandler { protected final String filePath; @@ -29,6 +36,71 @@ public SubscriptionFileHandler(final String filePath) { this.filePath = filePath; } + /** + * @return a new File instance of the corresponding file + */ + public File getFile() { + return new File(filePath); + } + + /** + * @return a new Path instance of the corresponding file + */ + public Path getPath() { + return Paths.get(filePath); + } + + /** + * @throws IOException if an I/O error occurs + */ + public void deleteFile() throws IOException { + Files.delete(getPath()); + } + + /** + * @param targetFilePath the path to the target file + * @return the path to the target file + * @throws IOException if an I/O error occurs + */ + public Path moveFile(final String targetFilePath) throws IOException { + return this.moveFile(Paths.get(targetFilePath)); + } + + /** + * @param targetFilePath the path to the target file + * @return the path to the target file + * @throws IOException if an I/O error occurs + */ + public Path moveFile(final Path targetFilePath) throws IOException { + return Files.move( + getPath(), + targetFilePath, + StandardCopyOption.REPLACE_EXISTING, + StandardCopyOption.ATOMIC_MOVE); + } + + /** + * @param targetFilePath the path to the target file + * @return the path to the target file + * @throws IOException if an I/O error occurs + */ + public Path copyFile(final String targetFilePath) throws IOException { + return this.copyFile(Paths.get(targetFilePath)); + } + + /** + * @param targetFilePath the path to the target file + * @return the path to the target file + * @throws IOException if an I/O error occurs + */ + public Path copyFile(final Path targetFilePath) throws IOException { + return Files.copy( + getPath(), + targetFilePath, + StandardCopyOption.REPLACE_EXISTING, + StandardCopyOption.COPY_ATTRIBUTES); + } + @Override public SubscriptionSessionDataSetsHandler getSessionDataSetsHandler() { throw new SubscriptionIncompatibleHandlerException( From b48c809d64b5102b603939f2ed5bdb88d39999fa Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Tue, 21 May 2024 00:48:28 +0800 Subject: [PATCH 85/93] intro future style commit async API --- .../subscription/AsyncCommitCallback.java | 2 +- .../subscription/SubscriptionConsumer.java | 105 +++++++++++------- .../SubscriptionPullConsumer.java | 9 +- 3 files changed, 68 insertions(+), 48 deletions(-) diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/AsyncCommitCallback.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/AsyncCommitCallback.java index 0d538dfd6715..52e0eb7e0916 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/AsyncCommitCallback.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/AsyncCommitCallback.java @@ -24,7 +24,7 @@ default void onComplete() { // Do nothing } - default void onFailure(Throwable e) { + default void onFailure(final Throwable e) { // Do nothing } } diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java index 054eb36ec432..6cf84e47c3ec 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java @@ -66,6 +66,7 @@ import java.util.Optional; import java.util.Properties; import java.util.Set; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; @@ -88,7 +89,7 @@ public abstract class SubscriptionConsumer implements AutoCloseable { private final long heartbeatIntervalMs; private final long endpointsSyncIntervalMs; - private SubscriptionProviders subscriptionProviders; + private final SubscriptionProviders subscriptionProviders; private ScheduledExecutorService heartbeatWorkerExecutor; private ScheduledExecutorService endpointsSyncerExecutor; @@ -230,8 +231,11 @@ public synchronized void close() throws SubscriptionException, IoTDBConnectionEx // shutdown endpoints syncer shutdownEndpointsSyncer(); - // shutdown workers: heartbeat worker and async commit executor - shutdownWorkers(); + // shutdown heartbeat worker + shutdownHeartbeatWorker(); + + // shutdown async commit worker if needed + shutdownAsyncCommitWorkerIfNeeded(); // close subscription providers subscriptionProviders.acquireWriteLock(); @@ -311,18 +315,9 @@ private void launchHeartbeatWorker() { TimeUnit.MILLISECONDS); } - /** - * Shut down workers upon close. There are currently two workers: heartbeat worker and async - * commit executor. - */ - private void shutdownWorkers() { + private void shutdownHeartbeatWorker() { heartbeatWorkerExecutor.shutdown(); heartbeatWorkerExecutor = null; - - if (asyncCommitExecutor != null) { - asyncCommitExecutor.shutdown(); - asyncCommitExecutor = null; - } } /////////////////////////////// sync endpoints /////////////////////////////// @@ -735,42 +730,31 @@ private void commitInternal( /////////////////////////////// commit async /////////////////////////////// - protected void commitAsync(final Iterable messages) { - commitAsync(messages, new AsyncCommitCallback() {}); - } - protected void commitAsync( final Iterable messages, final AsyncCommitCallback callback) { - // Initiate executor if needed - if (asyncCommitExecutor == null) { - synchronized (this) { - if (asyncCommitExecutor != null) { - return; - } - - asyncCommitExecutor = - Executors.newSingleThreadExecutor( - r -> { - final Thread t = - new Thread( - Thread.currentThread().getThreadGroup(), - r, - "SubscriptionConsumerAsyncCommitWorker", - 0); - if (!t.isDaemon()) { - t.setDaemon(true); - } - if (t.getPriority() != Thread.NORM_PRIORITY) { - t.setPriority(Thread.NORM_PRIORITY); - } - return t; - }); - } - } + // launch async commit worker if needed + launchAsyncCommitWorkerIfNeeded(); asyncCommitExecutor.submit(new AsyncCommitWorker(messages, callback)); } + protected CompletableFuture commitAsync(final Iterable messages) { + // launch async commit worker if needed + launchAsyncCommitWorkerIfNeeded(); + + final CompletableFuture future = new CompletableFuture<>(); + asyncCommitExecutor.submit( + () -> { + try { + ack(messages); + future.complete(null); + } catch (final Throwable e) { + future.completeExceptionally(e); + } + }); + return future; + } + /////////////////////////////// redirection /////////////////////////////// private void subscribeWithRedirection(final Set topicNames) throws SubscriptionException { @@ -937,6 +921,41 @@ public Builder fileSaveDir(final String fileSaveDir) { /////////////////////////////// commit async worker /////////////////////////////// + private void launchAsyncCommitWorkerIfNeeded() { + if (asyncCommitExecutor == null) { + synchronized (this) { + if (asyncCommitExecutor != null) { + return; + } + + asyncCommitExecutor = + Executors.newSingleThreadExecutor( + r -> { + final Thread t = + new Thread( + Thread.currentThread().getThreadGroup(), + r, + "SubscriptionConsumerAsyncCommitWorker", + 0); + if (!t.isDaemon()) { + t.setDaemon(true); + } + if (t.getPriority() != Thread.NORM_PRIORITY) { + t.setPriority(Thread.NORM_PRIORITY); + } + return t; + }); + } + } + } + + private void shutdownAsyncCommitWorkerIfNeeded() { + if (asyncCommitExecutor != null) { + asyncCommitExecutor.shutdown(); + asyncCommitExecutor = null; + } + } + class AsyncCommitWorker implements Runnable { private final Iterable messages; private final AsyncCommitCallback callback; diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java index a14e67c820b9..65a6b97030e9 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java @@ -34,6 +34,7 @@ import java.util.Properties; import java.util.Set; import java.util.SortedMap; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.ConcurrentSkipListSet; import java.util.concurrent.Executors; @@ -170,12 +171,12 @@ public void commitSync(final Iterable messages) super.ack(messages); } - public void commitAsync(final SubscriptionMessage message) { - super.commitAsync(Collections.singletonList(message)); + public CompletableFuture commitAsync(final SubscriptionMessage message) { + return super.commitAsync(Collections.singletonList(message)); } - public void commitAsync(final Iterable messages) { - super.commitAsync(messages); + public CompletableFuture commitAsync(final Iterable messages) { + return super.commitAsync(messages); } public void commitAsync(final SubscriptionMessage message, final AsyncCommitCallback callback) { From bfed7f368a3010dd829e7a448b7d5d3a0cc8407e Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Tue, 21 May 2024 11:43:39 +0800 Subject: [PATCH 86/93] fix AtomicMoveNotSupportedException --- .../subscription/payload/SubscriptionFileHandler.java | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionFileHandler.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionFileHandler.java index 3b8c7993367e..a75d969fce61 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionFileHandler.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionFileHandler.java @@ -72,11 +72,7 @@ public Path moveFile(final String targetFilePath) throws IOException { * @throws IOException if an I/O error occurs */ public Path moveFile(final Path targetFilePath) throws IOException { - return Files.move( - getPath(), - targetFilePath, - StandardCopyOption.REPLACE_EXISTING, - StandardCopyOption.ATOMIC_MOVE); + return Files.move(getPath(), targetFilePath, StandardCopyOption.REPLACE_EXISTING); } /** From 79014bff5ded4ba3ed9ac6bbcfa020a88f1c1e2d Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Tue, 21 May 2024 18:30:20 +0800 Subject: [PATCH 87/93] fixup --- .../iotdb/db/subscription/broker/SubscriptionBroker.java | 4 ++-- .../subscription/broker/SubscriptionPrefetchingQueue.java | 6 +++--- .../broker/SubscriptionPrefetchingTabletsQueue.java | 4 ++-- .../broker/SubscriptionPrefetchingTsFileQueue.java | 4 ++-- 4 files changed, 9 insertions(+), 9 deletions(-) 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 f7fc08af5c7e..c0224b86c363 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.agent.SubscriptionAgent; import org.apache.iotdb.db.subscription.event.SubscriptionEvent; import org.apache.iotdb.db.subscription.metric.SubscriptionPrefetchingQueueMetrics; @@ -133,7 +133,7 @@ public List commit( /////////////////////////////// prefetching queue /////////////////////////////// public void bindPrefetchingQueue( - final String topicName, final BoundedBlockingPendingQueue inputPendingQueue) { + final String topicName, final 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 1818880f6210..43aef1956814 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 @@ -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.conf.IoTDBDescriptor; import org.apache.iotdb.db.pipe.agent.PipeAgent; import org.apache.iotdb.db.subscription.event.SubscriptionEvent; @@ -41,7 +41,7 @@ public abstract class SubscriptionPrefetchingQueue { protected final String brokerId; // consumer group id protected final String topicName; - protected final BoundedBlockingPendingQueue inputPendingQueue; + protected final UnboundedBlockingPendingQueue inputPendingQueue; protected final Map uncommittedEvents; private final AtomicLong subscriptionCommitIdGenerator = new AtomicLong(0); @@ -49,7 +49,7 @@ public abstract class SubscriptionPrefetchingQueue { public SubscriptionPrefetchingQueue( final String brokerId, final String topicName, - final BoundedBlockingPendingQueue inputPendingQueue) { + final 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/broker/SubscriptionPrefetchingTabletsQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTabletsQueue.java index 0b15133a36da..195194ef3c89 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTabletsQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTabletsQueue.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.pipe.event.UserDefinedEnrichedEvent; import org.apache.iotdb.db.pipe.event.common.tablet.PipeInsertNodeTabletInsertionEvent; @@ -57,7 +57,7 @@ public class SubscriptionPrefetchingTabletsQueue extends SubscriptionPrefetching public SubscriptionPrefetchingTabletsQueue( final String brokerId, final String topicName, - final BoundedBlockingPendingQueue inputPendingQueue) { + final UnboundedBlockingPendingQueue inputPendingQueue) { super(brokerId, topicName, inputPendingQueue); this.prefetchingQueue = new LinkedBlockingQueue<>(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java index 80b834a93466..a40a225c6a16 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.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.db.pipe.event.UserDefinedEnrichedEvent; import org.apache.iotdb.db.pipe.event.common.tsfile.PipeTsFileInsertionEvent; import org.apache.iotdb.db.subscription.event.SubscriptionTsFileEvent; @@ -56,7 +56,7 @@ public class SubscriptionPrefetchingTsFileQueue extends SubscriptionPrefetchingQ public SubscriptionPrefetchingTsFileQueue( final String brokerId, final String topicName, - final BoundedBlockingPendingQueue inputPendingQueue) { + final UnboundedBlockingPendingQueue inputPendingQueue) { super(brokerId, topicName, inputPendingQueue); this.consumerIdToCurrentEventMap = new ConcurrentHashMap<>(); From d8520196dafa53bbe650802ff35442339ea2d646 Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Wed, 22 May 2024 00:57:54 +0800 Subject: [PATCH 88/93] rename format in topic constant --- .../java/org/apache/iotdb/SubscriptionSessionExample.java | 2 +- .../it/dual/IoTDBSubscriptionConsumerGroupIT.java | 2 +- .../subscription/it/dual/IoTDBSubscriptionTopicIT.java | 6 +++--- .../subscription/it/local/IoTDBSubscriptionBasicIT.java | 2 +- .../apache/iotdb/rpc/subscription/config/TopicConfig.java | 4 ++-- .../iotdb/rpc/subscription/config/TopicConstant.java | 7 ++++--- .../iotdb/db/subscription/broker/SubscriptionBroker.java | 2 +- 7 files changed, 13 insertions(+), 12 deletions(-) diff --git a/example/session/src/main/java/org/apache/iotdb/SubscriptionSessionExample.java b/example/session/src/main/java/org/apache/iotdb/SubscriptionSessionExample.java index bde6014e0c46..f836ed324c91 100644 --- a/example/session/src/main/java/org/apache/iotdb/SubscriptionSessionExample.java +++ b/example/session/src/main/java/org/apache/iotdb/SubscriptionSessionExample.java @@ -156,7 +156,7 @@ private static void subscriptionExample2() throws Exception { try (final SubscriptionSession subscriptionSession = new SubscriptionSession(HOST, PORT)) { subscriptionSession.open(); final Properties config = new Properties(); - config.put(TopicConstant.FORMAT_KEY, TopicConstant.FORMAT_TS_FILE_READER_VALUE); + config.put(TopicConstant.FORMAT_KEY, TopicConstant.FORMAT_TS_FILE_HANDLER_VALUE); subscriptionSession.createTopic(TOPIC_2, config); } diff --git a/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionConsumerGroupIT.java b/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionConsumerGroupIT.java index b472628798f1..827abbc26605 100644 --- a/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionConsumerGroupIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionConsumerGroupIT.java @@ -830,7 +830,7 @@ private void createTopics(final long currentTime) { } { final Properties config = new Properties(); - config.put(TopicConstant.FORMAT_KEY, TopicConstant.FORMAT_TS_FILE_READER_VALUE); + config.put(TopicConstant.FORMAT_KEY, TopicConstant.FORMAT_TS_FILE_HANDLER_VALUE); session.createTopic("all", config); } } catch (final Exception e) { diff --git a/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionTopicIT.java b/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionTopicIT.java index edf36030840b..b6994b7eaf62 100644 --- a/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionTopicIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionTopicIT.java @@ -526,19 +526,19 @@ public void testTopicInvalidConfig() throws Exception { final List configs = new ArrayList<>(); { final Properties config = new Properties(); - config.put(TopicConstant.FORMAT_KEY, TopicConstant.FORMAT_TS_FILE_READER_VALUE); + config.put(TopicConstant.FORMAT_KEY, TopicConstant.FORMAT_TS_FILE_HANDLER_VALUE); config.put(TopicConstant.PATH_KEY, "root.db.*.s"); configs.add(config); } { final Properties config = new Properties(); - config.put(TopicConstant.FORMAT_KEY, TopicConstant.FORMAT_TS_FILE_READER_VALUE); + config.put(TopicConstant.FORMAT_KEY, TopicConstant.FORMAT_TS_FILE_HANDLER_VALUE); config.put(TopicConstant.START_TIME_KEY, System.currentTimeMillis()); configs.add(config); } { final Properties config = new Properties(); - config.put(TopicConstant.FORMAT_KEY, TopicConstant.FORMAT_TS_FILE_READER_VALUE); + config.put(TopicConstant.FORMAT_KEY, TopicConstant.FORMAT_TS_FILE_HANDLER_VALUE); config.put("processor", "tumbling-time-sampling-processor"); config.put("processor.tumbling-time.interval-seconds", "1"); config.put("processor.down-sampling.split-file", "true"); diff --git a/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionBasicIT.java b/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionBasicIT.java index 9911e4b53cc9..88a8c15dbd18 100644 --- a/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionBasicIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/IoTDBSubscriptionBasicIT.java @@ -187,7 +187,7 @@ public void testBasicSubscribeTsFile() throws Exception { try (final SubscriptionSession session = new SubscriptionSession(host, port)) { session.open(); final Properties config = new Properties(); - config.put(TopicConstant.FORMAT_KEY, TopicConstant.FORMAT_TS_FILE_READER_VALUE); + config.put(TopicConstant.FORMAT_KEY, TopicConstant.FORMAT_TS_FILE_HANDLER_VALUE); session.createTopic(topicName, config); } catch (final Exception e) { e.printStackTrace(); diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/config/TopicConfig.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/config/TopicConfig.java index 92b90e1b02b3..d42b254d9798 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/config/TopicConfig.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/config/TopicConfig.java @@ -83,7 +83,7 @@ public Map getAttributesWithTimeRange(long creationTime) { } public Map getAttributesWithSourceRealtimeMode() { - if (TopicConstant.FORMAT_TS_FILE_READER_VALUE.equals( + if (TopicConstant.FORMAT_TS_FILE_HANDLER_VALUE.equals( attributes.getOrDefault(TopicConstant.FORMAT_KEY, TopicConstant.FORMAT_DEFAULT_VALUE))) { return Collections.singletonMap("realtime.mode", "batch"); } @@ -103,7 +103,7 @@ public Map getAttributesWithProcessorPrefix() { } public boolean isValid() { - if (!TopicConstant.FORMAT_TS_FILE_READER_VALUE.equals( + if (!TopicConstant.FORMAT_TS_FILE_HANDLER_VALUE.equals( attributes.getOrDefault(TopicConstant.FORMAT_KEY, TopicConstant.FORMAT_DEFAULT_VALUE))) { return true; } diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/config/TopicConstant.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/config/TopicConstant.java index 27f3d771117c..6a193aba1629 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/config/TopicConstant.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/config/TopicConstant.java @@ -31,9 +31,10 @@ public class TopicConstant { public static final String NOW_TIME_VALUE = "now"; public static final String FORMAT_KEY = "format"; - public static final String FORMAT_SESSION_DATA_SET_VALUE = "SessionDataSet"; - public static final String FORMAT_TS_FILE_READER_VALUE = "TsFileReader"; - public static final String FORMAT_DEFAULT_VALUE = FORMAT_SESSION_DATA_SET_VALUE; + public static final String FORMAT_SESSION_DATA_SETS_HANDLER_VALUE = + "SubscriptionSessionDataSetsHandler"; + public static final String FORMAT_TS_FILE_HANDLER_VALUE = "SubscriptionTsFileHandler"; + public static final String FORMAT_DEFAULT_VALUE = FORMAT_SESSION_DATA_SETS_HANDLER_VALUE; private TopicConstant() { throw new IllegalStateException("Utility class"); 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 c0224b86c363..90785c1602e6 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 @@ -142,7 +142,7 @@ public void bindPrefetchingQueue( return; } final String topicFormat = SubscriptionAgent.topic().getTopicFormat(topicName); - if (TopicConstant.FORMAT_TS_FILE_READER_VALUE.equals(topicFormat)) { + if (TopicConstant.FORMAT_TS_FILE_HANDLER_VALUE.equals(topicFormat)) { final SubscriptionPrefetchingQueue queue = new SubscriptionPrefetchingTsFileQueue(brokerId, topicName, inputPendingQueue); SubscriptionPrefetchingQueueMetrics.getInstance().register(queue); From 18c8c677252eccdc897a2b98f177c7e13c58eac8 Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Thu, 23 May 2024 15:07:55 +0800 Subject: [PATCH 89/93] downgrade consumer close ex to log --- .../subscription/SubscriptionConsumer.java | 11 ++--- .../subscription/SubscriptionProviders.java | 10 +++-- .../SubscriptionPullConsumer.java | 5 +-- .../SubscriptionPushConsumer.java | 5 +-- .../payload/SubscriptionFileHandler.java | 44 +++++++++---------- .../payload/SubscriptionMessage.java | 5 ++- .../payload/SubscriptionTsFileHandler.java | 6 +-- 7 files changed, 42 insertions(+), 44 deletions(-) diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java index 6cf84e47c3ec..133e8f1c5187 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java @@ -21,7 +21,6 @@ import org.apache.iotdb.common.rpc.thrift.TEndPoint; import org.apache.iotdb.isession.SessionConfig; -import org.apache.iotdb.rpc.IoTDBConnectionException; import org.apache.iotdb.rpc.subscription.config.ConsumerConstant; import org.apache.iotdb.rpc.subscription.exception.SubscriptionConnectionException; import org.apache.iotdb.rpc.subscription.exception.SubscriptionException; @@ -198,7 +197,7 @@ protected SubscriptionConsumer(final Builder builder, final Properties propertie /////////////////////////////// open & close /////////////////////////////// - public synchronized void open() throws SubscriptionException, IoTDBConnectionException { + public synchronized void open() throws SubscriptionException { if (!isClosed.get()) { return; } @@ -206,8 +205,7 @@ public synchronized void open() throws SubscriptionException, IoTDBConnectionExc // open subscription providers subscriptionProviders.acquireWriteLock(); try { - subscriptionProviders.openProviders( - this); // throw SubscriptionException or IoTDBConnectionException + subscriptionProviders.openProviders(this); // throw SubscriptionException } finally { subscriptionProviders.releaseWriteLock(); } @@ -222,7 +220,7 @@ public synchronized void open() throws SubscriptionException, IoTDBConnectionExc } @Override - public synchronized void close() throws SubscriptionException, IoTDBConnectionException { + public synchronized void close() { if (isClosed.get()) { return; } @@ -240,8 +238,7 @@ public synchronized void close() throws SubscriptionException, IoTDBConnectionEx // close subscription providers subscriptionProviders.acquireWriteLock(); try { - subscriptionProviders - .closeProviders(); // throw SubscriptionException or IoTDBConnectionException + subscriptionProviders.closeProviders(); } finally { subscriptionProviders.releaseWriteLock(); } diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionProviders.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionProviders.java index 092a36f34397..8038dc724d69 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionProviders.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionProviders.java @@ -73,8 +73,7 @@ void releaseWriteLock() { /////////////////////////////// CRUD /////////////////////////////// /** Caller should ensure that the method is called in the lock {@link #acquireWriteLock()}. */ - void openProviders(final SubscriptionConsumer consumer) - throws SubscriptionException, IoTDBConnectionException { + void openProviders(final SubscriptionConsumer consumer) throws SubscriptionException { // close stale providers closeProviders(); @@ -129,9 +128,12 @@ void openProviders(final SubscriptionConsumer consumer) } /** Caller should ensure that the method is called in the lock {@link #acquireWriteLock()}. */ - void closeProviders() throws SubscriptionException, IoTDBConnectionException { + void closeProviders() { for (final SubscriptionProvider provider : getAllProviders()) { - provider.close(); + try { + provider.close(); + } catch (final Exception ignored) { + } } subscriptionProviders.clear(); } diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java index 65a6b97030e9..25315f8c6244 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPullConsumer.java @@ -19,7 +19,6 @@ package org.apache.iotdb.session.subscription; -import org.apache.iotdb.rpc.IoTDBConnectionException; import org.apache.iotdb.rpc.subscription.config.ConsumerConstant; import org.apache.iotdb.rpc.subscription.exception.SubscriptionException; import org.apache.iotdb.session.subscription.payload.SubscriptionMessage; @@ -92,7 +91,7 @@ private SubscriptionPullConsumer( /////////////////////////////// open & close /////////////////////////////// - public synchronized void open() throws IoTDBConnectionException { + public synchronized void open() throws SubscriptionException { if (!isClosed.get()) { return; } @@ -107,7 +106,7 @@ public synchronized void open() throws IoTDBConnectionException { } @Override - public synchronized void close() throws IoTDBConnectionException { + public synchronized void close() { if (isClosed.get()) { return; } diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPushConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPushConsumer.java index 25e0e66aaf9e..9dbb2df59a5e 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPushConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionPushConsumer.java @@ -19,7 +19,6 @@ package org.apache.iotdb.session.subscription; -import org.apache.iotdb.rpc.IoTDBConnectionException; import org.apache.iotdb.rpc.subscription.config.ConsumerConstant; import org.apache.iotdb.rpc.subscription.exception.SubscriptionException; import org.apache.iotdb.session.subscription.payload.SubscriptionMessage; @@ -77,7 +76,7 @@ private SubscriptionPushConsumer( /////////////////////////////// open & close /////////////////////////////// - public synchronized void open() throws IoTDBConnectionException { + public synchronized void open() throws SubscriptionException { if (!isClosed.get()) { return; } @@ -90,7 +89,7 @@ public synchronized void open() throws IoTDBConnectionException { } @Override - public synchronized void close() throws IoTDBConnectionException { + public synchronized void close() { if (isClosed.get()) { return; } diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionFileHandler.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionFileHandler.java index a75d969fce61..60c269b86370 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionFileHandler.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionFileHandler.java @@ -30,71 +30,71 @@ public abstract class SubscriptionFileHandler implements SubscriptionMessageHandler { - protected final String filePath; + protected final String absolutePath; - public SubscriptionFileHandler(final String filePath) { - this.filePath = filePath; + public SubscriptionFileHandler(final String absolutePath) { + this.absolutePath = absolutePath; } /** * @return a new File instance of the corresponding file */ public File getFile() { - return new File(filePath); + return new File(absolutePath); } /** * @return a new Path instance of the corresponding file */ public Path getPath() { - return Paths.get(filePath); + return Paths.get(absolutePath); } /** + * @return the path to the source file * @throws IOException if an I/O error occurs */ - public void deleteFile() throws IOException { - Files.delete(getPath()); + public Path deleteFile() throws IOException { + final Path sourcePath = getPath(); + Files.delete(sourcePath); + return sourcePath; } /** - * @param targetFilePath the path to the target file + * @param target the path to the target file * @return the path to the target file * @throws IOException if an I/O error occurs */ - public Path moveFile(final String targetFilePath) throws IOException { - return this.moveFile(Paths.get(targetFilePath)); + public Path moveFile(final String target) throws IOException { + return this.moveFile(Paths.get(target)); } /** - * @param targetFilePath the path to the target file + * @param target the path to the target file * @return the path to the target file * @throws IOException if an I/O error occurs */ - public Path moveFile(final Path targetFilePath) throws IOException { - return Files.move(getPath(), targetFilePath, StandardCopyOption.REPLACE_EXISTING); + public Path moveFile(final Path target) throws IOException { + return Files.move(getPath(), target, StandardCopyOption.REPLACE_EXISTING); } /** - * @param targetFilePath the path to the target file + * @param target the path to the target file * @return the path to the target file * @throws IOException if an I/O error occurs */ - public Path copyFile(final String targetFilePath) throws IOException { - return this.copyFile(Paths.get(targetFilePath)); + public Path copyFile(final String target) throws IOException { + return this.copyFile(Paths.get(target)); } /** - * @param targetFilePath the path to the target file + * @param target the path to the target file * @return the path to the target file * @throws IOException if an I/O error occurs */ - public Path copyFile(final Path targetFilePath) throws IOException { + public Path copyFile(final Path target) throws IOException { return Files.copy( - getPath(), - targetFilePath, - StandardCopyOption.REPLACE_EXISTING, - StandardCopyOption.COPY_ATTRIBUTES); + getPath(), target, StandardCopyOption.REPLACE_EXISTING, StandardCopyOption.COPY_ATTRIBUTES); } @Override diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionMessage.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionMessage.java index becb88e90f42..36cfdd23af17 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionMessage.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionMessage.java @@ -42,10 +42,11 @@ public SubscriptionMessage( this.handler = new SubscriptionSessionDataSetsHandler(tablets); } - public SubscriptionMessage(final SubscriptionCommitContext commitContext, final String filePath) { + public SubscriptionMessage( + final SubscriptionCommitContext commitContext, final String absolutePath) { this.commitContext = commitContext; this.messageType = SubscriptionMessageType.SUBSCRIPTION_TS_FILE_HANDLER.getType(); - this.handler = new SubscriptionTsFileHandler(filePath); + this.handler = new SubscriptionTsFileHandler(absolutePath); } public SubscriptionCommitContext getCommitContext() { diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionTsFileHandler.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionTsFileHandler.java index 136a4204a5b3..c1c84c79d78f 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionTsFileHandler.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionTsFileHandler.java @@ -26,12 +26,12 @@ public class SubscriptionTsFileHandler extends SubscriptionFileHandler { - public SubscriptionTsFileHandler(final String filePath) { - super(filePath); + public SubscriptionTsFileHandler(final String absolutePath) { + super(absolutePath); } public TsFileReader openReader() throws IOException { - return new TsFileReader(new TsFileSequenceReader(filePath)); + return new TsFileReader(new TsFileSequenceReader(absolutePath)); } @Override From 13795c16f3ad0bc10a7b5d8aadff73340fbb6332 Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Thu, 23 May 2024 15:16:29 +0800 Subject: [PATCH 90/93] remove sub prefix --- .../it/dual/IoTDBSubscriptionConsumerGroupIT.java | 4 ++-- .../apache/iotdb/rpc/subscription/config/TopicConstant.java | 5 ++--- .../session/subscription/payload/SubscriptionMessage.java | 4 ++-- .../subscription/payload/SubscriptionMessageType.java | 4 ++-- 4 files changed, 8 insertions(+), 9 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionConsumerGroupIT.java b/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionConsumerGroupIT.java index 827abbc26605..7eccef3f76af 100644 --- a/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionConsumerGroupIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionConsumerGroupIT.java @@ -938,7 +938,7 @@ private void pollMessagesAndCheck( final short messageType = message.getMessageType(); if (SubscriptionMessageType.isValidatedMessageType(messageType)) { switch (SubscriptionMessageType.valueOf(messageType)) { - case SUBSCRIPTION_SESSION_DATA_SETS_HANDLER: + case SESSION_DATA_SETS_HANDLER: { for (final SubscriptionSessionDataSet dataSet : message.getSessionDataSetsHandler()) { @@ -954,7 +954,7 @@ private void pollMessagesAndCheck( } break; } - case SUBSCRIPTION_TS_FILE_HANDLER: + case TS_FILE_HANDLER: { try (final TsFileReader tsFileReader = message.getTsFileHandler().openReader()) { diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/config/TopicConstant.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/config/TopicConstant.java index 6a193aba1629..bd44cf1bcc95 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/config/TopicConstant.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/config/TopicConstant.java @@ -31,9 +31,8 @@ public class TopicConstant { public static final String NOW_TIME_VALUE = "now"; public static final String FORMAT_KEY = "format"; - public static final String FORMAT_SESSION_DATA_SETS_HANDLER_VALUE = - "SubscriptionSessionDataSetsHandler"; - public static final String FORMAT_TS_FILE_HANDLER_VALUE = "SubscriptionTsFileHandler"; + public static final String FORMAT_SESSION_DATA_SETS_HANDLER_VALUE = "SessionDataSetsHandler"; + public static final String FORMAT_TS_FILE_HANDLER_VALUE = "TsFileHandler"; public static final String FORMAT_DEFAULT_VALUE = FORMAT_SESSION_DATA_SETS_HANDLER_VALUE; private TopicConstant() { diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionMessage.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionMessage.java index 36cfdd23af17..40182692178b 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionMessage.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionMessage.java @@ -38,14 +38,14 @@ public class SubscriptionMessage public SubscriptionMessage( final SubscriptionCommitContext commitContext, final List tablets) { this.commitContext = commitContext; - this.messageType = SubscriptionMessageType.SUBSCRIPTION_SESSION_DATA_SETS_HANDLER.getType(); + this.messageType = SubscriptionMessageType.SESSION_DATA_SETS_HANDLER.getType(); this.handler = new SubscriptionSessionDataSetsHandler(tablets); } public SubscriptionMessage( final SubscriptionCommitContext commitContext, final String absolutePath) { this.commitContext = commitContext; - this.messageType = SubscriptionMessageType.SUBSCRIPTION_TS_FILE_HANDLER.getType(); + this.messageType = SubscriptionMessageType.TS_FILE_HANDLER.getType(); this.handler = new SubscriptionTsFileHandler(absolutePath); } diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionMessageType.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionMessageType.java index ef142b3ec7cf..5dabf3711ccc 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionMessageType.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/payload/SubscriptionMessageType.java @@ -24,8 +24,8 @@ import java.util.Map; public enum SubscriptionMessageType { - SUBSCRIPTION_SESSION_DATA_SETS_HANDLER((short) 0), - SUBSCRIPTION_TS_FILE_HANDLER((short) 1), + SESSION_DATA_SETS_HANDLER((short) 0), + TS_FILE_HANDLER((short) 1), ; private final short type; From e9efcbbf212467995a945aa38933db9c1bf75fc8 Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Thu, 23 May 2024 22:06:48 +0800 Subject: [PATCH 91/93] fix review --- .../subscription/config/ConsumerConstant.java | 3 ++- .../rpc/subscription/config/TopicConfig.java | 26 +++++++++---------- 2 files changed, 14 insertions(+), 15 deletions(-) diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/config/ConsumerConstant.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/config/ConsumerConstant.java index 2b8784c1cda4..ccfe74e3cc2f 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/config/ConsumerConstant.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/config/ConsumerConstant.java @@ -43,7 +43,8 @@ public class ConsumerConstant { public static final long ENDPOINTS_SYNC_INTERVAL_MS_MIN_VALUE = 5_000; public static final String FILE_SAVE_DIR_KEY = "file-save-dir"; - public static final String FILE_SAVE_DIR_DEFAULT_VALUE = System.getProperty("user.dir"); + public static final String FILE_SAVE_DIR_DEFAULT_VALUE = + System.getProperty("user.dir") + "iotdb-subscription"; /////////////////////////////// pull consumer /////////////////////////////// diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/config/TopicConfig.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/config/TopicConfig.java index d42b254d9798..6003cc99d31a 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/config/TopicConfig.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/config/TopicConfig.java @@ -36,17 +36,17 @@ public TopicConfig() { super(Collections.emptyMap()); } - public TopicConfig(Map attributes) { + public TopicConfig(final Map attributes) { super(attributes); } /////////////////////////////// de/ser /////////////////////////////// - public void serialize(DataOutputStream stream) throws IOException { + public void serialize(final DataOutputStream stream) throws IOException { ReadWriteIOUtils.write(attributes, stream); } - public static TopicConfig deserialize(ByteBuffer buffer) { + public static TopicConfig deserialize(final ByteBuffer buffer) { return new TopicConfig(ReadWriteIOUtils.readMap(buffer)); } @@ -63,16 +63,16 @@ public Map getAttributesWithSourcePathOrPattern() { attributes.getOrDefault(TopicConstant.PATH_KEY, TopicConstant.PATH_DEFAULT_VALUE)); } - public Map getAttributesWithTimeRange(long creationTime) { - Map attributesWithTimeRange = new HashMap<>(); - String startTime = + public Map getAttributesWithTimeRange(final long creationTime) { + final Map attributesWithTimeRange = new HashMap<>(); + final String startTime = attributes.getOrDefault(TopicConstant.START_TIME_KEY, String.valueOf(Long.MIN_VALUE)); if (TopicConstant.NOW_TIME_VALUE.equals(startTime)) { attributesWithTimeRange.put(TopicConstant.START_TIME_KEY, String.valueOf(creationTime)); } else { attributesWithTimeRange.put(TopicConstant.START_TIME_KEY, startTime); } - String endTime = + final String endTime = attributes.getOrDefault(TopicConstant.END_TIME_KEY, String.valueOf(Long.MAX_VALUE)); if (TopicConstant.NOW_TIME_VALUE.equals(endTime)) { attributesWithTimeRange.put(TopicConstant.END_TIME_KEY, String.valueOf(creationTime)); @@ -83,16 +83,14 @@ public Map getAttributesWithTimeRange(long creationTime) { } public Map getAttributesWithSourceRealtimeMode() { - if (TopicConstant.FORMAT_TS_FILE_HANDLER_VALUE.equals( - attributes.getOrDefault(TopicConstant.FORMAT_KEY, TopicConstant.FORMAT_DEFAULT_VALUE))) { - return Collections.singletonMap("realtime.mode", "batch"); - } - - return Collections.singletonMap("realtime.mode", "hybrid"); + return TopicConstant.FORMAT_TS_FILE_HANDLER_VALUE.equals( + attributes.getOrDefault(TopicConstant.FORMAT_KEY, TopicConstant.FORMAT_DEFAULT_VALUE)) + ? Collections.singletonMap("realtime.mode", "batch") + : Collections.singletonMap("realtime.mode", "hybrid"); } public Map getAttributesWithProcessorPrefix() { - Map attributesWithProcessorPrefix = new HashMap<>(); + final Map attributesWithProcessorPrefix = new HashMap<>(); attributes.forEach( (key, value) -> { if (key.toLowerCase().startsWith("processor")) { From 73905bf47879a706ec73718f6a31a927f1fc5325 Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Fri, 24 May 2024 20:01:34 +0800 Subject: [PATCH 92/93] notify the client when the topic does not meet the restrictions --- .../iotdb/SubscriptionSessionExample.java | 3 +- .../it/dual/IoTDBSubscriptionTopicIT.java | 139 ++++++++++++++---- .../subscription/config/ConsumerConstant.java | 4 +- .../rpc/subscription/config/TopicConfig.java | 51 ++----- .../subscription/SubscriptionConsumer.java | 15 ++ .../executor/ClusterConfigTaskExecutor.java | 12 -- .../SubscriptionPrefetchingTsFileQueue.java | 10 ++ .../subscription/meta/topic/TopicMeta.java | 37 ++--- 8 files changed, 172 insertions(+), 99 deletions(-) diff --git a/example/session/src/main/java/org/apache/iotdb/SubscriptionSessionExample.java b/example/session/src/main/java/org/apache/iotdb/SubscriptionSessionExample.java index f836ed324c91..383b16a345a7 100644 --- a/example/session/src/main/java/org/apache/iotdb/SubscriptionSessionExample.java +++ b/example/session/src/main/java/org/apache/iotdb/SubscriptionSessionExample.java @@ -21,7 +21,6 @@ import org.apache.iotdb.isession.SessionDataSet; import org.apache.iotdb.isession.util.Version; -import org.apache.iotdb.rpc.IoTDBConnectionException; import org.apache.iotdb.rpc.subscription.config.ConsumerConstant; import org.apache.iotdb.rpc.subscription.config.TopicConstant; import org.apache.iotdb.session.Session; @@ -194,7 +193,7 @@ private static void subscriptionExample2() throws Exception { consumer2.commitSync(messages); } consumer2.unsubscribe(TOPIC_2); - } catch (IOException | IoTDBConnectionException e) { + } catch (final IOException e) { throw new RuntimeException(e); } }); diff --git a/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionTopicIT.java b/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionTopicIT.java index b6994b7eaf62..b6ef690236e8 100644 --- a/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionTopicIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionTopicIT.java @@ -27,6 +27,7 @@ import org.apache.iotdb.it.framework.IoTDBTestRunner; import org.apache.iotdb.itbase.category.MultiClusterIT2Subscription; import org.apache.iotdb.rpc.subscription.config.TopicConstant; +import org.apache.iotdb.rpc.subscription.exception.SubscriptionRuntimeCriticalException; import org.apache.iotdb.session.subscription.SubscriptionPullConsumer; import org.apache.iotdb.session.subscription.SubscriptionSession; import org.apache.iotdb.session.subscription.payload.SubscriptionMessage; @@ -52,6 +53,7 @@ import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.locks.LockSupport; import static org.junit.Assert.fail; @@ -494,7 +496,8 @@ public void testTopicNameWithBackQuote() throws Exception { } } - public void testTopicInvalidConfig() throws Exception { + @Test + public void testTopicInvalidTimeRangeConfig() throws Exception { final String host = senderEnv.getIP(); final int port = Integer.parseInt(senderEnv.getPort()); @@ -521,39 +524,117 @@ public void testTopicInvalidConfig() throws Exception { } catch (final Exception ignored) { } assertTopicCount(0); + } - // Scenario 3: test when 'format' is 'TsFileReader' - final List configs = new ArrayList<>(); - { - final Properties config = new Properties(); - config.put(TopicConstant.FORMAT_KEY, TopicConstant.FORMAT_TS_FILE_HANDLER_VALUE); - config.put(TopicConstant.PATH_KEY, "root.db.*.s"); - configs.add(config); - } - { - final Properties config = new Properties(); - config.put(TopicConstant.FORMAT_KEY, TopicConstant.FORMAT_TS_FILE_HANDLER_VALUE); - config.put(TopicConstant.START_TIME_KEY, System.currentTimeMillis()); - configs.add(config); + @Test + public void testTopicInvalidPathConfig() throws Exception { + // Test invalid path when using tsfile format + final Properties config = new Properties(); + config.put(TopicConstant.FORMAT_KEY, TopicConstant.FORMAT_TS_FILE_HANDLER_VALUE); + config.put(TopicConstant.PATH_KEY, "root.db.*.s"); + testTopicInvalidRuntimeConfigTemplate("topic3", config); + } + + @Test + public void testTopicInvalidProcessorConfig() throws Exception { + // Test invalid processor when using tsfile format + final Properties config = new Properties(); + config.put(TopicConstant.FORMAT_KEY, TopicConstant.FORMAT_TS_FILE_HANDLER_VALUE); + config.put("processor", "tumbling-time-sampling-processor"); + config.put("processor.tumbling-time.interval-seconds", "1"); + config.put("processor.down-sampling.split-file", "true"); + testTopicInvalidRuntimeConfigTemplate("topic4", config); + } + + private void testTopicInvalidRuntimeConfigTemplate( + final String topicName, final Properties config) throws Exception { + // Create topic + final String host = senderEnv.getIP(); + final int port = Integer.parseInt(senderEnv.getPort()); + try (final SubscriptionSession session = new SubscriptionSession(host, port)) { + session.open(); + session.createTopic(topicName, config); + } catch (final Exception e) { + e.printStackTrace(); + fail(e.getMessage()); } - { - final Properties config = new Properties(); - config.put(TopicConstant.FORMAT_KEY, TopicConstant.FORMAT_TS_FILE_HANDLER_VALUE); - config.put("processor", "tumbling-time-sampling-processor"); - config.put("processor.tumbling-time.interval-seconds", "1"); - config.put("processor.down-sampling.split-file", "true"); - configs.add(config); + assertTopicCount(1); + + final AtomicBoolean dataPrepared = new AtomicBoolean(false); + final AtomicBoolean topicSubscribed = new AtomicBoolean(false); + final AtomicBoolean result = new AtomicBoolean(false); + final List threads = new ArrayList<>(); + + // Subscribe on sender + threads.add( + new Thread( + () -> { + final AtomicInteger retryCount = new AtomicInteger(); + final int maxRetryTimes = 32; + try (final SubscriptionPullConsumer consumer = + new SubscriptionPullConsumer.Builder() + .host(host) + .port(port) + .consumerId("c1") + .consumerGroupId("cg1") + .autoCommit(false) + .buildPullConsumer()) { + consumer.open(); + consumer.subscribe(topicName); + topicSubscribed.set(true); + while (retryCount.getAndIncrement() < maxRetryTimes) { + LockSupport.parkNanos(IoTDBSubscriptionITConstant.SLEEP_NS); // wait some time + if (dataPrepared.get()) { + final List messages = + consumer.poll(IoTDBSubscriptionITConstant.POLL_TIMEOUT_MS); + consumer.commitSync(messages); + } + } + consumer.unsubscribe(topicName); + } catch (final SubscriptionRuntimeCriticalException e) { + result.set(true); + } catch (final Exception e) { + e.printStackTrace(); + // Avoid failure + } finally { + LOGGER.info("consumer exiting..."); + } + })); + + // Insert some realtime data on sender + threads.add( + new Thread( + () -> { + while (!topicSubscribed.get()) { + LockSupport.parkNanos(IoTDBSubscriptionITConstant.SLEEP_NS); // wait some time + } + try (final ISession session = senderEnv.getSessionConnection()) { + for (int i = 0; i < 100; ++i) { + session.executeNonQueryStatement( + String.format("insert into root.db.d1(time, s) values (%s, 1)", i)); + } + session.executeNonQueryStatement("flush"); + } catch (final Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + dataPrepared.set(true); + })); + + for (final Thread thread : threads) { + thread.start(); } - for (final Properties config : configs) { - try (final SubscriptionSession session = new SubscriptionSession(host, port)) { - session.open(); - session.createTopic("topic3", config); - fail(); - } catch (final Exception ignored) { - } + Awaitility.await() + .pollDelay(IoTDBSubscriptionITConstant.AWAITILITY_POLL_DELAY_SECOND, TimeUnit.SECONDS) + .pollInterval(IoTDBSubscriptionITConstant.AWAITILITY_POLL_INTERVAL_SECOND, TimeUnit.SECONDS) + .atMost(IoTDBSubscriptionITConstant.AWAITILITY_AT_MOST_SECOND, TimeUnit.SECONDS) + // The expected SubscriptionRuntimeCriticalException was not thrown if result is false + .untilTrue(result); + + for (final Thread thread : threads) { + thread.join(); } - assertTopicCount(0); } private void assertTopicCount(final int count) throws Exception { diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/config/ConsumerConstant.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/config/ConsumerConstant.java index ccfe74e3cc2f..efb0e53a6493 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/config/ConsumerConstant.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/config/ConsumerConstant.java @@ -19,6 +19,8 @@ package org.apache.iotdb.rpc.subscription.config; +import java.nio.file.Paths; + public class ConsumerConstant { /////////////////////////////// common /////////////////////////////// @@ -44,7 +46,7 @@ public class ConsumerConstant { public static final String FILE_SAVE_DIR_KEY = "file-save-dir"; public static final String FILE_SAVE_DIR_DEFAULT_VALUE = - System.getProperty("user.dir") + "iotdb-subscription"; + Paths.get(System.getProperty("user.dir"), "iotdb-subscription").toString(); /////////////////////////////// pull consumer /////////////////////////////// diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/config/TopicConfig.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/config/TopicConfig.java index 6003cc99d31a..fe62ba622a96 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/config/TopicConfig.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/subscription/config/TopicConfig.java @@ -52,7 +52,7 @@ public static TopicConfig deserialize(final ByteBuffer buffer) { /////////////////////////////// utilities /////////////////////////////// - public Map getAttributesWithSourcePathOrPattern() { + public Map getAttributesWithPathOrPattern() { if (attributes.containsKey(TopicConstant.PATTERN_KEY)) { return Collections.singletonMap( TopicConstant.PATTERN_KEY, attributes.get(TopicConstant.PATTERN_KEY)); @@ -65,6 +65,8 @@ public Map getAttributesWithSourcePathOrPattern() { public Map getAttributesWithTimeRange(final long creationTime) { final Map attributesWithTimeRange = new HashMap<>(); + + // parse start time final String startTime = attributes.getOrDefault(TopicConstant.START_TIME_KEY, String.valueOf(Long.MIN_VALUE)); if (TopicConstant.NOW_TIME_VALUE.equals(startTime)) { @@ -72,6 +74,8 @@ public Map getAttributesWithTimeRange(final long creationTime) { } else { attributesWithTimeRange.put(TopicConstant.START_TIME_KEY, startTime); } + + // parse end time final String endTime = attributes.getOrDefault(TopicConstant.END_TIME_KEY, String.valueOf(Long.MAX_VALUE)); if (TopicConstant.NOW_TIME_VALUE.equals(endTime)) { @@ -79,10 +83,18 @@ public Map getAttributesWithTimeRange(final long creationTime) { } else { attributesWithTimeRange.put(TopicConstant.END_TIME_KEY, endTime); } + + // enable loose range when using tsfile format + if (TopicConstant.FORMAT_TS_FILE_HANDLER_VALUE.equals( + attributes.getOrDefault(TopicConstant.FORMAT_KEY, TopicConstant.FORMAT_DEFAULT_VALUE))) { + attributesWithTimeRange.put("history.loose-range", "time"); + attributesWithTimeRange.put("realtime.loose-range", "time"); + } + return attributesWithTimeRange; } - public Map getAttributesWithSourceRealtimeMode() { + public Map getAttributesWithRealtimeMode() { return TopicConstant.FORMAT_TS_FILE_HANDLER_VALUE.equals( attributes.getOrDefault(TopicConstant.FORMAT_KEY, TopicConstant.FORMAT_DEFAULT_VALUE)) ? Collections.singletonMap("realtime.mode", "batch") @@ -99,39 +111,4 @@ public Map getAttributesWithProcessorPrefix() { }); return attributesWithProcessorPrefix; } - - public boolean isValid() { - if (!TopicConstant.FORMAT_TS_FILE_HANDLER_VALUE.equals( - attributes.getOrDefault(TopicConstant.FORMAT_KEY, TopicConstant.FORMAT_DEFAULT_VALUE))) { - return true; - } - - // check processor - if (!getAttributesWithProcessorPrefix().isEmpty()) { - return false; - } - - // check time range - final String startTime = - attributes.getOrDefault(TopicConstant.START_TIME_KEY, String.valueOf(Long.MIN_VALUE)); - if (!String.valueOf(Long.MIN_VALUE).equals(startTime)) { - return false; - } - final String endTime = - attributes.getOrDefault(TopicConstant.END_TIME_KEY, String.valueOf(Long.MAX_VALUE)); - if (!String.valueOf(Long.MAX_VALUE).equals(endTime)) { - return false; - } - - // check path or pattern - if (!TopicConstant.PATH_DEFAULT_VALUE.equals( - attributes.getOrDefault(TopicConstant.PATH_KEY, TopicConstant.PATH_DEFAULT_VALUE)) - || !TopicConstant.PATTERN_DEFAULT_VALUE.equals( - attributes.getOrDefault( - TopicConstant.PATTERN_KEY, TopicConstant.PATTERN_DEFAULT_VALUE))) { - return false; - } - - return true; - } } diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java index 133e8f1c5187..58655aa2f091 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/SubscriptionConsumer.java @@ -406,6 +406,21 @@ protected List poll(final Set topicNames, final lon ((FileInitPayload) pollResponse.getPayload()).getFileName()) .ifPresent(messages::add); break; + case ERROR: + final ErrorPayload payload = (ErrorPayload) pollResponse.getPayload(); + final String errorMessage = payload.getErrorMessage(); + final boolean critical = payload.isCritical(); + LOGGER.warn( + "Error occurred when SubscriptionConsumer {} polling topics {}: {}, critical: {}", + this, + topicNames, + errorMessage, + critical); + if (critical) { + throw new SubscriptionRuntimeCriticalException(errorMessage); + } else { + throw new SubscriptionRuntimeNonCriticalException(errorMessage); + } default: LOGGER.warn("unexpected response type: {}", responseType); break; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java index c8546f8d2634..912e6d53777b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java @@ -215,7 +215,6 @@ import org.apache.iotdb.rpc.RpcUtils; import org.apache.iotdb.rpc.StatementExecutionException; import org.apache.iotdb.rpc.TSStatusCode; -import org.apache.iotdb.rpc.subscription.config.TopicConfig; import org.apache.iotdb.service.rpc.thrift.TPipeTransferReq; import org.apache.iotdb.service.rpc.thrift.TPipeTransferResp; import org.apache.iotdb.trigger.api.Trigger; @@ -1947,17 +1946,6 @@ public SettableFuture createTopic(CreateTopicStatement createT return future; } - final TopicConfig topicConfig = new TopicConfig(topicAttributes); - if (!topicConfig.isValid()) { - final String exceptionMessage = - String.format( - "Failed to create topic %s, topic config %s is invalid.", topicName, topicConfig); - LOGGER.warn(exceptionMessage); - future.setException( - new IoTDBException(exceptionMessage, TSStatusCode.CREATE_TOPIC_ERROR.getStatusCode())); - return future; - } - try (final ConfigNodeClient configNodeClient = CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) { final TCreateTopicReq req = diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java index a40a225c6a16..fc5a1a8a81ee 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/broker/SubscriptionPrefetchingTsFileQueue.java @@ -25,6 +25,7 @@ import org.apache.iotdb.db.pipe.event.common.tsfile.PipeTsFileInsertionEvent; import org.apache.iotdb.db.subscription.event.SubscriptionTsFileEvent; import org.apache.iotdb.pipe.api.event.Event; +import org.apache.iotdb.pipe.api.event.dml.insertion.TabletInsertionEvent; import org.apache.iotdb.rpc.subscription.payload.poll.ErrorPayload; import org.apache.iotdb.rpc.subscription.payload.poll.FileInitPayload; import org.apache.iotdb.rpc.subscription.payload.poll.FilePiecePayload; @@ -77,6 +78,15 @@ public SubscriptionTsFileEvent poll(final String consumerId) { Event event; while (Objects.nonNull( event = UserDefinedEnrichedEvent.maybeOf(inputPendingQueue.waitedPoll()))) { + if (event instanceof TabletInsertionEvent) { + final String errorMessage = + String.format( + "A TabletInsertionEvent was pulled from topic %s which is formatted as TsFile by SubscriptionPrefetchingTsFileQueue %s. This event %s will be ignored. Please check the topic configuration.", + topicName, this, event); + LOGGER.warn(errorMessage); + return generateSubscriptionPollErrorResponse(errorMessage, true); + } + if (!(event instanceof PipeTsFileInsertionEvent)) { LOGGER.warn( "Subscription: SubscriptionPrefetchingTsFileQueue {} only support poll PipeTsFileInsertionEvent. Ignore {}.", diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/subscription/meta/topic/TopicMeta.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/subscription/meta/topic/TopicMeta.java index 2a39b32a2138..33f5065cbae6 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/subscription/meta/topic/TopicMeta.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/subscription/meta/topic/TopicMeta.java @@ -50,7 +50,8 @@ private TopicMeta() { this.subscribedConsumerGroupIds = new HashSet<>(); } - public TopicMeta(String topicName, long creationTime, Map topicAttributes) { + public TopicMeta( + final String topicName, final long creationTime, final Map topicAttributes) { this.topicName = topicName; this.creationTime = creationTime; this.config = new TopicConfig(topicAttributes); @@ -83,11 +84,11 @@ public TopicConfig getConfig() { /** * @return true if the consumer group did not already subscribe this topic */ - public boolean addSubscribedConsumerGroup(String consumerGroupId) { + public boolean addSubscribedConsumerGroup(final String consumerGroupId) { return subscribedConsumerGroupIds.add(consumerGroupId); } - public void removeSubscribedConsumerGroup(String consumerGroupId) { + public void removeSubscribedConsumerGroup(final String consumerGroupId) { subscribedConsumerGroupIds.remove(consumerGroupId); } @@ -95,7 +96,7 @@ public Set getSubscribedConsumerGroupIds() { return subscribedConsumerGroupIds; } - public boolean isSubscribedByConsumerGroup(String consumerGroupId) { + public boolean isSubscribedByConsumerGroup(final String consumerGroupId) { return subscribedConsumerGroupIds.contains(consumerGroupId); } @@ -106,29 +107,29 @@ public boolean hasSubscribedConsumerGroup() { ////////////////////////////////////// de/ser //////////////////////////////// public ByteBuffer serialize() throws IOException { - PublicBAOS byteArrayOutputStream = new PublicBAOS(); - DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream); + final PublicBAOS byteArrayOutputStream = new PublicBAOS(); + final DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream); serialize(outputStream); return ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size()); } - public void serialize(OutputStream outputStream) throws IOException { + public void serialize(final OutputStream outputStream) throws IOException { ReadWriteIOUtils.write(topicName, outputStream); ReadWriteIOUtils.write(creationTime, outputStream); ReadWriteIOUtils.write(config.getAttribute().size(), outputStream); - for (Map.Entry entry : config.getAttribute().entrySet()) { + for (final Map.Entry entry : config.getAttribute().entrySet()) { ReadWriteIOUtils.write(entry.getKey(), outputStream); ReadWriteIOUtils.write(entry.getValue(), outputStream); } ReadWriteIOUtils.write(subscribedConsumerGroupIds.size(), outputStream); - for (String subscribedConsumerGroupID : subscribedConsumerGroupIds) { + for (final String subscribedConsumerGroupID : subscribedConsumerGroupIds) { ReadWriteIOUtils.write(subscribedConsumerGroupID, outputStream); } } - public static TopicMeta deserialize(InputStream inputStream) throws IOException { + public static TopicMeta deserialize(final InputStream inputStream) throws IOException { final TopicMeta topicMeta = new TopicMeta(); topicMeta.topicName = ReadWriteIOUtils.readString(inputStream); @@ -149,7 +150,7 @@ public static TopicMeta deserialize(InputStream inputStream) throws IOException return topicMeta; } - public static TopicMeta deserialize(ByteBuffer byteBuffer) { + public static TopicMeta deserialize(final ByteBuffer byteBuffer) { final TopicMeta topicMeta = new TopicMeta(); topicMeta.topicName = ReadWriteIOUtils.readString(byteBuffer); @@ -173,17 +174,17 @@ public static TopicMeta deserialize(ByteBuffer byteBuffer) { /////////////////////////////// utilities /////////////////////////////// public Map generateExtractorAttributes() { - Map extractorAttributes = new HashMap<>(); + final Map extractorAttributes = new HashMap<>(); // disable meta sync extractorAttributes.put("source", "iotdb-source"); extractorAttributes.put("inclusion", "data.insert"); extractorAttributes.put("inclusion.exclusion", "data.delete"); // path - extractorAttributes.putAll(config.getAttributesWithSourcePathOrPattern()); + extractorAttributes.putAll(config.getAttributesWithPathOrPattern()); // time extractorAttributes.putAll(config.getAttributesWithTimeRange(creationTime)); // realtime mode - extractorAttributes.putAll(config.getAttributesWithSourceRealtimeMode()); + extractorAttributes.putAll(config.getAttributesWithRealtimeMode()); return extractorAttributes; } @@ -191,8 +192,8 @@ public Map generateProcessorAttributes() { return config.getAttributesWithProcessorPrefix(); } - public Map generateConnectorAttributes(String consumerGroupId) { - Map connectorAttributes = new HashMap<>(); + public Map generateConnectorAttributes(final String consumerGroupId) { + final Map connectorAttributes = new HashMap<>(); connectorAttributes.put("sink", "subscription-sink"); connectorAttributes.put(PipeConnectorConstant.SINK_TOPIC_KEY, topicName); connectorAttributes.put(PipeConnectorConstant.SINK_CONSUMER_GROUP_KEY, consumerGroupId); @@ -202,14 +203,14 @@ public Map generateConnectorAttributes(String consumerGroupId) { ////////////////////////////////////// Object //////////////////////////////// @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; } - TopicMeta that = (TopicMeta) obj; + final TopicMeta that = (TopicMeta) obj; return creationTime == that.creationTime && Objects.equals(topicName, that.topicName) && Objects.equals(config, that.config) From 4bcb9eae857f7e0575a3139e03873b9fb56e8207 Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Mon, 27 May 2024 14:48:46 +0800 Subject: [PATCH 93/93] fix IT --- .../subscription/it/dual/IoTDBSubscriptionTopicIT.java | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionTopicIT.java b/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionTopicIT.java index b6ef690236e8..f2d4e2191bf0 100644 --- a/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionTopicIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionTopicIT.java @@ -613,6 +613,14 @@ private void testTopicInvalidRuntimeConfigTemplate( session.executeNonQueryStatement( String.format("insert into root.db.d1(time, s) values (%s, 1)", i)); } + for (int i = 0; i < 100; ++i) { + session.executeNonQueryStatement( + String.format("insert into root.db.d2(time, s) values (%s, 1)", i)); + } + for (int i = 0; i < 100; ++i) { + session.executeNonQueryStatement( + String.format("insert into root.db.d3(time, t) values (%s, 1)", i)); + } session.executeNonQueryStatement("flush"); } catch (final Exception e) { e.printStackTrace();