From db5658a2b6b95b949338aa0b7879a70514006406 Mon Sep 17 00:00:00 2001 From: ysc <2725843507@qq.com> Date: Fri, 16 May 2025 15:16:48 +0800 Subject: [PATCH 01/11] Collector: Add string format processing for kafka value & Add kafka processor --- .../plugin/builtin/BuiltinPlugin.java | 6 +- .../builtin/processor/KafkaProcessor.java | 56 ++++++++ .../event/PipeRawTabletInsertionEvent.java | 16 ++- .../IoTDBDataRegionSyncConnector.java | 5 +- .../source/event/common/PipeRowCollector.java | 2 +- .../builtin/source/iotdb/SubDemoEvent.java | 15 +-- .../builtin/source/kafka/KafkaEvent.java | 31 +++++ .../builtin/source/kafka/KafkaSource.java | 126 +++++++++++++----- .../source/kafka/KafkaSourceConstant.java | 4 +- .../constructor/ProcessorConstructor.java | 2 + .../task/source/push/PushSourceTask.java | 42 +++--- 11 files changed, 226 insertions(+), 79 deletions(-) create mode 100644 iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/processor/KafkaProcessor.java create mode 100644 iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/kafka/KafkaEvent.java diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/BuiltinPlugin.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/BuiltinPlugin.java index ba75247a..43505caa 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/BuiltinPlugin.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/BuiltinPlugin.java @@ -20,6 +20,7 @@ package org.apache.iotdb.collector.plugin.builtin; import org.apache.iotdb.collector.plugin.builtin.processor.DoNothingProcessor; +import org.apache.iotdb.collector.plugin.builtin.processor.KafkaProcessor; import org.apache.iotdb.collector.plugin.builtin.processor.SubscriptionProcessor; import org.apache.iotdb.collector.plugin.builtin.sink.DemoSink; import org.apache.iotdb.collector.plugin.builtin.sink.protocol.IoTDBDataRegionSyncConnector; @@ -33,14 +34,15 @@ public enum BuiltinPlugin { // Push Sources HTTP_PUSH_SOURCE("http-push-source", HttpPushSource.class), KAFKA_SOURCE("kafka-source", KafkaSource.class), + SUBSCRIPTION_SOURCE("subscription-source", IoTDBPushSource.class), - // Pull Sources + // Pull Sources, HTTP_PULL_SOURCE("http-pull-source", HttpPullSource.class), - SUBSCRIPTION_SOURCE("subscription-source", IoTDBPushSource.class), // Processors DO_NOTHING_PROCESSOR("do-nothing-processor", DoNothingProcessor.class), SUBSCRIPTION_PROCESSOR("subscription-processor", SubscriptionProcessor.class), + KAFKA_PROCESSOR("kafka-processor", KafkaProcessor.class), // Sinks IOTDB_DEMO_SINK("iotdb-demo-sink", DemoSink.class), diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/processor/KafkaProcessor.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/processor/KafkaProcessor.java new file mode 100644 index 00000000..49765e1b --- /dev/null +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/processor/KafkaProcessor.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.collector.plugin.builtin.processor; + +import org.apache.iotdb.collector.plugin.builtin.source.kafka.KafkaEvent; +import org.apache.iotdb.pipe.api.PipeProcessor; +import org.apache.iotdb.pipe.api.collector.EventCollector; +import org.apache.iotdb.pipe.api.customizer.configuration.PipeProcessorRuntimeConfiguration; +import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameterValidator; +import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameters; +import org.apache.iotdb.pipe.api.event.Event; +import org.apache.iotdb.pipe.api.event.dml.insertion.TabletInsertionEvent; + +public class KafkaProcessor implements PipeProcessor { + @Override + public void validate(PipeParameterValidator pipeParameterValidator) throws Exception {} + + @Override + public void customize( + PipeParameters pipeParameters, + PipeProcessorRuntimeConfiguration pipeProcessorRuntimeConfiguration) + throws Exception {} + + @Override + public void process(TabletInsertionEvent tabletInsertionEvent, EventCollector eventCollector) + throws Exception { + eventCollector.collect(tabletInsertionEvent); + } + + @Override + public void process(Event event, EventCollector eventCollector) throws Exception { + if (event instanceof KafkaEvent) { + process((KafkaEvent) event, eventCollector); + } + } + + @Override + public void close() throws Exception {} +} diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/event/PipeRawTabletInsertionEvent.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/event/PipeRawTabletInsertionEvent.java index bee813a7..37153c94 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/event/PipeRawTabletInsertionEvent.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/event/PipeRawTabletInsertionEvent.java @@ -19,6 +19,7 @@ package org.apache.iotdb.collector.plugin.builtin.sink.event; +import org.apache.iotdb.collector.plugin.builtin.source.event.common.PipeRowCollector; import org.apache.iotdb.pipe.api.access.Row; import org.apache.iotdb.pipe.api.collector.RowCollector; import org.apache.iotdb.pipe.api.event.dml.insertion.TabletInsertionEvent; @@ -30,16 +31,17 @@ public class PipeRawTabletInsertionEvent extends PipeInsertionEvent implements TabletInsertionEvent, AutoCloseable { - public PipeRawTabletInsertionEvent(Tablet tablet, String deviceId) { + public String deviceId; + protected Tablet tablet; + private boolean isAligned; + + public PipeRawTabletInsertionEvent(final Tablet tablet, final String deviceId) { this.deviceId = deviceId; this.tablet = tablet; + this.tablet.setDeviceId(deviceId); } - public String deviceId; - protected Tablet tablet; - private boolean isAligned; - @Override public Iterable processRowByRow( final BiConsumer consumer) { @@ -49,7 +51,9 @@ public Iterable processRowByRow( @Override public Iterable processTablet( final BiConsumer consumer) { - throw new UnsupportedOperationException(); + final PipeRowCollector rowCollector = new PipeRowCollector(); + consumer.accept(convertToTablet(), rowCollector); + return rowCollector.convertToTabletInsertionEvents(); } public Tablet getTablet() { diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/protocol/IoTDBDataRegionSyncConnector.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/protocol/IoTDBDataRegionSyncConnector.java index cfd74576..2cde1b5a 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/protocol/IoTDBDataRegionSyncConnector.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/protocol/IoTDBDataRegionSyncConnector.java @@ -19,6 +19,7 @@ package org.apache.iotdb.collector.plugin.builtin.sink.protocol; +import org.apache.iotdb.collector.plugin.api.event.PeriodicalEvent; import org.apache.iotdb.collector.plugin.builtin.sink.client.IoTDBDataNodeSyncClientManager; import org.apache.iotdb.collector.plugin.builtin.sink.client.IoTDBSyncClient; import org.apache.iotdb.collector.plugin.builtin.sink.client.IoTDBSyncClientManager; @@ -158,7 +159,9 @@ public void transfer(final TsFileInsertionEvent tsFileInsertionEvent) throws Exc @Override public void transfer(final Event event) throws Exception { - if (isTabletBatchModeEnabled && !tabletBatchBuilder.isEmpty()) { + if (event instanceof PeriodicalEvent) { + doTransferWrapper(); + } else if (isTabletBatchModeEnabled && !tabletBatchBuilder.isEmpty()) { doTransferWrapper(); } } diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/event/common/PipeRowCollector.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/event/common/PipeRowCollector.java index c2e31d66..cd222e05 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/event/common/PipeRowCollector.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/event/common/PipeRowCollector.java @@ -95,7 +95,7 @@ private void collectTabletInsertionEvent() { this.tablet = null; } - public List convertToTabletInsertionEvents(final boolean shouldReport) { + public List convertToTabletInsertionEvents() { collectTabletInsertionEvent(); return tabletInsertionEventList; diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/SubDemoEvent.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/SubDemoEvent.java index ea547f8e..a0f3749c 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/SubDemoEvent.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/SubDemoEvent.java @@ -20,25 +20,12 @@ package org.apache.iotdb.collector.plugin.builtin.source.iotdb; import org.apache.iotdb.collector.plugin.builtin.sink.event.PipeRawTabletInsertionEvent; -import org.apache.iotdb.collector.plugin.builtin.source.event.common.PipeRowCollector; -import org.apache.iotdb.pipe.api.collector.RowCollector; -import org.apache.iotdb.pipe.api.event.dml.insertion.TabletInsertionEvent; import org.apache.tsfile.write.record.Tablet; -import java.util.function.BiConsumer; - public class SubDemoEvent extends PipeRawTabletInsertionEvent { - public SubDemoEvent(Tablet tablet, String deviceId) { + public SubDemoEvent(final Tablet tablet, final String deviceId) { super(tablet, deviceId); } - - @Override - public Iterable processTablet( - final BiConsumer consumer) { - final PipeRowCollector collector = new PipeRowCollector(); - consumer.accept(tablet, collector); - return collector.convertToTabletInsertionEvents(false); - } } diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/kafka/KafkaEvent.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/kafka/KafkaEvent.java new file mode 100644 index 00000000..f978a6c4 --- /dev/null +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/kafka/KafkaEvent.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.collector.plugin.builtin.source.kafka; + +import org.apache.iotdb.collector.plugin.builtin.sink.event.PipeRawTabletInsertionEvent; + +import org.apache.tsfile.write.record.Tablet; + +public class KafkaEvent extends PipeRawTabletInsertionEvent { + + public KafkaEvent(final Tablet tablet, final String deviceId) { + super(tablet, deviceId); + } +} diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/kafka/KafkaSource.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/kafka/KafkaSource.java index 54bdba0a..151bac55 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/kafka/KafkaSource.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/kafka/KafkaSource.java @@ -32,12 +32,20 @@ import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.WakeupException; +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.utils.Binary; +import org.apache.tsfile.write.record.Tablet; +import org.apache.tsfile.write.schema.IMeasurementSchema; +import org.apache.tsfile.write.schema.MeasurementSchema; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.nio.charset.StandardCharsets; import java.time.Duration; +import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; +import java.util.List; import java.util.Objects; import java.util.Optional; import java.util.Properties; @@ -100,8 +108,9 @@ public class KafkaSource extends PushSource { @Override public void validate(PipeParameterValidator validator) throws Exception { - validateRequiredParam(validator, KAFKA_SOURCE_TOPIC_KEY); - validateRequiredParam(validator, KAFKA_SOURCE_GROUP_ID_KEY); + validateRequiredParam(validator, validator.getParameters().getString(KAFKA_SOURCE_TOPIC_KEY)); + validateRequiredParam( + validator, validator.getParameters().getString(KAFKA_SOURCE_GROUP_ID_KEY)); validateParam( validator, @@ -115,10 +124,31 @@ public void validate(PipeParameterValidator validator) throws Exception { enableAutoCommit -> BOOLEAN_SET.contains(String.valueOf(enableAutoCommit)), KAFKA_SOURCE_ENABLE_AUTO_COMMIT_DEFAULT_VALUE); - validateIntegerParam(validator, KAFKA_SOURCE_SESSION_TIMEOUT_MS_KEY, value -> value > 0); - validateIntegerParam(validator, KAFKA_SOURCE_MAX_POLL_INTERVAL_MS_KEY, value -> value > 0); - validateIntegerParam(validator, KAFKA_SOURCE_MAX_POLL_RECORDS_KEY, value -> value > 0); - validateIntegerParam(validator, REPORT_TIME_INTERVAL_KEY, value -> value > 0); + validateIntegerParam( + validator, + KAFKA_SOURCE_SESSION_TIMEOUT_MS_KEY, + KAFKA_SOURCE_SESSION_TIMEOUT_MS_DEFAULT_VALUE, + value -> value > 0); + validateIntegerParam( + validator, + KAFKA_SOURCE_MAX_POLL_INTERVAL_MS_KEY, + KAFKA_SOURCE_MAX_POLL_INTERVAL_MS_DEFAULT_VALUE, + value -> value > 0); + validateIntegerParam( + validator, + KAFKA_SOURCE_MAX_POLL_RECORDS_KEY, + KAFKA_SOURCE_MAX_POLL_RECORDS_DEFAULT_VALUE, + value -> value > 0); + validateIntegerParam( + validator, + REPORT_TIME_INTERVAL_KEY, + REPORT_TIME_INTERVAL_DEFAULT_VALUE, + value -> value > 0); + } + + private void validateRequiredParam( + final PipeParameterValidator validator, final String paramKey) { + validator.validate(Objects::nonNull, String.format("%s is required", paramKey), paramKey); } private void validateParam( @@ -134,17 +164,13 @@ private void validateParam( paramValue); } - private void validateRequiredParam( - final PipeParameterValidator validator, final String paramKey) { - validator.validate(Objects::nonNull, String.format("%s is required", paramKey)); - } - private void validateIntegerParam( final PipeParameterValidator validator, final String paramKey, + final String paramDefaultValue, final Predicate validationCondition) { final int paramValue = - validator.getParameters().getIntOrDefault(paramKey, Integer.parseInt(paramKey)); + validator.getParameters().getIntOrDefault(paramKey, Integer.parseInt(paramDefaultValue)); validator.validate( value -> validationCondition.test((Integer) value), @@ -243,7 +269,7 @@ public void doWork() { while (isStarted && !Thread.currentThread().isInterrupted()) { markPausePosition(); - processRecords(consumer.poll(Duration.ofMillis(100))); + process(consumer.poll(Duration.ofMillis(100))); if (enableAutoCommit) { consumer.commitSync(); @@ -278,26 +304,64 @@ private void initConsumer() { consumer = new KafkaConsumer<>(props); } - private void processRecords(final ConsumerRecords records) { - records.forEach( - record -> { - try { - supplyRecord(record); - } catch (final Exception e) { - LOGGER.warn("Failed to process record at offset {}", record.offset(), e); - } - }); - } + private void process(final ConsumerRecords records) { + if (!records.isEmpty()) { + for (final ConsumerRecord record : records) { + final List schemaList = new ArrayList<>(); + final String[] dataArray = record.value().trim().split(","); + final String deviceId = dataArray[0]; + long timestamp = Long.parseLong(dataArray[1]); + final String[] measurements = dataArray[2].trim().split(":"); + final String[] typeStrings = dataArray[3].trim().split(":"); + final TSDataType[] types = new TSDataType[measurements.length]; + final String[] valueStrings = dataArray[4].trim().split(":"); + final Object[] values = new Object[measurements.length]; + + for (int i = 0; i < typeStrings.length; i++) { + types[i] = TSDataType.valueOf(typeStrings[i]); + } - private void supplyRecord(final ConsumerRecord record) { - offset = record.offset(); + for (int i = 0; i < measurements.length; i++) { + schemaList.add(new MeasurementSchema(measurements[i], types[i])); + } - LOGGER.debug( - "Consumed record: partition={}, offset={}, key={}, value={}", - instanceIndex, - record.offset(), - record.key(), - record.value()); + for (int i = 0; i < valueStrings.length; i++) { + switch (types[i]) { + case INT64: + values[i] = new long[] {Long.parseLong(valueStrings[i])}; + break; + case DOUBLE: + values[i] = new double[] {Double.parseDouble(valueStrings[i])}; + break; + case INT32: + values[i] = new int[] {Integer.parseInt(valueStrings[i])}; + break; + case TEXT: + values[i] = + new Binary[] {new Binary(valueStrings[i].getBytes(StandardCharsets.UTF_8))}; + break; + case FLOAT: + values[i] = new float[] {Float.parseFloat(valueStrings[i])}; + break; + case BOOLEAN: + values[i] = new boolean[] {Boolean.parseBoolean(valueStrings[i])}; + break; + default: + } + } + final Tablet tablet = new Tablet(deviceId, schemaList, values.length); + tablet.setTimestamps(new long[] {timestamp}); + tablet.setRowSize(values.length); + tablet.setValues(values); + + final KafkaEvent event = new KafkaEvent(tablet, deviceId); + try { + supply(event); + } catch (final Exception e) { + LOGGER.warn("failed to supply KafkaEvent {}", event, e); + } + } + } } @Override diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/kafka/KafkaSourceConstant.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/kafka/KafkaSourceConstant.java index bf9d6505..830ee803 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/kafka/KafkaSourceConstant.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/kafka/KafkaSourceConstant.java @@ -42,9 +42,9 @@ public class KafkaSourceConstant { public static final String KAFKA_SOURCE_PARTITION_ASSIGN_STRATEGY_KEY = "partition.assign.strategy"; - public static final String KAFKA_SOURCE_TOPIC_DEFAULT_VALUE = ""; + public static final String KAFKA_SOURCE_TOPIC_DEFAULT_VALUE = "my_topic"; public static final String KAFKA_SOURCE_BOOTSTRAP_SERVERS_DEFAULT_VALUE = "localhost:9092"; - public static final String KAFKA_SOURCE_GROUP_ID_DEFAULT_VALUE = ""; + public static final String KAFKA_SOURCE_GROUP_ID_DEFAULT_VALUE = "multi-thread-group"; public static final String KAFKA_SOURCE_KEY_DESERIALIZER_DEFAULT_VALUE = StringDeserializer.class.getName(); public static final String KAFKA_SOURCE_VALUE_DESERIALIZER_DEFAULT_VALUE = diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/runtime/plugin/constructor/ProcessorConstructor.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/runtime/plugin/constructor/ProcessorConstructor.java index 1e696504..9d3dbabf 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/runtime/plugin/constructor/ProcessorConstructor.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/runtime/plugin/constructor/ProcessorConstructor.java @@ -21,6 +21,7 @@ import org.apache.iotdb.collector.plugin.builtin.BuiltinPlugin; import org.apache.iotdb.collector.plugin.builtin.processor.DoNothingProcessor; +import org.apache.iotdb.collector.plugin.builtin.processor.KafkaProcessor; import org.apache.iotdb.collector.plugin.builtin.processor.SubscriptionProcessor; import org.apache.iotdb.collector.runtime.plugin.meta.PluginMetaKeeper; import org.apache.iotdb.pipe.api.PipeProcessor; @@ -38,6 +39,7 @@ protected void initConstructors() { BuiltinPlugin.DO_NOTHING_PROCESSOR.getPluginName(), DoNothingProcessor::new); pluginConstructors.put( BuiltinPlugin.SUBSCRIPTION_PROCESSOR.getPluginName(), SubscriptionProcessor::new); + pluginConstructors.put(BuiltinPlugin.KAFKA_PROCESSOR.getPluginName(), KafkaProcessor::new); } @Override diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/runtime/task/source/push/PushSourceTask.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/runtime/task/source/push/PushSourceTask.java index 1c74234b..5b9f491b 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/runtime/task/source/push/PushSourceTask.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/runtime/task/source/push/PushSourceTask.java @@ -89,29 +89,27 @@ public void createInternal() throws Exception { // register storage progress schedule job ScheduleService.reportProgress() .ifPresent( - reportEvent -> { - reportEvent.register( - taskId, - () -> { - if (pushSources != null && pushSources.length > 0) { - Map progresses = new HashMap<>(); - for (int i = 0; i < pushSources.length; i++) { - final int finalI = i; - pushSources[i] - .report() - .ifPresent(progressIndex -> progresses.put(finalI, progressIndex)); + reportEvent -> + reportEvent.register( + taskId, + () -> { + if (pushSources != null && pushSources.length > 0) { + Map progresses = new HashMap<>(); + for (int i = 0; i < pushSources.length; i++) { + final int finalI = i; + pushSources[i] + .report() + .ifPresent(progressIndex -> progresses.put(finalI, progressIndex)); + } + + PersistenceService.task() + .ifPresent( + task -> + task.tryReportTaskProgress( + new ProgressReportEvent(taskId, progresses))); } - - PersistenceService.task() - .ifPresent( - task -> - task.tryReportTaskProgress( - new ProgressReportEvent(taskId, progresses))); - LOGGER.info("successfully reported task progress {}", progresses); - } - }, - TaskRuntimeOptions.TASK_PROGRESS_REPORT_INTERVAL.value()); - }); + }, + TaskRuntimeOptions.TASK_PROGRESS_REPORT_INTERVAL.value())); } @Override From 7cb35390a4833fc4d95e91d4fe80a51b9eaf8c32 Mon Sep 17 00:00:00 2001 From: ysc <2725843507@qq.com> Date: Wed, 21 May 2025 15:22:04 +0800 Subject: [PATCH 02/11] improve iotdb subscription --- .../collector/plugin/api/BaseSource.java | 95 +++++++++ .../collector/plugin/api/PullSource.java | 22 ++- .../collector/plugin/api/PushSource.java | 23 +-- .../api/customizer/CollectorParameters.java | 58 ++++++ .../plugin/builtin/BuiltinPlugin.java | 14 +- .../builtin/processor/KafkaProcessor.java | 6 +- .../processor/SubscriptionProcessor.java | 6 +- .../event/PipeRawTabletInsertionEvent.java | 67 ++++++- .../source/constant/SourceConstant.java | 21 +- .../source/event/common/PipeRowCollector.java | 2 +- .../builtin/source/iotdb/IoTDBPushSource.java | 136 ------------- .../source/iotdb/IoTDBPushSourceConstant.java | 34 ---- .../source/iotdb/IoTDBSubscription.java | 186 ++++++++++++++++++ .../iotdb/IoTDBSubscriptionPullSource.java | 81 ++++++++ .../iotdb/IoTDBSubscriptionPushSource.java | 103 ++++++++++ .../IoTDBSubscriptionSourceConstant.java | 93 +++++++++ .../IoTDBSubscriptionTablePullSource.java | 103 ++++++++++ .../IoTDBSubscriptionTablePushSource.java | 96 +++++++++ .../IoTDBSubscriptionTreePullSource.java | 103 ++++++++++ .../IoTDBSubscriptionTreePushSource.java | 96 +++++++++ .../builtin/source/iotdb/SubDemoEvent.java | 31 --- .../builtin/source/kafka/KafkaEvent.java | 31 --- .../builtin/source/kafka/KafkaSource.java | 86 ++------ .../source/kafka/KafkaSourceConstant.java | 42 ++-- .../plugin/constructor/SourceConstructor.java | 18 +- .../task/source/pull/PullSourceTask.java | 4 +- 26 files changed, 1195 insertions(+), 362 deletions(-) create mode 100644 iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/api/BaseSource.java delete mode 100644 iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBPushSource.java delete mode 100644 iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBPushSourceConstant.java create mode 100644 iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscription.java create mode 100644 iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionPullSource.java create mode 100644 iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionPushSource.java create mode 100644 iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionSourceConstant.java create mode 100644 iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTablePullSource.java create mode 100644 iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTablePushSource.java create mode 100644 iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTreePullSource.java create mode 100644 iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTreePushSource.java delete mode 100644 iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/SubDemoEvent.java delete mode 100644 iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/kafka/KafkaEvent.java diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/api/BaseSource.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/api/BaseSource.java new file mode 100644 index 00000000..a438e81e --- /dev/null +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/api/BaseSource.java @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.collector.plugin.api; + +import org.apache.iotdb.collector.plugin.api.customizer.CollectorParameters; +import org.apache.iotdb.collector.runtime.progress.ProgressIndex; +import org.apache.iotdb.pipe.api.PipeSource; +import org.apache.iotdb.pipe.api.customizer.configuration.PipeExtractorRuntimeConfiguration; +import org.apache.iotdb.pipe.api.customizer.configuration.PipeSourceRuntimeConfiguration; +import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameterValidator; +import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameters; + +import java.util.Optional; + +import static org.apache.iotdb.collector.plugin.builtin.source.constant.SourceConstant.SOURCE_DEVICE_ID_DEFAULT_VALUE; +import static org.apache.iotdb.collector.plugin.builtin.source.constant.SourceConstant.SOURCE_DEVICE_ID_KEY; +import static org.apache.iotdb.collector.plugin.builtin.source.constant.SourceConstant.SOURCE_IS_ALIGNED_DEFAULT_VALUE; +import static org.apache.iotdb.collector.plugin.builtin.source.constant.SourceConstant.SOURCE_IS_ALIGNED_KEY; +import static org.apache.iotdb.collector.plugin.builtin.source.constant.SourceConstant.SOURCE_REPORT_TIME_INTERVAL_DEFAULT_VALUE; +import static org.apache.iotdb.collector.plugin.builtin.source.constant.SourceConstant.SOURCE_REPORT_TIME_INTERVAL_KEY; +import static org.apache.iotdb.collector.plugin.builtin.source.constant.SourceConstant.SOURCE_SQL_DIALECT_DEFAULT_VALUE; +import static org.apache.iotdb.collector.plugin.builtin.source.constant.SourceConstant.SOURCE_SQL_DIALECT_KEY; +import static org.apache.iotdb.collector.plugin.builtin.source.constant.SourceConstant.SOURCE_SQL_DIALECT_VALUE_SET; + +public abstract class BaseSource implements PipeSource { + + protected ProgressIndex startIndex; + protected int instanceIndex; + + protected String deviceId; + protected Boolean isAligned; + protected String sqlDialect; + protected int reportTimeInterval; + + @Override + public void validate(final PipeParameterValidator validator) throws Exception { + CollectorParameters.validateBooleanParam( + validator, SOURCE_IS_ALIGNED_KEY, SOURCE_IS_ALIGNED_DEFAULT_VALUE); + + CollectorParameters.validateSetParam( + validator, + SOURCE_SQL_DIALECT_KEY, + SOURCE_SQL_DIALECT_VALUE_SET, + SOURCE_SQL_DIALECT_DEFAULT_VALUE); + + CollectorParameters.validateIntegerParam( + validator, + SOURCE_REPORT_TIME_INTERVAL_KEY, + SOURCE_REPORT_TIME_INTERVAL_DEFAULT_VALUE, + value -> value > 0); + } + + @Override + public void customize( + final PipeParameters pipeParameters, + final PipeSourceRuntimeConfiguration pipeSourceRuntimeConfiguration) + throws Exception { + deviceId = + pipeParameters.getStringOrDefault(SOURCE_DEVICE_ID_KEY, SOURCE_DEVICE_ID_DEFAULT_VALUE); + isAligned = + pipeParameters.getBooleanOrDefault(SOURCE_IS_ALIGNED_KEY, SOURCE_IS_ALIGNED_DEFAULT_VALUE); + sqlDialect = + pipeParameters.getStringOrDefault(SOURCE_SQL_DIALECT_KEY, SOURCE_SQL_DIALECT_DEFAULT_VALUE); + reportTimeInterval = + pipeParameters.getIntOrDefault( + SOURCE_REPORT_TIME_INTERVAL_KEY, SOURCE_REPORT_TIME_INTERVAL_DEFAULT_VALUE); + } + + @Override + public final void customize( + PipeParameters pipeParameters, + PipeExtractorRuntimeConfiguration pipeExtractorRuntimeConfiguration) + throws Exception { + throw new UnsupportedOperationException(); + } + + public abstract Optional report(); +} diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/api/PullSource.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/api/PullSource.java index 8e5030fb..bbfaf089 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/api/PullSource.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/api/PullSource.java @@ -19,21 +19,23 @@ package org.apache.iotdb.collector.plugin.api; -import org.apache.iotdb.collector.runtime.progress.ProgressIndex; import org.apache.iotdb.pipe.api.PipeSource; -import org.apache.iotdb.pipe.api.customizer.configuration.PipeExtractorRuntimeConfiguration; +import org.apache.iotdb.pipe.api.customizer.configuration.PipeSourceRuntimeConfiguration; +import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameterValidator; import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameters; -import java.util.Optional; - -public abstract class PullSource implements PipeSource { +public abstract class PullSource extends BaseSource implements PipeSource { @Override - public final void customize( - PipeParameters pipeParameters, - PipeExtractorRuntimeConfiguration pipeExtractorRuntimeConfiguration) { - throw new UnsupportedOperationException(); + public void validate(final PipeParameterValidator validator) throws Exception { + super.validate(validator); } - public abstract Optional report(); + @Override + public void customize( + final PipeParameters pipeParameters, + final PipeSourceRuntimeConfiguration pipeSourceRuntimeConfiguration) + throws Exception { + super.customize(pipeParameters, pipeSourceRuntimeConfiguration); + } } diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/api/PushSource.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/api/PushSource.java index 90daa8f5..90a79178 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/api/PushSource.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/api/PushSource.java @@ -19,17 +19,15 @@ package org.apache.iotdb.collector.plugin.api; -import org.apache.iotdb.collector.runtime.progress.ProgressIndex; import org.apache.iotdb.collector.runtime.task.TaskDispatch; import org.apache.iotdb.pipe.api.PipeSource; import org.apache.iotdb.pipe.api.collector.EventCollector; -import org.apache.iotdb.pipe.api.customizer.configuration.PipeExtractorRuntimeConfiguration; +import org.apache.iotdb.pipe.api.customizer.configuration.PipeSourceRuntimeConfiguration; +import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameterValidator; import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameters; import org.apache.iotdb.pipe.api.event.Event; -import java.util.Optional; - -public abstract class PushSource implements PipeSource { +public abstract class PushSource extends BaseSource implements PipeSource { private TaskDispatch dispatch; @@ -40,11 +38,16 @@ public PushSource() { } @Override - public final void customize( - PipeParameters pipeParameters, - PipeExtractorRuntimeConfiguration pipeExtractorRuntimeConfiguration) + public void validate(final PipeParameterValidator validator) throws Exception { + super.validate(validator); + } + + @Override + public void customize( + final PipeParameters pipeParameters, + final PipeSourceRuntimeConfiguration pipeSourceRuntimeConfiguration) throws Exception { - throw new UnsupportedOperationException(); + super.customize(pipeParameters, pipeSourceRuntimeConfiguration); } public final void setCollector(final EventCollector collector) { @@ -75,6 +78,4 @@ public final void supply(final Event event) throws Exception { public final void setDispatch(final TaskDispatch dispatch) { this.dispatch = dispatch; } - - public abstract Optional report(); } diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/api/customizer/CollectorParameters.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/api/customizer/CollectorParameters.java index 6a7fe4b1..fe49320d 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/api/customizer/CollectorParameters.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/api/customizer/CollectorParameters.java @@ -19,10 +19,16 @@ package org.apache.iotdb.collector.plugin.api.customizer; +import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameterValidator; + import java.util.Arrays; import java.util.Collections; import java.util.HashSet; +import java.util.Objects; import java.util.Set; +import java.util.function.Predicate; + +import static org.apache.iotdb.collector.plugin.builtin.source.constant.SourceConstant.BOOLEAN_SET; public class CollectorParameters { private static final Set PARAM_SET = @@ -31,4 +37,56 @@ public class CollectorParameters { public static boolean matchAnyParam(final String param) { return PARAM_SET.contains(param); } + + public static void validateStringRequiredParam( + final PipeParameterValidator validator, final String paramKey) { + validator.validate( + o -> Objects.nonNull(validator.getParameters().getString((String) o)), + String.format("%s is required, but git null.", paramKey), + paramKey); + } + + public static void validateBooleanParam( + final PipeParameterValidator validator, final String paramKey, final boolean defaultValue) { + validateSetParam(validator, paramKey, BOOLEAN_SET, String.valueOf(defaultValue)); + } + + public static void validateSetParam( + final PipeParameterValidator validator, + final String paramKey, + final Set valueSet, + final String defaultValue) { + final String paramValue = validator.getParameters().getStringOrDefault(paramKey, defaultValue); + + validator.validate( + o -> valueSet.contains(o.toString()), + String.format("%s must be one of %s, but got %s.", paramKey, valueSet, paramValue), + paramValue); + } + + public static void validateIntegerParam( + final PipeParameterValidator validator, + final String paramKey, + final Integer paramDefaultValue, + final Predicate validationCondition) { + final int paramValue = validator.getParameters().getIntOrDefault(paramKey, paramDefaultValue); + + validator.validate( + value -> validationCondition.test((Integer) value), + String.format("%s must be > 0, but got %d.", paramKey, paramValue), + paramValue); + } + + public static void validateLongParam( + final PipeParameterValidator validator, + final String paramKey, + final Long paramDefaultValue, + final Predicate validationCondition) { + final long paramValue = validator.getParameters().getLongOrDefault(paramKey, paramDefaultValue); + + validator.validate( + value -> validationCondition.test((Long) value), + String.format("%s must be > 0, but got %d.", paramKey, paramValue), + paramValue); + } } diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/BuiltinPlugin.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/BuiltinPlugin.java index 43505caa..f2dccccd 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/BuiltinPlugin.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/BuiltinPlugin.java @@ -26,7 +26,10 @@ import org.apache.iotdb.collector.plugin.builtin.sink.protocol.IoTDBDataRegionSyncConnector; import org.apache.iotdb.collector.plugin.builtin.source.HttpPullSource; import org.apache.iotdb.collector.plugin.builtin.source.HttpPushSource; -import org.apache.iotdb.collector.plugin.builtin.source.iotdb.IoTDBPushSource; +import org.apache.iotdb.collector.plugin.builtin.source.iotdb.IoTDBSubscriptionTablePullSource; +import org.apache.iotdb.collector.plugin.builtin.source.iotdb.IoTDBSubscriptionTablePushSource; +import org.apache.iotdb.collector.plugin.builtin.source.iotdb.IoTDBSubscriptionTreePullSource; +import org.apache.iotdb.collector.plugin.builtin.source.iotdb.IoTDBSubscriptionTreePushSource; import org.apache.iotdb.collector.plugin.builtin.source.kafka.KafkaSource; public enum BuiltinPlugin { @@ -34,7 +37,14 @@ public enum BuiltinPlugin { // Push Sources HTTP_PUSH_SOURCE("http-push-source", HttpPushSource.class), KAFKA_SOURCE("kafka-source", KafkaSource.class), - SUBSCRIPTION_SOURCE("subscription-source", IoTDBPushSource.class), + IOTDB_SUBSCRIPTION_TREE_PUSH_SOURCE( + "iotdb-subscription-tree-push-source", IoTDBSubscriptionTreePushSource.class), + IOTDB_SUBSCRIPTION_TREE_PULL_SOURCE( + "iotdb-subscription-tree-pull-source", IoTDBSubscriptionTreePullSource.class), + IOTDB_SUBSCRIPTION_TABLE_PUSH_SOURCE( + "iotdb-subscription-table-push-source", IoTDBSubscriptionTablePushSource.class), + IOTDB_SUBSCRIPTION_TABLE_PULL_SOURCE( + "iotdb-subscription-table-pull-source", IoTDBSubscriptionTablePullSource.class), // Pull Sources, HTTP_PULL_SOURCE("http-pull-source", HttpPullSource.class), diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/processor/KafkaProcessor.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/processor/KafkaProcessor.java index 49765e1b..46ba3389 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/processor/KafkaProcessor.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/processor/KafkaProcessor.java @@ -19,7 +19,7 @@ package org.apache.iotdb.collector.plugin.builtin.processor; -import org.apache.iotdb.collector.plugin.builtin.source.kafka.KafkaEvent; +import org.apache.iotdb.collector.plugin.builtin.sink.event.PipeRawTabletInsertionEvent; import org.apache.iotdb.pipe.api.PipeProcessor; import org.apache.iotdb.pipe.api.collector.EventCollector; import org.apache.iotdb.pipe.api.customizer.configuration.PipeProcessorRuntimeConfiguration; @@ -46,8 +46,8 @@ public void process(TabletInsertionEvent tabletInsertionEvent, EventCollector ev @Override public void process(Event event, EventCollector eventCollector) throws Exception { - if (event instanceof KafkaEvent) { - process((KafkaEvent) event, eventCollector); + if (event instanceof PipeRawTabletInsertionEvent) { + process((PipeRawTabletInsertionEvent) event, eventCollector); } } diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/processor/SubscriptionProcessor.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/processor/SubscriptionProcessor.java index 1b933325..d2a425dc 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/processor/SubscriptionProcessor.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/processor/SubscriptionProcessor.java @@ -19,7 +19,7 @@ package org.apache.iotdb.collector.plugin.builtin.processor; -import org.apache.iotdb.collector.plugin.builtin.source.iotdb.SubDemoEvent; +import org.apache.iotdb.collector.plugin.builtin.sink.event.PipeRawTabletInsertionEvent; import org.apache.iotdb.pipe.api.PipeProcessor; import org.apache.iotdb.pipe.api.collector.EventCollector; import org.apache.iotdb.pipe.api.customizer.configuration.PipeProcessorRuntimeConfiguration; @@ -46,8 +46,8 @@ public void process(TabletInsertionEvent tabletInsertionEvent, EventCollector ev @Override public void process(Event event, EventCollector eventCollector) throws Exception { - if (event instanceof SubDemoEvent) { - process((SubDemoEvent) event, eventCollector); + if (event instanceof PipeRawTabletInsertionEvent) { + process((PipeRawTabletInsertionEvent) event, eventCollector); } } diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/event/PipeRawTabletInsertionEvent.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/event/PipeRawTabletInsertionEvent.java index 37153c94..b533e084 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/event/PipeRawTabletInsertionEvent.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/event/PipeRawTabletInsertionEvent.java @@ -19,38 +19,91 @@ package org.apache.iotdb.collector.plugin.builtin.sink.event; +import org.apache.iotdb.collector.plugin.builtin.source.event.common.PipeRow; import org.apache.iotdb.collector.plugin.builtin.source.event.common.PipeRowCollector; import org.apache.iotdb.pipe.api.access.Row; import org.apache.iotdb.pipe.api.collector.RowCollector; import org.apache.iotdb.pipe.api.event.dml.insertion.TabletInsertionEvent; +import org.apache.tsfile.enums.TSDataType; import org.apache.tsfile.write.record.Tablet; +import org.apache.tsfile.write.schema.IMeasurementSchema; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import java.util.Collections; import java.util.function.BiConsumer; public class PipeRawTabletInsertionEvent extends PipeInsertionEvent implements TabletInsertionEvent, AutoCloseable { - public String deviceId; + private static final Logger LOGGER = LoggerFactory.getLogger(PipeRawTabletInsertionEvent.class); + protected Tablet tablet; - private boolean isAligned; + private final boolean isAligned; + + private final boolean isTableMod; - public PipeRawTabletInsertionEvent(final Tablet tablet, final String deviceId) { - this.deviceId = deviceId; + public PipeRawTabletInsertionEvent(final Tablet tablet, final boolean isAligned) { this.tablet = tablet; + this.isAligned = isAligned; - this.tablet.setDeviceId(deviceId); + this.isTableMod = tablet.getTableName() != null; } @Override public Iterable processRowByRow( final BiConsumer consumer) { - throw new UnsupportedOperationException(); + if (isTableMod) { + if (LOGGER.isDebugEnabled()) { + LOGGER.warn("TablePatternParser does not support row by row processing"); + } + + return Collections.emptyList(); + } + + if (tablet.getValues().length == 0 || tablet.getTimestamps().length == 0) { + return Collections.emptyList(); + } + + final IMeasurementSchema[] schemas = tablet.getSchemas().toArray(new IMeasurementSchema[0]); + final String[] columnNames = new String[schemas.length]; + final TSDataType[] valueColumnTypes = new TSDataType[schemas.length]; + for (int i = 0; i < schemas.length; i++) { + columnNames[i] = schemas[i].getMeasurementName(); + valueColumnTypes[i] = schemas[i].getType(); + } + + final PipeRowCollector rowCollector = new PipeRowCollector(); + for (int i = 0; i < tablet.getTimestamps().length; i++) { + consumer.accept( + new PipeRow( + i, + tablet.getDeviceId(), + isAligned, + schemas, + tablet.getTimestamps(), + valueColumnTypes, + tablet.getValues(), + tablet.getBitMaps(), + columnNames), + rowCollector); + } + + return rowCollector.convertToTabletInsertionEvents(); } @Override public Iterable processTablet( final BiConsumer consumer) { + if (isTableMod) { + if (LOGGER.isDebugEnabled()) { + LOGGER.warn("TablePatternParser does not support tablet processing"); + } + + return Collections.emptyList(); + } + final PipeRowCollector rowCollector = new PipeRowCollector(); consumer.accept(convertToTablet(), rowCollector); return rowCollector.convertToTabletInsertionEvents(); @@ -73,7 +126,7 @@ public boolean isTableModelEvent() { } public String getDeviceId() { - return deviceId; + return tablet.getDeviceId(); } public String getTableModelDatabaseName() { diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/constant/SourceConstant.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/constant/SourceConstant.java index 1da7ff65..23b644fe 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/constant/SourceConstant.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/constant/SourceConstant.java @@ -19,9 +19,26 @@ package org.apache.iotdb.collector.plugin.builtin.source.constant; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.Set; + public class SourceConstant { - public static final String REPORT_TIME_INTERVAL_KEY = "report-time-interval"; + public static final String SOURCE_REPORT_TIME_INTERVAL_KEY = "report-time-interval"; + public static final String SOURCE_IS_ALIGNED_KEY = "is-aligned"; + public static final String SOURCE_DEVICE_ID_KEY = "device-id"; + + public static final int SOURCE_REPORT_TIME_INTERVAL_DEFAULT_VALUE = 60; + public static final boolean SOURCE_IS_ALIGNED_DEFAULT_VALUE = false; + public static final String SOURCE_DEVICE_ID_DEFAULT_VALUE = "root.test"; + + public static final String SOURCE_SQL_DIALECT_KEY = "sql-dialect"; + public static final String SOURCE_SQL_DIALECT_DEFAULT_VALUE = "tree"; + public static final Set SOURCE_SQL_DIALECT_VALUE_SET = + Collections.unmodifiableSet(new HashSet<>(Arrays.asList("tree", "table"))); - public static final String REPORT_TIME_INTERVAL_DEFAULT_VALUE = "60"; + public static final Set BOOLEAN_SET = + Collections.unmodifiableSet(new HashSet<>(Arrays.asList("true", "false"))); } diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/event/common/PipeRowCollector.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/event/common/PipeRowCollector.java index cd222e05..8ad21b7d 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/event/common/PipeRowCollector.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/event/common/PipeRowCollector.java @@ -90,7 +90,7 @@ public void collectRow(Row row) { private void collectTabletInsertionEvent() { if (tablet != null) { // TODO: non-PipeInsertionEvent sourceEvent is not supported? - tabletInsertionEventList.add(new PipeRawTabletInsertionEvent(tablet, tablet.getDeviceId())); + tabletInsertionEventList.add(new PipeRawTabletInsertionEvent(tablet, false)); } this.tablet = null; } diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBPushSource.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBPushSource.java deleted file mode 100644 index 7b97703a..00000000 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBPushSource.java +++ /dev/null @@ -1,136 +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.collector.plugin.builtin.source.iotdb; - -import org.apache.iotdb.collector.plugin.api.PushSource; -import org.apache.iotdb.collector.runtime.progress.ProgressIndex; -import org.apache.iotdb.pipe.api.customizer.configuration.PipeSourceRuntimeConfiguration; -import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameterValidator; -import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameters; -import org.apache.iotdb.rpc.subscription.config.ConsumerConstant; -import org.apache.iotdb.session.subscription.consumer.tree.SubscriptionTreePullConsumer; -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.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.List; -import java.util.Optional; -import java.util.Properties; - -public class IoTDBPushSource extends PushSource { - - private static final Logger LOGGER = LoggerFactory.getLogger(IoTDBPushSource.class); - - private String host; - private Integer port; - private String topic; - private Long timeout; - private String deviceId; - - private volatile boolean isStarted = true; - private Thread workerThread; - - @Override - public void validate(PipeParameterValidator validator) throws Exception {} - - @Override - public void customize( - PipeParameters pipeParameters, PipeSourceRuntimeConfiguration pipeSourceRuntimeConfiguration) - throws Exception { - host = - pipeParameters.getStringOrDefault( - IoTDBPushSourceConstant.HOST_KEY, IoTDBPushSourceConstant.HOST_VALUE); - port = - pipeParameters.getIntOrDefault( - IoTDBPushSourceConstant.PORT_KEY, IoTDBPushSourceConstant.PORT_VALUE); - topic = - pipeParameters.getStringOrDefault( - IoTDBPushSourceConstant.TOPIC_KEY, IoTDBPushSourceConstant.TOPIC_VALUE); - timeout = - pipeParameters.getLongOrDefault( - IoTDBPushSourceConstant.TIMEOUT_KEY, IoTDBPushSourceConstant.TIMEOUT_VALUE); - deviceId = - pipeParameters.getStringOrDefault( - IoTDBPushSourceConstant.DEVICE_ID_KEY, IoTDBPushSourceConstant.DEVICE_ID_VALUE); - } - - @Override - public void start() throws Exception { - if (workerThread == null || !workerThread.isAlive()) { - isStarted = true; - workerThread = new Thread(this::doWork); - workerThread.start(); - } - } - - private void doWork() { - final Properties pullProperties = new Properties(); - pullProperties.put(IoTDBPushSourceConstant.HOST_KEY, host); - pullProperties.put(IoTDBPushSourceConstant.PORT_KEY, port); - pullProperties.put(ConsumerConstant.CONSUMER_ID_KEY, "r1"); - pullProperties.put(ConsumerConstant.CONSUMER_GROUP_ID_KEY, "rg1"); - - try (final SubscriptionTreePullConsumer consumer = - new SubscriptionTreePullConsumer(pullProperties)) { - consumer.open(); - consumer.subscribe(topic); - - while (isStarted && !Thread.currentThread().isInterrupted()) { - markPausePosition(); - - final List messages = consumer.poll(timeout); - for (final SubscriptionMessage message : messages) { - final short messageType = message.getMessageType(); - if (SubscriptionMessageType.isValidatedMessageType(messageType)) { - for (final SubscriptionSessionDataSet dataSet : message.getSessionDataSetsHandler()) { - final SubDemoEvent event = new SubDemoEvent(dataSet.getTablet(), deviceId); - supply(event); - } - } - } - } - } catch (final Exception e) { - Thread.currentThread().interrupt(); - LOGGER.error("Error in push source", e); - } - } - - @Override - public void close() throws Exception { - isStarted = false; - if (workerThread != null) { - workerThread.interrupt(); - try { - workerThread.join(1000); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - } - workerThread = null; - } - } - - @Override - public Optional report() { - return Optional.empty(); - } -} diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBPushSourceConstant.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBPushSourceConstant.java deleted file mode 100644 index aefa0198..00000000 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBPushSourceConstant.java +++ /dev/null @@ -1,34 +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.collector.plugin.builtin.source.iotdb; - -public class IoTDBPushSourceConstant { - public static final String HOST_KEY = "host"; - public static final String PORT_KEY = "port"; - public static final String TOPIC_KEY = "topic"; - public static final String TIMEOUT_KEY = "timeout"; - public static final String DEVICE_ID_KEY = "deviceId"; - - public static final String HOST_VALUE = "127.0.0.1"; - public static final Integer PORT_VALUE = 6668; - public static final String TOPIC_VALUE = "root_all"; - public static final Long TIMEOUT_VALUE = 10000L; - public static final String DEVICE_ID_VALUE = "root.test.demo"; -} diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscription.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscription.java new file mode 100644 index 00000000..253758b6 --- /dev/null +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscription.java @@ -0,0 +1,186 @@ +/* + * 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.collector.plugin.builtin.source.iotdb; + +import org.apache.iotdb.collector.plugin.api.customizer.CollectorParameters; +import org.apache.iotdb.pipe.api.customizer.configuration.PipeSourceRuntimeConfiguration; +import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameterValidator; +import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameters; +import org.apache.iotdb.pipe.api.event.Event; +import org.apache.iotdb.session.subscription.consumer.base.AbstractSubscriptionConsumerBuilder; + +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.locks.LockSupport; + +import static org.apache.iotdb.collector.plugin.builtin.source.constant.SourceConstant.SOURCE_IS_ALIGNED_DEFAULT_VALUE; +import static org.apache.iotdb.collector.plugin.builtin.source.constant.SourceConstant.SOURCE_IS_ALIGNED_KEY; +import static org.apache.iotdb.collector.plugin.builtin.source.constant.SourceConstant.SOURCE_SQL_DIALECT_DEFAULT_VALUE; +import static org.apache.iotdb.collector.plugin.builtin.source.constant.SourceConstant.SOURCE_SQL_DIALECT_KEY; +import static org.apache.iotdb.collector.plugin.builtin.source.constant.SourceConstant.SOURCE_SQL_DIALECT_VALUE_SET; +import static org.apache.iotdb.collector.plugin.builtin.source.iotdb.IoTDBSubscriptionSourceConstant.IOTDB_SUBSCRIPTION_SOURCE_AUTO_POLL_TIMEOUT_MS_DEFAULT_VALUE; +import static org.apache.iotdb.collector.plugin.builtin.source.iotdb.IoTDBSubscriptionSourceConstant.IOTDB_SUBSCRIPTION_SOURCE_AUTO_POLL_TIMEOUT_MS_KEY; +import static org.apache.iotdb.collector.plugin.builtin.source.iotdb.IoTDBSubscriptionSourceConstant.IOTDB_SUBSCRIPTION_SOURCE_AUTO_POLL_TIMEOUT_MS_MIN_VALUE; +import static org.apache.iotdb.collector.plugin.builtin.source.iotdb.IoTDBSubscriptionSourceConstant.IOTDB_SUBSCRIPTION_SOURCE_CONSUMER_ID_KEY; +import static org.apache.iotdb.collector.plugin.builtin.source.iotdb.IoTDBSubscriptionSourceConstant.IOTDB_SUBSCRIPTION_SOURCE_ENDPOINT_SYNC_INTERVAL_MS_DEFAULT_VALUE; +import static org.apache.iotdb.collector.plugin.builtin.source.iotdb.IoTDBSubscriptionSourceConstant.IOTDB_SUBSCRIPTION_SOURCE_ENDPOINT_SYNC_INTERVAL_MS_KEY; +import static org.apache.iotdb.collector.plugin.builtin.source.iotdb.IoTDBSubscriptionSourceConstant.IOTDB_SUBSCRIPTION_SOURCE_ENDPOINT_SYNC_INTERVAL_MS_MIN_VALUE; +import static org.apache.iotdb.collector.plugin.builtin.source.iotdb.IoTDBSubscriptionSourceConstant.IOTDB_SUBSCRIPTION_SOURCE_GROUP_ID_KEY; +import static org.apache.iotdb.collector.plugin.builtin.source.iotdb.IoTDBSubscriptionSourceConstant.IOTDB_SUBSCRIPTION_SOURCE_HEARTBEAT_INTERVAL_MS_DEFAULT_VALUE; +import static org.apache.iotdb.collector.plugin.builtin.source.iotdb.IoTDBSubscriptionSourceConstant.IOTDB_SUBSCRIPTION_SOURCE_HEARTBEAT_INTERVAL_MS_KEY; +import static org.apache.iotdb.collector.plugin.builtin.source.iotdb.IoTDBSubscriptionSourceConstant.IOTDB_SUBSCRIPTION_SOURCE_HEARTBEAT_INTERVAL_MS_MIN_VALUE; +import static org.apache.iotdb.collector.plugin.builtin.source.iotdb.IoTDBSubscriptionSourceConstant.IOTDB_SUBSCRIPTION_SOURCE_HOST_DEFAULT_VALUE; +import static org.apache.iotdb.collector.plugin.builtin.source.iotdb.IoTDBSubscriptionSourceConstant.IOTDB_SUBSCRIPTION_SOURCE_HOST_KEY; +import static org.apache.iotdb.collector.plugin.builtin.source.iotdb.IoTDBSubscriptionSourceConstant.IOTDB_SUBSCRIPTION_SOURCE_MAX_POLL_PARALLELISM_DEFAULT_VALUE; +import static org.apache.iotdb.collector.plugin.builtin.source.iotdb.IoTDBSubscriptionSourceConstant.IOTDB_SUBSCRIPTION_SOURCE_MAX_POLL_PARALLELISM_KEY; +import static org.apache.iotdb.collector.plugin.builtin.source.iotdb.IoTDBSubscriptionSourceConstant.IOTDB_SUBSCRIPTION_SOURCE_PORT_DEFAULT_VALUE; +import static org.apache.iotdb.collector.plugin.builtin.source.iotdb.IoTDBSubscriptionSourceConstant.IOTDB_SUBSCRIPTION_SOURCE_PORT_KEY; +import static org.apache.iotdb.collector.plugin.builtin.source.iotdb.IoTDBSubscriptionSourceConstant.IOTDB_SUBSCRIPTION_SOURCE_THRIFT_MAX_FRAME_SIZE_DEFAULT_VALUE; +import static org.apache.iotdb.collector.plugin.builtin.source.iotdb.IoTDBSubscriptionSourceConstant.IOTDB_SUBSCRIPTION_SOURCE_THRIFT_MAX_FRAME_SIZE_KEY; +import static org.apache.iotdb.collector.plugin.builtin.source.iotdb.IoTDBSubscriptionSourceConstant.IOTDB_SUBSCRIPTION_SOURCE_TOPIC_DEFAULT_VALUE; +import static org.apache.iotdb.collector.plugin.builtin.source.iotdb.IoTDBSubscriptionSourceConstant.IOTDB_SUBSCRIPTION_SOURCE_TOPIC_KEY; + +public class IoTDBSubscription { + + private String host; + private Integer port; + private String consumerId; + private String groupId; + private Long heartbeatIntervalMs; + private Long endpointsSyncIntervalMs; + private Integer thriftMaxFrameSize; + private Integer maxPollParallelism; + + private String topic; + + protected final BlockingQueue eventQueue = new LinkedBlockingQueue<>(); + protected final Integer eventQueueCapacity = 1000; + protected final Long eventQueuePauseIntervalMs = 100_000_000L; + + public void validate(final PipeParameterValidator validator) { + CollectorParameters.validateStringRequiredParam(validator, IOTDB_SUBSCRIPTION_SOURCE_TOPIC_KEY); + CollectorParameters.validateStringRequiredParam( + validator, IOTDB_SUBSCRIPTION_SOURCE_GROUP_ID_KEY); + CollectorParameters.validateStringRequiredParam( + validator, IOTDB_SUBSCRIPTION_SOURCE_CONSUMER_ID_KEY); + + CollectorParameters.validateBooleanParam( + validator, SOURCE_IS_ALIGNED_KEY, SOURCE_IS_ALIGNED_DEFAULT_VALUE); + + CollectorParameters.validateIntegerParam( + validator, + IOTDB_SUBSCRIPTION_SOURCE_PORT_KEY, + IOTDB_SUBSCRIPTION_SOURCE_PORT_DEFAULT_VALUE, + value -> value > 0); + CollectorParameters.validateIntegerParam( + validator, + IOTDB_SUBSCRIPTION_SOURCE_THRIFT_MAX_FRAME_SIZE_KEY, + IOTDB_SUBSCRIPTION_SOURCE_THRIFT_MAX_FRAME_SIZE_DEFAULT_VALUE, + value -> value > 0); + CollectorParameters.validateIntegerParam( + validator, + IOTDB_SUBSCRIPTION_SOURCE_MAX_POLL_PARALLELISM_KEY, + IOTDB_SUBSCRIPTION_SOURCE_MAX_POLL_PARALLELISM_DEFAULT_VALUE, + value -> value > 0); + + CollectorParameters.validateLongParam( + validator, + IOTDB_SUBSCRIPTION_SOURCE_AUTO_POLL_TIMEOUT_MS_KEY, + IOTDB_SUBSCRIPTION_SOURCE_AUTO_POLL_TIMEOUT_MS_DEFAULT_VALUE, + value -> value >= IOTDB_SUBSCRIPTION_SOURCE_AUTO_POLL_TIMEOUT_MS_MIN_VALUE); + CollectorParameters.validateLongParam( + validator, + IOTDB_SUBSCRIPTION_SOURCE_HEARTBEAT_INTERVAL_MS_KEY, + IOTDB_SUBSCRIPTION_SOURCE_HEARTBEAT_INTERVAL_MS_DEFAULT_VALUE, + value -> value >= IOTDB_SUBSCRIPTION_SOURCE_HEARTBEAT_INTERVAL_MS_MIN_VALUE); + CollectorParameters.validateLongParam( + validator, + IOTDB_SUBSCRIPTION_SOURCE_ENDPOINT_SYNC_INTERVAL_MS_KEY, + IOTDB_SUBSCRIPTION_SOURCE_ENDPOINT_SYNC_INTERVAL_MS_DEFAULT_VALUE, + value -> value >= IOTDB_SUBSCRIPTION_SOURCE_ENDPOINT_SYNC_INTERVAL_MS_MIN_VALUE); + + CollectorParameters.validateSetParam( + validator, + SOURCE_SQL_DIALECT_KEY, + SOURCE_SQL_DIALECT_VALUE_SET, + SOURCE_SQL_DIALECT_DEFAULT_VALUE); + } + + public void customize( + final PipeParameters pipeParameters, final PipeSourceRuntimeConfiguration configuration) { + host = + pipeParameters.getStringOrDefault( + IOTDB_SUBSCRIPTION_SOURCE_HOST_KEY, IOTDB_SUBSCRIPTION_SOURCE_HOST_DEFAULT_VALUE); + port = + pipeParameters.getIntOrDefault( + IOTDB_SUBSCRIPTION_SOURCE_PORT_KEY, IOTDB_SUBSCRIPTION_SOURCE_PORT_DEFAULT_VALUE); + topic = + pipeParameters.getStringOrDefault( + IOTDB_SUBSCRIPTION_SOURCE_TOPIC_KEY, IOTDB_SUBSCRIPTION_SOURCE_TOPIC_DEFAULT_VALUE); + consumerId = pipeParameters.getString(IOTDB_SUBSCRIPTION_SOURCE_CONSUMER_ID_KEY); + groupId = pipeParameters.getString(IOTDB_SUBSCRIPTION_SOURCE_GROUP_ID_KEY); + heartbeatIntervalMs = + pipeParameters.getLongOrDefault( + IOTDB_SUBSCRIPTION_SOURCE_HEARTBEAT_INTERVAL_MS_KEY, + IOTDB_SUBSCRIPTION_SOURCE_HEARTBEAT_INTERVAL_MS_DEFAULT_VALUE); + endpointsSyncIntervalMs = + pipeParameters.getLongOrDefault( + IOTDB_SUBSCRIPTION_SOURCE_ENDPOINT_SYNC_INTERVAL_MS_KEY, + IOTDB_SUBSCRIPTION_SOURCE_ENDPOINT_SYNC_INTERVAL_MS_DEFAULT_VALUE); + thriftMaxFrameSize = + pipeParameters.getIntOrDefault( + IOTDB_SUBSCRIPTION_SOURCE_THRIFT_MAX_FRAME_SIZE_KEY, + IOTDB_SUBSCRIPTION_SOURCE_THRIFT_MAX_FRAME_SIZE_DEFAULT_VALUE); + maxPollParallelism = + pipeParameters.getIntOrDefault( + IOTDB_SUBSCRIPTION_SOURCE_MAX_POLL_PARALLELISM_KEY, + IOTDB_SUBSCRIPTION_SOURCE_MAX_POLL_PARALLELISM_DEFAULT_VALUE); + } + + public AbstractSubscriptionConsumerBuilder getSubscriptionConsumerBuilder() { + return new AbstractSubscriptionConsumerBuilder() + .host(host) + .port(port) + .consumerId(consumerId) + .consumerGroupId(groupId) + .heartbeatIntervalMs(heartbeatIntervalMs) + .endpointsSyncIntervalMs(endpointsSyncIntervalMs) + .thriftMaxFrameSize(thriftMaxFrameSize) + .maxPollParallelism(maxPollParallelism); + } + + public Event take() throws InterruptedException { + return eventQueue.take(); + } + + public void put(final Event event) throws InterruptedException { + eventQueue.put(event); + } + + public void checkIfNeedPause() { + while (eventQueue.size() >= eventQueueCapacity) { + LockSupport.parkNanos(eventQueuePauseIntervalMs); + } + } + + public String getTopic() { + return topic; + } +} diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionPullSource.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionPullSource.java new file mode 100644 index 00000000..83d32e0a --- /dev/null +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionPullSource.java @@ -0,0 +1,81 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.collector.plugin.builtin.source.iotdb; + +import org.apache.iotdb.collector.plugin.api.PullSource; +import org.apache.iotdb.collector.plugin.api.customizer.CollectorParameters; +import org.apache.iotdb.pipe.api.customizer.configuration.PipeSourceRuntimeConfiguration; +import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameterValidator; +import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameters; +import org.apache.iotdb.session.subscription.consumer.base.AbstractSubscriptionPullConsumerBuilder; + +import static org.apache.iotdb.collector.plugin.builtin.source.iotdb.IoTDBSubscriptionSourceConstant.IOTDB_SUBSCRIPTION_SOURCE_AUTO_COMMIT_DEFAULT_VALUE; +import static org.apache.iotdb.collector.plugin.builtin.source.iotdb.IoTDBSubscriptionSourceConstant.IOTDB_SUBSCRIPTION_SOURCE_AUTO_COMMIT_INTERVAL_MS_DEFAULT_VALUE; +import static org.apache.iotdb.collector.plugin.builtin.source.iotdb.IoTDBSubscriptionSourceConstant.IOTDB_SUBSCRIPTION_SOURCE_AUTO_COMMIT_INTERVAL_MS_KEY; +import static org.apache.iotdb.collector.plugin.builtin.source.iotdb.IoTDBSubscriptionSourceConstant.IOTDB_SUBSCRIPTION_SOURCE_AUTO_COMMIT_INTERVAL_MS_MIN_VALUE; +import static org.apache.iotdb.collector.plugin.builtin.source.iotdb.IoTDBSubscriptionSourceConstant.IOTDB_SUBSCRIPTION_SOURCE_AUTO_COMMIT_KEY; + +public abstract class IoTDBSubscriptionPullSource extends PullSource { + + protected final IoTDBSubscription subscription = new IoTDBSubscription(); + + private Boolean autoCommit; + private Long autoCommitIntervalMs; + + @Override + public void validate(final PipeParameterValidator validator) throws Exception { + super.validate(validator); + subscription.validate(validator); + + CollectorParameters.validateBooleanParam( + validator, + IOTDB_SUBSCRIPTION_SOURCE_AUTO_COMMIT_KEY, + IOTDB_SUBSCRIPTION_SOURCE_AUTO_COMMIT_DEFAULT_VALUE); + + CollectorParameters.validateLongParam( + validator, + IOTDB_SUBSCRIPTION_SOURCE_AUTO_COMMIT_INTERVAL_MS_KEY, + IOTDB_SUBSCRIPTION_SOURCE_AUTO_COMMIT_INTERVAL_MS_DEFAULT_VALUE, + value -> value >= IOTDB_SUBSCRIPTION_SOURCE_AUTO_COMMIT_INTERVAL_MS_MIN_VALUE); + } + + @Override + public void customize( + PipeParameters pipeParameters, PipeSourceRuntimeConfiguration pipeSourceRuntimeConfiguration) + throws Exception { + super.customize(pipeParameters, pipeSourceRuntimeConfiguration); + subscription.customize(pipeParameters, pipeSourceRuntimeConfiguration); + + autoCommit = + pipeParameters.getBooleanOrDefault( + IOTDB_SUBSCRIPTION_SOURCE_AUTO_COMMIT_KEY, + IOTDB_SUBSCRIPTION_SOURCE_AUTO_COMMIT_DEFAULT_VALUE); + autoCommitIntervalMs = + pipeParameters.getLongOrDefault( + IOTDB_SUBSCRIPTION_SOURCE_AUTO_COMMIT_INTERVAL_MS_KEY, + IOTDB_SUBSCRIPTION_SOURCE_AUTO_COMMIT_INTERVAL_MS_DEFAULT_VALUE); + } + + protected AbstractSubscriptionPullConsumerBuilder getSubscriptionPullConsumerBuilder() { + return ((AbstractSubscriptionPullConsumerBuilder) subscription.getSubscriptionConsumerBuilder()) + .autoCommit(autoCommit) + .autoCommitIntervalMs(autoCommitIntervalMs); + } +} diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionPushSource.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionPushSource.java new file mode 100644 index 00000000..cc518eed --- /dev/null +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionPushSource.java @@ -0,0 +1,103 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.collector.plugin.builtin.source.iotdb; + +import org.apache.iotdb.collector.plugin.api.PushSource; +import org.apache.iotdb.collector.plugin.api.customizer.CollectorParameters; +import org.apache.iotdb.pipe.api.customizer.configuration.PipeSourceRuntimeConfiguration; +import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameterValidator; +import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameters; +import org.apache.iotdb.session.subscription.consumer.AckStrategy; +import org.apache.iotdb.session.subscription.consumer.base.AbstractSubscriptionPushConsumerBuilder; + +import static org.apache.iotdb.collector.plugin.builtin.source.iotdb.IoTDBSubscriptionSourceConstant.IOTDB_SUBSCRIPTION_SOURCE_ACK_STRATEGY_DEFAULT_VALUE; +import static org.apache.iotdb.collector.plugin.builtin.source.iotdb.IoTDBSubscriptionSourceConstant.IOTDB_SUBSCRIPTION_SOURCE_ACK_STRATEGY_KEY; +import static org.apache.iotdb.collector.plugin.builtin.source.iotdb.IoTDBSubscriptionSourceConstant.IOTDB_SUBSCRIPTION_SOURCE_ACK_STRATEGY_VALUE_MAP; +import static org.apache.iotdb.collector.plugin.builtin.source.iotdb.IoTDBSubscriptionSourceConstant.IOTDB_SUBSCRIPTION_SOURCE_AUTO_POLL_INTERVAL_MS_DEFAULT_VALUE; +import static org.apache.iotdb.collector.plugin.builtin.source.iotdb.IoTDBSubscriptionSourceConstant.IOTDB_SUBSCRIPTION_SOURCE_AUTO_POLL_INTERVAL_MS_KEY; +import static org.apache.iotdb.collector.plugin.builtin.source.iotdb.IoTDBSubscriptionSourceConstant.IOTDB_SUBSCRIPTION_SOURCE_AUTO_POLL_TIMEOUT_MS_DEFAULT_VALUE; +import static org.apache.iotdb.collector.plugin.builtin.source.iotdb.IoTDBSubscriptionSourceConstant.IOTDB_SUBSCRIPTION_SOURCE_AUTO_POLL_TIMEOUT_MS_KEY; +import static org.apache.iotdb.collector.plugin.builtin.source.iotdb.IoTDBSubscriptionSourceConstant.IOTDB_SUBSCRIPTION_SOURCE_AUTO_POLL_TIMEOUT_MS_MIN_VALUE; + +public abstract class IoTDBSubscriptionPushSource extends PushSource { + + protected volatile boolean isStarted = true; + protected Thread workerThread; + + private Long autoPollIntervalMs; + private Long autoPollTimeoutMs; + private AckStrategy ackStrategy; + + protected final IoTDBSubscription subscription = new IoTDBSubscription(); + + @Override + public void validate(final PipeParameterValidator validator) throws Exception { + super.validate(validator); + subscription.validate(validator); + + CollectorParameters.validateSetParam( + validator, + IOTDB_SUBSCRIPTION_SOURCE_ACK_STRATEGY_KEY, + IOTDB_SUBSCRIPTION_SOURCE_ACK_STRATEGY_VALUE_MAP.keySet(), + IOTDB_SUBSCRIPTION_SOURCE_ACK_STRATEGY_DEFAULT_VALUE); + + CollectorParameters.validateLongParam( + validator, + IOTDB_SUBSCRIPTION_SOURCE_AUTO_POLL_INTERVAL_MS_KEY, + IOTDB_SUBSCRIPTION_SOURCE_AUTO_POLL_INTERVAL_MS_DEFAULT_VALUE, + value -> value > 0); + CollectorParameters.validateLongParam( + validator, + IOTDB_SUBSCRIPTION_SOURCE_AUTO_POLL_TIMEOUT_MS_KEY, + IOTDB_SUBSCRIPTION_SOURCE_AUTO_POLL_TIMEOUT_MS_DEFAULT_VALUE, + value -> value >= IOTDB_SUBSCRIPTION_SOURCE_AUTO_POLL_TIMEOUT_MS_MIN_VALUE); + } + + @Override + public void customize( + final PipeParameters pipeParameters, + final PipeSourceRuntimeConfiguration pipeSourceRuntimeConfiguration) + throws Exception { + super.customize(pipeParameters, pipeSourceRuntimeConfiguration); + subscription.customize(pipeParameters, pipeSourceRuntimeConfiguration); + + ackStrategy = + IOTDB_SUBSCRIPTION_SOURCE_ACK_STRATEGY_VALUE_MAP.get( + pipeParameters.getStringOrDefault( + IOTDB_SUBSCRIPTION_SOURCE_ACK_STRATEGY_KEY, + IOTDB_SUBSCRIPTION_SOURCE_ACK_STRATEGY_DEFAULT_VALUE)); + + autoPollIntervalMs = + pipeParameters.getLongOrDefault( + IOTDB_SUBSCRIPTION_SOURCE_AUTO_POLL_INTERVAL_MS_KEY, + IOTDB_SUBSCRIPTION_SOURCE_AUTO_POLL_INTERVAL_MS_DEFAULT_VALUE); + autoPollTimeoutMs = + pipeParameters.getLongOrDefault( + IOTDB_SUBSCRIPTION_SOURCE_AUTO_POLL_TIMEOUT_MS_KEY, + IOTDB_SUBSCRIPTION_SOURCE_AUTO_POLL_TIMEOUT_MS_DEFAULT_VALUE); + } + + protected AbstractSubscriptionPushConsumerBuilder getSubscriptionPushConsumerBuilder() { + return ((AbstractSubscriptionPushConsumerBuilder) subscription.getSubscriptionConsumerBuilder()) + .ackStrategy(ackStrategy) + .autoPollIntervalMs(autoPollIntervalMs) + .autoPollTimeoutMs(autoPollTimeoutMs); + } +} diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionSourceConstant.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionSourceConstant.java new file mode 100644 index 00000000..85034f30 --- /dev/null +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionSourceConstant.java @@ -0,0 +1,93 @@ +/* + * 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.collector.plugin.builtin.source.iotdb; + +import org.apache.iotdb.session.subscription.consumer.AckStrategy; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +public class IoTDBSubscriptionSourceConstant { + + public static final String IOTDB_SUBSCRIPTION_SOURCE_TOPIC_KEY = "topic"; + public static final String IOTDB_SUBSCRIPTION_SOURCE_TOPIC_DEFAULT_VALUE = "root_all"; + + public static final String IOTDB_SUBSCRIPTION_SOURCE_HOST_KEY = "host"; + public static final String IOTDB_SUBSCRIPTION_SOURCE_HOST_DEFAULT_VALUE = "localhost"; + + public static final String IOTDB_SUBSCRIPTION_SOURCE_PORT_KEY = "port"; + public static final int IOTDB_SUBSCRIPTION_SOURCE_PORT_DEFAULT_VALUE = 6667; + + public static final String IOTDB_SUBSCRIPTION_SOURCE_CONSUMER_ID_KEY = "consumer-id"; + + public static final String IOTDB_SUBSCRIPTION_SOURCE_GROUP_ID_KEY = "group-id"; + + public static final String IOTDB_SUBSCRIPTION_SOURCE_HEARTBEAT_INTERVAL_MS_KEY = + "heartbeat-interval-ms"; + public static final long IOTDB_SUBSCRIPTION_SOURCE_HEARTBEAT_INTERVAL_MS_DEFAULT_VALUE = 30_000L; + public static final long IOTDB_SUBSCRIPTION_SOURCE_HEARTBEAT_INTERVAL_MS_MIN_VALUE = 1_000L; + + public static final String IOTDB_SUBSCRIPTION_SOURCE_ENDPOINT_SYNC_INTERVAL_MS_KEY = + "endpoint-sync-interval-ms"; + public static final long IOTDB_SUBSCRIPTION_SOURCE_ENDPOINT_SYNC_INTERVAL_MS_DEFAULT_VALUE = + 120_000L; + public static final long IOTDB_SUBSCRIPTION_SOURCE_ENDPOINT_SYNC_INTERVAL_MS_MIN_VALUE = 5_000L; + + public static final String IOTDB_SUBSCRIPTION_SOURCE_THRIFT_MAX_FRAME_SIZE_KEY = + "thrift-max-frame-size"; + public static final int IOTDB_SUBSCRIPTION_SOURCE_THRIFT_MAX_FRAME_SIZE_DEFAULT_VALUE = 67108864; + + public static final String IOTDB_SUBSCRIPTION_SOURCE_MAX_POLL_PARALLELISM_KEY = + "max-poll-parallelism"; + public static final int IOTDB_SUBSCRIPTION_SOURCE_MAX_POLL_PARALLELISM_DEFAULT_VALUE = 1; + + public static final String IOTDB_SUBSCRIPTION_SOURCE_AUTO_COMMIT_KEY = "auto-commit"; + public static final boolean IOTDB_SUBSCRIPTION_SOURCE_AUTO_COMMIT_DEFAULT_VALUE = true; + + public static final String IOTDB_SUBSCRIPTION_SOURCE_AUTO_COMMIT_INTERVAL_MS_KEY = + "auto-commit-interval-ms"; + public static final long IOTDB_SUBSCRIPTION_SOURCE_AUTO_COMMIT_INTERVAL_MS_DEFAULT_VALUE = 5_000L; + public static final long IOTDB_SUBSCRIPTION_SOURCE_AUTO_COMMIT_INTERVAL_MS_MIN_VALUE = 500L; + + public static final String IOTDB_SUBSCRIPTION_SOURCE_ACK_STRATEGY_KEY = "ack-strategy"; + public static final String IOTDB_SUBSCRIPTION_SOURCE_ACK_STRATEGY_DEFAULT_VALUE = "after"; + public static final Map IOTDB_SUBSCRIPTION_SOURCE_ACK_STRATEGY_VALUE_MAP = + Collections.unmodifiableMap( + new HashMap() { + { + put("before", AckStrategy.BEFORE_CONSUME); + put("after", AckStrategy.AFTER_CONSUME); + } + }); + + public static final String IOTDB_SUBSCRIPTION_SOURCE_AUTO_POLL_INTERVAL_MS_KEY = + "auto-poll-interval-ms"; + public static final long IOTDB_SUBSCRIPTION_SOURCE_AUTO_POLL_INTERVAL_MS_DEFAULT_VALUE = 100L; + + public static final String IOTDB_SUBSCRIPTION_SOURCE_AUTO_POLL_TIMEOUT_MS_KEY = + "auto-poll-timeout-ms"; + public static final long IOTDB_SUBSCRIPTION_SOURCE_AUTO_POLL_TIMEOUT_MS_DEFAULT_VALUE = 10_000L; + public static final long IOTDB_SUBSCRIPTION_SOURCE_AUTO_POLL_TIMEOUT_MS_MIN_VALUE = 1_000L; + + private IoTDBSubscriptionSourceConstant() { + throw new IllegalStateException("Utility class"); + } +} diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTablePullSource.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTablePullSource.java new file mode 100644 index 00000000..45fcc321 --- /dev/null +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTablePullSource.java @@ -0,0 +1,103 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.collector.plugin.builtin.source.iotdb; + +import org.apache.iotdb.collector.plugin.builtin.sink.event.PipeRawTabletInsertionEvent; +import org.apache.iotdb.collector.runtime.progress.ProgressIndex; +import org.apache.iotdb.pipe.api.event.Event; +import org.apache.iotdb.session.subscription.consumer.ISubscriptionTablePullConsumer; +import org.apache.iotdb.session.subscription.payload.SubscriptionMessage; +import org.apache.iotdb.session.subscription.payload.SubscriptionSessionDataSet; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.List; +import java.util.Optional; + +public class IoTDBSubscriptionTablePullSource extends IoTDBSubscriptionPullSource { + + private static final Logger LOGGER = + LoggerFactory.getLogger(IoTDBSubscriptionTablePullSource.class); + + private ISubscriptionTablePullConsumer consumer; + private static final Long POLL_TIMEOUT_MS = 10_000L; + + private volatile boolean isStarted; + private Thread workerThread; + + @Override + public void start() throws Exception { + consumer = getSubscriptionPullConsumerBuilder().buildTablePullConsumer(); + consumer.open(); + consumer.subscribe(subscription.getTopic()); + + if (workerThread == null) { + isStarted = true; + + workerThread = new Thread(this::doWork); + workerThread.setName("iotdb-subscription-tree-pull-source"); + workerThread.start(); + } + } + + private void doWork() { + while (isStarted && !Thread.currentThread().isInterrupted()) { + final List messages = consumer.poll(POLL_TIMEOUT_MS); + + for (final SubscriptionMessage message : messages) { + for (final SubscriptionSessionDataSet dataSet : message.getSessionDataSetsHandler()) { + subscription.checkIfNeedPause(); + + try { + subscription.put(new PipeRawTabletInsertionEvent(dataSet.getTablet(), isAligned)); + } catch (final InterruptedException e) { + LOGGER.warn("iotdb subscription tree model pull consumer thread interrupted", e); + Thread.currentThread().interrupt(); + } + } + } + } + } + + @Override + public Event supply() throws InterruptedException { + return subscription.take(); + } + + @Override + public void close() throws Exception { + isStarted = false; + if (workerThread != null) { + workerThread.interrupt(); + try { + workerThread.join(1000); + } catch (final InterruptedException e) { + Thread.currentThread().interrupt(); + } + workerThread = null; + } + } + + @Override + public Optional report() { + return Optional.empty(); + } +} diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTablePushSource.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTablePushSource.java new file mode 100644 index 00000000..65ff644c --- /dev/null +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTablePushSource.java @@ -0,0 +1,96 @@ +/* + * 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.collector.plugin.builtin.source.iotdb; + +import org.apache.iotdb.collector.plugin.builtin.sink.event.PipeRawTabletInsertionEvent; +import org.apache.iotdb.collector.runtime.progress.ProgressIndex; +import org.apache.iotdb.session.subscription.consumer.ConsumeResult; +import org.apache.iotdb.session.subscription.consumer.ISubscriptionTablePushConsumer; +import org.apache.iotdb.session.subscription.payload.SubscriptionSessionDataSet; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Optional; + +public class IoTDBSubscriptionTablePushSource extends IoTDBSubscriptionPushSource { + + private static final Logger LOGGER = + LoggerFactory.getLogger(IoTDBSubscriptionTablePushSource.class); + + @Override + public void start() throws Exception { + if (workerThread == null || !workerThread.isAlive()) { + isStarted = true; + + workerThread = new Thread(this::doWork); + workerThread.setName("iotdb-subscription-tree-push-worker"); + workerThread.start(); + } + } + + private void doWork() { + try (final ISubscriptionTablePushConsumer consumer = + getSubscriptionPushConsumerBuilder() + .consumeListener( + message -> { + for (final SubscriptionSessionDataSet dataSet : + message.getSessionDataSetsHandler()) { + try { + subscription.put( + new PipeRawTabletInsertionEvent(dataSet.getTablet(), isStarted)); + } catch (final InterruptedException e) { + LOGGER.warn( + "iotdb subscription tree model push consumer thread interrupted", e); + Thread.currentThread().interrupt(); + + return ConsumeResult.FAILURE; + } + } + + return ConsumeResult.SUCCESS; + }) + .buildTablePushConsumer()) { + consumer.open(); + consumer.subscribe(subscription.getTopic()); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + @Override + public void close() throws Exception { + isStarted = false; + if (workerThread != null) { + workerThread.interrupt(); + try { + workerThread.join(1000); + } catch (final InterruptedException e) { + Thread.currentThread().interrupt(); + } + workerThread = null; + } + } + + @Override + public Optional report() { + return Optional.empty(); + } +} diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTreePullSource.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTreePullSource.java new file mode 100644 index 00000000..f6619c23 --- /dev/null +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTreePullSource.java @@ -0,0 +1,103 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.collector.plugin.builtin.source.iotdb; + +import org.apache.iotdb.collector.plugin.builtin.sink.event.PipeRawTabletInsertionEvent; +import org.apache.iotdb.collector.runtime.progress.ProgressIndex; +import org.apache.iotdb.pipe.api.event.Event; +import org.apache.iotdb.session.subscription.consumer.ISubscriptionTreePullConsumer; +import org.apache.iotdb.session.subscription.payload.SubscriptionMessage; +import org.apache.iotdb.session.subscription.payload.SubscriptionSessionDataSet; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.List; +import java.util.Optional; + +public class IoTDBSubscriptionTreePullSource extends IoTDBSubscriptionPullSource { + + private static final Logger LOGGER = + LoggerFactory.getLogger(IoTDBSubscriptionTreePullSource.class); + + private ISubscriptionTreePullConsumer consumer; + private static final Long POLL_TIMEOUT_MS = 10_000L; + + private volatile boolean isStarted; + private Thread workerThread; + + @Override + public void start() throws Exception { + consumer = getSubscriptionPullConsumerBuilder().buildPullConsumer(); + consumer.open(); + consumer.subscribe(subscription.getTopic()); + + if (workerThread == null) { + isStarted = true; + + workerThread = new Thread(this::doWork); + workerThread.setName("iotdb-subscription-tree-pull-source"); + workerThread.start(); + } + } + + private void doWork() { + while (isStarted && !Thread.currentThread().isInterrupted()) { + final List messages = consumer.poll(POLL_TIMEOUT_MS); + + for (final SubscriptionMessage message : messages) { + for (final SubscriptionSessionDataSet dataSet : message.getSessionDataSetsHandler()) { + subscription.checkIfNeedPause(); + + try { + subscription.put(new PipeRawTabletInsertionEvent(dataSet.getTablet(), isAligned)); + } catch (final InterruptedException e) { + LOGGER.warn("iotdb subscription tree model pull consumer thread interrupted", e); + Thread.currentThread().interrupt(); + } + } + } + } + } + + @Override + public Event supply() throws InterruptedException { + return subscription.take(); + } + + @Override + public void close() throws Exception { + isStarted = false; + if (workerThread != null) { + workerThread.interrupt(); + try { + workerThread.join(1000); + } catch (final InterruptedException e) { + Thread.currentThread().interrupt(); + } + workerThread = null; + } + } + + @Override + public Optional report() { + return Optional.empty(); + } +} diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTreePushSource.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTreePushSource.java new file mode 100644 index 00000000..85c29497 --- /dev/null +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTreePushSource.java @@ -0,0 +1,96 @@ +/* + * 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.collector.plugin.builtin.source.iotdb; + +import org.apache.iotdb.collector.plugin.builtin.sink.event.PipeRawTabletInsertionEvent; +import org.apache.iotdb.collector.runtime.progress.ProgressIndex; +import org.apache.iotdb.session.subscription.consumer.ConsumeResult; +import org.apache.iotdb.session.subscription.consumer.ISubscriptionTreePushConsumer; +import org.apache.iotdb.session.subscription.payload.SubscriptionSessionDataSet; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Optional; + +public class IoTDBSubscriptionTreePushSource extends IoTDBSubscriptionPushSource { + + private static final Logger LOGGER = + LoggerFactory.getLogger(IoTDBSubscriptionTreePushSource.class); + + @Override + public void start() throws Exception { + if (workerThread == null || !workerThread.isAlive()) { + isStarted = true; + + workerThread = new Thread(this::doWork); + workerThread.setName("iotdb-subscription-tree-push-worker"); + workerThread.start(); + } + } + + private void doWork() { + try (final ISubscriptionTreePushConsumer consumer = + getSubscriptionPushConsumerBuilder() + .consumeListener( + message -> { + for (final SubscriptionSessionDataSet dataSet : + message.getSessionDataSetsHandler()) { + try { + subscription.put( + new PipeRawTabletInsertionEvent(dataSet.getTablet(), isStarted)); + } catch (final InterruptedException e) { + LOGGER.warn( + "iotdb subscription tree model push consumer thread interrupted", e); + Thread.currentThread().interrupt(); + + return ConsumeResult.FAILURE; + } + } + + return ConsumeResult.SUCCESS; + }) + .buildPushConsumer()) { + consumer.open(); + consumer.subscribe(subscription.getTopic()); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + @Override + public void close() throws Exception { + isStarted = false; + if (workerThread != null) { + workerThread.interrupt(); + try { + workerThread.join(1000); + } catch (final InterruptedException e) { + Thread.currentThread().interrupt(); + } + workerThread = null; + } + } + + @Override + public Optional report() { + return Optional.empty(); + } +} diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/SubDemoEvent.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/SubDemoEvent.java deleted file mode 100644 index a0f3749c..00000000 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/SubDemoEvent.java +++ /dev/null @@ -1,31 +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.collector.plugin.builtin.source.iotdb; - -import org.apache.iotdb.collector.plugin.builtin.sink.event.PipeRawTabletInsertionEvent; - -import org.apache.tsfile.write.record.Tablet; - -public class SubDemoEvent extends PipeRawTabletInsertionEvent { - - public SubDemoEvent(final Tablet tablet, final String deviceId) { - super(tablet, deviceId); - } -} diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/kafka/KafkaEvent.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/kafka/KafkaEvent.java deleted file mode 100644 index f978a6c4..00000000 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/kafka/KafkaEvent.java +++ /dev/null @@ -1,31 +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.collector.plugin.builtin.source.kafka; - -import org.apache.iotdb.collector.plugin.builtin.sink.event.PipeRawTabletInsertionEvent; - -import org.apache.tsfile.write.record.Tablet; - -public class KafkaEvent extends PipeRawTabletInsertionEvent { - - public KafkaEvent(final Tablet tablet, final String deviceId) { - super(tablet, deviceId); - } -} diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/kafka/KafkaSource.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/kafka/KafkaSource.java index 151bac55..2cf80bd8 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/kafka/KafkaSource.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/kafka/KafkaSource.java @@ -20,7 +20,9 @@ package org.apache.iotdb.collector.plugin.builtin.source.kafka; import org.apache.iotdb.collector.plugin.api.PushSource; +import org.apache.iotdb.collector.plugin.api.customizer.CollectorParameters; import org.apache.iotdb.collector.plugin.api.customizer.CollectorRuntimeEnvironment; +import org.apache.iotdb.collector.plugin.builtin.sink.event.PipeRawTabletInsertionEvent; import org.apache.iotdb.collector.runtime.progress.ProgressIndex; import org.apache.iotdb.collector.service.RuntimeService; import org.apache.iotdb.pipe.api.customizer.configuration.PipeSourceRuntimeConfiguration; @@ -46,17 +48,13 @@ import java.util.Collections; import java.util.HashMap; import java.util.List; -import java.util.Objects; import java.util.Optional; import java.util.Properties; -import java.util.function.Predicate; -import static org.apache.iotdb.collector.plugin.builtin.source.constant.SourceConstant.REPORT_TIME_INTERVAL_DEFAULT_VALUE; -import static org.apache.iotdb.collector.plugin.builtin.source.constant.SourceConstant.REPORT_TIME_INTERVAL_KEY; -import static org.apache.iotdb.collector.plugin.builtin.source.kafka.KafkaSourceConstant.AUTO_OFFSET_RESET_SET; -import static org.apache.iotdb.collector.plugin.builtin.source.kafka.KafkaSourceConstant.BOOLEAN_SET; +import static org.apache.iotdb.collector.plugin.builtin.source.constant.SourceConstant.SOURCE_REPORT_TIME_INTERVAL_KEY; import static org.apache.iotdb.collector.plugin.builtin.source.kafka.KafkaSourceConstant.KAFKA_SOURCE_AUTO_OFFSET_RESET_DEFAULT_VALUE; import static org.apache.iotdb.collector.plugin.builtin.source.kafka.KafkaSourceConstant.KAFKA_SOURCE_AUTO_OFFSET_RESET_KEY; +import static org.apache.iotdb.collector.plugin.builtin.source.kafka.KafkaSourceConstant.KAFKA_SOURCE_AUTO_OFFSET_RESET_VALUE_SET; import static org.apache.iotdb.collector.plugin.builtin.source.kafka.KafkaSourceConstant.KAFKA_SOURCE_BOOTSTRAP_SERVERS_DEFAULT_VALUE; import static org.apache.iotdb.collector.plugin.builtin.source.kafka.KafkaSourceConstant.KAFKA_SOURCE_BOOTSTRAP_SERVERS_KEY; import static org.apache.iotdb.collector.plugin.builtin.source.kafka.KafkaSourceConstant.KAFKA_SOURCE_ENABLE_AUTO_COMMIT_DEFAULT_VALUE; @@ -82,9 +80,6 @@ public class KafkaSource extends PushSource { private static final Logger LOGGER = LoggerFactory.getLogger(KafkaSource.class); - private ProgressIndex startIndex; - private int instanceIndex; - private Thread workerThread; private volatile boolean isStarted = false; private volatile KafkaConsumer consumer; @@ -104,78 +99,37 @@ public class KafkaSource extends PushSource { private long offset; - private int reportTimeInterval; - @Override public void validate(PipeParameterValidator validator) throws Exception { - validateRequiredParam(validator, validator.getParameters().getString(KAFKA_SOURCE_TOPIC_KEY)); - validateRequiredParam( - validator, validator.getParameters().getString(KAFKA_SOURCE_GROUP_ID_KEY)); + CollectorParameters.validateStringRequiredParam(validator, KAFKA_SOURCE_TOPIC_KEY); + CollectorParameters.validateStringRequiredParam(validator, KAFKA_SOURCE_GROUP_ID_KEY); - validateParam( + CollectorParameters.validateSetParam( validator, KAFKA_SOURCE_AUTO_OFFSET_RESET_KEY, - autoOffsetReset -> AUTO_OFFSET_RESET_SET.contains(String.valueOf(autoOffsetReset)), + KAFKA_SOURCE_AUTO_OFFSET_RESET_VALUE_SET, KAFKA_SOURCE_AUTO_OFFSET_RESET_DEFAULT_VALUE); - validateParam( + CollectorParameters.validateBooleanParam( validator, KAFKA_SOURCE_ENABLE_AUTO_COMMIT_KEY, - enableAutoCommit -> BOOLEAN_SET.contains(String.valueOf(enableAutoCommit)), KAFKA_SOURCE_ENABLE_AUTO_COMMIT_DEFAULT_VALUE); - validateIntegerParam( + CollectorParameters.validateIntegerParam( validator, KAFKA_SOURCE_SESSION_TIMEOUT_MS_KEY, KAFKA_SOURCE_SESSION_TIMEOUT_MS_DEFAULT_VALUE, value -> value > 0); - validateIntegerParam( + CollectorParameters.validateIntegerParam( validator, KAFKA_SOURCE_MAX_POLL_INTERVAL_MS_KEY, KAFKA_SOURCE_MAX_POLL_INTERVAL_MS_DEFAULT_VALUE, value -> value > 0); - validateIntegerParam( + CollectorParameters.validateIntegerParam( validator, KAFKA_SOURCE_MAX_POLL_RECORDS_KEY, KAFKA_SOURCE_MAX_POLL_RECORDS_DEFAULT_VALUE, value -> value > 0); - validateIntegerParam( - validator, - REPORT_TIME_INTERVAL_KEY, - REPORT_TIME_INTERVAL_DEFAULT_VALUE, - value -> value > 0); - } - - private void validateRequiredParam( - final PipeParameterValidator validator, final String paramKey) { - validator.validate(Objects::nonNull, String.format("%s is required", paramKey), paramKey); - } - - private void validateParam( - final PipeParameterValidator validator, - final String paramKey, - final Predicate validationCondition, - final String defaultValue) { - final String paramValue = validator.getParameters().getStringOrDefault(paramKey, defaultValue); - - validator.validate( - validationCondition::test, - String.format("%s must be one of %s, but got %s", paramKey, BOOLEAN_SET, paramValue), - paramValue); - } - - private void validateIntegerParam( - final PipeParameterValidator validator, - final String paramKey, - final String paramDefaultValue, - final Predicate validationCondition) { - final int paramValue = - validator.getParameters().getIntOrDefault(paramKey, Integer.parseInt(paramDefaultValue)); - - validator.validate( - value -> validationCondition.test((Integer) value), - String.format("%s must be > 0, but got %d", paramKey, paramValue), - paramValue); } @Override @@ -216,23 +170,17 @@ public void customize( pipeParameters.getBooleanOrDefault(KAFKA_SOURCE_ENABLE_AUTO_COMMIT_KEY, false); sessionTimeoutMs = pipeParameters.getIntOrDefault( - KAFKA_SOURCE_SESSION_TIMEOUT_MS_KEY, - Integer.parseInt(KAFKA_SOURCE_SESSION_TIMEOUT_MS_DEFAULT_VALUE)); + KAFKA_SOURCE_SESSION_TIMEOUT_MS_KEY, KAFKA_SOURCE_SESSION_TIMEOUT_MS_DEFAULT_VALUE); maxPollRecords = pipeParameters.getIntOrDefault( - KAFKA_SOURCE_MAX_POLL_RECORDS_KEY, - Integer.parseInt(KAFKA_SOURCE_MAX_POLL_RECORDS_DEFAULT_VALUE)); + KAFKA_SOURCE_MAX_POLL_RECORDS_KEY, KAFKA_SOURCE_MAX_POLL_RECORDS_DEFAULT_VALUE); maxPollIntervalMs = pipeParameters.getIntOrDefault( - KAFKA_SOURCE_MAX_POLL_INTERVAL_MS_KEY, - Integer.parseInt(KAFKA_SOURCE_MAX_POLL_INTERVAL_MS_DEFAULT_VALUE)); + KAFKA_SOURCE_MAX_POLL_INTERVAL_MS_KEY, KAFKA_SOURCE_MAX_POLL_INTERVAL_MS_DEFAULT_VALUE); partitionAssignmentStrategy = pipeParameters.getStringOrDefault( KAFKA_SOURCE_PARTITION_ASSIGN_STRATEGY_KEY, KAFKA_SOURCE_PARTITION_ASSIGN_STRATEGY_DEFAULT_VALUE); - reportTimeInterval = - pipeParameters.getIntOrDefault( - REPORT_TIME_INTERVAL_KEY, Integer.parseInt(REPORT_TIME_INTERVAL_DEFAULT_VALUE)); } @Override @@ -354,7 +302,7 @@ private void process(final ConsumerRecords records) { tablet.setRowSize(values.length); tablet.setValues(values); - final KafkaEvent event = new KafkaEvent(tablet, deviceId); + final PipeRawTabletInsertionEvent event = new PipeRawTabletInsertionEvent(tablet, false); try { supply(event); } catch (final Exception e) { @@ -386,7 +334,7 @@ public void close() throws Exception { public Optional report() { final HashMap progressInfo = new HashMap<>(); progressInfo.put("offset", String.valueOf(offset)); - progressInfo.put(REPORT_TIME_INTERVAL_KEY, String.valueOf(reportTimeInterval)); + progressInfo.put(SOURCE_REPORT_TIME_INTERVAL_KEY, String.valueOf(reportTimeInterval)); return Optional.of(new ProgressIndex(instanceIndex, progressInfo)); } diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/kafka/KafkaSourceConstant.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/kafka/KafkaSourceConstant.java index 830ee803..0691e17a 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/kafka/KafkaSourceConstant.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/kafka/KafkaSourceConstant.java @@ -30,38 +30,44 @@ public class KafkaSourceConstant { public static final String KAFKA_SOURCE_TOPIC_KEY = "topic"; + public static final String KAFKA_SOURCE_TOPIC_DEFAULT_VALUE = "my_topic"; + public static final String KAFKA_SOURCE_BOOTSTRAP_SERVERS_KEY = "bootstrap.servers"; + public static final String KAFKA_SOURCE_BOOTSTRAP_SERVERS_DEFAULT_VALUE = "localhost:9092"; + public static final String KAFKA_SOURCE_GROUP_ID_KEY = "group.id"; + public static final String KAFKA_SOURCE_GROUP_ID_DEFAULT_VALUE = "multi-thread-group"; + public static final String KAFKA_SOURCE_KEY_DESERIALIZER_KEY = "key.deserializer"; + public static final String KAFKA_SOURCE_KEY_DESERIALIZER_DEFAULT_VALUE = + StringDeserializer.class.getName(); + public static final String KAFKA_SOURCE_VALUE_DESERIALIZER_KEY = "value.deserializer"; + public static final String KAFKA_SOURCE_VALUE_DESERIALIZER_DEFAULT_VALUE = + StringDeserializer.class.getName(); + public static final String KAFKA_SOURCE_AUTO_OFFSET_RESET_KEY = "auto.offset.reset"; + public static final String KAFKA_SOURCE_AUTO_OFFSET_RESET_DEFAULT_VALUE = "none"; + public static final Set KAFKA_SOURCE_AUTO_OFFSET_RESET_VALUE_SET = + Collections.unmodifiableSet(new HashSet<>(Arrays.asList("none", "earliest", "latest"))); + public static final String KAFKA_SOURCE_ENABLE_AUTO_COMMIT_KEY = "enable.auto.commit"; + public static final boolean KAFKA_SOURCE_ENABLE_AUTO_COMMIT_DEFAULT_VALUE = false; + public static final String KAFKA_SOURCE_SESSION_TIMEOUT_MS_KEY = "session.timeout.ms"; + public static final int KAFKA_SOURCE_SESSION_TIMEOUT_MS_DEFAULT_VALUE = 10000; + public static final String KAFKA_SOURCE_MAX_POLL_RECORDS_KEY = "max.poll.records"; + public static final int KAFKA_SOURCE_MAX_POLL_RECORDS_DEFAULT_VALUE = 500; + public static final String KAFKA_SOURCE_MAX_POLL_INTERVAL_MS_KEY = "max.poll.interval.ms"; + public static final int KAFKA_SOURCE_MAX_POLL_INTERVAL_MS_DEFAULT_VALUE = 300000; + public static final String KAFKA_SOURCE_PARTITION_ASSIGN_STRATEGY_KEY = "partition.assign.strategy"; - - public static final String KAFKA_SOURCE_TOPIC_DEFAULT_VALUE = "my_topic"; - public static final String KAFKA_SOURCE_BOOTSTRAP_SERVERS_DEFAULT_VALUE = "localhost:9092"; - public static final String KAFKA_SOURCE_GROUP_ID_DEFAULT_VALUE = "multi-thread-group"; - public static final String KAFKA_SOURCE_KEY_DESERIALIZER_DEFAULT_VALUE = - StringDeserializer.class.getName(); - public static final String KAFKA_SOURCE_VALUE_DESERIALIZER_DEFAULT_VALUE = - StringDeserializer.class.getName(); - public static final String KAFKA_SOURCE_AUTO_OFFSET_RESET_DEFAULT_VALUE = "none"; - public static final String KAFKA_SOURCE_ENABLE_AUTO_COMMIT_DEFAULT_VALUE = "false"; - public static final String KAFKA_SOURCE_SESSION_TIMEOUT_MS_DEFAULT_VALUE = "10000"; - public static final String KAFKA_SOURCE_MAX_POLL_RECORDS_DEFAULT_VALUE = "500"; - public static final String KAFKA_SOURCE_MAX_POLL_INTERVAL_MS_DEFAULT_VALUE = "300000"; public static final String KAFKA_SOURCE_PARTITION_ASSIGN_STRATEGY_DEFAULT_VALUE = RangeAssignor.class.getName(); - public static final Set AUTO_OFFSET_RESET_SET = - Collections.unmodifiableSet(new HashSet<>(Arrays.asList("none", "earliest", "latest"))); - public static final Set BOOLEAN_SET = - Collections.unmodifiableSet(new HashSet<>(Arrays.asList("true", "false"))); - private KafkaSourceConstant() { throw new IllegalStateException("Utility class"); } diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/runtime/plugin/constructor/SourceConstructor.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/runtime/plugin/constructor/SourceConstructor.java index d85a1952..1e24d825 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/runtime/plugin/constructor/SourceConstructor.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/runtime/plugin/constructor/SourceConstructor.java @@ -24,7 +24,10 @@ import org.apache.iotdb.collector.plugin.builtin.BuiltinPlugin; import org.apache.iotdb.collector.plugin.builtin.source.HttpPullSource; import org.apache.iotdb.collector.plugin.builtin.source.HttpPushSource; -import org.apache.iotdb.collector.plugin.builtin.source.iotdb.IoTDBPushSource; +import org.apache.iotdb.collector.plugin.builtin.source.iotdb.IoTDBSubscriptionTablePullSource; +import org.apache.iotdb.collector.plugin.builtin.source.iotdb.IoTDBSubscriptionTablePushSource; +import org.apache.iotdb.collector.plugin.builtin.source.iotdb.IoTDBSubscriptionTreePullSource; +import org.apache.iotdb.collector.plugin.builtin.source.iotdb.IoTDBSubscriptionTreePushSource; import org.apache.iotdb.collector.plugin.builtin.source.kafka.KafkaSource; import org.apache.iotdb.collector.runtime.plugin.meta.PluginMetaKeeper; import org.apache.iotdb.pipe.api.PipeSource; @@ -40,7 +43,18 @@ public SourceConstructor(PluginMetaKeeper pluginMetaKeeper) { protected void initConstructors() { pluginConstructors.put(BuiltinPlugin.HTTP_PULL_SOURCE.getPluginName(), HttpPullSource::new); pluginConstructors.put(BuiltinPlugin.HTTP_PUSH_SOURCE.getPluginName(), HttpPushSource::new); - pluginConstructors.put(BuiltinPlugin.SUBSCRIPTION_SOURCE.getPluginName(), IoTDBPushSource::new); + pluginConstructors.put( + BuiltinPlugin.IOTDB_SUBSCRIPTION_TREE_PUSH_SOURCE.getPluginName(), + IoTDBSubscriptionTreePushSource::new); + pluginConstructors.put( + BuiltinPlugin.IOTDB_SUBSCRIPTION_TREE_PULL_SOURCE.getPluginName(), + IoTDBSubscriptionTreePullSource::new); + pluginConstructors.put( + BuiltinPlugin.IOTDB_SUBSCRIPTION_TABLE_PULL_SOURCE.getClassName(), + IoTDBSubscriptionTablePushSource::new); + pluginConstructors.put( + BuiltinPlugin.IOTDB_SUBSCRIPTION_TABLE_PUSH_SOURCE.getPluginName(), + IoTDBSubscriptionTablePullSource::new); pluginConstructors.put(BuiltinPlugin.KAFKA_SOURCE.getPluginName(), KafkaSource::new); } diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/runtime/task/source/pull/PullSourceTask.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/runtime/task/source/pull/PullSourceTask.java index 5cf7468e..769ba32c 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/runtime/task/source/pull/PullSourceTask.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/runtime/task/source/pull/PullSourceTask.java @@ -44,7 +44,7 @@ import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; -import static org.apache.iotdb.collector.plugin.builtin.source.constant.SourceConstant.REPORT_TIME_INTERVAL_KEY; +import static org.apache.iotdb.collector.plugin.builtin.source.constant.SourceConstant.SOURCE_REPORT_TIME_INTERVAL_KEY; public class PullSourceTask extends SourceTask { @@ -152,7 +152,7 @@ public void createInternal() throws Exception { .get() .getProgressInfo() .getOrDefault( - REPORT_TIME_INTERVAL_KEY, + SOURCE_REPORT_TIME_INTERVAL_KEY, String.valueOf( TaskRuntimeOptions.TASK_PROGRESS_REPORT_INTERVAL.value()))) : TaskRuntimeOptions.TASK_PROGRESS_REPORT_INTERVAL.value()); From 68b272f1f1955a31f6587729115dcddc813f462b Mon Sep 17 00:00:00 2001 From: ysc <2725843507@qq.com> Date: Wed, 21 May 2025 15:33:26 +0800 Subject: [PATCH 03/11] update processor --- .../plugin/builtin/BuiltinPlugin.java | 6 +- ...fkaProcessor.java => CommonProcessor.java} | 11 ++-- .../processor/SubscriptionProcessor.java | 56 ------------------- .../constructor/ProcessorConstructor.java | 7 +-- 4 files changed, 10 insertions(+), 70 deletions(-) rename iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/processor/{KafkaProcessor.java => CommonProcessor.java} (82%) delete mode 100644 iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/processor/SubscriptionProcessor.java diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/BuiltinPlugin.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/BuiltinPlugin.java index f2dccccd..c759216c 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/BuiltinPlugin.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/BuiltinPlugin.java @@ -19,9 +19,8 @@ package org.apache.iotdb.collector.plugin.builtin; +import org.apache.iotdb.collector.plugin.builtin.processor.CommonProcessor; import org.apache.iotdb.collector.plugin.builtin.processor.DoNothingProcessor; -import org.apache.iotdb.collector.plugin.builtin.processor.KafkaProcessor; -import org.apache.iotdb.collector.plugin.builtin.processor.SubscriptionProcessor; import org.apache.iotdb.collector.plugin.builtin.sink.DemoSink; import org.apache.iotdb.collector.plugin.builtin.sink.protocol.IoTDBDataRegionSyncConnector; import org.apache.iotdb.collector.plugin.builtin.source.HttpPullSource; @@ -51,8 +50,7 @@ public enum BuiltinPlugin { // Processors DO_NOTHING_PROCESSOR("do-nothing-processor", DoNothingProcessor.class), - SUBSCRIPTION_PROCESSOR("subscription-processor", SubscriptionProcessor.class), - KAFKA_PROCESSOR("kafka-processor", KafkaProcessor.class), + COMMON_PROCESSOR("common-processor", CommonProcessor.class), // Sinks IOTDB_DEMO_SINK("iotdb-demo-sink", DemoSink.class), diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/processor/KafkaProcessor.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/processor/CommonProcessor.java similarity index 82% rename from iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/processor/KafkaProcessor.java rename to iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/processor/CommonProcessor.java index 46ba3389..8e127483 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/processor/KafkaProcessor.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/processor/CommonProcessor.java @@ -28,24 +28,25 @@ import org.apache.iotdb.pipe.api.event.Event; import org.apache.iotdb.pipe.api.event.dml.insertion.TabletInsertionEvent; -public class KafkaProcessor implements PipeProcessor { +public class CommonProcessor implements PipeProcessor { @Override public void validate(PipeParameterValidator pipeParameterValidator) throws Exception {} @Override public void customize( - PipeParameters pipeParameters, - PipeProcessorRuntimeConfiguration pipeProcessorRuntimeConfiguration) + final PipeParameters pipeParameters, + final PipeProcessorRuntimeConfiguration pipeProcessorRuntimeConfiguration) throws Exception {} @Override - public void process(TabletInsertionEvent tabletInsertionEvent, EventCollector eventCollector) + public void process( + final TabletInsertionEvent tabletInsertionEvent, final EventCollector eventCollector) throws Exception { eventCollector.collect(tabletInsertionEvent); } @Override - public void process(Event event, EventCollector eventCollector) throws Exception { + public void process(final Event event, final EventCollector eventCollector) throws Exception { if (event instanceof PipeRawTabletInsertionEvent) { process((PipeRawTabletInsertionEvent) event, eventCollector); } diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/processor/SubscriptionProcessor.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/processor/SubscriptionProcessor.java deleted file mode 100644 index d2a425dc..00000000 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/processor/SubscriptionProcessor.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.collector.plugin.builtin.processor; - -import org.apache.iotdb.collector.plugin.builtin.sink.event.PipeRawTabletInsertionEvent; -import org.apache.iotdb.pipe.api.PipeProcessor; -import org.apache.iotdb.pipe.api.collector.EventCollector; -import org.apache.iotdb.pipe.api.customizer.configuration.PipeProcessorRuntimeConfiguration; -import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameterValidator; -import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameters; -import org.apache.iotdb.pipe.api.event.Event; -import org.apache.iotdb.pipe.api.event.dml.insertion.TabletInsertionEvent; - -public class SubscriptionProcessor implements PipeProcessor { - @Override - public void validate(PipeParameterValidator pipeParameterValidator) throws Exception {} - - @Override - public void customize( - PipeParameters pipeParameters, - PipeProcessorRuntimeConfiguration pipeProcessorRuntimeConfiguration) - throws Exception {} - - @Override - public void process(TabletInsertionEvent tabletInsertionEvent, EventCollector eventCollector) - throws Exception { - eventCollector.collect(tabletInsertionEvent); - } - - @Override - public void process(Event event, EventCollector eventCollector) throws Exception { - if (event instanceof PipeRawTabletInsertionEvent) { - process((PipeRawTabletInsertionEvent) event, eventCollector); - } - } - - @Override - public void close() throws Exception {} -} diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/runtime/plugin/constructor/ProcessorConstructor.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/runtime/plugin/constructor/ProcessorConstructor.java index 9d3dbabf..853ff7b6 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/runtime/plugin/constructor/ProcessorConstructor.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/runtime/plugin/constructor/ProcessorConstructor.java @@ -20,9 +20,8 @@ package org.apache.iotdb.collector.runtime.plugin.constructor; import org.apache.iotdb.collector.plugin.builtin.BuiltinPlugin; +import org.apache.iotdb.collector.plugin.builtin.processor.CommonProcessor; import org.apache.iotdb.collector.plugin.builtin.processor.DoNothingProcessor; -import org.apache.iotdb.collector.plugin.builtin.processor.KafkaProcessor; -import org.apache.iotdb.collector.plugin.builtin.processor.SubscriptionProcessor; import org.apache.iotdb.collector.runtime.plugin.meta.PluginMetaKeeper; import org.apache.iotdb.pipe.api.PipeProcessor; import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameters; @@ -37,9 +36,7 @@ public ProcessorConstructor(PluginMetaKeeper pluginMetaKeeper) { protected void initConstructors() { pluginConstructors.put( BuiltinPlugin.DO_NOTHING_PROCESSOR.getPluginName(), DoNothingProcessor::new); - pluginConstructors.put( - BuiltinPlugin.SUBSCRIPTION_PROCESSOR.getPluginName(), SubscriptionProcessor::new); - pluginConstructors.put(BuiltinPlugin.KAFKA_PROCESSOR.getPluginName(), KafkaProcessor::new); + pluginConstructors.put(BuiltinPlugin.COMMON_PROCESSOR.getPluginName(), CommonProcessor::new); } @Override From 2ee4e9ff874e6201a6b11921994369e57085faf7 Mon Sep 17 00:00:00 2001 From: ysc <2725843507@qq.com> Date: Thu, 22 May 2025 15:41:47 +0800 Subject: [PATCH 04/11] refactor --- .../source/constant/SourceConstant.java | 13 +++- ...tion.java => IoTDBSubscriptionCommon.java} | 15 ++-- .../iotdb/IoTDBSubscriptionPullSource.java | 75 ++++++++++++++++++- .../iotdb/IoTDBSubscriptionPushSource.java | 57 +++++++++++++- .../IoTDBSubscriptionSourceConstant.java | 6 +- .../IoTDBSubscriptionTablePullSource.java | 59 ++------------- .../IoTDBSubscriptionTablePushSource.java | 54 ++----------- .../IoTDBSubscriptionTreePullSource.java | 59 ++------------- .../IoTDBSubscriptionTreePushSource.java | 54 ++----------- .../source/kafka/KafkaSourceConstant.java | 10 ++- 10 files changed, 183 insertions(+), 219 deletions(-) rename iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/{IoTDBSubscription.java => IoTDBSubscriptionCommon.java} (95%) diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/constant/SourceConstant.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/constant/SourceConstant.java index 23b644fe..62423c66 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/constant/SourceConstant.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/constant/SourceConstant.java @@ -27,17 +27,22 @@ public class SourceConstant { public static final String SOURCE_REPORT_TIME_INTERVAL_KEY = "report-time-interval"; - public static final String SOURCE_IS_ALIGNED_KEY = "is-aligned"; - public static final String SOURCE_DEVICE_ID_KEY = "device-id"; - public static final int SOURCE_REPORT_TIME_INTERVAL_DEFAULT_VALUE = 60; + + public static final String SOURCE_IS_ALIGNED_KEY = "is-aligned"; public static final boolean SOURCE_IS_ALIGNED_DEFAULT_VALUE = false; + + public static final String SOURCE_DEVICE_ID_KEY = "device-id"; public static final String SOURCE_DEVICE_ID_DEFAULT_VALUE = "root.test"; public static final String SOURCE_SQL_DIALECT_KEY = "sql-dialect"; + public static final String SOURCE_SQL_DIALECT_TREE_VALUE = "tree"; + public static final String SOURCE_SQL_DIALECT_TABLE_VALUE = "table"; public static final String SOURCE_SQL_DIALECT_DEFAULT_VALUE = "tree"; public static final Set SOURCE_SQL_DIALECT_VALUE_SET = - Collections.unmodifiableSet(new HashSet<>(Arrays.asList("tree", "table"))); + Collections.unmodifiableSet( + new HashSet<>( + Arrays.asList(SOURCE_SQL_DIALECT_TREE_VALUE, SOURCE_SQL_DIALECT_TABLE_VALUE))); public static final Set BOOLEAN_SET = Collections.unmodifiableSet(new HashSet<>(Arrays.asList("true", "false"))); diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscription.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionCommon.java similarity index 95% rename from iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscription.java rename to iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionCommon.java index 253758b6..5534c61b 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscription.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionCommon.java @@ -57,7 +57,7 @@ import static org.apache.iotdb.collector.plugin.builtin.source.iotdb.IoTDBSubscriptionSourceConstant.IOTDB_SUBSCRIPTION_SOURCE_TOPIC_DEFAULT_VALUE; import static org.apache.iotdb.collector.plugin.builtin.source.iotdb.IoTDBSubscriptionSourceConstant.IOTDB_SUBSCRIPTION_SOURCE_TOPIC_KEY; -public class IoTDBSubscription { +public class IoTDBSubscriptionCommon { private String host; private Integer port; @@ -70,10 +70,11 @@ public class IoTDBSubscription { private String topic; - protected final BlockingQueue eventQueue = new LinkedBlockingQueue<>(); - protected final Integer eventQueueCapacity = 1000; - protected final Long eventQueuePauseIntervalMs = 100_000_000L; + private final BlockingQueue eventQueue = new LinkedBlockingQueue<>(); + private static final Integer EVENT_QUEUE_CAPACITY = 1000; + private static final Long EVENT_QUEUE_PAUSE_INTERVAL_MS = 100_000_000L; + // validate common parameters public void validate(final PipeParameterValidator validator) { CollectorParameters.validateStringRequiredParam(validator, IOTDB_SUBSCRIPTION_SOURCE_TOPIC_KEY); CollectorParameters.validateStringRequiredParam( @@ -123,6 +124,7 @@ public void validate(final PipeParameterValidator validator) { SOURCE_SQL_DIALECT_DEFAULT_VALUE); } + // customize common parameters public void customize( final PipeParameters pipeParameters, final PipeSourceRuntimeConfiguration configuration) { host = @@ -154,6 +156,7 @@ public void customize( IOTDB_SUBSCRIPTION_SOURCE_MAX_POLL_PARALLELISM_DEFAULT_VALUE); } + // common consumer builder public AbstractSubscriptionConsumerBuilder getSubscriptionConsumerBuilder() { return new AbstractSubscriptionConsumerBuilder() .host(host) @@ -175,8 +178,8 @@ public void put(final Event event) throws InterruptedException { } public void checkIfNeedPause() { - while (eventQueue.size() >= eventQueueCapacity) { - LockSupport.parkNanos(eventQueuePauseIntervalMs); + while (eventQueue.size() >= EVENT_QUEUE_CAPACITY) { + LockSupport.parkNanos(EVENT_QUEUE_PAUSE_INTERVAL_MS); } } diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionPullSource.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionPullSource.java index 83d32e0a..4cfa7b22 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionPullSource.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionPullSource.java @@ -21,10 +21,19 @@ import org.apache.iotdb.collector.plugin.api.PullSource; import org.apache.iotdb.collector.plugin.api.customizer.CollectorParameters; +import org.apache.iotdb.collector.plugin.builtin.sink.event.PipeRawTabletInsertionEvent; import org.apache.iotdb.pipe.api.customizer.configuration.PipeSourceRuntimeConfiguration; import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameterValidator; import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameters; +import org.apache.iotdb.pipe.api.event.Event; import org.apache.iotdb.session.subscription.consumer.base.AbstractSubscriptionPullConsumerBuilder; +import org.apache.iotdb.session.subscription.payload.SubscriptionMessage; +import org.apache.iotdb.session.subscription.payload.SubscriptionSessionDataSet; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.List; import static org.apache.iotdb.collector.plugin.builtin.source.iotdb.IoTDBSubscriptionSourceConstant.IOTDB_SUBSCRIPTION_SOURCE_AUTO_COMMIT_DEFAULT_VALUE; import static org.apache.iotdb.collector.plugin.builtin.source.iotdb.IoTDBSubscriptionSourceConstant.IOTDB_SUBSCRIPTION_SOURCE_AUTO_COMMIT_INTERVAL_MS_DEFAULT_VALUE; @@ -34,11 +43,18 @@ public abstract class IoTDBSubscriptionPullSource extends PullSource { - protected final IoTDBSubscription subscription = new IoTDBSubscription(); + private static final Logger LOGGER = LoggerFactory.getLogger(IoTDBSubscriptionPullSource.class); + + protected static final Long POLL_TIMEOUT_MS = 10_000L; + + protected final IoTDBSubscriptionCommon subscription = new IoTDBSubscriptionCommon(); private Boolean autoCommit; private Long autoCommitIntervalMs; + protected volatile boolean isStarted; + protected Thread workerThread; + @Override public void validate(final PipeParameterValidator validator) throws Exception { super.validate(validator); @@ -73,6 +89,63 @@ public void customize( IOTDB_SUBSCRIPTION_SOURCE_AUTO_COMMIT_INTERVAL_MS_DEFAULT_VALUE); } + @Override + public void start() throws Exception { + initPullConsumer(); + + if (workerThread == null) { + isStarted = true; + + workerThread = new Thread(this::doWork); + workerThread.setName(getPullConsumerThreadName()); + workerThread.start(); + } + } + + protected abstract void initPullConsumer(); + + protected abstract String getPullConsumerThreadName(); + + private void doWork() { + while (isStarted && !Thread.currentThread().isInterrupted()) { + final List messages = poll(); + + for (final SubscriptionMessage message : messages) { + for (final SubscriptionSessionDataSet dataSet : message.getSessionDataSetsHandler()) { + subscription.checkIfNeedPause(); + + try { + subscription.put(new PipeRawTabletInsertionEvent(dataSet.getTablet(), isAligned)); + } catch (final InterruptedException e) { + LOGGER.warn("{} thread interrupted", getPullConsumerThreadName(), e); + Thread.currentThread().interrupt(); + } + } + } + } + } + + protected abstract List poll(); + + @Override + public Event supply() throws InterruptedException { + return subscription.take(); + } + + @Override + public void close() throws Exception { + isStarted = false; + if (workerThread != null) { + workerThread.interrupt(); + try { + workerThread.join(1000); + } catch (final InterruptedException e) { + Thread.currentThread().interrupt(); + } + workerThread = null; + } + } + protected AbstractSubscriptionPullConsumerBuilder getSubscriptionPullConsumerBuilder() { return ((AbstractSubscriptionPullConsumerBuilder) subscription.getSubscriptionConsumerBuilder()) .autoCommit(autoCommit) diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionPushSource.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionPushSource.java index cc518eed..c41acaf8 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionPushSource.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionPushSource.java @@ -21,11 +21,17 @@ import org.apache.iotdb.collector.plugin.api.PushSource; import org.apache.iotdb.collector.plugin.api.customizer.CollectorParameters; +import org.apache.iotdb.collector.plugin.builtin.sink.event.PipeRawTabletInsertionEvent; import org.apache.iotdb.pipe.api.customizer.configuration.PipeSourceRuntimeConfiguration; import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameterValidator; import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameters; import org.apache.iotdb.session.subscription.consumer.AckStrategy; +import org.apache.iotdb.session.subscription.consumer.ConsumeResult; import org.apache.iotdb.session.subscription.consumer.base.AbstractSubscriptionPushConsumerBuilder; +import org.apache.iotdb.session.subscription.payload.SubscriptionSessionDataSet; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import static org.apache.iotdb.collector.plugin.builtin.source.iotdb.IoTDBSubscriptionSourceConstant.IOTDB_SUBSCRIPTION_SOURCE_ACK_STRATEGY_DEFAULT_VALUE; import static org.apache.iotdb.collector.plugin.builtin.source.iotdb.IoTDBSubscriptionSourceConstant.IOTDB_SUBSCRIPTION_SOURCE_ACK_STRATEGY_KEY; @@ -38,6 +44,8 @@ public abstract class IoTDBSubscriptionPushSource extends PushSource { + private static final Logger LOGGER = LoggerFactory.getLogger(IoTDBSubscriptionPushSource.class); + protected volatile boolean isStarted = true; protected Thread workerThread; @@ -45,7 +53,7 @@ public abstract class IoTDBSubscriptionPushSource extends PushSource { private Long autoPollTimeoutMs; private AckStrategy ackStrategy; - protected final IoTDBSubscription subscription = new IoTDBSubscription(); + protected final IoTDBSubscriptionCommon subscription = new IoTDBSubscriptionCommon(); @Override public void validate(final PipeParameterValidator validator) throws Exception { @@ -94,10 +102,57 @@ public void customize( IOTDB_SUBSCRIPTION_SOURCE_AUTO_POLL_TIMEOUT_MS_DEFAULT_VALUE); } + @Override + public void start() throws Exception { + if (workerThread == null || !workerThread.isAlive()) { + isStarted = true; + + workerThread = new Thread(this::doWork); + workerThread.setName(getPushConsumerThreadName()); + workerThread.start(); + } + } + + protected abstract void doWork(); + + protected abstract String getPushConsumerThreadName(); + + protected AbstractSubscriptionPushConsumerBuilder getPushConsumerBuilder() { + return getSubscriptionPushConsumerBuilder() + .consumeListener( + message -> { + for (final SubscriptionSessionDataSet dataSet : message.getSessionDataSetsHandler()) { + try { + subscription.put(new PipeRawTabletInsertionEvent(dataSet.getTablet(), isStarted)); + } catch (final InterruptedException e) { + LOGGER.warn("{} thread interrupted", getPushConsumerThreadName(), e); + Thread.currentThread().interrupt(); + + return ConsumeResult.FAILURE; + } + } + return ConsumeResult.SUCCESS; + }); + } + protected AbstractSubscriptionPushConsumerBuilder getSubscriptionPushConsumerBuilder() { return ((AbstractSubscriptionPushConsumerBuilder) subscription.getSubscriptionConsumerBuilder()) .ackStrategy(ackStrategy) .autoPollIntervalMs(autoPollIntervalMs) .autoPollTimeoutMs(autoPollTimeoutMs); } + + @Override + public void close() throws Exception { + isStarted = false; + if (workerThread != null) { + workerThread.interrupt(); + try { + workerThread.join(1000); + } catch (final InterruptedException e) { + Thread.currentThread().interrupt(); + } + workerThread = null; + } + } } diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionSourceConstant.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionSourceConstant.java index 85034f30..ac91175d 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionSourceConstant.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionSourceConstant.java @@ -68,13 +68,15 @@ public class IoTDBSubscriptionSourceConstant { public static final long IOTDB_SUBSCRIPTION_SOURCE_AUTO_COMMIT_INTERVAL_MS_MIN_VALUE = 500L; public static final String IOTDB_SUBSCRIPTION_SOURCE_ACK_STRATEGY_KEY = "ack-strategy"; + public static final String IOTDB_SUBSCRIPTION_SOURCE_ACK_STRATEGY_BEFORE_VALUE = "before"; + public static final String IOTDB_SUBSCRIPTION_SOURCE_ACK_STRATEGY_AFTER_VALUE = "after"; public static final String IOTDB_SUBSCRIPTION_SOURCE_ACK_STRATEGY_DEFAULT_VALUE = "after"; public static final Map IOTDB_SUBSCRIPTION_SOURCE_ACK_STRATEGY_VALUE_MAP = Collections.unmodifiableMap( new HashMap() { { - put("before", AckStrategy.BEFORE_CONSUME); - put("after", AckStrategy.AFTER_CONSUME); + put(IOTDB_SUBSCRIPTION_SOURCE_ACK_STRATEGY_BEFORE_VALUE, AckStrategy.BEFORE_CONSUME); + put(IOTDB_SUBSCRIPTION_SOURCE_ACK_STRATEGY_AFTER_VALUE, AckStrategy.AFTER_CONSUME); } }); diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTablePullSource.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTablePullSource.java index 45fcc321..f61b5728 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTablePullSource.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTablePullSource.java @@ -19,81 +19,32 @@ package org.apache.iotdb.collector.plugin.builtin.source.iotdb; -import org.apache.iotdb.collector.plugin.builtin.sink.event.PipeRawTabletInsertionEvent; import org.apache.iotdb.collector.runtime.progress.ProgressIndex; -import org.apache.iotdb.pipe.api.event.Event; import org.apache.iotdb.session.subscription.consumer.ISubscriptionTablePullConsumer; import org.apache.iotdb.session.subscription.payload.SubscriptionMessage; -import org.apache.iotdb.session.subscription.payload.SubscriptionSessionDataSet; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.util.List; import java.util.Optional; public class IoTDBSubscriptionTablePullSource extends IoTDBSubscriptionPullSource { - private static final Logger LOGGER = - LoggerFactory.getLogger(IoTDBSubscriptionTablePullSource.class); - private ISubscriptionTablePullConsumer consumer; - private static final Long POLL_TIMEOUT_MS = 10_000L; - - private volatile boolean isStarted; - private Thread workerThread; @Override - public void start() throws Exception { + protected void initPullConsumer() { consumer = getSubscriptionPullConsumerBuilder().buildTablePullConsumer(); consumer.open(); consumer.subscribe(subscription.getTopic()); - - if (workerThread == null) { - isStarted = true; - - workerThread = new Thread(this::doWork); - workerThread.setName("iotdb-subscription-tree-pull-source"); - workerThread.start(); - } - } - - private void doWork() { - while (isStarted && !Thread.currentThread().isInterrupted()) { - final List messages = consumer.poll(POLL_TIMEOUT_MS); - - for (final SubscriptionMessage message : messages) { - for (final SubscriptionSessionDataSet dataSet : message.getSessionDataSetsHandler()) { - subscription.checkIfNeedPause(); - - try { - subscription.put(new PipeRawTabletInsertionEvent(dataSet.getTablet(), isAligned)); - } catch (final InterruptedException e) { - LOGGER.warn("iotdb subscription tree model pull consumer thread interrupted", e); - Thread.currentThread().interrupt(); - } - } - } - } } @Override - public Event supply() throws InterruptedException { - return subscription.take(); + protected String getPullConsumerThreadName() { + return "iotdb-subscription-table-pull-source"; } @Override - public void close() throws Exception { - isStarted = false; - if (workerThread != null) { - workerThread.interrupt(); - try { - workerThread.join(1000); - } catch (final InterruptedException e) { - Thread.currentThread().interrupt(); - } - workerThread = null; - } + protected List poll() { + return consumer.poll(POLL_TIMEOUT_MS); } @Override diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTablePushSource.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTablePushSource.java index 65ff644c..76785c0a 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTablePushSource.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTablePushSource.java @@ -19,11 +19,8 @@ package org.apache.iotdb.collector.plugin.builtin.source.iotdb; -import org.apache.iotdb.collector.plugin.builtin.sink.event.PipeRawTabletInsertionEvent; import org.apache.iotdb.collector.runtime.progress.ProgressIndex; -import org.apache.iotdb.session.subscription.consumer.ConsumeResult; import org.apache.iotdb.session.subscription.consumer.ISubscriptionTablePushConsumer; -import org.apache.iotdb.session.subscription.payload.SubscriptionSessionDataSet; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -35,58 +32,19 @@ public class IoTDBSubscriptionTablePushSource extends IoTDBSubscriptionPushSourc private static final Logger LOGGER = LoggerFactory.getLogger(IoTDBSubscriptionTablePushSource.class); - @Override - public void start() throws Exception { - if (workerThread == null || !workerThread.isAlive()) { - isStarted = true; - - workerThread = new Thread(this::doWork); - workerThread.setName("iotdb-subscription-tree-push-worker"); - workerThread.start(); - } - } - - private void doWork() { + protected void doWork() { try (final ISubscriptionTablePushConsumer consumer = - getSubscriptionPushConsumerBuilder() - .consumeListener( - message -> { - for (final SubscriptionSessionDataSet dataSet : - message.getSessionDataSetsHandler()) { - try { - subscription.put( - new PipeRawTabletInsertionEvent(dataSet.getTablet(), isStarted)); - } catch (final InterruptedException e) { - LOGGER.warn( - "iotdb subscription tree model push consumer thread interrupted", e); - Thread.currentThread().interrupt(); - - return ConsumeResult.FAILURE; - } - } - - return ConsumeResult.SUCCESS; - }) - .buildTablePushConsumer()) { + getPushConsumerBuilder().buildTablePushConsumer()) { consumer.open(); consumer.subscribe(subscription.getTopic()); - } catch (Exception e) { - throw new RuntimeException(e); + } catch (final Exception e) { + LOGGER.warn("Error occurred while {} thread", getPushConsumerThreadName(), e); } } @Override - public void close() throws Exception { - isStarted = false; - if (workerThread != null) { - workerThread.interrupt(); - try { - workerThread.join(1000); - } catch (final InterruptedException e) { - Thread.currentThread().interrupt(); - } - workerThread = null; - } + protected String getPushConsumerThreadName() { + return "iotdb-subscription-table-push-source"; } @Override diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTreePullSource.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTreePullSource.java index f6619c23..8c168d36 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTreePullSource.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTreePullSource.java @@ -19,81 +19,32 @@ package org.apache.iotdb.collector.plugin.builtin.source.iotdb; -import org.apache.iotdb.collector.plugin.builtin.sink.event.PipeRawTabletInsertionEvent; import org.apache.iotdb.collector.runtime.progress.ProgressIndex; -import org.apache.iotdb.pipe.api.event.Event; import org.apache.iotdb.session.subscription.consumer.ISubscriptionTreePullConsumer; import org.apache.iotdb.session.subscription.payload.SubscriptionMessage; -import org.apache.iotdb.session.subscription.payload.SubscriptionSessionDataSet; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.util.List; import java.util.Optional; public class IoTDBSubscriptionTreePullSource extends IoTDBSubscriptionPullSource { - private static final Logger LOGGER = - LoggerFactory.getLogger(IoTDBSubscriptionTreePullSource.class); - private ISubscriptionTreePullConsumer consumer; - private static final Long POLL_TIMEOUT_MS = 10_000L; - - private volatile boolean isStarted; - private Thread workerThread; @Override - public void start() throws Exception { + protected void initPullConsumer() { consumer = getSubscriptionPullConsumerBuilder().buildPullConsumer(); consumer.open(); consumer.subscribe(subscription.getTopic()); - - if (workerThread == null) { - isStarted = true; - - workerThread = new Thread(this::doWork); - workerThread.setName("iotdb-subscription-tree-pull-source"); - workerThread.start(); - } - } - - private void doWork() { - while (isStarted && !Thread.currentThread().isInterrupted()) { - final List messages = consumer.poll(POLL_TIMEOUT_MS); - - for (final SubscriptionMessage message : messages) { - for (final SubscriptionSessionDataSet dataSet : message.getSessionDataSetsHandler()) { - subscription.checkIfNeedPause(); - - try { - subscription.put(new PipeRawTabletInsertionEvent(dataSet.getTablet(), isAligned)); - } catch (final InterruptedException e) { - LOGGER.warn("iotdb subscription tree model pull consumer thread interrupted", e); - Thread.currentThread().interrupt(); - } - } - } - } } @Override - public Event supply() throws InterruptedException { - return subscription.take(); + protected String getPullConsumerThreadName() { + return "iotdb-subscription-tree-pull-source"; } @Override - public void close() throws Exception { - isStarted = false; - if (workerThread != null) { - workerThread.interrupt(); - try { - workerThread.join(1000); - } catch (final InterruptedException e) { - Thread.currentThread().interrupt(); - } - workerThread = null; - } + protected List poll() { + return consumer.poll(POLL_TIMEOUT_MS); } @Override diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTreePushSource.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTreePushSource.java index 85c29497..5af6c27e 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTreePushSource.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTreePushSource.java @@ -19,11 +19,8 @@ package org.apache.iotdb.collector.plugin.builtin.source.iotdb; -import org.apache.iotdb.collector.plugin.builtin.sink.event.PipeRawTabletInsertionEvent; import org.apache.iotdb.collector.runtime.progress.ProgressIndex; -import org.apache.iotdb.session.subscription.consumer.ConsumeResult; import org.apache.iotdb.session.subscription.consumer.ISubscriptionTreePushConsumer; -import org.apache.iotdb.session.subscription.payload.SubscriptionSessionDataSet; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -35,58 +32,19 @@ public class IoTDBSubscriptionTreePushSource extends IoTDBSubscriptionPushSource private static final Logger LOGGER = LoggerFactory.getLogger(IoTDBSubscriptionTreePushSource.class); - @Override - public void start() throws Exception { - if (workerThread == null || !workerThread.isAlive()) { - isStarted = true; - - workerThread = new Thread(this::doWork); - workerThread.setName("iotdb-subscription-tree-push-worker"); - workerThread.start(); - } - } - - private void doWork() { + protected void doWork() { try (final ISubscriptionTreePushConsumer consumer = - getSubscriptionPushConsumerBuilder() - .consumeListener( - message -> { - for (final SubscriptionSessionDataSet dataSet : - message.getSessionDataSetsHandler()) { - try { - subscription.put( - new PipeRawTabletInsertionEvent(dataSet.getTablet(), isStarted)); - } catch (final InterruptedException e) { - LOGGER.warn( - "iotdb subscription tree model push consumer thread interrupted", e); - Thread.currentThread().interrupt(); - - return ConsumeResult.FAILURE; - } - } - - return ConsumeResult.SUCCESS; - }) - .buildPushConsumer()) { + getPushConsumerBuilder().buildPushConsumer()) { consumer.open(); consumer.subscribe(subscription.getTopic()); - } catch (Exception e) { - throw new RuntimeException(e); + } catch (final Exception e) { + LOGGER.warn("Error occurred while {} thread", getPushConsumerThreadName(), e); } } @Override - public void close() throws Exception { - isStarted = false; - if (workerThread != null) { - workerThread.interrupt(); - try { - workerThread.join(1000); - } catch (final InterruptedException e) { - Thread.currentThread().interrupt(); - } - workerThread = null; - } + protected String getPushConsumerThreadName() { + return "iotdb-subscription-tree-push-source"; } @Override diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/kafka/KafkaSourceConstant.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/kafka/KafkaSourceConstant.java index 0691e17a..d4655d48 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/kafka/KafkaSourceConstant.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/kafka/KafkaSourceConstant.java @@ -47,9 +47,17 @@ public class KafkaSourceConstant { StringDeserializer.class.getName(); public static final String KAFKA_SOURCE_AUTO_OFFSET_RESET_KEY = "auto.offset.reset"; + public static final String KAFKA_SOURCE_AUTO_OFFSET_RESET_NONE_VALUE = "none"; + public static final String KAFKA_SOURCE_AUTO_OFFSET_RESET_EARLIEST_VALUE = "earliest"; + public static final String KAFKA_SOURCE_AUTO_OFFSET_RESET_LATEST_VALUE = "latest"; public static final String KAFKA_SOURCE_AUTO_OFFSET_RESET_DEFAULT_VALUE = "none"; public static final Set KAFKA_SOURCE_AUTO_OFFSET_RESET_VALUE_SET = - Collections.unmodifiableSet(new HashSet<>(Arrays.asList("none", "earliest", "latest"))); + Collections.unmodifiableSet( + new HashSet<>( + Arrays.asList( + KAFKA_SOURCE_AUTO_OFFSET_RESET_NONE_VALUE, + KAFKA_SOURCE_AUTO_OFFSET_RESET_EARLIEST_VALUE, + KAFKA_SOURCE_AUTO_OFFSET_RESET_LATEST_VALUE))); public static final String KAFKA_SOURCE_ENABLE_AUTO_COMMIT_KEY = "enable.auto.commit"; public static final boolean KAFKA_SOURCE_ENABLE_AUTO_COMMIT_DEFAULT_VALUE = false; From 02c99cfc0e70964d763b0f08e58c2d124716b4e9 Mon Sep 17 00:00:00 2001 From: ysc <2725843507@qq.com> Date: Wed, 28 May 2025 15:54:59 +0800 Subject: [PATCH 05/11] migrate memory control --- .../resources/conf/application.properties | 11 +- .../collector/config/PipeRuntimeOptions.java | 34 +- .../api/customizer/CollectorParameters.java | 9 - ...IoTDBDataNodeCacheLeaderClientManager.java | 52 ++- .../sink/client/IoTDBSyncClientManager.java | 2 +- .../evolvable/batch/PipeTabletEventBatch.java | 40 +- .../batch/PipeTabletEventPlainBatch.java | 6 +- .../batch/PipeTransferBatchReqBuilder.java | 8 +- .../request/PipeTransferTabletBatchReqV2.java | 38 -- .../IoTDBDataRegionSyncConnector.java | 7 +- .../memory/AtomicLongMemoryBlock.java | 70 +++ .../sink/resource/memory/IMemoryBlock.java | 86 ++++ .../sink/resource/memory/MemoryBlockType.java | 28 ++ .../sink/resource/memory/MemoryException.java | 53 +++ .../sink/resource/memory/MemoryManager.java | 405 ++++++++++++++++++ .../resource/memory/PipeFixedMemoryBlock.java | 64 +++ .../sink/resource/memory/PipeMemoryBlock.java | 175 ++++++++ .../resource/memory/PipeMemoryBlockType.java | 26 ++ .../resource/memory/PipeMemoryManager.java | 163 +++++++ .../memory/PipeTabletMemoryBlock.java | 27 ++ .../memory/PipeTsFileMemoryBlock.java | 27 ++ .../source/constant/SourceConstant.java | 2 +- .../source/iotdb/IoTDBSubscriptionCommon.java | 54 ++- .../iotdb/IoTDBSubscriptionPullSource.java | 31 +- .../iotdb/IoTDBSubscriptionPushSource.java | 72 +++- .../IoTDBSubscriptionSourceConstant.java | 2 +- .../IoTDBSubscriptionTablePullSource.java | 2 +- .../IoTDBSubscriptionTablePushSource.java | 2 +- .../IoTDBSubscriptionTreePullSource.java | 18 +- .../IoTDBSubscriptionTreePushSource.java | 2 +- .../source/kafka/KafkaSourceConstant.java | 2 +- .../collector/runtime/task/TaskRuntime.java | 45 +- 32 files changed, 1408 insertions(+), 155 deletions(-) create mode 100644 iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/resource/memory/AtomicLongMemoryBlock.java create mode 100644 iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/resource/memory/IMemoryBlock.java create mode 100644 iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/resource/memory/MemoryBlockType.java create mode 100644 iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/resource/memory/MemoryException.java create mode 100644 iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/resource/memory/MemoryManager.java create mode 100644 iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/resource/memory/PipeFixedMemoryBlock.java create mode 100644 iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/resource/memory/PipeMemoryBlock.java create mode 100644 iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/resource/memory/PipeMemoryBlockType.java create mode 100644 iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/resource/memory/PipeMemoryManager.java create mode 100644 iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/resource/memory/PipeTabletMemoryBlock.java create mode 100644 iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/resource/memory/PipeTsFileMemoryBlock.java diff --git a/iotdb-collector/collector-core/src/assembly/resources/conf/application.properties b/iotdb-collector/collector-core/src/assembly/resources/conf/application.properties index d72b18c1..26567883 100644 --- a/iotdb-collector/collector-core/src/assembly/resources/conf/application.properties +++ b/iotdb-collector/collector-core/src/assembly/resources/conf/application.properties @@ -63,7 +63,7 @@ task_database_file_path=system/database/task.db # Proactively triggers the interval for batch deliveries # Effective mode: on every start # Data type: long -executor_cron_heartbeat_event_interval_seconds=20 +executor_cron_heartbeat_event_interval_seconds=5 # Task progress storage interval # Effective mode: on every start @@ -114,7 +114,7 @@ max_retry_times=5 # Bind with rpc_address # Effective mode: on every start # Data type: int -rpc_port=6667 +rpc_port=6668 # Used for connection of IoTDB native clients(Session) # Could set 127.0.0.1(for local test) or ipv4 address @@ -156,4 +156,9 @@ pipe_leader_cache_memory_usage_percentage=0.1 # Enable/disable reference tracking for pipe events # Effective mode: on every start # Data type: boolean -pipe_event_reference_tracking_enabled=true \ No newline at end of file +pipe_event_reference_tracking_enabled=true + +# The maximum number of tablets that can be in a batch +# Effective mode: on every start +# Data type: int +pipe_max_allowed_event_count_in_tablet_batch=100 \ No newline at end of file diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/config/PipeRuntimeOptions.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/config/PipeRuntimeOptions.java index c5f42020..fe47a092 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/config/PipeRuntimeOptions.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/config/PipeRuntimeOptions.java @@ -265,11 +265,43 @@ public void setValue(final String valueString) { } }; - public static volatile Option DATA_NODE_ID = + public static final Option DATA_NODE_ID = new Option("data_node_id", -1) { @Override public void setValue(final String valueString) { value = Integer.parseInt(valueString); } }; + + public static final Option PIPE_MEMORY_MANAGEMENT_ENABLED = + new Option("pipe_memory_management_enabled", true) { + @Override + public void setValue(String valueString) { + value = Boolean.parseBoolean(valueString); + } + }; + + public static final Option PIPE_MEMORY_ALLOCATE_MIN_SIZE_IN_BYTES = + new Option("pipe_memory_allocate_min_size_in_bytes", 32) { + @Override + public void setValue(String valueString) { + value = Integer.parseInt(valueString); + } + }; + + public static final Option PIPE_TOTAL_FLOATING_MEMORY_PROPORTION = + new Option("pipe_total_floating_memory_proportion", 0.2) { + @Override + public void setValue(String valueString) { + value = Double.parseDouble(valueString); + } + }; + + public static final Option PIPE_MAX_ALLOWED_EVENT_COUNT_IN_TABLET_BATCH = + new Option("pipe_max_allowed_event_count_in_tablet_batch", 100) { + @Override + public void setValue(String valueString) { + value = Integer.parseInt(valueString); + } + }; } diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/api/customizer/CollectorParameters.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/api/customizer/CollectorParameters.java index fe49320d..d1b0c022 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/api/customizer/CollectorParameters.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/api/customizer/CollectorParameters.java @@ -21,9 +21,6 @@ import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameterValidator; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashSet; import java.util.Objects; import java.util.Set; import java.util.function.Predicate; @@ -31,12 +28,6 @@ import static org.apache.iotdb.collector.plugin.builtin.source.constant.SourceConstant.BOOLEAN_SET; public class CollectorParameters { - private static final Set PARAM_SET = - Collections.unmodifiableSet(new HashSet<>(Arrays.asList("source", "processor", "sink"))); - - public static boolean matchAnyParam(final String param) { - return PARAM_SET.contains(param); - } public static void validateStringRequiredParam( final PipeParameterValidator validator, final String paramKey) { diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/client/IoTDBDataNodeCacheLeaderClientManager.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/client/IoTDBDataNodeCacheLeaderClientManager.java index dff5f2de..b680c611 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/client/IoTDBDataNodeCacheLeaderClientManager.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/client/IoTDBDataNodeCacheLeaderClientManager.java @@ -19,12 +19,17 @@ package org.apache.iotdb.collector.plugin.builtin.sink.client; +import org.apache.iotdb.collector.config.PipeRuntimeOptions; +import org.apache.iotdb.collector.plugin.builtin.sink.resource.memory.PipeMemoryBlock; +import org.apache.iotdb.collector.plugin.builtin.sink.resource.memory.PipeMemoryManager; import org.apache.iotdb.common.rpc.thrift.TEndPoint; import com.github.benmanes.caffeine.cache.Cache; import com.github.benmanes.caffeine.cache.Caffeine; import com.github.benmanes.caffeine.cache.Weigher; import com.google.common.util.concurrent.AtomicDouble; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.util.concurrent.ConcurrentHashMap; @@ -32,7 +37,9 @@ public interface IoTDBDataNodeCacheLeaderClientManager { LeaderCacheManager LEADER_CACHE_MANAGER = new LeaderCacheManager(); - class LeaderCacheManager { + class LeaderCacheManager implements AutoCloseable { + + private static final Logger LOGGER = LoggerFactory.getLogger(LeaderCacheManager.class); private final AtomicDouble memoryUsageCheatFactor = new AtomicDouble(1); @@ -41,9 +48,45 @@ class LeaderCacheManager { // a hashmap to reuse the created endpoint private final ConcurrentHashMap endPoints = new ConcurrentHashMap<>(); + private final PipeMemoryBlock allocatedMemoryBlock; + public LeaderCacheManager() { + final long initMemorySizeInBytes = + PipeMemoryManager.getInstance().getTotalNonFloatingMemorySizeInBytes() / 10; + final long maxMemorySizeInBytes = + (long) + (PipeMemoryManager.getInstance().getTotalNonFloatingMemorySizeInBytes() + * PipeRuntimeOptions.PIPE_LEADER_CACHE_MEMORY_USAGE_PERCENTAGE.value()); + + // properties required by pipe memory control framework + allocatedMemoryBlock = + PipeMemoryManager.getInstance() + .tryAllocate(initMemorySizeInBytes) + .setShrinkMethod(oldMemory -> Math.max(oldMemory / 2, 1)) + .setShrinkCallback( + (oldMemory, newMemory) -> { + memoryUsageCheatFactor.updateAndGet( + factor -> factor * ((double) oldMemory / newMemory)); + LOGGER.info( + "LeaderCacheManager.allocatedMemoryBlock has shrunk from {} to {}.", + oldMemory, + newMemory); + }) + .setExpandMethod( + oldMemory -> Math.min(Math.max(oldMemory, 1) * 2, maxMemorySizeInBytes)) + .setExpandCallback( + (oldMemory, newMemory) -> { + memoryUsageCheatFactor.updateAndGet( + factor -> factor / ((double) newMemory / oldMemory)); + LOGGER.info( + "LeaderCacheManager.allocatedMemoryBlock has expanded from {} to {}.", + oldMemory, + newMemory); + }); + device2endpoint = Caffeine.newBuilder() + .maximumWeight(allocatedMemoryBlock.getMemoryUsageInBytes()) .weigher( (Weigher) (device, endPoint) -> { @@ -75,5 +118,12 @@ public void updateLeaderEndPoint(final String deviceId, final TEndPoint endPoint device2endpoint.put(deviceId, endPoint); } } + + @Override + public void close() throws Exception { + if (allocatedMemoryBlock != null) { + allocatedMemoryBlock.close(); + } + } } } diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/client/IoTDBSyncClientManager.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/client/IoTDBSyncClientManager.java index fadb2d2b..e70ced28 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/client/IoTDBSyncClientManager.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/client/IoTDBSyncClientManager.java @@ -177,7 +177,7 @@ private boolean initClientAndStatus( new IoTDBSyncClient( new ThriftClientProperty.Builder() .setConnectionTimeoutMs( - PipeRuntimeOptions.PIPE_CONNECTOR_TRANSFER_TIMEOUT_MS.value()) + PipeRuntimeOptions.PIPE_CONNECTOR_HANDSHAKE_TIMEOUT_MS.value()) .setRpcThriftCompressionEnabled( PipeRuntimeOptions.IS_PIPE_CONNECTOR_RPC_THRIFT_COMPRESSION_ENABLED.value()) .build(), diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/payload/evolvable/batch/PipeTabletEventBatch.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/payload/evolvable/batch/PipeTabletEventBatch.java index b26a065a..aaf0a026 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/payload/evolvable/batch/PipeTabletEventBatch.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/payload/evolvable/batch/PipeTabletEventBatch.java @@ -19,10 +19,16 @@ package org.apache.iotdb.collector.plugin.builtin.sink.payload.evolvable.batch; +import org.apache.iotdb.collector.config.PipeRuntimeOptions; import org.apache.iotdb.collector.plugin.builtin.sink.event.PipeRawTabletInsertionEvent; +import org.apache.iotdb.collector.plugin.builtin.sink.resource.memory.PipeMemoryBlock; +import org.apache.iotdb.collector.plugin.builtin.sink.resource.memory.PipeMemoryManager; import org.apache.iotdb.pipe.api.event.Event; import org.apache.iotdb.pipe.api.event.dml.insertion.TabletInsertionEvent; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.io.IOException; import java.util.ArrayList; import java.util.List; @@ -30,17 +36,45 @@ public abstract class PipeTabletEventBatch implements AutoCloseable { + private static final Logger LOGGER = LoggerFactory.getLogger(PipeTabletEventBatch.class); + protected final List events = new ArrayList<>(); private final int maxDelayInMs; private long firstEventProcessingTime = Long.MIN_VALUE; + private final PipeMemoryBlock allocatedMemoryBlock; + protected long totalBufferSize = 0; protected volatile boolean isClosed = false; protected PipeTabletEventBatch(final int maxDelayInMs, final long requestMaxBatchSizeInBytes) { this.maxDelayInMs = maxDelayInMs; + + // limit in buffer size + this.allocatedMemoryBlock = + PipeMemoryManager.getInstance() + .tryAllocate(requestMaxBatchSizeInBytes) + .setShrinkMethod(oldMemory -> Math.max(oldMemory / 2, 0)) + .setShrinkCallback( + (oldMemory, newMemory) -> + LOGGER.info( + "The batch size limit has shrunk from {} to {}.", oldMemory, newMemory)) + .setExpandMethod( + oldMemory -> Math.min(Math.max(oldMemory, 1) * 2, requestMaxBatchSizeInBytes)) + .setExpandCallback( + (oldMemory, newMemory) -> + LOGGER.info( + "The batch size limit has expanded from {} to {}.", oldMemory, newMemory)); + + if (getMaxBatchSizeInBytes() != requestMaxBatchSizeInBytes) { + LOGGER.info( + "PipeTabletEventBatch: the max batch size is adjusted from {} to {} due to the " + + "memory restriction", + requestMaxBatchSizeInBytes, + getMaxBatchSizeInBytes()); + } } /** @@ -81,12 +115,12 @@ public synchronized boolean onEvent(final TabletInsertionEvent event) throws IOE public boolean shouldEmit() { return totalBufferSize >= getMaxBatchSizeInBytes() - || System.currentTimeMillis() - firstEventProcessingTime >= maxDelayInMs; + || System.currentTimeMillis() - firstEventProcessingTime >= maxDelayInMs + || events.size() > PipeRuntimeOptions.PIPE_MAX_ALLOWED_EVENT_COUNT_IN_TABLET_BATCH.value(); } private long getMaxBatchSizeInBytes() { - // return allocatedMemoryBlock.getMemoryUsageInBytes(); - return 16777216; + return allocatedMemoryBlock.getMemoryUsageInBytes(); } public synchronized void onSuccess() { diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/payload/evolvable/batch/PipeTabletEventPlainBatch.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/payload/evolvable/batch/PipeTabletEventPlainBatch.java index e1477887..54eacf79 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/payload/evolvable/batch/PipeTabletEventPlainBatch.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/payload/evolvable/batch/PipeTabletEventPlainBatch.java @@ -26,8 +26,6 @@ import org.apache.tsfile.utils.Pair; import org.apache.tsfile.utils.PublicBAOS; import org.apache.tsfile.utils.ReadWriteIOUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.io.DataOutputStream; import java.io.IOException; @@ -39,8 +37,6 @@ public class PipeTabletEventPlainBatch extends PipeTabletEventBatch { - private static final Logger LOGGER = LoggerFactory.getLogger(PipeTabletEventPlainBatch.class); - private final List binaryBuffers = new ArrayList<>(); private final List insertNodeBuffers = new ArrayList<>(); private final List tabletBuffers = new ArrayList<>(); @@ -62,7 +58,7 @@ protected boolean constructBatch(final TabletInsertionEvent event) throws IOExce final int bufferSize = buildTabletInsertionBuffer(event); totalBufferSize += bufferSize; pipe2BytesAccumulated.compute( - new Pair<>("", 0L), + new Pair<>("", System.currentTimeMillis()), (pipeName, bytesAccumulated) -> bytesAccumulated == null ? bufferSize : bytesAccumulated + bufferSize); return true; diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/payload/evolvable/batch/PipeTransferBatchReqBuilder.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/payload/evolvable/batch/PipeTransferBatchReqBuilder.java index 5cd29e47..95c6322d 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/payload/evolvable/batch/PipeTransferBatchReqBuilder.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/payload/evolvable/batch/PipeTransferBatchReqBuilder.java @@ -19,7 +19,7 @@ package org.apache.iotdb.collector.plugin.builtin.sink.payload.evolvable.batch; -import org.apache.iotdb.collector.config.PipeRuntimeOptions; +import org.apache.iotdb.collector.plugin.builtin.sink.client.IoTDBDataNodeCacheLeaderClientManager; import org.apache.iotdb.collector.plugin.builtin.sink.event.PipeRawTabletInsertionEvent; import org.apache.iotdb.common.rpc.thrift.TEndPoint; import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameters; @@ -142,7 +142,11 @@ public synchronized Pair onEvent( } final TEndPoint endPoint = - new TEndPoint(PipeRuntimeOptions.RPC_ADDRESS.value(), PipeRuntimeOptions.RPC_PORT.value()); + IoTDBDataNodeCacheLeaderClientManager.LEADER_CACHE_MANAGER.getLeaderEndPoint(deviceId); + + if (Objects.isNull(endPoint)) { + return defaultBatch.onEvent(event) ? new Pair<>(null, defaultBatch) : null; + } final PipeTabletEventPlainBatch batch = endPointToBatch.computeIfAbsent( diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/payload/thrift/request/PipeTransferTabletBatchReqV2.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/payload/thrift/request/PipeTransferTabletBatchReqV2.java index f59678e7..a21e9907 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/payload/thrift/request/PipeTransferTabletBatchReqV2.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/payload/thrift/request/PipeTransferTabletBatchReqV2.java @@ -85,44 +85,6 @@ public static PipeTransferTabletBatchReqV2 toTPipeTransferReq( return batchReq; } - // public static PipeTransferTabletBatchReqV2 fromTPipeTransferReq( - // final TPipeTransferReq transferReq) { - // final PipeTransferTabletBatchReqV2 batchReq = new PipeTransferTabletBatchReqV2(); - // - // int size = ReadWriteIOUtils.readInt(transferReq.body); - // for (int i = 0; i < size; ++i) { - // final int length = ReadWriteIOUtils.readInt(transferReq.body); - // final byte[] body = new byte[length]; - // transferReq.body.get(body); - // batchReq.binaryReqs.add( - // PipeTransferTabletBinaryReqV2.toTPipeTransferBinaryReq( - // ByteBuffer.wrap(body), ReadWriteIOUtils.readString(transferReq.body))); - // } - // - // size = ReadWriteIOUtils.readInt(transferReq.body); - // for (int i = 0; i < size; ++i) { - // batchReq.insertNodeReqs.add( - // PipeTransferTabletInsertNodeReqV2.toTabletInsertNodeReq( - // (InsertNode) PlanFragment.deserializeHelper(transferReq.body, null), - // ReadWriteIOUtils.readString(transferReq.body))); - // } - // - // size = ReadWriteIOUtils.readInt(transferReq.body); - // for (int i = 0; i < size; ++i) { - // batchReq.tabletReqs.add( - // PipeTransferTabletRawReqV2.toTPipeTransferRawReq( - // Tablet.deserialize(transferReq.body), - // ReadWriteIOUtils.readBool(transferReq.body), - // ReadWriteIOUtils.readString(transferReq.body))); - // } - // - // batchReq.version = transferReq.version; - // batchReq.type = transferReq.type; - // batchReq.body = transferReq.body; - // - // return batchReq; - // } - /////////////////////////////// Object /////////////////////////////// @Override diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/protocol/IoTDBDataRegionSyncConnector.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/protocol/IoTDBDataRegionSyncConnector.java index 2cde1b5a..008133ec 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/protocol/IoTDBDataRegionSyncConnector.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/protocol/IoTDBDataRegionSyncConnector.java @@ -19,7 +19,6 @@ package org.apache.iotdb.collector.plugin.builtin.sink.protocol; -import org.apache.iotdb.collector.plugin.api.event.PeriodicalEvent; import org.apache.iotdb.collector.plugin.builtin.sink.client.IoTDBDataNodeSyncClientManager; import org.apache.iotdb.collector.plugin.builtin.sink.client.IoTDBSyncClient; import org.apache.iotdb.collector.plugin.builtin.sink.client.IoTDBSyncClientManager; @@ -59,7 +58,6 @@ import java.io.IOException; import java.nio.file.NoSuchFileException; import java.util.List; -import java.util.Map; import java.util.Objects; public class IoTDBDataRegionSyncConnector extends IoTDBSslSyncConnector { @@ -159,9 +157,7 @@ public void transfer(final TsFileInsertionEvent tsFileInsertionEvent) throws Exc @Override public void transfer(final Event event) throws Exception { - if (event instanceof PeriodicalEvent) { - doTransferWrapper(); - } else if (isTabletBatchModeEnabled && !tabletBatchBuilder.isEmpty()) { + if (isTabletBatchModeEnabled && !tabletBatchBuilder.isEmpty()) { doTransferWrapper(); } } @@ -222,7 +218,6 @@ private void doTransfer( private void doTransfer(final PipeTabletEventTsFileBatch batchToTransfer) throws IOException, WriteProcessException { final List> dbTsFilePairs = batchToTransfer.sealTsFiles(); - final Map, Double> pipe2WeightMap = batchToTransfer.deepCopyPipe2WeightMap(); for (final Pair dbTsFile : dbTsFilePairs) { doTransfer(dbTsFile.right, null, dbTsFile.left); diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/resource/memory/AtomicLongMemoryBlock.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/resource/memory/AtomicLongMemoryBlock.java new file mode 100644 index 00000000..f73c3a53 --- /dev/null +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/resource/memory/AtomicLongMemoryBlock.java @@ -0,0 +1,70 @@ +/* + * 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.collector.plugin.builtin.sink.resource.memory; + +import java.util.concurrent.atomic.AtomicLong; + +public class AtomicLongMemoryBlock extends IMemoryBlock { + /** The memory usage in byte of this memory block */ + protected final AtomicLong usedMemoryInBytes = new AtomicLong(0); + + public AtomicLongMemoryBlock( + final String name, + final MemoryManager memoryManager, + final long maxMemorySizeInByte, + final MemoryBlockType memoryBlockType) { + this.name = name; + this.memoryManager = memoryManager; + this.totalMemorySizeInBytes = maxMemorySizeInByte; + this.memoryBlockType = memoryBlockType; + } + + @Override + public void forceAllocateWithoutLimitation(long sizeInByte) { + usedMemoryInBytes.addAndGet(sizeInByte); + } + + public long getUsedMemoryInBytes() { + return usedMemoryInBytes.get(); + } + + @Override + public String toString() { + return "IoTDBMemoryBlock{" + + "name=" + + name + + ", isReleased=" + + isReleased + + ", memoryBlockType=" + + memoryBlockType + + ", totalMemorySizeInBytes=" + + totalMemorySizeInBytes + + ", usedMemoryInBytes=" + + usedMemoryInBytes + + '}'; + } + + @Override + public void close() throws Exception { + if (memoryManager != null) { + memoryManager.release(this); + } + } +} diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/resource/memory/IMemoryBlock.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/resource/memory/IMemoryBlock.java new file mode 100644 index 00000000..a7acde3a --- /dev/null +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/resource/memory/IMemoryBlock.java @@ -0,0 +1,86 @@ +/* + * 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.collector.plugin.builtin.sink.resource.memory; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public abstract class IMemoryBlock implements AutoCloseable { + private static final Logger LOGGER = LoggerFactory.getLogger(IMemoryBlock.class); + + /** The memory manager that manages this memory block */ + protected MemoryManager memoryManager; + + /** The name of this memory block */ + protected String name; + + /** The type of this memory block */ + protected MemoryBlockType memoryBlockType; + + /** The flag that indicates whether this memory block is released */ + protected volatile boolean isReleased = false; + + /** The total memory size in byte of this memory block */ + protected long totalMemorySizeInBytes; + + /** + * Forcibly allocate memory without the limit of totalMemorySizeInBytes + * + * @param sizeInByte the size of memory to be allocated, should be positive + */ + public abstract void forceAllocateWithoutLimitation(final long sizeInByte); + + /** Get the memory usage in byte of this memory block */ + public abstract long getUsedMemoryInBytes(); + + /** Get the name of memory block */ + public String getName() { + return name; + } + + /** Update maximum memory size in byte of this memory block */ + public void setTotalMemorySizeInBytes(final long totalMemorySizeInBytes) { + this.totalMemorySizeInBytes = totalMemorySizeInBytes; + } + + /** Get the maximum memory size in byte of this memory block */ + public long getTotalMemorySizeInBytes() { + return totalMemorySizeInBytes; + } + + /** Get whether this memory block is released */ + public boolean isReleased() { + return isReleased; + } + + /** Mark this memory block as released */ + public void markAsReleased() { + isReleased = true; + } + + public void print(int indent) { + StringBuilder sb = new StringBuilder(); + for (int i = 0; i < indent; i++) { + sb.append(" "); + } + sb.append(this); + LOGGER.info(sb.toString()); + } +} diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/resource/memory/MemoryBlockType.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/resource/memory/MemoryBlockType.java new file mode 100644 index 00000000..66a3774c --- /dev/null +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/resource/memory/MemoryBlockType.java @@ -0,0 +1,28 @@ +/* + * 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.collector.plugin.builtin.sink.resource.memory; + +public enum MemoryBlockType { + NONE, + // static memory + STATIC, + // dynamic memory + DYNAMIC, +} diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/resource/memory/MemoryException.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/resource/memory/MemoryException.java new file mode 100644 index 00000000..da81f1fe --- /dev/null +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/resource/memory/MemoryException.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.collector.plugin.builtin.sink.resource.memory; + +import java.util.Objects; + +public class MemoryException extends RuntimeException { + + private final long timestamp; + + public MemoryException(final String message) { + super(message); + this.timestamp = System.currentTimeMillis(); + } + + public long getTimestamp() { + return timestamp; + } + + @Override + public boolean equals(Object obj) { + return obj instanceof MemoryException + && Objects.equals(getMessage(), ((MemoryException) obj).getMessage()) + && Objects.equals(getTimestamp(), ((MemoryException) obj).getTimestamp()); + } + + @Override + public int hashCode() { + return Objects.hash(getMessage(), getTimestamp()); + } + + @Override + public String toString() { + return "MemoryException{" + "message='" + getMessage() + "', timestamp=" + getTimestamp() + "}"; + } +} diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/resource/memory/MemoryManager.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/resource/memory/MemoryManager.java new file mode 100644 index 00000000..04ef3c45 --- /dev/null +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/resource/memory/MemoryManager.java @@ -0,0 +1,405 @@ +/* + * 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.collector.plugin.builtin.sink.resource.memory; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.function.LongUnaryOperator; + +public class MemoryManager { + private static final Logger LOGGER = LoggerFactory.getLogger(MemoryManager.class); + + /** The max retry times for memory allocation */ + private static final int MEMORY_ALLOCATE_MAX_RETRIES = 3; + + /** The retry interval for memory allocation */ + private static final long MEMORY_ALLOCATE_RETRY_INTERVAL_IN_MS = 1000; + + /** The min memory size to allocate */ + private static final long MEMORY_ALLOCATE_MIN_SIZE_IN_BYTES = 32; + + /** The name of memory manager */ + private final String name; + + /** Whether memory management is enabled */ + private final boolean enabled; + + /** The total memory size in byte of memory manager */ + private volatile long totalMemorySizeInBytes; + + /** The allocated memory size */ + private volatile long allocatedMemorySizeInBytes = 0L; + + /** The parent memory manager */ + private final MemoryManager parentMemoryManager; + + /** The child memory manager */ + private final Map children = new ConcurrentHashMap<>(); + + /** The allocated memory blocks of this memory manager */ + private final Map allocatedMemoryBlocks = new ConcurrentHashMap<>(); + + MemoryManager(String name, MemoryManager parentMemoryManager, long totalMemorySizeInBytes) { + this.name = name; + this.parentMemoryManager = parentMemoryManager; + this.totalMemorySizeInBytes = totalMemorySizeInBytes; + this.enabled = false; + } + + private MemoryManager( + String name, + MemoryManager parentMemoryManager, + long totalMemorySizeInBytes, + boolean enabled) { + this.name = name; + this.parentMemoryManager = parentMemoryManager; + this.totalMemorySizeInBytes = totalMemorySizeInBytes; + this.enabled = enabled; + } + + // region The Methods Of IMemoryBlock Management + + /** + * Allocate memory block with exact specified size in bytes + * + * @param name the name of memory block + * @param sizeInBytes the size in bytes of memory block try to allocate + * @param type the type of memory block + * @return the allocated memory block + * @throw MemoryException if fail to allocate after MEMORY_ALLOCATE_MAX_RETRIES retries + */ + public synchronized IMemoryBlock exactAllocate( + String name, long sizeInBytes, MemoryBlockType type) { + if (!enabled) { + return getOrRegisterMemoryBlock(name, sizeInBytes, type); + } + for (int i = 0; i < MEMORY_ALLOCATE_MAX_RETRIES; i++) { + if (totalMemorySizeInBytes - allocatedMemorySizeInBytes >= sizeInBytes) { + return getOrRegisterMemoryBlock(name, sizeInBytes, type); + } + + try { + this.wait(MEMORY_ALLOCATE_RETRY_INTERVAL_IN_MS); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + LOGGER.warn("exactAllocate: interrupted while waiting for available memory", e); + } + } + + throw new MemoryException( + String.format( + "exactAllocate: failed to allocate memory after %d retries, " + + "total memory size %d bytes, used memory size %d bytes, " + + "requested memory size %d bytes", + MEMORY_ALLOCATE_MAX_RETRIES, + totalMemorySizeInBytes, + allocatedMemorySizeInBytes, + sizeInBytes)); + } + + /** + * Try to allocate memory block with total memory size in bytes + * + * @param name the name of memory block + * @param memoryBlockType the type of memory block + */ + public synchronized IMemoryBlock exactAllocate(String name, MemoryBlockType memoryBlockType) { + return exactAllocate( + name, totalMemorySizeInBytes - allocatedMemorySizeInBytes, memoryBlockType); + } + + /** + * Try to allocate memory block with customAllocateStrategy + * + * @param name the name of memory block + * @param sizeInBytes the size in bytes of memory block try to allocate + * @param customAllocateStrategy the custom allocate strategy when memory is insufficient + * @param type the type of memory block + * @return the memory block if success, otherwise null + */ + public synchronized IMemoryBlock tryAllocate( + String name, + long sizeInBytes, + LongUnaryOperator customAllocateStrategy, + MemoryBlockType type) { + if (!enabled) { + return getOrRegisterMemoryBlock(name, sizeInBytes, type); + } + + if (totalMemorySizeInBytes - allocatedMemorySizeInBytes >= sizeInBytes) { + return getOrRegisterMemoryBlock(name, sizeInBytes, type); + } + + long sizeToAllocateInBytes = sizeInBytes; + while (sizeToAllocateInBytes >= MEMORY_ALLOCATE_MIN_SIZE_IN_BYTES) { + if (totalMemorySizeInBytes - allocatedMemorySizeInBytes >= sizeToAllocateInBytes) { + LOGGER.debug( + "tryAllocate: allocated memory, " + + "total memory size {} bytes, used memory size {} bytes, " + + "original requested memory size {} bytes, " + + "actual requested memory size {} bytes", + totalMemorySizeInBytes, + allocatedMemorySizeInBytes, + sizeInBytes, + sizeToAllocateInBytes); + return getOrRegisterMemoryBlock(name, sizeToAllocateInBytes, type); + } + + sizeToAllocateInBytes = + Math.max( + customAllocateStrategy.applyAsLong(sizeToAllocateInBytes), + MEMORY_ALLOCATE_MIN_SIZE_IN_BYTES); + } + + // TODO @spricoder: consider to find more memory in active way + LOGGER.warn( + "tryAllocate: failed to allocate memory, " + + "total memory size {} bytes, used memory size {} bytes, " + + "requested memory size {} bytes", + totalMemorySizeInBytes, + allocatedMemorySizeInBytes, + sizeInBytes); + return null; + } + + /** + * Try to register memory block with specified size in bytes + * + * @param name the name of memory block, UNIQUE + * @param sizeInBytes the size in bytes of memory block + * @param type the type of memory block + * @return the memory block + */ + private IMemoryBlock getOrRegisterMemoryBlock( + String name, long sizeInBytes, MemoryBlockType type) { + if (sizeInBytes < 0) { + throw new MemoryException( + String.format( + "register memory block %s failed: sizeInBytes should be non-negative", name)); + } + return allocatedMemoryBlocks.compute( + name, + (blockName, block) -> { + if (block != null) { + if (block.getTotalMemorySizeInBytes() != sizeInBytes) { + LOGGER.warn( + "getOrRegisterMemoryBlock failed: memory block {} already exists, " + + "it's size is {}, requested size is {}", + blockName, + block.getTotalMemorySizeInBytes(), + sizeInBytes); + } + return block; + } else { + allocatedMemorySizeInBytes += sizeInBytes; + return new AtomicLongMemoryBlock(name, this, sizeInBytes, type); + } + }); + } + + /** + * Release memory block and notify all waiting threads + * + * @param block the memory block to release + */ + public synchronized void release(IMemoryBlock block) { + if (block == null || block.isReleased()) { + return; + } + releaseWithOutNotify(block); + this.notifyAll(); + } + + /** + * Release memory block without notify + * + * @param block the memory block to release + */ + public synchronized void releaseWithOutNotify(IMemoryBlock block) { + if (block == null || block.isReleased()) { + return; + } + + block.markAsReleased(); + allocatedMemorySizeInBytes -= block.getTotalMemorySizeInBytes(); + allocatedMemoryBlocks.remove(block.getName()); + try { + block.close(); + } catch (Exception e) { + LOGGER.error("releaseWithOutNotify: failed to close memory block {}", block, e); + } + } + + // endregion + + // region The Methods Of MemoryManager Management + + /** + * Try to create a new memory manager with specified name and total memory size in bytes, then put + * it into children map. NOTICE: if there are same name memory manager, it will return the + * existing one instead of creating a new one or update the existing one. + * + * @param name the name of memory manager + * @param sizeInBytes the total memory size in bytes of memory manager + * @param enabled whether memory management is enabled + * @return the memory manager + */ + public synchronized MemoryManager getOrCreateMemoryManager( + String name, long sizeInBytes, boolean enabled) { + return children.compute( + name, + (managerName, manager) -> { + if (sizeInBytes < 0) { + LOGGER.warn("getOrCreateMemoryManager {}: sizeInBytes should be positive", name); + return null; + } + if (manager != null) { + LOGGER.debug( + "getMemoryManager: memory manager {} already exists, it's size is {}, enabled is {}", + managerName, + manager.getTotalMemorySizeInBytes(), + manager.isEnable()); + return manager; + } else { + if (this.enabled + && sizeInBytes + this.allocatedMemorySizeInBytes > this.totalMemorySizeInBytes) { + LOGGER.warn( + "getOrCreateMemoryManager failed: total memory size {} bytes is less than allocated memory size {} bytes", + sizeInBytes, + allocatedMemorySizeInBytes); + return null; + } + allocatedMemorySizeInBytes += sizeInBytes; + return new MemoryManager(name, this, sizeInBytes, enabled); + } + }); + } + + /** + * Try to create a new memory manager with specified name and total memory size in bytes, then put + * it into children map. NOTICE: if there are same name memory manager, it will return the + * existing one. + * + * @param name the name of memory manager + * @param totalMemorySizeInBytes the total memory size in bytes of memory manager + * @return the memory manager + */ + public synchronized MemoryManager getOrCreateMemoryManager( + String name, long totalMemorySizeInBytes) { + return getOrCreateMemoryManager(name, totalMemorySizeInBytes, false); + } + + /** + * Re-allocate memory according to ratio + * + * @param ratio the ratio of new total memory size to old total memory size + */ + private void reAllocateMemoryAccordingToRatio(double ratio) { + // first increase the total memory size of this memory manager + this.totalMemorySizeInBytes *= ratio; + // then re-allocate memory for all memory blocks + for (IMemoryBlock block : allocatedMemoryBlocks.values()) { + block.setTotalMemorySizeInBytes((long) (block.getTotalMemorySizeInBytes() * ratio)); + } + // finally re-allocate memory for all child memory managers + for (Map.Entry entry : children.entrySet()) { + entry.getValue().reAllocateMemoryAccordingToRatio(ratio); + } + } + + public String getName() { + return name; + } + + public boolean isEnable() { + return enabled; + } + + /** Get total memory size in bytes of memory manager */ + public long getTotalMemorySizeInBytes() { + return totalMemorySizeInBytes; + } + + public void setTotalMemorySizeInBytes(long totalMemorySizeInBytes) { + this.totalMemorySizeInBytes = totalMemorySizeInBytes; + } + + public void setTotalMemorySizeInBytesWithReload(long totalMemorySizeInBytes) { + reAllocateMemoryAccordingToRatio((double) totalMemorySizeInBytes / this.totalMemorySizeInBytes); + } + + /** Get available memory size in bytes of memory manager */ + public long getAvailableMemorySizeInBytes() { + return totalMemorySizeInBytes - allocatedMemorySizeInBytes; + } + + /** Get allocated memory size in bytes of memory manager */ + public long getAllocatedMemorySizeInBytes() { + return allocatedMemorySizeInBytes; + } + + /** Get actual used memory size in bytes of memory manager */ + public long getUsedMemorySizeInBytes() { + long memorySize = + allocatedMemoryBlocks.values().stream().mapToLong(IMemoryBlock::getUsedMemoryInBytes).sum(); + for (MemoryManager child : children.values()) { + memorySize += child.getUsedMemorySizeInBytes(); + } + return memorySize; + } + + // endregion + + @Override + public String toString() { + return "MemoryManager{" + + "name=" + + name + + ", enabled=" + + enabled + + ", totalMemorySizeInBytes=" + + totalMemorySizeInBytes + + ", allocatedMemorySizeInBytes=" + + allocatedMemorySizeInBytes + + '}'; + } + + public void print() { + print(0); + } + + private void print(int indent) { + StringBuilder sb = new StringBuilder(); + for (int i = 0; i < indent; i++) { + sb.append(" "); + } + sb.append(this); + LOGGER.info(sb.toString()); + for (IMemoryBlock block : allocatedMemoryBlocks.values()) { + block.print(indent + 2); + } + for (MemoryManager child : children.values()) { + child.print(indent + 1); + } + } +} diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/resource/memory/PipeFixedMemoryBlock.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/resource/memory/PipeFixedMemoryBlock.java new file mode 100644 index 00000000..45064b54 --- /dev/null +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/resource/memory/PipeFixedMemoryBlock.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.collector.plugin.builtin.sink.resource.memory; + +import java.util.function.BiConsumer; +import java.util.function.LongUnaryOperator; + +public abstract class PipeFixedMemoryBlock extends PipeMemoryBlock { + + public PipeFixedMemoryBlock(long memoryUsageInBytes) { + super(memoryUsageInBytes); + } + + @Override + boolean shrink() { + return false; + } + + @Override + boolean expand() { + return false; + } + + @Override + public PipeMemoryBlock setShrinkMethod(LongUnaryOperator shrinkMethod) { + throw new UnsupportedOperationException( + "Shrink method is not supported in PipeFixedMemoryBlock"); + } + + @Override + public PipeMemoryBlock setShrinkCallback(BiConsumer shrinkCallback) { + throw new UnsupportedOperationException( + "Shrink callback is not supported in PipeFixedMemoryBlock"); + } + + @Override + public PipeMemoryBlock setExpandMethod(LongUnaryOperator extendMethod) { + throw new UnsupportedOperationException( + "Expand method is not supported in PipeFixedMemoryBlock"); + } + + @Override + public PipeMemoryBlock setExpandCallback(BiConsumer expandCallback) { + throw new UnsupportedOperationException( + "Expand callback is not supported in PipeFixedMemoryBlock"); + } +} diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/resource/memory/PipeMemoryBlock.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/resource/memory/PipeMemoryBlock.java new file mode 100644 index 00000000..e8a9ac9a --- /dev/null +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/resource/memory/PipeMemoryBlock.java @@ -0,0 +1,175 @@ +/* + * 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.collector.plugin.builtin.sink.resource.memory; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.locks.ReentrantLock; +import java.util.function.BiConsumer; +import java.util.function.LongUnaryOperator; + +public class PipeMemoryBlock implements AutoCloseable { + + private static final Logger LOGGER = LoggerFactory.getLogger(PipeMemoryBlock.class); + + private final ReentrantLock lock = new ReentrantLock(); + + private final AtomicLong memoryUsageInBytes = new AtomicLong(0); + + private final AtomicReference shrinkMethod = new AtomicReference<>(); + private final AtomicReference> shrinkCallback = new AtomicReference<>(); + private final AtomicReference expandMethod = new AtomicReference<>(); + private final AtomicReference> expandCallback = new AtomicReference<>(); + + public PipeMemoryBlock(final long memoryUsageInBytes) { + this.memoryUsageInBytes.set(memoryUsageInBytes); + } + + public long getMemoryUsageInBytes() { + return memoryUsageInBytes.get(); + } + + public PipeMemoryBlock setShrinkMethod(final LongUnaryOperator shrinkMethod) { + this.shrinkMethod.set(shrinkMethod); + return this; + } + + public PipeMemoryBlock setShrinkCallback(final BiConsumer shrinkCallback) { + this.shrinkCallback.set(shrinkCallback); + return this; + } + + public PipeMemoryBlock setExpandMethod(final LongUnaryOperator extendMethod) { + this.expandMethod.set(extendMethod); + return this; + } + + public PipeMemoryBlock setExpandCallback(final BiConsumer expandCallback) { + this.expandCallback.set(expandCallback); + return this; + } + + boolean shrink() { + if (lock.tryLock()) { + try { + return doShrink(); + } finally { + lock.unlock(); + } + } + return false; + } + + private boolean doShrink() { + if (shrinkMethod.get() == null) { + return false; + } + + final long oldMemorySizeInBytes = memoryUsageInBytes.get(); + final long newMemorySizeInBytes = shrinkMethod.get().applyAsLong(memoryUsageInBytes.get()); + + final long memoryInBytesCanBeReleased = oldMemorySizeInBytes - newMemorySizeInBytes; + if (memoryInBytesCanBeReleased <= 0) { + return false; + } + + if (shrinkCallback.get() != null) { + try { + shrinkCallback.get().accept(oldMemorySizeInBytes, newMemorySizeInBytes); + } catch (Exception e) { + LOGGER.warn("Failed to execute the shrink callback.", e); + } + } + return true; + } + + boolean expand() { + if (lock.tryLock()) { + try { + return doExpand(); + } finally { + lock.unlock(); + } + } + return false; + } + + private boolean doExpand() { + if (expandMethod.get() == null) { + return false; + } + + final long oldMemorySizeInBytes = memoryUsageInBytes.get(); + final long newMemorySizeInBytes = expandMethod.get().applyAsLong(memoryUsageInBytes.get()); + + final long memoryInBytesNeededToBeAllocated = newMemorySizeInBytes - oldMemorySizeInBytes; + if (memoryInBytesNeededToBeAllocated <= 0) { + return false; + } + + if (expandCallback.get() != null) { + try { + expandCallback.get().accept(oldMemorySizeInBytes, newMemorySizeInBytes); + } catch (Exception e) { + LOGGER.warn("Failed to execute the expand callback.", e); + } + } + return true; + } + + @Override + public String toString() { + return "PipeMemoryBlock{" + "usedMemoryInBytes=" + memoryUsageInBytes.get() + '}'; + } + + @Override + public void close() { + boolean isInterrupted = false; + + while (true) { + try { + if (lock.tryLock(50, TimeUnit.MICROSECONDS)) { + try { + if (isInterrupted) { + LOGGER.warn("{} is released after thread interruption.", this); + } + break; + } finally { + lock.unlock(); + } + } + } catch (final InterruptedException e) { + // Each time the close task is run, it means that the interrupt status left by the previous + // tryLock does not need to be retained. Otherwise, it will lead to an infinite loop. + isInterrupted = true; + LOGGER.warn("Interrupted while waiting for the lock.", e); + } + } + + // Restore the interrupt status of the current thread + if (isInterrupted) { + Thread.currentThread().interrupt(); + } + } +} diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/resource/memory/PipeMemoryBlockType.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/resource/memory/PipeMemoryBlockType.java new file mode 100644 index 00000000..ae0ec315 --- /dev/null +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/resource/memory/PipeMemoryBlockType.java @@ -0,0 +1,26 @@ +/* + * 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.collector.plugin.builtin.sink.resource.memory; + +public enum PipeMemoryBlockType { + NORMAL, + TABLET, + TS_FILE, +} diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/resource/memory/PipeMemoryManager.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/resource/memory/PipeMemoryManager.java new file mode 100644 index 00000000..d4309329 --- /dev/null +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/resource/memory/PipeMemoryManager.java @@ -0,0 +1,163 @@ +/* + * 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.collector.plugin.builtin.sink.resource.memory; + +import org.apache.iotdb.collector.config.PipeRuntimeOptions; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.function.LongUnaryOperator; + +public class PipeMemoryManager { + + private static final Logger LOGGER = LoggerFactory.getLogger(PipeMemoryManager.class); + + private static final boolean PIPE_MEMORY_MANAGEMENT_ENABLED = + PipeRuntimeOptions.PIPE_MEMORY_MANAGEMENT_ENABLED.value(); + + private static final long MEMORY_ALLOCATE_MIN_SIZE_IN_BYTES = + PipeRuntimeOptions.PIPE_MEMORY_ALLOCATE_MIN_SIZE_IN_BYTES.value(); + + private final IMemoryBlock memoryBlock = + new MemoryManager("pipeMemoryManager", null, Runtime.getRuntime().totalMemory() / 10) + .exactAllocate("Stream", MemoryBlockType.DYNAMIC); + + private static final double FLOATING_MEMORY_RATIO = + PipeRuntimeOptions.PIPE_TOTAL_FLOATING_MEMORY_PROPORTION.value(); + + // Only non-zero memory blocks will be added to this set. + private final Set allocatedBlocks = new HashSet<>(); + + private PipeMemoryManager() {} + + public synchronized PipeMemoryBlock tryAllocate(long sizeInBytes) { + return tryAllocate(sizeInBytes, currentSize -> currentSize * 2 / 3); + } + + public synchronized PipeMemoryBlock tryAllocate( + long sizeInBytes, LongUnaryOperator customAllocateStrategy) { + if (!PIPE_MEMORY_MANAGEMENT_ENABLED) { + return new PipeMemoryBlock(sizeInBytes); + } + + if (sizeInBytes == 0 + || getTotalNonFloatingMemorySizeInBytes() - memoryBlock.getUsedMemoryInBytes() + >= sizeInBytes) { + return registerMemoryBlock(sizeInBytes); + } + + long sizeToAllocateInBytes = sizeInBytes; + while (sizeToAllocateInBytes > MEMORY_ALLOCATE_MIN_SIZE_IN_BYTES) { + if (getTotalNonFloatingMemorySizeInBytes() - memoryBlock.getUsedMemoryInBytes() + >= sizeToAllocateInBytes) { + LOGGER.info( + "tryAllocate: allocated memory, " + + "total memory size {} bytes, used memory size {} bytes, " + + "original requested memory size {} bytes, " + + "actual requested memory size {} bytes", + getTotalNonFloatingMemorySizeInBytes(), + memoryBlock.getUsedMemoryInBytes(), + sizeInBytes, + sizeToAllocateInBytes); + return registerMemoryBlock(sizeToAllocateInBytes); + } + + sizeToAllocateInBytes = + Math.max( + customAllocateStrategy.applyAsLong(sizeToAllocateInBytes), + MEMORY_ALLOCATE_MIN_SIZE_IN_BYTES); + } + + if (tryShrinkUntilFreeMemorySatisfy(sizeToAllocateInBytes)) { + LOGGER.info( + "tryAllocate: allocated memory, " + + "total memory size {} bytes, used memory size {} bytes, " + + "original requested memory size {} bytes, " + + "actual requested memory size {} bytes", + getTotalNonFloatingMemorySizeInBytes(), + memoryBlock.getUsedMemoryInBytes(), + sizeInBytes, + sizeToAllocateInBytes); + return registerMemoryBlock(sizeToAllocateInBytes); + } else { + LOGGER.warn( + "tryAllocate: failed to allocate memory, " + + "total memory size {} bytes, used memory size {} bytes, " + + "requested memory size {} bytes", + getTotalNonFloatingMemorySizeInBytes(), + memoryBlock.getUsedMemoryInBytes(), + sizeInBytes); + return registerMemoryBlock(0); + } + } + + private PipeMemoryBlock registerMemoryBlock(long sizeInBytes) { + // For memory block whose size is 0, we do not need to add it to the allocated blocks now. + // It's good for performance and will not trigger concurrent issues. + // If forceResize is called on it, we will add it to the allocated blocks. + final PipeMemoryBlock returnedMemoryBlock = new PipeMemoryBlock(sizeInBytes); + + if (sizeInBytes > 0) { + memoryBlock.forceAllocateWithoutLimitation(sizeInBytes); + allocatedBlocks.add(returnedMemoryBlock); + } + + return returnedMemoryBlock; + } + + private boolean tryShrinkUntilFreeMemorySatisfy(long sizeInBytes) { + final List shuffledBlocks = new ArrayList<>(allocatedBlocks); + Collections.shuffle(shuffledBlocks); + + while (true) { + boolean hasAtLeastOneBlockShrinkable = false; + for (final PipeMemoryBlock block : shuffledBlocks) { + if (block.shrink()) { + hasAtLeastOneBlockShrinkable = true; + if (getTotalNonFloatingMemorySizeInBytes() - memoryBlock.getUsedMemoryInBytes() + >= sizeInBytes) { + return true; + } + } + } + if (!hasAtLeastOneBlockShrinkable) { + return false; + } + } + } + + public long getTotalNonFloatingMemorySizeInBytes() { + return (long) (memoryBlock.getTotalMemorySizeInBytes() * (1 - FLOATING_MEMORY_RATIO)); + } + + public static PipeMemoryManager getInstance() { + return PipeMemoryManagerHolder.INSTANCE; + } + + private static class PipeMemoryManagerHolder { + private static final PipeMemoryManager INSTANCE = new PipeMemoryManager(); + } +} diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/resource/memory/PipeTabletMemoryBlock.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/resource/memory/PipeTabletMemoryBlock.java new file mode 100644 index 00000000..d0dc30da --- /dev/null +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/resource/memory/PipeTabletMemoryBlock.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.collector.plugin.builtin.sink.resource.memory; + +public class PipeTabletMemoryBlock extends PipeFixedMemoryBlock { + + public PipeTabletMemoryBlock(long memoryUsageInBytes) { + super(memoryUsageInBytes); + } +} diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/resource/memory/PipeTsFileMemoryBlock.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/resource/memory/PipeTsFileMemoryBlock.java new file mode 100644 index 00000000..5b6c44f5 --- /dev/null +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/resource/memory/PipeTsFileMemoryBlock.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.collector.plugin.builtin.sink.resource.memory; + +public class PipeTsFileMemoryBlock extends PipeFixedMemoryBlock { + + public PipeTsFileMemoryBlock(long memoryUsageInBytes) { + super(memoryUsageInBytes); + } +} diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/constant/SourceConstant.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/constant/SourceConstant.java index 62423c66..a7e2503e 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/constant/SourceConstant.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/constant/SourceConstant.java @@ -27,7 +27,7 @@ public class SourceConstant { public static final String SOURCE_REPORT_TIME_INTERVAL_KEY = "report-time-interval"; - public static final int SOURCE_REPORT_TIME_INTERVAL_DEFAULT_VALUE = 60; + public static final int SOURCE_REPORT_TIME_INTERVAL_DEFAULT_VALUE = 5; public static final String SOURCE_IS_ALIGNED_KEY = "is-aligned"; public static final boolean SOURCE_IS_ALIGNED_DEFAULT_VALUE = false; diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionCommon.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionCommon.java index 5534c61b..473d65bb 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionCommon.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionCommon.java @@ -24,11 +24,9 @@ import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameterValidator; import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameters; import org.apache.iotdb.pipe.api.event.Event; -import org.apache.iotdb.session.subscription.consumer.base.AbstractSubscriptionConsumerBuilder; +import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; -import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.locks.LockSupport; import static org.apache.iotdb.collector.plugin.builtin.source.constant.SourceConstant.SOURCE_IS_ALIGNED_DEFAULT_VALUE; import static org.apache.iotdb.collector.plugin.builtin.source.constant.SourceConstant.SOURCE_IS_ALIGNED_KEY; @@ -70,9 +68,8 @@ public class IoTDBSubscriptionCommon { private String topic; - private final BlockingQueue eventQueue = new LinkedBlockingQueue<>(); private static final Integer EVENT_QUEUE_CAPACITY = 1000; - private static final Long EVENT_QUEUE_PAUSE_INTERVAL_MS = 100_000_000L; + private final BlockingQueue eventQueue = new ArrayBlockingQueue<>(EVENT_QUEUE_CAPACITY); // validate common parameters public void validate(final PipeParameterValidator validator) { @@ -156,19 +153,6 @@ public void customize( IOTDB_SUBSCRIPTION_SOURCE_MAX_POLL_PARALLELISM_DEFAULT_VALUE); } - // common consumer builder - public AbstractSubscriptionConsumerBuilder getSubscriptionConsumerBuilder() { - return new AbstractSubscriptionConsumerBuilder() - .host(host) - .port(port) - .consumerId(consumerId) - .consumerGroupId(groupId) - .heartbeatIntervalMs(heartbeatIntervalMs) - .endpointsSyncIntervalMs(endpointsSyncIntervalMs) - .thriftMaxFrameSize(thriftMaxFrameSize) - .maxPollParallelism(maxPollParallelism); - } - public Event take() throws InterruptedException { return eventQueue.take(); } @@ -177,10 +161,36 @@ public void put(final Event event) throws InterruptedException { eventQueue.put(event); } - public void checkIfNeedPause() { - while (eventQueue.size() >= EVENT_QUEUE_CAPACITY) { - LockSupport.parkNanos(EVENT_QUEUE_PAUSE_INTERVAL_MS); - } + public String getHost() { + return host; + } + + public Integer getPort() { + return port; + } + + public String getConsumerId() { + return consumerId; + } + + public String getGroupId() { + return groupId; + } + + public Long getHeartbeatIntervalMs() { + return heartbeatIntervalMs; + } + + public Long getEndpointsSyncIntervalMs() { + return endpointsSyncIntervalMs; + } + + public Integer getThriftMaxFrameSize() { + return thriftMaxFrameSize; + } + + public Integer getMaxPollParallelism() { + return maxPollParallelism; } public String getTopic() { diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionPullSource.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionPullSource.java index 4cfa7b22..aac66387 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionPullSource.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionPullSource.java @@ -26,7 +26,8 @@ import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameterValidator; import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameters; import org.apache.iotdb.pipe.api.event.Event; -import org.apache.iotdb.session.subscription.consumer.base.AbstractSubscriptionPullConsumerBuilder; +import org.apache.iotdb.session.subscription.consumer.table.SubscriptionTablePullConsumerBuilder; +import org.apache.iotdb.session.subscription.consumer.tree.SubscriptionTreePullConsumerBuilder; import org.apache.iotdb.session.subscription.payload.SubscriptionMessage; import org.apache.iotdb.session.subscription.payload.SubscriptionSessionDataSet; @@ -112,8 +113,6 @@ private void doWork() { for (final SubscriptionMessage message : messages) { for (final SubscriptionSessionDataSet dataSet : message.getSessionDataSetsHandler()) { - subscription.checkIfNeedPause(); - try { subscription.put(new PipeRawTabletInsertionEvent(dataSet.getTablet(), isAligned)); } catch (final InterruptedException e) { @@ -146,8 +145,30 @@ public void close() throws Exception { } } - protected AbstractSubscriptionPullConsumerBuilder getSubscriptionPullConsumerBuilder() { - return ((AbstractSubscriptionPullConsumerBuilder) subscription.getSubscriptionConsumerBuilder()) + protected SubscriptionTreePullConsumerBuilder getSubscriptionTreePullConsumerBuilder() { + return new SubscriptionTreePullConsumerBuilder() + .host(subscription.getHost()) + .port(subscription.getPort()) + .consumerId(subscription.getConsumerId()) + .consumerGroupId(subscription.getGroupId()) + .heartbeatIntervalMs(subscription.getHeartbeatIntervalMs()) + .endpointsSyncIntervalMs(subscription.getEndpointsSyncIntervalMs()) + .thriftMaxFrameSize(subscription.getThriftMaxFrameSize()) + .maxPollParallelism(subscription.getMaxPollParallelism()) + .autoCommit(autoCommit) + .autoCommitIntervalMs(autoCommitIntervalMs); + } + + protected SubscriptionTablePullConsumerBuilder getSubscriptionTablePullConsumerBuilder() { + return new SubscriptionTablePullConsumerBuilder() + .host(subscription.getHost()) + .port(subscription.getPort()) + .consumerId(subscription.getConsumerId()) + .consumerGroupId(subscription.getGroupId()) + .heartbeatIntervalMs(subscription.getHeartbeatIntervalMs()) + .endpointsSyncIntervalMs(subscription.getEndpointsSyncIntervalMs()) + .thriftMaxFrameSize(subscription.getThriftMaxFrameSize()) + .maxPollParallelism(subscription.getMaxPollParallelism()) .autoCommit(autoCommit) .autoCommitIntervalMs(autoCommitIntervalMs); } diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionPushSource.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionPushSource.java index c41acaf8..ef358a05 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionPushSource.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionPushSource.java @@ -26,13 +26,17 @@ import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameterValidator; import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameters; import org.apache.iotdb.session.subscription.consumer.AckStrategy; +import org.apache.iotdb.session.subscription.consumer.ConsumeListener; import org.apache.iotdb.session.subscription.consumer.ConsumeResult; -import org.apache.iotdb.session.subscription.consumer.base.AbstractSubscriptionPushConsumerBuilder; +import org.apache.iotdb.session.subscription.consumer.table.SubscriptionTablePushConsumerBuilder; +import org.apache.iotdb.session.subscription.consumer.tree.SubscriptionTreePushConsumerBuilder; import org.apache.iotdb.session.subscription.payload.SubscriptionSessionDataSet; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.concurrent.locks.LockSupport; + import static org.apache.iotdb.collector.plugin.builtin.source.iotdb.IoTDBSubscriptionSourceConstant.IOTDB_SUBSCRIPTION_SOURCE_ACK_STRATEGY_DEFAULT_VALUE; import static org.apache.iotdb.collector.plugin.builtin.source.iotdb.IoTDBSubscriptionSourceConstant.IOTDB_SUBSCRIPTION_SOURCE_ACK_STRATEGY_KEY; import static org.apache.iotdb.collector.plugin.builtin.source.iotdb.IoTDBSubscriptionSourceConstant.IOTDB_SUBSCRIPTION_SOURCE_ACK_STRATEGY_VALUE_MAP; @@ -117,26 +121,56 @@ public void start() throws Exception { protected abstract String getPushConsumerThreadName(); - protected AbstractSubscriptionPushConsumerBuilder getPushConsumerBuilder() { - return getSubscriptionPushConsumerBuilder() - .consumeListener( - message -> { - for (final SubscriptionSessionDataSet dataSet : message.getSessionDataSetsHandler()) { - try { - subscription.put(new PipeRawTabletInsertionEvent(dataSet.getTablet(), isStarted)); - } catch (final InterruptedException e) { - LOGGER.warn("{} thread interrupted", getPushConsumerThreadName(), e); - Thread.currentThread().interrupt(); - - return ConsumeResult.FAILURE; - } - } - return ConsumeResult.SUCCESS; - }); + protected ConsumeListener getConsumeListener() { + return message -> { + for (final SubscriptionSessionDataSet dataSet : message.getSessionDataSetsHandler()) { + try { + markPausePosition(); + + subscription.put(new PipeRawTabletInsertionEvent(dataSet.getTablet(), isAligned)); + } catch (final InterruptedException e) { + LOGGER.warn("{} thread interrupted", getPushConsumerThreadName(), e); + Thread.currentThread().interrupt(); + + return ConsumeResult.FAILURE; + } + } + + while (isStarted && !Thread.currentThread().isInterrupted()) { + LockSupport.park(); + } + + return ConsumeResult.SUCCESS; + }; + } + + protected SubscriptionTreePushConsumerBuilder getSubscriptionTreePushConsumerBuilder() { + return new SubscriptionTreePushConsumerBuilder() + .host(subscription.getHost()) + .port(subscription.getPort()) + .consumerId(subscription.getConsumerId() + instanceIndex) + .consumerGroupId(subscription.getGroupId()) + .heartbeatIntervalMs(subscription.getHeartbeatIntervalMs()) + .endpointsSyncIntervalMs(subscription.getEndpointsSyncIntervalMs()) + .thriftMaxFrameSize(subscription.getThriftMaxFrameSize()) + .maxPollParallelism(subscription.getMaxPollParallelism()) + .consumeListener(getConsumeListener()) + .ackStrategy(ackStrategy) + .autoPollIntervalMs(autoPollIntervalMs) + .autoPollTimeoutMs(autoPollTimeoutMs); } - protected AbstractSubscriptionPushConsumerBuilder getSubscriptionPushConsumerBuilder() { - return ((AbstractSubscriptionPushConsumerBuilder) subscription.getSubscriptionConsumerBuilder()) + protected SubscriptionTablePushConsumerBuilder getSubscriptionTablePushConsumer() { + return new SubscriptionTablePushConsumerBuilder() + .host(subscription.getHost()) + .port(subscription.getPort()) + .consumerId(subscription.getConsumerId() + instanceIndex) + .consumerGroupId(subscription.getGroupId()) + .heartbeatIntervalMs(subscription.getHeartbeatIntervalMs()) + .endpointsSyncIntervalMs(subscription.getEndpointsSyncIntervalMs()) + .thriftMaxFrameSize(subscription.getThriftMaxFrameSize()) + .maxPollParallelism(subscription.getMaxPollParallelism()) + .consumeListener(getConsumeListener()) .ackStrategy(ackStrategy) .autoPollIntervalMs(autoPollIntervalMs) .autoPollTimeoutMs(autoPollTimeoutMs); diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionSourceConstant.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionSourceConstant.java index ac91175d..a5a3b1df 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionSourceConstant.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionSourceConstant.java @@ -60,7 +60,7 @@ public class IoTDBSubscriptionSourceConstant { public static final int IOTDB_SUBSCRIPTION_SOURCE_MAX_POLL_PARALLELISM_DEFAULT_VALUE = 1; public static final String IOTDB_SUBSCRIPTION_SOURCE_AUTO_COMMIT_KEY = "auto-commit"; - public static final boolean IOTDB_SUBSCRIPTION_SOURCE_AUTO_COMMIT_DEFAULT_VALUE = true; + public static final boolean IOTDB_SUBSCRIPTION_SOURCE_AUTO_COMMIT_DEFAULT_VALUE = false; public static final String IOTDB_SUBSCRIPTION_SOURCE_AUTO_COMMIT_INTERVAL_MS_KEY = "auto-commit-interval-ms"; diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTablePullSource.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTablePullSource.java index f61b5728..6d0864f0 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTablePullSource.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTablePullSource.java @@ -32,7 +32,7 @@ public class IoTDBSubscriptionTablePullSource extends IoTDBSubscriptionPullSourc @Override protected void initPullConsumer() { - consumer = getSubscriptionPullConsumerBuilder().buildTablePullConsumer(); + consumer = getSubscriptionTablePullConsumerBuilder().buildTablePullConsumer(); consumer.open(); consumer.subscribe(subscription.getTopic()); } diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTablePushSource.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTablePushSource.java index 76785c0a..61955fe4 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTablePushSource.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTablePushSource.java @@ -34,7 +34,7 @@ public class IoTDBSubscriptionTablePushSource extends IoTDBSubscriptionPushSourc protected void doWork() { try (final ISubscriptionTablePushConsumer consumer = - getPushConsumerBuilder().buildTablePushConsumer()) { + getSubscriptionTablePushConsumer().buildTablePushConsumer()) { consumer.open(); consumer.subscribe(subscription.getTopic()); } catch (final Exception e) { diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTreePullSource.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTreePullSource.java index 8c168d36..1c2130bd 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTreePullSource.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTreePullSource.java @@ -23,16 +23,20 @@ import org.apache.iotdb.session.subscription.consumer.ISubscriptionTreePullConsumer; import org.apache.iotdb.session.subscription.payload.SubscriptionMessage; +import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.Optional; +import static org.apache.iotdb.collector.plugin.builtin.source.constant.SourceConstant.SOURCE_REPORT_TIME_INTERVAL_KEY; + public class IoTDBSubscriptionTreePullSource extends IoTDBSubscriptionPullSource { private ISubscriptionTreePullConsumer consumer; @Override protected void initPullConsumer() { - consumer = getSubscriptionPullConsumerBuilder().buildPullConsumer(); + consumer = getSubscriptionTreePullConsumerBuilder().buildPullConsumer(); consumer.open(); consumer.subscribe(subscription.getTopic()); } @@ -44,11 +48,19 @@ protected String getPullConsumerThreadName() { @Override protected List poll() { - return consumer.poll(POLL_TIMEOUT_MS); + return consumer.poll(Collections.singleton(subscription.getTopic()), POLL_TIMEOUT_MS); } @Override public Optional report() { - return Optional.empty(); + final ProgressIndex progress = + new ProgressIndex( + instanceIndex, + new HashMap() { + { + put(SOURCE_REPORT_TIME_INTERVAL_KEY, String.valueOf(reportTimeInterval)); + } + }); + return Optional.of(progress); } } diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTreePushSource.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTreePushSource.java index 5af6c27e..27deb13e 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTreePushSource.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTreePushSource.java @@ -34,7 +34,7 @@ public class IoTDBSubscriptionTreePushSource extends IoTDBSubscriptionPushSource protected void doWork() { try (final ISubscriptionTreePushConsumer consumer = - getPushConsumerBuilder().buildPushConsumer()) { + getSubscriptionTreePushConsumerBuilder().buildPushConsumer()) { consumer.open(); consumer.subscribe(subscription.getTopic()); } catch (final Exception e) { diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/kafka/KafkaSourceConstant.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/kafka/KafkaSourceConstant.java index d4655d48..ff9a857c 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/kafka/KafkaSourceConstant.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/kafka/KafkaSourceConstant.java @@ -50,7 +50,7 @@ public class KafkaSourceConstant { public static final String KAFKA_SOURCE_AUTO_OFFSET_RESET_NONE_VALUE = "none"; public static final String KAFKA_SOURCE_AUTO_OFFSET_RESET_EARLIEST_VALUE = "earliest"; public static final String KAFKA_SOURCE_AUTO_OFFSET_RESET_LATEST_VALUE = "latest"; - public static final String KAFKA_SOURCE_AUTO_OFFSET_RESET_DEFAULT_VALUE = "none"; + public static final String KAFKA_SOURCE_AUTO_OFFSET_RESET_DEFAULT_VALUE = "earliest"; public static final Set KAFKA_SOURCE_AUTO_OFFSET_RESET_VALUE_SET = Collections.unmodifiableSet( new HashSet<>( diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/runtime/task/TaskRuntime.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/runtime/task/TaskRuntime.java index 0dd1d355..f82ee544 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/runtime/task/TaskRuntime.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/runtime/task/TaskRuntime.java @@ -19,7 +19,6 @@ package org.apache.iotdb.collector.runtime.task; -import org.apache.iotdb.collector.plugin.api.customizer.CollectorParameters; import org.apache.iotdb.collector.runtime.task.event.ProgressReportEvent; import org.apache.iotdb.collector.runtime.task.processor.ProcessorTask; import org.apache.iotdb.collector.runtime.task.sink.SinkTask; @@ -33,7 +32,6 @@ import javax.ws.rs.core.Response; import java.util.HashMap; -import java.util.Iterator; import java.util.Map; import java.util.Objects; import java.util.concurrent.ConcurrentHashMap; @@ -72,12 +70,11 @@ public synchronized Response createTask( .orElseGet(HashMap::new) : new HashMap<>()))); - final SinkTask sinkTask = new SinkTask(taskId, convert(sinkAttribute)); + final SinkTask sinkTask = new SinkTask(taskId, sinkAttribute); final ProcessorTask processorTask = - new ProcessorTask(taskId, convert(processorAttribute), sinkTask.makeProducer()); + new ProcessorTask(taskId, processorAttribute, sinkTask.makeProducer()); final SourceTask sourceTask = - SourceTask.construct( - taskId, convert(sourceAttribute), processorTask.makeProducer(), taskState); + SourceTask.construct(taskId, sourceAttribute, processorTask.makeProducer(), taskState); final TaskCombiner taskCombiner = new TaskCombiner(sourceTask, processorTask, sinkTask); taskCombiner.create(); @@ -169,17 +166,20 @@ public synchronized Response stopTask(final String taskId) { } public synchronized Response dropTask(final String taskId) { - if (Objects.isNull(tasks.get(taskId)) || !tasks.containsKey(taskId)) { - return Response.status(Response.Status.NOT_FOUND) - .entity(String.format("task %s not found", taskId)) - .build(); + if (Objects.nonNull(tasks.get(taskId)) && tasks.containsKey(taskId)) { + try { + final TaskCombiner task = tasks.get(taskId); + task.drop(); + tasks.remove(taskId); + } catch (Exception e) { + LOGGER.warn("Failed to drop task {} because {}", taskId, e.getMessage(), e); + return Response.serverError() + .entity(String.format("Failed to drop task %s, because %s", taskId, e.getMessage())) + .build(); + } } try { - final TaskCombiner task = tasks.get(taskId); - task.drop(); - tasks.remove(taskId); - // remove task info from sqlite PersistenceService.task().ifPresent(taskPersistence -> taskPersistence.tryDeleteTask(taskId)); @@ -210,21 +210,4 @@ public synchronized void close() throws Exception { tasks.clear(); LOGGER.info("Task runtime closed in {}ms", System.currentTimeMillis() - currentTime); } - - private Map convert(final Map attributes) { - final Map modifyParamMap = new HashMap<>(); - final Iterator> paramIterator = attributes.entrySet().iterator(); - - while (paramIterator.hasNext()) { - final Map.Entry param = paramIterator.next(); - - if (!CollectorParameters.matchAnyParam(param.getKey())) { - modifyParamMap.put(param.getKey().replace("-", "_"), param.getValue()); - paramIterator.remove(); - } - } - attributes.putAll(modifyParamMap); - - return attributes; - } } From 955e5da39a8b69c51d181853ef6cc72eed7c34ad Mon Sep 17 00:00:00 2001 From: ysc <2725843507@qq.com> Date: Thu, 29 May 2025 14:33:11 +0800 Subject: [PATCH 06/11] update memory control --- .../collector/config/PipeRuntimeOptions.java | 36 ++ .../evolvable/batch/PipeTabletEventBatch.java | 10 +- .../memory/AtomicLongMemoryBlock.java | 26 + .../sink/resource/memory/IMemoryBlock.java | 10 + .../sink/resource/memory/PipeMemoryBlock.java | 19 +- .../resource/memory/PipeMemoryManager.java | 449 +++++++++++++++++- 6 files changed, 544 insertions(+), 6 deletions(-) diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/config/PipeRuntimeOptions.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/config/PipeRuntimeOptions.java index fe47a092..abbb693b 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/config/PipeRuntimeOptions.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/config/PipeRuntimeOptions.java @@ -304,4 +304,40 @@ public void setValue(String valueString) { value = Integer.parseInt(valueString); } }; + + public static final Option PIPE_MEMORY_ALLOCATE_MAX_RETRIES = + new Option("pipe_memory_allocate_max_retries", 10) { + @Override + public void setValue(String valueString) { + value = Integer.parseInt(valueString); + } + }; + + public static final Option PIPE_MEMORY_ALLOCATE_RETRY_INTERVAL_MS = + new Option("pipe_memory_allocate_retry_interval_ms", 50) { + @Override + public void setValue(String valueString) { + value = Integer.parseInt(valueString); + } + }; + + public static final Option + PIPE_DATA_STRUCTURE_TABLET_MEMORY_BLOCK_ALLOCATION_REJECT_THRESHOLD = + new Option( + "pipe_data_structure_tablet_memory_block_allocation_reject_threshold", 0.4) { + @Override + public void setValue(String valueString) { + value = Double.parseDouble(valueString); + } + }; + + public static final Option + PIPE_DATA_STRUCTURE_TS_FILE_MEMORY_BLOCK_ALLOCATION_REJECT_THRESHOLD = + new Option( + "pipe_data_structure_ts_file_memory_block_allocation_reject_threshold", 0.4) { + @Override + public void setValue(String valueString) { + value = Double.parseDouble(valueString); + } + }; } diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/payload/evolvable/batch/PipeTabletEventBatch.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/payload/evolvable/batch/PipeTabletEventBatch.java index aaf0a026..93e77b6f 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/payload/evolvable/batch/PipeTabletEventBatch.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/payload/evolvable/batch/PipeTabletEventBatch.java @@ -19,7 +19,6 @@ package org.apache.iotdb.collector.plugin.builtin.sink.payload.evolvable.batch; -import org.apache.iotdb.collector.config.PipeRuntimeOptions; import org.apache.iotdb.collector.plugin.builtin.sink.event.PipeRawTabletInsertionEvent; import org.apache.iotdb.collector.plugin.builtin.sink.resource.memory.PipeMemoryBlock; import org.apache.iotdb.collector.plugin.builtin.sink.resource.memory.PipeMemoryManager; @@ -84,6 +83,7 @@ protected PipeTabletEventBatch(final int maxDelayInMs, final long requestMaxBatc * @return {@code true} if the batch can be transferred */ public synchronized boolean onEvent(final TabletInsertionEvent event) throws IOException { + // TODO consider using a more generic event instead if (isClosed || !(event instanceof PipeRawTabletInsertionEvent)) { return false; } @@ -115,8 +115,8 @@ public synchronized boolean onEvent(final TabletInsertionEvent event) throws IOE public boolean shouldEmit() { return totalBufferSize >= getMaxBatchSizeInBytes() - || System.currentTimeMillis() - firstEventProcessingTime >= maxDelayInMs - || events.size() > PipeRuntimeOptions.PIPE_MAX_ALLOWED_EVENT_COUNT_IN_TABLET_BATCH.value(); + || System.currentTimeMillis() - firstEventProcessingTime >= maxDelayInMs; + // || events.size() > PipeRuntimeOptions.PIPE_MAX_ALLOWED_EVENT_COUNT_IN_TABLET_BATCH.value(); } private long getMaxBatchSizeInBytes() { @@ -136,6 +136,10 @@ public synchronized void close() { isClosed = true; events.clear(); + + if (allocatedMemoryBlock != null) { + allocatedMemoryBlock.close(); + } } public List deepCopyEvents() { diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/resource/memory/AtomicLongMemoryBlock.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/resource/memory/AtomicLongMemoryBlock.java index f73c3a53..df52c18b 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/resource/memory/AtomicLongMemoryBlock.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/resource/memory/AtomicLongMemoryBlock.java @@ -19,9 +19,15 @@ package org.apache.iotdb.collector.plugin.builtin.sink.resource.memory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.util.concurrent.atomic.AtomicLong; public class AtomicLongMemoryBlock extends IMemoryBlock { + + private static final Logger LOGGER = LoggerFactory.getLogger(AtomicLongMemoryBlock.class); + /** The memory usage in byte of this memory block */ protected final AtomicLong usedMemoryInBytes = new AtomicLong(0); @@ -36,6 +42,26 @@ public AtomicLongMemoryBlock( this.memoryBlockType = memoryBlockType; } + @Override + public void release(long sizeInByte) { + usedMemoryInBytes.updateAndGet( + memCost -> { + if (sizeInByte > memCost) { + LOGGER.warn( + "The memory cost to be released is larger than the memory cost of memory block {}", + this); + return 0; + } + return memCost - sizeInByte; + }); + } + + /** Get the free memory in byte of this memory block */ + @Override + public long getFreeMemoryInBytes() { + return totalMemorySizeInBytes - usedMemoryInBytes.get(); + } + @Override public void forceAllocateWithoutLimitation(long sizeInByte) { usedMemoryInBytes.addAndGet(sizeInByte); diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/resource/memory/IMemoryBlock.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/resource/memory/IMemoryBlock.java index a7acde3a..738b54cc 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/resource/memory/IMemoryBlock.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/resource/memory/IMemoryBlock.java @@ -40,6 +40,16 @@ public abstract class IMemoryBlock implements AutoCloseable { /** The total memory size in byte of this memory block */ protected long totalMemorySizeInBytes; + /** + * Try to release memory managed by this memory block + * + * @param sizeInByte the size of memory to be released, should be positive + */ + public abstract void release(final long sizeInByte); + + /** Get the free memory in byte of this memory block */ + public abstract long getFreeMemoryInBytes(); + /** * Forcibly allocate memory without the limit of totalMemorySizeInBytes * diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/resource/memory/PipeMemoryBlock.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/resource/memory/PipeMemoryBlock.java index e8a9ac9a..9017a34b 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/resource/memory/PipeMemoryBlock.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/resource/memory/PipeMemoryBlock.java @@ -33,6 +33,8 @@ public class PipeMemoryBlock implements AutoCloseable { private static final Logger LOGGER = LoggerFactory.getLogger(PipeMemoryBlock.class); + private final PipeMemoryManager pipeMemoryManager = PipeMemoryManager.getInstance(); + private final ReentrantLock lock = new ReentrantLock(); private final AtomicLong memoryUsageInBytes = new AtomicLong(0); @@ -42,6 +44,8 @@ public class PipeMemoryBlock implements AutoCloseable { private final AtomicReference expandMethod = new AtomicReference<>(); private final AtomicReference> expandCallback = new AtomicReference<>(); + private volatile boolean isReleased = false; + public PipeMemoryBlock(final long memoryUsageInBytes) { this.memoryUsageInBytes.set(memoryUsageInBytes); } @@ -90,7 +94,8 @@ private boolean doShrink() { final long newMemorySizeInBytes = shrinkMethod.get().applyAsLong(memoryUsageInBytes.get()); final long memoryInBytesCanBeReleased = oldMemorySizeInBytes - newMemorySizeInBytes; - if (memoryInBytesCanBeReleased <= 0) { + if (memoryInBytesCanBeReleased <= 0 + || !pipeMemoryManager.release(this, memoryInBytesCanBeReleased)) { return false; } @@ -138,6 +143,18 @@ private boolean doExpand() { return true; } + public void setMemoryUsageInBytes(final long memoryUsageInBytes) { + this.memoryUsageInBytes.set(memoryUsageInBytes); + } + + boolean isReleased() { + return isReleased; + } + + void markAsReleased() { + isReleased = true; + } + @Override public String toString() { return "PipeMemoryBlock{" + "usedMemoryInBytes=" + memoryUsageInBytes.get() + '}'; diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/resource/memory/PipeMemoryManager.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/resource/memory/PipeMemoryManager.java index d4309329..01a63cac 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/resource/memory/PipeMemoryManager.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/resource/memory/PipeMemoryManager.java @@ -20,6 +20,7 @@ package org.apache.iotdb.collector.plugin.builtin.sink.resource.memory; import org.apache.iotdb.collector.config.PipeRuntimeOptions; +import org.apache.iotdb.collector.plugin.builtin.sink.exception.PipeRuntimeOutOfMemoryCriticalException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -38,6 +39,10 @@ public class PipeMemoryManager { private static final boolean PIPE_MEMORY_MANAGEMENT_ENABLED = PipeRuntimeOptions.PIPE_MEMORY_MANAGEMENT_ENABLED.value(); + private static final int MEMORY_ALLOCATE_MAX_RETRIES = + PipeRuntimeOptions.PIPE_MEMORY_ALLOCATE_MAX_RETRIES.value(); + private static final long MEMORY_ALLOCATE_RETRY_INTERVAL_IN_MS = + PipeRuntimeOptions.PIPE_MEMORY_ALLOCATE_RETRY_INTERVAL_MS.value(); private static final long MEMORY_ALLOCATE_MIN_SIZE_IN_BYTES = PipeRuntimeOptions.PIPE_MEMORY_ALLOCATE_MIN_SIZE_IN_BYTES.value(); @@ -45,6 +50,21 @@ public class PipeMemoryManager { new MemoryManager("pipeMemoryManager", null, Runtime.getRuntime().totalMemory() / 10) .exactAllocate("Stream", MemoryBlockType.DYNAMIC); + private static final double EXCEED_PROTECT_THRESHOLD = 0.95; + + // To avoid too much parsed events causing OOM. If total tablet memory size exceeds this + // threshold, allocations of memory block for tablets will be rejected. + private static final double TABLET_MEMORY_REJECT_THRESHOLD = + PipeRuntimeOptions.PIPE_DATA_STRUCTURE_TABLET_MEMORY_BLOCK_ALLOCATION_REJECT_THRESHOLD + .value(); + private volatile long usedMemorySizeInBytesOfTablets; + + // Used to control the memory allocated for managing slice tsfile. + private static final double TS_FILE_MEMORY_REJECT_THRESHOLD = + PipeRuntimeOptions.PIPE_DATA_STRUCTURE_TS_FILE_MEMORY_BLOCK_ALLOCATION_REJECT_THRESHOLD + .value(); + private volatile long usedMemorySizeInBytesOfTsFiles; + private static final double FLOATING_MEMORY_RATIO = PipeRuntimeOptions.PIPE_TOTAL_FLOATING_MEMORY_PROPORTION.value(); @@ -53,6 +73,297 @@ public class PipeMemoryManager { private PipeMemoryManager() {} + // NOTE: Here we unify the memory threshold judgment for tablet and tsfile memory block, because + // introducing too many heuristic rules not conducive to flexible dynamic adjustment of memory + // configuration: + // 1. Proportion of memory occupied by tablet memory block: [TABLET_MEMORY_REJECT_THRESHOLD / 2, + // TABLET_MEMORY_REJECT_THRESHOLD + TS_FILE_MEMORY_REJECT_THRESHOLD / 2] + // 2. Proportion of memory occupied by tsfile memory block: [TS_FILE_MEMORY_REJECT_THRESHOLD / 2, + // TS_FILE_MEMORY_REJECT_THRESHOLD + TABLET_MEMORY_REJECT_THRESHOLD / 2] + // 3. The sum of the memory proportion occupied by the tablet memory block and the tsfile memory + // block does not exceed TABLET_MEMORY_REJECT_THRESHOLD + TS_FILE_MEMORY_REJECT_THRESHOLD + + private double allowedMaxMemorySizeInBytesOfTabletsAndTsFiles() { + return (TABLET_MEMORY_REJECT_THRESHOLD + TS_FILE_MEMORY_REJECT_THRESHOLD) + * getTotalNonFloatingMemorySizeInBytes(); + } + + private double allowedMaxMemorySizeInBytesOfTablets() { + return (TABLET_MEMORY_REJECT_THRESHOLD + TS_FILE_MEMORY_REJECT_THRESHOLD / 2) + * getTotalNonFloatingMemorySizeInBytes(); + } + + private double allowedMaxMemorySizeInBytesOfTsTiles() { + return (TS_FILE_MEMORY_REJECT_THRESHOLD + TABLET_MEMORY_REJECT_THRESHOLD / 2) + * getTotalNonFloatingMemorySizeInBytes(); + } + + public boolean isEnough4TabletParsing() { + return (double) usedMemorySizeInBytesOfTablets + (double) usedMemorySizeInBytesOfTsFiles + < EXCEED_PROTECT_THRESHOLD * allowedMaxMemorySizeInBytesOfTabletsAndTsFiles() + && (double) usedMemorySizeInBytesOfTablets + < EXCEED_PROTECT_THRESHOLD * allowedMaxMemorySizeInBytesOfTablets(); + } + + private boolean isHardEnough4TabletParsing() { + return (double) usedMemorySizeInBytesOfTablets + (double) usedMemorySizeInBytesOfTsFiles + < allowedMaxMemorySizeInBytesOfTabletsAndTsFiles() + && (double) usedMemorySizeInBytesOfTablets < allowedMaxMemorySizeInBytesOfTablets(); + } + + public boolean isEnough4TsFileSlicing() { + return (double) usedMemorySizeInBytesOfTablets + (double) usedMemorySizeInBytesOfTsFiles + < EXCEED_PROTECT_THRESHOLD * allowedMaxMemorySizeInBytesOfTabletsAndTsFiles() + && (double) usedMemorySizeInBytesOfTsFiles + < EXCEED_PROTECT_THRESHOLD * allowedMaxMemorySizeInBytesOfTsTiles(); + } + + private boolean isHardEnough4TsFileSlicing() { + return (double) usedMemorySizeInBytesOfTablets + (double) usedMemorySizeInBytesOfTsFiles + < allowedMaxMemorySizeInBytesOfTabletsAndTsFiles() + && (double) usedMemorySizeInBytesOfTsFiles < allowedMaxMemorySizeInBytesOfTsTiles(); + } + + public synchronized PipeMemoryBlock forceAllocate(long sizeInBytes) + throws PipeRuntimeOutOfMemoryCriticalException { + if (!PIPE_MEMORY_MANAGEMENT_ENABLED) { + // No need to calculate the tablet size, skip it to save time + return new PipeMemoryBlock(0); + } + + if (sizeInBytes == 0) { + return registerMemoryBlock(0); + } + + return forceAllocateWithRetry(sizeInBytes, PipeMemoryBlockType.NORMAL); + } + + public PipeTabletMemoryBlock forceAllocateForTabletWithRetry(long tabletSizeInBytes) + throws PipeRuntimeOutOfMemoryCriticalException { + if (!PIPE_MEMORY_MANAGEMENT_ENABLED) { + // No need to calculate the tablet size, skip it to save time + return new PipeTabletMemoryBlock(0); + } + + if (tabletSizeInBytes == 0) { + return (PipeTabletMemoryBlock) registerMemoryBlock(0, PipeMemoryBlockType.TABLET); + } + + for (int i = 1; i <= MEMORY_ALLOCATE_MAX_RETRIES; i++) { + if (isHardEnough4TabletParsing()) { + break; + } + + try { + Thread.sleep(MEMORY_ALLOCATE_RETRY_INTERVAL_IN_MS); + } catch (InterruptedException ex) { + Thread.currentThread().interrupt(); + LOGGER.warn("forceAllocateWithRetry: interrupted while waiting for available memory", ex); + } + } + + if (!isHardEnough4TabletParsing()) { + throw new PipeRuntimeOutOfMemoryCriticalException( + String.format( + "forceAllocateForTablet: failed to allocate because there's too much memory for tablets, " + + "total memory size %d bytes, used memory for tablet size %d bytes, requested memory size %d bytes", + getTotalNonFloatingMemorySizeInBytes(), + usedMemorySizeInBytesOfTablets, + tabletSizeInBytes)); + } + + synchronized (this) { + final PipeTabletMemoryBlock block = + (PipeTabletMemoryBlock) + forceAllocateWithRetry(tabletSizeInBytes, PipeMemoryBlockType.TABLET); + usedMemorySizeInBytesOfTablets += block.getMemoryUsageInBytes(); + return block; + } + } + + public PipeTsFileMemoryBlock forceAllocateForTsFileWithRetry(long tsFileSizeInBytes) + throws PipeRuntimeOutOfMemoryCriticalException { + if (!PIPE_MEMORY_MANAGEMENT_ENABLED) { + return new PipeTsFileMemoryBlock(0); + } + + if (tsFileSizeInBytes == 0) { + return (PipeTsFileMemoryBlock) registerMemoryBlock(0, PipeMemoryBlockType.TS_FILE); + } + + for (int i = 1; i <= MEMORY_ALLOCATE_MAX_RETRIES; i++) { + if (isHardEnough4TsFileSlicing()) { + break; + } + + try { + Thread.sleep(MEMORY_ALLOCATE_RETRY_INTERVAL_IN_MS); + } catch (InterruptedException ex) { + Thread.currentThread().interrupt(); + LOGGER.warn("forceAllocateWithRetry: interrupted while waiting for available memory", ex); + } + } + + if (!isHardEnough4TsFileSlicing()) { + throw new PipeRuntimeOutOfMemoryCriticalException( + String.format( + "forceAllocateForTsFile: failed to allocate because there's too much memory for tsfiles, " + + "total memory size %d bytes, used memory for tsfile size %d bytes, requested memory size %d bytes", + getTotalNonFloatingMemorySizeInBytes(), + usedMemorySizeInBytesOfTsFiles, + tsFileSizeInBytes)); + } + + synchronized (this) { + final PipeTsFileMemoryBlock block = + (PipeTsFileMemoryBlock) + forceAllocateWithRetry(tsFileSizeInBytes, PipeMemoryBlockType.TS_FILE); + usedMemorySizeInBytesOfTsFiles += block.getMemoryUsageInBytes(); + return block; + } + } + + private PipeMemoryBlock forceAllocateWithRetry(long sizeInBytes, PipeMemoryBlockType type) + throws PipeRuntimeOutOfMemoryCriticalException { + if (!PIPE_MEMORY_MANAGEMENT_ENABLED) { + switch (type) { + case TABLET: + return new PipeTabletMemoryBlock(sizeInBytes); + case TS_FILE: + return new PipeTsFileMemoryBlock(sizeInBytes); + default: + return new PipeMemoryBlock(sizeInBytes); + } + } + + for (int i = 1; i <= MEMORY_ALLOCATE_MAX_RETRIES; i++) { + if (getTotalNonFloatingMemorySizeInBytes() - memoryBlock.getUsedMemoryInBytes() + >= sizeInBytes) { + return registerMemoryBlock(sizeInBytes, type); + } + + try { + tryShrinkUntilFreeMemorySatisfy(sizeInBytes); + this.wait(MEMORY_ALLOCATE_RETRY_INTERVAL_IN_MS); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + LOGGER.warn("forceAllocate: interrupted while waiting for available memory", e); + } + } + + throw new PipeRuntimeOutOfMemoryCriticalException( + String.format( + "forceAllocate: failed to allocate memory after %d retries, " + + "total memory size %d bytes, used memory size %d bytes, " + + "requested memory size %d bytes", + MEMORY_ALLOCATE_MAX_RETRIES, + getTotalNonFloatingMemorySizeInBytes(), + memoryBlock.getUsedMemoryInBytes(), + sizeInBytes)); + } + + public synchronized void forceResize(PipeMemoryBlock block, long targetSize) { + if (block == null || block.isReleased()) { + LOGGER.warn("forceResize: cannot resize a null or released memory block"); + return; + } + + if (!PIPE_MEMORY_MANAGEMENT_ENABLED) { + block.setMemoryUsageInBytes(targetSize); + return; + } + + final long oldSize = block.getMemoryUsageInBytes(); + if (oldSize == 0) { + // If the memory block is not registered, we need to register it first. + // Otherwise, the memory usage will be inconsistent. + // See registerMemoryBlock for more details. + allocatedBlocks.add(block); + } + + if (oldSize >= targetSize) { + memoryBlock.release(oldSize - targetSize); + if (block instanceof PipeTabletMemoryBlock) { + usedMemorySizeInBytesOfTablets -= oldSize - targetSize; + } + if (block instanceof PipeTsFileMemoryBlock) { + usedMemorySizeInBytesOfTsFiles -= oldSize - targetSize; + } + block.setMemoryUsageInBytes(targetSize); + + // If no memory is used in the block, we can remove it from the allocated blocks. + if (targetSize == 0) { + allocatedBlocks.remove(block); + } + return; + } + + long sizeInBytes = targetSize - oldSize; + for (int i = 1; i <= MEMORY_ALLOCATE_MAX_RETRIES; i++) { + if (getTotalNonFloatingMemorySizeInBytes() - memoryBlock.getUsedMemoryInBytes() + >= sizeInBytes) { + memoryBlock.forceAllocateWithoutLimitation(sizeInBytes); + if (block instanceof PipeTabletMemoryBlock) { + usedMemorySizeInBytesOfTablets += sizeInBytes; + } + if (block instanceof PipeTsFileMemoryBlock) { + usedMemorySizeInBytesOfTsFiles += sizeInBytes; + } + block.setMemoryUsageInBytes(targetSize); + return; + } + + try { + tryShrinkUntilFreeMemorySatisfy(sizeInBytes); + this.wait(MEMORY_ALLOCATE_RETRY_INTERVAL_IN_MS); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + LOGGER.warn("forceResize: interrupted while waiting for available memory", e); + } + } + + throw new PipeRuntimeOutOfMemoryCriticalException( + String.format( + "forceResize: failed to allocate memory after %d retries, " + + "total memory size %d bytes, used memory size %d bytes, " + + "requested memory size %d bytes", + MEMORY_ALLOCATE_MAX_RETRIES, + getTotalNonFloatingMemorySizeInBytes(), + memoryBlock.getUsedMemoryInBytes(), + sizeInBytes)); + } + + /** + * Allocate a {@link PipeMemoryBlock} for pipe only if memory used after allocation is less than + * the specified threshold. + * + * @param sizeInBytes size of memory needed to allocate + * @param usedThreshold proportion of memory used, ranged from 0.0 to 1.0 + * @return {@code null} if the proportion of memory used after allocation exceeds {@code + * usedThreshold}. Will return a memory block otherwise. + */ + public synchronized PipeMemoryBlock forceAllocateIfSufficient( + long sizeInBytes, float usedThreshold) { + if (usedThreshold < 0.0f || usedThreshold > 1.0f) { + return null; + } + + if (!PIPE_MEMORY_MANAGEMENT_ENABLED) { + return new PipeMemoryBlock(sizeInBytes); + } + + if (sizeInBytes == 0) { + return registerMemoryBlock(0); + } + + if ((float) (memoryBlock.getUsedMemoryInBytes() + sizeInBytes) + <= getTotalNonFloatingMemorySizeInBytes() * usedThreshold) { + return forceAllocate(sizeInBytes); + } + + return null; + } + public synchronized PipeMemoryBlock tryAllocate(long sizeInBytes) { return tryAllocate(sizeInBytes, currentSize -> currentSize * 2 / 3); } @@ -114,12 +425,49 @@ public synchronized PipeMemoryBlock tryAllocate( } } + public synchronized boolean tryAllocate( + PipeMemoryBlock block, long memoryInBytesNeededToBeAllocated) { + if (!PIPE_MEMORY_MANAGEMENT_ENABLED || block == null || block.isReleased()) { + return false; + } + + if (getTotalNonFloatingMemorySizeInBytes() - memoryBlock.getUsedMemoryInBytes() + >= memoryInBytesNeededToBeAllocated) { + memoryBlock.forceAllocateWithoutLimitation(memoryInBytesNeededToBeAllocated); + if (block instanceof PipeTabletMemoryBlock) { + usedMemorySizeInBytesOfTablets += memoryInBytesNeededToBeAllocated; + } + if (block instanceof PipeTsFileMemoryBlock) { + usedMemorySizeInBytesOfTsFiles += memoryInBytesNeededToBeAllocated; + } + block.setMemoryUsageInBytes(block.getMemoryUsageInBytes() + memoryInBytesNeededToBeAllocated); + return true; + } + + return false; + } + private PipeMemoryBlock registerMemoryBlock(long sizeInBytes) { + return registerMemoryBlock(sizeInBytes, PipeMemoryBlockType.NORMAL); + } + + private PipeMemoryBlock registerMemoryBlock(long sizeInBytes, PipeMemoryBlockType type) { + final PipeMemoryBlock returnedMemoryBlock; + switch (type) { + case TABLET: + returnedMemoryBlock = new PipeTabletMemoryBlock(sizeInBytes); + break; + case TS_FILE: + returnedMemoryBlock = new PipeTsFileMemoryBlock(sizeInBytes); + break; + default: + returnedMemoryBlock = new PipeMemoryBlock(sizeInBytes); + break; + } + // For memory block whose size is 0, we do not need to add it to the allocated blocks now. // It's good for performance and will not trigger concurrent issues. // If forceResize is called on it, we will add it to the allocated blocks. - final PipeMemoryBlock returnedMemoryBlock = new PipeMemoryBlock(sizeInBytes); - if (sizeInBytes > 0) { memoryBlock.forceAllocateWithoutLimitation(sizeInBytes); allocatedBlocks.add(returnedMemoryBlock); @@ -149,10 +497,107 @@ private boolean tryShrinkUntilFreeMemorySatisfy(long sizeInBytes) { } } + public synchronized void tryExpandAllAndCheckConsistency() { + allocatedBlocks.forEach(PipeMemoryBlock::expand); + + long blockSum = + allocatedBlocks.stream().mapToLong(PipeMemoryBlock::getMemoryUsageInBytes).sum(); + if (blockSum != memoryBlock.getUsedMemoryInBytes()) { + LOGGER.warn( + "tryExpandAllAndCheckConsistency: memory usage is not consistent with allocated blocks," + + " usedMemorySizeInBytes is {} but sum of all blocks is {}", + memoryBlock.getUsedMemoryInBytes(), + blockSum); + } + + long tabletBlockSum = + allocatedBlocks.stream() + .filter(PipeTabletMemoryBlock.class::isInstance) + .mapToLong(PipeMemoryBlock::getMemoryUsageInBytes) + .sum(); + if (tabletBlockSum != usedMemorySizeInBytesOfTablets) { + LOGGER.warn( + "tryExpandAllAndCheckConsistency: memory usage of tablets is not consistent with allocated blocks," + + " usedMemorySizeInBytesOfTablets is {} but sum of all tablet blocks is {}", + usedMemorySizeInBytesOfTablets, + tabletBlockSum); + } + + long tsFileBlockSum = + allocatedBlocks.stream() + .filter(PipeTsFileMemoryBlock.class::isInstance) + .mapToLong(PipeMemoryBlock::getMemoryUsageInBytes) + .sum(); + if (tsFileBlockSum != usedMemorySizeInBytesOfTsFiles) { + LOGGER.warn( + "tryExpandAllAndCheckConsistency: memory usage of tsfiles is not consistent with allocated blocks," + + " usedMemorySizeInBytesOfTsFiles is {} but sum of all tsfile blocks is {}", + usedMemorySizeInBytesOfTsFiles, + tsFileBlockSum); + } + } + + public synchronized void release(PipeMemoryBlock block) { + if (!PIPE_MEMORY_MANAGEMENT_ENABLED || block == null || block.isReleased()) { + return; + } + + allocatedBlocks.remove(block); + memoryBlock.release(block.getMemoryUsageInBytes()); + if (block instanceof PipeTabletMemoryBlock) { + usedMemorySizeInBytesOfTablets -= block.getMemoryUsageInBytes(); + } + if (block instanceof PipeTsFileMemoryBlock) { + usedMemorySizeInBytesOfTsFiles -= block.getMemoryUsageInBytes(); + } + block.markAsReleased(); + + this.notifyAll(); + } + + public synchronized boolean release(PipeMemoryBlock block, long sizeInBytes) { + if (!PIPE_MEMORY_MANAGEMENT_ENABLED || block == null || block.isReleased()) { + return false; + } + + memoryBlock.release(sizeInBytes); + if (block instanceof PipeTabletMemoryBlock) { + usedMemorySizeInBytesOfTablets -= sizeInBytes; + } + if (block instanceof PipeTsFileMemoryBlock) { + usedMemorySizeInBytesOfTsFiles -= sizeInBytes; + } + block.setMemoryUsageInBytes(block.getMemoryUsageInBytes() - sizeInBytes); + + this.notifyAll(); + + return true; + } + + public long getUsedMemorySizeInBytes() { + return memoryBlock.getUsedMemoryInBytes(); + } + + public long getUsedMemorySizeInBytesOfTablets() { + return usedMemorySizeInBytesOfTablets; + } + + public long getUsedMemorySizeInBytesOfTsFiles() { + return usedMemorySizeInBytesOfTsFiles; + } + + public long getFreeMemorySizeInBytes() { + return memoryBlock.getFreeMemoryInBytes(); + } + public long getTotalNonFloatingMemorySizeInBytes() { return (long) (memoryBlock.getTotalMemorySizeInBytes() * (1 - FLOATING_MEMORY_RATIO)); } + public long getTotalFloatingMemorySizeInBytes() { + return (long) (memoryBlock.getTotalMemorySizeInBytes() * FLOATING_MEMORY_RATIO); + } + public static PipeMemoryManager getInstance() { return PipeMemoryManagerHolder.INSTANCE; } From d0c0eb83e1bb5d1bc497c83243ec9f41f035b110 Mon Sep 17 00:00:00 2001 From: ysc <2725843507@qq.com> Date: Tue, 3 Jun 2025 11:17:36 +0800 Subject: [PATCH 07/11] update exception when creating task --- .../resources/conf/application.properties | 12 --- .../persistence/PluginPersistence.java | 49 ++++++++---- .../persistence/TaskPersistence.java | 78 +++++++++++++------ .../iotdb/IoTDBSubscriptionPullSource.java | 6 ++ .../IoTDBSubscriptionSourceConstant.java | 2 +- .../IoTDBSubscriptionTablePullSource.java | 5 ++ .../IoTDBSubscriptionTreePullSource.java | 5 ++ .../runtime/task/processor/ProcessorTask.java | 20 ++++- .../collector/runtime/task/sink/SinkTask.java | 18 ++++- .../task/source/pull/PullSourceTask.java | 19 ++++- .../task/source/push/PushSourceTask.java | 19 ++++- 11 files changed, 175 insertions(+), 58 deletions(-) diff --git a/iotdb-collector/collector-core/src/assembly/resources/conf/application.properties b/iotdb-collector/collector-core/src/assembly/resources/conf/application.properties index 26567883..4d0e1350 100644 --- a/iotdb-collector/collector-core/src/assembly/resources/conf/application.properties +++ b/iotdb-collector/collector-core/src/assembly/resources/conf/application.properties @@ -110,18 +110,6 @@ rate_limiter_hot_reload_check_interval_ms=1000 # Data type: int max_retry_times=5 -# Used for connection of IoTDB native clients -# Bind with rpc_address -# Effective mode: on every start -# Data type: int -rpc_port=6668 - -# Used for connection of IoTDB native clients(Session) -# Could set 127.0.0.1(for local test) or ipv4 address -# Effective mode: on every start -# Data type: String -rpc_address=0.0.0.0 - # Buffer size for reading files in pipe connector (8MB default) # Effective mode: on every start # Data type: int diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/persistence/PluginPersistence.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/persistence/PluginPersistence.java index f575fd97..c12d23e4 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/persistence/PluginPersistence.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/persistence/PluginPersistence.java @@ -37,7 +37,8 @@ import java.sql.ResultSet; import java.sql.SQLException; import java.sql.Timestamp; -import java.util.Objects; +import java.util.ArrayList; +import java.util.List; public class PluginPersistence extends Persistence { @@ -76,6 +77,8 @@ public void tryResume() { final String queryAllPluginSQL = "SELECT plugin_name, class_name, jar_name, jar_md5 FROM plugin"; + final List notExistPluginNames = new ArrayList<>(); + try (final Connection connection = getConnection()) { final PreparedStatement statement = connection.prepareStatement(queryAllPluginSQL); final ResultSet pluginResultSet = statement.executeQuery(); @@ -87,15 +90,25 @@ public void tryResume() { final String jarMd5 = pluginResultSet.getString("jar_md5"); if (!isPluginJarFileWithMD5NameExists(pluginName, jarName, jarMd5)) { - tryDeletePlugin(pluginName); + notExistPluginNames.add(pluginName); + LOGGER.warn( + "Plugin {} has not existed, will try delete from database later.", pluginName); continue; } - tryRecoverPlugin(pluginName, className, jarName, jarMd5); + final Response response = tryRecoverPlugin(pluginName, className, jarName, jarMd5); + if (response.getStatus() != Response.Status.OK.getStatusCode()) { + LOGGER.warn( + "Failed to recover plugin message from plugin {}, because {}", pluginName, response); + } } } catch (final SQLException e) { LOGGER.warn("Failed to resume plugin persistence message, because {}", e.getMessage()); } + + if (!notExistPluginNames.isEmpty()) { + tryDeleteNotExistPlugins(notExistPluginNames); + } } private boolean isPluginJarFileWithMD5NameExists( @@ -108,17 +121,27 @@ private boolean isPluginJarFileWithMD5NameExists( return Files.exists(pluginJarFileWithMD5Path); } - private void tryRecoverPlugin( + private Response tryRecoverPlugin( final String pluginName, final String className, final String jarName, final String jarMD5) { - final Response response = - RuntimeService.plugin().isPresent() - ? RuntimeService.plugin() - .get() - .createPlugin(pluginName, className, jarName, jarMD5, false) - : null; - - if (Objects.isNull(response) || response.getStatus() != Response.Status.OK.getStatusCode()) { - LOGGER.warn("Failed to recover plugin message from plugin {}: {}", pluginName, response); + return RuntimeService.plugin().isPresent() + ? RuntimeService.plugin().get().createPlugin(pluginName, className, jarName, jarMD5, false) + : Response.serverError().entity("[RuntimeService] the plugin runtime is null.").build(); + } + + private void tryDeleteNotExistPlugins(final List pluginNames) { + final String batchDeleteSQL = "DELETE FROM plugin WHERE plugin_name = ?"; + + try (final Connection connection = getConnection(); + final PreparedStatement statement = connection.prepareStatement(batchDeleteSQL)) { + for (final String pluginName : pluginNames) { + statement.setString(1, pluginName); + statement.addBatch(); + } + statement.executeBatch(); + + LOGGER.info("Successfully delete plugin names: {}", pluginNames); + } catch (final SQLException e) { + LOGGER.warn("Failed to delete plugin names: {}, because {}", pluginNames, e.getMessage()); } } diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/persistence/TaskPersistence.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/persistence/TaskPersistence.java index 88c4d9ca..86daa165 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/persistence/TaskPersistence.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/persistence/TaskPersistence.java @@ -44,7 +44,6 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Map; -import java.util.Objects; import java.util.Optional; public class TaskPersistence extends Persistence { @@ -84,6 +83,7 @@ public void tryResume() { final String queryAllTaskSQL = "SELECT task_id, task_state, source_attribute, processor_attribute, sink_attribute, create_time FROM task"; + final List failedResumeTaskIds = new ArrayList<>(); try (final Connection connection = getConnection(); final PreparedStatement statement = connection.prepareStatement(queryAllTaskSQL); final ResultSet taskResultSet = statement.executeQuery()) { @@ -94,35 +94,63 @@ public void tryResume() { final byte[] processorAttribute = taskResultSet.getBytes(4); final byte[] sinkAttribute = taskResultSet.getBytes(5); - tryRecoverTask( - taskId, - taskState, - SerializationUtil.deserialize(sourceAttribute), - SerializationUtil.deserialize(processorAttribute), - SerializationUtil.deserialize(sinkAttribute)); + final Response response = + tryRecoverTask( + taskId, + taskState, + SerializationUtil.deserialize(sourceAttribute), + SerializationUtil.deserialize(processorAttribute), + SerializationUtil.deserialize(sinkAttribute)); + + if (response.getStatus() != Response.Status.OK.getStatusCode()) { + LOGGER.warn( + "Failed to resume task {}, because {}, will delete from database later.", + taskId, + response); + failedResumeTaskIds.add(taskId); + } } } catch (final SQLException e) { - LOGGER.warn("Failed to resume task persistence message, because {}", e.getMessage()); + LOGGER.warn( + "Failed to resume task persistence message, task ids is {}, because {}", + failedResumeTaskIds, + e.getMessage()); + } + + if (!failedResumeTaskIds.isEmpty()) { + tryDeleteBatchTaskIds(failedResumeTaskIds); } } - public void tryRecoverTask( + public Response tryRecoverTask( final String taskId, final TaskStateEnum taskState, final Map sourceAttribute, final Map processorAttribute, final Map sinkAttribute) { - final Response response = - RuntimeService.task().isPresent() - ? RuntimeService.task() - .get() - .createTask( - taskId, taskState, sourceAttribute, processorAttribute, sinkAttribute, false) - : null; - - if (Objects.isNull(response) || response.getStatus() != Response.Status.OK.getStatusCode()) { - LOGGER.warn("Failed to recover task persistence message, because {}", response); - tryDeleteTask(taskId); + + return RuntimeService.task().isPresent() + ? RuntimeService.task() + .get() + .createTask( + taskId, taskState, sourceAttribute, processorAttribute, sinkAttribute, false) + : Response.serverError().entity("[RuntimeService] the task runtime is null.").build(); + } + + public void tryDeleteBatchTaskIds(final List taskIds) { + final String batchDeleteSQL = "DELETE FROM task WHERE task_id = ?"; + + try (final Connection connection = getConnection(); + final PreparedStatement statement = connection.prepareStatement(batchDeleteSQL)) { + for (final String taskId : taskIds) { + statement.setString(1, taskId); + statement.addBatch(); + } + statement.executeBatch(); + + LOGGER.info("Successfully deleted task ids: {}", taskIds); + } catch (final SQLException e) { + LOGGER.warn("Failed to delete task ids: {}, because {}", taskIds, e.getMessage()); } } @@ -151,7 +179,7 @@ public void tryPersistenceTask( statement.setString(7, String.valueOf(new Timestamp(System.currentTimeMillis()))); statement.executeUpdate(); - LOGGER.info("successfully persisted task {} info", taskId); + LOGGER.info("Successfully persisted task {} info", taskId); } catch (final SQLException | IOException e) { LOGGER.warn("Failed to persistence task message, because {}", e.getMessage()); } @@ -165,7 +193,7 @@ public void tryDeleteTask(final String taskId) { statement.setString(1, taskId); statement.executeUpdate(); - LOGGER.info("successfully deleted task {}", taskId); + LOGGER.info("Successfully deleted task {}", taskId); } catch (final SQLException e) { LOGGER.warn("Failed to delete task persistence message, because {}", e.getMessage()); } @@ -180,7 +208,11 @@ public void tryAlterTaskState(final String taskId, final TaskStateEnum taskState statement.setString(2, taskId); statement.executeUpdate(); - LOGGER.info("successfully altered task {}", taskId); + LOGGER.info( + "Successfully altered task {} status, from {} to {} ", + taskId, + (taskState.getTaskState() + 1) % 2, + taskState); } catch (final SQLException e) { LOGGER.warn("Failed to alter task persistence message, because {}", e.getMessage()); } diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionPullSource.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionPullSource.java index aac66387..e68c223b 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionPullSource.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionPullSource.java @@ -121,11 +121,17 @@ private void doWork() { } } } + + if (!autoCommit) { + commitAsync(messages); + } } } protected abstract List poll(); + protected abstract void commitAsync(List messages); + @Override public Event supply() throws InterruptedException { return subscription.take(); diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionSourceConstant.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionSourceConstant.java index a5a3b1df..ac91175d 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionSourceConstant.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionSourceConstant.java @@ -60,7 +60,7 @@ public class IoTDBSubscriptionSourceConstant { public static final int IOTDB_SUBSCRIPTION_SOURCE_MAX_POLL_PARALLELISM_DEFAULT_VALUE = 1; public static final String IOTDB_SUBSCRIPTION_SOURCE_AUTO_COMMIT_KEY = "auto-commit"; - public static final boolean IOTDB_SUBSCRIPTION_SOURCE_AUTO_COMMIT_DEFAULT_VALUE = false; + public static final boolean IOTDB_SUBSCRIPTION_SOURCE_AUTO_COMMIT_DEFAULT_VALUE = true; public static final String IOTDB_SUBSCRIPTION_SOURCE_AUTO_COMMIT_INTERVAL_MS_KEY = "auto-commit-interval-ms"; diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTablePullSource.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTablePullSource.java index 6d0864f0..e5352e8a 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTablePullSource.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTablePullSource.java @@ -47,6 +47,11 @@ protected List poll() { return consumer.poll(POLL_TIMEOUT_MS); } + @Override + protected void commitAsync(final List messages) { + consumer.commitAsync(messages); + } + @Override public Optional report() { return Optional.empty(); diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTreePullSource.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTreePullSource.java index 1c2130bd..916c952f 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTreePullSource.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTreePullSource.java @@ -51,6 +51,11 @@ protected List poll() { return consumer.poll(Collections.singleton(subscription.getTopic()), POLL_TIMEOUT_MS); } + @Override + protected void commitAsync(final List messages) { + consumer.commitAsync(messages); + } + @Override public Optional report() { final ProgressIndex progress = diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/runtime/task/processor/ProcessorTask.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/runtime/task/processor/ProcessorTask.java index 03353a7f..9c747e6f 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/runtime/task/processor/ProcessorTask.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/runtime/task/processor/ProcessorTask.java @@ -89,13 +89,15 @@ public ProcessorTask( } @Override - public void createInternal() throws Exception { + public void createInternal() { final PluginRuntime pluginRuntime = RuntimeService.plugin().isPresent() ? RuntimeService.plugin().get() : null; if (pluginRuntime == null) { throw new IllegalStateException("Plugin runtime is down"); } + String processorCreateErrorMsg = ""; + final long creationTime = System.currentTimeMillis(); processorConsumers = new ProcessorConsumer[parallelism]; for (int i = 0; i < parallelism; i++) { @@ -111,11 +113,23 @@ public void createInternal() throws Exception { new CollectorProcessorRuntimeConfiguration(taskId, creationTime, parallelism, i)); } catch (final Exception e) { try { + processorCreateErrorMsg = + String.format( + "Error occurred when create processor-task-%s, instance index %s, because %s tying to close it.", + taskId, i, e); + LOGGER.warn(processorCreateErrorMsg); + processorConsumers[i].consumer().close(); } catch (final Exception ex) { - LOGGER.warn("Failed to close sink on creation failure", ex); + final String processorCloseErrorMsg = + String.format( + "Error occurred when closing processor-task-%s, instance index %s, because %s", + taskId, i, ex); + LOGGER.warn(processorCloseErrorMsg); + throw new RuntimeException(processorCloseErrorMsg + "\n" + processorCreateErrorMsg); } - throw e; + + throw new RuntimeException(processorCreateErrorMsg); } } disruptor.handleEventsWithWorkerPool(processorConsumers); diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/runtime/task/sink/SinkTask.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/runtime/task/sink/SinkTask.java index 3def6d9a..65a71192 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/runtime/task/sink/SinkTask.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/runtime/task/sink/SinkTask.java @@ -88,6 +88,8 @@ public void createInternal() throws Exception { throw new IllegalStateException("Plugin runtime is down"); } + String sinkCreateErrorMsg = ""; + final long creationTime = System.currentTimeMillis(); consumers = new SinkConsumer[parallelism]; for (int i = 0; i < parallelism; i++) { @@ -103,11 +105,23 @@ public void createInternal() throws Exception { consumers[i].consumer().handshake(); } catch (final Exception e) { try { + sinkCreateErrorMsg = + String.format( + "Error occurred when create sink-task-%s, instance index %s, because %s tying to close it.", + taskId, i, e); + LOGGER.warn(sinkCreateErrorMsg); + consumers[i].consumer().close(); } catch (final Exception ex) { - LOGGER.warn("Failed to close sink on creation failure", ex); + final String sinkCloseErrorMsg = + String.format( + "Error occurred when closing sink-task-%s, instance index %s, because %s", + taskId, i, ex); + LOGGER.warn(sinkCloseErrorMsg); + throw new RuntimeException("Create: " + sinkCreateErrorMsg + "\n" + sinkCloseErrorMsg); } - throw e; + + throw new RuntimeException(sinkCreateErrorMsg); } } disruptor.handleEventsWithWorkerPool(consumers); diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/runtime/task/source/pull/PullSourceTask.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/runtime/task/source/pull/PullSourceTask.java index 769ba32c..775236ab 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/runtime/task/source/pull/PullSourceTask.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/runtime/task/source/pull/PullSourceTask.java @@ -71,6 +71,8 @@ public void createInternal() throws Exception { throw new IllegalStateException("Plugin runtime is down"); } + String pullSourceCreateErrorMsg = ""; + REGISTERED_EXECUTOR_SERVICES.putIfAbsent( taskId, new ThreadPoolExecutor( @@ -96,11 +98,24 @@ public void createInternal() throws Exception { consumers[i].consumer().start(); } catch (final Exception e) { try { + pullSourceCreateErrorMsg = + String.format( + "Error occurred when creating pull-source-task-%s, instance index %s, because %s tying to close it.", + taskId, i, e); + LOGGER.warn(pullSourceCreateErrorMsg); + consumers[i].consumer().close(); } catch (final Exception ex) { - LOGGER.warn("Failed to close source on creation failure", ex); - throw e; + final String pullSourceCloseErrorMsg = + String.format( + "Error occurred when closing pull-source-task-%s, instance index %s, because %s", + taskId, i, ex); + LOGGER.warn(pullSourceCloseErrorMsg); + + throw new RuntimeException(pullSourceCreateErrorMsg + "\n" + pullSourceCloseErrorMsg); } + + throw new RuntimeException(pullSourceCreateErrorMsg); } int finalI = i; diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/runtime/task/source/push/PushSourceTask.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/runtime/task/source/push/PushSourceTask.java index 5b9f491b..a271bc18 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/runtime/task/source/push/PushSourceTask.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/runtime/task/source/push/PushSourceTask.java @@ -61,6 +61,8 @@ public void createInternal() throws Exception { throw new IllegalStateException("Plugin runtime is down"); } + String pushSourceCreateErrorMsg = ""; + final long creationTime = System.currentTimeMillis(); pushSources = new PushSource[parallelism]; for (int i = 0; i < parallelism; i++) { @@ -78,11 +80,24 @@ public void createInternal() throws Exception { pushSources[i].start(); } catch (final Exception e) { try { + pushSourceCreateErrorMsg = + String.format( + "Error occurred when create push-source-task-%s, instance index %s, because %s trying to close it.", + taskId, i, e); + LOGGER.warn(pushSourceCreateErrorMsg); + pushSources[i].close(); } catch (final Exception ex) { - LOGGER.warn("Failed to close source on creation failure", ex); - throw e; + final String pushSourceCloseErrorMsg = + String.format( + "Error occurred when closing push-source-task-%s, instance index %s, because %s", + taskId, i, ex); + LOGGER.warn(pushSourceCloseErrorMsg); + + throw new RuntimeException(pushSourceCreateErrorMsg + "\n" + pushSourceCloseErrorMsg); } + + throw new RuntimeException(pushSourceCreateErrorMsg); } } From 64ecc19b7bff8b23e48cc5eb5e4d8effd1568e64 Mon Sep 17 00:00:00 2001 From: ysc <2725843507@qq.com> Date: Thu, 5 Jun 2025 08:33:41 +0800 Subject: [PATCH 08/11] update params --- .../resources/conf/application.properties | 72 ------- .../collector/config/PipeRuntimeOptions.java | 184 +----------------- .../config/PluginRuntimeOptions.java | 1 + .../collector/config/TaskRuntimeOptions.java | 51 ++--- .../collector/config/TrimProperties.java | 1 + .../sink/client/IoTDBSyncClientManager.java | 2 +- .../evolvable/batch/PipeTabletEventBatch.java | 5 +- .../runtime/task/processor/ProcessorTask.java | 18 +- .../collector/runtime/task/sink/SinkTask.java | 18 +- .../runtime/task/source/SourceTask.java | 11 +- 10 files changed, 51 insertions(+), 312 deletions(-) diff --git a/iotdb-collector/collector-core/src/assembly/resources/conf/application.properties b/iotdb-collector/collector-core/src/assembly/resources/conf/application.properties index 4d0e1350..8003da25 100644 --- a/iotdb-collector/collector-core/src/assembly/resources/conf/application.properties +++ b/iotdb-collector/collector-core/src/assembly/resources/conf/application.properties @@ -30,31 +30,6 @@ api_service_port=17070 ### Task Runtime Configuration #################### -# The number of concurrent threads for the source task. -# Effective mode: on every start -# Data type: int -task_source_parallelism_num=4 - -# The number of concurrent threads for the process task. -# Effective mode: on every start -# Data type: int -task_process_parallelism_num=4 - -# The number of concurrent threads for the sink task. -# Effective mode: on every start -# Data type: int -task_sink_parallelism_num=4 - -# The ring buffer size for the processor task. -# Effective mode: on every start -# Data type: int -task_processor_ring_buffer_size=1024 - -# The ring buffer size for the sink task. -# Effective mode: on every start -# Data type: int -task_sink_ring_buffer_size=1024 - # Database file location of task # Effective mode: on every start # Data type: string @@ -93,59 +68,12 @@ plugin_database_file_path=system/database/plugin.db ### Pipe Configuration #################### -# The total bytes that all pipe sinks can transfer per second. -# When given a value less than or equal to 0, it means no limit. -# default value is -1, which means no limit. -# Effective mode: on every start -# Data type: double -pipe_all_sinks_rate_limit_bytes_per_second=-1 - -# Rate limiter configuration interval in milliseconds for hot reloading -# Effective mode: on every start -# Data type: int -rate_limiter_hot_reload_check_interval_ms=1000 - -# Maximum number of retry attempts for operations -# Effective mode: on every start -# Data type: int -max_retry_times=5 - -# Buffer size for reading files in pipe connector (8MB default) -# Effective mode: on every start -# Data type: int -pipe_connector_read_file_buffer_size=8388608 - -# Timeout duration for pipe connector data transfer in milliseconds -# Effective mode: on every start -# Data type: int -pipe_connector_transfer_timeout_ms=900000 - -# Maximum allowed frame size for Thrift communication -# Effective mode: on every start -# Data type: int -thrift_frame_max_size=536870912 - -# Enable/disable thrift compression for pipe connector RPC -# Effective mode: on every start -# Data type: boolean -is_pipe_connector_rpc_thrift_compression_enabled=false - # Use this value to set timestamp precision as "ms", "us" or "ns". # Once the precision has been set, it can not be changed. # Effective mode: on every start # Data type: string timestamp_precision=ms -# Memory allocation ratio for pipe leader cache management -# Effective mode: on every start -# Data type: float -pipe_leader_cache_memory_usage_percentage=0.1 - -# Enable/disable reference tracking for pipe events -# Effective mode: on every start -# Data type: boolean -pipe_event_reference_tracking_enabled=true - # The maximum number of tablets that can be in a batch # Effective mode: on every start # Data type: int diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/config/PipeRuntimeOptions.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/config/PipeRuntimeOptions.java index abbb693b..b38d2a30 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/config/PipeRuntimeOptions.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/config/PipeRuntimeOptions.java @@ -19,50 +19,7 @@ package org.apache.iotdb.collector.config; -import org.apache.iotdb.rpc.RpcUtils; - -import java.util.concurrent.TimeUnit; - public class PipeRuntimeOptions extends Options { - public static final Option PIPE_ALL_SINK_RATE_LIMIT_BYTES_PER_SECOND = - new Option("pipe_all_sinks_rate_limit_bytes_per_second", -1d) { - @Override - public void setValue(final String valueString) { - value = Double.parseDouble(valueString); - } - }; - - public static final Option RATE_LIMITER_HOT_RELOAD_CHECK_INTERVAL_MS = - new Option("rate_limiter_hot_reload_check_interval_ms", 1000) { - @Override - public void setValue(final String valueString) { - value = Integer.parseInt(valueString); - } - }; - - public static final Option MAX_RETRY_TIMES = - new Option("max_retry_times", 5) { - @Override - public void setValue(final String valueString) { - value = Integer.parseInt(valueString); - } - }; - - public static final Option RPC_PORT = - new Option("rpc_port", 6667) { - @Override - public void setValue(final String valueString) { - value = Integer.parseInt(valueString); - } - }; - - public static final Option RPC_ADDRESS = - new Option("rpc_address", "0.0.0.0") { - @Override - public void setValue(final String valueString) { - value = valueString; - } - }; public static final Option PIPE_CONNECTOR_READ_FILE_BUFFER_SIZE = new Option("pipe_connector_read_file_buffer_size", 8388608) { @@ -88,8 +45,8 @@ public void setValue(final String valueString) { } }; - public static final Option IS_PIPE_CONNECTOR_RPC_THRIFT_COMPRESSION_ENABLED = - new Option("is_pipe_connector_rpc_thrift_compression_enabled", false) { + public static final Option PIPE_CONNECTOR_RPC_THRIFT_COMPRESSION_ENABLED = + new Option("pipe_connector_rpc_thrift_compression_enabled", false) { @Override public void setValue(final String valueString) { value = Boolean.parseBoolean(valueString); @@ -120,115 +77,6 @@ public void setValue(String valueString) { } }; - public static final Option PIPE_EVENT_REFERENCE_TRACKING_ENABLED = - new Option("pipe_event_reference_tracking_enabled", true) { - @Override - public void setValue(String valueString) { - value = Boolean.parseBoolean(valueString); - } - }; - - public static volatile Option PIPE_CHECK_MEMORY_ENOUGH_INTERVAL_MS = - new Option("pipe_check_memory_enough_interval_ms", 10L) { - @Override - public void setValue(final String valueString) { - value = Long.parseLong(valueString); - } - }; - - public static final Option PIPE_CONNECTOR_READ_FILE_BUFFER_MEMORY_CONTROL = - new Option("pipe_connector_read_file_buffer_memory_control", false) { - @Override - public void setValue(final String valueString) { - value = Boolean.parseBoolean(valueString); - } - }; - - public static final Option PIPE_SUBTASK_EXECUTOR_MAX_THREAD_NUM = - new Option( - "pipe_subtask_executor_max_thread_num", - Math.max(5, Runtime.getRuntime().availableProcessors() / 2)) { - @Override - public void setValue(final String valueString) { - value = Integer.parseInt(valueString); - } - }; - - public static final Option PIPE_DATA_STRUCTURE_TABLET_SIZE_IN_BYTES = - new Option("pipe_data_structure_tablet_size_in_bytes", 2097152) { - @Override - public void setValue(final String valueString) { - value = Integer.parseInt(valueString); - } - }; - - public static final Option PIPE_DATA_STRUCTURE_TABLET_ROW_SIZE = - new Option("pipe_data_structure_tablet_row_size", 2048) { - @Override - public void setValue(final String valueString) { - value = Integer.parseInt(valueString); - } - }; - - public static final Option PIPE_AIR_GAP_RECEIVER_ENABLED = - new Option("pipe_air_gap_receiver_enabled", true) { - - @Override - public void setValue(final String valueString) { - value = Boolean.parseBoolean(valueString); - } - }; - - public static final Option PIPE_AIR_GAP_RECEIVER_PORT = - new Option("pipe_air_gap_receiver_port", 9780) { - @Override - public void setValue(final String valueString) { - value = Integer.parseInt(valueString); - } - }; - - public static final Option PIPE_SUBTASK_EXECUTOR_PENDING_QUEUE_MAX_BLOCKING_TIME_MS = - new Option("pipe_subtask_executor_pending_queue_max_blocking_time_ms", 1000L) { - @Override - public void setValue(final String valueString) { - value = Long.parseLong(valueString); - } - }; - - public static final Option TIMESTAMP_PRECISION_CHECK_ENABLED = - new Option("timestamp_precision_check_enabled", true) { - @Override - public void setValue(final String valueString) { - value = Boolean.parseBoolean(valueString); - } - }; - - public static final Option DN_CONNECTION_TIMEOUT_IN_MS = - new Option("dn_connection_timeout_in_ms", (int) TimeUnit.SECONDS.toMillis(60)) { - @Override - public void setValue(final String valueString) { - value = Integer.parseInt(valueString); - } - }; - - public static final Option IS_RPC_THRIFT_COMPRESSION_ENABLED = - new Option("is_rpc_thrift_compression_enabled", false) { - @Override - public void setValue(final String valueString) { - value = Boolean.parseBoolean(valueString); - } - }; - - public static final Option PIPE_CONNECTOR_REQUEST_SLICE_THRESHOLD_BYTES = - new Option( - "pipe_connector_request_slice_threshold_bytes", - (int) (RpcUtils.THRIFT_FRAME_MAX_SIZE * 0.8)) { - @Override - public void setValue(final String valueString) { - value = Integer.parseInt(valueString); - } - }; - public static final Option PIPE_CONNECTOR_HANDSHAKE_TIMEOUT_MS = new Option("pipe_connector_handshake_timeout_ms", 10 * 1000) { @Override @@ -237,34 +85,6 @@ public void setValue(final String valueString) { } }; - public static final Option PIPE_CONNECTOR_RPC_THRIFT_COMPRESSION_ENABLED = - new Option("pipe_connector_rpc_thrift_compression_enabled", false) { - @Override - public void setValue(final String valueString) { - value = Boolean.parseBoolean(valueString); - } - }; - - public static final Option PIPE_ASYNC_CONNECTOR_SELECTOR_NUMBER = - new Option( - "pipe_async_connector_selector_number", - Math.max(4, Runtime.getRuntime().availableProcessors() / 2)) { - @Override - public void setValue(final String valueString) { - value = Integer.parseInt(valueString); - } - }; - - public static final Option PIPE_ASYNC_CONNECTOR_MAX_CLIENT_NUMBER = - new Option( - "pipe_async_connector_max_client_number", - Math.max(16, Runtime.getRuntime().availableProcessors() / 2)) { - @Override - public void setValue(final String valueString) { - value = Integer.parseInt(valueString); - } - }; - public static final Option DATA_NODE_ID = new Option("data_node_id", -1) { @Override diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/config/PluginRuntimeOptions.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/config/PluginRuntimeOptions.java index 92a745eb..766da95b 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/config/PluginRuntimeOptions.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/config/PluginRuntimeOptions.java @@ -22,6 +22,7 @@ import java.io.File; public class PluginRuntimeOptions extends Options { + public static final Option PLUGIN_LIB_DIR = new Option("plugin_lib_dir", "system" + File.separator + "plugin") { @Override diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/config/TaskRuntimeOptions.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/config/TaskRuntimeOptions.java index 35dda43c..0b430819 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/config/TaskRuntimeOptions.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/config/TaskRuntimeOptions.java @@ -23,45 +23,24 @@ public class TaskRuntimeOptions extends Options { - public static final Option TASK_SOURCE_PARALLELISM_NUM = - new Option("task_source_parallelism_num", 4) { - @Override - public void setValue(final String valueString) { - value = Integer.parseInt(valueString); - } - }; + //////////////////////////// Param from rest request //////////////////////////// - public static final Option TASK_PROCESS_PARALLELISM_NUM = - new Option("task_process_parallelism_num", 4) { - @Override - public void setValue(final String valueString) { - value = Integer.parseInt(valueString); - } - }; + public static final String TASK_SOURCE_PARALLELISM_NUM_KEY = "source-parallelism-num"; + public static final Integer TASK_SOURCE_PARALLELISM_NUM_DEFAULT_VALUE = 4; - public static final Option TASK_SINK_PARALLELISM_NUM = - new Option("task_sink_parallelism_num", 4) { - @Override - public void setValue(final String valueString) { - value = Integer.parseInt(valueString); - } - }; + public static final String TASK_PROCESSOR_PARALLELISM_NUM_KEY = "processor-parallelism-num"; + public static final Integer TASK_PROCESSOR_PARALLELISM_NUM_DEFAULT_VALUE = 4; - public static final Option TASK_PROCESSOR_RING_BUFFER_SIZE = - new Option("task_processor_ring_buffer_size", 1024) { - @Override - public void setValue(final String valueString) { - value = Integer.parseInt(valueString); - } - }; + public static final String TASK_SINK_PARALLELISM_NUM_KEY = "sink-parallelism-num"; + public static final Integer TASK_SINK_PARALLELISM_NUM_DEFAULT_VALUE = 4; - public static final Option TASK_SINK_RING_BUFFER_SIZE = - new Option("task_sink_ring_buffer_size", 1024) { - @Override - public void setValue(final String valueString) { - value = Integer.parseInt(valueString); - } - }; + public static final String TASK_PROCESSOR_RING_BUFFER_SIZE_KEY = "processor-ring-buffer-size"; + public static final Integer TASK_PROCESSOR_RING_BUFFER_SIZE_DEFAULT_VALUE = 1024; + + public static final String TASK_SINK_RING_BUFFER_SIZE_KEY = "sink-ring-buffer-size"; + public static final Integer TASK_SINK_RING_BUFFER_SIZE_DEFAULT_VALUE = 1024; + + //////////////////////////// Param from application.properties //////////////////////////// public static final Option TASK_DATABASE_FILE_PATH = new Option( @@ -74,7 +53,7 @@ public void setValue(final String valueString) { }; public static final Option EXECUTOR_CRON_HEARTBEAT_EVENT_INTERVAL_SECONDS = - new Option("executor_cron_heartbeat_event_interval_seconds", 20L) { + new Option("executor_cron_heartbeat_event_interval_seconds", 5L) { @Override public void setValue(final String valueString) { value = Long.parseLong(valueString); diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/config/TrimProperties.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/config/TrimProperties.java index 1cf766d5..ae41b1b8 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/config/TrimProperties.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/config/TrimProperties.java @@ -23,6 +23,7 @@ import java.util.Properties; public class TrimProperties extends Properties { + @Override public synchronized Object get(Object key) { Object value = super.get(key); diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/client/IoTDBSyncClientManager.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/client/IoTDBSyncClientManager.java index e70ced28..95b8af2b 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/client/IoTDBSyncClientManager.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/client/IoTDBSyncClientManager.java @@ -179,7 +179,7 @@ private boolean initClientAndStatus( .setConnectionTimeoutMs( PipeRuntimeOptions.PIPE_CONNECTOR_HANDSHAKE_TIMEOUT_MS.value()) .setRpcThriftCompressionEnabled( - PipeRuntimeOptions.IS_PIPE_CONNECTOR_RPC_THRIFT_COMPRESSION_ENABLED.value()) + PipeRuntimeOptions.PIPE_CONNECTOR_RPC_THRIFT_COMPRESSION_ENABLED.value()) .build(), endPoint.getIp(), endPoint.getPort(), diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/payload/evolvable/batch/PipeTabletEventBatch.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/payload/evolvable/batch/PipeTabletEventBatch.java index 93e77b6f..0660cf96 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/payload/evolvable/batch/PipeTabletEventBatch.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/payload/evolvable/batch/PipeTabletEventBatch.java @@ -19,6 +19,7 @@ package org.apache.iotdb.collector.plugin.builtin.sink.payload.evolvable.batch; +import org.apache.iotdb.collector.config.PipeRuntimeOptions; import org.apache.iotdb.collector.plugin.builtin.sink.event.PipeRawTabletInsertionEvent; import org.apache.iotdb.collector.plugin.builtin.sink.resource.memory.PipeMemoryBlock; import org.apache.iotdb.collector.plugin.builtin.sink.resource.memory.PipeMemoryManager; @@ -115,8 +116,8 @@ public synchronized boolean onEvent(final TabletInsertionEvent event) throws IOE public boolean shouldEmit() { return totalBufferSize >= getMaxBatchSizeInBytes() - || System.currentTimeMillis() - firstEventProcessingTime >= maxDelayInMs; - // || events.size() > PipeRuntimeOptions.PIPE_MAX_ALLOWED_EVENT_COUNT_IN_TABLET_BATCH.value(); + || System.currentTimeMillis() - firstEventProcessingTime >= maxDelayInMs + || events.size() > PipeRuntimeOptions.PIPE_MAX_ALLOWED_EVENT_COUNT_IN_TABLET_BATCH.value(); } private long getMaxBatchSizeInBytes() { diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/runtime/task/processor/ProcessorTask.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/runtime/task/processor/ProcessorTask.java index 9c747e6f..613fb72c 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/runtime/task/processor/ProcessorTask.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/runtime/task/processor/ProcessorTask.java @@ -43,8 +43,10 @@ import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; -import static org.apache.iotdb.collector.config.TaskRuntimeOptions.TASK_PROCESSOR_RING_BUFFER_SIZE; -import static org.apache.iotdb.collector.config.TaskRuntimeOptions.TASK_PROCESS_PARALLELISM_NUM; +import static org.apache.iotdb.collector.config.TaskRuntimeOptions.TASK_PROCESSOR_PARALLELISM_NUM_DEFAULT_VALUE; +import static org.apache.iotdb.collector.config.TaskRuntimeOptions.TASK_PROCESSOR_PARALLELISM_NUM_KEY; +import static org.apache.iotdb.collector.config.TaskRuntimeOptions.TASK_PROCESSOR_RING_BUFFER_SIZE_DEFAULT_VALUE; +import static org.apache.iotdb.collector.config.TaskRuntimeOptions.TASK_PROCESSOR_RING_BUFFER_SIZE_KEY; public class ProcessorTask extends Task { @@ -64,10 +66,10 @@ public ProcessorTask( super( taskId, attributes, - TASK_PROCESS_PARALLELISM_NUM.key(), - attributes.containsKey(TASK_PROCESS_PARALLELISM_NUM.key()) - ? Integer.parseInt(TASK_PROCESS_PARALLELISM_NUM.key()) - : TASK_PROCESS_PARALLELISM_NUM.value()); + TASK_PROCESSOR_PARALLELISM_NUM_KEY, + attributes.containsKey(TASK_PROCESSOR_PARALLELISM_NUM_KEY) + ? Integer.parseInt(attributes.get(TASK_PROCESSOR_PARALLELISM_NUM_KEY)) + : TASK_PROCESSOR_PARALLELISM_NUM_DEFAULT_VALUE); REGISTERED_EXECUTOR_SERVICES.putIfAbsent( taskId, @@ -81,7 +83,9 @@ public ProcessorTask( disruptor = new Disruptor<>( EventContainer::new, - TASK_PROCESSOR_RING_BUFFER_SIZE.value(), + attributes.containsKey(TASK_PROCESSOR_RING_BUFFER_SIZE_KEY) + ? Integer.parseInt(attributes.get(TASK_PROCESSOR_RING_BUFFER_SIZE_KEY)) + : TASK_PROCESSOR_RING_BUFFER_SIZE_DEFAULT_VALUE, REGISTERED_EXECUTOR_SERVICES.get(taskId), ProducerType.MULTI, new BlockingWaitStrategy()); diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/runtime/task/sink/SinkTask.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/runtime/task/sink/SinkTask.java index 65a71192..573d642c 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/runtime/task/sink/SinkTask.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/runtime/task/sink/SinkTask.java @@ -40,8 +40,10 @@ import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; -import static org.apache.iotdb.collector.config.TaskRuntimeOptions.TASK_SINK_PARALLELISM_NUM; -import static org.apache.iotdb.collector.config.TaskRuntimeOptions.TASK_SINK_RING_BUFFER_SIZE; +import static org.apache.iotdb.collector.config.TaskRuntimeOptions.TASK_SINK_PARALLELISM_NUM_DEFAULT_VALUE; +import static org.apache.iotdb.collector.config.TaskRuntimeOptions.TASK_SINK_PARALLELISM_NUM_KEY; +import static org.apache.iotdb.collector.config.TaskRuntimeOptions.TASK_SINK_RING_BUFFER_SIZE_DEFAULT_VALUE; +import static org.apache.iotdb.collector.config.TaskRuntimeOptions.TASK_SINK_RING_BUFFER_SIZE_KEY; public class SinkTask extends Task { @@ -57,10 +59,10 @@ public SinkTask(final String taskId, final Map attributes) { super( taskId, attributes, - TASK_SINK_PARALLELISM_NUM.key(), - attributes.containsKey(TASK_SINK_PARALLELISM_NUM.key()) - ? Integer.parseInt(TASK_SINK_PARALLELISM_NUM.key()) - : TASK_SINK_PARALLELISM_NUM.value()); + TASK_SINK_PARALLELISM_NUM_KEY, + attributes.containsKey(TASK_SINK_PARALLELISM_NUM_KEY) + ? Integer.parseInt(attributes.get(TASK_SINK_PARALLELISM_NUM_KEY)) + : TASK_SINK_PARALLELISM_NUM_DEFAULT_VALUE); REGISTERED_EXECUTOR_SERVICES.putIfAbsent( taskId, @@ -74,7 +76,9 @@ public SinkTask(final String taskId, final Map attributes) { disruptor = new Disruptor<>( EventContainer::new, - TASK_SINK_RING_BUFFER_SIZE.value(), + attributes.containsKey(TASK_SINK_RING_BUFFER_SIZE_KEY) + ? Integer.parseInt(attributes.get(TASK_SINK_RING_BUFFER_SIZE_KEY)) + : TASK_SINK_RING_BUFFER_SIZE_DEFAULT_VALUE, REGISTERED_EXECUTOR_SERVICES.get(taskId), ProducerType.MULTI, new BlockingWaitStrategy()); diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/runtime/task/source/SourceTask.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/runtime/task/source/SourceTask.java index 206b4ca8..ed16fd44 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/runtime/task/source/SourceTask.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/runtime/task/source/SourceTask.java @@ -30,7 +30,8 @@ import java.util.Map; -import static org.apache.iotdb.collector.config.TaskRuntimeOptions.TASK_SOURCE_PARALLELISM_NUM; +import static org.apache.iotdb.collector.config.TaskRuntimeOptions.TASK_SOURCE_PARALLELISM_NUM_DEFAULT_VALUE; +import static org.apache.iotdb.collector.config.TaskRuntimeOptions.TASK_SOURCE_PARALLELISM_NUM_KEY; public abstract class SourceTask extends Task { @@ -45,10 +46,10 @@ protected SourceTask( super( taskId, attributes, - TASK_SOURCE_PARALLELISM_NUM.key(), - attributes.containsKey(TASK_SOURCE_PARALLELISM_NUM.key()) - ? Integer.parseInt(attributes.get(TASK_SOURCE_PARALLELISM_NUM.key())) - : TASK_SOURCE_PARALLELISM_NUM.value()); + TASK_SOURCE_PARALLELISM_NUM_KEY, + attributes.containsKey(TASK_SOURCE_PARALLELISM_NUM_KEY) + ? Integer.parseInt(attributes.get(TASK_SOURCE_PARALLELISM_NUM_KEY)) + : TASK_SOURCE_PARALLELISM_NUM_DEFAULT_VALUE); this.processorProducer = processorProducer; this.taskState = taskState; } From 94d4eb559d6ce6c8101a9b95d27a1dbf0d16077f Mon Sep 17 00:00:00 2001 From: ysc <2725843507@qq.com> Date: Thu, 5 Jun 2025 15:05:12 +0800 Subject: [PATCH 09/11] update iotdb subscription push source --- .../iotdb/collector/plugin/api/BaseSource.java | 4 ++++ .../source/iotdb/IoTDBSubscriptionCommon.java | 15 --------------- .../iotdb/IoTDBSubscriptionPullSource.java | 13 +++++++++---- .../iotdb/IoTDBSubscriptionPushSource.java | 18 +++++------------- .../IoTDBSubscriptionTablePushSource.java | 5 +++++ .../iotdb/IoTDBSubscriptionTreePushSource.java | 5 +++++ .../builtin/source/kafka/KafkaSource.java | 3 +-- 7 files changed, 29 insertions(+), 34 deletions(-) diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/api/BaseSource.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/api/BaseSource.java index a438e81e..dea3de00 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/api/BaseSource.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/api/BaseSource.java @@ -20,6 +20,7 @@ package org.apache.iotdb.collector.plugin.api; import org.apache.iotdb.collector.plugin.api.customizer.CollectorParameters; +import org.apache.iotdb.collector.plugin.api.customizer.CollectorRuntimeEnvironment; import org.apache.iotdb.collector.runtime.progress.ProgressIndex; import org.apache.iotdb.pipe.api.PipeSource; import org.apache.iotdb.pipe.api.customizer.configuration.PipeExtractorRuntimeConfiguration; @@ -81,6 +82,9 @@ public void customize( reportTimeInterval = pipeParameters.getIntOrDefault( SOURCE_REPORT_TIME_INTERVAL_KEY, SOURCE_REPORT_TIME_INTERVAL_DEFAULT_VALUE); + instanceIndex = + ((CollectorRuntimeEnvironment) pipeSourceRuntimeConfiguration.getRuntimeEnvironment()) + .getInstanceIndex(); } @Override diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionCommon.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionCommon.java index 473d65bb..f2d8af9b 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionCommon.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionCommon.java @@ -23,10 +23,6 @@ import org.apache.iotdb.pipe.api.customizer.configuration.PipeSourceRuntimeConfiguration; import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameterValidator; import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameters; -import org.apache.iotdb.pipe.api.event.Event; - -import java.util.concurrent.ArrayBlockingQueue; -import java.util.concurrent.BlockingQueue; import static org.apache.iotdb.collector.plugin.builtin.source.constant.SourceConstant.SOURCE_IS_ALIGNED_DEFAULT_VALUE; import static org.apache.iotdb.collector.plugin.builtin.source.constant.SourceConstant.SOURCE_IS_ALIGNED_KEY; @@ -68,9 +64,6 @@ public class IoTDBSubscriptionCommon { private String topic; - private static final Integer EVENT_QUEUE_CAPACITY = 1000; - private final BlockingQueue eventQueue = new ArrayBlockingQueue<>(EVENT_QUEUE_CAPACITY); - // validate common parameters public void validate(final PipeParameterValidator validator) { CollectorParameters.validateStringRequiredParam(validator, IOTDB_SUBSCRIPTION_SOURCE_TOPIC_KEY); @@ -153,14 +146,6 @@ public void customize( IOTDB_SUBSCRIPTION_SOURCE_MAX_POLL_PARALLELISM_DEFAULT_VALUE); } - public Event take() throws InterruptedException { - return eventQueue.take(); - } - - public void put(final Event event) throws InterruptedException { - eventQueue.put(event); - } - public String getHost() { return host; } diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionPullSource.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionPullSource.java index e68c223b..c5f28a10 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionPullSource.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionPullSource.java @@ -35,6 +35,8 @@ import org.slf4j.LoggerFactory; import java.util.List; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.BlockingQueue; import static org.apache.iotdb.collector.plugin.builtin.source.iotdb.IoTDBSubscriptionSourceConstant.IOTDB_SUBSCRIPTION_SOURCE_AUTO_COMMIT_DEFAULT_VALUE; import static org.apache.iotdb.collector.plugin.builtin.source.iotdb.IoTDBSubscriptionSourceConstant.IOTDB_SUBSCRIPTION_SOURCE_AUTO_COMMIT_INTERVAL_MS_DEFAULT_VALUE; @@ -56,6 +58,9 @@ public abstract class IoTDBSubscriptionPullSource extends PullSource { protected volatile boolean isStarted; protected Thread workerThread; + private static final Integer EVENT_QUEUE_CAPACITY = 1000; + private final BlockingQueue eventQueue = new ArrayBlockingQueue<>(EVENT_QUEUE_CAPACITY); + @Override public void validate(final PipeParameterValidator validator) throws Exception { super.validate(validator); @@ -114,7 +119,7 @@ private void doWork() { for (final SubscriptionMessage message : messages) { for (final SubscriptionSessionDataSet dataSet : message.getSessionDataSetsHandler()) { try { - subscription.put(new PipeRawTabletInsertionEvent(dataSet.getTablet(), isAligned)); + eventQueue.put(new PipeRawTabletInsertionEvent(dataSet.getTablet(), isAligned)); } catch (final InterruptedException e) { LOGGER.warn("{} thread interrupted", getPullConsumerThreadName(), e); Thread.currentThread().interrupt(); @@ -134,7 +139,7 @@ private void doWork() { @Override public Event supply() throws InterruptedException { - return subscription.take(); + return eventQueue.take(); } @Override @@ -155,7 +160,7 @@ protected SubscriptionTreePullConsumerBuilder getSubscriptionTreePullConsumerBui return new SubscriptionTreePullConsumerBuilder() .host(subscription.getHost()) .port(subscription.getPort()) - .consumerId(subscription.getConsumerId()) + .consumerId(subscription.getConsumerId() + instanceIndex) .consumerGroupId(subscription.getGroupId()) .heartbeatIntervalMs(subscription.getHeartbeatIntervalMs()) .endpointsSyncIntervalMs(subscription.getEndpointsSyncIntervalMs()) @@ -169,7 +174,7 @@ protected SubscriptionTablePullConsumerBuilder getSubscriptionTablePullConsumerB return new SubscriptionTablePullConsumerBuilder() .host(subscription.getHost()) .port(subscription.getPort()) - .consumerId(subscription.getConsumerId()) + .consumerId(subscription.getConsumerId() + instanceIndex) .consumerGroupId(subscription.getGroupId()) .heartbeatIntervalMs(subscription.getHeartbeatIntervalMs()) .endpointsSyncIntervalMs(subscription.getEndpointsSyncIntervalMs()) diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionPushSource.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionPushSource.java index ef358a05..d7b60599 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionPushSource.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionPushSource.java @@ -35,8 +35,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.concurrent.locks.LockSupport; - import static org.apache.iotdb.collector.plugin.builtin.source.iotdb.IoTDBSubscriptionSourceConstant.IOTDB_SUBSCRIPTION_SOURCE_ACK_STRATEGY_DEFAULT_VALUE; import static org.apache.iotdb.collector.plugin.builtin.source.iotdb.IoTDBSubscriptionSourceConstant.IOTDB_SUBSCRIPTION_SOURCE_ACK_STRATEGY_KEY; import static org.apache.iotdb.collector.plugin.builtin.source.iotdb.IoTDBSubscriptionSourceConstant.IOTDB_SUBSCRIPTION_SOURCE_ACK_STRATEGY_VALUE_MAP; @@ -127,19 +125,13 @@ protected ConsumeListener getConsumeListener() { try { markPausePosition(); - subscription.put(new PipeRawTabletInsertionEvent(dataSet.getTablet(), isAligned)); - } catch (final InterruptedException e) { - LOGGER.warn("{} thread interrupted", getPushConsumerThreadName(), e); - Thread.currentThread().interrupt(); - + supply(new PipeRawTabletInsertionEvent(dataSet.getTablet(), isAligned)); + } catch (final Exception e) { + LOGGER.warn("Error occurred when supply event, because {}", e.getMessage()); return ConsumeResult.FAILURE; } } - while (isStarted && !Thread.currentThread().isInterrupted()) { - LockSupport.park(); - } - return ConsumeResult.SUCCESS; }; } @@ -154,8 +146,8 @@ protected SubscriptionTreePushConsumerBuilder getSubscriptionTreePushConsumerBui .endpointsSyncIntervalMs(subscription.getEndpointsSyncIntervalMs()) .thriftMaxFrameSize(subscription.getThriftMaxFrameSize()) .maxPollParallelism(subscription.getMaxPollParallelism()) - .consumeListener(getConsumeListener()) .ackStrategy(ackStrategy) + .consumeListener(getConsumeListener()) .autoPollIntervalMs(autoPollIntervalMs) .autoPollTimeoutMs(autoPollTimeoutMs); } @@ -170,8 +162,8 @@ protected SubscriptionTablePushConsumerBuilder getSubscriptionTablePushConsumer( .endpointsSyncIntervalMs(subscription.getEndpointsSyncIntervalMs()) .thriftMaxFrameSize(subscription.getThriftMaxFrameSize()) .maxPollParallelism(subscription.getMaxPollParallelism()) - .consumeListener(getConsumeListener()) .ackStrategy(ackStrategy) + .consumeListener(getConsumeListener()) .autoPollIntervalMs(autoPollIntervalMs) .autoPollTimeoutMs(autoPollTimeoutMs); } diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTablePushSource.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTablePushSource.java index 61955fe4..1f5073ae 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTablePushSource.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTablePushSource.java @@ -26,6 +26,7 @@ import org.slf4j.LoggerFactory; import java.util.Optional; +import java.util.concurrent.locks.LockSupport; public class IoTDBSubscriptionTablePushSource extends IoTDBSubscriptionPushSource { @@ -37,6 +38,10 @@ protected void doWork() { getSubscriptionTablePushConsumer().buildTablePushConsumer()) { consumer.open(); consumer.subscribe(subscription.getTopic()); + + while (!Thread.currentThread().isInterrupted()) { + LockSupport.park(); + } } catch (final Exception e) { LOGGER.warn("Error occurred while {} thread", getPushConsumerThreadName(), e); } diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTreePushSource.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTreePushSource.java index 27deb13e..f07f3e2e 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTreePushSource.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTreePushSource.java @@ -26,6 +26,7 @@ import org.slf4j.LoggerFactory; import java.util.Optional; +import java.util.concurrent.locks.LockSupport; public class IoTDBSubscriptionTreePushSource extends IoTDBSubscriptionPushSource { @@ -37,6 +38,10 @@ protected void doWork() { getSubscriptionTreePushConsumerBuilder().buildPushConsumer()) { consumer.open(); consumer.subscribe(subscription.getTopic()); + + while (!Thread.currentThread().isInterrupted()) { + LockSupport.park(); + } } catch (final Exception e) { LOGGER.warn("Error occurred while {} thread", getPushConsumerThreadName(), e); } diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/kafka/KafkaSource.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/kafka/KafkaSource.java index 2cf80bd8..85c54b8a 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/kafka/KafkaSource.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/kafka/KafkaSource.java @@ -138,9 +138,8 @@ public void customize( throws Exception { final CollectorRuntimeEnvironment environment = (CollectorRuntimeEnvironment) pipeSourceRuntimeConfiguration.getRuntimeEnvironment(); - final String taskId = environment.getPipeName(); - instanceIndex = environment.getInstanceIndex(); + startIndex = RuntimeService.progress().isPresent() ? RuntimeService.progress().get().getInstanceProgressIndex(taskId, instanceIndex) From 0f5fe3415b40ee780a90ff61ef6a48fccb1769fd Mon Sep 17 00:00:00 2001 From: ysc <2725843507@qq.com> Date: Fri, 6 Jun 2025 15:14:42 +0800 Subject: [PATCH 10/11] update close method --- .../source/iotdb/IoTDBSubscriptionPushSource.java | 9 ++++++--- .../source/iotdb/IoTDBSubscriptionTablePushSource.java | 5 +---- .../source/iotdb/IoTDBSubscriptionTreePushSource.java | 5 +---- 3 files changed, 8 insertions(+), 11 deletions(-) diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionPushSource.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionPushSource.java index d7b60599..266e47e3 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionPushSource.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionPushSource.java @@ -35,6 +35,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.concurrent.CountDownLatch; + import static org.apache.iotdb.collector.plugin.builtin.source.iotdb.IoTDBSubscriptionSourceConstant.IOTDB_SUBSCRIPTION_SOURCE_ACK_STRATEGY_DEFAULT_VALUE; import static org.apache.iotdb.collector.plugin.builtin.source.iotdb.IoTDBSubscriptionSourceConstant.IOTDB_SUBSCRIPTION_SOURCE_ACK_STRATEGY_KEY; import static org.apache.iotdb.collector.plugin.builtin.source.iotdb.IoTDBSubscriptionSourceConstant.IOTDB_SUBSCRIPTION_SOURCE_ACK_STRATEGY_VALUE_MAP; @@ -48,7 +50,6 @@ public abstract class IoTDBSubscriptionPushSource extends PushSource { private static final Logger LOGGER = LoggerFactory.getLogger(IoTDBSubscriptionPushSource.class); - protected volatile boolean isStarted = true; protected Thread workerThread; private Long autoPollIntervalMs; @@ -57,6 +58,8 @@ public abstract class IoTDBSubscriptionPushSource extends PushSource { protected final IoTDBSubscriptionCommon subscription = new IoTDBSubscriptionCommon(); + protected final CountDownLatch shutdownLatch = new CountDownLatch(1); + @Override public void validate(final PipeParameterValidator validator) throws Exception { super.validate(validator); @@ -107,7 +110,6 @@ public void customize( @Override public void start() throws Exception { if (workerThread == null || !workerThread.isAlive()) { - isStarted = true; workerThread = new Thread(this::doWork); workerThread.setName(getPushConsumerThreadName()); @@ -170,7 +172,8 @@ protected SubscriptionTablePushConsumerBuilder getSubscriptionTablePushConsumer( @Override public void close() throws Exception { - isStarted = false; + shutdownLatch.countDown(); + if (workerThread != null) { workerThread.interrupt(); try { diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTablePushSource.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTablePushSource.java index 1f5073ae..660f8f1d 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTablePushSource.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTablePushSource.java @@ -26,7 +26,6 @@ import org.slf4j.LoggerFactory; import java.util.Optional; -import java.util.concurrent.locks.LockSupport; public class IoTDBSubscriptionTablePushSource extends IoTDBSubscriptionPushSource { @@ -39,9 +38,7 @@ protected void doWork() { consumer.open(); consumer.subscribe(subscription.getTopic()); - while (!Thread.currentThread().isInterrupted()) { - LockSupport.park(); - } + shutdownLatch.await(); } catch (final Exception e) { LOGGER.warn("Error occurred while {} thread", getPushConsumerThreadName(), e); } diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTreePushSource.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTreePushSource.java index f07f3e2e..f1f256a2 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTreePushSource.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTreePushSource.java @@ -26,7 +26,6 @@ import org.slf4j.LoggerFactory; import java.util.Optional; -import java.util.concurrent.locks.LockSupport; public class IoTDBSubscriptionTreePushSource extends IoTDBSubscriptionPushSource { @@ -39,9 +38,7 @@ protected void doWork() { consumer.open(); consumer.subscribe(subscription.getTopic()); - while (!Thread.currentThread().isInterrupted()) { - LockSupport.park(); - } + shutdownLatch.await(); } catch (final Exception e) { LOGGER.warn("Error occurred while {} thread", getPushConsumerThreadName(), e); } From 9540880893d3fe91b2e2d02d3cb7dc2524ddf2b2 Mon Sep 17 00:00:00 2001 From: ysc <2725843507@qq.com> Date: Wed, 11 Jun 2025 14:25:38 +0800 Subject: [PATCH 11/11] fix close source but not unsubscribe topic --- .../builtin/sink/event/PipeInsertionEvent.java | 7 ++++++- .../sink/event/PipeRawTabletInsertionEvent.java | 8 -------- .../iotdb/IoTDBSubscriptionTablePullSource.java | 10 ++++++++++ .../iotdb/IoTDBSubscriptionTablePushSource.java | 2 ++ .../iotdb/IoTDBSubscriptionTreePullSource.java | 13 +++++++++++-- .../iotdb/IoTDBSubscriptionTreePushSource.java | 2 ++ .../utils/{PathUtil.java => PathUtils.java} | 13 ++++++++++++- 7 files changed, 43 insertions(+), 12 deletions(-) rename iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/utils/{PathUtil.java => PathUtils.java} (68%) diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/event/PipeInsertionEvent.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/event/PipeInsertionEvent.java index 1027ece7..65166b27 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/event/PipeInsertionEvent.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/event/PipeInsertionEvent.java @@ -19,10 +19,13 @@ package org.apache.iotdb.collector.plugin.builtin.sink.event; +import org.apache.iotdb.collector.utils.PathUtils; + public abstract class PipeInsertionEvent { protected Boolean isTableModelEvent; + protected String sourceDatabaseNameFromDataRegion; protected String treeModelDatabaseName; protected String tableModelDatabaseName; @@ -35,6 +38,8 @@ public String getTreeModelDatabaseName() { } public String getTableModelDatabaseName() { - return tableModelDatabaseName; + return tableModelDatabaseName == null + ? tableModelDatabaseName = PathUtils.unQualifyDatabaseName(sourceDatabaseNameFromDataRegion) + : tableModelDatabaseName; } } diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/event/PipeRawTabletInsertionEvent.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/event/PipeRawTabletInsertionEvent.java index b533e084..35ba7d41 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/event/PipeRawTabletInsertionEvent.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/sink/event/PipeRawTabletInsertionEvent.java @@ -121,18 +121,10 @@ public Tablet convertToTablet() { return tablet; } - public boolean isTableModelEvent() { - return false; - } - public String getDeviceId() { return tablet.getDeviceId(); } - public String getTableModelDatabaseName() { - return null; - } - @Override public void close() {} } diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTablePullSource.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTablePullSource.java index e5352e8a..e16c3a47 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTablePullSource.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTablePullSource.java @@ -56,4 +56,14 @@ protected void commitAsync(final List messages) { public Optional report() { return Optional.empty(); } + + @Override + public void close() throws Exception { + super.close(); + + if (consumer != null) { + consumer.unsubscribe(subscription.getTopic()); + consumer.close(); + } + } } diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTablePushSource.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTablePushSource.java index 660f8f1d..d6b939a1 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTablePushSource.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTablePushSource.java @@ -39,6 +39,8 @@ protected void doWork() { consumer.subscribe(subscription.getTopic()); shutdownLatch.await(); + + consumer.unsubscribe(subscription.getTopic()); } catch (final Exception e) { LOGGER.warn("Error occurred while {} thread", getPushConsumerThreadName(), e); } diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTreePullSource.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTreePullSource.java index 916c952f..72ebdd03 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTreePullSource.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTreePullSource.java @@ -23,7 +23,6 @@ import org.apache.iotdb.session.subscription.consumer.ISubscriptionTreePullConsumer; import org.apache.iotdb.session.subscription.payload.SubscriptionMessage; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Optional; @@ -48,7 +47,7 @@ protected String getPullConsumerThreadName() { @Override protected List poll() { - return consumer.poll(Collections.singleton(subscription.getTopic()), POLL_TIMEOUT_MS); + return consumer.poll(POLL_TIMEOUT_MS); } @Override @@ -68,4 +67,14 @@ public Optional report() { }); return Optional.of(progress); } + + @Override + public void close() throws Exception { + super.close(); + + if (consumer != null) { + consumer.unsubscribe(subscription.getTopic()); + consumer.close(); + } + } } diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTreePushSource.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTreePushSource.java index f1f256a2..fcc61ed2 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTreePushSource.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/plugin/builtin/source/iotdb/IoTDBSubscriptionTreePushSource.java @@ -39,6 +39,8 @@ protected void doWork() { consumer.subscribe(subscription.getTopic()); shutdownLatch.await(); + + consumer.unsubscribe(subscription.getTopic()); } catch (final Exception e) { LOGGER.warn("Error occurred while {} thread", getPushConsumerThreadName(), e); } diff --git a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/utils/PathUtil.java b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/utils/PathUtils.java similarity index 68% rename from iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/utils/PathUtil.java rename to iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/utils/PathUtils.java index a0ab021d..e155cd7c 100644 --- a/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/utils/PathUtil.java +++ b/iotdb-collector/collector-core/src/main/java/org/apache/iotdb/collector/utils/PathUtils.java @@ -19,4 +19,15 @@ package org.apache.iotdb.collector.utils; -public class PathUtil {} +public class PathUtils { + public static boolean isTableModelDatabase(final String databaseName) { + return !databaseName.startsWith("root."); + } + + public static String unQualifyDatabaseName(String databaseName) { + if (databaseName != null && databaseName.startsWith("root.")) { + databaseName = databaseName.substring(5).toLowerCase(); + } + return databaseName; + } +}