EDQS: refactoring, OOM handling, healthcheck
This commit is contained in:
parent
342280105a
commit
bb7ea08e59
@ -72,7 +72,7 @@ public class TbRuleEngineQueueConsumerManager extends MainQueueConsumerManager<T
|
|||||||
ExecutorService consumerExecutor,
|
ExecutorService consumerExecutor,
|
||||||
ScheduledExecutorService scheduler,
|
ScheduledExecutorService scheduler,
|
||||||
ExecutorService taskExecutor) {
|
ExecutorService taskExecutor) {
|
||||||
super(queueKey, null, null, ctx.getQueueFactory()::createToRuleEngineMsgConsumer, consumerExecutor, scheduler, taskExecutor);
|
super(queueKey, null, null, ctx.getQueueFactory()::createToRuleEngineMsgConsumer, consumerExecutor, scheduler, taskExecutor, null);
|
||||||
this.ctx = ctx;
|
this.ctx = ctx;
|
||||||
this.stats = new TbRuleEngineConsumerStats(queueKey, ctx.getStatsFactory());
|
this.stats = new TbRuleEngineConsumerStats(queueKey, ctx.getStatsFactory());
|
||||||
}
|
}
|
||||||
|
|||||||
@ -28,6 +28,7 @@
|
|||||||
<logger name="org.thingsboard.server" level="INFO"/>
|
<logger name="org.thingsboard.server" level="INFO"/>
|
||||||
<logger name="org.apache.kafka.common.utils.AppInfoParser" level="WARN"/>
|
<logger name="org.apache.kafka.common.utils.AppInfoParser" level="WARN"/>
|
||||||
<logger name="org.apache.kafka.clients" level="WARN"/>
|
<logger name="org.apache.kafka.clients" level="WARN"/>
|
||||||
|
<logger name="org.thingsboard.server.actors" level="WARN"/>
|
||||||
<!-- To enable the logging of scanned rule engine components-->
|
<!-- To enable the logging of scanned rule engine components-->
|
||||||
<!-- <logger name="org.thingsboard.server.service.component.AnnotationComponentDiscoveryService" level="DEBUG" />-->
|
<!-- <logger name="org.thingsboard.server.service.component.AnnotationComponentDiscoveryService" level="DEBUG" />-->
|
||||||
<!-- To enable the debug logging of rule node upgrade -->
|
<!-- To enable the debug logging of rule node upgrade -->
|
||||||
|
|||||||
@ -28,7 +28,6 @@ import org.thingsboard.server.dao.service.DaoSqlTest;
|
|||||||
import org.thingsboard.server.edqs.util.EdqsRocksDb;
|
import org.thingsboard.server.edqs.util.EdqsRocksDb;
|
||||||
|
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
import java.util.function.BiPredicate;
|
|
||||||
|
|
||||||
import static org.awaitility.Awaitility.await;
|
import static org.awaitility.Awaitility.await;
|
||||||
|
|
||||||
@ -45,7 +44,7 @@ public class EdqsEntityQueryControllerTest extends EntityQueryControllerTest {
|
|||||||
@Autowired
|
@Autowired
|
||||||
private EdqsService edqsService;
|
private EdqsService edqsService;
|
||||||
|
|
||||||
@MockBean
|
@MockBean // so that we don't do backup for tests
|
||||||
private EdqsRocksDb edqsRocksDb;
|
private EdqsRocksDb edqsRocksDb;
|
||||||
|
|
||||||
@Before
|
@Before
|
||||||
|
|||||||
@ -15,9 +15,8 @@
|
|||||||
*/
|
*/
|
||||||
package org.thingsboard.server.common.data;
|
package org.thingsboard.server.common.data;
|
||||||
|
|
||||||
import java.util.Arrays;
|
|
||||||
import java.util.EnumSet;
|
import java.util.EnumSet;
|
||||||
import java.util.HashSet;
|
import java.util.List;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
|
||||||
public enum ObjectType {
|
public enum ObjectType {
|
||||||
@ -68,12 +67,13 @@ public enum ObjectType {
|
|||||||
TENANT, TENANT_PROFILE, CUSTOMER, DEVICE_PROFILE, DEVICE, ASSET_PROFILE, ASSET, EDGE, ENTITY_VIEW, USER, DASHBOARD,
|
TENANT, TENANT_PROFILE, CUSTOMER, DEVICE_PROFILE, DEVICE, ASSET_PROFILE, ASSET, EDGE, ENTITY_VIEW, USER, DASHBOARD,
|
||||||
RULE_CHAIN, WIDGET_TYPE, WIDGETS_BUNDLE, API_USAGE_STATE, QUEUE_STATS
|
RULE_CHAIN, WIDGET_TYPE, WIDGETS_BUNDLE, API_USAGE_STATE, QUEUE_STATS
|
||||||
);
|
);
|
||||||
public static final Set<ObjectType> edqsTypes = new HashSet<>(edqsTenantTypes);
|
public static final Set<ObjectType> edqsTypes = EnumSet.copyOf(edqsTenantTypes);
|
||||||
public static final Set<ObjectType> edqsSystemTypes = EnumSet.of(TENANT, TENANT_PROFILE, USER, DASHBOARD,
|
public static final Set<ObjectType> edqsSystemTypes = EnumSet.of(TENANT, TENANT_PROFILE, USER, DASHBOARD,
|
||||||
API_USAGE_STATE, ATTRIBUTE_KV, LATEST_TS_KV);
|
API_USAGE_STATE, ATTRIBUTE_KV, LATEST_TS_KV);
|
||||||
|
public static final Set<ObjectType> unversionedTypes = EnumSet.of(QUEUE_STATS);
|
||||||
|
|
||||||
static {
|
static {
|
||||||
edqsTypes.addAll(Arrays.asList(RELATION, ATTRIBUTE_KV, LATEST_TS_KV));
|
edqsTypes.addAll(List.of(RELATION, ATTRIBUTE_KV, LATEST_TS_KV));
|
||||||
}
|
}
|
||||||
|
|
||||||
public EntityType toEntityType() {
|
public EntityType toEntityType() {
|
||||||
|
|||||||
@ -68,6 +68,10 @@
|
|||||||
<groupId>org.thingsboard.common</groupId>
|
<groupId>org.thingsboard.common</groupId>
|
||||||
<artifactId>queue</artifactId>
|
<artifactId>queue</artifactId>
|
||||||
</dependency>
|
</dependency>
|
||||||
|
<dependency>
|
||||||
|
<groupId>org.springframework.boot</groupId>
|
||||||
|
<artifactId>spring-boot-starter-web</artifactId>
|
||||||
|
</dependency>
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>org.apache.kafka</groupId>
|
<groupId>org.apache.kafka</groupId>
|
||||||
<artifactId>kafka-clients</artifactId>
|
<artifactId>kafka-clients</artifactId>
|
||||||
|
|||||||
@ -21,10 +21,12 @@ import com.google.common.util.concurrent.ListeningExecutorService;
|
|||||||
import com.google.common.util.concurrent.MoreExecutors;
|
import com.google.common.util.concurrent.MoreExecutors;
|
||||||
import jakarta.annotation.PostConstruct;
|
import jakarta.annotation.PostConstruct;
|
||||||
import jakarta.annotation.PreDestroy;
|
import jakarta.annotation.PreDestroy;
|
||||||
|
import lombok.Getter;
|
||||||
import lombok.RequiredArgsConstructor;
|
import lombok.RequiredArgsConstructor;
|
||||||
import lombok.extern.slf4j.Slf4j;
|
import lombok.extern.slf4j.Slf4j;
|
||||||
import org.apache.commons.lang3.exception.ExceptionUtils;
|
import org.apache.commons.lang3.exception.ExceptionUtils;
|
||||||
import org.springframework.beans.factory.annotation.Autowired;
|
import org.springframework.beans.factory.annotation.Autowired;
|
||||||
|
import org.springframework.context.ConfigurableApplicationContext;
|
||||||
import org.springframework.context.annotation.Lazy;
|
import org.springframework.context.annotation.Lazy;
|
||||||
import org.springframework.context.event.EventListener;
|
import org.springframework.context.event.EventListener;
|
||||||
import org.springframework.stereotype.Service;
|
import org.springframework.stereotype.Service;
|
||||||
@ -72,6 +74,7 @@ import java.util.concurrent.ExecutorService;
|
|||||||
import java.util.concurrent.Executors;
|
import java.util.concurrent.Executors;
|
||||||
import java.util.concurrent.ScheduledExecutorService;
|
import java.util.concurrent.ScheduledExecutorService;
|
||||||
import java.util.concurrent.atomic.AtomicInteger;
|
import java.util.concurrent.atomic.AtomicInteger;
|
||||||
|
import java.util.function.Consumer;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
@EdqsComponent
|
@EdqsComponent
|
||||||
@ -85,8 +88,8 @@ public class EdqsProcessor implements TbQueueHandler<TbProtoQueueMsg<ToEdqsMsg>,
|
|||||||
private final EdqRepository repository;
|
private final EdqRepository repository;
|
||||||
private final EdqsConfig config;
|
private final EdqsConfig config;
|
||||||
private final EdqsPartitionService partitionService;
|
private final EdqsPartitionService partitionService;
|
||||||
@Autowired
|
private final ConfigurableApplicationContext applicationContext;
|
||||||
@Lazy
|
@Autowired @Lazy
|
||||||
private EdqsStateService stateService;
|
private EdqsStateService stateService;
|
||||||
|
|
||||||
private MainQueueConsumerManager<TbProtoQueueMsg<ToEdqsMsg>, QueueConfig> eventsConsumer;
|
private MainQueueConsumerManager<TbProtoQueueMsg<ToEdqsMsg>, QueueConfig> eventsConsumer;
|
||||||
@ -96,17 +99,30 @@ public class EdqsProcessor implements TbQueueHandler<TbProtoQueueMsg<ToEdqsMsg>,
|
|||||||
private ExecutorService mgmtExecutor;
|
private ExecutorService mgmtExecutor;
|
||||||
private ScheduledExecutorService scheduler;
|
private ScheduledExecutorService scheduler;
|
||||||
private ListeningExecutorService requestExecutor;
|
private ListeningExecutorService requestExecutor;
|
||||||
|
private ExecutorService repartitionExecutor;
|
||||||
|
|
||||||
private final VersionsStore versionsStore = new VersionsStore();
|
private final VersionsStore versionsStore = new VersionsStore();
|
||||||
|
|
||||||
private final AtomicInteger counter = new AtomicInteger(); // FIXME: TMP
|
private final AtomicInteger counter = new AtomicInteger(); // FIXME: TMP
|
||||||
|
|
||||||
|
@Getter
|
||||||
|
private Consumer<Throwable> errorHandler;
|
||||||
|
|
||||||
@PostConstruct
|
@PostConstruct
|
||||||
private void init() {
|
private void init() {
|
||||||
consumersExecutor = Executors.newCachedThreadPool(ThingsBoardThreadFactory.forName("edqs-consumer"));
|
consumersExecutor = Executors.newCachedThreadPool(ThingsBoardThreadFactory.forName("edqs-consumer"));
|
||||||
mgmtExecutor = ThingsBoardExecutors.newWorkStealingPool(4, "edqs-consumer-mgmt");
|
mgmtExecutor = ThingsBoardExecutors.newWorkStealingPool(4, "edqs-consumer-mgmt");
|
||||||
scheduler = ThingsBoardExecutors.newSingleThreadScheduledExecutor("edqs-scheduler");
|
scheduler = ThingsBoardExecutors.newSingleThreadScheduledExecutor("edqs-scheduler");
|
||||||
requestExecutor = MoreExecutors.listeningDecorator(ThingsBoardExecutors.newWorkStealingPool(12, "edqs-requests"));
|
requestExecutor = MoreExecutors.listeningDecorator(ThingsBoardExecutors.newWorkStealingPool(12, "edqs-requests"));
|
||||||
|
repartitionExecutor = Executors.newSingleThreadExecutor(ThingsBoardThreadFactory.forName("edqs-repartition"));
|
||||||
|
errorHandler = error -> {
|
||||||
|
if (error instanceof OutOfMemoryError) {
|
||||||
|
log.error("OOM detected, shutting down");
|
||||||
|
repository.clear();
|
||||||
|
Executors.newSingleThreadExecutor(ThingsBoardThreadFactory.forName("edqs-shutdown"))
|
||||||
|
.execute(applicationContext::close);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
eventsConsumer = MainQueueConsumerManager.<TbProtoQueueMsg<ToEdqsMsg>, QueueConfig>builder()
|
eventsConsumer = MainQueueConsumerManager.<TbProtoQueueMsg<ToEdqsMsg>, QueueConfig>builder()
|
||||||
.queueKey(new QueueKey(ServiceType.EDQS, EdqsQueue.EVENTS.getTopic()))
|
.queueKey(new QueueKey(ServiceType.EDQS, EdqsQueue.EVENTS.getTopic()))
|
||||||
@ -117,7 +133,7 @@ public class EdqsProcessor implements TbQueueHandler<TbProtoQueueMsg<ToEdqsMsg>,
|
|||||||
ToEdqsMsg msg = queueMsg.getValue();
|
ToEdqsMsg msg = queueMsg.getValue();
|
||||||
log.trace("Processing message: {}", msg);
|
log.trace("Processing message: {}", msg);
|
||||||
process(msg, EdqsQueue.EVENTS);
|
process(msg, EdqsQueue.EVENTS);
|
||||||
} catch (Throwable t) {
|
} catch (Exception t) {
|
||||||
log.error("Failed to process message: {}", queueMsg, t);
|
log.error("Failed to process message: {}", queueMsg, t);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -127,6 +143,7 @@ public class EdqsProcessor implements TbQueueHandler<TbProtoQueueMsg<ToEdqsMsg>,
|
|||||||
.consumerExecutor(consumersExecutor)
|
.consumerExecutor(consumersExecutor)
|
||||||
.taskExecutor(mgmtExecutor)
|
.taskExecutor(mgmtExecutor)
|
||||||
.scheduler(scheduler)
|
.scheduler(scheduler)
|
||||||
|
.uncaughtErrorHandler(errorHandler)
|
||||||
.build();
|
.build();
|
||||||
responseTemplate = queueFactory.createEdqsResponseTemplate();
|
responseTemplate = queueFactory.createEdqsResponseTemplate();
|
||||||
}
|
}
|
||||||
@ -141,7 +158,7 @@ public class EdqsProcessor implements TbQueueHandler<TbProtoQueueMsg<ToEdqsMsg>,
|
|||||||
if (event.getServiceType() != ServiceType.EDQS) {
|
if (event.getServiceType() != ServiceType.EDQS) {
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
consumersExecutor.submit(() -> {
|
repartitionExecutor.submit(() -> { // todo: maybe cancel the task if new event comes
|
||||||
try {
|
try {
|
||||||
Set<TopicPartitionInfo> newPartitions = event.getNewPartitions().get(new QueueKey(ServiceType.EDQS));
|
Set<TopicPartitionInfo> newPartitions = event.getNewPartitions().get(new QueueKey(ServiceType.EDQS));
|
||||||
Set<TopicPartitionInfo> partitions = newPartitions.stream()
|
Set<TopicPartitionInfo> partitions = newPartitions.stream()
|
||||||
@ -220,8 +237,8 @@ public class EdqsProcessor implements TbQueueHandler<TbProtoQueueMsg<ToEdqsMsg>,
|
|||||||
if (!versionsStore.isNew(key, version)) {
|
if (!versionsStore.isNew(key, version)) {
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
} else {
|
} else if (!ObjectType.unversionedTypes.contains(objectType)) {
|
||||||
log.warn("[{}] {} doesn't have version: {}", tenantId, objectType, edqsMsg);
|
log.warn("[{}] {} {} doesn't have version", tenantId, objectType, key);
|
||||||
}
|
}
|
||||||
if (queue != EdqsQueue.STATE) {
|
if (queue != EdqsQueue.STATE) {
|
||||||
stateService.save(tenantId, objectType, key, eventType, edqsMsg);
|
stateService.save(tenantId, objectType, key, eventType, edqsMsg);
|
||||||
@ -272,6 +289,7 @@ public class EdqsProcessor implements TbQueueHandler<TbProtoQueueMsg<ToEdqsMsg>,
|
|||||||
mgmtExecutor.shutdownNow();
|
mgmtExecutor.shutdownNow();
|
||||||
scheduler.shutdownNow();
|
scheduler.shutdownNow();
|
||||||
requestExecutor.shutdownNow();
|
requestExecutor.shutdownNow();
|
||||||
|
repartitionExecutor.shutdownNow();
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|||||||
@ -53,7 +53,7 @@ public class EdqsProducer {
|
|||||||
TbQueueCallback callback = new TbQueueCallback() {
|
TbQueueCallback callback = new TbQueueCallback() {
|
||||||
@Override
|
@Override
|
||||||
public void onSuccess(TbQueueMsgMetadata metadata) {
|
public void onSuccess(TbQueueMsgMetadata metadata) {
|
||||||
log.debug("[{}][{}][{}] Published msg to {}: {}", tenantId, type, key, topic, msg); // fixme log levels
|
log.trace("[{}][{}][{}] Published msg to {}: {}", tenantId, type, key, topic, msg); // fixme log levels
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|||||||
@ -153,7 +153,7 @@ public class TenantRepo {
|
|||||||
EntityData<?> to = getOrCreate(entity.getTo());
|
EntityData<?> to = getOrCreate(entity.getTo());
|
||||||
boolean added = repo.add(from, to, TbStringPool.intern(entity.getType()));
|
boolean added = repo.add(from, to, TbStringPool.intern(entity.getType()));
|
||||||
if (added) {
|
if (added) {
|
||||||
edqsStatsService.ifPresent(statService -> statService.reportTenantEdqsObject(tenantId, ObjectType.RELATION, EdqsEventType.UPDATED));
|
edqsStatsService.ifPresent(statService -> statService.reportEvent(tenantId, ObjectType.RELATION, EdqsEventType.UPDATED));
|
||||||
}
|
}
|
||||||
} else if (RelationTypeGroup.DASHBOARD.equals(entity.getTypeGroup())) {
|
} else if (RelationTypeGroup.DASHBOARD.equals(entity.getTypeGroup())) {
|
||||||
if (EntityRelation.CONTAINS_TYPE.equals(entity.getType()) && entity.getFrom().getEntityType() == EntityType.CUSTOMER) {
|
if (EntityRelation.CONTAINS_TYPE.equals(entity.getType()) && entity.getFrom().getEntityType() == EntityType.CUSTOMER) {
|
||||||
@ -172,7 +172,7 @@ public class TenantRepo {
|
|||||||
if (relationsRepo != null) {
|
if (relationsRepo != null) {
|
||||||
boolean removed = relationsRepo.remove(entityRelation.getFrom().getId(), entityRelation.getTo().getId(), entityRelation.getType());
|
boolean removed = relationsRepo.remove(entityRelation.getFrom().getId(), entityRelation.getTo().getId(), entityRelation.getType());
|
||||||
if (removed) {
|
if (removed) {
|
||||||
edqsStatsService.ifPresent(statService -> statService.reportTenantEdqsObject(tenantId, ObjectType.RELATION, EdqsEventType.DELETED));
|
edqsStatsService.ifPresent(statService -> statService.reportEvent(tenantId, ObjectType.RELATION, EdqsEventType.DELETED));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
} else if (RelationTypeGroup.DASHBOARD.equals(entityRelation.getTypeGroup())) {
|
} else if (RelationTypeGroup.DASHBOARD.equals(entityRelation.getTypeGroup())) {
|
||||||
@ -223,7 +223,7 @@ public class TenantRepo {
|
|||||||
EntityData<?> removed = getEntityMap(entityType).remove(entityId);
|
EntityData<?> removed = getEntityMap(entityType).remove(entityId);
|
||||||
if (removed != null) {
|
if (removed != null) {
|
||||||
getEntitySet(entityType).remove(removed);
|
getEntitySet(entityType).remove(removed);
|
||||||
edqsStatsService.ifPresent(statService -> statService.reportTenantEdqsObject(tenantId, ObjectType.fromEntityType(entityType), EdqsEventType.DELETED));
|
edqsStatsService.ifPresent(statService -> statService.reportEvent(tenantId, ObjectType.fromEntityType(entityType), EdqsEventType.DELETED));
|
||||||
UUID customerId = removed.getCustomerId();
|
UUID customerId = removed.getCustomerId();
|
||||||
if (customerId != null) {
|
if (customerId != null) {
|
||||||
CustomerData customerData = (CustomerData) getEntityMap(EntityType.CUSTOMER).get(customerId);
|
CustomerData customerData = (CustomerData) getEntityMap(EntityType.CUSTOMER).get(customerId);
|
||||||
@ -244,7 +244,7 @@ public class TenantRepo {
|
|||||||
Integer keyId = KeyDictionary.get(attributeKv.getKey());
|
Integer keyId = KeyDictionary.get(attributeKv.getKey());
|
||||||
boolean added = entityData.putAttr(keyId, attributeKv.getScope(), toDataPoint(attributeKv.getLastUpdateTs(), value));
|
boolean added = entityData.putAttr(keyId, attributeKv.getScope(), toDataPoint(attributeKv.getLastUpdateTs(), value));
|
||||||
if (added) {
|
if (added) {
|
||||||
edqsStatsService.ifPresent(statService -> statService.reportTenantEdqsObject(tenantId, ObjectType.ATTRIBUTE_KV, EdqsEventType.UPDATED));
|
edqsStatsService.ifPresent(statService -> statService.reportEvent(tenantId, ObjectType.ATTRIBUTE_KV, EdqsEventType.UPDATED));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -254,7 +254,7 @@ public class TenantRepo {
|
|||||||
if (entityData != null) {
|
if (entityData != null) {
|
||||||
boolean removed = entityData.removeAttr(KeyDictionary.get(attributeKv.getKey()), attributeKv.getScope());
|
boolean removed = entityData.removeAttr(KeyDictionary.get(attributeKv.getKey()), attributeKv.getScope());
|
||||||
if (removed) {
|
if (removed) {
|
||||||
edqsStatsService.ifPresent(statService -> statService.reportTenantEdqsObject(tenantId, ObjectType.ATTRIBUTE_KV, EdqsEventType.DELETED));
|
edqsStatsService.ifPresent(statService -> statService.reportEvent(tenantId, ObjectType.ATTRIBUTE_KV, EdqsEventType.DELETED));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -266,7 +266,7 @@ public class TenantRepo {
|
|||||||
Integer keyId = KeyDictionary.get(latestTsKv.getKey());
|
Integer keyId = KeyDictionary.get(latestTsKv.getKey());
|
||||||
boolean added = entityData.putTs(keyId, toDataPoint(latestTsKv.getTs(), value));
|
boolean added = entityData.putTs(keyId, toDataPoint(latestTsKv.getTs(), value));
|
||||||
if (added) {
|
if (added) {
|
||||||
edqsStatsService.ifPresent(statService -> statService.reportTenantEdqsObject(tenantId, ObjectType.LATEST_TS_KV, EdqsEventType.UPDATED));
|
edqsStatsService.ifPresent(statService -> statService.reportEvent(tenantId, ObjectType.LATEST_TS_KV, EdqsEventType.UPDATED));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -276,7 +276,7 @@ public class TenantRepo {
|
|||||||
if (entityData != null) {
|
if (entityData != null) {
|
||||||
boolean removed = entityData.removeTs(KeyDictionary.get(latestTsKv.getKey()));
|
boolean removed = entityData.removeTs(KeyDictionary.get(latestTsKv.getKey()));
|
||||||
if (removed) {
|
if (removed) {
|
||||||
edqsStatsService.ifPresent(statService -> statService.reportTenantEdqsObject(tenantId, ObjectType.LATEST_TS_KV, EdqsEventType.DELETED));
|
edqsStatsService.ifPresent(statService -> statService.reportEvent(tenantId, ObjectType.LATEST_TS_KV, EdqsEventType.DELETED));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -331,7 +331,7 @@ public class TenantRepo {
|
|||||||
log.debug("[{}] Adding {} {}", tenantId, entityType, id);
|
log.debug("[{}] Adding {} {}", tenantId, entityType, id);
|
||||||
EntityData<?> entityData = constructEntityData(entityType, entityId);
|
EntityData<?> entityData = constructEntityData(entityType, entityId);
|
||||||
getEntitySet(entityType).add(entityData);
|
getEntitySet(entityType).add(entityData);
|
||||||
edqsStatsService.ifPresent(statService -> statService.reportTenantEdqsObject(tenantId, ObjectType.fromEntityType(entityType), EdqsEventType.UPDATED));
|
edqsStatsService.ifPresent(statService -> statService.reportEvent(tenantId, ObjectType.fromEntityType(entityType), EdqsEventType.UPDATED));
|
||||||
return entityData;
|
return entityData;
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|||||||
@ -0,0 +1,40 @@
|
|||||||
|
/**
|
||||||
|
* Copyright © 2016-2024 The Thingsboard Authors
|
||||||
|
*
|
||||||
|
* Licensed 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.thingsboard.server.edqs.state;
|
||||||
|
|
||||||
|
import lombok.RequiredArgsConstructor;
|
||||||
|
import org.springframework.http.ResponseEntity;
|
||||||
|
import org.springframework.web.bind.annotation.GetMapping;
|
||||||
|
import org.springframework.web.bind.annotation.RequestMapping;
|
||||||
|
import org.springframework.web.bind.annotation.RestController;
|
||||||
|
|
||||||
|
@RestController
|
||||||
|
@RequiredArgsConstructor
|
||||||
|
@RequestMapping("/api/edqs")
|
||||||
|
public class EdqsController {
|
||||||
|
|
||||||
|
private final EdqsStateService edqsStateService;
|
||||||
|
|
||||||
|
@GetMapping("/ready")
|
||||||
|
public ResponseEntity<Void> isReady() {
|
||||||
|
if (edqsStateService.isReady()) {
|
||||||
|
return ResponseEntity.ok().build();
|
||||||
|
} else {
|
||||||
|
return ResponseEntity.badRequest().build();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
@ -29,4 +29,6 @@ public interface EdqsStateService {
|
|||||||
|
|
||||||
void save(TenantId tenantId, ObjectType type, String key, EdqsEventType eventType, ToEdqsMsg msg);
|
void save(TenantId tenantId, ObjectType type, String key, EdqsEventType eventType, ToEdqsMsg msg);
|
||||||
|
|
||||||
|
boolean isReady();
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|||||||
@ -42,10 +42,10 @@ import org.thingsboard.server.queue.edqs.EdqsConfig;
|
|||||||
import org.thingsboard.server.queue.edqs.EdqsQueue;
|
import org.thingsboard.server.queue.edqs.EdqsQueue;
|
||||||
import org.thingsboard.server.queue.edqs.EdqsQueueFactory;
|
import org.thingsboard.server.queue.edqs.EdqsQueueFactory;
|
||||||
import org.thingsboard.server.queue.edqs.KafkaEdqsComponent;
|
import org.thingsboard.server.queue.edqs.KafkaEdqsComponent;
|
||||||
import org.thingsboard.server.queue.util.AfterStartUp;
|
|
||||||
|
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.UUID;
|
import java.util.UUID;
|
||||||
|
import java.util.concurrent.CountDownLatch;
|
||||||
import java.util.concurrent.ExecutorService;
|
import java.util.concurrent.ExecutorService;
|
||||||
import java.util.concurrent.Executors;
|
import java.util.concurrent.Executors;
|
||||||
import java.util.concurrent.ScheduledExecutorService;
|
import java.util.concurrent.ScheduledExecutorService;
|
||||||
@ -66,6 +66,8 @@ public class KafkaEdqsStateService implements EdqsStateService {
|
|||||||
private QueueConsumerManager<TbProtoQueueMsg<ToEdqsMsg>> eventsConsumer;
|
private QueueConsumerManager<TbProtoQueueMsg<ToEdqsMsg>> eventsConsumer;
|
||||||
private EdqsProducer stateProducer;
|
private EdqsProducer stateProducer;
|
||||||
|
|
||||||
|
private boolean initialRestoreDone;
|
||||||
|
|
||||||
private ExecutorService consumersExecutor;
|
private ExecutorService consumersExecutor;
|
||||||
private ExecutorService mgmtExecutor;
|
private ExecutorService mgmtExecutor;
|
||||||
private ScheduledExecutorService scheduler;
|
private ScheduledExecutorService scheduler;
|
||||||
@ -76,7 +78,7 @@ public class KafkaEdqsStateService implements EdqsStateService {
|
|||||||
|
|
||||||
@PostConstruct
|
@PostConstruct
|
||||||
private void init() {
|
private void init() {
|
||||||
consumersExecutor = Executors.newCachedThreadPool(ThingsBoardThreadFactory.forName("edqs-backup-consumer"));
|
consumersExecutor = Executors.newCachedThreadPool(ThingsBoardThreadFactory.forName("edqs-consumer"));
|
||||||
mgmtExecutor = ThingsBoardExecutors.newWorkStealingPool(4, "edqs-backup-consumer-mgmt");
|
mgmtExecutor = ThingsBoardExecutors.newWorkStealingPool(4, "edqs-backup-consumer-mgmt");
|
||||||
scheduler = ThingsBoardExecutors.newSingleThreadScheduledExecutor("edqs-backup-scheduler");
|
scheduler = ThingsBoardExecutors.newSingleThreadScheduledExecutor("edqs-backup-scheduler");
|
||||||
|
|
||||||
@ -92,8 +94,8 @@ public class KafkaEdqsStateService implements EdqsStateService {
|
|||||||
if (stateReadCount.incrementAndGet() % 100000 == 0) {
|
if (stateReadCount.incrementAndGet() % 100000 == 0) {
|
||||||
log.info("[state] Processed {} msgs", stateReadCount.get());
|
log.info("[state] Processed {} msgs", stateReadCount.get());
|
||||||
}
|
}
|
||||||
} catch (Throwable t) {
|
} catch (Exception e) {
|
||||||
log.error("Failed to process message: {}", queueMsg, t);
|
log.error("Failed to process message: {}", queueMsg, e); // TODO: do something about the error - e.g. reprocess
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
consumer.commit();
|
consumer.commit();
|
||||||
@ -102,39 +104,48 @@ public class KafkaEdqsStateService implements EdqsStateService {
|
|||||||
.consumerExecutor(consumersExecutor)
|
.consumerExecutor(consumersExecutor)
|
||||||
.taskExecutor(mgmtExecutor)
|
.taskExecutor(mgmtExecutor)
|
||||||
.scheduler(scheduler)
|
.scheduler(scheduler)
|
||||||
|
.uncaughtErrorHandler(edqsProcessor.getErrorHandler())
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
eventsConsumer = QueueConsumerManager.<TbProtoQueueMsg<ToEdqsMsg>>builder() // FIXME Slavik writes to the state while we read it, slows down the start
|
ExecutorService backupExecutor = ThingsBoardExecutors.newLimitedTasksExecutor(12, 1000, "events-to-backup-executor");
|
||||||
|
eventsConsumer = QueueConsumerManager.<TbProtoQueueMsg<ToEdqsMsg>>builder() // FIXME Slavik writes to the state while we read it, slows down the start. maybe start backup consumer after restore is finished
|
||||||
.name("edqs-events-to-backup-consumer")
|
.name("edqs-events-to-backup-consumer")
|
||||||
.pollInterval(config.getPollInterval())
|
.pollInterval(config.getPollInterval())
|
||||||
.msgPackProcessor((msgs, consumer) -> {
|
.msgPackProcessor((msgs, consumer) -> {
|
||||||
|
CountDownLatch resultLatch = new CountDownLatch(msgs.size());
|
||||||
for (TbProtoQueueMsg<ToEdqsMsg> queueMsg : msgs) {
|
for (TbProtoQueueMsg<ToEdqsMsg> queueMsg : msgs) {
|
||||||
try {
|
backupExecutor.submit(() -> {
|
||||||
ToEdqsMsg msg = queueMsg.getValue();
|
try {
|
||||||
log.trace("Processing message: {}", msg);
|
ToEdqsMsg msg = queueMsg.getValue();
|
||||||
|
log.trace("Processing message: {}", msg);
|
||||||
|
|
||||||
if (msg.hasEventMsg()) {
|
if (msg.hasEventMsg()) {
|
||||||
EdqsEventMsg eventMsg = msg.getEventMsg();
|
EdqsEventMsg eventMsg = msg.getEventMsg();
|
||||||
String key = eventMsg.getKey();
|
String key = eventMsg.getKey();
|
||||||
if (eventsReadCount.incrementAndGet() % 100000 == 0) {
|
int count = eventsReadCount.incrementAndGet();
|
||||||
log.info("[events-to-backup] Processed {} msgs", eventsReadCount.get());
|
if (count % 100000 == 0) {
|
||||||
}
|
log.info("[events-to-backup] Processed {} msgs", count);
|
||||||
if (eventMsg.hasVersion()) {
|
}
|
||||||
if (!versionsStore.isNew(key, eventMsg.getVersion())) {
|
if (eventMsg.hasVersion()) {
|
||||||
return;
|
if (!versionsStore.isNew(key, eventMsg.getVersion())) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
|
||||||
|
|
||||||
TenantId tenantId = getTenantId(msg);
|
TenantId tenantId = getTenantId(msg);
|
||||||
ObjectType objectType = ObjectType.valueOf(eventMsg.getObjectType());
|
ObjectType objectType = ObjectType.valueOf(eventMsg.getObjectType());
|
||||||
EdqsEventType eventType = EdqsEventType.valueOf(eventMsg.getEventType());
|
EdqsEventType eventType = EdqsEventType.valueOf(eventMsg.getEventType());
|
||||||
log.debug("[{}] Saving to backup [{}] [{}] [{}]", tenantId, objectType, eventType, key);
|
log.debug("[{}] Saving to backup [{}] [{}] [{}]", tenantId, objectType, eventType, key);
|
||||||
stateProducer.send(tenantId, objectType, key, msg);
|
stateProducer.send(tenantId, objectType, key, msg);
|
||||||
|
}
|
||||||
|
} catch (Throwable t) {
|
||||||
|
log.error("Failed to process message: {}", queueMsg, t);
|
||||||
|
} finally {
|
||||||
|
resultLatch.countDown();
|
||||||
}
|
}
|
||||||
} catch (Throwable t) {
|
});
|
||||||
log.error("Failed to process message: {}", queueMsg, t);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
resultLatch.await();
|
||||||
consumer.commit();
|
consumer.commit();
|
||||||
})
|
})
|
||||||
.consumerCreator(() -> queueFactory.createEdqsMsgConsumer(EdqsQueue.EVENTS, "events-to-backup-consumer-group")) // shared by all instances consumer group
|
.consumerCreator(() -> queueFactory.createEdqsMsgConsumer(EdqsQueue.EVENTS, "events-to-backup-consumer-group")) // shared by all instances consumer group
|
||||||
@ -149,22 +160,21 @@ public class KafkaEdqsStateService implements EdqsStateService {
|
|||||||
.build();
|
.build();
|
||||||
}
|
}
|
||||||
|
|
||||||
@AfterStartUp(order = AfterStartUp.REGULAR_SERVICE)
|
|
||||||
public void afterStartUp() {
|
|
||||||
eventsConsumer.subscribe();
|
|
||||||
eventsConsumer.launch();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void restore(Set<TopicPartitionInfo> partitions) {
|
public void restore(Set<TopicPartitionInfo> partitions) {
|
||||||
stateReadCount.set(0); //TODO Slavik: do not support remote mode in monolith setup
|
stateReadCount.set(0); //TODO Slavik: do not support remote mode in monolith setup
|
||||||
long startTs = System.currentTimeMillis();
|
long startTs = System.currentTimeMillis();
|
||||||
log.info("Restore started for partitions {}", partitions.stream().map(tpi -> tpi.getPartition().orElse(-1)).sorted().toList());
|
log.info("Restore started for partitions {}", partitions.stream().map(tpi -> tpi.getPartition().orElse(-1)).sorted().toList());
|
||||||
|
|
||||||
stateConsumer.doUpdate(partitions); // calling blocking doUpdate instead of update
|
stateConsumer.doUpdate(partitions); // calling blocking doUpdate instead of update
|
||||||
stateConsumer.awaitStop(0); // consumers should stop on their own because EdqsQueue.STATE.stopWhenRead is true, we just need to wait
|
stateConsumer.awaitStop(0); // consumers should stop on their own because EdqsQueue.STATE.stopWhenRead is true, we just need to wait
|
||||||
|
|
||||||
log.info("Restore finished in {} ms. Processed {} msgs", (System.currentTimeMillis() - startTs), stateReadCount.get());
|
log.info("Restore finished in {} ms. Processed {} msgs", (System.currentTimeMillis() - startTs), stateReadCount.get());
|
||||||
|
|
||||||
|
if (!initialRestoreDone) {
|
||||||
|
initialRestoreDone = true;
|
||||||
|
|
||||||
|
eventsConsumer.subscribe();
|
||||||
|
eventsConsumer.launch();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@ -172,6 +182,11 @@ public class KafkaEdqsStateService implements EdqsStateService {
|
|||||||
// do nothing here, backup is done by events consumer
|
// do nothing here, backup is done by events consumer
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean isReady() {
|
||||||
|
return initialRestoreDone;
|
||||||
|
}
|
||||||
|
|
||||||
private TenantId getTenantId(ToEdqsMsg edqsMsg) {
|
private TenantId getTenantId(ToEdqsMsg edqsMsg) {
|
||||||
return TenantId.fromUUID(new UUID(edqsMsg.getTenantIdMSB(), edqsMsg.getTenantIdLSB()));
|
return TenantId.fromUUID(new UUID(edqsMsg.getTenantIdMSB(), edqsMsg.getTenantIdLSB()));
|
||||||
}
|
}
|
||||||
|
|||||||
@ -43,13 +43,11 @@ public class LocalEdqsStateService implements EdqsStateService {
|
|||||||
@Autowired
|
@Autowired
|
||||||
private EdqsRocksDb db;
|
private EdqsRocksDb db;
|
||||||
|
|
||||||
private Set<TopicPartitionInfo> partitions;
|
private boolean restoreDone;
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void restore(Set<TopicPartitionInfo> partitions) {
|
public void restore(Set<TopicPartitionInfo> partitions) {
|
||||||
if (this.partitions == null) {
|
if (restoreDone) {
|
||||||
this.partitions = partitions;
|
|
||||||
} else {
|
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -62,6 +60,7 @@ public class LocalEdqsStateService implements EdqsStateService {
|
|||||||
log.error("[{}] Failed to restore value", key, e);
|
log.error("[{}] Failed to restore value", key, e);
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
|
restoreDone = true;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@ -78,4 +77,9 @@ public class LocalEdqsStateService implements EdqsStateService {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean isReady() {
|
||||||
|
return restoreDone;
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|||||||
@ -52,9 +52,9 @@ public class EdqsStatsService {
|
|||||||
log.info("EDQS Stats: {}", values);
|
log.info("EDQS Stats: {}", values);
|
||||||
}
|
}
|
||||||
|
|
||||||
public void reportTenantEdqsObject(TenantId tenantId, ObjectType objectType, EdqsEventType eventType) {
|
public void reportEvent(TenantId tenantId, ObjectType objectType, EdqsEventType eventType) {
|
||||||
statsMap.computeIfAbsent(tenantId, id -> new EdqsStats(tenantId, statsFactory))
|
statsMap.computeIfAbsent(tenantId, id -> new EdqsStats(tenantId, statsFactory))
|
||||||
.reportEdqsObject(objectType, eventType);
|
.reportEvent(objectType, eventType);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Getter
|
@Getter
|
||||||
@ -78,7 +78,7 @@ public class EdqsStatsService {
|
|||||||
.collect(Collectors.joining(", "));
|
.collect(Collectors.joining(", "));
|
||||||
}
|
}
|
||||||
|
|
||||||
public void reportEdqsObject(ObjectType objectType, EdqsEventType eventType) {
|
public void reportEvent(ObjectType objectType, EdqsEventType eventType) {
|
||||||
AtomicInteger objectCounter = getOrCreateObjectCounter(objectType);
|
AtomicInteger objectCounter = getOrCreateObjectCounter(objectType);
|
||||||
if (eventType == EdqsEventType.UPDATED){
|
if (eventType == EdqsEventType.UPDATED){
|
||||||
objectCounter.incrementAndGet();
|
objectCounter.incrementAndGet();
|
||||||
|
|||||||
@ -39,6 +39,7 @@ import java.util.concurrent.ScheduledExecutorService;
|
|||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
import java.util.concurrent.locks.ReentrantLock;
|
import java.util.concurrent.locks.ReentrantLock;
|
||||||
import java.util.function.BiFunction;
|
import java.util.function.BiFunction;
|
||||||
|
import java.util.function.Consumer;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
@Slf4j
|
@Slf4j
|
||||||
@ -53,6 +54,7 @@ public class MainQueueConsumerManager<M extends TbQueueMsg, C extends QueueConfi
|
|||||||
protected final ExecutorService consumerExecutor;
|
protected final ExecutorService consumerExecutor;
|
||||||
protected final ScheduledExecutorService scheduler;
|
protected final ScheduledExecutorService scheduler;
|
||||||
protected final ExecutorService taskExecutor;
|
protected final ExecutorService taskExecutor;
|
||||||
|
protected final Consumer<Throwable> uncaughtErrorHandler;
|
||||||
|
|
||||||
private final java.util.Queue<TbQueueConsumerManagerTask> tasks = new ConcurrentLinkedQueue<>();
|
private final java.util.Queue<TbQueueConsumerManagerTask> tasks = new ConcurrentLinkedQueue<>();
|
||||||
private final ReentrantLock lock = new ReentrantLock();
|
private final ReentrantLock lock = new ReentrantLock();
|
||||||
@ -68,7 +70,8 @@ public class MainQueueConsumerManager<M extends TbQueueMsg, C extends QueueConfi
|
|||||||
BiFunction<C, Integer, TbQueueConsumer<M>> consumerCreator,
|
BiFunction<C, Integer, TbQueueConsumer<M>> consumerCreator,
|
||||||
ExecutorService consumerExecutor,
|
ExecutorService consumerExecutor,
|
||||||
ScheduledExecutorService scheduler,
|
ScheduledExecutorService scheduler,
|
||||||
ExecutorService taskExecutor) {
|
ExecutorService taskExecutor,
|
||||||
|
Consumer<Throwable> uncaughtErrorHandler) {
|
||||||
this.queueKey = queueKey;
|
this.queueKey = queueKey;
|
||||||
this.config = config;
|
this.config = config;
|
||||||
this.msgPackProcessor = msgPackProcessor;
|
this.msgPackProcessor = msgPackProcessor;
|
||||||
@ -76,6 +79,7 @@ public class MainQueueConsumerManager<M extends TbQueueMsg, C extends QueueConfi
|
|||||||
this.consumerExecutor = consumerExecutor;
|
this.consumerExecutor = consumerExecutor;
|
||||||
this.scheduler = scheduler;
|
this.scheduler = scheduler;
|
||||||
this.taskExecutor = taskExecutor;
|
this.taskExecutor = taskExecutor;
|
||||||
|
this.uncaughtErrorHandler = uncaughtErrorHandler;
|
||||||
if (config != null) {
|
if (config != null) {
|
||||||
init(config);
|
init(config);
|
||||||
}
|
}
|
||||||
@ -189,36 +193,40 @@ public class MainQueueConsumerManager<M extends TbQueueMsg, C extends QueueConfi
|
|||||||
log.info("[{}] Launching consumer", consumerTask.getKey());
|
log.info("[{}] Launching consumer", consumerTask.getKey());
|
||||||
Future<?> consumerLoop = consumerExecutor.submit(() -> {
|
Future<?> consumerLoop = consumerExecutor.submit(() -> {
|
||||||
ThingsBoardThreadFactory.updateCurrentThreadName(consumerTask.getKey().toString());
|
ThingsBoardThreadFactory.updateCurrentThreadName(consumerTask.getKey().toString());
|
||||||
try {
|
consumerLoop(consumerTask.getConsumer());
|
||||||
consumerLoop(consumerTask.getConsumer());
|
|
||||||
} catch (Throwable e) {
|
|
||||||
log.error("Failure in consumer loop", e);
|
|
||||||
}
|
|
||||||
log.info("[{}] Consumer stopped", consumerTask.getKey());
|
log.info("[{}] Consumer stopped", consumerTask.getKey());
|
||||||
});
|
});
|
||||||
consumerTask.setTask(consumerLoop);
|
consumerTask.setTask(consumerLoop);
|
||||||
}
|
}
|
||||||
|
|
||||||
private void consumerLoop(TbQueueConsumer<M> consumer) {
|
private void consumerLoop(TbQueueConsumer<M> consumer) {
|
||||||
while (!stopped && !consumer.isStopped()) {
|
try {
|
||||||
try {
|
while (!stopped && !consumer.isStopped()) {
|
||||||
List<M> msgs = consumer.poll(config.getPollInterval());
|
try {
|
||||||
if (msgs.isEmpty()) {
|
List<M> msgs = consumer.poll(config.getPollInterval());
|
||||||
continue;
|
if (msgs.isEmpty()) {
|
||||||
}
|
continue;
|
||||||
processMsgs(msgs, consumer, config);
|
}
|
||||||
} catch (Exception e) {
|
processMsgs(msgs, consumer, config);
|
||||||
if (!consumer.isStopped()) {
|
} catch (Exception e) {
|
||||||
log.warn("Failed to process messages from queue", e);
|
if (!consumer.isStopped()) {
|
||||||
try {
|
log.warn("Failed to process messages from queue", e);
|
||||||
Thread.sleep(config.getPollInterval());
|
try {
|
||||||
} catch (InterruptedException e2) {
|
Thread.sleep(config.getPollInterval());
|
||||||
log.trace("Failed to wait until the server has capacity to handle new requests", e2);
|
} catch (InterruptedException e2) {
|
||||||
|
log.trace("Failed to wait until the server has capacity to handle new requests", e2);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
if (consumer.isStopped()) {
|
||||||
if (consumer.isStopped()) {
|
consumer.unsubscribe();
|
||||||
|
}
|
||||||
|
} catch (Throwable t) {
|
||||||
|
log.error("Failure in consumer loop", t);
|
||||||
|
if (uncaughtErrorHandler != null) {
|
||||||
|
uncaughtErrorHandler.accept(t);
|
||||||
|
}
|
||||||
consumer.unsubscribe();
|
consumer.unsubscribe();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@ -84,7 +84,7 @@ public class TbQueueConsumerTask<M extends TbQueueMsg> {
|
|||||||
}
|
}
|
||||||
log.trace("[{}] Awaited finish", key);
|
log.trace("[{}] Awaited finish", key);
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
log.warn("[{}] Failed to await for consumer to stop", key, e);
|
log.warn("[{}] Failed to await for consumer to stop (timeout {} sec)", key, timeoutSec, e);
|
||||||
}
|
}
|
||||||
task = null;
|
task = null;
|
||||||
}
|
}
|
||||||
|
|||||||
@ -26,15 +26,10 @@ import org.apache.kafka.clients.consumer.ConsumerConfig;
|
|||||||
import org.apache.kafka.clients.consumer.KafkaConsumer;
|
import org.apache.kafka.clients.consumer.KafkaConsumer;
|
||||||
import org.apache.kafka.clients.consumer.OffsetAndMetadata;
|
import org.apache.kafka.clients.consumer.OffsetAndMetadata;
|
||||||
import org.apache.kafka.common.TopicPartition;
|
import org.apache.kafka.common.TopicPartition;
|
||||||
import org.springframework.beans.factory.annotation.Autowired;
|
|
||||||
import org.springframework.boot.autoconfigure.condition.ConditionalOnProperty;
|
import org.springframework.boot.autoconfigure.condition.ConditionalOnProperty;
|
||||||
import org.springframework.context.annotation.Lazy;
|
|
||||||
import org.springframework.stereotype.Component;
|
import org.springframework.stereotype.Component;
|
||||||
import org.thingsboard.common.util.ThingsBoardExecutors;
|
import org.thingsboard.common.util.ThingsBoardExecutors;
|
||||||
import org.thingsboard.server.common.data.StringUtils;
|
import org.thingsboard.server.common.data.StringUtils;
|
||||||
import org.thingsboard.server.common.data.id.TenantId;
|
|
||||||
import org.thingsboard.server.common.msg.queue.ServiceType;
|
|
||||||
import org.thingsboard.server.queue.discovery.PartitionService;
|
|
||||||
|
|
||||||
import java.time.Duration;
|
import java.time.Duration;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
@ -56,10 +51,6 @@ public class TbKafkaConsumerStatsService {
|
|||||||
private final TbKafkaSettings kafkaSettings;
|
private final TbKafkaSettings kafkaSettings;
|
||||||
private final TbKafkaConsumerStatisticConfig statsConfig;
|
private final TbKafkaConsumerStatisticConfig statsConfig;
|
||||||
|
|
||||||
@Lazy
|
|
||||||
@Autowired
|
|
||||||
private PartitionService partitionService;
|
|
||||||
|
|
||||||
private Consumer<String, byte[]> consumer;
|
private Consumer<String, byte[]> consumer;
|
||||||
private ScheduledExecutorService statsPrintScheduler;
|
private ScheduledExecutorService statsPrintScheduler;
|
||||||
|
|
||||||
@ -111,9 +102,7 @@ public class TbKafkaConsumerStatsService {
|
|||||||
}
|
}
|
||||||
|
|
||||||
private boolean isStatsPrintRequired() {
|
private boolean isStatsPrintRequired() {
|
||||||
boolean isMyRuleEnginePartition = partitionService.isMyPartition(ServiceType.TB_RULE_ENGINE, TenantId.SYS_TENANT_ID, TenantId.SYS_TENANT_ID);
|
return log.isInfoEnabled();
|
||||||
boolean isMyCorePartition = partitionService.isMyPartition(ServiceType.TB_CORE, TenantId.SYS_TENANT_ID, TenantId.SYS_TENANT_ID);
|
|
||||||
return log.isInfoEnabled() && (isMyRuleEnginePartition || isMyCorePartition);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private List<GroupTopicStats> getTopicsStatsWithLag(Map<TopicPartition, OffsetAndMetadata> groupOffsets, Map<TopicPartition, Long> endOffsets) {
|
private List<GroupTopicStats> getTopicsStatsWithLag(Map<TopicPartition, OffsetAndMetadata> groupOffsets, Map<TopicPartition, Long> endOffsets) {
|
||||||
|
|||||||
Loading…
x
Reference in New Issue
Block a user