mirror of
https://github.com/kestra-io/kestra.git
synced 2025-12-19 18:05:41 -05:00
feat(core): move kafka to ee (#995)
Co-authored-by: Loïc Mathieu <loikeseke@gmail.com>
This commit is contained in:
@@ -70,8 +70,6 @@ allprojects {
|
||||
//
|
||||
configurations.all {
|
||||
resolutionStrategy {
|
||||
force("org.apache.kafka:kafka-clients:" + kafkaVersion)
|
||||
force("org.apache.kafka:kafka-streams:" + kafkaVersion)
|
||||
force("org.jooq:jooq:3.16.10")
|
||||
force("org.slf4j:slf4j-api:1.7.36")
|
||||
|
||||
|
||||
@@ -26,12 +26,8 @@ dependencies {
|
||||
implementation project(":core")
|
||||
|
||||
implementation project(":repository-memory")
|
||||
implementation project(":repository-elasticsearch")
|
||||
|
||||
implementation project(":runner-memory")
|
||||
implementation project(":runner-kafka")
|
||||
|
||||
implementation project(":indexer-kafka-elasticsearch")
|
||||
|
||||
implementation project(":jdbc")
|
||||
implementation project(":jdbc-h2")
|
||||
|
||||
@@ -106,7 +106,7 @@ abstract public class AbstractCommand implements Callable<Integer> {
|
||||
(
|
||||
this.internalLog && (
|
||||
logger.getName().startsWith("io.kestra") &&
|
||||
!logger.getName().startsWith("io.kestra.runner.kafka.services"))
|
||||
!logger.getName().startsWith("io.kestra.ee.runner.kafka.services"))
|
||||
)
|
||||
|| logger.getName().startsWith("flow")
|
||||
)
|
||||
|
||||
@@ -10,14 +10,12 @@ import io.kestra.core.queues.QueueInterface;
|
||||
import io.kestra.core.repositories.FlowRepositoryInterface;
|
||||
import io.kestra.core.repositories.TemplateRepositoryInterface;
|
||||
import io.kestra.core.repositories.TriggerRepositoryInterface;
|
||||
import io.kestra.runner.kafka.services.KafkaAdminService;
|
||||
import io.micronaut.context.ApplicationContext;
|
||||
import io.micronaut.inject.qualifiers.Qualifiers;
|
||||
import lombok.SneakyThrows;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Collectors;
|
||||
import jakarta.inject.Inject;
|
||||
import jakarta.inject.Singleton;
|
||||
@@ -59,19 +57,6 @@ public class RestoreQueueService {
|
||||
@SneakyThrows
|
||||
@SuppressWarnings("unchecked")
|
||||
public <T> int send(List<T> list, String queueName, Class<?> cls, boolean noRecreate) {
|
||||
Optional<String> queueType = applicationContext.getProperty("kestra.queue.type", String.class);
|
||||
|
||||
if (queueType.isPresent() && queueType.get().equals("kafka")) {
|
||||
KafkaAdminService kafkaAdminService = applicationContext.getBean(KafkaAdminService.class);
|
||||
if (!noRecreate) {
|
||||
kafkaAdminService.delete(cls);
|
||||
}
|
||||
|
||||
// need some wait to be sure the topic are deleted before recreated with right configuration
|
||||
Thread.sleep(2000);
|
||||
kafkaAdminService.createIfNotExist(cls);
|
||||
}
|
||||
|
||||
QueueInterface<T> queue = (QueueInterface<T>) applicationContext.getBean(
|
||||
QueueInterface.class,
|
||||
Qualifiers.byName(queueName)
|
||||
|
||||
@@ -68,156 +68,6 @@ kestra:
|
||||
delay: 1s
|
||||
maxDelay: ""
|
||||
|
||||
kafka:
|
||||
defaults:
|
||||
topic-prefix: "kestra_"
|
||||
consumer-prefix: "kestra_"
|
||||
topic:
|
||||
partitions: 16
|
||||
replication-factor: 1
|
||||
|
||||
client:
|
||||
properties:
|
||||
metrics.recording.level: TRACE
|
||||
|
||||
consumer:
|
||||
properties:
|
||||
isolation.level: "read_committed"
|
||||
auto.offset.reset: "earliest"
|
||||
enable.auto.commit: "false"
|
||||
|
||||
producer:
|
||||
properties:
|
||||
acks: "all"
|
||||
compression.type: "lz4"
|
||||
max.request.size: "10485760"
|
||||
|
||||
stream:
|
||||
properties:
|
||||
processing.guarantee: "exactly_once_v2"
|
||||
replication.factor: "${kestra.kafka.defaults.topic.replication-factor}"
|
||||
acks: "all"
|
||||
compression.type: "lz4"
|
||||
max.request.size: "10485760"
|
||||
cache.max.bytes.buffering: 0
|
||||
|
||||
topics:
|
||||
execution:
|
||||
name: "${kestra.kafka.defaults.topic-prefix}execution"
|
||||
cls: io.kestra.core.models.executions.Execution
|
||||
properties:
|
||||
cleanup.policy: "compact"
|
||||
retention.ms: "-1"
|
||||
min.compaction.lag.ms: 86400000
|
||||
|
||||
flow:
|
||||
name: "${kestra.kafka.defaults.topic-prefix}flow"
|
||||
cls: io.kestra.core.models.flows.Flow
|
||||
properties:
|
||||
cleanup.policy: "compact"
|
||||
retention.ms: "-1"
|
||||
segment.bytes: "10485760"
|
||||
|
||||
flowlast:
|
||||
name: "${kestra.kafka.defaults.topic-prefix}flow_last"
|
||||
properties:
|
||||
cleanup.policy: "compact"
|
||||
retention.ms: "-1"
|
||||
segment.bytes: "10485760"
|
||||
|
||||
template:
|
||||
name: "${kestra.kafka.defaults.topic-prefix}template"
|
||||
cls: io.kestra.core.models.templates.Template
|
||||
properties:
|
||||
cleanup.policy: "compact"
|
||||
retention.ms: "-1"
|
||||
segment.bytes: "10485760"
|
||||
|
||||
executor:
|
||||
name: "${kestra.kafka.defaults.topic-prefix}executor_main-executor-changelog"
|
||||
cls: io.kestra.core.runners.Executor
|
||||
properties:
|
||||
cleanup.policy: "delete,compact"
|
||||
min.compaction.lag.ms: 86400000
|
||||
delete.retention.ms: 86400000
|
||||
|
||||
killed:
|
||||
name: "${kestra.kafka.defaults.topic-prefix}executor_killed"
|
||||
cls: io.kestra.core.models.executions.ExecutionKilled
|
||||
properties:
|
||||
cleanup.policy: "delete,compact"
|
||||
min.compaction.lag.ms: 86400000
|
||||
delete.retention.ms: 86400000
|
||||
|
||||
executorflowtrigger:
|
||||
name: "${kestra.kafka.defaults.topic-prefix}executor_flowtrigger"
|
||||
cls: io.kestra.runner.kafka.streams.ExecutorFlowTrigger
|
||||
properties:
|
||||
cleanup.policy: "delete,compact"
|
||||
min.compaction.lag.ms: 86400000
|
||||
delete.retention.ms: 86400000
|
||||
|
||||
executorworkertaskexecution:
|
||||
cls: io.kestra.core.runners.WorkerTaskExecution
|
||||
name: "${kestra.kafka.defaults.topic-prefix}executor_workertaskexecution"
|
||||
properties:
|
||||
cleanup.policy: "delete,compact"
|
||||
min.compaction.lag.ms: 86400000
|
||||
delete.retention.ms: 86400000
|
||||
|
||||
workertask:
|
||||
name: "${kestra.kafka.defaults.topic-prefix}workertask"
|
||||
cls: io.kestra.core.runners.WorkerTask
|
||||
properties:
|
||||
cleanup.policy: "delete,compact"
|
||||
min.compaction.lag.ms: 86400000
|
||||
delete.retention.ms: 86400000
|
||||
|
||||
workertaskresult:
|
||||
name: "${kestra.kafka.defaults.topic-prefix}workertaskresult"
|
||||
cls: io.kestra.core.runners.WorkerTaskResult
|
||||
properties:
|
||||
cleanup.policy: "delete,compact"
|
||||
min.compaction.lag.ms: 86400000
|
||||
delete.retention.ms: 86400000
|
||||
|
||||
workerinstance:
|
||||
cls: io.kestra.core.runners.WorkerInstance
|
||||
name: "${kestra.kafka.defaults.topic-prefix}workerinstance"
|
||||
properties:
|
||||
cleanup.policy: "compact"
|
||||
min.compaction.lag.ms: 86400000
|
||||
delete.retention.ms: 86400000
|
||||
|
||||
executorworkerinstance:
|
||||
name: "${kestra.kafka.defaults.topic-prefix}executor_workerinstance"
|
||||
properties:
|
||||
cleanup.policy: "compact"
|
||||
min.compaction.lag.ms: 86400000
|
||||
delete.retention.ms: 86400000
|
||||
|
||||
workertaskrunning:
|
||||
cls: io.kestra.core.runners.WorkerTaskRunning
|
||||
name: "${kestra.kafka.defaults.topic-prefix}workertaskrunning"
|
||||
properties:
|
||||
cleanup.policy: "delete,compact"
|
||||
min.compaction.lag.ms: 86400000
|
||||
delete.retention.ms: 86400000
|
||||
|
||||
logentry:
|
||||
cls: io.kestra.core.models.executions.LogEntry
|
||||
name: "${kestra.kafka.defaults.topic-prefix}logs"
|
||||
properties:
|
||||
cleanup.policy: "delete"
|
||||
|
||||
trigger:
|
||||
cls: io.kestra.core.models.triggers.Trigger
|
||||
name: "${kestra.kafka.defaults.topic-prefix}trigger"
|
||||
properties:
|
||||
cleanup.policy: "compact"
|
||||
min.compaction.lag.ms: 86400000
|
||||
delete.retention.ms: 86400000
|
||||
|
||||
jdbc:
|
||||
tables:
|
||||
queues:
|
||||
|
||||
@@ -7,10 +7,10 @@
|
||||
<logger name="io.kestra" level="INFO" />
|
||||
<logger name="flow" level="INFO" />
|
||||
|
||||
<logger name="io.kestra.runner.kafka.services.KafkaConsumerService" level="WARN" />
|
||||
<logger name="io.kestra.runner.kafka.services.KafkaProducerService" level="WARN" />
|
||||
<logger name="io.kestra.runner.kafka.services.KafkaStreamService" level="WARN" />
|
||||
<logger name="io.kestra.runner.kafka.services.KafkaAdminService" level="WARN" />
|
||||
<logger name="io.kestra.ee.runner.kafka.services.KafkaConsumerService" level="WARN" />
|
||||
<logger name="io.kestra.ee.runner.kafka.services.KafkaProducerService" level="WARN" />
|
||||
<logger name="io.kestra.ee.runner.kafka.services.KafkaStreamService" level="WARN" />
|
||||
<logger name="io.kestra.ee.runner.kafka.services.KafkaAdminService" level="WARN" />
|
||||
|
||||
<!-- The configuration '%s' was supplied but isn't a known config. > https://github.com/apache/kafka/pull/5876 -->
|
||||
<logger name="org.apache.kafka.clients.producer.ProducerConfig" level="ERROR" />
|
||||
|
||||
@@ -59,7 +59,7 @@ abstract public class FlowListenersTest {
|
||||
});
|
||||
|
||||
// resend on startup done for kafka
|
||||
if (flowListenersService.getClass().getName().equals("io.kestra.runner.kafka.KafkaFlowListeners")) {
|
||||
if (flowListenersService.getClass().getName().equals("io.kestra.ee.runner.kafka.KafkaFlowListeners")) {
|
||||
wait(ref, () -> {
|
||||
assertThat(count.get(), is(0));
|
||||
assertThat(flowListenersService.flows().size(), is(0));
|
||||
|
||||
@@ -1,57 +1,4 @@
|
||||
services:
|
||||
zookeeper:
|
||||
image: confluentinc/cp-zookeeper
|
||||
environment:
|
||||
ZOOKEEPER_CLIENT_PORT: 2181
|
||||
KAFKA_OPTS: "-Dzookeeper.4lw.commands.whitelist=ruok"
|
||||
healthcheck:
|
||||
test: echo "ruok" | nc -w 2 localhost 2181 | grep -q "imok" || exit 1
|
||||
interval: 10s
|
||||
timeout: 5s
|
||||
retries: 10
|
||||
start_period: 10s
|
||||
|
||||
kafka:
|
||||
image: confluentinc/cp-kafka
|
||||
environment:
|
||||
KAFKA_ZOOKEEPER_CONNECT: zookeeper:2181
|
||||
KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR: 1
|
||||
KAFKA_TRANSACTION_STATE_LOG_REPLICATION_FACTOR: 1
|
||||
KAFKA_TRANSACTION_STATE_LOG_MIN_ISR: 1
|
||||
KAFKA_ADVERTISED_LISTENERS: PLAINTEXT://localhost:9092
|
||||
KAFKA_CONFLUENT_SUPPORT_METRICS_ENABLE: 'false'
|
||||
depends_on:
|
||||
zookeeper:
|
||||
condition: service_healthy
|
||||
healthcheck:
|
||||
test: kafka-topics --bootstrap-server localhost:9092 --list || exit 1
|
||||
interval: 30s
|
||||
timeout: 10s
|
||||
retries: 10
|
||||
ports:
|
||||
- 9092:9092
|
||||
|
||||
elasticsearch:
|
||||
image: docker.elastic.co/elasticsearch/elasticsearch:8.2.3
|
||||
environment:
|
||||
discovery.type: single-node
|
||||
ES_JAVA_OPTS: "-Xms256m -Xmx256m"
|
||||
xpack.security.enabled: "false"
|
||||
ulimits:
|
||||
memlock:
|
||||
soft: -1
|
||||
hard: -1
|
||||
nofile:
|
||||
soft: 65536
|
||||
hard: 65536
|
||||
ports:
|
||||
- 9200:9200
|
||||
healthcheck:
|
||||
test: curl -s -f localhost:9200/_cat/health >/dev/null || exit 1
|
||||
interval: 30s
|
||||
timeout: 10s
|
||||
retries: 10
|
||||
|
||||
mysql:
|
||||
image: mysql
|
||||
environment:
|
||||
|
||||
@@ -1,5 +1,3 @@
|
||||
version=0.6.1-SNAPSHOT
|
||||
opensearchVersion=2.4.1
|
||||
micronautVersion=3.8.0
|
||||
kafkaVersion=3.1.0
|
||||
lombokVersion=1.18.24
|
||||
@@ -1,15 +0,0 @@
|
||||
publishSonatypePublicationPublicationToSonatypeRepository.enabled = false
|
||||
|
||||
dependencies {
|
||||
implementation project(":core")
|
||||
implementation project(":repository-elasticsearch")
|
||||
implementation project(":runner-kafka")
|
||||
|
||||
implementation (group: 'org.opensearch.client', name: 'opensearch-rest-high-level-client', version: "$opensearchVersion") {
|
||||
exclude group: 'org.apache.logging.log4j'
|
||||
}
|
||||
implementation group: "org.apache.kafka", name: "kafka-clients", version: kafkaVersion
|
||||
implementation group: 'net.jodah', name: 'failsafe', version: '2.4.4'
|
||||
|
||||
testImplementation 'org.mockito:mockito-junit-jupiter:4.11.0'
|
||||
}
|
||||
@@ -1,16 +0,0 @@
|
||||
package io.kestra.indexer;
|
||||
|
||||
import io.micronaut.context.annotation.ConfigurationProperties;
|
||||
import lombok.Getter;
|
||||
|
||||
import java.time.Duration;
|
||||
import java.time.temporal.ChronoUnit;
|
||||
import java.util.List;
|
||||
|
||||
@ConfigurationProperties("kestra.indexer")
|
||||
@Getter
|
||||
public class IndexerConfig {
|
||||
Integer batchSize = 500;
|
||||
Duration batchDuration = Duration.of(1, ChronoUnit.SECONDS);
|
||||
List<Class<?>> models;
|
||||
}
|
||||
@@ -1,263 +0,0 @@
|
||||
package io.kestra.indexer;
|
||||
|
||||
import io.kestra.core.metrics.MetricRegistry;
|
||||
import io.kestra.core.runners.Indexer;
|
||||
import io.kestra.core.runners.IndexerInterface;
|
||||
import io.kestra.core.utils.DurationOrSizeTrigger;
|
||||
import io.kestra.core.utils.ExecutorsUtils;
|
||||
import io.kestra.repository.elasticsearch.ElasticSearchIndicesService;
|
||||
import io.kestra.repository.elasticsearch.ElasticSearchRepositoryEnabled;
|
||||
import io.kestra.repository.elasticsearch.configs.IndicesConfig;
|
||||
import io.kestra.runner.kafka.KafkaQueueEnabled;
|
||||
import io.kestra.runner.kafka.configs.TopicsConfig;
|
||||
import io.kestra.runner.kafka.services.KafkaConsumerService;
|
||||
import io.micrometer.core.instrument.Timer;
|
||||
import io.micronaut.context.annotation.Replaces;
|
||||
import lombok.extern.slf4j.Slf4j;
|
||||
import net.jodah.failsafe.Failsafe;
|
||||
import net.jodah.failsafe.RetryPolicy;
|
||||
import org.apache.kafka.clients.consumer.ConsumerRecord;
|
||||
import org.apache.kafka.common.errors.WakeupException;
|
||||
import org.apache.kafka.common.serialization.Serdes;
|
||||
import org.opensearch.action.bulk.BulkItemResponse;
|
||||
import org.opensearch.action.bulk.BulkRequest;
|
||||
import org.opensearch.action.bulk.BulkResponse;
|
||||
import org.opensearch.action.delete.DeleteRequest;
|
||||
import org.opensearch.action.index.IndexRequest;
|
||||
import org.opensearch.client.RequestOptions;
|
||||
import org.opensearch.client.RestHighLevelClient;
|
||||
import org.opensearch.common.xcontent.XContentType;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.time.Duration;
|
||||
import java.time.temporal.ChronoUnit;
|
||||
import java.util.*;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.StreamSupport;
|
||||
import jakarta.inject.Inject;
|
||||
import jakarta.inject.Singleton;
|
||||
|
||||
@Singleton
|
||||
@Replaces(Indexer.class)
|
||||
@ElasticSearchRepositoryEnabled
|
||||
@KafkaQueueEnabled
|
||||
@Slf4j
|
||||
public class KafkaElasticIndexer implements IndexerInterface, Cloneable {
|
||||
private final MetricRegistry metricRegistry;
|
||||
private final RestHighLevelClient elasticClient;
|
||||
private final KafkaConsumerService kafkaConsumerService;
|
||||
private final ExecutorService poolExecutor;
|
||||
|
||||
private final Map<String, String> mapping;
|
||||
private final Set<String> subscriptions;
|
||||
|
||||
private final AtomicBoolean running = new AtomicBoolean(true);;
|
||||
private final DurationOrSizeTrigger<ConsumerRecord<String, String>> trigger;
|
||||
|
||||
private org.apache.kafka.clients.consumer.Consumer<String, String> kafkaConsumer;
|
||||
|
||||
@Inject
|
||||
public KafkaElasticIndexer(
|
||||
MetricRegistry metricRegistry,
|
||||
RestHighLevelClient elasticClient,
|
||||
IndexerConfig indexerConfig,
|
||||
List<TopicsConfig> topicsConfig,
|
||||
List<IndicesConfig> indicesConfigs,
|
||||
ElasticSearchIndicesService elasticSearchIndicesService,
|
||||
KafkaConsumerService kafkaConsumerService,
|
||||
ExecutorsUtils executorsUtils
|
||||
) {
|
||||
this.metricRegistry = metricRegistry;
|
||||
this.elasticClient = elasticClient;
|
||||
this.kafkaConsumerService = kafkaConsumerService;
|
||||
this.poolExecutor = executorsUtils.cachedThreadPool("kakfa-elastic-indexer");
|
||||
|
||||
this.subscriptions = subscriptions(topicsConfig, indexerConfig);
|
||||
this.mapping = mapTopicToIndices(topicsConfig, indicesConfigs);
|
||||
|
||||
this.trigger = new DurationOrSizeTrigger<>(
|
||||
indexerConfig.getBatchDuration(),
|
||||
indexerConfig.getBatchSize()
|
||||
);
|
||||
|
||||
elasticSearchIndicesService.createIndice(null);
|
||||
elasticSearchIndicesService.updateMapping(null);
|
||||
}
|
||||
|
||||
public void run() {
|
||||
poolExecutor.execute(() -> {
|
||||
kafkaConsumer = kafkaConsumerService.of(Indexer.class, Serdes.String(), Indexer.class);
|
||||
kafkaConsumer.subscribe(this.subscriptions);
|
||||
|
||||
List<ConsumerRecord<String, String>> rows = new ArrayList<>();
|
||||
|
||||
while (running.get()) {
|
||||
try {
|
||||
List<ConsumerRecord<String, String>> records = StreamSupport
|
||||
.stream(kafkaConsumer.poll(Duration.ofMillis(500)).spliterator(), false)
|
||||
.collect(Collectors.toList());
|
||||
|
||||
records
|
||||
.stream()
|
||||
.collect(Collectors.groupingBy(ConsumerRecord::topic))
|
||||
.forEach((topic, consumerRecords) -> {
|
||||
this.metricRegistry
|
||||
.counter(MetricRegistry.METRIC_INDEXER_MESSAGE_IN_COUNT, "topic", topic)
|
||||
.increment(consumerRecords.size());
|
||||
});
|
||||
|
||||
for (ConsumerRecord<String, String> record : records) {
|
||||
rows.add(record);
|
||||
this.send(rows, kafkaConsumer);
|
||||
}
|
||||
|
||||
this.send(rows, kafkaConsumer);
|
||||
} catch (WakeupException e) {
|
||||
log.debug("Received Wakeup on {}", this.getClass().getName());
|
||||
running.set(false);
|
||||
}
|
||||
}
|
||||
|
||||
this.kafkaConsumer.close();
|
||||
});
|
||||
}
|
||||
|
||||
private void send(List<ConsumerRecord<String, String>> rows, org.apache.kafka.clients.consumer.Consumer<String, String> consumer) {
|
||||
if (trigger.test(rows)) {
|
||||
BulkRequest request = bulkRequest(rows);
|
||||
|
||||
try {
|
||||
this.metricRegistry
|
||||
.counter(MetricRegistry.METRIC_INDEXER_REQUEST_COUNT)
|
||||
.increment();
|
||||
|
||||
Timer timer = this.metricRegistry.timer(MetricRegistry.METRIC_INDEXER_REQUEST_DURATION);
|
||||
|
||||
timer.record(() -> Failsafe
|
||||
.with(retryPolicy())
|
||||
.run(() -> {
|
||||
this.insert(request);
|
||||
}));
|
||||
|
||||
consumer.commitSync(KafkaConsumerService.maxOffsets(rows));
|
||||
rows.clear();
|
||||
} catch (RuntimeException exception) {
|
||||
consumer.close();
|
||||
throw new RuntimeException(exception);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private BulkRequest bulkRequest(List<ConsumerRecord<String, String>> rows) {
|
||||
BulkRequest request = new BulkRequest();
|
||||
|
||||
rows
|
||||
.forEach(record -> {
|
||||
if (record.value() == null) {
|
||||
request
|
||||
.add(new DeleteRequest(this.indexName(record))
|
||||
.id(record.key())
|
||||
);
|
||||
} else {
|
||||
request
|
||||
.add(new IndexRequest(this.indexName(record))
|
||||
.id(record.key())
|
||||
.source(record.value(), XContentType.JSON)
|
||||
);
|
||||
}
|
||||
});
|
||||
|
||||
return request;
|
||||
}
|
||||
|
||||
private void insert(BulkRequest bulkRequest) throws IOException {
|
||||
BulkResponse bulkResponse = this.elasticClient.bulk(bulkRequest, RequestOptions.DEFAULT);
|
||||
|
||||
Map<String, List<BulkItemResponse>> grouped = StreamSupport.stream(bulkResponse.spliterator(), false)
|
||||
.collect(Collectors.groupingBy(BulkItemResponse::getIndex));
|
||||
|
||||
grouped.
|
||||
forEach((index, bulkItemResponses) -> this.metricRegistry
|
||||
.counter(MetricRegistry.METRIC_INDEXER_MESSAGE_OUT_COUNT, "index", index)
|
||||
.increment(bulkItemResponses.size()));
|
||||
|
||||
if (bulkResponse.hasFailures()) {
|
||||
grouped.
|
||||
forEach((index, bulkItemResponses) -> {
|
||||
long count = bulkItemResponses.stream()
|
||||
.filter(BulkItemResponse::isFailed)
|
||||
.count();
|
||||
|
||||
if (count > 0) {
|
||||
this.metricRegistry
|
||||
.counter(MetricRegistry.METRIC_INDEXER_MESSAGE_FAILED_COUNT, "index", index)
|
||||
.increment(bulkItemResponses.size());
|
||||
}
|
||||
});
|
||||
|
||||
throw new IOException("Indexer failed bulk '" + bulkResponse.buildFailureMessage() + "'");
|
||||
}
|
||||
|
||||
this.metricRegistry.timer(MetricRegistry.METRIC_INDEXER_SERVER_DURATION)
|
||||
.record(bulkResponse.getTook().getNanos(), TimeUnit.NANOSECONDS);
|
||||
|
||||
if (log.isDebugEnabled()) {
|
||||
log.debug("Indexer request with {} elements in {}", bulkRequest.numberOfActions(), bulkResponse.getTook().toString());
|
||||
}
|
||||
}
|
||||
|
||||
private RetryPolicy<Object> retryPolicy() {
|
||||
return new RetryPolicy<>()
|
||||
.onFailedAttempt(event -> {
|
||||
this.metricRegistry.counter(MetricRegistry.METRIC_INDEXER_REQUEST_RETRY_COUNT)
|
||||
.increment();
|
||||
|
||||
log.warn(
|
||||
"Indexer failed in " + event.getStartTime().toString() + " (" + event.getElapsedTime().toSeconds() + "s elapsed / " + event.getAttemptCount() + " retries)",
|
||||
event.getLastFailure()
|
||||
);;
|
||||
})
|
||||
.withBackoff(2, 300, ChronoUnit.SECONDS);
|
||||
}
|
||||
|
||||
protected Map<String, String> mapTopicToIndices(List<TopicsConfig> topicsConfig, List<IndicesConfig> indicesConfigs) {
|
||||
return topicsConfig
|
||||
.stream()
|
||||
.filter(topic -> indicesConfigs
|
||||
.stream()
|
||||
.anyMatch(indicesConfig -> indicesConfig.getCls() == topic.getCls())
|
||||
)
|
||||
.map(topic -> new AbstractMap.SimpleEntry<>(
|
||||
topic.getName(),
|
||||
indicesConfigs
|
||||
.stream()
|
||||
.filter(indicesConfig -> indicesConfig.getCls() == topic.getCls())
|
||||
.findFirst()
|
||||
.orElseThrow()
|
||||
.getIndex()
|
||||
))
|
||||
.collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
|
||||
}
|
||||
|
||||
protected Set<String> subscriptions(List<TopicsConfig> topicsConfig, IndexerConfig indexerConfig) {
|
||||
return topicsConfig
|
||||
.stream()
|
||||
.filter(t -> indexerConfig.getModels().contains(t.getCls()))
|
||||
.map(TopicsConfig::getName)
|
||||
.collect(Collectors.toSet());
|
||||
}
|
||||
|
||||
protected String indexName(ConsumerRecord<?, ?> record) {
|
||||
return this.mapping.get(record.topic());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
if (kafkaConsumer != null) {
|
||||
this.kafkaConsumer.wakeup();
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -1,143 +0,0 @@
|
||||
package io.kestra.indexer;
|
||||
|
||||
import com.fasterxml.jackson.core.JsonProcessingException;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import io.micronaut.test.extensions.junit5.annotation.MicronautTest;
|
||||
import org.apache.kafka.clients.consumer.ConsumerRecord;
|
||||
import org.apache.kafka.clients.consumer.MockConsumer;
|
||||
import org.apache.kafka.clients.consumer.OffsetResetStrategy;
|
||||
import org.apache.kafka.common.TopicPartition;
|
||||
import org.opensearch.client.RestHighLevelClient;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import io.kestra.core.metrics.MetricRegistry;
|
||||
import io.kestra.core.models.executions.Execution;
|
||||
import io.kestra.core.models.flows.Flow;
|
||||
import io.kestra.core.serializers.JacksonMapper;
|
||||
import io.kestra.core.utils.ExecutorsUtils;
|
||||
import io.kestra.core.utils.TestsUtils;
|
||||
import io.kestra.repository.elasticsearch.ElasticSearchIndicesService;
|
||||
import io.kestra.repository.elasticsearch.configs.IndicesConfig;
|
||||
import io.kestra.runner.kafka.configs.TopicsConfig;
|
||||
import io.kestra.runner.kafka.services.KafkaConsumerService;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import jakarta.inject.Inject;
|
||||
|
||||
import static org.hamcrest.MatcherAssert.assertThat;
|
||||
import static org.hamcrest.Matchers.is;
|
||||
import static org.mockito.ArgumentMatchers.any;
|
||||
import static org.mockito.Mockito.*;
|
||||
|
||||
|
||||
@MicronautTest
|
||||
class KafkaElasticIndexerTest {
|
||||
@Inject
|
||||
MetricRegistry metricRegistry;
|
||||
|
||||
@Inject
|
||||
RestHighLevelClient elasticClient;
|
||||
|
||||
@Inject
|
||||
IndexerConfig indexerConfig;
|
||||
|
||||
@Inject
|
||||
List<TopicsConfig> topicsConfig;
|
||||
|
||||
@Inject
|
||||
List<IndicesConfig> indicesConfigs;
|
||||
|
||||
@Inject
|
||||
ElasticSearchIndicesService elasticSearchIndicesService;
|
||||
|
||||
@Inject
|
||||
KafkaConsumerService kafkaConsumerService;
|
||||
|
||||
@Inject
|
||||
ExecutorsUtils executorsUtils;
|
||||
|
||||
@Test
|
||||
void run() throws IOException, InterruptedException {
|
||||
String topic = this.topicsConfig
|
||||
.stream()
|
||||
.filter(indicesConfig -> indicesConfig.getCls() == Execution.class)
|
||||
.findFirst()
|
||||
.orElseThrow()
|
||||
.getName();
|
||||
|
||||
CountDownLatch countDownLatch = new CountDownLatch(1);
|
||||
|
||||
RestHighLevelClient elasticClientSpy = spy(elasticClient);
|
||||
doAnswer(invocation -> {
|
||||
countDownLatch.countDown();
|
||||
return invocation.callRealMethod();
|
||||
}).when(elasticClientSpy).bulk(any(), any());
|
||||
|
||||
KafkaConsumerService kafkaConsumerServiceSpy = mock(KafkaConsumerService.class);
|
||||
MockConsumer<String, String> mockConsumer = mockConsumer(topic);
|
||||
doReturn(mockConsumer).when(kafkaConsumerServiceSpy).of(any(), any(), any());
|
||||
|
||||
ConsumerRecord<String, String> first = buildExecutionRecord(topic, 0);
|
||||
|
||||
mockConsumer.addRecord(first);
|
||||
mockConsumer.addRecord(buildExecutionRecord(topic, 1));
|
||||
mockConsumer.addRecord(buildExecutionRecord(topic, 2));
|
||||
mockConsumer.addRecord(buildExecutionRecord(topic, 3));
|
||||
mockConsumer.addRecord(buildExecutionRecord(topic, 4));
|
||||
mockConsumer.addRecord(buildRecord(topic, first.key(), null, 5));
|
||||
|
||||
KafkaElasticIndexer indexer = new KafkaElasticIndexer(
|
||||
metricRegistry,
|
||||
elasticClientSpy,
|
||||
indexerConfig,
|
||||
topicsConfig,
|
||||
indicesConfigs,
|
||||
elasticSearchIndicesService,
|
||||
kafkaConsumerServiceSpy,
|
||||
executorsUtils
|
||||
);
|
||||
|
||||
Thread thread = new Thread(indexer);
|
||||
thread.start();
|
||||
|
||||
countDownLatch.await();
|
||||
assertThat(countDownLatch.getCount(), is(0L));
|
||||
}
|
||||
|
||||
private ConsumerRecord<String, String> buildExecutionRecord(String topic, int offset) throws JsonProcessingException {
|
||||
Flow flow = TestsUtils.mockFlow();
|
||||
Execution execution = TestsUtils.mockExecution(flow, ImmutableMap.of());
|
||||
|
||||
return buildRecord(
|
||||
topic,
|
||||
execution.getId(),
|
||||
JacksonMapper.ofJson().writeValueAsString(execution),
|
||||
offset
|
||||
);
|
||||
}
|
||||
|
||||
private ConsumerRecord<String, String> buildRecord(String topic, String key, String value, int offset) {
|
||||
return new ConsumerRecord<>(
|
||||
topic,
|
||||
0,
|
||||
offset,
|
||||
key,
|
||||
value
|
||||
);
|
||||
}
|
||||
|
||||
private MockConsumer<String, String> mockConsumer(String index) {
|
||||
MockConsumer<String, String> consumer = spy(new MockConsumer<>(OffsetResetStrategy.EARLIEST));
|
||||
doNothing().when(consumer).subscribe(Collections.singleton(any()));
|
||||
|
||||
consumer.assign(Collections.singletonList(new TopicPartition(index, 0)));
|
||||
HashMap<TopicPartition, Long> beginningOffsets = new HashMap<>();
|
||||
beginningOffsets.put(new TopicPartition(index, 0), 0L);
|
||||
consumer.updateBeginningOffsets(beginningOffsets);
|
||||
|
||||
return consumer;
|
||||
}
|
||||
}
|
||||
@@ -1,49 +0,0 @@
|
||||
kestra:
|
||||
queue:
|
||||
type: kafka
|
||||
repository:
|
||||
type: elasticsearch
|
||||
|
||||
|
||||
elasticsearch:
|
||||
client:
|
||||
httpHosts: http://localhost:9200
|
||||
|
||||
defaults:
|
||||
indice-prefix: "unit_kestra_${random.shortuuid}_"
|
||||
|
||||
indices:
|
||||
executions:
|
||||
index: "${kestra.elasticsearch.defaults.indice-prefix}executions"
|
||||
cls: io.kestra.core.models.executions.Execution
|
||||
mapping-file: execution
|
||||
settings: &settings |
|
||||
{
|
||||
"index.number_of_replicas": 0
|
||||
}
|
||||
logs:
|
||||
index: "${kestra.elasticsearch.defaults.indice-prefix}logs"
|
||||
cls: io.kestra.core.models.executions.LogEntry
|
||||
mapping-file: log
|
||||
settings: *settings
|
||||
|
||||
kafka:
|
||||
client:
|
||||
properties:
|
||||
bootstrap.servers: "localhost:9092"
|
||||
|
||||
topics:
|
||||
execution:
|
||||
name: "${kestra.elasticsearch.defaults.indice-prefix}execution"
|
||||
cls: io.kestra.core.models.executions.Execution
|
||||
|
||||
logentry:
|
||||
cls: io.kestra.core.models.executions.LogEntry
|
||||
name: "${kestra.elasticsearch.defaults.indice-prefix}logs"
|
||||
|
||||
indexer:
|
||||
batchSize: 100
|
||||
models:
|
||||
- io.kestra.core.models.executions.Execution
|
||||
- io.kestra.core.models.triggers.Trigger
|
||||
- io.kestra.core.models.executions.LogEntry
|
||||
@@ -1,10 +0,0 @@
|
||||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<configuration debug="false">
|
||||
<include resource="logback/base.xml" />
|
||||
<include resource="logback/text.xml" />
|
||||
|
||||
<root level="WARN">
|
||||
<appender-ref ref="STDOUT" />
|
||||
<appender-ref ref="STDERR" />
|
||||
</root>
|
||||
</configuration>
|
||||
@@ -1 +0,0 @@
|
||||
mock-maker-inline
|
||||
@@ -1,14 +0,0 @@
|
||||
publishSonatypePublicationPublicationToSonatypeRepository.enabled = false
|
||||
|
||||
dependencies {
|
||||
implementation project(":core")
|
||||
|
||||
implementation (group: 'org.opensearch.client', name: 'opensearch-rest-high-level-client', version: "$opensearchVersion") {
|
||||
exclude group: 'org.apache.logging.log4j'
|
||||
}
|
||||
|
||||
testImplementation project(':core').sourceSets.test.output
|
||||
testImplementation project(':runner-memory')
|
||||
testImplementation project(':storage-local')
|
||||
testImplementation 'org.mockito:mockito-junit-jupiter:4.11.0'
|
||||
}
|
||||
@@ -1,529 +0,0 @@
|
||||
package io.kestra.repository.elasticsearch;
|
||||
|
||||
import com.fasterxml.jackson.core.JsonProcessingException;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.fasterxml.jackson.databind.exc.InvalidTypeIdException;
|
||||
import io.kestra.core.exceptions.DeserializationException;
|
||||
import io.kestra.core.models.flows.State;
|
||||
import io.kestra.core.repositories.ArrayListTotal;
|
||||
import io.kestra.core.serializers.JacksonMapper;
|
||||
import io.kestra.core.utils.ExecutorsUtils;
|
||||
import io.kestra.repository.elasticsearch.configs.IndicesConfig;
|
||||
import io.micronaut.data.model.Pageable;
|
||||
import io.micronaut.data.model.Sort;
|
||||
import lombok.SneakyThrows;
|
||||
import lombok.extern.slf4j.Slf4j;
|
||||
import org.apache.lucene.queryparser.classic.QueryParser;
|
||||
import org.opensearch.action.DocWriteResponse;
|
||||
import org.opensearch.action.ShardOperationFailedException;
|
||||
import org.opensearch.action.admin.indices.settings.get.GetSettingsRequest;
|
||||
import org.opensearch.action.admin.indices.settings.get.GetSettingsResponse;
|
||||
import org.opensearch.action.delete.DeleteRequest;
|
||||
import org.opensearch.action.delete.DeleteResponse;
|
||||
import org.opensearch.action.get.GetRequest;
|
||||
import org.opensearch.action.get.GetResponse;
|
||||
import org.opensearch.action.index.IndexRequest;
|
||||
import org.opensearch.action.index.IndexResponse;
|
||||
import org.opensearch.action.search.ClearScrollRequest;
|
||||
import org.opensearch.action.search.SearchRequest;
|
||||
import org.opensearch.action.search.SearchResponse;
|
||||
import org.opensearch.action.search.SearchScrollRequest;
|
||||
import org.opensearch.action.support.WriteRequest;
|
||||
import org.opensearch.action.support.replication.ReplicationResponse;
|
||||
import org.opensearch.action.update.UpdateRequest;
|
||||
import org.opensearch.action.update.UpdateResponse;
|
||||
import org.opensearch.client.RequestOptions;
|
||||
import org.opensearch.client.RestHighLevelClient;
|
||||
import org.opensearch.common.xcontent.XContentBuilder;
|
||||
import org.opensearch.common.xcontent.XContentType;
|
||||
import org.opensearch.common.unit.TimeValue;
|
||||
import org.opensearch.index.query.BoolQueryBuilder;
|
||||
import org.opensearch.index.query.QueryBuilder;
|
||||
import org.opensearch.index.query.QueryBuilders;
|
||||
import org.opensearch.index.query.QueryStringQueryBuilder;
|
||||
import org.opensearch.script.Script;
|
||||
import org.opensearch.script.ScriptType;
|
||||
import org.opensearch.search.SearchHit;
|
||||
import org.opensearch.search.aggregations.AggregationBuilder;
|
||||
import org.opensearch.search.aggregations.AggregationBuilders;
|
||||
import org.opensearch.search.aggregations.BucketOrder;
|
||||
import org.opensearch.search.aggregations.bucket.terms.Terms;
|
||||
import org.opensearch.search.aggregations.bucket.terms.TermsAggregationBuilder;
|
||||
import org.opensearch.search.builder.SearchSourceBuilder;
|
||||
import org.opensearch.search.sort.*;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.*;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.function.Consumer;
|
||||
import java.util.function.Predicate;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
import io.micronaut.core.annotation.Nullable;
|
||||
import javax.annotation.PostConstruct;
|
||||
import jakarta.inject.Inject;
|
||||
|
||||
import static org.opensearch.common.xcontent.XContentFactory.jsonBuilder;
|
||||
|
||||
@Slf4j
|
||||
abstract public class AbstractElasticSearchRepository<T> {
|
||||
protected static final ObjectMapper MAPPER = JacksonMapper.ofJson(false);
|
||||
private static ExecutorService poolExecutor;
|
||||
protected Class<T> cls;
|
||||
protected RestHighLevelClient client;
|
||||
protected ElasticSearchIndicesService elasticSearchIndicesService;
|
||||
protected Map<String, IndicesConfig> indicesConfigs;
|
||||
|
||||
@Inject
|
||||
public AbstractElasticSearchRepository(
|
||||
RestHighLevelClient client,
|
||||
ElasticSearchIndicesService elasticSearchIndicesService,
|
||||
ExecutorsUtils executorsUtils,
|
||||
Class<T> cls
|
||||
) {
|
||||
this.startExecutor(executorsUtils);
|
||||
|
||||
this.client = client;
|
||||
this.cls = cls;
|
||||
this.elasticSearchIndicesService = elasticSearchIndicesService;
|
||||
|
||||
this.indicesConfigs = elasticSearchIndicesService.findConfig(cls);
|
||||
}
|
||||
|
||||
private synchronized void startExecutor(ExecutorsUtils executorsUtils) {
|
||||
if (poolExecutor == null) {
|
||||
poolExecutor = executorsUtils.cachedThreadPool("elasticsearch-repository");
|
||||
}
|
||||
}
|
||||
|
||||
protected BoolQueryBuilder defaultFilter() {
|
||||
return QueryBuilders.boolQuery()
|
||||
.must(QueryBuilders.matchQuery("deleted", false));
|
||||
}
|
||||
|
||||
protected static QueryStringQueryBuilder queryString(@Nullable String query) {
|
||||
if (query == null) {
|
||||
return QueryBuilders.queryStringQuery("*");
|
||||
}
|
||||
|
||||
List<String> words = Arrays.stream(query.split("[^a-zA-Z0-9_.-]+"))
|
||||
.filter(r -> !r.equals(""))
|
||||
.map(QueryParser::escape)
|
||||
.collect(Collectors.toList());
|
||||
|
||||
String lucene = "(*" + String.join("*", words) + "*)^3 OR (*" + String.join("* AND *", words) + "*)";
|
||||
|
||||
|
||||
if (words.size() == 1) {
|
||||
lucene = "(" + QueryParser.escape(query) + ")^5 OR " + lucene;
|
||||
}
|
||||
|
||||
return QueryBuilders.queryStringQuery(lucene);
|
||||
}
|
||||
|
||||
protected Optional<T> getRequest(String index, String id) {
|
||||
BoolQueryBuilder bool = this.defaultFilter()
|
||||
.must(QueryBuilders.termQuery("_id", id));
|
||||
|
||||
SearchSourceBuilder sourceBuilder = new SearchSourceBuilder()
|
||||
.query(bool)
|
||||
.size(1);
|
||||
|
||||
SearchRequest searchRequest = this.searchRequest(
|
||||
index,
|
||||
sourceBuilder,
|
||||
false
|
||||
);
|
||||
|
||||
try {
|
||||
SearchResponse searchResponse = client.search(searchRequest, RequestOptions.DEFAULT);
|
||||
|
||||
return this.map(List.of(searchResponse.getHits().getHits()))
|
||||
.stream()
|
||||
.findFirst();
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
protected Optional<T> rawGetRequest(String index, String id) {
|
||||
GetRequest getRequest = new GetRequest(
|
||||
indexName(index),
|
||||
id
|
||||
);
|
||||
|
||||
try {
|
||||
GetResponse getResponse = client.get(getRequest, RequestOptions.DEFAULT);
|
||||
|
||||
if (!getResponse.isExists()) {
|
||||
return Optional.empty();
|
||||
}
|
||||
|
||||
return Optional.of(this.deserialize(getResponse.getSourceAsString()));
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
protected static void handleWriteErrors(DocWriteResponse indexResponse) throws Exception {
|
||||
ReplicationResponse.ShardInfo shardInfo = indexResponse.getShardInfo();
|
||||
if (shardInfo.getTotal() != shardInfo.getSuccessful()) {
|
||||
log.warn("Replication incomplete, expected " + shardInfo.getTotal() + ", got " + shardInfo.getSuccessful());
|
||||
}
|
||||
|
||||
if (shardInfo.getFailed() > 0) {
|
||||
throw new Exception(
|
||||
Stream.of(shardInfo.getFailures())
|
||||
.map(ShardOperationFailedException::reason)
|
||||
.collect(Collectors.joining("\n"))
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
protected IndexResponse putRequest(String index, String id, String json) {
|
||||
IndexRequest request = new IndexRequest(indexName(index));
|
||||
request.id(id);
|
||||
request.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE);
|
||||
|
||||
request.source(json, XContentType.JSON);
|
||||
|
||||
try {
|
||||
IndexResponse response = client.index(request, RequestOptions.DEFAULT);
|
||||
handleWriteErrors(response);
|
||||
|
||||
return response;
|
||||
} catch (Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
protected String indexName(String index) {
|
||||
return this.indicesConfigs.get(index).getIndex();
|
||||
}
|
||||
|
||||
protected IndexResponse putRequest(String index, String id, T source) {
|
||||
try {
|
||||
String json = MAPPER.writeValueAsString(source);
|
||||
return this.putRequest(index, id, json);
|
||||
} catch (JsonProcessingException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
protected UpdateResponse updateRequest(String index, String id, XContentBuilder doc) {
|
||||
UpdateRequest request = new UpdateRequest(indexName(index), id);
|
||||
request.doc(doc);
|
||||
request.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE);
|
||||
|
||||
try {
|
||||
UpdateResponse response = client.update(request, RequestOptions.DEFAULT);
|
||||
handleWriteErrors(response);
|
||||
|
||||
return response;
|
||||
} catch (Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@SneakyThrows
|
||||
protected UpdateResponse deleteRequest(String index, String id) {
|
||||
XContentBuilder delete = jsonBuilder()
|
||||
.startObject()
|
||||
.field("deleted", true)
|
||||
.endObject();
|
||||
|
||||
return this.updateRequest(index, id, delete);
|
||||
}
|
||||
|
||||
protected DeleteResponse rawDeleteRequest(String index, String id) {
|
||||
DeleteRequest request = new DeleteRequest(indexName(index), id);
|
||||
request.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE);
|
||||
|
||||
try {
|
||||
DeleteResponse response = client.delete(request, RequestOptions.DEFAULT);
|
||||
handleWriteErrors(response);
|
||||
|
||||
return response;
|
||||
} catch (Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
protected SearchRequest searchRequest(String index, SearchSourceBuilder sourceBuilder, boolean scroll) {
|
||||
SearchRequest searchRequest = new SearchRequest()
|
||||
.indices(indexName(index))
|
||||
.source(sourceBuilder);
|
||||
|
||||
if (scroll) {
|
||||
searchRequest.scroll(new TimeValue(60000));
|
||||
}
|
||||
|
||||
return searchRequest;
|
||||
}
|
||||
|
||||
protected Predicate<Sort.Order> isDurationSort() {
|
||||
return order -> order != null
|
||||
&& order.getProperty() != null
|
||||
&& order.getProperty().contains("state.duration");
|
||||
}
|
||||
|
||||
protected SortOrder toSortOrder(Sort.Order.Direction sortDirection) {
|
||||
return sortDirection == Sort.Order.Direction.ASC ? SortOrder.ASC : SortOrder.DESC;
|
||||
}
|
||||
|
||||
protected SortBuilder<FieldSortBuilder> toFieldSortBuilder(Sort.Order order) {
|
||||
return SortBuilders
|
||||
.fieldSort(order.getProperty())
|
||||
.order(toSortOrder(order.getDirection()));
|
||||
}
|
||||
|
||||
public static final String DURATION_SORT_SCRIPT_CODE = "" +
|
||||
"ZonedDateTime start = doc[params.fieldPrefix+'state.startDate'].value;\n" +
|
||||
"ZonedDateTime end = ZonedDateTime.ofInstant(Instant.ofEpochMilli(params.now), ZoneId.of('Z'));\n" +
|
||||
"\n" +
|
||||
"if(!params.runningStates.contains(doc[params.fieldPrefix+'state.current'].value) && doc[params.fieldPrefix+'state.endDate'].size() > 0){\n" +
|
||||
" end = doc[params.fieldPrefix+'state.endDate'].value; \n" +
|
||||
"}\n" +
|
||||
"return ChronoUnit.MILLIS.between(start,end);";
|
||||
|
||||
protected SortBuilder<ScriptSortBuilder> createDurationSortScript(Sort.Order sortByDuration, boolean nested) {
|
||||
return SortBuilders
|
||||
.scriptSort(new Script(ScriptType.INLINE,
|
||||
Script.DEFAULT_SCRIPT_LANG,
|
||||
DURATION_SORT_SCRIPT_CODE,
|
||||
Collections.emptyMap(),
|
||||
Map.of("now", new Date().getTime(),
|
||||
"runningStates", Arrays.stream(State.runningTypes()).map(type -> type.name()).toArray(String[]::new),
|
||||
"fieldPrefix", nested ? "taskRunList." : ""
|
||||
)
|
||||
), ScriptSortBuilder.ScriptSortType.NUMBER
|
||||
)
|
||||
.order(toSortOrder(sortByDuration.getDirection()));
|
||||
}
|
||||
|
||||
protected List<SortBuilder<?>> defaultSorts(Pageable pageable, boolean nested) {
|
||||
List<SortBuilder<?>> sorts = new ArrayList<>();
|
||||
|
||||
// Use script sort for duration field
|
||||
pageable
|
||||
.getSort()
|
||||
.getOrderBy()
|
||||
.stream()
|
||||
.filter(isDurationSort())
|
||||
.findFirst()
|
||||
.ifPresent(order -> sorts.add(createDurationSortScript(order, nested)));
|
||||
|
||||
// Use field sort for all other fields
|
||||
sorts.addAll(pageable
|
||||
.getSort()
|
||||
.getOrderBy()
|
||||
.stream()
|
||||
.filter(Predicate.not(isDurationSort()))
|
||||
.map(this::toFieldSortBuilder)
|
||||
.collect(Collectors.toList()));
|
||||
|
||||
return sorts;
|
||||
}
|
||||
|
||||
protected SearchSourceBuilder searchSource(
|
||||
QueryBuilder query,
|
||||
Optional<List<AggregationBuilder>> aggregations,
|
||||
@Nullable Pageable pageable
|
||||
) {
|
||||
SearchSourceBuilder sourceBuilder = new SearchSourceBuilder()
|
||||
.query(query);
|
||||
|
||||
if (pageable != null && pageable.getSize() == -1) {
|
||||
sourceBuilder.size(1000);
|
||||
} else if (pageable != null) {
|
||||
sourceBuilder
|
||||
.size(pageable.getSize())
|
||||
.from(Math.toIntExact(pageable.getOffset() - pageable.getSize()));
|
||||
|
||||
for (SortBuilder<?> s : defaultSorts(pageable, false)) {
|
||||
sourceBuilder = sourceBuilder.sort(s);
|
||||
}
|
||||
} else {
|
||||
sourceBuilder.size(0);
|
||||
}
|
||||
|
||||
if (aggregations.isPresent()) {
|
||||
for (AggregationBuilder aggregation : aggregations.get()) {
|
||||
sourceBuilder.aggregation(aggregation);
|
||||
}
|
||||
}
|
||||
|
||||
return sourceBuilder;
|
||||
}
|
||||
|
||||
protected ArrayListTotal<T> findQueryString(String index, String query, Pageable pageable) {
|
||||
BoolQueryBuilder bool = this.defaultFilter();
|
||||
|
||||
if (query != null) {
|
||||
bool.must(queryString(query).field("*.fulltext"));
|
||||
}
|
||||
|
||||
SearchSourceBuilder sourceBuilder = this.searchSource(bool, Optional.empty(), pageable);
|
||||
|
||||
return this.query(index, sourceBuilder);
|
||||
}
|
||||
|
||||
protected List<T> searchByIds(String index, List<String> ids) {
|
||||
if (ids == null) {
|
||||
return new ArrayList<>();
|
||||
}
|
||||
|
||||
BoolQueryBuilder bool = this.defaultFilter()
|
||||
.must(QueryBuilders.idsQuery()
|
||||
.addIds(ids.toArray(String[]::new))
|
||||
);
|
||||
|
||||
SearchSourceBuilder sourceBuilder = new SearchSourceBuilder()
|
||||
.query(bool);
|
||||
|
||||
return this.scroll(index, sourceBuilder);
|
||||
}
|
||||
|
||||
protected List<T> map(List<SearchHit> searchHits) {
|
||||
return searchHits
|
||||
.stream()
|
||||
.map(searchHit -> this.deserialize(searchHit.getSourceAsString()))
|
||||
.filter(Objects::nonNull)
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
|
||||
protected T deserialize(String source) {
|
||||
try {
|
||||
return MAPPER.readValue(source, this.cls);
|
||||
} catch (InvalidTypeIdException e) {
|
||||
throw new DeserializationException(e);
|
||||
} catch (IOException e) {
|
||||
throw new DeserializationException(e);
|
||||
}
|
||||
}
|
||||
|
||||
protected ArrayListTotal<T> query(String index, SearchSourceBuilder sourceBuilder) {
|
||||
SearchRequest searchRequest = searchRequest(index, sourceBuilder, false);
|
||||
|
||||
try {
|
||||
SearchResponse searchResponse = client.search(searchRequest, RequestOptions.DEFAULT);
|
||||
return new ArrayListTotal<>(this.map(List.of(searchResponse.getHits().getHits())), searchResponse.getHits().getTotalHits().value);
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
protected List<T> scroll(String index, SearchSourceBuilder sourceBuilder) {
|
||||
List<T> result = new ArrayList<>();
|
||||
|
||||
this.internalScroll(index, sourceBuilder, documentFields -> {
|
||||
result.addAll(this.map(List.of(documentFields)));
|
||||
});
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
protected void scroll(String index, SearchSourceBuilder sourceBuilder, Consumer<T> consumer) {
|
||||
this.internalScroll(
|
||||
index,
|
||||
sourceBuilder,
|
||||
documentFields -> this.map(List.of(documentFields)).forEach(consumer)
|
||||
);
|
||||
}
|
||||
|
||||
protected void internalScroll(String index, SearchSourceBuilder sourceBuilder, Consumer<SearchHit> consumer) {
|
||||
String scrollId = null;
|
||||
SearchRequest searchRequest = searchRequest(index, sourceBuilder, true);
|
||||
try {
|
||||
SearchResponse searchResponse;
|
||||
searchResponse = client.search(searchRequest, RequestOptions.DEFAULT);
|
||||
scrollId = searchResponse.getScrollId();
|
||||
|
||||
do {
|
||||
List.of(searchResponse.getHits().getHits())
|
||||
.forEach(consumer);
|
||||
|
||||
SearchScrollRequest searchScrollRequest = new SearchScrollRequest()
|
||||
.scrollId(scrollId)
|
||||
.scroll(new TimeValue(60000));
|
||||
|
||||
searchResponse = client.scroll(searchScrollRequest, RequestOptions.DEFAULT);
|
||||
} while (searchResponse.getHits().getHits().length != 0);
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
} finally {
|
||||
this.clearScrollId(scrollId);
|
||||
}
|
||||
}
|
||||
|
||||
private void clearScrollId(String scrollId) {
|
||||
if (scrollId == null) {
|
||||
return;
|
||||
}
|
||||
|
||||
poolExecutor.execute(() -> {
|
||||
ClearScrollRequest clearScrollRequest = new ClearScrollRequest();
|
||||
clearScrollRequest.addScrollId(scrollId);
|
||||
|
||||
try {
|
||||
client.clearScroll(clearScrollRequest, RequestOptions.DEFAULT);
|
||||
} catch (IOException e) {
|
||||
log.warn("Failed to clear scroll", e);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
@PostConstruct
|
||||
public void initMapping() {
|
||||
this.elasticSearchIndicesService.createIndice(this.indicesConfigs);
|
||||
this.elasticSearchIndicesService.updateMapping(this.indicesConfigs);
|
||||
}
|
||||
|
||||
|
||||
protected List<String> findDistinct(String index, String field) {
|
||||
BoolQueryBuilder query = this.defaultFilter();
|
||||
|
||||
// We want to keep only "distinct" values of field "namespace"
|
||||
// @TODO: use includeExclude(new IncludeExclude(0, 10)) to partition results
|
||||
TermsAggregationBuilder termsAggregationBuilder = AggregationBuilders
|
||||
.terms("distinct_values")
|
||||
.field(field)
|
||||
.size(10000)
|
||||
.order(BucketOrder.key(true));
|
||||
|
||||
SearchSourceBuilder sourceBuilder = new SearchSourceBuilder()
|
||||
.query(query)
|
||||
.aggregation(termsAggregationBuilder);
|
||||
|
||||
SearchRequest searchRequest = searchRequest(index, sourceBuilder, false);
|
||||
|
||||
try {
|
||||
SearchResponse searchResponse = client.search(searchRequest, RequestOptions.DEFAULT);
|
||||
|
||||
Terms namespaces = searchResponse.getAggregations().get("distinct_values");
|
||||
|
||||
return new ArrayListTotal<>(
|
||||
namespaces.getBuckets()
|
||||
.stream()
|
||||
.map(o -> {
|
||||
return o.getKey().toString();
|
||||
})
|
||||
.collect(Collectors.toList()),
|
||||
namespaces.getBuckets().size()
|
||||
);
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
protected GetSettingsResponse getSettings(String index, boolean includeDefaults) {
|
||||
GetSettingsRequest request = new GetSettingsRequest()
|
||||
.indices(indexName(index))
|
||||
.includeDefaults(includeDefaults);
|
||||
try {
|
||||
return this.client.indices().getSettings(request, RequestOptions.DEFAULT);
|
||||
} catch (IOException ioe) {
|
||||
throw new RuntimeException(ioe);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -1,587 +0,0 @@
|
||||
package io.kestra.repository.elasticsearch;
|
||||
|
||||
import io.kestra.core.events.CrudEvent;
|
||||
import io.kestra.core.events.CrudEventType;
|
||||
import io.kestra.core.models.executions.Execution;
|
||||
import io.kestra.core.models.executions.TaskRun;
|
||||
import io.kestra.core.models.executions.statistics.DailyExecutionStatistics;
|
||||
import io.kestra.core.models.executions.statistics.ExecutionCount;
|
||||
import io.kestra.core.models.executions.statistics.Flow;
|
||||
import io.kestra.core.models.flows.State;
|
||||
import io.kestra.core.queues.QueueFactoryInterface;
|
||||
import io.kestra.core.queues.QueueInterface;
|
||||
import io.kestra.core.repositories.ArrayListTotal;
|
||||
import io.kestra.core.repositories.ExecutionRepositoryInterface;
|
||||
import io.kestra.core.utils.ExecutorsUtils;
|
||||
import io.micronaut.context.event.ApplicationEventPublisher;
|
||||
import io.micronaut.data.model.Pageable;
|
||||
import io.reactivex.BackpressureStrategy;
|
||||
import io.reactivex.Flowable;
|
||||
import jakarta.inject.Inject;
|
||||
import jakarta.inject.Named;
|
||||
import jakarta.inject.Singleton;
|
||||
import org.opensearch.action.search.SearchRequest;
|
||||
import org.opensearch.action.search.SearchResponse;
|
||||
import org.opensearch.client.RequestOptions;
|
||||
import org.opensearch.client.RestHighLevelClient;
|
||||
import org.opensearch.common.settings.Settings;
|
||||
import org.opensearch.index.IndexSettings;
|
||||
import org.opensearch.index.query.BoolQueryBuilder;
|
||||
import org.opensearch.index.query.QueryBuilders;
|
||||
import org.opensearch.search.aggregations.AggregationBuilder;
|
||||
import org.opensearch.search.aggregations.AggregationBuilders;
|
||||
import org.opensearch.search.aggregations.bucket.filter.Filter;
|
||||
import org.opensearch.search.aggregations.bucket.filter.FiltersAggregator;
|
||||
import org.opensearch.search.aggregations.bucket.filter.ParsedFilters;
|
||||
import org.opensearch.search.aggregations.bucket.histogram.DateHistogramAggregationBuilder;
|
||||
import org.opensearch.search.aggregations.bucket.histogram.DateHistogramInterval;
|
||||
import org.opensearch.search.aggregations.bucket.histogram.LongBounds;
|
||||
import org.opensearch.search.aggregations.bucket.histogram.ParsedDateHistogram;
|
||||
import org.opensearch.search.aggregations.bucket.nested.NestedAggregationBuilder;
|
||||
import org.opensearch.search.aggregations.bucket.nested.ParsedNested;
|
||||
import org.opensearch.search.aggregations.bucket.terms.ParsedStringTerms;
|
||||
import org.opensearch.search.aggregations.bucket.terms.Terms;
|
||||
import org.opensearch.search.aggregations.bucket.terms.TermsAggregationBuilder;
|
||||
import org.opensearch.search.aggregations.metrics.ParsedStats;
|
||||
import org.opensearch.search.aggregations.metrics.TopHits;
|
||||
import org.opensearch.search.builder.SearchSourceBuilder;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.time.Duration;
|
||||
import java.time.LocalDate;
|
||||
import java.time.ZoneId;
|
||||
import java.time.ZonedDateTime;
|
||||
import java.time.format.DateTimeFormatter;
|
||||
import java.util.*;
|
||||
import java.util.stream.Collectors;
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
@Singleton
|
||||
@ElasticSearchRepositoryEnabled
|
||||
public class ElasticSearchExecutionRepository extends AbstractElasticSearchRepository<Execution> implements ExecutionRepositoryInterface {
|
||||
private static final String INDEX_NAME = "executions";
|
||||
public static final String DATE_FORMAT = "yyyy-MM-dd";
|
||||
|
||||
public static final String NESTED_AGG = "NESTED";
|
||||
public static final String DATE_AGG = "DATE";
|
||||
public static final String NAMESPACE_AGG = "NAMESPACE";
|
||||
public static final String FLOW_AGG = "FLOW";
|
||||
public static final String COUNT_AGG = "COUNT";
|
||||
public static final String DURATION_AGG = "DURATION";
|
||||
|
||||
private final ApplicationEventPublisher<CrudEvent<Execution>> eventPublisher;
|
||||
private final QueueInterface<Execution> executionQueue;
|
||||
|
||||
@Inject
|
||||
public ElasticSearchExecutionRepository(
|
||||
RestHighLevelClient client,
|
||||
ElasticSearchIndicesService elasticSearchIndicesService,
|
||||
ExecutorsUtils executorsUtils,
|
||||
@Named(QueueFactoryInterface.EXECUTION_NAMED) QueueInterface<Execution> executionQueue,
|
||||
ApplicationEventPublisher<CrudEvent<Execution>> eventPublisher
|
||||
) {
|
||||
super(client, elasticSearchIndicesService, executorsUtils, Execution.class);
|
||||
|
||||
this.eventPublisher = eventPublisher;
|
||||
this.executionQueue = executionQueue;
|
||||
}
|
||||
|
||||
public Boolean isTaskRunEnabled() {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<Execution> findById(String id) {
|
||||
return this.getRequest(INDEX_NAME, id);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<String, Map<String, List<DailyExecutionStatistics>>> dailyGroupByFlowStatistics(
|
||||
@Nullable String query,
|
||||
@Nullable String namespace,
|
||||
@Nullable String flowId,
|
||||
@Nullable List<FlowFilter> flows,
|
||||
@Nullable ZonedDateTime startDate,
|
||||
@Nullable ZonedDateTime endDate,
|
||||
boolean groupByNamespaceOnly
|
||||
) {
|
||||
if (startDate == null) {
|
||||
startDate = ZonedDateTime.now().minusDays(30);
|
||||
}
|
||||
|
||||
if (endDate == null) {
|
||||
endDate = ZonedDateTime.now();
|
||||
}
|
||||
|
||||
TermsAggregationBuilder agg = AggregationBuilders.terms(NAMESPACE_AGG)
|
||||
.size(groupByNamespaceOnly ? 25 : 10000)
|
||||
.field("namespace")
|
||||
.subAggregation(
|
||||
groupByNamespaceOnly ?
|
||||
dailyExecutionStatisticsFinalAgg(startDate, endDate, false) :
|
||||
AggregationBuilders.terms(FLOW_AGG)
|
||||
.size(10000)
|
||||
.field("flowId")
|
||||
.subAggregation(dailyExecutionStatisticsFinalAgg(startDate, endDate, false))
|
||||
);
|
||||
|
||||
SearchSourceBuilder sourceBuilder = this.searchSource(
|
||||
this.filters(query, startDate, endDate, namespace, flowId, flows, null),
|
||||
Optional.of(Collections.singletonList(
|
||||
agg
|
||||
)),
|
||||
null
|
||||
);
|
||||
|
||||
try {
|
||||
SearchRequest searchRequest = searchRequest(INDEX_NAME, sourceBuilder, false);
|
||||
SearchResponse searchResponse = client.search(searchRequest, RequestOptions.DEFAULT);
|
||||
|
||||
Map<String, Map<String, List<DailyExecutionStatistics>>> result = new HashMap<>();
|
||||
|
||||
((ParsedStringTerms) searchResponse.getAggregations().get(NAMESPACE_AGG)).getBuckets()
|
||||
.forEach(namespaceBucket -> {
|
||||
final String currentNamespace = namespaceBucket.getKeyAsString();
|
||||
|
||||
if (groupByNamespaceOnly) {
|
||||
this.parseDateAgg(namespaceBucket, result, currentNamespace, "*");
|
||||
} else {
|
||||
((ParsedStringTerms) namespaceBucket.getAggregations().get(FLOW_AGG)).getBuckets()
|
||||
.forEach(flowBucket -> {
|
||||
final String currentFlowId = flowBucket.getKeyAsString();
|
||||
|
||||
this.parseDateAgg(flowBucket, result, currentNamespace, currentFlowId);
|
||||
});
|
||||
}
|
||||
});
|
||||
|
||||
return result;
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
private <T extends Terms.Bucket> void parseDateAgg(T bucket, Map<String, Map<String, List<DailyExecutionStatistics>>> result, String namespace, String flowId) {
|
||||
((ParsedDateHistogram) bucket.getAggregations().get(DATE_AGG)).getBuckets()
|
||||
.forEach(dateBucket -> {
|
||||
final LocalDate currentStartDate = LocalDate.parse(
|
||||
dateBucket.getKeyAsString(),
|
||||
DateTimeFormatter.ISO_LOCAL_DATE
|
||||
);
|
||||
|
||||
ParsedStringTerms countAgg = dateBucket.getAggregations().get(COUNT_AGG);
|
||||
ParsedStats durationAgg = dateBucket.getAggregations().get(DURATION_AGG);
|
||||
|
||||
result.compute(namespace, (namespaceKey, namespaceMap) -> {
|
||||
if (namespaceMap == null) {
|
||||
namespaceMap = new HashMap<>();
|
||||
}
|
||||
|
||||
namespaceMap.compute(flowId, (flowKey, flowList) -> {
|
||||
if (flowList == null) {
|
||||
flowList = new ArrayList<>();
|
||||
}
|
||||
|
||||
flowList.add(dailyExecutionStatisticsMap(countAgg, durationAgg, currentStartDate));
|
||||
|
||||
return flowList;
|
||||
});
|
||||
|
||||
return namespaceMap;
|
||||
});
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<ExecutionCount> executionCounts(
|
||||
List<Flow> flows,
|
||||
@Nullable List<State.Type> states,
|
||||
@Nullable ZonedDateTime startDate,
|
||||
@Nullable ZonedDateTime endDate
|
||||
) {
|
||||
if (startDate == null) {
|
||||
startDate = ZonedDateTime.now().minusDays(30);
|
||||
}
|
||||
|
||||
if (endDate == null) {
|
||||
endDate = ZonedDateTime.now();
|
||||
}
|
||||
|
||||
SearchSourceBuilder sourceBuilder = this.searchSource(
|
||||
this.filters(null, startDate, endDate, null, null, null, states),
|
||||
Optional.of(Collections.singletonList(
|
||||
AggregationBuilders.filters(
|
||||
"FILTERS",
|
||||
flows
|
||||
.stream()
|
||||
.map(flow -> new FiltersAggregator.KeyedFilter(
|
||||
flow.getNamespace() + "/" + flow.getFlowId(),
|
||||
QueryBuilders.boolQuery()
|
||||
.must(QueryBuilders.termQuery("namespace", flow.getNamespace()))
|
||||
.must(QueryBuilders.termQuery("flowId", flow.getFlowId()))
|
||||
))
|
||||
.toArray(FiltersAggregator.KeyedFilter[]::new)
|
||||
)
|
||||
)),
|
||||
null
|
||||
);
|
||||
|
||||
try {
|
||||
SearchRequest searchRequest = searchRequest(INDEX_NAME, sourceBuilder, false);
|
||||
SearchResponse searchResponse = client.search(searchRequest, RequestOptions.DEFAULT);
|
||||
|
||||
List<ExecutionCount> result = new ArrayList<>();
|
||||
|
||||
((ParsedFilters) searchResponse.getAggregations().get("FILTERS")).getBuckets()
|
||||
.forEach(filtersBuckets -> {
|
||||
final String key = filtersBuckets.getKeyAsString();
|
||||
|
||||
result.add(new ExecutionCount(
|
||||
key.split("/")[0],
|
||||
key.split("/")[1],
|
||||
filtersBuckets.getDocCount()
|
||||
));
|
||||
});
|
||||
|
||||
return result;
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<DailyExecutionStatistics> dailyStatistics(
|
||||
@Nullable String query,
|
||||
@Nullable String namespace,
|
||||
@Nullable String flowId,
|
||||
@Nullable ZonedDateTime startDate,
|
||||
@Nullable ZonedDateTime endDate,
|
||||
boolean isTaskRun
|
||||
) {
|
||||
if (startDate == null) {
|
||||
startDate = ZonedDateTime.now().minusDays(30);
|
||||
}
|
||||
|
||||
if (endDate == null) {
|
||||
endDate = ZonedDateTime.now();
|
||||
}
|
||||
|
||||
AggregationBuilder agg = dailyExecutionStatisticsFinalAgg(startDate, endDate, isTaskRun);
|
||||
|
||||
if (isTaskRun) {
|
||||
agg = AggregationBuilders.nested(NESTED_AGG, "taskRunList")
|
||||
.subAggregation(agg);
|
||||
}
|
||||
|
||||
SearchSourceBuilder sourceBuilder = this.searchSource(
|
||||
this.filters(query, startDate, endDate, namespace, flowId, null, null),
|
||||
Optional.of(Collections.singletonList(
|
||||
agg
|
||||
)),
|
||||
null
|
||||
);
|
||||
|
||||
try {
|
||||
SearchRequest searchRequest = searchRequest(INDEX_NAME, sourceBuilder, false);
|
||||
SearchResponse searchResponse = client.search(searchRequest, RequestOptions.DEFAULT);
|
||||
ParsedDateHistogram groupAgg = isTaskRun ?
|
||||
((ParsedNested) searchResponse.getAggregations().get(NESTED_AGG)).getAggregations().get(DATE_AGG) :
|
||||
searchResponse.getAggregations().get(DATE_AGG);
|
||||
|
||||
List<DailyExecutionStatistics> result = new ArrayList<>();
|
||||
|
||||
groupAgg.getBuckets().forEach(bucket -> {
|
||||
ParsedStringTerms countAgg = bucket.getAggregations().get(COUNT_AGG);
|
||||
ParsedStats durationAgg = bucket.getAggregations().get(DURATION_AGG);
|
||||
|
||||
final LocalDate currentStartDate = LocalDate.parse(bucket.getKeyAsString(), DateTimeFormatter.ISO_LOCAL_DATE);
|
||||
|
||||
result.add(dailyExecutionStatisticsMap(countAgg, durationAgg, currentStartDate));
|
||||
|
||||
});
|
||||
|
||||
return result;
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
private BoolQueryBuilder filters(
|
||||
String query,
|
||||
ZonedDateTime startDate,
|
||||
ZonedDateTime endDate,
|
||||
String namespace,
|
||||
String flowId,
|
||||
List<FlowFilter> flows,
|
||||
List<State.Type> state
|
||||
) {
|
||||
return this.filters(query, startDate, endDate, namespace, flowId, flows, state, false);
|
||||
}
|
||||
|
||||
private BoolQueryBuilder filters(
|
||||
String query,
|
||||
ZonedDateTime startDate,
|
||||
ZonedDateTime endDate,
|
||||
String namespace,
|
||||
String flowId,
|
||||
List<FlowFilter> flows,
|
||||
List<State.Type> state,
|
||||
boolean isTaskRun
|
||||
) {
|
||||
String prefix = isTaskRun ? "taskRunList." : "";
|
||||
|
||||
BoolQueryBuilder bool = isTaskRun ? QueryBuilders.boolQuery() : this.defaultFilter();
|
||||
|
||||
if (query != null) {
|
||||
if (isTaskRun) {
|
||||
bool.must(queryString(query));
|
||||
} else {
|
||||
bool.must(queryString(query).field("*.fulltext"));
|
||||
}
|
||||
}
|
||||
|
||||
if (flows != null) {
|
||||
BoolQueryBuilder flowsBool = QueryBuilders.boolQuery()
|
||||
.minimumShouldMatch(1);
|
||||
|
||||
flows
|
||||
.stream()
|
||||
.map(e -> QueryBuilders.boolQuery()
|
||||
.must(QueryBuilders.matchQuery(prefix + "flowId", e.getId()))
|
||||
.must(QueryBuilders.matchQuery(prefix + "namespace", e.getNamespace()))
|
||||
)
|
||||
.forEach(flowsBool::should);
|
||||
|
||||
bool.must(flowsBool);
|
||||
}
|
||||
|
||||
if (flowId != null && namespace != null) {
|
||||
bool = bool.must(QueryBuilders.matchQuery(prefix + "flowId", flowId));
|
||||
bool = bool.must(QueryBuilders.matchQuery(prefix + "namespace", namespace));
|
||||
} else if (namespace != null) {
|
||||
bool = bool.must(QueryBuilders.prefixQuery(prefix + "namespace", namespace));
|
||||
}
|
||||
|
||||
if (startDate != null) {
|
||||
bool.must(QueryBuilders.rangeQuery(prefix + "state.startDate").gte(startDate));
|
||||
}
|
||||
|
||||
if (endDate != null) {
|
||||
bool.must(QueryBuilders.rangeQuery(prefix + "state.startDate").lte(endDate));
|
||||
}
|
||||
|
||||
if (state != null) {
|
||||
bool = bool.must(QueryBuilders.termsQuery(prefix + "state.current", stateConvert(state)));
|
||||
}
|
||||
|
||||
return bool;
|
||||
}
|
||||
|
||||
private static DateHistogramAggregationBuilder dailyExecutionStatisticsFinalAgg(
|
||||
ZonedDateTime startDate,
|
||||
ZonedDateTime endDate,
|
||||
boolean isTaskRun
|
||||
) {
|
||||
return AggregationBuilders.dateHistogram(DATE_AGG)
|
||||
.field((isTaskRun ? "taskRunList." : "") + "state.startDate")
|
||||
.format(DATE_FORMAT)
|
||||
.minDocCount(0)
|
||||
.fixedInterval(DateHistogramInterval.DAY)
|
||||
.extendedBounds(new LongBounds(
|
||||
startDate.format(DateTimeFormatter.ofPattern(DATE_FORMAT)),
|
||||
endDate.format(DateTimeFormatter.ofPattern(DATE_FORMAT))
|
||||
))
|
||||
.timeZone(ZoneId.systemDefault())
|
||||
.subAggregation(AggregationBuilders.stats(DURATION_AGG).
|
||||
field((isTaskRun ? "taskRunList." : "") + "state.duration")
|
||||
)
|
||||
.subAggregation(AggregationBuilders.terms(COUNT_AGG)
|
||||
.size(10000)
|
||||
.field((isTaskRun ? "taskRunList." : "") + "state.current")
|
||||
);
|
||||
}
|
||||
|
||||
private static DailyExecutionStatistics dailyExecutionStatisticsMap(ParsedStringTerms countAgg, ParsedStats durationAgg, LocalDate startDate) {
|
||||
DailyExecutionStatistics build = DailyExecutionStatistics.builder()
|
||||
.startDate(startDate)
|
||||
.duration(DailyExecutionStatistics.Duration.builder()
|
||||
.avg(durationFromDouble(durationAgg.getAvg()))
|
||||
.min(durationFromDouble(durationAgg.getMin()))
|
||||
.max(durationFromDouble(durationAgg.getMax()))
|
||||
.sum(durationFromDouble(durationAgg.getSum()))
|
||||
.count(durationAgg.getCount())
|
||||
.build()
|
||||
)
|
||||
.build();
|
||||
|
||||
countAgg.getBuckets()
|
||||
.forEach(item -> build.getExecutionCounts()
|
||||
.compute(
|
||||
State.Type.valueOf(item.getKeyAsString()),
|
||||
(type, current) -> item.getDocCount()
|
||||
)
|
||||
);
|
||||
|
||||
return build;
|
||||
}
|
||||
|
||||
private static Duration durationFromDouble(double val) {
|
||||
return Duration.ofMillis(
|
||||
(long) (val * 1000)
|
||||
);
|
||||
}
|
||||
|
||||
private static List<String> stateConvert(List<State.Type> state) {
|
||||
return state
|
||||
.stream()
|
||||
.map(Enum::name)
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
|
||||
@Override
|
||||
public ArrayListTotal<Execution> find(
|
||||
Pageable pageable,
|
||||
@Nullable String query,
|
||||
@Nullable String namespace,
|
||||
@Nullable String flowId,
|
||||
@Nullable ZonedDateTime startDate,
|
||||
@Nullable ZonedDateTime endDate,
|
||||
@Nullable List<State.Type> state
|
||||
) {
|
||||
BoolQueryBuilder bool = this.filters(query, startDate, endDate, namespace, flowId, null, state);
|
||||
|
||||
SearchSourceBuilder sourceBuilder = this.searchSource(bool, Optional.empty(), pageable);
|
||||
|
||||
return this.query(INDEX_NAME, sourceBuilder);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Flowable<Execution> find(
|
||||
@Nullable String query,
|
||||
@Nullable String namespace,
|
||||
@Nullable String flowId,
|
||||
@Nullable ZonedDateTime startDate,
|
||||
@Nullable ZonedDateTime endDate,
|
||||
@Nullable List<State.Type> state
|
||||
) {
|
||||
BoolQueryBuilder bool = this.filters(null, null, endDate, namespace, flowId, null, state);
|
||||
|
||||
SearchSourceBuilder sourceBuilder = new SearchSourceBuilder()
|
||||
.query(bool);
|
||||
|
||||
return Flowable.create(
|
||||
emitter -> {
|
||||
this.scroll(INDEX_NAME, sourceBuilder, emitter::onNext);
|
||||
|
||||
emitter.onComplete();
|
||||
},
|
||||
BackpressureStrategy.BUFFER
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ArrayListTotal<TaskRun> findTaskRun(
|
||||
Pageable pageable,
|
||||
@Nullable String query,
|
||||
@Nullable String namespace,
|
||||
@Nullable String flowId,
|
||||
@Nullable ZonedDateTime startDate,
|
||||
@Nullable ZonedDateTime endDate,
|
||||
@Nullable List<State.Type> states
|
||||
) {
|
||||
BoolQueryBuilder filterAggQuery = filters(query, startDate, endDate, namespace, flowId, null, states, true);
|
||||
|
||||
NestedAggregationBuilder nestedAgg = AggregationBuilders
|
||||
.nested("NESTED", "taskRunList")
|
||||
.subAggregation(
|
||||
AggregationBuilders.filter("FILTER", filterAggQuery)
|
||||
.subAggregation(
|
||||
AggregationBuilders
|
||||
.topHits("TOPHITS")
|
||||
.size(pageable.getSize())
|
||||
.sorts(defaultSorts(pageable, true))
|
||||
.from(Math.toIntExact(pageable.getOffset() - pageable.getSize()))
|
||||
)
|
||||
);
|
||||
|
||||
BoolQueryBuilder mainQuery = this.defaultFilter();
|
||||
|
||||
SearchSourceBuilder sourceBuilder = this.searchSource(mainQuery, Optional.of(List.of(nestedAgg)), null)
|
||||
.fetchSource(false);
|
||||
|
||||
SearchRequest searchRequest = searchRequest(INDEX_NAME, sourceBuilder, false);
|
||||
|
||||
try {
|
||||
SearchResponse searchResponse = client.search(searchRequest, RequestOptions.DEFAULT);
|
||||
|
||||
ParsedNested pn = searchResponse.getAggregations().get("NESTED");
|
||||
Filter fa = pn.getAggregations().get("FILTER");
|
||||
long docCount = fa.getDocCount();
|
||||
TopHits th = fa.getAggregations().get("TOPHITS");
|
||||
|
||||
List<TaskRun> collect = Arrays.stream(th.getHits().getHits())
|
||||
.map(documentFields -> {
|
||||
try {
|
||||
return MAPPER.readValue(documentFields.getSourceAsString(), TaskRun.class);
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
})
|
||||
.collect(Collectors.toList());
|
||||
|
||||
return new ArrayListTotal<>(collect, docCount);
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Execution delete(Execution execution) {
|
||||
Execution deleted = execution.toDeleted();
|
||||
|
||||
executionQueue.emit(deleted);
|
||||
this.deleteRequest(INDEX_NAME, execution.getId());
|
||||
|
||||
eventPublisher.publishEvent(new CrudEvent<>(deleted, CrudEventType.DELETE));
|
||||
|
||||
return deleted;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Integer purge(Execution execution) {
|
||||
this.rawDeleteRequest(INDEX_NAME, execution.getId());
|
||||
|
||||
return 1;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ArrayListTotal<Execution> findByFlowId(String namespace, String id, Pageable pageable) {
|
||||
BoolQueryBuilder bool = QueryBuilders.boolQuery()
|
||||
.must(QueryBuilders.termQuery("namespace", namespace))
|
||||
.must(QueryBuilders.termQuery("flowId", id));
|
||||
|
||||
SearchSourceBuilder sourceBuilder = this.searchSource(bool, Optional.empty(), pageable);
|
||||
|
||||
return this.query(INDEX_NAME, sourceBuilder);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Execution save(Execution execution) {
|
||||
this.putRequest(INDEX_NAME, execution.getId(), execution);
|
||||
|
||||
return execution;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Integer maxTaskRunSetting() {
|
||||
String max = this.getSettings(INDEX_NAME, true)
|
||||
.getSetting(
|
||||
this.indicesConfigs.get(INDEX_NAME).getIndex(),
|
||||
IndexSettings.MAX_INNER_RESULT_WINDOW_SETTING.getKey()
|
||||
);
|
||||
|
||||
return Integer
|
||||
.valueOf(max == null ? IndexSettings.MAX_INNER_RESULT_WINDOW_SETTING.getDefault(Settings.EMPTY) : Integer.valueOf(max));
|
||||
}
|
||||
}
|
||||
@@ -1,386 +0,0 @@
|
||||
package io.kestra.repository.elasticsearch;
|
||||
|
||||
import com.fasterxml.jackson.core.JsonProcessingException;
|
||||
import com.fasterxml.jackson.databind.JsonNode;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import io.kestra.core.events.CrudEvent;
|
||||
import io.kestra.core.events.CrudEventType;
|
||||
import io.kestra.core.exceptions.DeserializationException;
|
||||
import io.kestra.core.models.SearchResult;
|
||||
import io.kestra.core.models.flows.Flow;
|
||||
import io.kestra.core.models.flows.FlowWithException;
|
||||
import io.kestra.core.models.flows.FlowWithSource;
|
||||
import io.kestra.core.models.triggers.Trigger;
|
||||
import io.kestra.core.models.validations.ManualConstraintViolation;
|
||||
import io.kestra.core.models.validations.ModelValidator;
|
||||
import io.kestra.core.queues.QueueFactoryInterface;
|
||||
import io.kestra.core.queues.QueueInterface;
|
||||
import io.kestra.core.repositories.ArrayListTotal;
|
||||
import io.kestra.core.repositories.FlowRepositoryInterface;
|
||||
import io.kestra.core.serializers.JacksonMapper;
|
||||
import io.kestra.core.services.FlowService;
|
||||
import io.kestra.core.utils.ExecutorsUtils;
|
||||
import io.kestra.core.utils.ListUtils;
|
||||
import io.micronaut.context.event.ApplicationEventPublisher;
|
||||
import io.micronaut.data.model.Pageable;
|
||||
import jakarta.inject.Inject;
|
||||
import jakarta.inject.Named;
|
||||
import jakarta.inject.Singleton;
|
||||
import org.opensearch.action.search.SearchRequest;
|
||||
import org.opensearch.action.search.SearchResponse;
|
||||
import org.opensearch.client.RequestOptions;
|
||||
import org.opensearch.client.RestHighLevelClient;
|
||||
import org.opensearch.common.text.Text;
|
||||
import org.opensearch.index.query.BoolQueryBuilder;
|
||||
import org.opensearch.index.query.MatchQueryBuilder;
|
||||
import org.opensearch.index.query.QueryBuilder;
|
||||
import org.opensearch.index.query.QueryBuilders;
|
||||
import org.opensearch.search.builder.SearchSourceBuilder;
|
||||
import org.opensearch.search.fetch.subphase.highlight.HighlightBuilder;
|
||||
import org.opensearch.search.sort.FieldSortBuilder;
|
||||
import org.opensearch.search.sort.SortOrder;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.*;
|
||||
import java.util.stream.Collectors;
|
||||
import javax.annotation.Nullable;
|
||||
import javax.validation.ConstraintViolationException;
|
||||
|
||||
@Singleton
|
||||
@ElasticSearchRepositoryEnabled
|
||||
public class ElasticSearchFlowRepository extends AbstractElasticSearchRepository<Flow> implements FlowRepositoryInterface {
|
||||
private static final String INDEX_NAME = "flows";
|
||||
protected static final String REVISIONS_NAME = "flows-revisions";
|
||||
protected static final ObjectMapper JSON_MAPPER = JacksonMapper.ofJson();
|
||||
|
||||
private final QueueInterface<Flow> flowQueue;
|
||||
private final QueueInterface<Trigger> triggerQueue;
|
||||
private final ApplicationEventPublisher<CrudEvent<Flow>> eventPublisher;
|
||||
@Inject
|
||||
private ModelValidator modelValidator;
|
||||
@Inject
|
||||
public ElasticSearchFlowRepository(
|
||||
RestHighLevelClient client,
|
||||
ElasticSearchIndicesService elasticSearchIndicesService,
|
||||
ExecutorsUtils executorsUtils,
|
||||
@Named(QueueFactoryInterface.FLOW_NAMED) QueueInterface<Flow> flowQueue,
|
||||
@Named(QueueFactoryInterface.TRIGGER_NAMED) QueueInterface<Trigger> triggerQueue,
|
||||
ApplicationEventPublisher<CrudEvent<Flow>> eventPublisher
|
||||
) {
|
||||
super(client, elasticSearchIndicesService, executorsUtils, Flow.class);
|
||||
|
||||
this.flowQueue = flowQueue;
|
||||
this.triggerQueue = triggerQueue;
|
||||
this.eventPublisher = eventPublisher;
|
||||
}
|
||||
|
||||
private static String flowId(Flow flow) {
|
||||
return String.join("_", Arrays.asList(
|
||||
flow.getNamespace(),
|
||||
flow.getId()
|
||||
));
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Flow deserialize(String source) {
|
||||
try {
|
||||
return super.deserialize(source);
|
||||
} catch (DeserializationException e) {
|
||||
try {
|
||||
JsonNode jsonNode = MAPPER.readTree(source);
|
||||
return FlowWithException.builder()
|
||||
.id(jsonNode.get("id").asText())
|
||||
.namespace(jsonNode.get("namespace").asText())
|
||||
.revision(jsonNode.get("revision").asInt())
|
||||
.source(JacksonMapper.ofJson().writeValueAsString(JacksonMapper.toMap(source)))
|
||||
.exception(e.getMessage())
|
||||
.tasks(List.of())
|
||||
.build();
|
||||
} catch (JsonProcessingException ex) {
|
||||
throw new DeserializationException(ex);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private SearchSourceBuilder searchById(String namespace, String id, Optional<Integer> revision) {
|
||||
BoolQueryBuilder bool = this.defaultFilter()
|
||||
.must(QueryBuilders.termQuery("namespace", namespace))
|
||||
.must(QueryBuilders.termQuery("id", id));
|
||||
|
||||
revision
|
||||
.ifPresent(v -> {
|
||||
this.removeDeleted(bool);
|
||||
bool.must(QueryBuilders.termQuery("revision", v));
|
||||
});
|
||||
|
||||
return new SearchSourceBuilder()
|
||||
.query(bool)
|
||||
.sort(new FieldSortBuilder("revision").order(SortOrder.DESC))
|
||||
.size(1);
|
||||
}
|
||||
|
||||
public Optional<Flow> findById(String namespace, String id, Optional<Integer> revision) {
|
||||
SearchSourceBuilder sourceBuilder = this.searchById(namespace, id, revision)
|
||||
.fetchSource("*", "sourceCode");
|
||||
|
||||
List<Flow> query = this.query(revision.isPresent() ? REVISIONS_NAME : INDEX_NAME, sourceBuilder);
|
||||
|
||||
return query.size() > 0 ? Optional.of(query.get(0)) : Optional.empty();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<FlowWithSource> findByIdWithSource(String namespace, String id, Optional<Integer> revision) {
|
||||
SearchSourceBuilder sourceBuilder = this.searchById(namespace, id, revision);
|
||||
SearchRequest searchRequest = searchRequest(revision.isPresent() ? REVISIONS_NAME : INDEX_NAME, sourceBuilder, false);
|
||||
|
||||
try {
|
||||
SearchResponse searchResponse = client.search(searchRequest, RequestOptions.DEFAULT);
|
||||
|
||||
if (searchResponse.getHits().getHits().length == 0) {
|
||||
return Optional.empty();
|
||||
}
|
||||
|
||||
return Optional.of(FlowWithSource.of(
|
||||
this.deserialize(searchResponse.getHits().getHits()[0].getSourceAsString()),
|
||||
(String) searchResponse.getHits().getHits()[0].getSourceAsMap().get("sourceCode")
|
||||
));
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
private void removeDeleted(BoolQueryBuilder bool) {
|
||||
QueryBuilder deleted = bool
|
||||
.must()
|
||||
.stream()
|
||||
.filter(queryBuilder -> queryBuilder instanceof MatchQueryBuilder && ((MatchQueryBuilder) queryBuilder).fieldName().equals("deleted"))
|
||||
.findFirst()
|
||||
.orElseThrow();
|
||||
|
||||
bool.must().remove(deleted);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<FlowWithSource> findRevisions(String namespace, String id) {
|
||||
BoolQueryBuilder defaultFilter = this.defaultFilter();
|
||||
|
||||
BoolQueryBuilder bool = defaultFilter
|
||||
.must(QueryBuilders.termQuery("namespace", namespace))
|
||||
.must(QueryBuilders.termQuery("id", id));
|
||||
|
||||
this.removeDeleted(defaultFilter);
|
||||
|
||||
SearchSourceBuilder sourceBuilder = new SearchSourceBuilder()
|
||||
.query(bool)
|
||||
.sort(new FieldSortBuilder("revision").order(SortOrder.ASC));
|
||||
|
||||
List<FlowWithSource> result = new ArrayList<>();
|
||||
|
||||
this.internalScroll(
|
||||
REVISIONS_NAME,
|
||||
sourceBuilder,
|
||||
documentFields -> result.add(
|
||||
FlowWithSource.of(
|
||||
this.deserialize(documentFields.getSourceAsString()),
|
||||
(String) documentFields.getSourceAsMap().get("sourceCode")
|
||||
)
|
||||
)
|
||||
);
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<Flow> findAll() {
|
||||
SearchSourceBuilder sourceBuilder = new SearchSourceBuilder()
|
||||
.fetchSource("*", "sourceCode")
|
||||
.query(this.defaultFilter());
|
||||
|
||||
return this.scroll(INDEX_NAME, sourceBuilder);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<Flow> findByNamespace(String namespace) {
|
||||
BoolQueryBuilder bool = this.defaultFilter()
|
||||
.must(QueryBuilders.termQuery("namespace", namespace));
|
||||
|
||||
SearchSourceBuilder sourceBuilder = new SearchSourceBuilder()
|
||||
.fetchSource("*", "sourceCode")
|
||||
.query(bool);
|
||||
|
||||
return this.scroll(INDEX_NAME, sourceBuilder);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ArrayListTotal<Flow> find(
|
||||
Pageable pageable,
|
||||
@Nullable String query,
|
||||
@Nullable String namespace,
|
||||
@Nullable Map<String, String> labels
|
||||
) {
|
||||
BoolQueryBuilder bool = this.defaultFilter();
|
||||
|
||||
if (query != null) {
|
||||
bool.must(queryString(query).field("*.fulltext"));
|
||||
}
|
||||
|
||||
if (namespace != null) {
|
||||
bool.must(QueryBuilders.prefixQuery("namespace", namespace));
|
||||
}
|
||||
|
||||
if (labels != null) {
|
||||
labels.forEach((key, value) -> {
|
||||
bool.must(QueryBuilders.termQuery("labelsMap.key", key));
|
||||
|
||||
if (value != null) {
|
||||
bool.must(QueryBuilders.termQuery("labelsMap.value", value));
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
SearchSourceBuilder sourceBuilder = this.searchSource(bool, Optional.empty(), pageable);
|
||||
sourceBuilder.fetchSource("*", "sourceCode");
|
||||
|
||||
return this.query(INDEX_NAME, sourceBuilder);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ArrayListTotal<SearchResult<Flow>> findSourceCode(Pageable pageable, @Nullable String query, @Nullable String namespace) {
|
||||
BoolQueryBuilder bool = this.defaultFilter()
|
||||
.must(queryString(query).field("sourceCode"));
|
||||
|
||||
if (namespace != null) {
|
||||
bool.must(QueryBuilders.prefixQuery("namespace", namespace));
|
||||
}
|
||||
|
||||
SearchSourceBuilder sourceBuilder = this.searchSource(bool, Optional.empty(), pageable);
|
||||
sourceBuilder.fetchSource("*", "sourceCode");
|
||||
sourceBuilder.highlighter(new HighlightBuilder()
|
||||
.preTags("[mark]")
|
||||
.postTags("[/mark]")
|
||||
.field("sourceCode")
|
||||
);
|
||||
|
||||
SearchRequest searchRequest = searchRequest(INDEX_NAME, sourceBuilder, false);
|
||||
|
||||
try {
|
||||
SearchResponse searchResponse = client.search(searchRequest, RequestOptions.DEFAULT);
|
||||
return new ArrayListTotal<>(
|
||||
Arrays.stream(searchResponse.getHits().getHits())
|
||||
.map(documentFields -> {
|
||||
try {
|
||||
return new SearchResult<>(
|
||||
MAPPER.readValue(documentFields.getSourceAsString(), this.cls),
|
||||
documentFields.getHighlightFields().get("sourceCode") != null ?
|
||||
Arrays.stream(documentFields.getHighlightFields().get("sourceCode").getFragments())
|
||||
.map(Text::string)
|
||||
.collect(Collectors.toList()) :
|
||||
Collections.emptyList()
|
||||
);
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
})
|
||||
.collect(Collectors.toList()),
|
||||
searchResponse.getHits().getTotalHits().value
|
||||
);
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
public FlowWithSource create(Flow flow, String flowSource, Flow flowWithDefaults) throws ConstraintViolationException {
|
||||
if (this.findById(flow.getNamespace(), flow.getId()).isPresent()) {
|
||||
throw new ConstraintViolationException(Collections.singleton(ManualConstraintViolation.of(
|
||||
"Flow namespace:'" + flow.getNamespace() + "', id:'" + flow.getId() + "' already exists",
|
||||
flow,
|
||||
Flow.class,
|
||||
"flow.id",
|
||||
flow.getId()
|
||||
)));
|
||||
}
|
||||
|
||||
// Check flow with defaults injected
|
||||
modelValidator.validate(flowWithDefaults);
|
||||
|
||||
return this.save(flow, CrudEventType.CREATE, flowSource);
|
||||
}
|
||||
|
||||
public FlowWithSource update(Flow flow, Flow previous, String flowSource, Flow flowWithDefaults) throws ConstraintViolationException {
|
||||
// Check flow with defaults injected
|
||||
modelValidator.validate(flowWithDefaults);
|
||||
|
||||
// control if update is valid
|
||||
Optional<ConstraintViolationException> checkUpdate = previous.validateUpdate(flowWithDefaults);
|
||||
if(checkUpdate.isPresent()){
|
||||
throw checkUpdate.get();
|
||||
}
|
||||
|
||||
FlowService
|
||||
.findRemovedTrigger(flow, previous)
|
||||
.forEach(abstractTrigger -> triggerQueue.delete(Trigger.of(flow, abstractTrigger)));
|
||||
|
||||
return this.save(flow, CrudEventType.UPDATE, flowSource);
|
||||
}
|
||||
|
||||
public FlowWithSource save(Flow flow, CrudEventType crudEventType, String flowSource) throws ConstraintViolationException {
|
||||
Optional<FlowWithSource> exists = this.findByIdWithSource(flow.getNamespace(), flow.getId());
|
||||
if (exists.isPresent() && exists.get().isUpdatable(flow, flowSource)) {
|
||||
return exists.get();
|
||||
}
|
||||
|
||||
List<FlowWithSource> revisions = this.findRevisions(flow.getNamespace(), flow.getId());
|
||||
|
||||
if (revisions.size() > 0) {
|
||||
flow = flow.withRevision(revisions.get(revisions.size() - 1).getRevision() + 1);
|
||||
} else {
|
||||
flow = flow.withRevision(1);
|
||||
}
|
||||
|
||||
String json;
|
||||
try {
|
||||
Map<String, Object> flowMap = JacksonMapper.toMap(flow);
|
||||
flowMap.put("sourceCode", flowSource);
|
||||
if (flow.getLabels() != null) {
|
||||
flowMap.put("labelsMap", flow.getLabels()
|
||||
.entrySet()
|
||||
.stream()
|
||||
.map(e -> Map.of("key", e.getKey(), "value", e.getValue()))
|
||||
.collect(Collectors.toList())
|
||||
);
|
||||
}
|
||||
json = JSON_MAPPER.writeValueAsString(flowMap);
|
||||
} catch (JsonProcessingException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
|
||||
this.putRequest(INDEX_NAME, flowId(flow), json);
|
||||
this.putRequest(REVISIONS_NAME, flow.uid(), json);
|
||||
|
||||
flowQueue.emit(flow);
|
||||
|
||||
eventPublisher.publishEvent(new CrudEvent<>(flow, crudEventType));
|
||||
|
||||
return FlowWithSource.of(flow, flowSource);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Flow delete(Flow flow) {
|
||||
Flow deleted = flow.toDeleted();
|
||||
|
||||
flowQueue.emit(deleted);
|
||||
this.deleteRequest(INDEX_NAME, flowId(deleted));
|
||||
this.putRequest(REVISIONS_NAME, deleted.uid(), deleted);
|
||||
|
||||
ListUtils.emptyOnNull(flow.getTriggers())
|
||||
.forEach(abstractTrigger -> triggerQueue.delete(Trigger.of(flow, abstractTrigger)));
|
||||
|
||||
eventPublisher.publishEvent(new CrudEvent<>(deleted, CrudEventType.DELETE));
|
||||
|
||||
return deleted;
|
||||
}
|
||||
|
||||
public List<String> findDistinctNamespace() {
|
||||
return findDistinct(INDEX_NAME, "namespace");
|
||||
}
|
||||
}
|
||||
@@ -1,78 +0,0 @@
|
||||
package io.kestra.repository.elasticsearch;
|
||||
|
||||
import org.opensearch.client.RequestOptions;
|
||||
import org.opensearch.client.RestHighLevelClient;
|
||||
import org.opensearch.client.indices.CreateIndexRequest;
|
||||
import org.opensearch.client.indices.GetIndexRequest;
|
||||
import org.opensearch.client.indices.PutMappingRequest;
|
||||
import org.opensearch.common.xcontent.XContentType;
|
||||
import io.kestra.repository.elasticsearch.configs.IndicesConfig;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.AbstractMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
import io.micronaut.core.annotation.Nullable;
|
||||
import jakarta.inject.Inject;
|
||||
import jakarta.inject.Singleton;
|
||||
|
||||
@Singleton
|
||||
@ElasticSearchRepositoryEnabled
|
||||
public class ElasticSearchIndicesService {
|
||||
private final Map<String, IndicesConfig> indicesConfigs;
|
||||
|
||||
private final RestHighLevelClient client;
|
||||
|
||||
@Inject
|
||||
public ElasticSearchIndicesService(
|
||||
RestHighLevelClient client,
|
||||
List<IndicesConfig> indicesConfigs
|
||||
) {
|
||||
this.client = client;
|
||||
|
||||
this.indicesConfigs = indicesConfigs
|
||||
.stream()
|
||||
.map(r -> new AbstractMap.SimpleEntry<>(r.getName(), r))
|
||||
.collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
|
||||
}
|
||||
|
||||
public <T> Map<String, IndicesConfig> findConfig(Class<T> cls) {
|
||||
return indicesConfigs
|
||||
.entrySet()
|
||||
.stream()
|
||||
.filter(r -> r.getValue().getCls() == cls)
|
||||
.collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
|
||||
}
|
||||
|
||||
public void createIndice(@Nullable Map<String, IndicesConfig> indicesConfigs) {
|
||||
try {
|
||||
for (Map.Entry<String, IndicesConfig> index : (indicesConfigs == null ? this.indicesConfigs : indicesConfigs).entrySet()) {
|
||||
GetIndexRequest exists = new GetIndexRequest(index.getValue().getIndex());
|
||||
if (!client.indices().exists(exists, RequestOptions.DEFAULT)) {
|
||||
CreateIndexRequest request = new CreateIndexRequest(index.getValue().getIndex());
|
||||
request.settings(index.getValue().getSettingsContent(), XContentType.JSON);
|
||||
|
||||
client.indices().create(request, RequestOptions.DEFAULT);
|
||||
}
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
public void updateMapping(@Nullable Map<String, IndicesConfig> indicesConfigs) {
|
||||
for (Map.Entry<String, IndicesConfig> index : (indicesConfigs == null ? this.indicesConfigs : indicesConfigs).entrySet()) {
|
||||
if (index.getValue().getMappingContent() != null) {
|
||||
try {
|
||||
PutMappingRequest request = new PutMappingRequest(index.getValue().getIndex());
|
||||
request.source(index.getValue().getMappingContent(), XContentType.JSON);
|
||||
|
||||
client.indices().putMapping(request, RequestOptions.DEFAULT);
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -1,162 +0,0 @@
|
||||
package io.kestra.repository.elasticsearch;
|
||||
|
||||
import io.kestra.core.models.executions.Execution;
|
||||
import io.kestra.core.models.executions.LogEntry;
|
||||
import io.kestra.core.models.validations.ModelValidator;
|
||||
import io.kestra.core.repositories.ArrayListTotal;
|
||||
import io.kestra.core.repositories.LogRepositoryInterface;
|
||||
import io.kestra.core.utils.ExecutorsUtils;
|
||||
import io.kestra.core.utils.IdUtils;
|
||||
import io.micronaut.data.model.Pageable;
|
||||
import jakarta.inject.Inject;
|
||||
import jakarta.inject.Singleton;
|
||||
import org.opensearch.client.RequestOptions;
|
||||
import org.opensearch.client.RestHighLevelClient;
|
||||
import org.opensearch.index.query.BoolQueryBuilder;
|
||||
import org.opensearch.index.query.QueryBuilders;
|
||||
import org.opensearch.index.query.TermsQueryBuilder;
|
||||
import org.opensearch.index.reindex.BulkByScrollResponse;
|
||||
import org.opensearch.index.reindex.DeleteByQueryRequest;
|
||||
import org.opensearch.search.builder.SearchSourceBuilder;
|
||||
import org.opensearch.search.sort.SortOrder;
|
||||
import org.slf4j.event.Level;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.time.ZonedDateTime;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
@Singleton
|
||||
@ElasticSearchRepositoryEnabled
|
||||
public class ElasticSearchLogRepository extends AbstractElasticSearchRepository<LogEntry> implements LogRepositoryInterface {
|
||||
private static final String INDEX_NAME = "logs";
|
||||
|
||||
@Inject
|
||||
public ElasticSearchLogRepository(
|
||||
RestHighLevelClient client,
|
||||
ElasticSearchIndicesService elasticSearchIndicesService,
|
||||
ExecutorsUtils executorsUtils
|
||||
) {
|
||||
super(client, elasticSearchIndicesService, executorsUtils, LogEntry.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ArrayListTotal<LogEntry> find(
|
||||
Pageable pageable,
|
||||
@Nullable String query,
|
||||
@Nullable String namespace,
|
||||
@Nullable String flowId,
|
||||
@Nullable Level minLevel,
|
||||
@Nullable ZonedDateTime startDate,
|
||||
@Nullable ZonedDateTime endDate
|
||||
) {
|
||||
BoolQueryBuilder bool = this.defaultFilter();
|
||||
|
||||
if (query != null) {
|
||||
bool.must(QueryBuilders.queryStringQuery(query).field("*.fulltext"));
|
||||
}
|
||||
|
||||
if (namespace != null) {
|
||||
bool.must(QueryBuilders.prefixQuery("namespace", namespace));
|
||||
}
|
||||
|
||||
if (flowId != null) {
|
||||
bool.must(QueryBuilders.termQuery("flowId", flowId));
|
||||
}
|
||||
|
||||
if (minLevel != null) {
|
||||
bool.must(minLevel(minLevel));
|
||||
}
|
||||
|
||||
if (startDate != null) {
|
||||
bool.must(QueryBuilders.rangeQuery("timestamp").gte(startDate));
|
||||
}
|
||||
|
||||
if (endDate != null) {
|
||||
bool.must(QueryBuilders.rangeQuery("timestamp").lte(endDate));
|
||||
}
|
||||
|
||||
SearchSourceBuilder sourceBuilder = this.searchSource(bool, Optional.empty(), pageable)
|
||||
.sort("timestamp", SortOrder.DESC);
|
||||
|
||||
return this.query(INDEX_NAME, sourceBuilder);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<LogEntry> findByExecutionId(String id, Level minLevel) {
|
||||
BoolQueryBuilder bool = this.defaultFilter()
|
||||
.must(QueryBuilders.termQuery("executionId", id));
|
||||
|
||||
if (minLevel != null) {
|
||||
bool.must(minLevel(minLevel));
|
||||
}
|
||||
|
||||
SearchSourceBuilder sourceBuilder = new SearchSourceBuilder()
|
||||
.query(bool)
|
||||
.sort("timestamp", SortOrder.ASC);
|
||||
|
||||
return this.scroll(INDEX_NAME, sourceBuilder);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<LogEntry> findByExecutionIdAndTaskId(String executionId, String taskId, Level minLevel) {
|
||||
BoolQueryBuilder bool = this.defaultFilter()
|
||||
.must(QueryBuilders.termQuery("executionId", executionId))
|
||||
.must(QueryBuilders.termQuery("taskId", taskId));
|
||||
|
||||
if (minLevel != null) {
|
||||
bool.must(minLevel(minLevel));
|
||||
}
|
||||
|
||||
SearchSourceBuilder sourceBuilder = new SearchSourceBuilder()
|
||||
.query(bool)
|
||||
.sort("timestamp", SortOrder.ASC);
|
||||
|
||||
return this.scroll(INDEX_NAME, sourceBuilder);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<LogEntry> findByExecutionIdAndTaskRunId(String executionId, String taskRunId, Level minLevel) {
|
||||
BoolQueryBuilder bool = this.defaultFilter()
|
||||
.must(QueryBuilders.termQuery("executionId", executionId))
|
||||
.must(QueryBuilders.termQuery("taskRunId", taskRunId));
|
||||
|
||||
if (minLevel != null) {
|
||||
bool.must(minLevel(minLevel));
|
||||
}
|
||||
|
||||
SearchSourceBuilder sourceBuilder = new SearchSourceBuilder()
|
||||
.query(bool)
|
||||
.sort("timestamp", SortOrder.ASC);
|
||||
|
||||
return this.scroll(INDEX_NAME, sourceBuilder);
|
||||
}
|
||||
|
||||
@Override
|
||||
public LogEntry save(LogEntry log) {
|
||||
this.putRequest(INDEX_NAME, IdUtils.create(), log);
|
||||
|
||||
return log;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Integer purge(Execution execution) {
|
||||
DeleteByQueryRequest request = new DeleteByQueryRequest(this.indexName(INDEX_NAME));
|
||||
request.setConflicts("proceed");
|
||||
request.setQuery(new TermsQueryBuilder("executionId", execution.getId()));
|
||||
request.setRefresh(true);
|
||||
|
||||
try {
|
||||
BulkByScrollResponse bulkByScrollResponse = this.client.deleteByQuery(request, RequestOptions.DEFAULT);
|
||||
|
||||
return Long.valueOf(bulkByScrollResponse.getDeleted()).intValue();
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
private TermsQueryBuilder minLevel(Level minLevel) {
|
||||
return QueryBuilders.termsQuery("level", LogEntry.findLevelsByMin(minLevel));
|
||||
}
|
||||
}
|
||||
@@ -1,12 +0,0 @@
|
||||
package io.kestra.repository.elasticsearch;
|
||||
|
||||
import io.micronaut.context.annotation.Requires;
|
||||
|
||||
import java.lang.annotation.*;
|
||||
|
||||
@Documented
|
||||
@Retention(RetentionPolicy.RUNTIME)
|
||||
@Target({ElementType.PACKAGE, ElementType.TYPE})
|
||||
@Requires(property = "kestra.repository.type", value = "elasticsearch")
|
||||
public @interface ElasticSearchRepositoryEnabled {
|
||||
}
|
||||
@@ -1,165 +0,0 @@
|
||||
package io.kestra.repository.elasticsearch;
|
||||
|
||||
import com.fasterxml.jackson.core.JsonProcessingException;
|
||||
import com.fasterxml.jackson.databind.JsonNode;
|
||||
import io.kestra.core.events.CrudEvent;
|
||||
import io.kestra.core.events.CrudEventType;
|
||||
import io.kestra.core.exceptions.DeserializationException;
|
||||
import io.kestra.core.models.templates.Template;
|
||||
import io.kestra.core.models.templates.TemplateSource;
|
||||
import io.kestra.core.queues.QueueFactoryInterface;
|
||||
import io.kestra.core.queues.QueueInterface;
|
||||
import io.kestra.core.repositories.ArrayListTotal;
|
||||
import io.kestra.core.repositories.TemplateRepositoryInterface;
|
||||
import io.kestra.core.serializers.JacksonMapper;
|
||||
import io.kestra.core.utils.ExecutorsUtils;
|
||||
import io.micronaut.context.event.ApplicationEventPublisher;
|
||||
import io.micronaut.data.model.Pageable;
|
||||
import jakarta.inject.Inject;
|
||||
import jakarta.inject.Named;
|
||||
import jakarta.inject.Singleton;
|
||||
import org.opensearch.client.RestHighLevelClient;
|
||||
import org.opensearch.index.query.BoolQueryBuilder;
|
||||
import org.opensearch.index.query.QueryBuilders;
|
||||
import org.opensearch.search.builder.SearchSourceBuilder;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import javax.annotation.Nullable;
|
||||
import javax.validation.ConstraintViolationException;
|
||||
|
||||
@Singleton
|
||||
@ElasticSearchRepositoryEnabled
|
||||
public class ElasticSearchTemplateRepository extends AbstractElasticSearchRepository<Template> implements TemplateRepositoryInterface {
|
||||
private static final String INDEX_NAME = "templates";
|
||||
private final QueueInterface<Template> templateQueue;
|
||||
private final ApplicationEventPublisher<CrudEvent<Template>> eventPublisher;
|
||||
|
||||
@Inject
|
||||
public ElasticSearchTemplateRepository(
|
||||
RestHighLevelClient client,
|
||||
ElasticSearchIndicesService elasticSearchIndicesService,
|
||||
ExecutorsUtils executorsUtils,
|
||||
@Named(QueueFactoryInterface.TEMPLATE_NAMED) QueueInterface<Template> templateQueue,
|
||||
ApplicationEventPublisher<CrudEvent<Template>> eventPublisher
|
||||
) {
|
||||
super(client, elasticSearchIndicesService, executorsUtils, Template.class);
|
||||
|
||||
this.templateQueue = templateQueue;
|
||||
this.eventPublisher = eventPublisher;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Template deserialize(String source) {
|
||||
try {
|
||||
return super.deserialize(source);
|
||||
} catch (DeserializationException e) {
|
||||
try {
|
||||
JsonNode jsonNode = MAPPER.readTree(source);
|
||||
return TemplateSource.builder()
|
||||
.id(jsonNode.get("id").asText())
|
||||
.namespace(jsonNode.get("namespace").asText())
|
||||
.source(JacksonMapper.ofYaml().writeValueAsString(JacksonMapper.toMap(source)))
|
||||
.exception(e.getMessage())
|
||||
.tasks(List.of())
|
||||
.build();
|
||||
} catch (JsonProcessingException ex) {
|
||||
throw new DeserializationException(ex);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<Template> findById(String namespace, String id) {
|
||||
BoolQueryBuilder bool = this.defaultFilter()
|
||||
.must(QueryBuilders.termQuery("namespace", namespace))
|
||||
.must(QueryBuilders.termQuery("id", id));
|
||||
|
||||
SearchSourceBuilder sourceBuilder = new SearchSourceBuilder()
|
||||
.query(bool)
|
||||
.size(1);
|
||||
|
||||
List<Template> query = this.query(INDEX_NAME, sourceBuilder);
|
||||
|
||||
return query.size() > 0 ? Optional.of(query.get(0)) : Optional.empty();
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<Template> findAll() {
|
||||
SearchSourceBuilder sourceBuilder = new SearchSourceBuilder()
|
||||
.query(this.defaultFilter());
|
||||
|
||||
return this.scroll(INDEX_NAME, sourceBuilder);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ArrayListTotal<Template> find(
|
||||
Pageable pageable,
|
||||
@Nullable String query,
|
||||
@Nullable String namespace
|
||||
) {
|
||||
BoolQueryBuilder bool = this.defaultFilter();
|
||||
|
||||
if (query != null) {
|
||||
bool.must(queryString(query).field("*.fulltext"));
|
||||
}
|
||||
|
||||
if (namespace != null) {
|
||||
bool.must(QueryBuilders.prefixQuery("namespace", namespace));
|
||||
}
|
||||
|
||||
|
||||
SearchSourceBuilder sourceBuilder = this.searchSource(bool, Optional.empty(), pageable);
|
||||
|
||||
return this.query(INDEX_NAME, sourceBuilder);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<Template> findByNamespace(String namespace) {
|
||||
BoolQueryBuilder bool = this.defaultFilter()
|
||||
.must(QueryBuilders.termQuery("namespace", namespace));
|
||||
|
||||
SearchSourceBuilder sourceBuilder = new SearchSourceBuilder()
|
||||
.query(bool);
|
||||
|
||||
return this.scroll(INDEX_NAME, sourceBuilder);
|
||||
}
|
||||
|
||||
public Template create(Template template) throws ConstraintViolationException {
|
||||
return this.save(template, CrudEventType.CREATE);
|
||||
}
|
||||
|
||||
public Template update(Template template, Template previous) throws ConstraintViolationException {
|
||||
this
|
||||
.findById(previous.getNamespace(), previous.getId())
|
||||
.map(current -> current.validateUpdate(template))
|
||||
.filter(Optional::isPresent)
|
||||
.map(Optional::get)
|
||||
.ifPresent(s -> {
|
||||
throw s;
|
||||
});
|
||||
|
||||
return this.save(template, CrudEventType.UPDATE);
|
||||
}
|
||||
|
||||
public Template save(Template template, CrudEventType crudEventType) {
|
||||
this.putRequest(INDEX_NAME, template.uid(), template);
|
||||
|
||||
templateQueue.emit(template);
|
||||
|
||||
eventPublisher.publishEvent(new CrudEvent<>(template, crudEventType));
|
||||
|
||||
return template;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void delete(Template template) {
|
||||
this.deleteRequest(INDEX_NAME, template.uid());
|
||||
|
||||
eventPublisher.publishEvent(new CrudEvent<>(template, CrudEventType.DELETE));
|
||||
}
|
||||
|
||||
public List<String> findDistinctNamespace() {
|
||||
return findDistinct(INDEX_NAME, "namespace");
|
||||
}
|
||||
}
|
||||
@@ -1,107 +0,0 @@
|
||||
package io.kestra.repository.elasticsearch;
|
||||
|
||||
import com.google.common.base.Charsets;
|
||||
import io.kestra.core.serializers.JacksonMapper;
|
||||
import io.micronaut.context.annotation.Requires;
|
||||
import io.micronaut.health.HealthStatus;
|
||||
import io.micronaut.management.endpoint.health.HealthEndpoint;
|
||||
import io.micronaut.management.health.indicator.HealthIndicator;
|
||||
import io.micronaut.management.health.indicator.HealthResult;
|
||||
import jakarta.inject.Singleton;
|
||||
import lombok.extern.slf4j.Slf4j;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.http.client.methods.HttpGet;
|
||||
import org.opensearch.client.Request;
|
||||
import org.opensearch.client.Response;
|
||||
import org.opensearch.client.ResponseListener;
|
||||
import org.opensearch.client.RestHighLevelClient;
|
||||
import org.opensearch.cluster.health.ClusterHealthStatus;
|
||||
import org.reactivestreams.Publisher;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
|
||||
import static io.micronaut.health.HealthStatus.DOWN;
|
||||
import static io.micronaut.health.HealthStatus.UP;
|
||||
import static org.opensearch.cluster.health.ClusterHealthStatus.GREEN;
|
||||
import static org.opensearch.cluster.health.ClusterHealthStatus.YELLOW;
|
||||
|
||||
/**
|
||||
* A {@link HealthIndicator} for Elasticsearch High Level REST client.
|
||||
* Mostly a copy from the micronaut one, since it don't work with yellow cluster for example
|
||||
*/
|
||||
@Requires(beans = HealthEndpoint.class)
|
||||
@ElasticSearchRepositoryEnabled
|
||||
@Singleton
|
||||
@Slf4j
|
||||
public class ElasticsearchHealthIndicator implements HealthIndicator {
|
||||
public static final String NAME = "elasticsearch";
|
||||
private final RestHighLevelClient esClient;
|
||||
private final Request request;
|
||||
/**
|
||||
* Constructor.
|
||||
*
|
||||
* @param esClient The Elasticsearch high level REST client.
|
||||
*/
|
||||
public ElasticsearchHealthIndicator(RestHighLevelClient esClient) {
|
||||
this.esClient = esClient;
|
||||
|
||||
request = new Request(HttpGet.METHOD_NAME, "/_cluster/health");
|
||||
request.addParameter("master_timeout", "30s");
|
||||
request.addParameter("level", "cluster");
|
||||
request.addParameter("timeout", "30s");
|
||||
}
|
||||
|
||||
@Override
|
||||
public Publisher<HealthResult> getResult() {
|
||||
return (subscriber -> {
|
||||
try {
|
||||
esClient.getLowLevelClient()
|
||||
.performRequestAsync(
|
||||
request,
|
||||
new ResponseListener() {
|
||||
private final HealthResult.Builder resultBuilder = HealthResult.builder(NAME);
|
||||
|
||||
@Override
|
||||
public void onSuccess(Response response) {
|
||||
HealthResult result;
|
||||
|
||||
try {
|
||||
Map<String, Object> map = JacksonMapper.toMap(
|
||||
IOUtils.toString(response.getEntity().getContent(), Charsets.UTF_8)
|
||||
);
|
||||
|
||||
result = resultBuilder
|
||||
.status(healthResultStatus((String) map.get("status")))
|
||||
.details(map)
|
||||
.build();
|
||||
} catch (IOException e) {
|
||||
result = resultBuilder.status(DOWN).exception(e).build();
|
||||
}
|
||||
|
||||
subscriber.onNext(result);
|
||||
subscriber.onComplete();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFailure(Exception e) {
|
||||
subscriber.onNext(resultBuilder.status(DOWN).exception(e).build());
|
||||
subscriber.onComplete();
|
||||
}
|
||||
}
|
||||
);
|
||||
} catch (Exception e) {
|
||||
HealthResult.Builder resultBuilder = HealthResult.builder(NAME);
|
||||
subscriber.onNext(resultBuilder.status(DOWN).exception(e).build());
|
||||
subscriber.onComplete();
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
|
||||
private HealthStatus healthResultStatus(String status) {
|
||||
ClusterHealthStatus clusterHealthStatus = ClusterHealthStatus.fromString(status);
|
||||
|
||||
return clusterHealthStatus == GREEN || clusterHealthStatus == YELLOW ? UP : DOWN;
|
||||
}
|
||||
}
|
||||
@@ -1,77 +0,0 @@
|
||||
package io.kestra.repository.elasticsearch;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import io.kestra.core.events.CrudEvent;
|
||||
import io.kestra.core.events.CrudEventType;
|
||||
import io.kestra.core.models.Setting;
|
||||
import io.kestra.core.repositories.SettingRepositoryInterface;
|
||||
import io.kestra.core.utils.ExecutorsUtils;
|
||||
import io.micronaut.context.event.ApplicationEventPublisher;
|
||||
import jakarta.inject.Inject;
|
||||
import jakarta.inject.Singleton;
|
||||
import org.opensearch.client.RestHighLevelClient;
|
||||
import org.opensearch.index.query.QueryBuilders;
|
||||
import org.opensearch.search.builder.SearchSourceBuilder;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
|
||||
@Singleton
|
||||
@ElasticSearchRepositoryEnabled
|
||||
public class ElasticsearchSettingRepository extends AbstractElasticSearchRepository<Setting> implements SettingRepositoryInterface {
|
||||
private static final String INDEX_NAME = "settings";
|
||||
|
||||
private final ApplicationEventPublisher<CrudEvent<Setting>> eventPublisher;
|
||||
|
||||
@Inject
|
||||
public ElasticsearchSettingRepository(
|
||||
RestHighLevelClient client,
|
||||
ElasticSearchIndicesService elasticSearchIndicesService,
|
||||
ExecutorsUtils executorsUtils,
|
||||
ApplicationEventPublisher<CrudEvent<Setting>> eventPublisher
|
||||
) {
|
||||
super(client, elasticSearchIndicesService, executorsUtils, Setting.class);
|
||||
|
||||
this.eventPublisher = eventPublisher;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<Setting> findByKey(String key) {
|
||||
SearchSourceBuilder sourceBuilder = new SearchSourceBuilder()
|
||||
.query(QueryBuilders.termQuery("key", key))
|
||||
.size(1);
|
||||
|
||||
List<Setting> query = this.query(INDEX_NAME, sourceBuilder);
|
||||
|
||||
return query.size() > 0 ? Optional.of(query.get(0)) : Optional.empty();
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<Setting> findAll() {
|
||||
SearchSourceBuilder sourceBuilder = new SearchSourceBuilder()
|
||||
.query(QueryBuilders.matchAllQuery());
|
||||
|
||||
return this.scroll(INDEX_NAME, sourceBuilder);
|
||||
}
|
||||
|
||||
@Override
|
||||
@VisibleForTesting
|
||||
public Setting save(Setting setting) {
|
||||
this.putRequest(INDEX_NAME, setting.getKey(), setting);
|
||||
|
||||
eventPublisher.publishEvent(new CrudEvent<>(setting, CrudEventType.UPDATE));
|
||||
|
||||
return setting;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public Setting delete(Setting setting) {
|
||||
this.rawDeleteRequest(INDEX_NAME, setting.getKey());
|
||||
|
||||
eventPublisher.publishEvent(new CrudEvent<>(setting, CrudEventType.DELETE));
|
||||
|
||||
return setting;
|
||||
}
|
||||
|
||||
}
|
||||
@@ -1,63 +0,0 @@
|
||||
package io.kestra.repository.elasticsearch;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import io.kestra.core.models.executions.Execution;
|
||||
import io.kestra.core.models.validations.ModelValidator;
|
||||
import org.opensearch.client.RestHighLevelClient;
|
||||
import org.opensearch.index.query.QueryBuilders;
|
||||
import org.opensearch.search.builder.SearchSourceBuilder;
|
||||
import io.kestra.core.models.triggers.Trigger;
|
||||
import io.kestra.core.models.triggers.TriggerContext;
|
||||
import io.kestra.core.repositories.TriggerRepositoryInterface;
|
||||
import io.kestra.core.utils.ExecutorsUtils;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import jakarta.inject.Inject;
|
||||
import jakarta.inject.Singleton;
|
||||
|
||||
@Singleton
|
||||
@ElasticSearchRepositoryEnabled
|
||||
public class ElasticsearchTriggerRepository extends AbstractElasticSearchRepository<Trigger> implements TriggerRepositoryInterface {
|
||||
private static final String INDEX_NAME = "triggers";
|
||||
|
||||
@Inject
|
||||
public ElasticsearchTriggerRepository(
|
||||
RestHighLevelClient client,
|
||||
ElasticSearchIndicesService elasticSearchIndicesService,
|
||||
ExecutorsUtils executorsUtils
|
||||
) {
|
||||
super(client, elasticSearchIndicesService, executorsUtils, Trigger.class);
|
||||
}
|
||||
|
||||
public Optional<Trigger> findLast(TriggerContext trigger) {
|
||||
return this.rawGetRequest(INDEX_NAME, trigger.uid());
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<Trigger> findByExecution(Execution execution) {
|
||||
SearchSourceBuilder sourceBuilder = new SearchSourceBuilder()
|
||||
.query(QueryBuilders.termQuery("executionId", execution.getId()))
|
||||
.size(1);
|
||||
|
||||
List<Trigger> query = this.query(INDEX_NAME, sourceBuilder);
|
||||
|
||||
return query.size() > 0 ? Optional.of(query.get(0)) : Optional.empty();
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<Trigger> findAll() {
|
||||
SearchSourceBuilder sourceBuilder = new SearchSourceBuilder()
|
||||
.query(QueryBuilders.matchAllQuery());
|
||||
|
||||
return this.scroll(INDEX_NAME, sourceBuilder);
|
||||
}
|
||||
|
||||
@Override
|
||||
@VisibleForTesting
|
||||
public Trigger save(Trigger trigger) {
|
||||
this.putRequest(INDEX_NAME, trigger.uid(), trigger);
|
||||
|
||||
return trigger;
|
||||
}
|
||||
}
|
||||
@@ -1,94 +0,0 @@
|
||||
package io.kestra.repository.elasticsearch.configs;
|
||||
|
||||
import io.micronaut.context.annotation.ConfigurationBuilder;
|
||||
import io.micronaut.context.annotation.ConfigurationProperties;
|
||||
import lombok.Getter;
|
||||
import lombok.SneakyThrows;
|
||||
import org.apache.http.Header;
|
||||
import org.apache.http.HttpHost;
|
||||
import org.apache.http.auth.AuthScope;
|
||||
import org.apache.http.auth.UsernamePasswordCredentials;
|
||||
import org.apache.http.client.CredentialsProvider;
|
||||
import org.apache.http.client.config.RequestConfig;
|
||||
import org.apache.http.conn.ssl.NoopHostnameVerifier;
|
||||
import org.apache.http.conn.ssl.SSLConnectionSocketFactory;
|
||||
import org.apache.http.conn.ssl.SSLSocketFactory;
|
||||
import org.apache.http.conn.ssl.TrustStrategy;
|
||||
import org.apache.http.impl.client.BasicCredentialsProvider;
|
||||
import org.apache.http.impl.nio.client.HttpAsyncClientBuilder;
|
||||
import org.apache.http.message.BasicHeader;
|
||||
import org.apache.http.ssl.SSLContextBuilder;
|
||||
import org.apache.http.ssl.SSLContexts;
|
||||
|
||||
import java.net.URI;
|
||||
import java.security.cert.CertificateException;
|
||||
import java.security.cert.X509Certificate;
|
||||
import java.util.Arrays;
|
||||
import javax.net.ssl.SSLContext;
|
||||
|
||||
@ConfigurationProperties("kestra.elasticsearch.client")
|
||||
@Getter
|
||||
public class ElasticsearchConfig {
|
||||
String[] httpHosts;
|
||||
String[] defaultHeaders;
|
||||
String pathPrefix;
|
||||
Boolean strictDeprecationMode;
|
||||
Boolean trustAllSsl;
|
||||
BasicAuth basicAuth;
|
||||
|
||||
@Getter
|
||||
@ConfigurationProperties("basic-auth")
|
||||
public static class BasicAuth {
|
||||
String username;
|
||||
String password;
|
||||
}
|
||||
|
||||
@ConfigurationBuilder(configurationPrefix = "request.default")
|
||||
@SuppressWarnings("WeakerAccess")
|
||||
protected RequestConfig.Builder requestConfigBuilder = RequestConfig.custom();
|
||||
|
||||
@SneakyThrows
|
||||
public HttpAsyncClientBuilder httpAsyncClientBuilder() {
|
||||
HttpAsyncClientBuilder builder = HttpAsyncClientBuilder.create();
|
||||
|
||||
if (basicAuth != null) {
|
||||
final CredentialsProvider basicCredential = new BasicCredentialsProvider();
|
||||
basicCredential.setCredentials(
|
||||
AuthScope.ANY,
|
||||
new UsernamePasswordCredentials(this.basicAuth.username, this.basicAuth.password)
|
||||
);
|
||||
|
||||
builder.setDefaultCredentialsProvider(basicCredential);
|
||||
}
|
||||
|
||||
if (trustAllSsl != null && trustAllSsl) {
|
||||
SSLContextBuilder sslContextBuilder = new SSLContextBuilder();
|
||||
sslContextBuilder.loadTrustMaterial(null, (TrustStrategy) (chain, authType) -> true);
|
||||
SSLContext sslContext = sslContextBuilder.build();
|
||||
|
||||
builder.setSSLContext(sslContext);
|
||||
builder.setSSLHostnameVerifier(new NoopHostnameVerifier());
|
||||
}
|
||||
|
||||
return builder;
|
||||
}
|
||||
|
||||
public HttpHost[] httpHosts() {
|
||||
return Arrays.stream(this.httpHosts)
|
||||
.map(s -> {
|
||||
URI uri = URI.create(s);
|
||||
return new HttpHost(uri.getHost(), uri.getPort(), uri.getScheme());
|
||||
})
|
||||
.toArray(HttpHost[]::new);
|
||||
}
|
||||
|
||||
public Header[] defaultHeaders() {
|
||||
return Arrays.stream(this.defaultHeaders)
|
||||
.map(header -> {
|
||||
String[] nameAndValue = header.split(":");
|
||||
return new BasicHeader(nameAndValue[0], nameAndValue[1]);
|
||||
})
|
||||
.toArray(Header[]::new);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -1,63 +0,0 @@
|
||||
package io.kestra.repository.elasticsearch.configs;
|
||||
|
||||
import com.fasterxml.jackson.core.type.TypeReference;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import io.micronaut.context.annotation.EachProperty;
|
||||
import io.micronaut.context.annotation.Parameter;
|
||||
import lombok.Getter;
|
||||
import lombok.SneakyThrows;
|
||||
import io.kestra.core.serializers.JacksonMapper;
|
||||
import io.kestra.core.utils.MapUtils;
|
||||
|
||||
import java.net.URL;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
||||
@EachProperty("kestra.elasticsearch.indices")
|
||||
@Getter
|
||||
public class IndicesConfig {
|
||||
private final static ObjectMapper yamlMapper = JacksonMapper.ofYaml();
|
||||
private final static ObjectMapper jsonMapper = JacksonMapper.ofJson();
|
||||
private final static TypeReference<Map<String, Object>> typeReference = new TypeReference<>() {};
|
||||
|
||||
String name;
|
||||
|
||||
Class<?> cls;
|
||||
|
||||
String index;
|
||||
|
||||
String settings;
|
||||
|
||||
String mappingFile;
|
||||
|
||||
public IndicesConfig(@Parameter String name) {
|
||||
this.name = name;
|
||||
}
|
||||
|
||||
@SneakyThrows
|
||||
private Map<String, Object> readYamlFile(String path) {
|
||||
URL url = Objects.requireNonNull(this.getClass().getClassLoader()
|
||||
.getResource(path));
|
||||
|
||||
return yamlMapper.readValue(url, typeReference);
|
||||
}
|
||||
|
||||
@SneakyThrows
|
||||
public String getSettingsContent() {
|
||||
Map<String, Object> defaults = this.readYamlFile("settings.yml");
|
||||
|
||||
Map<String, Object> override = this.getSettings() == null ? Map.of() : jsonMapper.readValue(this.getSettings(), typeReference);
|
||||
|
||||
return jsonMapper.writeValueAsString(
|
||||
MapUtils.merge(defaults, override)
|
||||
);
|
||||
}
|
||||
|
||||
@SneakyThrows
|
||||
public String getMappingContent() {
|
||||
return jsonMapper.writeValueAsString(
|
||||
this.readYamlFile("mappings/" + this.getMappingFile() + ".yml")
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -1,104 +0,0 @@
|
||||
package io.kestra.repository.elasticsearch.services;
|
||||
|
||||
import io.kestra.core.metrics.MetricRegistry;
|
||||
import io.kestra.repository.elasticsearch.configs.ElasticsearchConfig;
|
||||
import io.micrometer.core.instrument.MeterRegistry;
|
||||
import io.micrometer.core.instrument.binder.httpcomponents.MicrometerHttpClientInterceptor;
|
||||
import io.micronaut.context.annotation.Bean;
|
||||
import io.micronaut.context.annotation.Factory;
|
||||
import io.micronaut.context.annotation.Requires;
|
||||
import jakarta.inject.Inject;
|
||||
import jakarta.inject.Singleton;
|
||||
import org.opensearch.client.RestClient;
|
||||
import org.opensearch.client.RestClientBuilder;
|
||||
import org.opensearch.client.RestHighLevelClient;
|
||||
|
||||
import java.net.URI;
|
||||
|
||||
|
||||
@Requires(beans = ElasticsearchConfig.class)
|
||||
@Factory
|
||||
public class ElasticsearchClientFactory {
|
||||
@Inject
|
||||
MetricRegistry metricRegistry;
|
||||
|
||||
@Inject
|
||||
MeterRegistry meterRegistry;
|
||||
|
||||
/**
|
||||
* Create the {@link RestHighLevelClient} bean for the given configuration.
|
||||
*
|
||||
* @param config The {@link ElasticsearchConfig} object
|
||||
* @return A {@link RestHighLevelClient} bean
|
||||
*/
|
||||
@Bean(preDestroy = "close")
|
||||
@Inject
|
||||
@Singleton
|
||||
RestHighLevelClient restHighLevelClient(ElasticsearchConfig config) {
|
||||
return new RestHighLevelClient(restClientBuilder(config));
|
||||
}
|
||||
|
||||
/**
|
||||
* @param config The {@link ElasticsearchConfig} object
|
||||
* @return The {@link RestClient} bean
|
||||
*/
|
||||
@Bean(preDestroy = "close")
|
||||
@Inject
|
||||
@Singleton
|
||||
RestClient restClient(ElasticsearchConfig config) {
|
||||
return restClientBuilder(config).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* @param config The {@link ElasticsearchConfig} object
|
||||
* @return The {@link RestClientBuilder}
|
||||
*/
|
||||
protected RestClientBuilder restClientBuilder(ElasticsearchConfig config) {
|
||||
RestClientBuilder builder = RestClient
|
||||
.builder(config.httpHosts())
|
||||
.setRequestConfigCallback(requestConfigBuilder -> {
|
||||
requestConfigBuilder = config.getRequestConfigBuilder();
|
||||
return requestConfigBuilder;
|
||||
})
|
||||
.setHttpClientConfigCallback(httpClientBuilder -> {
|
||||
httpClientBuilder = config.httpAsyncClientBuilder();
|
||||
|
||||
MicrometerHttpClientInterceptor interceptor = new MicrometerHttpClientInterceptor(
|
||||
meterRegistry,
|
||||
request -> {
|
||||
String path = URI.create(request.getRequestLine().getUri()).getPath();
|
||||
|
||||
int i = path.indexOf("/_doc/");
|
||||
|
||||
if (i < 0) {
|
||||
return path;
|
||||
} else {
|
||||
return path.substring(0, i + 5) + "/{id}";
|
||||
}
|
||||
},
|
||||
metricRegistry.tags("type", "elasticsearch"),
|
||||
true
|
||||
);
|
||||
|
||||
httpClientBuilder
|
||||
.addInterceptorFirst(interceptor.getRequestInterceptor())
|
||||
.addInterceptorLast(interceptor.getResponseInterceptor());
|
||||
|
||||
return httpClientBuilder;
|
||||
});
|
||||
|
||||
if (config.getDefaultHeaders() != null) {
|
||||
builder.setDefaultHeaders(config.defaultHeaders());
|
||||
}
|
||||
|
||||
if (config.getPathPrefix() != null) {
|
||||
builder.setPathPrefix(config.getPathPrefix());
|
||||
}
|
||||
|
||||
if (config.getStrictDeprecationMode() != null) {
|
||||
builder.setStrictDeprecationMode(config.getStrictDeprecationMode());
|
||||
}
|
||||
|
||||
return builder;
|
||||
}
|
||||
}
|
||||
@@ -1,143 +0,0 @@
|
||||
dynamic: false
|
||||
properties:
|
||||
flowId:
|
||||
type: keyword
|
||||
fields:
|
||||
fulltext:
|
||||
type: keyword
|
||||
normalizer: keyword
|
||||
flowRevision:
|
||||
type: integer
|
||||
id:
|
||||
type: keyword
|
||||
fields:
|
||||
fulltext:
|
||||
type: keyword
|
||||
inputs:
|
||||
type: object
|
||||
properties: {}
|
||||
namespace:
|
||||
type: keyword
|
||||
fields:
|
||||
fulltext:
|
||||
type: keyword
|
||||
normalizer: keyword
|
||||
parentId:
|
||||
type: keyword
|
||||
state:
|
||||
properties:
|
||||
current:
|
||||
type: keyword
|
||||
startDate:
|
||||
type: date
|
||||
endDate:
|
||||
type: date
|
||||
duration:
|
||||
type: double
|
||||
histories:
|
||||
properties:
|
||||
date:
|
||||
type: date
|
||||
state:
|
||||
type: keyword
|
||||
taskRunList:
|
||||
type: nested
|
||||
properties:
|
||||
id:
|
||||
type: text
|
||||
fields:
|
||||
fulltext:
|
||||
type: keyword
|
||||
executionId:
|
||||
type: text
|
||||
fields:
|
||||
fulltext:
|
||||
type: keyword
|
||||
namespace:
|
||||
type: text
|
||||
fields:
|
||||
fulltext:
|
||||
type: keyword
|
||||
normalizer: keyword
|
||||
flowId:
|
||||
type: text
|
||||
fields:
|
||||
fulltext:
|
||||
type: keyword
|
||||
normalizer: keyword
|
||||
taskId:
|
||||
type: text
|
||||
fields:
|
||||
fulltext:
|
||||
type: keyword
|
||||
normalizer: keyword
|
||||
parentTaskRunId:
|
||||
type: text
|
||||
fields:
|
||||
fulltext:
|
||||
type: keyword
|
||||
value:
|
||||
type: text
|
||||
fields:
|
||||
fulltext:
|
||||
type: keyword
|
||||
attempts:
|
||||
properties:
|
||||
state:
|
||||
properties:
|
||||
current:
|
||||
type: keyword
|
||||
startDate:
|
||||
type: date
|
||||
endDate:
|
||||
type: date
|
||||
duration:
|
||||
type: double
|
||||
histories:
|
||||
properties:
|
||||
date:
|
||||
type: date
|
||||
state:
|
||||
type: keyword
|
||||
metrics:
|
||||
properties:
|
||||
name:
|
||||
type: keyword
|
||||
type:
|
||||
type: keyword
|
||||
value:
|
||||
type: double
|
||||
tags:
|
||||
type: object
|
||||
dynamic: true
|
||||
properties: {}
|
||||
outputs:
|
||||
type: object
|
||||
properties: {}
|
||||
state:
|
||||
properties:
|
||||
current:
|
||||
type: keyword
|
||||
startDate:
|
||||
type: date
|
||||
endDate:
|
||||
type: date
|
||||
duration:
|
||||
type: double
|
||||
histories:
|
||||
properties:
|
||||
date:
|
||||
type: date
|
||||
state:
|
||||
type: keyword
|
||||
trigger:
|
||||
properties:
|
||||
id:
|
||||
type: keyword
|
||||
type:
|
||||
type: keyword
|
||||
variables:
|
||||
type: object
|
||||
properties: {}
|
||||
deleted:
|
||||
type: boolean
|
||||
@@ -1,51 +0,0 @@
|
||||
dynamic: false
|
||||
properties:
|
||||
id:
|
||||
type: keyword
|
||||
fields:
|
||||
fulltext:
|
||||
type: keyword
|
||||
normalizer: keyword
|
||||
namespace:
|
||||
type: keyword
|
||||
fields:
|
||||
fulltext:
|
||||
type: keyword
|
||||
normalizer: keyword
|
||||
revision:
|
||||
type: integer
|
||||
inputs:
|
||||
type: nested
|
||||
properties:
|
||||
name:
|
||||
type: text
|
||||
type:
|
||||
type: keyword
|
||||
required:
|
||||
type: boolean
|
||||
tasks:
|
||||
type: nested
|
||||
properties:
|
||||
id:
|
||||
type: keyword
|
||||
type:
|
||||
type: keyword
|
||||
errors:
|
||||
type: nested
|
||||
properties:
|
||||
id:
|
||||
type: keyword
|
||||
type:
|
||||
type: keyword
|
||||
deleted:
|
||||
type: boolean
|
||||
sourceCode:
|
||||
type: text
|
||||
term_vector: with_positions_offsets
|
||||
labelsMap:
|
||||
type: object
|
||||
properties:
|
||||
key:
|
||||
type: keyword
|
||||
value:
|
||||
type: keyword
|
||||
@@ -1,56 +0,0 @@
|
||||
dynamic: false
|
||||
properties:
|
||||
namespace:
|
||||
type: keyword
|
||||
fields:
|
||||
fulltext:
|
||||
type: keyword
|
||||
normalizer: keyword
|
||||
flowId:
|
||||
type: keyword
|
||||
fields:
|
||||
fulltext:
|
||||
type: keyword
|
||||
normalizer: keyword
|
||||
taskId:
|
||||
type: keyword
|
||||
fields:
|
||||
fulltext:
|
||||
type: keyword
|
||||
normalizer: keyword
|
||||
executionId:
|
||||
type: keyword
|
||||
fields:
|
||||
fulltext:
|
||||
type: keyword
|
||||
normalizer: keyword
|
||||
taskRunId:
|
||||
type: keyword
|
||||
fields:
|
||||
fulltext:
|
||||
type: keyword
|
||||
attemptNumber:
|
||||
type: integer
|
||||
triggerId:
|
||||
type: keyword
|
||||
level:
|
||||
type: keyword
|
||||
fields:
|
||||
fulltext:
|
||||
type: keyword
|
||||
message:
|
||||
type: text
|
||||
fields:
|
||||
fulltext:
|
||||
type: text
|
||||
analyzer: search
|
||||
thread:
|
||||
type: text
|
||||
fields:
|
||||
fulltext:
|
||||
type: text
|
||||
analyzer: search
|
||||
timestamp:
|
||||
type: date
|
||||
deleted:
|
||||
type: boolean
|
||||
@@ -1,11 +0,0 @@
|
||||
dynamic: false
|
||||
properties:
|
||||
key:
|
||||
type: keyword
|
||||
fields:
|
||||
fulltext:
|
||||
type: keyword
|
||||
normalizer: keyword
|
||||
outputs:
|
||||
type: object
|
||||
properties: {}
|
||||
@@ -1,28 +0,0 @@
|
||||
dynamic: false
|
||||
properties:
|
||||
id:
|
||||
type: keyword
|
||||
fields:
|
||||
fulltext:
|
||||
type: keyword
|
||||
normalizer: keyword
|
||||
namespace:
|
||||
type: keyword
|
||||
fields:
|
||||
fulltext:
|
||||
type: keyword
|
||||
normalizer: keyword
|
||||
tasks:
|
||||
properties:
|
||||
id:
|
||||
type: keyword
|
||||
type:
|
||||
type: keyword
|
||||
errors:
|
||||
properties:
|
||||
id:
|
||||
type: keyword
|
||||
type:
|
||||
type: keyword
|
||||
deleted:
|
||||
type: boolean
|
||||
@@ -1,31 +0,0 @@
|
||||
dynamic: false
|
||||
properties:
|
||||
namespace:
|
||||
type: keyword
|
||||
fields:
|
||||
fulltext:
|
||||
type: keyword
|
||||
normalizer: keyword
|
||||
flowId:
|
||||
type: keyword
|
||||
fields:
|
||||
fulltext:
|
||||
type: keyword
|
||||
normalizer: keyword
|
||||
flowRevision:
|
||||
type: integer
|
||||
triggerId:
|
||||
type: keyword
|
||||
fields:
|
||||
fulltext:
|
||||
type: keyword
|
||||
normalizer: keyword
|
||||
executionId:
|
||||
type: keyword
|
||||
fields:
|
||||
fulltext:
|
||||
type: keyword
|
||||
date:
|
||||
type: date
|
||||
deleted:
|
||||
type: boolean
|
||||
@@ -1,55 +0,0 @@
|
||||
index:
|
||||
analysis:
|
||||
normalizer:
|
||||
keyword:
|
||||
type: custom
|
||||
char_filter: []
|
||||
filter:
|
||||
- lowercase
|
||||
- asciifolding
|
||||
analyzer:
|
||||
prefix:
|
||||
type: custom
|
||||
tokenizer: standard
|
||||
filter:
|
||||
- word_delimiter
|
||||
- asciifolding
|
||||
- lowercase
|
||||
- prefix_ngrams
|
||||
search:
|
||||
type: custom
|
||||
tokenizer: standard
|
||||
filter:
|
||||
- word_delimiter
|
||||
- asciifolding
|
||||
- lowercase
|
||||
analyser:
|
||||
type: custom
|
||||
tokenizer: standard
|
||||
filter:
|
||||
- english_possessive_stemmer
|
||||
- lowercase
|
||||
- english_stop
|
||||
- english_stemmer
|
||||
filter:
|
||||
english_stop:
|
||||
type: stop
|
||||
stopwords: _english_
|
||||
english_stemmer:
|
||||
type: stemmer
|
||||
name: english
|
||||
word_delimiter:
|
||||
type: word_delimiter
|
||||
preserve_original: true
|
||||
english_possessive_stemmer:
|
||||
type: stemmer
|
||||
name: possessive_english
|
||||
remove_numbers:
|
||||
type: pattern_replace
|
||||
pattern: "([^\\p{L}]+)"
|
||||
replacement: ''
|
||||
prefix_ngrams:
|
||||
side: front
|
||||
max_gram: 30
|
||||
min_gram: 1
|
||||
type: edge_ngram
|
||||
@@ -1,95 +0,0 @@
|
||||
package io.kestra.core.tasks.executions;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import io.kestra.core.models.executions.statistics.Flow;
|
||||
import io.kestra.core.models.flows.State;
|
||||
import io.kestra.core.runners.RunContext;
|
||||
import io.kestra.core.runners.RunContextFactory;
|
||||
import io.kestra.repository.elasticsearch.ElasticSearchExecutionRepository;
|
||||
import io.kestra.repository.elasticsearch.ElasticSearchExecutionRepositoryTest;
|
||||
import io.kestra.repository.elasticsearch.ElasticSearchRepositoryTestUtils;
|
||||
import io.micronaut.test.extensions.junit5.annotation.MicronautTest;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import jakarta.inject.Inject;
|
||||
|
||||
import static org.hamcrest.MatcherAssert.assertThat;
|
||||
import static org.hamcrest.Matchers.is;
|
||||
|
||||
@MicronautTest
|
||||
class CountsTest {
|
||||
@Inject
|
||||
RunContextFactory runContextFactory;
|
||||
|
||||
@Inject
|
||||
ElasticSearchExecutionRepository executionRepository;
|
||||
|
||||
@Inject
|
||||
ElasticSearchRepositoryTestUtils utils;
|
||||
|
||||
@Test
|
||||
void run() throws Exception {
|
||||
for (int i = 0; i < 28; i++) {
|
||||
executionRepository.save(ElasticSearchExecutionRepositoryTest.builder(
|
||||
i < 5 ? State.Type.RUNNING : (i < 8 ? State.Type.FAILED : State.Type.SUCCESS),
|
||||
i < 4 ? "first" : (i < 10 ? "second" : "third")
|
||||
).build());
|
||||
}
|
||||
|
||||
RunContext runContext = runContextFactory.of(ImmutableMap.of("namespace", "io.kestra.unittest"));
|
||||
|
||||
// matching one
|
||||
Counts.Output run = Counts.builder()
|
||||
.flows(List.of(
|
||||
new Flow(ElasticSearchExecutionRepositoryTest.NAMESPACE, "first"),
|
||||
new Flow(ElasticSearchExecutionRepositoryTest.NAMESPACE, "second"),
|
||||
new Flow(ElasticSearchExecutionRepositoryTest.NAMESPACE, "third")
|
||||
))
|
||||
.expression("{{ count >= 5 }}")
|
||||
.startDate("{{ now() | dateAdd (-30, 'DAYS') }}")
|
||||
.endDate("{{ now() }}")
|
||||
.build()
|
||||
.run(runContext);
|
||||
|
||||
assertThat(run.getResults().size(), is(2));
|
||||
assertThat(run.getResults().stream().filter(f -> f.getFlowId().equals("second")).count(), is(1L));
|
||||
assertThat(run.getResults().stream().filter(f -> f.getFlowId().equals("third")).count(), is(1L));
|
||||
|
||||
// add state filter no result
|
||||
run = Counts.builder()
|
||||
.flows(List.of(
|
||||
new Flow(ElasticSearchExecutionRepositoryTest.NAMESPACE, "first"),
|
||||
new Flow(ElasticSearchExecutionRepositoryTest.NAMESPACE, "second"),
|
||||
new Flow(ElasticSearchExecutionRepositoryTest.NAMESPACE, "third")
|
||||
))
|
||||
.states(List.of(State.Type.RUNNING))
|
||||
.expression("{{ count >= 5 }}")
|
||||
.build()
|
||||
.run(runContext);
|
||||
|
||||
assertThat(run.getResults().size(), is(0));
|
||||
|
||||
// non matching entry
|
||||
run = Counts.builder()
|
||||
.flows(List.of(
|
||||
new Flow("io.kestra.test", "missing"),
|
||||
new Flow(ElasticSearchExecutionRepositoryTest.NAMESPACE, "second"),
|
||||
new Flow(ElasticSearchExecutionRepositoryTest.NAMESPACE, "third")
|
||||
))
|
||||
.expression("{{ count == 0 }}")
|
||||
.build()
|
||||
.run(runContext);
|
||||
|
||||
assertThat(run.getResults().size(), is(1));
|
||||
assertThat(run.getResults().stream().filter(f -> f.getFlowId().equals("missing")).count(), is(1L));
|
||||
}
|
||||
|
||||
@AfterEach
|
||||
protected void tearDown() throws IOException {
|
||||
utils.tearDown();
|
||||
executionRepository.initMapping();
|
||||
}
|
||||
}
|
||||
@@ -1,21 +0,0 @@
|
||||
package io.kestra.repository.elasticsearch;
|
||||
|
||||
import io.kestra.core.repositories.AbstractExecutionRepositoryTest;
|
||||
import jakarta.inject.Inject;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
public class ElasticSearchExecutionRepositoryTest extends AbstractExecutionRepositoryTest {
|
||||
@Inject
|
||||
ElasticSearchRepositoryTestUtils utils;
|
||||
|
||||
@Inject
|
||||
protected ElasticSearchExecutionRepository elasticExecutionRepository;
|
||||
|
||||
@AfterEach
|
||||
protected void tearDown() throws IOException {
|
||||
utils.tearDown();
|
||||
elasticExecutionRepository.initMapping();
|
||||
}
|
||||
}
|
||||
@@ -1,21 +0,0 @@
|
||||
package io.kestra.repository.elasticsearch;
|
||||
|
||||
import io.kestra.core.repositories.AbstractExecutionServiceTest;
|
||||
import jakarta.inject.Inject;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
class ElasticSearchExecutionServiceTest extends AbstractExecutionServiceTest {
|
||||
@Inject
|
||||
ElasticSearchRepositoryTestUtils utils;
|
||||
|
||||
@Inject
|
||||
ElasticSearchExecutionRepository elasticExecutionRepository;
|
||||
|
||||
@AfterEach
|
||||
protected void tearDown() throws IOException {
|
||||
utils.tearDown();
|
||||
elasticExecutionRepository.initMapping();
|
||||
}
|
||||
}
|
||||
@@ -1,72 +0,0 @@
|
||||
package io.kestra.repository.elasticsearch;
|
||||
|
||||
import io.kestra.core.Helpers;
|
||||
import io.kestra.core.models.SearchResult;
|
||||
import io.kestra.core.models.flows.Flow;
|
||||
import io.kestra.core.repositories.AbstractFlowRepositoryTest;
|
||||
import io.kestra.repository.elasticsearch.configs.IndicesConfig;
|
||||
import io.micronaut.data.model.Pageable;
|
||||
import io.micronaut.data.model.Sort;
|
||||
import io.micronaut.test.extensions.junit5.annotation.MicronautTest;
|
||||
import org.opensearch.client.RestHighLevelClient;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import jakarta.inject.Inject;
|
||||
|
||||
import static org.hamcrest.MatcherAssert.assertThat;
|
||||
import static org.hamcrest.Matchers.containsString;
|
||||
import static org.hamcrest.Matchers.is;
|
||||
|
||||
@MicronautTest
|
||||
class ElasticSearchFlowRepositoryTest extends AbstractFlowRepositoryTest {
|
||||
@Inject
|
||||
RestHighLevelClient client;
|
||||
|
||||
@Inject
|
||||
List<IndicesConfig> indicesConfigs;
|
||||
|
||||
@Inject
|
||||
ElasticSearchFlowRepository elasticSearchFlowRepository;
|
||||
|
||||
@Inject
|
||||
ElasticSearchTemplateRepository elasticSearchTemplateRepository;
|
||||
|
||||
@Inject
|
||||
private ElasticSearchRepositoryTestUtils utils;
|
||||
|
||||
@Test
|
||||
void find() {
|
||||
List<Flow> save = flowRepository.find(Pageable.from(1, 100, Sort.UNSORTED), null, null, null);
|
||||
assertThat((long) save.size(), is(Helpers.FLOWS_COUNT));
|
||||
|
||||
save = flowRepository.find(Pageable.from(1, 100, Sort.UNSORTED), null, null, Map.of("country", "FR"));
|
||||
assertThat(save.size(), is(1));
|
||||
|
||||
HashMap<String, String> map = new HashMap<>();
|
||||
map.put("region", null);
|
||||
save = flowRepository.find(Pageable.from(1, 100, Sort.UNSORTED), null, null, map);
|
||||
assertThat(save.size(), is(1));
|
||||
}
|
||||
|
||||
@Test
|
||||
void findSourceCode() {
|
||||
List<SearchResult<Flow>> search = flowRepository.findSourceCode(Pageable.from(1, 10, Sort.UNSORTED), "*types.MultipleCondition*", null);
|
||||
|
||||
assertThat((long) search.size(), is(1L));
|
||||
assertThat(search.get(0).getModel().getId(), is("trigger-multiplecondition-listener"));
|
||||
assertThat(search.get(0).getFragments().get(0), containsString("types.MultipleCondition[/mark]"));
|
||||
}
|
||||
|
||||
@AfterEach
|
||||
protected void tearDown() throws IOException {
|
||||
utils.tearDown();
|
||||
elasticSearchFlowRepository.initMapping();
|
||||
elasticSearchTemplateRepository.initMapping();
|
||||
}
|
||||
}
|
||||
@@ -1,21 +0,0 @@
|
||||
package io.kestra.repository.elasticsearch;
|
||||
|
||||
import io.kestra.core.repositories.AbstractLogRepositoryTest;
|
||||
import jakarta.inject.Inject;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
class ElasticSearchLogRepositoryTest extends AbstractLogRepositoryTest {
|
||||
@Inject
|
||||
ElasticSearchLogRepository elasticSearchLogRepository;
|
||||
|
||||
@Inject
|
||||
private ElasticSearchRepositoryTestUtils utils;
|
||||
|
||||
@AfterEach
|
||||
protected void tearDown() throws IOException {
|
||||
utils.tearDown();
|
||||
elasticSearchLogRepository.initMapping();
|
||||
}
|
||||
}
|
||||
@@ -1,31 +0,0 @@
|
||||
package io.kestra.repository.elasticsearch;
|
||||
|
||||
import org.opensearch.action.admin.indices.delete.DeleteIndexRequest;
|
||||
import org.opensearch.action.support.IndicesOptions;
|
||||
import org.opensearch.client.RequestOptions;
|
||||
import org.opensearch.client.RestHighLevelClient;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
import io.kestra.repository.elasticsearch.configs.IndicesConfig;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import jakarta.inject.Inject;
|
||||
import jakarta.inject.Singleton;
|
||||
|
||||
@Singleton
|
||||
public class ElasticSearchRepositoryTestUtils {
|
||||
@Inject
|
||||
RestHighLevelClient client;
|
||||
|
||||
@Inject
|
||||
List<IndicesConfig> indicesConfigs;
|
||||
|
||||
@AfterEach
|
||||
public void tearDown() throws IOException {
|
||||
DeleteIndexRequest deleteIndexRequest = new DeleteIndexRequest(indicesConfigs.stream()
|
||||
.map(IndicesConfig::getIndex)
|
||||
.toArray(String[]::new))
|
||||
.indicesOptions(IndicesOptions.LENIENT_EXPAND_OPEN);
|
||||
client.indices().delete(deleteIndexRequest, RequestOptions.DEFAULT);
|
||||
}
|
||||
}
|
||||
@@ -1,21 +0,0 @@
|
||||
package io.kestra.repository.elasticsearch;
|
||||
|
||||
import io.kestra.core.repositories.AbstracSettingRepositoryTest;
|
||||
import jakarta.inject.Inject;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
class ElasticSearchSettingRepositoryTest extends AbstracSettingRepositoryTest {
|
||||
@Inject
|
||||
ElasticsearchSettingRepository elasticsearchSettingRepository;
|
||||
|
||||
@Inject
|
||||
private ElasticSearchRepositoryTestUtils utils;
|
||||
|
||||
@AfterEach
|
||||
protected void tearDown() throws IOException {
|
||||
utils.tearDown();
|
||||
elasticsearchSettingRepository.initMapping();
|
||||
}
|
||||
}
|
||||
@@ -1,23 +0,0 @@
|
||||
package io.kestra.repository.elasticsearch;
|
||||
|
||||
import io.micronaut.test.extensions.junit5.annotation.MicronautTest;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
import io.kestra.core.repositories.AbstractTemplateRepositoryTest;
|
||||
|
||||
import java.io.IOException;
|
||||
import jakarta.inject.Inject;
|
||||
|
||||
@MicronautTest
|
||||
class ElasticSearchTemplateRepositoryTest extends AbstractTemplateRepositoryTest {
|
||||
@Inject
|
||||
ElasticSearchTemplateRepository elasticSearchTemplateRepository;
|
||||
|
||||
@Inject
|
||||
private ElasticSearchRepositoryTestUtils utils;
|
||||
|
||||
@AfterEach
|
||||
protected void tearDown() throws IOException {
|
||||
utils.tearDown();
|
||||
elasticSearchTemplateRepository.initMapping();
|
||||
}
|
||||
}
|
||||
@@ -1,21 +0,0 @@
|
||||
package io.kestra.repository.elasticsearch;
|
||||
|
||||
import io.kestra.core.repositories.AbstractTriggerRepositoryTest;
|
||||
import jakarta.inject.Inject;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
class ElasticSearchTriggerRepositoryTest extends AbstractTriggerRepositoryTest {
|
||||
@Inject
|
||||
ElasticsearchTriggerRepository elasticsearchTriggerRepository;
|
||||
|
||||
@Inject
|
||||
private ElasticSearchRepositoryTestUtils utils;
|
||||
|
||||
@AfterEach
|
||||
protected void tearDown() throws IOException {
|
||||
utils.tearDown();
|
||||
elasticsearchTriggerRepository.initMapping();
|
||||
}
|
||||
}
|
||||
@@ -1,60 +0,0 @@
|
||||
kestra:
|
||||
queue:
|
||||
type: memory
|
||||
repository:
|
||||
type: elasticsearch
|
||||
storage:
|
||||
type: local
|
||||
local:
|
||||
base-path: /tmp/unittest
|
||||
|
||||
elasticsearch:
|
||||
client:
|
||||
httpHosts: http://localhost:9200
|
||||
defaults:
|
||||
indice-prefix: "unit_kestra_${random.shortuuid}_"
|
||||
indices:
|
||||
flows:
|
||||
index: "${kestra.elasticsearch.defaults.indice-prefix}flows"
|
||||
cls: io.kestra.core.models.flows.Flow
|
||||
mapping-file: flow
|
||||
settings: &settings |
|
||||
{
|
||||
"index.number_of_replicas": 0
|
||||
}
|
||||
|
||||
flows-revisions:
|
||||
index: "${kestra.elasticsearch.defaults.indice-prefix}flows_revisions"
|
||||
cls: io.kestra.core.models.flows.Flow
|
||||
mapping-file: flow
|
||||
settings: *settings
|
||||
|
||||
executions:
|
||||
index: "${kestra.elasticsearch.defaults.indice-prefix}executions"
|
||||
cls: io.kestra.core.models.executions.Execution
|
||||
mapping-file: execution
|
||||
settings: *settings
|
||||
|
||||
templates:
|
||||
index: "${kestra.elasticsearch.defaults.indice-prefix}templates"
|
||||
cls: io.kestra.core.models.templates.Template
|
||||
mapping-file: template
|
||||
settings: *settings
|
||||
|
||||
triggers:
|
||||
index: "${kestra.elasticsearch.defaults.indice-prefix}triggers"
|
||||
cls: io.kestra.core.models.triggers.Trigger
|
||||
mapping-file: trigger
|
||||
settings: *settings
|
||||
|
||||
logs:
|
||||
index: "${kestra.elasticsearch.defaults.indice-prefix}logs"
|
||||
cls: io.kestra.core.models.executions.LogEntry
|
||||
mapping-file: log
|
||||
settings: *settings
|
||||
|
||||
settings:
|
||||
index: "${kestra.elasticsearch.defaults.indice-prefix}settings"
|
||||
cls: io.kestra.core.models.Setting
|
||||
mapping-file: setting
|
||||
settings: *settings
|
||||
@@ -1 +0,0 @@
|
||||
mock-maker-inline
|
||||
@@ -1,16 +0,0 @@
|
||||
publishSonatypePublicationPublicationToSonatypeRepository.enabled = false
|
||||
|
||||
dependencies {
|
||||
implementation project(":core")
|
||||
|
||||
implementation group: "org.apache.kafka", name: "kafka-clients", version: kafkaVersion
|
||||
implementation group: "org.apache.kafka", name: 'kafka-streams', version: kafkaVersion
|
||||
|
||||
testImplementation project(':core').sourceSets.test.output
|
||||
testImplementation project(':repository-memory')
|
||||
testImplementation project(':storage-local')
|
||||
|
||||
testImplementation group: 'org.apache.kafka', name: 'kafka-streams-test-utils', version: kafkaVersion
|
||||
|
||||
testImplementation 'org.mockito:mockito-junit-jupiter:4.11.0'
|
||||
}
|
||||
@@ -1,165 +0,0 @@
|
||||
package io.kestra.runner.kafka;
|
||||
|
||||
import com.fasterxml.jackson.core.JsonProcessingException;
|
||||
import io.kestra.core.serializers.JacksonMapper;
|
||||
import io.kestra.core.utils.ListUtils;
|
||||
import io.kestra.runner.kafka.configs.ClientConfig;
|
||||
import io.kestra.runner.kafka.configs.LoggerConfig;
|
||||
import io.kestra.runner.kafka.services.KafkaStreamService;
|
||||
import io.micronaut.context.ApplicationContext;
|
||||
import io.micronaut.core.annotation.Nullable;
|
||||
import lombok.extern.slf4j.Slf4j;
|
||||
import org.apache.kafka.common.serialization.StringDeserializer;
|
||||
import org.slf4j.event.Level;
|
||||
|
||||
import java.time.Instant;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
||||
@Slf4j
|
||||
public abstract class AbstractInterceptor {
|
||||
private final static StringDeserializer STRING_DESERIALIZER = new StringDeserializer();
|
||||
|
||||
ClientConfig clientConfig;
|
||||
|
||||
protected <K, V> void logRecord(
|
||||
LoggerConfig.Type type,
|
||||
String topic,
|
||||
Integer partition,
|
||||
@Nullable Long offset,
|
||||
Long timestamp,
|
||||
K key,
|
||||
V value
|
||||
) {
|
||||
Level level = isMatch(type, topic, key, value);
|
||||
|
||||
if (level == null) {
|
||||
return;
|
||||
}
|
||||
|
||||
String format = "[{} > {}{}{}{}] {} = {}";
|
||||
Object[] args = {
|
||||
type,
|
||||
topic,
|
||||
partition != null ? "[" + partition + "]" : "",
|
||||
offset != null ? "@" + offset : "",
|
||||
timestamp != null ? " " + Instant.ofEpochMilli(timestamp) : "",
|
||||
deserialize(key),
|
||||
deserialize(value)
|
||||
};
|
||||
|
||||
this.log(level, format, args);
|
||||
}
|
||||
|
||||
protected void log(Level level, String format, Object[] args) {
|
||||
if (level == Level.TRACE) {
|
||||
log.trace(format, args);
|
||||
} else if (level == Level.DEBUG) {
|
||||
log.debug(format, args);
|
||||
} else if (level == Level.INFO) {
|
||||
log.info(format, args);
|
||||
} else if (level == Level.WARN) {
|
||||
log.warn(format, args);
|
||||
} else if (level == Level.ERROR) {
|
||||
log.error(format, args);
|
||||
}
|
||||
}
|
||||
|
||||
protected <K, V> Level isMatch(LoggerConfig.Type type, String topic) {
|
||||
return ListUtils.emptyOnNull(this.clientConfig.getLoggers())
|
||||
.stream()
|
||||
.map(loggerConfig -> {
|
||||
if (logEnabled(loggerConfig.getLevel()) &&
|
||||
this.isMatchType(loggerConfig, type) &&
|
||||
this.isMatchTopic(loggerConfig, topic)
|
||||
) {
|
||||
return loggerConfig.getLevel();
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
})
|
||||
.filter(Objects::nonNull)
|
||||
.findFirst()
|
||||
.orElse(null);
|
||||
}
|
||||
|
||||
protected <K, V> Level isMatch(LoggerConfig.Type type, String topic, K key, V value) {
|
||||
return ListUtils.emptyOnNull(this.clientConfig.getLoggers())
|
||||
.stream()
|
||||
.map(loggerConfig -> {
|
||||
if (logEnabled(loggerConfig.getLevel()) &&
|
||||
this.isMatchType(loggerConfig, type) &&
|
||||
this.isMatchTopic(loggerConfig, topic) &&
|
||||
this.isMatchKey(loggerConfig, key) &&
|
||||
this.isMatchValue(loggerConfig, value)
|
||||
) {
|
||||
return loggerConfig.getLevel();
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
})
|
||||
.filter(Objects::nonNull)
|
||||
.findFirst()
|
||||
.orElse(null);
|
||||
}
|
||||
|
||||
private boolean isMatchType(LoggerConfig loggerConfig, LoggerConfig.Type type) {
|
||||
return (
|
||||
loggerConfig.getType() == null ||
|
||||
loggerConfig.getType() == type
|
||||
);
|
||||
}
|
||||
|
||||
private boolean isMatchTopic(LoggerConfig loggerConfig, String topic) {
|
||||
return (
|
||||
loggerConfig.getTopicRegexp() == null ||
|
||||
topic.matches(loggerConfig.getTopicRegexp())
|
||||
);
|
||||
}
|
||||
|
||||
private <K> boolean isMatchKey(LoggerConfig loggerConfig, K key) {
|
||||
return (
|
||||
loggerConfig.getKeyRegexp() == null ||
|
||||
deserialize(key).matches(loggerConfig.getKeyRegexp())
|
||||
);
|
||||
}
|
||||
|
||||
private <V> boolean isMatchValue(LoggerConfig loggerConfig, V value) {
|
||||
return (
|
||||
loggerConfig.getValueRegexp() == null ||
|
||||
deserialize(value).matches(loggerConfig.getValueRegexp())
|
||||
);
|
||||
}
|
||||
|
||||
private String deserialize(Object value) {
|
||||
if (value instanceof byte[]) {
|
||||
return STRING_DESERIALIZER.deserialize("", (byte[])value);
|
||||
} else if (value instanceof String) {
|
||||
return (String) value;
|
||||
} else {
|
||||
try {
|
||||
return JacksonMapper.ofJson(false).writeValueAsString(value);
|
||||
} catch (JsonProcessingException e) {
|
||||
return "";
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
protected boolean logEnabled(Level level) {
|
||||
return (level == Level.TRACE && log.isTraceEnabled()) ||
|
||||
(level == Level.DEBUG && log.isDebugEnabled()) ||
|
||||
(level == Level.INFO && log.isInfoEnabled()) ||
|
||||
(level == Level.WARN && log.isWarnEnabled()) ||
|
||||
(level == Level.ERROR && log.isErrorEnabled());
|
||||
}
|
||||
|
||||
public void configure(Map<String, ?> configs) {
|
||||
ApplicationContext applicationContext = (ApplicationContext) configs.get(KafkaStreamService.APPLICATION_CONTEXT_CONFIG);
|
||||
clientConfig = applicationContext.getBean(ClientConfig.class);
|
||||
}
|
||||
|
||||
public void close() {
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
@@ -1,61 +0,0 @@
|
||||
package io.kestra.runner.kafka;
|
||||
|
||||
import io.kestra.runner.kafka.configs.LoggerConfig;
|
||||
import io.micronaut.core.annotation.Nullable;
|
||||
import lombok.NoArgsConstructor;
|
||||
import org.apache.kafka.clients.consumer.ConsumerRecords;
|
||||
import org.apache.kafka.clients.consumer.OffsetAndMetadata;
|
||||
import org.apache.kafka.common.TopicPartition;
|
||||
import org.slf4j.event.Level;
|
||||
|
||||
import java.time.Instant;
|
||||
import java.util.Map;
|
||||
|
||||
@NoArgsConstructor
|
||||
public class ConsumerInterceptor<K,V> extends AbstractInterceptor implements org.apache.kafka.clients.consumer.ConsumerInterceptor<K, V> {
|
||||
@Override
|
||||
public ConsumerRecords<K, V> onConsume(ConsumerRecords<K, V> records) {
|
||||
records.forEach(record -> this.logRecord(
|
||||
LoggerConfig.Type.CONSUMER,
|
||||
record.topic(),
|
||||
record.partition(),
|
||||
record.offset(),
|
||||
record.timestamp(),
|
||||
record.key(),
|
||||
record.value()
|
||||
));
|
||||
|
||||
return records;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onCommit(Map<TopicPartition, OffsetAndMetadata> offsets) {
|
||||
offsets.forEach((topicPartition, offsetAndMetadata) -> this.logCommit(
|
||||
topicPartition.topic(),
|
||||
topicPartition.partition(),
|
||||
offsetAndMetadata.offset()
|
||||
));
|
||||
}
|
||||
|
||||
protected void logCommit(
|
||||
String topic,
|
||||
Integer partition,
|
||||
@Nullable Long offset
|
||||
) {
|
||||
Level level = isMatch(LoggerConfig.Type.CONSUMER_COMMIT, topic);
|
||||
|
||||
if (level == null) {
|
||||
return;
|
||||
}
|
||||
|
||||
String format = "[{}> {}{}{}]";
|
||||
Object[] args = {
|
||||
LoggerConfig.Type.CONSUMER_COMMIT,
|
||||
topic,
|
||||
partition != null ? "[" + partition + "]" : "",
|
||||
offset != null ? "@" + offset : "",
|
||||
};
|
||||
|
||||
this.log(level, format, args);
|
||||
}
|
||||
}
|
||||
@@ -1,48 +0,0 @@
|
||||
package io.kestra.runner.kafka;
|
||||
|
||||
import io.kestra.core.models.flows.Flow;
|
||||
import io.kestra.core.models.templates.Template;
|
||||
import io.kestra.runner.kafka.configs.TopicsConfig;
|
||||
import io.kestra.runner.kafka.services.KafkaStreamService;
|
||||
import io.micronaut.context.ApplicationContext;
|
||||
import lombok.extern.slf4j.Slf4j;
|
||||
import org.apache.kafka.clients.consumer.ConsumerRecord;
|
||||
import org.apache.kafka.streams.errors.DeserializationExceptionHandler;
|
||||
import org.apache.kafka.streams.processor.ProcessorContext;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
@Slf4j
|
||||
public class KafkaDeserializationExceptionHandler implements DeserializationExceptionHandler {
|
||||
private ApplicationContext applicationContext;
|
||||
|
||||
@Override
|
||||
public void configure(Map<String, ?> configs) {
|
||||
applicationContext = (ApplicationContext) configs.get(KafkaStreamService.APPLICATION_CONTEXT_CONFIG);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DeserializationHandlerResponse handle(ProcessorContext context, ConsumerRecord<byte[], byte[]> record, Exception exception) {
|
||||
String message = "Exception caught during deserialization, stream will continue! applicationId: {}, taskId: {}, topic: {}, partition: {}, offset: {}";
|
||||
Object[] args = {
|
||||
context.applicationId(),
|
||||
context.taskId(),
|
||||
record.topic(),
|
||||
record.partition(),
|
||||
record.offset(),
|
||||
exception
|
||||
};
|
||||
|
||||
TopicsConfig topicsConfig = KafkaQueue.topicsConfigByTopicName(applicationContext, record.topic());
|
||||
|
||||
if (topicsConfig.getCls() == Flow.class || topicsConfig.getCls() == Template.class) {
|
||||
if (log.isDebugEnabled()) {
|
||||
log.debug(message, args);
|
||||
}
|
||||
} else {
|
||||
log.warn(message, args);
|
||||
}
|
||||
|
||||
return DeserializationHandlerResponse.CONTINUE;
|
||||
}
|
||||
}
|
||||
@@ -1,97 +0,0 @@
|
||||
package io.kestra.runner.kafka;
|
||||
|
||||
import io.kestra.core.models.executions.Execution;
|
||||
import io.kestra.core.models.executions.ExecutionKilled;
|
||||
import io.kestra.core.models.executions.LogEntry;
|
||||
import io.kestra.core.models.flows.Flow;
|
||||
import io.kestra.core.models.templates.Template;
|
||||
import io.kestra.core.models.triggers.Trigger;
|
||||
import io.kestra.core.runners.*;
|
||||
import io.kestra.runner.kafka.executors.KafkaExecutorInterface;
|
||||
import io.kestra.runner.kafka.services.KafkaAdminService;
|
||||
import io.kestra.runner.kafka.services.KafkaStreamService;
|
||||
import io.kestra.runner.kafka.services.KafkaStreamSourceService;
|
||||
import io.kestra.runner.kafka.streams.ExecutorFlowTrigger;
|
||||
import io.micronaut.context.ApplicationContext;
|
||||
import jakarta.inject.Inject;
|
||||
import jakarta.inject.Singleton;
|
||||
import lombok.extern.slf4j.Slf4j;
|
||||
import org.apache.kafka.streams.Topology;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.time.Duration;
|
||||
import java.util.List;
|
||||
import java.util.Properties;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
@KafkaQueueEnabled
|
||||
@Singleton
|
||||
@Slf4j
|
||||
public class KafkaExecutor implements ExecutorInterface {
|
||||
private List<KafkaStreamService.Stream> streams;
|
||||
|
||||
@Inject
|
||||
protected ApplicationContext applicationContext;
|
||||
|
||||
@Inject
|
||||
protected KafkaStreamService kafkaStreamService;
|
||||
|
||||
@Inject
|
||||
protected KafkaAdminService kafkaAdminService;
|
||||
|
||||
@Inject
|
||||
protected List<KafkaExecutorInterface> kafkaExecutors;
|
||||
|
||||
@Inject
|
||||
protected ExecutorService executorService;
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
kafkaAdminService.createIfNotExist(WorkerTask.class);
|
||||
kafkaAdminService.createIfNotExist(WorkerTaskResult.class);
|
||||
kafkaAdminService.createIfNotExist(Execution.class);
|
||||
kafkaAdminService.createIfNotExist(Flow.class);
|
||||
kafkaAdminService.createIfNotExist(KafkaStreamSourceService.TOPIC_FLOWLAST);
|
||||
kafkaAdminService.createIfNotExist(Executor.class);
|
||||
kafkaAdminService.createIfNotExist(KafkaStreamSourceService.TOPIC_EXECUTOR_WORKERINSTANCE);
|
||||
kafkaAdminService.createIfNotExist(ExecutionKilled.class);
|
||||
kafkaAdminService.createIfNotExist(WorkerTaskExecution.class);
|
||||
kafkaAdminService.createIfNotExist(WorkerTaskRunning.class);
|
||||
kafkaAdminService.createIfNotExist(WorkerInstance.class);
|
||||
kafkaAdminService.createIfNotExist(Template.class);
|
||||
kafkaAdminService.createIfNotExist(LogEntry.class);
|
||||
kafkaAdminService.createIfNotExist(Trigger.class);
|
||||
kafkaAdminService.createIfNotExist(ExecutorFlowTrigger.class);
|
||||
|
||||
this.streams = this.kafkaExecutors
|
||||
.stream()
|
||||
.parallel()
|
||||
.map(executor -> {
|
||||
Properties properties = new Properties();
|
||||
// build
|
||||
Topology topology = executor.topology().build();
|
||||
|
||||
Logger logger = LoggerFactory.getLogger(executor.getClass());
|
||||
KafkaStreamService.Stream stream = kafkaStreamService.of(executor.getClass(), executor.getClass(), topology, properties, logger);
|
||||
stream.start();
|
||||
|
||||
executor.onCreated(applicationContext, stream);
|
||||
|
||||
applicationContext.inject(stream);
|
||||
|
||||
return stream;
|
||||
})
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
if (streams != null) {
|
||||
streams
|
||||
.parallelStream()
|
||||
.forEach(stream -> stream.close(Duration.ofSeconds(10)));
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -1,134 +0,0 @@
|
||||
package io.kestra.runner.kafka;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import io.kestra.core.runners.Executor;
|
||||
import io.kestra.runner.kafka.services.KafkaStreamService;
|
||||
import io.kestra.runner.kafka.streams.ExecutorFlowTrigger;
|
||||
import io.micronaut.context.ApplicationContext;
|
||||
import io.micronaut.inject.qualifiers.Qualifiers;
|
||||
import lombok.extern.slf4j.Slf4j;
|
||||
import org.apache.kafka.clients.producer.ProducerRecord;
|
||||
import org.apache.kafka.common.errors.RecordTooLargeException;
|
||||
import io.kestra.core.exceptions.InternalException;
|
||||
import io.kestra.core.models.executions.Execution;
|
||||
import io.kestra.core.models.executions.LogEntry;
|
||||
import io.kestra.core.models.executions.TaskRun;
|
||||
import io.kestra.core.queues.QueueFactoryInterface;
|
||||
import io.kestra.core.queues.QueueInterface;
|
||||
import io.kestra.runner.kafka.configs.TopicsConfig;
|
||||
import io.kestra.runner.kafka.serializers.JsonDeserializer;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.Map;
|
||||
|
||||
@Slf4j
|
||||
public class KafkaExecutorProductionExceptionHandler implements org.apache.kafka.streams.errors.ProductionExceptionHandler {
|
||||
private ApplicationContext applicationContext;
|
||||
private KafkaQueue<Execution> executionQueue;
|
||||
private KafkaQueue<Executor> executorQueue;
|
||||
private KafkaQueue<LogEntry> logQueue;
|
||||
|
||||
@Override
|
||||
public ProductionExceptionHandlerResponse handle(ProducerRecord<byte[], byte[]> record, Exception exception) {
|
||||
if (log.isDebugEnabled()) {
|
||||
log.error(
|
||||
"Failed to produced message on topic '{}', partition '{}', key '{}', value '{}' with exception '{}'",
|
||||
record.topic(),
|
||||
record.partition(),
|
||||
new String(record.key()),
|
||||
new String(record.value()),
|
||||
exception.getMessage(),
|
||||
exception
|
||||
);
|
||||
} else {
|
||||
log.error(
|
||||
"Failed to produced message on topic '{}', partition '{}', key '{}' with exception '{}'",
|
||||
record.topic(),
|
||||
record.partition(),
|
||||
new String(record.key()),
|
||||
exception.getMessage(),
|
||||
exception
|
||||
);
|
||||
}
|
||||
|
||||
try {
|
||||
TopicsConfig topicsConfig = KafkaQueue.topicsConfigByTopicName(applicationContext, record.topic());
|
||||
|
||||
|
||||
if (topicsConfig.getCls() == Executor.class || topicsConfig.getCls() == ExecutorFlowTrigger.class) {
|
||||
return ProductionExceptionHandlerResponse.CONTINUE;
|
||||
} else if (topicsConfig.getCls() == Execution.class) {
|
||||
Execution execution = getObject(Execution.class, record);
|
||||
|
||||
Execution.FailedExecutionWithLog failedExecutionWithLog = execution.failedExecutionFromExecutor(exception);
|
||||
Execution sendExecution = failedExecutionWithLog.getExecution();
|
||||
|
||||
failedExecutionWithLog.getLogs().forEach(logEntry -> logQueue.emitAsync(logEntry));
|
||||
|
||||
if (exception instanceof RecordTooLargeException) {
|
||||
boolean exit = false;
|
||||
while (!exit) {
|
||||
try {
|
||||
sendExecution = reduceOutputs(sendExecution);
|
||||
executionQueue.emit(sendExecution);
|
||||
executorQueue.emit(null);
|
||||
exit = true;
|
||||
} catch (Exception e) {
|
||||
exit = sendExecution.getTaskRunList().size() > 0;
|
||||
}
|
||||
}
|
||||
|
||||
return ProductionExceptionHandlerResponse.CONTINUE;
|
||||
} else {
|
||||
executionQueue.emit(sendExecution);
|
||||
}
|
||||
}
|
||||
} catch (Exception e) {
|
||||
log.warn("Can't resolve failed produce with exception '{}'", e.getMessage(), e);
|
||||
}
|
||||
|
||||
return ProductionExceptionHandlerResponse.FAIL;
|
||||
}
|
||||
|
||||
private Execution reduceOutputs(Execution execution) throws InternalException {
|
||||
if (execution.getTaskRunList().size() == 0) {
|
||||
return execution;
|
||||
}
|
||||
|
||||
ArrayList<TaskRun> reverse = new ArrayList<>(execution.getTaskRunList());
|
||||
Collections.reverse(reverse);
|
||||
|
||||
TaskRun taskRun = reverse.get(0).withOutputs(ImmutableMap.of());
|
||||
|
||||
return execution.withTaskRun(taskRun);
|
||||
}
|
||||
|
||||
@SuppressWarnings({"unchecked", "rawtypes"})
|
||||
private <T> T getObject(Class<T> cls, ProducerRecord<byte[], byte[]> record) {
|
||||
try (JsonDeserializer jsonDeserializer = new JsonDeserializer(cls)) {
|
||||
return (T) jsonDeserializer.deserialize(record.topic(), record.value());
|
||||
}
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
@Override
|
||||
public void configure(Map<String, ?> configs) {
|
||||
applicationContext = (ApplicationContext) configs.get(KafkaStreamService.APPLICATION_CONTEXT_CONFIG);
|
||||
|
||||
executionQueue = (KafkaQueue<Execution>) applicationContext.getBean(
|
||||
QueueInterface.class,
|
||||
Qualifiers.byName(QueueFactoryInterface.EXECUTION_NAMED)
|
||||
);
|
||||
|
||||
executorQueue = (KafkaQueue<Executor>) applicationContext.getBean(
|
||||
QueueInterface.class,
|
||||
Qualifiers.byName(QueueFactoryInterface.EXECUTOR_NAMED)
|
||||
);
|
||||
|
||||
logQueue = (KafkaQueue<LogEntry>) applicationContext.getBean(
|
||||
QueueInterface.class,
|
||||
Qualifiers.byName(QueueFactoryInterface.WORKERTASKLOG_NAMED)
|
||||
);
|
||||
}
|
||||
}
|
||||
@@ -1,75 +0,0 @@
|
||||
package io.kestra.runner.kafka;
|
||||
|
||||
import io.kestra.core.models.flows.Flow;
|
||||
import io.kestra.core.runners.FlowExecutorInterface;
|
||||
import io.kestra.core.services.FlowService;
|
||||
import io.kestra.core.utils.Await;
|
||||
import io.kestra.runner.kafka.services.SafeKeyValueStore;
|
||||
import jakarta.inject.Inject;
|
||||
import jakarta.inject.Singleton;
|
||||
import lombok.SneakyThrows;
|
||||
import lombok.extern.slf4j.Slf4j;
|
||||
|
||||
import java.time.Duration;
|
||||
import java.util.*;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
@Slf4j
|
||||
@Singleton
|
||||
public class KafkaFlowExecutor implements FlowExecutorInterface {
|
||||
@Inject
|
||||
private FlowService flowService;
|
||||
private SafeKeyValueStore<String, Flow> store;
|
||||
private Map<String, Flow> flowsLast;
|
||||
|
||||
public synchronized void setFlows(List<Flow> flows) {
|
||||
this.flowsLast = flowService.keepLastVersion(flows)
|
||||
.stream()
|
||||
.map(flow -> new AbstractMap.SimpleEntry<>(
|
||||
flow.uidWithoutRevision(),
|
||||
flow
|
||||
))
|
||||
.collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
|
||||
}
|
||||
|
||||
public synchronized void setStore(SafeKeyValueStore<String, Flow> store) {
|
||||
this.store = store;
|
||||
}
|
||||
|
||||
@SneakyThrows
|
||||
private void await() {
|
||||
if (flowsLast == null || store == null) {
|
||||
Await.until(() -> this.isReady() == true, Duration.ofMillis(100), Duration.ofMinutes(5));
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Collection<Flow> allLastVersion() {
|
||||
this.await();
|
||||
|
||||
return this.flowsLast.values();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<Flow> findById(String namespace, String id, Optional<Integer> revision) {
|
||||
this.await();
|
||||
|
||||
String uid = Flow.uidWithoutRevision(namespace, id);
|
||||
Flow flowLast = this.flowsLast.get(uid);
|
||||
|
||||
if (revision.isEmpty()) {
|
||||
return Optional.ofNullable(flowLast);
|
||||
}
|
||||
|
||||
if (flowLast != null && revision.get().equals(flowLast.getRevision())) {
|
||||
return Optional.of(flowLast);
|
||||
}
|
||||
|
||||
return this.store.get(Flow.uid(namespace, id, revision));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Boolean isReady() {
|
||||
return this.flowsLast != null && store != null;
|
||||
}
|
||||
}
|
||||
@@ -1,144 +0,0 @@
|
||||
package io.kestra.runner.kafka;
|
||||
|
||||
import io.kestra.core.runners.FlowListeners;
|
||||
import io.kestra.runner.kafka.services.*;
|
||||
import io.micronaut.context.annotation.Replaces;
|
||||
import lombok.extern.slf4j.Slf4j;
|
||||
import org.apache.commons.lang3.NotImplementedException;
|
||||
import org.apache.kafka.common.serialization.Serdes;
|
||||
import org.apache.kafka.common.utils.Bytes;
|
||||
import org.apache.kafka.streams.KafkaStreams;
|
||||
import org.apache.kafka.streams.StoreQueryParameters;
|
||||
import org.apache.kafka.streams.StreamsBuilder;
|
||||
import org.apache.kafka.streams.Topology;
|
||||
import org.apache.kafka.streams.errors.InvalidStateStoreException;
|
||||
import org.apache.kafka.streams.kstream.*;
|
||||
import org.apache.kafka.streams.state.KeyValueStore;
|
||||
import org.apache.kafka.streams.state.QueryableStoreTypes;
|
||||
import org.apache.kafka.streams.state.ReadOnlyKeyValueStore;
|
||||
import io.kestra.core.models.flows.Flow;
|
||||
import io.kestra.core.services.FlowListenersInterface;
|
||||
import io.kestra.runner.kafka.serializers.JsonSerde;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.function.BiConsumer;
|
||||
import java.util.function.Consumer;
|
||||
import java.util.stream.Collectors;
|
||||
import jakarta.inject.Inject;
|
||||
import jakarta.inject.Singleton;
|
||||
|
||||
@Singleton
|
||||
@Slf4j
|
||||
@KafkaQueueEnabled
|
||||
@Replaces(FlowListeners.class)
|
||||
public class KafkaFlowListeners implements FlowListenersInterface {
|
||||
private final KafkaAdminService kafkaAdminService;
|
||||
private final KafkaStreamService kafkaStreamService;
|
||||
private SafeKeyValueStore<String, Flow> store;
|
||||
private final List<Consumer<List<Flow>>> consumers = new ArrayList<>();
|
||||
private KafkaStreamService.Stream stream;
|
||||
private List<Flow> flows;
|
||||
|
||||
@Inject
|
||||
public KafkaFlowListeners(KafkaAdminService kafkaAdminService, KafkaStreamService kafkaStreamService) {
|
||||
this.kafkaAdminService = kafkaAdminService;
|
||||
this.kafkaStreamService = kafkaStreamService;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
kafkaAdminService.createIfNotExist(KafkaStreamSourceService.TOPIC_FLOWLAST);
|
||||
|
||||
stream = kafkaStreamService.of(FlowListener.class, FlowListener.class, new FlowListener().topology(), log);
|
||||
stream.start((newState, oldState) -> {
|
||||
if (newState == KafkaStreams.State.RUNNING) {
|
||||
try {
|
||||
ReadOnlyKeyValueStore<String, Flow> store = stream.store(StoreQueryParameters.fromNameAndType(
|
||||
"flow",
|
||||
QueryableStoreTypes.keyValueStore()
|
||||
));
|
||||
|
||||
this.store = new SafeKeyValueStore<>(store, "flow");
|
||||
this.send(this.flows());
|
||||
} catch (InvalidStateStoreException e) {
|
||||
this.store = null;
|
||||
log.warn(e.getMessage(), e);
|
||||
}
|
||||
} else {
|
||||
synchronized (this) {
|
||||
flows = null;
|
||||
}
|
||||
this.send(new ArrayList<>());
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
public class FlowListener {
|
||||
public Topology topology() {
|
||||
StreamsBuilder builder = new KafkaStreamsBuilder();
|
||||
|
||||
builder
|
||||
.table(
|
||||
kafkaAdminService.getTopicName(KafkaStreamSourceService.TOPIC_FLOWLAST),
|
||||
Consumed.with(Serdes.String(), JsonSerde.of(Flow.class, false)),
|
||||
Materialized.<String, Flow, KeyValueStore<Bytes, byte[]>>as("flow")
|
||||
.withKeySerde(Serdes.String())
|
||||
.withValueSerde(JsonSerde.of(Flow.class, false))
|
||||
)
|
||||
.filter((key, value) -> value != null)
|
||||
.toStream()
|
||||
.peek((key, value) -> {
|
||||
synchronized (this) {
|
||||
flows = null;
|
||||
}
|
||||
|
||||
send(flows());
|
||||
});
|
||||
|
||||
Topology topology = builder.build();
|
||||
|
||||
if (log.isTraceEnabled()) {
|
||||
log.trace(topology.describe().toString());
|
||||
}
|
||||
|
||||
return topology;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<Flow> flows() {
|
||||
if (this.store == null || stream.state() != KafkaStreams.State.RUNNING) {
|
||||
return Collections.emptyList();
|
||||
}
|
||||
|
||||
synchronized (this) {
|
||||
if (this.flows == null) {
|
||||
this.flows = this.store
|
||||
.all()
|
||||
.filter(flow -> flow != null && !flow.isDeleted())
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
}
|
||||
|
||||
//noinspection ConstantConditions
|
||||
return this.flows == null ? new ArrayList<>() : this.flows;
|
||||
}
|
||||
|
||||
private void send(List<Flow> flows) {
|
||||
this.consumers
|
||||
.forEach(consumer -> consumer.accept(flows));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void listen(Consumer<List<Flow>> consumer) {
|
||||
consumers.add(consumer);
|
||||
consumer.accept(this.flows());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void listen(BiConsumer<Flow, Flow> consumer) {
|
||||
throw new NotImplementedException("Kafka runner don't need to listen on each flow changed since it's handle by stream");
|
||||
}
|
||||
}
|
||||
@@ -1,73 +0,0 @@
|
||||
package io.kestra.runner.kafka;
|
||||
|
||||
import io.micronaut.core.util.CollectionUtils;
|
||||
import io.micronaut.health.HealthStatus;
|
||||
import io.micronaut.management.health.indicator.HealthIndicator;
|
||||
import io.micronaut.management.health.indicator.HealthResult;
|
||||
import io.reactivex.Flowable;
|
||||
import org.apache.kafka.clients.admin.*;
|
||||
import org.apache.kafka.common.Node;
|
||||
import org.apache.kafka.common.config.ConfigResource;
|
||||
import io.kestra.runner.kafka.services.KafkaAdminService;
|
||||
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.Map;
|
||||
import jakarta.inject.Singleton;
|
||||
|
||||
@Singleton
|
||||
|
||||
@KafkaQueueEnabled
|
||||
public class KafkaHealthIndicator implements HealthIndicator {
|
||||
private static final String ID = "kafka";
|
||||
private static final String REPLICATION_PROPERTY = "offsets.topic.replication.factor";
|
||||
private final AdminClient adminClient;
|
||||
|
||||
/**
|
||||
* Constructs a new Kafka health indicator for the given arguments.
|
||||
*
|
||||
* @param kafkaAdminService The admin service
|
||||
*/
|
||||
public KafkaHealthIndicator(KafkaAdminService kafkaAdminService) {
|
||||
this.adminClient = kafkaAdminService.of();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Flowable<HealthResult> getResult() {
|
||||
DescribeClusterResult result = adminClient.describeCluster();
|
||||
|
||||
Flowable<String> clusterId = Flowable.fromFuture(result.clusterId());
|
||||
Flowable<Collection<Node>> nodes = Flowable.fromFuture(result.nodes());
|
||||
Flowable<Node> controller = Flowable.fromFuture(result.controller());
|
||||
|
||||
return controller.switchMap(node -> {
|
||||
String brokerId = node.idString();
|
||||
ConfigResource configResource = new ConfigResource(ConfigResource.Type.BROKER, brokerId);
|
||||
DescribeConfigsResult configResult = adminClient.describeConfigs(Collections.singletonList(configResource));
|
||||
Flowable<Map<ConfigResource, Config>> configs = Flowable.fromFuture(configResult.all());
|
||||
return configs.switchMap(resources -> {
|
||||
Config config = resources.get(configResource);
|
||||
ConfigEntry ce = config.get(REPLICATION_PROPERTY);
|
||||
int replicationFactor = Integer.parseInt(ce.value());
|
||||
return nodes.switchMap(nodeList -> clusterId.map(clusterIdString -> {
|
||||
int nodeCount = nodeList.size();
|
||||
HealthResult.Builder builder;
|
||||
if (nodeCount >= replicationFactor) {
|
||||
builder = HealthResult.builder(ID, HealthStatus.UP);
|
||||
} else {
|
||||
builder = HealthResult.builder(ID, HealthStatus.DOWN);
|
||||
}
|
||||
return builder
|
||||
.details(CollectionUtils.mapOf(
|
||||
"brokerId", brokerId,
|
||||
"clusterId", clusterIdString,
|
||||
"nodes", nodeCount
|
||||
)).build();
|
||||
}));
|
||||
});
|
||||
}).onErrorReturn(throwable ->
|
||||
HealthResult.builder(ID, HealthStatus.DOWN)
|
||||
.exception(throwable).build()
|
||||
);
|
||||
}
|
||||
}
|
||||
@@ -1,39 +0,0 @@
|
||||
package io.kestra.runner.kafka;
|
||||
|
||||
import com.google.common.collect.Streams;
|
||||
import org.apache.kafka.streams.state.KeyValueIterator;
|
||||
import org.apache.kafka.streams.state.KeyValueStore;
|
||||
import io.kestra.core.models.flows.Flow;
|
||||
import io.kestra.core.models.triggers.multipleflows.MultipleConditionStorageInterface;
|
||||
import io.kestra.core.models.triggers.multipleflows.MultipleConditionWindow;
|
||||
|
||||
import java.time.ZonedDateTime;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
public class KafkaMultipleConditionStorage implements MultipleConditionStorageInterface {
|
||||
private final KeyValueStore<String, MultipleConditionWindow> store;
|
||||
|
||||
public KafkaMultipleConditionStorage(KeyValueStore<String, MultipleConditionWindow> store) {
|
||||
this.store = store;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<MultipleConditionWindow> get(Flow flow, String conditionId) {
|
||||
return Optional.ofNullable(this.store.get(MultipleConditionWindow.uid(flow, conditionId)));
|
||||
}
|
||||
|
||||
@SuppressWarnings("UnstableApiUsage")
|
||||
@Override
|
||||
public List<MultipleConditionWindow> expired() {
|
||||
ZonedDateTime now = ZonedDateTime.now();
|
||||
|
||||
try (KeyValueIterator<String, MultipleConditionWindow> all = this.store.all()) {
|
||||
return Streams.stream(all)
|
||||
.map(e -> e.value)
|
||||
.filter(e -> !e.isValid(now))
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -1,294 +0,0 @@
|
||||
package io.kestra.runner.kafka;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import io.kestra.core.queues.QueueException;
|
||||
import io.kestra.core.queues.QueueInterface;
|
||||
import io.kestra.core.queues.QueueService;
|
||||
import io.kestra.core.utils.ExecutorsUtils;
|
||||
import io.kestra.core.utils.RetryUtils;
|
||||
import io.kestra.runner.kafka.configs.TopicsConfig;
|
||||
import io.kestra.runner.kafka.serializers.JsonSerde;
|
||||
import io.kestra.runner.kafka.services.KafkaAdminService;
|
||||
import io.kestra.runner.kafka.services.KafkaConsumerService;
|
||||
import io.kestra.runner.kafka.services.KafkaProducerService;
|
||||
import io.micronaut.context.ApplicationContext;
|
||||
import lombok.extern.slf4j.Slf4j;
|
||||
import org.apache.kafka.clients.admin.AdminClient;
|
||||
import org.apache.kafka.clients.consumer.ConsumerRecords;
|
||||
import org.apache.kafka.clients.consumer.OffsetAndMetadata;
|
||||
import org.apache.kafka.clients.producer.KafkaProducer;
|
||||
import org.apache.kafka.clients.producer.ProducerRecord;
|
||||
import org.apache.kafka.clients.producer.RecordMetadata;
|
||||
import org.apache.kafka.common.TopicPartition;
|
||||
import org.apache.kafka.common.errors.TimeoutException;
|
||||
import org.apache.kafka.common.errors.WakeupException;
|
||||
|
||||
import java.time.Duration;
|
||||
import java.time.Instant;
|
||||
import java.util.*;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Future;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.function.Consumer;
|
||||
import java.util.stream.Collectors;
|
||||
import javax.annotation.PreDestroy;
|
||||
|
||||
@Slf4j
|
||||
public class KafkaQueue<T> implements QueueInterface<T>, AutoCloseable {
|
||||
private Class<T> cls;
|
||||
private final AdminClient adminClient;
|
||||
private final KafkaConsumerService kafkaConsumerService;
|
||||
private final List<org.apache.kafka.clients.consumer.Consumer<String, T>> kafkaConsumers = Collections.synchronizedList(new ArrayList<>());
|
||||
private final QueueService queueService;
|
||||
private final RetryUtils retryUtils;
|
||||
|
||||
private static ExecutorService poolExecutor;
|
||||
|
||||
private KafkaProducer<String, T> kafkaProducer;
|
||||
private TopicsConfig topicsConfig;
|
||||
|
||||
private KafkaQueue(ApplicationContext applicationContext) {
|
||||
if (poolExecutor == null) {
|
||||
ExecutorsUtils executorsUtils = applicationContext.getBean(ExecutorsUtils.class);
|
||||
poolExecutor = executorsUtils.cachedThreadPool("kakfa-queue");
|
||||
}
|
||||
|
||||
KafkaAdminService kafkaAdminService = applicationContext.getBean(KafkaAdminService.class);
|
||||
|
||||
this.adminClient = kafkaAdminService.of();
|
||||
this.kafkaConsumerService = applicationContext.getBean(KafkaConsumerService.class);
|
||||
this.queueService = applicationContext.getBean(QueueService.class);
|
||||
this.retryUtils = applicationContext.getBean(RetryUtils.class);
|
||||
}
|
||||
|
||||
public KafkaQueue(Class<T> cls, ApplicationContext applicationContext) {
|
||||
this(applicationContext);
|
||||
|
||||
this.cls = cls;
|
||||
this.kafkaProducer = applicationContext.getBean(KafkaProducerService.class).of(cls, JsonSerde.of(cls));
|
||||
this.topicsConfig = topicsConfig(applicationContext, this.cls);
|
||||
|
||||
KafkaAdminService kafkaAdminService = applicationContext.getBean(KafkaAdminService.class);
|
||||
kafkaAdminService.createIfNotExist(this.cls);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public KafkaQueue(String topicKey, Class<T> cls, ApplicationContext applicationContext) {
|
||||
this(applicationContext);
|
||||
|
||||
this.cls = cls;
|
||||
this.kafkaProducer = applicationContext.getBean(KafkaProducerService.class).of(cls, JsonSerde.of(cls));
|
||||
this.topicsConfig = topicsConfig(applicationContext, topicKey);
|
||||
|
||||
KafkaAdminService kafkaAdminService = applicationContext.getBean(KafkaAdminService.class);
|
||||
kafkaAdminService.createIfNotExist(topicKey);
|
||||
}
|
||||
|
||||
private Future<RecordMetadata> produceAsync(String key, T message) {
|
||||
return kafkaProducer
|
||||
.send(
|
||||
new ProducerRecord<>(
|
||||
topicsConfig.getName(),
|
||||
key, message
|
||||
),
|
||||
(metadata, e) -> {
|
||||
if (e != null) {
|
||||
log.error("Failed to produce on '{}' with key '{}', metadata '{}' ", this.cls, key, metadata, e);
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
private void produce(String key, T message) {
|
||||
try {
|
||||
this.produceAsync(key, message).get();
|
||||
} catch (InterruptedException | ExecutionException e) {
|
||||
throw new QueueException("Failed to produce on '" + this.cls + "' with key '" + key + "': ", e);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void emit(T message) throws QueueException {
|
||||
this.produce(this.queueService.key(message), message);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void emitAsync(T message) throws QueueException {
|
||||
this.produceAsync(this.queueService.key(message), message);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void delete(T message) throws QueueException {
|
||||
this.produce(this.queueService.key(message), null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Runnable receive(Consumer<T> consumer) {
|
||||
return this.receive(null, consumer);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Runnable receive(Class<?> consumerGroup, Consumer<T> consumer) {
|
||||
AtomicBoolean running = new AtomicBoolean(true);
|
||||
|
||||
// no consumer groups, we fetch actual offset and block until the response id ready
|
||||
// we need to be sure to get from actual time, so consume last 1 sec must enough
|
||||
Map<TopicPartition, Long> offsets = null;
|
||||
if (consumerGroup == null) {
|
||||
offsets = this.offsetForTime(Instant.now().minus(Duration.ofSeconds(1)));
|
||||
}
|
||||
Map<TopicPartition, Long> finalOffsets = offsets;
|
||||
|
||||
poolExecutor.execute(() -> {
|
||||
org.apache.kafka.clients.consumer.Consumer<String, T> kafkaConsumer = kafkaConsumerService.of(
|
||||
this.cls,
|
||||
JsonSerde.of(this.cls),
|
||||
consumerGroup
|
||||
);
|
||||
|
||||
kafkaConsumers.add(kafkaConsumer);
|
||||
|
||||
if (consumerGroup != null) {
|
||||
kafkaConsumer.subscribe(Collections.singleton(topicsConfig.getName()));
|
||||
} else {
|
||||
kafkaConsumer.assign(new ArrayList<>(finalOffsets.keySet()));
|
||||
finalOffsets.forEach(kafkaConsumer::seek);
|
||||
}
|
||||
|
||||
while (running.get()) {
|
||||
try {
|
||||
ConsumerRecords<String, T> records = kafkaConsumer.poll(Duration.ofMillis(500));
|
||||
|
||||
records.forEach(record -> {
|
||||
consumer.accept(record.value());
|
||||
|
||||
if (consumerGroup != null) {
|
||||
kafkaConsumer.commitSync(
|
||||
ImmutableMap.of(
|
||||
new TopicPartition(record.topic(), record.partition()),
|
||||
new OffsetAndMetadata(record.offset() + 1)
|
||||
)
|
||||
);
|
||||
}
|
||||
});
|
||||
} catch (WakeupException e) {
|
||||
log.debug("Received Wakeup on {} with type {}!", this.getClass().getName(), this.cls.getName());
|
||||
|
||||
// first call, we want to shutdown, so pause the consumer, will be closed after properly on second call
|
||||
if (kafkaConsumer.paused().size() == 0) {
|
||||
kafkaConsumer.pause(kafkaConsumer.assignment());
|
||||
} else {
|
||||
running.set(false);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
kafkaConsumers.remove(kafkaConsumer);
|
||||
kafkaConsumer.close();
|
||||
});
|
||||
|
||||
return () -> {
|
||||
running.set(false);
|
||||
};
|
||||
}
|
||||
|
||||
static TopicsConfig topicsConfigByTopicName(ApplicationContext applicationContext, String topicName) {
|
||||
return applicationContext
|
||||
.getBeansOfType(TopicsConfig.class)
|
||||
.stream()
|
||||
.filter(r -> r.getName().equals(topicName))
|
||||
.findFirst()
|
||||
.orElseThrow(() -> new IllegalArgumentException("Invalid topic name '" + topicName + "'"));
|
||||
}
|
||||
|
||||
static TopicsConfig topicsConfig(ApplicationContext applicationContext, Class<?> cls) {
|
||||
return applicationContext
|
||||
.getBeansOfType(TopicsConfig.class)
|
||||
.stream()
|
||||
.filter(r -> r.getCls() == cls)
|
||||
.findFirst()
|
||||
.orElseThrow(() -> new IllegalArgumentException("Invalid topic class '" + cls.getName() + "'"));
|
||||
}
|
||||
|
||||
static TopicsConfig topicsConfig(ApplicationContext applicationContext, String name) {
|
||||
return applicationContext
|
||||
.getBeansOfType(TopicsConfig.class)
|
||||
.stream()
|
||||
.filter(r -> r.getKey().equals(name))
|
||||
.findFirst()
|
||||
.orElseThrow(() -> new IllegalArgumentException("Invalid topic key '" + name + "'"));
|
||||
}
|
||||
|
||||
private List<TopicPartition> listTopicPartition() throws ExecutionException, InterruptedException {
|
||||
return this.adminClient
|
||||
.describeTopics(Collections.singleton(topicsConfig.getName()))
|
||||
.allTopicNames()
|
||||
.get()
|
||||
.entrySet()
|
||||
.stream()
|
||||
.flatMap(e -> e.getValue().partitions()
|
||||
.stream()
|
||||
.map(i -> new TopicPartition(e.getValue().name(), i.partition()))
|
||||
)
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
|
||||
Map<TopicPartition, Long> offsetForTime(Instant instant) {
|
||||
org.apache.kafka.clients.consumer.Consumer<String, T> consumer = kafkaConsumerService.of(
|
||||
this.cls,
|
||||
JsonSerde.of(this.cls),
|
||||
null
|
||||
);
|
||||
|
||||
try {
|
||||
List<TopicPartition> topicPartitions = this.listTopicPartition();
|
||||
Map<TopicPartition, Long> result = retryUtils.<Map<TopicPartition, Long>, TimeoutException>of().run(
|
||||
TimeoutException.class,
|
||||
() -> consumer.endOffsets(topicPartitions)
|
||||
);
|
||||
|
||||
result.putAll(consumer
|
||||
.offsetsForTimes(
|
||||
topicPartitions
|
||||
.stream()
|
||||
.map(e -> new AbstractMap.SimpleEntry<>(
|
||||
e,
|
||||
instant.toEpochMilli()
|
||||
))
|
||||
.collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue))
|
||||
)
|
||||
.entrySet()
|
||||
.stream()
|
||||
.filter(e -> e.getValue() != null)
|
||||
.map(e -> new AbstractMap.SimpleEntry<>(
|
||||
e.getKey(),
|
||||
e.getValue().offset()
|
||||
))
|
||||
.collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue))
|
||||
);
|
||||
|
||||
return result;
|
||||
} catch (ExecutionException | InterruptedException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void pause() {
|
||||
this.wakeup();
|
||||
}
|
||||
|
||||
@PreDestroy
|
||||
@Override
|
||||
public void close() {
|
||||
kafkaProducer.close();
|
||||
this.wakeup();
|
||||
}
|
||||
|
||||
private void wakeup() {
|
||||
kafkaConsumers.forEach(org.apache.kafka.clients.consumer.Consumer::wakeup);
|
||||
}
|
||||
}
|
||||
@@ -1,12 +0,0 @@
|
||||
package io.kestra.runner.kafka;
|
||||
|
||||
import io.micronaut.context.annotation.Requires;
|
||||
|
||||
import java.lang.annotation.*;
|
||||
|
||||
@Documented
|
||||
@Retention(RetentionPolicy.RUNTIME)
|
||||
@Target({ElementType.PACKAGE, ElementType.TYPE})
|
||||
@Requires(property = "kestra.queue.type", value = "kafka")
|
||||
public @interface KafkaQueueEnabled {
|
||||
}
|
||||
@@ -1,108 +0,0 @@
|
||||
package io.kestra.runner.kafka;
|
||||
|
||||
import io.kestra.core.models.executions.Execution;
|
||||
import io.kestra.core.models.executions.ExecutionKilled;
|
||||
import io.kestra.core.models.executions.LogEntry;
|
||||
import io.kestra.core.models.flows.Flow;
|
||||
import io.kestra.core.models.templates.Template;
|
||||
import io.kestra.core.models.triggers.Trigger;
|
||||
import io.kestra.core.queues.QueueFactoryInterface;
|
||||
import io.kestra.core.queues.QueueInterface;
|
||||
import io.kestra.core.queues.WorkerTaskQueueInterface;
|
||||
import io.kestra.core.runners.*;
|
||||
import io.micronaut.context.ApplicationContext;
|
||||
import io.micronaut.context.annotation.Factory;
|
||||
|
||||
import jakarta.inject.Inject;
|
||||
import jakarta.inject.Named;
|
||||
import jakarta.inject.Singleton;
|
||||
|
||||
@Factory
|
||||
@KafkaQueueEnabled
|
||||
public class KafkaQueueFactory implements QueueFactoryInterface {
|
||||
@Inject
|
||||
ApplicationContext applicationContext;
|
||||
|
||||
@Override
|
||||
@Singleton
|
||||
@Named(QueueFactoryInterface.EXECUTION_NAMED)
|
||||
public QueueInterface<Execution> execution() {
|
||||
return new KafkaQueue<>(Execution.class, applicationContext);
|
||||
}
|
||||
|
||||
@Override
|
||||
@Singleton
|
||||
@Named(QueueFactoryInterface.EXECUTOR_NAMED)
|
||||
public QueueInterface<Executor> executor() {
|
||||
return new KafkaQueue<>(Executor.class, applicationContext);
|
||||
}
|
||||
|
||||
@Override
|
||||
@Singleton
|
||||
@Named(QueueFactoryInterface.WORKERTASK_NAMED)
|
||||
public QueueInterface<WorkerTask> workerTask() {
|
||||
return new KafkaQueue<>(WorkerTask.class, applicationContext);
|
||||
}
|
||||
|
||||
@Override
|
||||
@Singleton
|
||||
@Named(QueueFactoryInterface.WORKERTASKRESULT_NAMED)
|
||||
public QueueInterface<WorkerTaskResult> workerTaskResult() {
|
||||
return new KafkaQueue<>(WorkerTaskResult.class, applicationContext);
|
||||
}
|
||||
|
||||
@Override
|
||||
@Singleton
|
||||
@Named(QueueFactoryInterface.WORKERTASKLOG_NAMED)
|
||||
public QueueInterface<LogEntry> logEntry() {
|
||||
return new KafkaQueue<>(LogEntry.class, applicationContext);
|
||||
}
|
||||
|
||||
@Override
|
||||
@Singleton
|
||||
@Named(QueueFactoryInterface.FLOW_NAMED)
|
||||
public QueueInterface<Flow> flow() {
|
||||
return new KafkaQueue<>(Flow.class, applicationContext);
|
||||
}
|
||||
|
||||
@Override
|
||||
@Singleton
|
||||
@Named(QueueFactoryInterface.KILL_NAMED)
|
||||
public QueueInterface<ExecutionKilled> kill() {
|
||||
return new KafkaQueue<>(ExecutionKilled.class, applicationContext);
|
||||
}
|
||||
|
||||
@Override
|
||||
@Singleton
|
||||
@Named(QueueFactoryInterface.TEMPLATE_NAMED)
|
||||
public QueueInterface<Template> template() {
|
||||
return new KafkaQueue<>(Template.class, applicationContext);
|
||||
}
|
||||
|
||||
@Override
|
||||
@Singleton
|
||||
@Named(QueueFactoryInterface.WORKERINSTANCE_NAMED)
|
||||
public QueueInterface<WorkerInstance> workerInstance() {
|
||||
return new KafkaQueue<>(WorkerInstance.class, applicationContext);
|
||||
}
|
||||
|
||||
@Override
|
||||
@Singleton
|
||||
@Named(QueueFactoryInterface.WORKERTASKRUNNING_NAMED)
|
||||
public QueueInterface<WorkerTaskRunning> workerTaskRunning() {
|
||||
return new KafkaQueue<>(WorkerTaskRunning.class, applicationContext);
|
||||
}
|
||||
|
||||
@Override
|
||||
@Singleton
|
||||
@Named(QueueFactoryInterface.TRIGGER_NAMED)
|
||||
public QueueInterface<Trigger> trigger() {
|
||||
return new KafkaQueue<>(Trigger.class, applicationContext);
|
||||
}
|
||||
|
||||
@Override
|
||||
@Singleton
|
||||
public WorkerTaskQueueInterface workerTaskQueue() {
|
||||
return new KafkaWorkerTaskQueue(applicationContext);
|
||||
}
|
||||
}
|
||||
@@ -1,204 +0,0 @@
|
||||
package io.kestra.runner.kafka;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import io.kestra.core.models.executions.Execution;
|
||||
import io.kestra.core.models.flows.Flow;
|
||||
import io.kestra.core.models.triggers.Trigger;
|
||||
import io.kestra.core.queues.QueueFactoryInterface;
|
||||
import io.kestra.core.queues.QueueInterface;
|
||||
import io.kestra.core.queues.QueueService;
|
||||
import io.kestra.core.runners.Executor;
|
||||
import io.kestra.core.schedulers.AbstractScheduler;
|
||||
import io.kestra.core.schedulers.DefaultScheduler;
|
||||
import io.kestra.core.schedulers.SchedulerExecutionWithTrigger;
|
||||
import io.kestra.core.services.FlowListenersInterface;
|
||||
import io.kestra.core.utils.IdUtils;
|
||||
import io.kestra.runner.kafka.configs.TopicsConfig;
|
||||
import io.kestra.runner.kafka.serializers.JsonSerde;
|
||||
import io.kestra.runner.kafka.services.KafkaAdminService;
|
||||
import io.kestra.runner.kafka.services.KafkaProducerService;
|
||||
import io.kestra.runner.kafka.services.KafkaStreamService;
|
||||
import io.kestra.runner.kafka.services.KafkaStreamsBuilder;
|
||||
import io.kestra.runner.kafka.streams.GlobalStateLockProcessor;
|
||||
import io.kestra.runner.kafka.streams.GlobalStateProcessor;
|
||||
import io.micronaut.context.ApplicationContext;
|
||||
import io.micronaut.context.annotation.Replaces;
|
||||
import io.micronaut.inject.qualifiers.Qualifiers;
|
||||
import jakarta.inject.Singleton;
|
||||
import lombok.extern.slf4j.Slf4j;
|
||||
import org.apache.kafka.clients.producer.KafkaProducer;
|
||||
import org.apache.kafka.clients.producer.ProducerRecord;
|
||||
import org.apache.kafka.common.serialization.Serdes;
|
||||
import org.apache.kafka.streams.KafkaStreams;
|
||||
import org.apache.kafka.streams.StoreQueryParameters;
|
||||
import org.apache.kafka.streams.StreamsBuilder;
|
||||
import org.apache.kafka.streams.Topology;
|
||||
import org.apache.kafka.streams.kstream.Consumed;
|
||||
import org.apache.kafka.streams.state.QueryableStoreTypes;
|
||||
import org.apache.kafka.streams.state.Stores;
|
||||
|
||||
import java.time.Duration;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
|
||||
@KafkaQueueEnabled
|
||||
@Singleton
|
||||
@Slf4j
|
||||
@Replaces(DefaultScheduler.class)
|
||||
public class KafkaScheduler extends AbstractScheduler {
|
||||
private static final String STATESTORE_EXECUTOR = "schedulerexecutor";
|
||||
private static final String STATESTORE_TRIGGER = "schedulertrigger";
|
||||
|
||||
private final KafkaAdminService kafkaAdminService;
|
||||
private final KafkaStreamService kafkaStreamService;
|
||||
private final QueueInterface<Trigger> triggerQueue;
|
||||
private final QueueService queueService;
|
||||
private final KafkaProducer<String, Object> kafkaProducer;
|
||||
private final TopicsConfig topicsConfigTrigger;
|
||||
private final TopicsConfig topicsConfigExecution;
|
||||
|
||||
private final Map<String, Trigger> triggerLock = new ConcurrentHashMap<>();
|
||||
|
||||
protected KafkaStreamService.Stream stateStream;
|
||||
protected KafkaStreamService.Stream cleanTriggerStream;
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public KafkaScheduler(
|
||||
ApplicationContext applicationContext,
|
||||
FlowListenersInterface flowListenersService
|
||||
) {
|
||||
super(
|
||||
applicationContext,
|
||||
flowListenersService
|
||||
);
|
||||
|
||||
this.triggerQueue = applicationContext.getBean(QueueInterface.class, Qualifiers.byName(QueueFactoryInterface.TRIGGER_NAMED));
|
||||
this.kafkaAdminService = applicationContext.getBean(KafkaAdminService.class);
|
||||
this.kafkaStreamService = applicationContext.getBean(KafkaStreamService.class);
|
||||
this.queueService = applicationContext.getBean(QueueService.class);
|
||||
this.kafkaProducer = applicationContext.getBean(KafkaProducerService.class).of(
|
||||
KafkaScheduler.class,
|
||||
JsonSerde.of(Object.class),
|
||||
ImmutableMap.of("transactional.id", IdUtils.create())
|
||||
);
|
||||
this.topicsConfigTrigger = KafkaQueue.topicsConfig(applicationContext, Trigger.class);
|
||||
this.topicsConfigExecution = KafkaQueue.topicsConfig(applicationContext, Execution.class);
|
||||
|
||||
this.kafkaProducer.initTransactions();
|
||||
}
|
||||
|
||||
public class SchedulerState {
|
||||
public StreamsBuilder topology() {
|
||||
StreamsBuilder builder = new KafkaStreamsBuilder();
|
||||
|
||||
// executor global state store
|
||||
builder.addGlobalStore(
|
||||
Stores.keyValueStoreBuilder(
|
||||
Stores.persistentKeyValueStore(STATESTORE_EXECUTOR),
|
||||
Serdes.String(),
|
||||
JsonSerde.of(Executor.class)
|
||||
),
|
||||
kafkaAdminService.getTopicName(Executor.class),
|
||||
Consumed.with(Serdes.String(), JsonSerde.of(Executor.class)).withName("GlobalStore.Executor"),
|
||||
() -> new GlobalStateProcessor<>(STATESTORE_EXECUTOR)
|
||||
);
|
||||
|
||||
// trigger global state store
|
||||
builder.addGlobalStore(
|
||||
Stores.keyValueStoreBuilder(
|
||||
Stores.persistentKeyValueStore(STATESTORE_TRIGGER),
|
||||
Serdes.String(),
|
||||
JsonSerde.of(Trigger.class)
|
||||
),
|
||||
kafkaAdminService.getTopicName(Trigger.class),
|
||||
Consumed.with(Serdes.String(), JsonSerde.of(Trigger.class)).withName("GlobalStore.Trigger"),
|
||||
() -> new GlobalStateLockProcessor<>(STATESTORE_TRIGGER, triggerLock)
|
||||
);
|
||||
|
||||
return builder;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* We saved the trigger in a local hash map that will be clean by {@link GlobalStateProcessor}.
|
||||
* The scheduler trust the STATESTORE_TRIGGER to know if a running execution exists. Since the store is filled async,
|
||||
* this can lead to empty trigger and launch of concurrent job.
|
||||
*
|
||||
* @param executionWithTrigger the execution trigger to save
|
||||
*/
|
||||
protected synchronized void saveLastTriggerAndEmitExecution(SchedulerExecutionWithTrigger executionWithTrigger) {
|
||||
Trigger trigger = Trigger.of(
|
||||
executionWithTrigger.getTriggerContext(),
|
||||
executionWithTrigger.getExecution()
|
||||
);
|
||||
|
||||
kafkaProducer.beginTransaction();
|
||||
|
||||
this.kafkaProducer.send(new ProducerRecord<>(
|
||||
topicsConfigTrigger.getName(),
|
||||
this.queueService.key(trigger),
|
||||
trigger
|
||||
));
|
||||
|
||||
|
||||
this.kafkaProducer.send(new ProducerRecord<>(
|
||||
topicsConfigExecution.getName(),
|
||||
this.queueService.key(executionWithTrigger.getExecution()),
|
||||
executionWithTrigger.getExecution()
|
||||
));
|
||||
|
||||
this.triggerLock.put(trigger.uid(), trigger);
|
||||
|
||||
kafkaProducer.commitTransaction();
|
||||
}
|
||||
|
||||
protected KafkaStreamService.Stream init(Class<?> group, StreamsBuilder builder) {
|
||||
Topology topology = builder.build();
|
||||
|
||||
if (log.isTraceEnabled()) {
|
||||
log.trace(topology.describe().toString());
|
||||
}
|
||||
|
||||
return kafkaStreamService.of(this.getClass(), group, topology);
|
||||
}
|
||||
|
||||
public void initStream() {
|
||||
kafkaAdminService.createIfNotExist(Flow.class);
|
||||
kafkaAdminService.createIfNotExist(Executor.class);
|
||||
kafkaAdminService.createIfNotExist(Trigger.class);
|
||||
|
||||
this.stateStream = this.init(SchedulerState.class, new SchedulerState().topology());
|
||||
this.stateStream.start((newState, oldState) -> {
|
||||
this.isReady = newState == KafkaStreams.State.RUNNING;
|
||||
});
|
||||
|
||||
this.triggerState = new KafkaSchedulerTriggerState(
|
||||
stateStream.store(StoreQueryParameters.fromNameAndType(STATESTORE_TRIGGER, QueryableStoreTypes.keyValueStore())),
|
||||
triggerQueue,
|
||||
triggerLock
|
||||
);
|
||||
|
||||
this.executionState = new KafkaSchedulerExecutionState(
|
||||
stateStream.store(StoreQueryParameters.fromNameAndType(STATESTORE_EXECUTOR, QueryableStoreTypes.keyValueStore()))
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
this.initStream();
|
||||
super.run();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
if (stateStream != null) {
|
||||
stateStream.close(Duration.ofSeconds(10));
|
||||
}
|
||||
|
||||
if (cleanTriggerStream != null) {
|
||||
cleanTriggerStream.close(Duration.ofSeconds(10));
|
||||
}
|
||||
|
||||
super.close();
|
||||
}
|
||||
}
|
||||
@@ -1,29 +0,0 @@
|
||||
package io.kestra.runner.kafka;
|
||||
|
||||
import io.kestra.core.runners.Executor;
|
||||
import io.kestra.core.schedulers.SchedulerExecutionState;
|
||||
import io.micronaut.context.annotation.Replaces;
|
||||
import lombok.extern.slf4j.Slf4j;
|
||||
import org.apache.kafka.streams.state.ReadOnlyKeyValueStore;
|
||||
import io.kestra.core.models.executions.Execution;
|
||||
import io.kestra.core.schedulers.SchedulerExecutionStateInterface;
|
||||
|
||||
import java.util.Optional;
|
||||
import jakarta.inject.Singleton;
|
||||
|
||||
@Slf4j
|
||||
@KafkaQueueEnabled
|
||||
@Singleton
|
||||
@Replaces(SchedulerExecutionState.class)
|
||||
public class KafkaSchedulerExecutionState implements SchedulerExecutionStateInterface {
|
||||
private final ReadOnlyKeyValueStore<String, Executor> store;
|
||||
|
||||
public KafkaSchedulerExecutionState(ReadOnlyKeyValueStore<String, Executor> store) {
|
||||
this.store = store;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<Execution> findById(String id) {
|
||||
return Optional.ofNullable(this.store.get(id)).map(Executor::getExecution);
|
||||
}
|
||||
}
|
||||
@@ -1,47 +0,0 @@
|
||||
package io.kestra.runner.kafka;
|
||||
|
||||
|
||||
import lombok.extern.slf4j.Slf4j;
|
||||
import org.apache.kafka.streams.state.ReadOnlyKeyValueStore;
|
||||
import io.kestra.core.models.triggers.Trigger;
|
||||
import io.kestra.core.models.triggers.TriggerContext;
|
||||
import io.kestra.core.queues.QueueInterface;
|
||||
import io.kestra.core.schedulers.SchedulerTriggerStateInterface;
|
||||
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import jakarta.inject.Singleton;
|
||||
import javax.validation.ConstraintViolationException;
|
||||
|
||||
@Slf4j
|
||||
@KafkaQueueEnabled
|
||||
@Singleton
|
||||
public class KafkaSchedulerTriggerState implements SchedulerTriggerStateInterface {
|
||||
private final ReadOnlyKeyValueStore<String, Trigger> store;
|
||||
private final QueueInterface<Trigger> triggerQueue;
|
||||
private final Map<String, Trigger> triggerLock;
|
||||
|
||||
public KafkaSchedulerTriggerState(
|
||||
ReadOnlyKeyValueStore<String, Trigger> store,
|
||||
QueueInterface<Trigger> triggerQueue,
|
||||
Map<String, Trigger> triggerLock
|
||||
) {
|
||||
this.store = store;
|
||||
this.triggerQueue = triggerQueue;
|
||||
this.triggerLock = triggerLock;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<Trigger> findLast(TriggerContext trigger) {
|
||||
return Optional
|
||||
.ofNullable(this.triggerLock.getOrDefault(trigger.uid(), null))
|
||||
.or(() -> Optional.ofNullable(this.store.get(trigger.uid())));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Trigger save(Trigger trigger) throws ConstraintViolationException {
|
||||
triggerQueue.emit(trigger);
|
||||
|
||||
return trigger;
|
||||
}
|
||||
}
|
||||
@@ -1,170 +0,0 @@
|
||||
package io.kestra.runner.kafka;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonInclude;
|
||||
import io.kestra.runner.kafka.services.KafkaStreamService;
|
||||
import io.micronaut.context.annotation.Requires;
|
||||
import io.micronaut.context.event.ApplicationEventListener;
|
||||
import io.micronaut.management.endpoint.annotation.Endpoint;
|
||||
import io.micronaut.management.endpoint.annotation.Read;
|
||||
import io.micronaut.management.endpoint.annotation.Selector;
|
||||
import lombok.AllArgsConstructor;
|
||||
import lombok.Builder;
|
||||
import lombok.Getter;
|
||||
import org.apache.kafka.streams.KafkaStreams;
|
||||
|
||||
import java.util.AbstractMap;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
@Endpoint(id = "kafkastreams", defaultSensitive = false)
|
||||
@Requires(property = "kestra.server-type", pattern = "(EXECUTOR|STANDALONE|SCHEDULER)")
|
||||
@KafkaQueueEnabled
|
||||
public class KafkaStreamEndpoint implements ApplicationEventListener<KafkaStreamEndpoint.Event> {
|
||||
private Map<String, KafkaStreamService.Stream> streams;
|
||||
|
||||
@Override
|
||||
public void onApplicationEvent(KafkaStreamEndpoint.Event event) {
|
||||
if (streams == null) {
|
||||
streams = new HashMap<>();
|
||||
}
|
||||
|
||||
streams.put(event.getClientId(), event.getStream());
|
||||
}
|
||||
|
||||
@Read
|
||||
public List<KafkaStream> global() {
|
||||
return (streams != null ? streams : new HashMap<String, KafkaStreamService.Stream>())
|
||||
.entrySet()
|
||||
.stream()
|
||||
.map(entry -> KafkaStream.of(entry.getKey(), entry.getValue()))
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
|
||||
@Read
|
||||
public Object detail(@Selector String clientId, @Selector String type) {
|
||||
if (!streams.containsKey(clientId)) {
|
||||
throw new IllegalArgumentException("Invalid clientId with name '" + clientId + "'");
|
||||
}
|
||||
|
||||
KafkaStreamService.Stream stream = this.streams.get(clientId);
|
||||
|
||||
switch (type) {
|
||||
case "lag":
|
||||
return stream
|
||||
.allLocalStorePartitionLags()
|
||||
.entrySet()
|
||||
.stream()
|
||||
.flatMap(e -> e.getValue()
|
||||
.entrySet()
|
||||
.stream()
|
||||
.map(f -> LagInfo.of(
|
||||
e.getKey(),
|
||||
f.getKey(),
|
||||
f.getValue()
|
||||
))
|
||||
)
|
||||
.collect(Collectors.toList());
|
||||
case "metrics":
|
||||
return stream
|
||||
.metrics()
|
||||
.values()
|
||||
.stream()
|
||||
.map(Metric::of)
|
||||
.collect(Collectors.toList());
|
||||
default:
|
||||
throw new IllegalArgumentException("Invalid type '" + type + "'");
|
||||
}
|
||||
}
|
||||
|
||||
@Getter
|
||||
@Builder
|
||||
@JsonInclude
|
||||
public static class KafkaStream {
|
||||
Boolean ready;
|
||||
String clientId;
|
||||
KafkaStreams.State state;
|
||||
@JsonInclude
|
||||
Map<String, Long> storeLags;
|
||||
|
||||
public static KafkaStream of(String clientId, KafkaStreamService.Stream stream) {
|
||||
return KafkaStream.builder()
|
||||
.ready(true)
|
||||
.clientId(clientId)
|
||||
.state(stream.state())
|
||||
.storeLags(stream
|
||||
.allLocalStorePartitionLags()
|
||||
.entrySet()
|
||||
.stream()
|
||||
.flatMap(e -> e.getValue()
|
||||
.entrySet()
|
||||
.stream()
|
||||
.map(f -> LagInfo.of(
|
||||
e.getKey(),
|
||||
f.getKey(),
|
||||
f.getValue()
|
||||
))
|
||||
)
|
||||
.collect(Collectors.groupingBy(LagInfo::getStore, Collectors.toList()))
|
||||
.entrySet()
|
||||
.stream()
|
||||
.map(entry -> new AbstractMap.SimpleEntry<>(
|
||||
entry.getKey(),
|
||||
entry.getValue()
|
||||
.stream()
|
||||
.mapToLong(LagInfo::getOffsetLag)
|
||||
.sum()
|
||||
))
|
||||
.collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue))
|
||||
)
|
||||
.build();
|
||||
}
|
||||
}
|
||||
|
||||
@Getter
|
||||
@Builder
|
||||
@JsonInclude
|
||||
public static class LagInfo {
|
||||
private String store;
|
||||
private Integer partition;
|
||||
private Long currentOffsetPosition;
|
||||
private Long endOffsetPosition;
|
||||
private Long offsetLag;
|
||||
|
||||
public static LagInfo of(String topic, Integer partition, org.apache.kafka.streams.LagInfo lagInfo) {
|
||||
return LagInfo.builder()
|
||||
.store(topic)
|
||||
.partition(partition)
|
||||
.currentOffsetPosition(lagInfo.currentOffsetPosition())
|
||||
.endOffsetPosition(lagInfo.endOffsetPosition())
|
||||
.offsetLag(lagInfo.offsetLag())
|
||||
.build();
|
||||
}
|
||||
}
|
||||
|
||||
@Getter
|
||||
@Builder
|
||||
public static class Metric {
|
||||
private String group;
|
||||
private String name;
|
||||
private Map<String, String> tags;
|
||||
private Object value;
|
||||
|
||||
public static <T extends org.apache.kafka.common.Metric> Metric of(T metric) {
|
||||
return Metric.builder()
|
||||
.group(metric.metricName().group())
|
||||
.name(metric.metricName().name())
|
||||
.tags(metric.metricName().tags())
|
||||
.value(metric.metricValue())
|
||||
.build();
|
||||
}
|
||||
}
|
||||
|
||||
@Getter
|
||||
@AllArgsConstructor
|
||||
public static class Event {
|
||||
String clientId;
|
||||
KafkaStreamService.Stream stream;
|
||||
}
|
||||
}
|
||||
@@ -1,49 +0,0 @@
|
||||
package io.kestra.runner.kafka;
|
||||
|
||||
import io.kestra.core.models.templates.Template;
|
||||
import io.kestra.core.utils.Await;
|
||||
import io.micronaut.context.annotation.Replaces;
|
||||
import io.micronaut.context.annotation.Requires;
|
||||
import jakarta.inject.Singleton;
|
||||
import lombok.SneakyThrows;
|
||||
import lombok.extern.slf4j.Slf4j;
|
||||
|
||||
import java.time.Duration;
|
||||
import java.util.AbstractMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
@Slf4j
|
||||
@KafkaQueueEnabled
|
||||
@Replaces(io.kestra.core.tasks.flows.Template.MemoryTemplateExecutor.class)
|
||||
@Requires(property = "kestra.server-type", pattern = "(EXECUTOR|STANDALONE)")
|
||||
@Singleton
|
||||
public class KafkaTemplateExecutor implements io.kestra.core.tasks.flows.Template.TemplateExecutorInterface {
|
||||
private Map<String, Template> templates;
|
||||
|
||||
|
||||
public synchronized void setTemplates(List<Template> templates) {
|
||||
this.templates = templates
|
||||
.stream()
|
||||
.map(template -> new AbstractMap.SimpleEntry<>(
|
||||
template.uid(),
|
||||
template
|
||||
))
|
||||
.collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
|
||||
}
|
||||
|
||||
@SneakyThrows
|
||||
private void await() {
|
||||
if (templates == null) {
|
||||
Await.until(() -> this.templates != null, Duration.ofMillis(100), Duration.ofMinutes(5));
|
||||
}
|
||||
}
|
||||
|
||||
public Optional<Template> findById(String namespace, String templateId) {
|
||||
this.await();
|
||||
|
||||
return Optional.ofNullable(this.templates.get(Template.uid(namespace, templateId)));
|
||||
}
|
||||
}
|
||||
@@ -1,201 +0,0 @@
|
||||
package io.kestra.runner.kafka;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import io.micronaut.context.ApplicationContext;
|
||||
import io.micronaut.inject.qualifiers.Qualifiers;
|
||||
import lombok.SneakyThrows;
|
||||
import lombok.extern.slf4j.Slf4j;
|
||||
import org.apache.kafka.clients.consumer.ConsumerGroupMetadata;
|
||||
import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
|
||||
import org.apache.kafka.clients.consumer.ConsumerRecords;
|
||||
import org.apache.kafka.clients.producer.KafkaProducer;
|
||||
import org.apache.kafka.clients.producer.ProducerRecord;
|
||||
import org.apache.kafka.common.TopicPartition;
|
||||
import org.apache.kafka.common.errors.WakeupException;
|
||||
import io.kestra.core.queues.QueueFactoryInterface;
|
||||
import io.kestra.core.queues.QueueInterface;
|
||||
import io.kestra.core.queues.QueueService;
|
||||
import io.kestra.core.queues.WorkerTaskQueueInterface;
|
||||
import io.kestra.core.runners.WorkerInstance;
|
||||
import io.kestra.core.runners.WorkerTask;
|
||||
import io.kestra.core.runners.WorkerTaskRunning;
|
||||
import io.kestra.core.utils.Await;
|
||||
import io.kestra.core.utils.ExecutorsUtils;
|
||||
import io.kestra.runner.kafka.configs.TopicsConfig;
|
||||
import io.kestra.runner.kafka.serializers.JsonSerde;
|
||||
import io.kestra.runner.kafka.services.KafkaAdminService;
|
||||
import io.kestra.runner.kafka.services.KafkaConfigService;
|
||||
import io.kestra.runner.kafka.services.KafkaConsumerService;
|
||||
import io.kestra.runner.kafka.services.KafkaProducerService;
|
||||
|
||||
import java.net.InetAddress;
|
||||
import java.time.Duration;
|
||||
import java.util.*;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import java.util.function.Consumer;
|
||||
import java.util.stream.Collectors;
|
||||
import jakarta.inject.Singleton;
|
||||
|
||||
@Slf4j
|
||||
@Singleton
|
||||
public class KafkaWorkerTaskQueue implements WorkerTaskQueueInterface {
|
||||
private final QueueInterface<WorkerInstance> workerInstanceQueue;
|
||||
private final TopicsConfig topicsConfigWorkerTask;
|
||||
private final TopicsConfig topicsConfigWorkerTaskRunning;
|
||||
private final KafkaProducer<String, WorkerTaskRunning> kafkaProducer;
|
||||
private final KafkaConsumerService kafkaConsumerService;
|
||||
private final KafkaConfigService kafkaConfigService;
|
||||
private final QueueService queueService;
|
||||
private final AtomicReference<WorkerInstance> workerInstance = new AtomicReference<>();
|
||||
private final UUID workerUuid;
|
||||
|
||||
private static ExecutorService poolExecutor;
|
||||
private final List<org.apache.kafka.clients.consumer.Consumer<String, WorkerTask>> kafkaConsumers = Collections.synchronizedList(new ArrayList<>());
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public KafkaWorkerTaskQueue(ApplicationContext applicationContext) {
|
||||
if (poolExecutor == null) {
|
||||
ExecutorsUtils executorsUtils = applicationContext.getBean(ExecutorsUtils.class);
|
||||
poolExecutor = executorsUtils.cachedThreadPool("kakfa-workertask-queue");
|
||||
}
|
||||
|
||||
this.workerUuid = UUID.randomUUID();
|
||||
this.kafkaProducer = applicationContext.getBean(KafkaProducerService.class).of(
|
||||
WorkerTaskRunning.class,
|
||||
JsonSerde.of(WorkerTaskRunning.class),
|
||||
ImmutableMap.of("transactional.id", this.workerUuid.toString())
|
||||
);
|
||||
this.topicsConfigWorkerTask = KafkaQueue.topicsConfig(applicationContext, WorkerTask.class);
|
||||
this.topicsConfigWorkerTaskRunning = KafkaQueue.topicsConfig(applicationContext, WorkerTaskRunning.class);
|
||||
this.kafkaConsumerService = applicationContext.getBean(KafkaConsumerService.class);
|
||||
this.kafkaConfigService = applicationContext.getBean(KafkaConfigService.class);
|
||||
this.queueService = applicationContext.getBean(QueueService.class);
|
||||
this.workerInstanceQueue = (QueueInterface<WorkerInstance>) applicationContext.getBean(
|
||||
QueueInterface.class,
|
||||
Qualifiers.byName(QueueFactoryInterface.WORKERINSTANCE_NAMED)
|
||||
);
|
||||
KafkaAdminService kafkaAdminService = applicationContext.getBean(KafkaAdminService.class);
|
||||
kafkaAdminService.createIfNotExist(WorkerTask.class);
|
||||
}
|
||||
|
||||
public Runnable receive(Class<?> consumerGroup, Consumer<WorkerTask> consumer) {
|
||||
AtomicBoolean running = new AtomicBoolean(true);
|
||||
|
||||
poolExecutor.execute(() -> {
|
||||
kafkaProducer.initTransactions();
|
||||
|
||||
org.apache.kafka.clients.consumer.Consumer<String, WorkerTask> kafkaConsumer = kafkaConsumerService.of(
|
||||
KafkaWorkerTaskQueue.class,
|
||||
JsonSerde.of(WorkerTask.class),
|
||||
consumerRebalanceListener(),
|
||||
consumerGroup
|
||||
);
|
||||
|
||||
kafkaConsumers.add(kafkaConsumer);
|
||||
|
||||
kafkaConsumer.subscribe(Collections.singleton(topicsConfigWorkerTask.getName()));
|
||||
|
||||
while (running.get()) {
|
||||
try {
|
||||
ConsumerRecords<String, WorkerTask> records = kafkaConsumer.poll(Duration.ofMillis(500));
|
||||
|
||||
if (!records.isEmpty()) {
|
||||
kafkaProducer.beginTransaction();
|
||||
|
||||
records.forEach(record -> {
|
||||
if (workerInstance.get() == null) {
|
||||
Await.until(() -> workerInstance.get() != null);
|
||||
}
|
||||
|
||||
WorkerTaskRunning workerTaskRunning = WorkerTaskRunning.of(
|
||||
record.value(),
|
||||
workerInstance.get(),
|
||||
record.partition()
|
||||
);
|
||||
|
||||
this.kafkaProducer.send(new ProducerRecord<>(
|
||||
topicsConfigWorkerTaskRunning.getName(),
|
||||
this.queueService.key(workerTaskRunning),
|
||||
workerTaskRunning
|
||||
));
|
||||
});
|
||||
|
||||
// we commit first all offset before submit task to worker
|
||||
|
||||
kafkaProducer.sendOffsetsToTransaction(
|
||||
KafkaConsumerService.maxOffsets(records),
|
||||
new ConsumerGroupMetadata(kafkaConfigService.getConsumerGroupName(consumerGroup))
|
||||
);
|
||||
kafkaProducer.commitTransaction();
|
||||
|
||||
// now, we can submit to worker to be sure we don't have a WorkerTaskResult before commiting the offset!
|
||||
records.forEach(record -> {
|
||||
consumer.accept(record.value());
|
||||
});
|
||||
}
|
||||
} catch (WakeupException e) {
|
||||
log.debug("Received Wakeup on {}!", this.getClass().getName());
|
||||
|
||||
// first call, we want to shutdown, so pause the consumer, will be closed after properly on second call
|
||||
if (kafkaConsumer.paused().size() == 0) {
|
||||
kafkaConsumer.pause(kafkaConsumer.assignment());
|
||||
} else {
|
||||
running.set(false);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
kafkaConsumers.remove(kafkaConsumer);
|
||||
kafkaConsumer.close();
|
||||
});
|
||||
|
||||
return () -> running.set(false);
|
||||
}
|
||||
|
||||
private ConsumerRebalanceListener consumerRebalanceListener() {
|
||||
return new ConsumerRebalanceListener() {
|
||||
@Override
|
||||
public void onPartitionsRevoked(Collection<TopicPartition> partitions) {
|
||||
this.send(partitions);
|
||||
}
|
||||
|
||||
@SneakyThrows
|
||||
@Override
|
||||
public void onPartitionsAssigned(Collection<TopicPartition> partitions) {
|
||||
this.send(partitions);
|
||||
}
|
||||
|
||||
@SneakyThrows
|
||||
private void send(Collection<TopicPartition> partitions) {
|
||||
workerInstance.set(WorkerInstance
|
||||
.builder()
|
||||
.partitions(partitions
|
||||
.stream()
|
||||
.map(TopicPartition::partition)
|
||||
.collect(Collectors.toList())
|
||||
)
|
||||
.workerUuid(workerUuid)
|
||||
.hostname(InetAddress.getLocalHost().getHostName())
|
||||
.build()
|
||||
);
|
||||
|
||||
workerInstanceQueue.emit(workerInstance.get());
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
public void pause() {
|
||||
this.wakeup();
|
||||
}
|
||||
|
||||
public void close() {
|
||||
kafkaProducer.close();
|
||||
this.wakeup();
|
||||
}
|
||||
|
||||
private void wakeup() {
|
||||
kafkaConsumers.forEach(org.apache.kafka.clients.consumer.Consumer::wakeup);
|
||||
}
|
||||
}
|
||||
@@ -1,62 +0,0 @@
|
||||
package io.kestra.runner.kafka;
|
||||
|
||||
import io.kestra.runner.kafka.configs.LoggerConfig;
|
||||
import io.micronaut.core.annotation.Nullable;
|
||||
import lombok.NoArgsConstructor;
|
||||
import org.apache.kafka.clients.producer.ProducerRecord;
|
||||
import org.apache.kafka.clients.producer.RecordMetadata;
|
||||
import org.slf4j.event.Level;
|
||||
|
||||
import java.time.Instant;
|
||||
|
||||
@NoArgsConstructor
|
||||
public class ProducerInterceptor<K, V> extends AbstractInterceptor implements org.apache.kafka.clients.producer.ProducerInterceptor<K, V> {
|
||||
@Override
|
||||
public ProducerRecord<K, V> onSend(ProducerRecord<K, V> record) {
|
||||
this.logRecord(
|
||||
LoggerConfig.Type.PRODUCER,
|
||||
record.topic(),
|
||||
record.partition(),
|
||||
null,
|
||||
record.timestamp(),
|
||||
record.key(),
|
||||
record.value()
|
||||
);
|
||||
|
||||
return record;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onAcknowledgement(RecordMetadata metadata, Exception exception) {
|
||||
this.logAck(
|
||||
metadata.topic(),
|
||||
metadata.partition(),
|
||||
metadata.offset(),
|
||||
metadata.timestamp()
|
||||
);
|
||||
}
|
||||
|
||||
protected void logAck(
|
||||
String topic,
|
||||
Integer partition,
|
||||
@Nullable Long offset,
|
||||
Long timestamp
|
||||
) {
|
||||
Level level = isMatch(LoggerConfig.Type.PRODUCER_ACK, topic);
|
||||
|
||||
if (level == null) {
|
||||
return;
|
||||
}
|
||||
|
||||
String format = "[{}> {}{}{}{}]";
|
||||
Object[] args = {
|
||||
LoggerConfig.Type.PRODUCER_ACK,
|
||||
topic,
|
||||
partition != null ? "[" + partition + "]" : "",
|
||||
offset != null ? "@" + offset : "",
|
||||
timestamp != null ? " " + Instant.ofEpochMilli(timestamp) : "",
|
||||
};
|
||||
|
||||
this.log(level, format, args);
|
||||
}
|
||||
}
|
||||
@@ -1,18 +0,0 @@
|
||||
package io.kestra.runner.kafka.configs;
|
||||
|
||||
import io.micronaut.context.annotation.ConfigurationProperties;
|
||||
import io.micronaut.core.convert.format.MapFormat;
|
||||
import lombok.Getter;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
@ConfigurationProperties("kestra.kafka.client")
|
||||
@Getter
|
||||
public class ClientConfig {
|
||||
@MapFormat(transformation = MapFormat.MapTransformation.FLAT)
|
||||
Map<String, String> properties;
|
||||
|
||||
List<LoggerConfig> loggers;
|
||||
}
|
||||
|
||||
@@ -1,15 +0,0 @@
|
||||
package io.kestra.runner.kafka.configs;
|
||||
|
||||
import io.micronaut.context.annotation.ConfigurationProperties;
|
||||
import io.micronaut.core.convert.format.MapFormat;
|
||||
import lombok.Getter;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
@ConfigurationProperties("kestra.kafka.defaults.consumer")
|
||||
@Getter
|
||||
public class ConsumerDefaultsConfig {
|
||||
@MapFormat(transformation = MapFormat.MapTransformation.FLAT)
|
||||
Map<String, String> properties;
|
||||
}
|
||||
|
||||
@@ -1,28 +0,0 @@
|
||||
package io.kestra.runner.kafka.configs;
|
||||
|
||||
import lombok.Getter;
|
||||
import org.slf4j.event.Level;
|
||||
|
||||
import javax.validation.constraints.NotNull;
|
||||
|
||||
@Getter
|
||||
public class LoggerConfig {
|
||||
@NotNull
|
||||
private Level level;
|
||||
|
||||
private String topicRegexp;
|
||||
|
||||
private Type type;
|
||||
|
||||
private String keyRegexp;
|
||||
|
||||
private String valueRegexp;
|
||||
|
||||
public enum Type {
|
||||
CONSUMER,
|
||||
CONSUMER_COMMIT,
|
||||
PRODUCER,
|
||||
PRODUCER_ACK,
|
||||
}
|
||||
}
|
||||
|
||||
@@ -1,15 +0,0 @@
|
||||
package io.kestra.runner.kafka.configs;
|
||||
|
||||
import io.micronaut.context.annotation.ConfigurationProperties;
|
||||
import io.micronaut.core.convert.format.MapFormat;
|
||||
import lombok.Getter;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
@ConfigurationProperties("kestra.kafka.defaults.producer")
|
||||
@Getter
|
||||
public class ProducerDefaultsConfig {
|
||||
@MapFormat(transformation = MapFormat.MapTransformation.FLAT)
|
||||
Map<String, String> properties;
|
||||
}
|
||||
|
||||
@@ -1,15 +0,0 @@
|
||||
package io.kestra.runner.kafka.configs;
|
||||
|
||||
import io.micronaut.context.annotation.ConfigurationProperties;
|
||||
import io.micronaut.core.convert.format.MapFormat;
|
||||
import lombok.Getter;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
@ConfigurationProperties("kestra.kafka.defaults.stream")
|
||||
@Getter
|
||||
public class StreamDefaultsConfig {
|
||||
@MapFormat(transformation = MapFormat.MapTransformation.FLAT)
|
||||
Map<String, String> properties;
|
||||
}
|
||||
|
||||
@@ -1,19 +0,0 @@
|
||||
package io.kestra.runner.kafka.configs;
|
||||
|
||||
import io.micronaut.context.annotation.ConfigurationProperties;
|
||||
import io.micronaut.core.convert.format.MapFormat;
|
||||
import lombok.Getter;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
@ConfigurationProperties("kestra.kafka.defaults.topic")
|
||||
@Getter
|
||||
public class TopicDefaultsConfig {
|
||||
int partitions = 6;
|
||||
|
||||
short replicationFactor = 1;
|
||||
|
||||
@MapFormat(transformation = MapFormat.MapTransformation.FLAT)
|
||||
Map<String, String> properties;
|
||||
}
|
||||
|
||||
@@ -1,30 +0,0 @@
|
||||
package io.kestra.runner.kafka.configs;
|
||||
|
||||
import io.micronaut.context.annotation.EachProperty;
|
||||
import io.micronaut.context.annotation.Parameter;
|
||||
import io.micronaut.core.convert.format.MapFormat;
|
||||
import lombok.Getter;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
@EachProperty("kestra.kafka.topics")
|
||||
@Getter
|
||||
public class TopicsConfig {
|
||||
String key;
|
||||
|
||||
Class<?> cls;
|
||||
|
||||
String name;
|
||||
|
||||
Integer partitions;
|
||||
|
||||
Short replicationFactor;
|
||||
|
||||
@MapFormat(transformation = MapFormat.MapTransformation.FLAT)
|
||||
Map<String, String> properties;
|
||||
|
||||
public TopicsConfig(@Parameter String key) {
|
||||
this.key = key;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -1,147 +0,0 @@
|
||||
package io.kestra.runner.kafka.executors;
|
||||
|
||||
import io.kestra.core.models.flows.Flow;
|
||||
import io.kestra.core.models.triggers.AbstractTrigger;
|
||||
import io.kestra.core.models.triggers.Trigger;
|
||||
import io.kestra.core.queues.QueueService;
|
||||
import io.kestra.core.services.FlowService;
|
||||
import io.kestra.core.utils.ListUtils;
|
||||
import io.kestra.runner.kafka.KafkaQueueEnabled;
|
||||
import io.kestra.runner.kafka.serializers.JsonSerde;
|
||||
import io.kestra.runner.kafka.services.KafkaAdminService;
|
||||
import io.kestra.runner.kafka.services.KafkaStreamSourceService;
|
||||
import io.kestra.runner.kafka.services.KafkaStreamsBuilder;
|
||||
import jakarta.inject.Inject;
|
||||
import jakarta.inject.Singleton;
|
||||
import lombok.AllArgsConstructor;
|
||||
import lombok.Getter;
|
||||
import lombok.extern.jackson.Jacksonized;
|
||||
import lombok.extern.slf4j.Slf4j;
|
||||
import org.apache.kafka.common.serialization.Serdes;
|
||||
import org.apache.kafka.common.utils.Bytes;
|
||||
import org.apache.kafka.streams.KeyValue;
|
||||
import org.apache.kafka.streams.StreamsBuilder;
|
||||
import org.apache.kafka.streams.kstream.*;
|
||||
import org.apache.kafka.streams.state.KeyValueStore;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
@KafkaQueueEnabled
|
||||
@Singleton
|
||||
@Slf4j
|
||||
public class ExecutorFlowLast implements KafkaExecutorInterface {
|
||||
@Inject
|
||||
private KafkaAdminService kafkaAdminService;
|
||||
|
||||
@Inject
|
||||
private QueueService queueService;
|
||||
|
||||
public StreamsBuilder topology() {
|
||||
StreamsBuilder builder = new KafkaStreamsBuilder();
|
||||
|
||||
// last global KTable
|
||||
GlobalKTable<String, Flow> flowGlobalKTable = builder
|
||||
.globalTable(
|
||||
kafkaAdminService.getTopicName(KafkaStreamSourceService.TOPIC_FLOWLAST),
|
||||
Consumed.with(Serdes.String(), JsonSerde.of(Flow.class)).withName("GlobalKTable.FlowLast"),
|
||||
Materialized.<String, Flow, KeyValueStore<Bytes, byte[]>>as("last")
|
||||
.withKeySerde(Serdes.String())
|
||||
.withValueSerde(JsonSerde.of(Flow.class))
|
||||
);
|
||||
|
||||
// stream
|
||||
KStream<String, Flow> stream = builder
|
||||
.stream(
|
||||
kafkaAdminService.getTopicName(Flow.class),
|
||||
Consumed.with(Serdes.String(), JsonSerde.of(Flow.class, false)).withName("Stream.Flow")
|
||||
);
|
||||
|
||||
// logs
|
||||
stream = KafkaStreamSourceService.logIfEnabled(
|
||||
log,
|
||||
stream,
|
||||
(key, value) -> log.trace(
|
||||
"Flow in '{}.{}' with revision {}",
|
||||
value.getNamespace(),
|
||||
value.getId(),
|
||||
value.getRevision()
|
||||
),
|
||||
"Main"
|
||||
);
|
||||
|
||||
// join with previous if more recent revision
|
||||
KStream<String, ExecutorFlowLast.FlowWithPrevious> streamWithPrevious = stream
|
||||
.filter((key, value) -> value != null, Named.as("Main.notNull"))
|
||||
.selectKey((key, value) -> value.uidWithoutRevision(), Named.as("Main.selectKey"))
|
||||
.leftJoin(
|
||||
flowGlobalKTable,
|
||||
(key, value) -> key,
|
||||
(readOnlyKey, current, previous) -> {
|
||||
if (previous == null) {
|
||||
return new ExecutorFlowLast.FlowWithPrevious(current, null);
|
||||
} else if (current.getRevision() < previous.getRevision()) {
|
||||
return null;
|
||||
} else {
|
||||
return new ExecutorFlowLast.FlowWithPrevious(current, previous);
|
||||
}
|
||||
},
|
||||
Named.as("Main.join")
|
||||
)
|
||||
.filter((key, value) -> value != null, Named.as("Main.joinNotNull"));
|
||||
|
||||
// remove triggers
|
||||
streamWithPrevious
|
||||
.flatMap(
|
||||
(key, value) -> {
|
||||
List<AbstractTrigger> deletedTriggers = new ArrayList<>();
|
||||
|
||||
if (value.getFlow().isDeleted()) {
|
||||
deletedTriggers = ListUtils.emptyOnNull(value.getFlow().getTriggers());
|
||||
} else if (value.getPrevious() != null) {
|
||||
deletedTriggers = FlowService.findRemovedTrigger(
|
||||
value.getFlow(),
|
||||
value.getPrevious()
|
||||
);
|
||||
}
|
||||
|
||||
return deletedTriggers
|
||||
.stream()
|
||||
.map(t -> new KeyValue<>(
|
||||
queueService.key(Trigger.of(value.getFlow(), t)),
|
||||
(Trigger) null
|
||||
))
|
||||
.collect(Collectors.toList());
|
||||
},
|
||||
Named.as("DeleteTrigger.flatMap")
|
||||
)
|
||||
.to(
|
||||
kafkaAdminService.getTopicName(Trigger.class),
|
||||
Produced.with(Serdes.String(), JsonSerde.of(Trigger.class)).withName("To.Trigger")
|
||||
);
|
||||
|
||||
// send to last and don't drop deleted flow in order to keep last version
|
||||
streamWithPrevious
|
||||
.map(
|
||||
(key, value) -> new KeyValue<>(
|
||||
value.getFlow().uidWithoutRevision(),
|
||||
value.getFlow()
|
||||
),
|
||||
Named.as("Main.Map")
|
||||
)
|
||||
.to(
|
||||
kafkaAdminService.getTopicName(KafkaStreamSourceService.TOPIC_FLOWLAST),
|
||||
Produced.with(Serdes.String(), JsonSerde.of(Flow.class)).withName("To.FlowLast")
|
||||
);
|
||||
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Getter
|
||||
@AllArgsConstructor
|
||||
public static class FlowWithPrevious {
|
||||
private Flow flow;
|
||||
private Flow previous;
|
||||
}
|
||||
}
|
||||
@@ -1,97 +0,0 @@
|
||||
package io.kestra.runner.kafka.executors;
|
||||
|
||||
import io.kestra.core.models.executions.Execution;
|
||||
import io.kestra.core.models.triggers.multipleflows.MultipleConditionWindow;
|
||||
import io.kestra.core.services.FlowService;
|
||||
import io.kestra.runner.kafka.KafkaFlowExecutor;
|
||||
import io.kestra.runner.kafka.KafkaQueueEnabled;
|
||||
import io.kestra.runner.kafka.serializers.JsonSerde;
|
||||
import io.kestra.runner.kafka.services.KafkaAdminService;
|
||||
import io.kestra.runner.kafka.services.KafkaStreamsBuilder;
|
||||
import io.kestra.runner.kafka.streams.FlowTriggerWithExecutionTransformer;
|
||||
import jakarta.inject.Inject;
|
||||
import jakarta.inject.Singleton;
|
||||
import org.apache.kafka.common.serialization.Serdes;
|
||||
import org.apache.kafka.streams.KeyValue;
|
||||
import org.apache.kafka.streams.StreamsBuilder;
|
||||
import org.apache.kafka.streams.kstream.Consumed;
|
||||
import org.apache.kafka.streams.kstream.KStream;
|
||||
import org.apache.kafka.streams.kstream.Named;
|
||||
import org.apache.kafka.streams.kstream.Produced;
|
||||
import org.apache.kafka.streams.state.Stores;
|
||||
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
@KafkaQueueEnabled
|
||||
@Singleton
|
||||
public class ExecutorFlowTrigger implements KafkaExecutorInterface {
|
||||
public static final String TRIGGER_MULTIPLE_STATE_STORE_NAME = "trigger_multiplecondition";
|
||||
|
||||
@Inject
|
||||
private KafkaAdminService kafkaAdminService;
|
||||
|
||||
@Inject
|
||||
private FlowService flowService;
|
||||
|
||||
@Inject
|
||||
private KafkaFlowExecutor kafkaFlowExecutor;
|
||||
|
||||
public StreamsBuilder topology() {
|
||||
StreamsBuilder builder = new KafkaStreamsBuilder();
|
||||
|
||||
// trigger
|
||||
builder.addStateStore(
|
||||
Stores.keyValueStoreBuilder(
|
||||
Stores.persistentKeyValueStore(TRIGGER_MULTIPLE_STATE_STORE_NAME),
|
||||
Serdes.String(),
|
||||
JsonSerde.of(MultipleConditionWindow.class)
|
||||
)
|
||||
);
|
||||
|
||||
KStream<String, io.kestra.runner.kafka.streams.ExecutorFlowTrigger> stream = builder
|
||||
.stream(
|
||||
kafkaAdminService.getTopicName(io.kestra.runner.kafka.streams.ExecutorFlowTrigger.class),
|
||||
Consumed.with(Serdes.String(), JsonSerde.of(io.kestra.runner.kafka.streams.ExecutorFlowTrigger.class))
|
||||
.withName("KStream.ExecutorFlowTrigger")
|
||||
)
|
||||
.filter((key, value) -> value != null, Named.as("ExecutorFlowTrigger.filterNotNull"));
|
||||
|
||||
stream
|
||||
.transformValues(
|
||||
() -> new FlowTriggerWithExecutionTransformer(
|
||||
TRIGGER_MULTIPLE_STATE_STORE_NAME,
|
||||
kafkaFlowExecutor,
|
||||
flowService
|
||||
),
|
||||
Named.as("ExecutorFlowTrigger.transformToExecutionList"),
|
||||
TRIGGER_MULTIPLE_STATE_STORE_NAME
|
||||
)
|
||||
.flatMap(
|
||||
(key, value) -> value
|
||||
.stream()
|
||||
.map(execution -> new KeyValue<>(execution.getId(), execution))
|
||||
.collect(Collectors.toList()),
|
||||
Named.as("ExecutorFlowTrigger.flapMapToExecution")
|
||||
)
|
||||
.to(
|
||||
kafkaAdminService.getTopicName(Execution.class),
|
||||
Produced
|
||||
.with(Serdes.String(), JsonSerde.of(Execution.class))
|
||||
.withName("ExecutorFlowTrigger.toExecution")
|
||||
);
|
||||
|
||||
stream
|
||||
.mapValues(
|
||||
(readOnlyKey, value) -> (io.kestra.runner.kafka.streams.ExecutorFlowTrigger)null,
|
||||
Named.as("ExecutorFlowTrigger.executorFlowTriggerToNull")
|
||||
)
|
||||
.to(
|
||||
kafkaAdminService.getTopicName(io.kestra.runner.kafka.streams.ExecutorFlowTrigger.class),
|
||||
Produced
|
||||
.with(Serdes.String(), JsonSerde.of(io.kestra.runner.kafka.streams.ExecutorFlowTrigger.class))
|
||||
.withName("ExecutorFlowTrigger.toExecutorFlowTrigger")
|
||||
);
|
||||
|
||||
return builder;
|
||||
}
|
||||
}
|
||||
@@ -1,717 +0,0 @@
|
||||
package io.kestra.runner.kafka.executors;
|
||||
|
||||
import io.kestra.core.metrics.MetricRegistry;
|
||||
import io.kestra.core.models.executions.Execution;
|
||||
import io.kestra.core.models.executions.ExecutionKilled;
|
||||
import io.kestra.core.models.executions.LogEntry;
|
||||
import io.kestra.core.models.executions.TaskRun;
|
||||
import io.kestra.core.models.flows.State;
|
||||
import io.kestra.core.queues.QueueService;
|
||||
import io.kestra.core.runners.*;
|
||||
import io.kestra.core.services.ConditionService;
|
||||
import io.kestra.core.services.ExecutionService;
|
||||
import io.kestra.core.services.FlowService;
|
||||
import io.kestra.runner.kafka.KafkaFlowExecutor;
|
||||
import io.kestra.runner.kafka.KafkaQueueEnabled;
|
||||
import io.kestra.runner.kafka.serializers.JsonSerde;
|
||||
import io.kestra.runner.kafka.services.KafkaAdminService;
|
||||
import io.kestra.runner.kafka.services.KafkaStreamSourceService;
|
||||
import io.kestra.runner.kafka.services.KafkaStreamsBuilder;
|
||||
import io.kestra.runner.kafka.streams.*;
|
||||
import jakarta.inject.Inject;
|
||||
import jakarta.inject.Singleton;
|
||||
import lombok.AllArgsConstructor;
|
||||
import lombok.Getter;
|
||||
import lombok.NoArgsConstructor;
|
||||
import lombok.extern.slf4j.Slf4j;
|
||||
import org.apache.commons.lang3.tuple.Pair;
|
||||
import org.apache.kafka.common.serialization.Serdes;
|
||||
import org.apache.kafka.common.utils.Bytes;
|
||||
import org.apache.kafka.streams.KeyValue;
|
||||
import org.apache.kafka.streams.StreamsBuilder;
|
||||
import org.apache.kafka.streams.kstream.*;
|
||||
import org.apache.kafka.streams.state.KeyValueStore;
|
||||
import org.apache.kafka.streams.state.Stores;
|
||||
import org.slf4j.event.Level;
|
||||
|
||||
import java.time.Duration;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
@KafkaQueueEnabled
|
||||
@Singleton
|
||||
@Slf4j
|
||||
public class ExecutorMain implements KafkaExecutorInterface {
|
||||
private static final String EXECUTOR_STATE_STORE_NAME = "executor";
|
||||
private static final String WORKERTASK_DEDUPLICATION_STATE_STORE_NAME = "workertask_deduplication";
|
||||
private static final String TRIGGER_DEDUPLICATION_STATE_STORE_NAME = "trigger_deduplication";
|
||||
private static final String NEXTS_DEDUPLICATION_STATE_STORE_NAME = "next_deduplication";
|
||||
private static final String EXECUTION_DELAY_STATE_STORE_NAME = "execution_delay";
|
||||
|
||||
@Inject
|
||||
private KafkaAdminService kafkaAdminService;
|
||||
|
||||
@Inject
|
||||
private FlowService flowService;
|
||||
|
||||
@Inject
|
||||
private KafkaStreamSourceService kafkaStreamSourceService;
|
||||
|
||||
@Inject
|
||||
private QueueService queueService;
|
||||
|
||||
@Inject
|
||||
private MetricRegistry metricRegistry;
|
||||
|
||||
@Inject
|
||||
private ConditionService conditionService;
|
||||
|
||||
@Inject
|
||||
private ExecutorService executorService;
|
||||
|
||||
@Inject
|
||||
private RunContextFactory runContextFactory;
|
||||
|
||||
@Inject
|
||||
private KafkaFlowExecutor kafkaFlowExecutor;
|
||||
|
||||
@Inject
|
||||
private ExecutionService executionService;
|
||||
|
||||
public StreamsBuilder topology() {
|
||||
StreamsBuilder builder = new KafkaStreamsBuilder();
|
||||
|
||||
// executor
|
||||
builder.addStateStore(Stores.keyValueStoreBuilder(
|
||||
Stores.persistentKeyValueStore(EXECUTOR_STATE_STORE_NAME),
|
||||
Serdes.String(),
|
||||
JsonSerde.of(Executor.class)
|
||||
));
|
||||
|
||||
// WorkerTask deduplication
|
||||
builder.addStateStore(Stores.keyValueStoreBuilder(
|
||||
Stores.persistentKeyValueStore(WORKERTASK_DEDUPLICATION_STATE_STORE_NAME),
|
||||
Serdes.String(),
|
||||
Serdes.String()
|
||||
));
|
||||
|
||||
// next deduplication
|
||||
builder.addStateStore(Stores.keyValueStoreBuilder(
|
||||
Stores.persistentKeyValueStore(NEXTS_DEDUPLICATION_STATE_STORE_NAME),
|
||||
Serdes.String(),
|
||||
JsonSerde.of(ExecutorNextTransformer.Store.class)
|
||||
));
|
||||
|
||||
// trigger deduplication
|
||||
builder.addStateStore(Stores.keyValueStoreBuilder(
|
||||
Stores.persistentKeyValueStore(TRIGGER_DEDUPLICATION_STATE_STORE_NAME),
|
||||
Serdes.String(),
|
||||
Serdes.String()
|
||||
));
|
||||
|
||||
// Execution delay
|
||||
builder.addStateStore(Stores.windowStoreBuilder(
|
||||
Stores.persistentWindowStore(EXECUTION_DELAY_STATE_STORE_NAME, Duration.ofDays(7), Duration.ofSeconds(1), false),
|
||||
Serdes.String(),
|
||||
JsonSerde.of(ExecutionDelay.class)
|
||||
));
|
||||
|
||||
// declare common stream
|
||||
KStream<String, WorkerTaskResult> workerTaskResultKStream = this.workerTaskResultKStream(builder);
|
||||
KStream<String, Executor> executorKStream = this.executorKStream(builder);
|
||||
|
||||
// join with killed
|
||||
KStream<String, ExecutionKilled> executionKilledKStream = this.executionKilledKStream(builder);
|
||||
KStream<String, Executor> executionWithKilled = this.joinExecutionKilled(executionKilledKStream, executorKStream);
|
||||
|
||||
// join with WorkerResult
|
||||
KStream<String, Executor> executionKStream = this.joinWorkerResult(workerTaskResultKStream, executionWithKilled);
|
||||
|
||||
// handle state on execution
|
||||
KStream<String, Executor> stream = kafkaStreamSourceService.executorWithFlow(executionKStream, true);
|
||||
|
||||
stream = this.handleExecutor(stream);
|
||||
|
||||
// save execution
|
||||
this.toExecution(stream, "Main");
|
||||
this.toWorkerTask(stream);
|
||||
this.handleExecutionDelay(stream);
|
||||
this.toWorkerTaskResult(stream);
|
||||
|
||||
this.toExecutorFlowTriggerTopic(stream);
|
||||
|
||||
// task Flow
|
||||
KTable<String, WorkerTaskExecution> workerTaskExecutionKTable = this.workerTaskExecutionStream(builder);
|
||||
|
||||
KStream<String, WorkerTaskExecution> workerTaskExecutionKStream = this.deduplicateWorkerTaskExecution(stream);
|
||||
this.toWorkerTaskExecution(workerTaskExecutionKStream);
|
||||
this.workerTaskExecutionToExecution(workerTaskExecutionKStream);
|
||||
this.handleWorkerTaskExecution(workerTaskExecutionKTable, stream);
|
||||
|
||||
// purge at end
|
||||
this.purgeExecutor(executorKStream);
|
||||
|
||||
this.purgeWorkerRunning(workerTaskResultKStream);
|
||||
|
||||
return builder;
|
||||
}
|
||||
|
||||
public KStream<String, Executor> executorKStream(StreamsBuilder builder) {
|
||||
KStream<String, Executor> result = builder
|
||||
.stream(
|
||||
kafkaAdminService.getTopicName(Execution.class),
|
||||
Consumed.with(Serdes.String(), JsonSerde.of(Execution.class)).withName("Executor.fromExecution")
|
||||
)
|
||||
.filter((key, value) -> value != null, Named.as("Executor.filterNotNull"))
|
||||
.transformValues(
|
||||
() -> new ExecutorFromExecutionTransformer(EXECUTOR_STATE_STORE_NAME),
|
||||
Named.as("Executor.toExecutor"),
|
||||
EXECUTOR_STATE_STORE_NAME
|
||||
);
|
||||
|
||||
// logs
|
||||
KafkaStreamSourceService.logIfEnabled(
|
||||
log,
|
||||
result,
|
||||
(key, value) -> executorService.log(log, true, value),
|
||||
"ExecutionIn"
|
||||
);
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
private KStream<String, ExecutionKilled> executionKilledKStream(StreamsBuilder builder) {
|
||||
return builder
|
||||
.stream(
|
||||
kafkaAdminService.getTopicName(ExecutionKilled.class),
|
||||
Consumed.with(Serdes.String(), JsonSerde.of(ExecutionKilled.class)).withName("KTable.ExecutionKilled")
|
||||
);
|
||||
}
|
||||
|
||||
private KStream<String, Executor> joinExecutionKilled(KStream<String, ExecutionKilled> executionKilledKStream, KStream<String, Executor> executorKStream) {
|
||||
return executorKStream
|
||||
.merge(
|
||||
executionKilledKStream
|
||||
.transformValues(
|
||||
() -> new ExecutorKilledJoinerTransformer(
|
||||
EXECUTOR_STATE_STORE_NAME
|
||||
),
|
||||
Named.as("JoinExecutionKilled.transformValues"),
|
||||
EXECUTOR_STATE_STORE_NAME
|
||||
)
|
||||
.filter((key, value) -> value != null, Named.as("JoinExecutionKilled.filterNotNull")),
|
||||
Named.as("JoinExecutionKilled.merge")
|
||||
);
|
||||
}
|
||||
|
||||
private KStream<String, WorkerTaskResult> workerTaskResultKStream(StreamsBuilder builder) {
|
||||
return builder
|
||||
.stream(
|
||||
kafkaAdminService.getTopicName(WorkerTaskResult.class),
|
||||
Consumed.with(Serdes.String(), JsonSerde.of(WorkerTaskResult.class)).withName("KStream.WorkerTaskResult")
|
||||
)
|
||||
.filter((key, value) -> value != null, Named.as("WorkerTaskResultKStream.filterNotNull"));
|
||||
}
|
||||
|
||||
private KStream<String, Executor> joinWorkerResult(KStream<String, WorkerTaskResult> workerTaskResultKstream, KStream<String, Executor> executorKStream) {
|
||||
return executorKStream
|
||||
.merge(
|
||||
workerTaskResultKstream
|
||||
.selectKey((key, value) -> value.getTaskRun().getExecutionId(), Named.as("JoinWorkerResult.selectKey"))
|
||||
.mapValues(
|
||||
(key, value) -> new Executor(value),
|
||||
Named.as("JoinWorkerResult.WorkerTaskResultMap")
|
||||
)
|
||||
.repartition(
|
||||
Repartitioned.<String, Executor>as("workertaskjoined")
|
||||
.withKeySerde(Serdes.String())
|
||||
.withValueSerde(JsonSerde.of(Executor.class))
|
||||
),
|
||||
Named.as("JoinWorkerResult.merge")
|
||||
)
|
||||
.transformValues(
|
||||
() -> new ExecutorJoinerTransformer(
|
||||
EXECUTOR_STATE_STORE_NAME,
|
||||
this.executorService,
|
||||
this.kafkaStreamSourceService,
|
||||
this.metricRegistry
|
||||
),
|
||||
Named.as("JoinWorkerResult.transformValues"),
|
||||
EXECUTOR_STATE_STORE_NAME
|
||||
)
|
||||
.filter(
|
||||
(key, value) -> value != null,
|
||||
Named.as("JoinWorkerResult.notNullFilter")
|
||||
);
|
||||
}
|
||||
|
||||
private KStream<String, Executor> handleExecutor(KStream<String, Executor> stream) {
|
||||
return stream
|
||||
.transformValues(
|
||||
() -> new ExecutorNextTransformer(
|
||||
NEXTS_DEDUPLICATION_STATE_STORE_NAME,
|
||||
this.executorService
|
||||
),
|
||||
Named.as("HandleExecutor.transformValues"),
|
||||
NEXTS_DEDUPLICATION_STATE_STORE_NAME
|
||||
);
|
||||
}
|
||||
|
||||
private void purgeExecutor(KStream<String, Executor> stream) {
|
||||
KStream<String, Executor> terminated = stream
|
||||
.filter(
|
||||
(key, value) -> executorService.canBePurged(value),
|
||||
Named.as("PurgeExecutor.filterTerminated")
|
||||
);
|
||||
|
||||
// clean up executor
|
||||
terminated
|
||||
.mapValues(
|
||||
(readOnlyKey, value) -> (Execution) null,
|
||||
Named.as("PurgeExecutor.executionToNull")
|
||||
)
|
||||
.to(
|
||||
kafkaAdminService.getTopicName(Executor.class),
|
||||
Produced.with(Serdes.String(), JsonSerde.of(Execution.class)).withName("PurgeExecutor.toExecutor")
|
||||
);
|
||||
|
||||
// flatMap taskRun
|
||||
KStream<String, TaskRun> taskRunKStream = terminated
|
||||
.filter(
|
||||
(key, value) -> value.getExecution().getTaskRunList() != null,
|
||||
Named.as("PurgeExecutor.notNullTaskRunList")
|
||||
)
|
||||
.flatMapValues(
|
||||
(readOnlyKey, value) -> value.getExecution().getTaskRunList(),
|
||||
Named.as("PurgeExecutor.flatMapTaskRunList")
|
||||
);
|
||||
|
||||
// clean up workerTaskResult
|
||||
taskRunKStream
|
||||
.map(
|
||||
(readOnlyKey, value) -> new KeyValue<>(
|
||||
value.getId(),
|
||||
(WorkerTaskResult) null
|
||||
),
|
||||
Named.as("PurgeExecutor.workerTaskResultToNull")
|
||||
)
|
||||
.to(
|
||||
kafkaAdminService.getTopicName(WorkerTaskResult.class),
|
||||
Produced.with(Serdes.String(), JsonSerde.of(WorkerTaskResult.class)).withName("PurgeExecutor.toWorkerTaskResult")
|
||||
);
|
||||
|
||||
// clean up WorkerTask deduplication state
|
||||
taskRunKStream
|
||||
.transformValues(
|
||||
() -> new DeduplicationPurgeTransformer<>(
|
||||
WORKERTASK_DEDUPLICATION_STATE_STORE_NAME,
|
||||
(key, value) -> value.getExecutionId() + "-" + value.getId()
|
||||
),
|
||||
Named.as("PurgeExecutor.purgeWorkerTaskDeduplication"),
|
||||
WORKERTASK_DEDUPLICATION_STATE_STORE_NAME
|
||||
);
|
||||
|
||||
taskRunKStream
|
||||
.transformValues(
|
||||
() -> new DeduplicationPurgeTransformer<>(
|
||||
WORKERTASK_DEDUPLICATION_STATE_STORE_NAME,
|
||||
(key, value) -> "WorkerTaskExecution-" + value.getExecutionId() + "-" + value.getId()
|
||||
),
|
||||
Named.as("PurgeExecutor.purgeWorkerTaskExecutionDeduplication"),
|
||||
WORKERTASK_DEDUPLICATION_STATE_STORE_NAME
|
||||
);
|
||||
|
||||
// clean up Execution Nexts deduplication state
|
||||
terminated
|
||||
.transformValues(
|
||||
() -> new DeduplicationPurgeTransformer<>(
|
||||
NEXTS_DEDUPLICATION_STATE_STORE_NAME,
|
||||
(key, value) -> value.getExecution().getId()
|
||||
),
|
||||
Named.as("PurgeExecutor.purgeNextsDeduplication"),
|
||||
NEXTS_DEDUPLICATION_STATE_STORE_NAME
|
||||
);
|
||||
|
||||
// clean up Flow Trigger deduplication state
|
||||
terminated
|
||||
.transformValues(
|
||||
() -> new DeduplicationPurgeTransformer<>(
|
||||
TRIGGER_DEDUPLICATION_STATE_STORE_NAME,
|
||||
(key, value) -> value.getExecution().getId()
|
||||
),
|
||||
Named.as("PurgeExecutor.purgeTriggerDeduplication"),
|
||||
TRIGGER_DEDUPLICATION_STATE_STORE_NAME
|
||||
);
|
||||
|
||||
// clean up killed
|
||||
terminated
|
||||
.filter(
|
||||
(key, value) -> value.getExecution().getState().getCurrent() == State.Type.KILLED,
|
||||
Named.as("PurgeExecutor.filterKilledToNull")
|
||||
)
|
||||
.mapValues(
|
||||
(readOnlyKey, value) -> (ExecutionKilled) null,
|
||||
Named.as("PurgeExecutor.executionKilledToNull")
|
||||
)
|
||||
.to(
|
||||
kafkaAdminService.getTopicName(ExecutionKilled.class),
|
||||
Produced.with(Serdes.String(), JsonSerde.of(ExecutionKilled.class)).withName("PurgeExecutor.toExecutionKilled")
|
||||
);
|
||||
}
|
||||
|
||||
private void toExecutorFlowTriggerTopic(KStream<String, Executor> stream) {
|
||||
stream
|
||||
.filter(
|
||||
(key, value) -> conditionService.isTerminatedWithListeners(value.getFlow(), value.getExecution()),
|
||||
Named.as("HandleExecutorFlowTriggerTopic.filterTerminated")
|
||||
)
|
||||
.transformValues(
|
||||
() -> new DeduplicationTransformer<>(
|
||||
"FlowTrigger",
|
||||
TRIGGER_DEDUPLICATION_STATE_STORE_NAME,
|
||||
(key, value) -> value.getExecution().getId(),
|
||||
(key, value) -> value.getExecution().getId()
|
||||
),
|
||||
Named.as("HandleExecutorFlowTriggerTopic.deduplication"),
|
||||
TRIGGER_DEDUPLICATION_STATE_STORE_NAME
|
||||
)
|
||||
.filter((key, value) -> value != null, Named.as("HandleExecutorFlowTriggerTopic.deduplicationNotNull"))
|
||||
.flatTransform(
|
||||
() -> new FlowWithTriggerTransformer(kafkaFlowExecutor, flowService),
|
||||
Named.as("HandleExecutorFlowTriggerTopic.flatMapToExecutorFlowTrigger")
|
||||
)
|
||||
.to(
|
||||
kafkaAdminService.getTopicName(io.kestra.runner.kafka.streams.ExecutorFlowTrigger.class),
|
||||
Produced
|
||||
.with(Serdes.String(), JsonSerde.of(io.kestra.runner.kafka.streams.ExecutorFlowTrigger.class))
|
||||
.withName("PurgeExecutor.toExecutorFlowTrigger")
|
||||
);
|
||||
}
|
||||
|
||||
private void toWorkerTask(KStream<String, Executor> stream) {
|
||||
// deduplication worker task
|
||||
KStream<String, WorkerTask> dedupWorkerTask = stream
|
||||
.flatMapValues(
|
||||
(readOnlyKey, value) -> value.getWorkerTasks(),
|
||||
Named.as("HandleWorkerTask.flatMapToWorkerTask")
|
||||
)
|
||||
.transformValues(
|
||||
() -> new DeduplicationTransformer<>(
|
||||
"WorkerTask",
|
||||
WORKERTASK_DEDUPLICATION_STATE_STORE_NAME,
|
||||
(key, value) -> value.getTaskRun().getExecutionId() + "-" + value.getTaskRun().getId(),
|
||||
(key, value) -> value.getTaskRun().getState().getCurrent().name()
|
||||
),
|
||||
Named.as("HandleWorkerTask.deduplication"),
|
||||
WORKERTASK_DEDUPLICATION_STATE_STORE_NAME
|
||||
)
|
||||
.filter((key, value) -> value != null, Named.as("HandleWorkerTask.notNullFilter"));
|
||||
|
||||
// flowable > running to WorkerTaskResult
|
||||
KStream<String, WorkerTaskResult> resultFlowable = dedupWorkerTask
|
||||
.filter((key, value) -> value.getTask().isFlowable(), Named.as("HandleWorkerTaskFlowable.filterIsFlowable"))
|
||||
.mapValues(
|
||||
(key, value) -> new WorkerTaskResult(value.withTaskRun(value.getTaskRun().withState(State.Type.RUNNING))),
|
||||
Named.as("HandleWorkerTaskFlowable.toRunning")
|
||||
)
|
||||
.map(
|
||||
(key, value) -> new KeyValue<>(queueService.key(value), value),
|
||||
Named.as("HandleWorkerTaskFlowable.mapWithKey")
|
||||
)
|
||||
.selectKey(
|
||||
(key, value) -> queueService.key(value),
|
||||
Named.as("HandleWorkerTaskFlowable.selectKey")
|
||||
);
|
||||
|
||||
KStream<String, WorkerTaskResult> workerTaskResultKStream = KafkaStreamSourceService.logIfEnabled(
|
||||
log,
|
||||
resultFlowable,
|
||||
(key, value) -> executorService.log(log, false, value),
|
||||
"HandleWorkerTaskFlowable"
|
||||
);
|
||||
|
||||
workerTaskResultKStream
|
||||
.to(
|
||||
kafkaAdminService.getTopicName(WorkerTaskResult.class),
|
||||
Produced.with(Serdes.String(), JsonSerde.of(WorkerTaskResult.class)).withName("HandleWorkerTaskFlowable.toWorkerTaskResult")
|
||||
);
|
||||
|
||||
// not flowable > to WorkerTask
|
||||
KStream<String, WorkerTask> resultNotFlowable = dedupWorkerTask
|
||||
.filter((key, value) -> value.getTask().isSendToWorkerTask(), Named.as("HandleWorkerTaskNotFlowable.filterIsNotFlowable"))
|
||||
.map((key, value) -> new KeyValue<>(queueService.key(value), value), Named.as("HandleWorkerTaskNotFlowable.mapWithKey"))
|
||||
.selectKey(
|
||||
(key, value) -> queueService.key(value),
|
||||
Named.as("HandleWorkerTaskNotFlowable.selectKey")
|
||||
);
|
||||
|
||||
KStream<String, WorkerTask> workerTaskKStream = KafkaStreamSourceService.logIfEnabled(
|
||||
log,
|
||||
resultNotFlowable,
|
||||
(key, value) -> executorService.log(log, false, value),
|
||||
"HandleWorkerTaskNotFlowable"
|
||||
);
|
||||
|
||||
workerTaskKStream
|
||||
.to(
|
||||
kafkaAdminService.getTopicName(WorkerTask.class),
|
||||
Produced.with(Serdes.String(), JsonSerde.of(WorkerTask.class)).withName("HandleWorkerTaskNotFlowable.toWorkerTask")
|
||||
);
|
||||
}
|
||||
|
||||
private KTable<String, WorkerTaskExecution> workerTaskExecutionStream(StreamsBuilder builder) {
|
||||
return builder
|
||||
.table(
|
||||
kafkaAdminService.getTopicName(WorkerTaskExecution.class),
|
||||
Consumed.with(Serdes.String(), JsonSerde.of(WorkerTaskExecution.class)).withName("WorkerTaskExecution.from"),
|
||||
Materialized.<String, WorkerTaskExecution, KeyValueStore<Bytes, byte[]>>as("workertaskexecution")
|
||||
.withKeySerde(Serdes.String())
|
||||
.withValueSerde(JsonSerde.of(WorkerTaskExecution.class))
|
||||
);
|
||||
}
|
||||
|
||||
private KStream<String, WorkerTaskExecution> deduplicateWorkerTaskExecution(KStream<String, Executor> stream) {
|
||||
return stream
|
||||
.flatMapValues(
|
||||
(readOnlyKey, value) -> value.getWorkerTaskExecutions(),
|
||||
Named.as("DeduplicateWorkerTaskExecution.flatMap")
|
||||
)
|
||||
.transformValues(
|
||||
() -> new DeduplicationTransformer<>(
|
||||
"DeduplicateWorkerTaskExecution",
|
||||
WORKERTASK_DEDUPLICATION_STATE_STORE_NAME,
|
||||
(key, value) -> "WorkerTaskExecution-" + value.getTaskRun().getExecutionId() + "-" + value.getTaskRun().getId(),
|
||||
(key, value) -> value.getTaskRun().getState().getCurrent().name()
|
||||
),
|
||||
Named.as("DeduplicateWorkerTaskExecution.deduplication"),
|
||||
WORKERTASK_DEDUPLICATION_STATE_STORE_NAME
|
||||
)
|
||||
.filter((key, value) -> value != null, Named.as("DeduplicateWorkerTaskExecution.notNullFilter"));
|
||||
}
|
||||
|
||||
private void toWorkerTaskExecution(KStream<String, WorkerTaskExecution> stream) {
|
||||
stream
|
||||
.selectKey(
|
||||
(key, value) -> value.getExecution().getId(),
|
||||
Named.as("ToWorkerTaskExecution.selectKey")
|
||||
)
|
||||
.to(
|
||||
kafkaAdminService.getTopicName(WorkerTaskExecution.class),
|
||||
Produced.with(Serdes.String(), JsonSerde.of(WorkerTaskExecution.class)).withName("ToWorkerTaskExecution.toWorkerTaskExecution")
|
||||
);
|
||||
}
|
||||
|
||||
private void workerTaskExecutionToExecution(KStream<String, WorkerTaskExecution> stream) {
|
||||
stream
|
||||
.mapValues(
|
||||
value -> {
|
||||
String message = "Create new execution for flow '" +
|
||||
value.getExecution().getNamespace() + "'." + value.getExecution().getFlowId() +
|
||||
"' with id '" + value.getExecution().getId() + "' from task '" + value.getTask().getId() +
|
||||
"' and taskrun '" + value.getTaskRun().getId() +
|
||||
(value.getTaskRun().getValue() != null ? " (" + value.getTaskRun().getValue() + ")" : "") + "'";
|
||||
|
||||
log.info(message);
|
||||
|
||||
LogEntry.LogEntryBuilder logEntryBuilder = LogEntry.of(value.getTaskRun()).toBuilder()
|
||||
.level(Level.INFO)
|
||||
.message(message)
|
||||
.timestamp(value.getTaskRun().getState().getStartDate())
|
||||
.thread(Thread.currentThread().getName());
|
||||
|
||||
return logEntryBuilder.build();
|
||||
},
|
||||
Named.as("WorkerTaskExecutionToExecution.mapToLog")
|
||||
)
|
||||
.selectKey((key, value) -> (String)null, Named.as("WorkerTaskExecutionToExecution.logRemoveKey"))
|
||||
.to(
|
||||
kafkaAdminService.getTopicName(LogEntry.class),
|
||||
Produced.with(Serdes.String(), JsonSerde.of(LogEntry.class)).withName("WorkerTaskExecutionToExecution.toLogEntry")
|
||||
);
|
||||
|
||||
KStream<String, Execution> executionKStream = stream
|
||||
.mapValues(
|
||||
(key, value) -> value.getExecution(),
|
||||
Named.as("WorkerTaskExecutionToExecution.map")
|
||||
)
|
||||
.selectKey(
|
||||
(key, value) -> value.getId(),
|
||||
Named.as("WorkerTaskExecutionToExecution.selectKey")
|
||||
);
|
||||
|
||||
executionKStream = KafkaStreamSourceService.logIfEnabled(
|
||||
log,
|
||||
executionKStream,
|
||||
(key, value) -> executorService.log(log, false, value),
|
||||
"WorkerTaskExecutionToExecution"
|
||||
);
|
||||
|
||||
executionKStream
|
||||
.to(
|
||||
kafkaAdminService.getTopicName(Execution.class),
|
||||
Produced.with(Serdes.String(), JsonSerde.of(Execution.class)).withName("WorkerTaskExecutionToExecution.toExecution")
|
||||
);
|
||||
}
|
||||
|
||||
private void handleWorkerTaskExecution(KTable<String, WorkerTaskExecution> workerTaskExecutionKTable, KStream<String, Executor> stream) {
|
||||
KStream<String, WorkerTaskResult> joinKStream = stream
|
||||
.filter(
|
||||
(key, value) -> conditionService.isTerminatedWithListeners(value.getFlow(), value.getExecution()),
|
||||
Named.as("HandleWorkerTaskExecution.isTerminated")
|
||||
)
|
||||
.transformValues(
|
||||
() -> new WorkerTaskExecutionTransformer(runContextFactory, workerTaskExecutionKTable.queryableStoreName(), kafkaFlowExecutor),
|
||||
Named.as("HandleWorkerTaskExecution.transform"),
|
||||
workerTaskExecutionKTable.queryableStoreName()
|
||||
)
|
||||
.filter((key, value) -> value != null, Named.as("HandleWorkerTaskExecution.joinNotNullFilter"));
|
||||
|
||||
toWorkerTaskResultSend(joinKStream, "HandleWorkerTaskExecution");
|
||||
}
|
||||
|
||||
private void handleExecutionDelay(KStream<String, Executor> stream) {
|
||||
KStream<String, Executor> executionDelayStream = stream
|
||||
.flatMapValues(
|
||||
(readOnlyKey, value) -> value.getExecutionDelays(),
|
||||
Named.as("HandleExecutionDelay.flapMap")
|
||||
)
|
||||
.transform(
|
||||
() -> new ExecutorPausedTransformer(EXECUTION_DELAY_STATE_STORE_NAME, EXECUTOR_STATE_STORE_NAME, executionService),
|
||||
Named.as("HandleExecutionDelay.transform"),
|
||||
EXECUTION_DELAY_STATE_STORE_NAME,
|
||||
EXECUTOR_STATE_STORE_NAME
|
||||
)
|
||||
.filter((key, value) -> value != null, Named.as("HandleExecutionDelay.notNullFilter"));
|
||||
|
||||
toExecution(executionDelayStream, "Delay");
|
||||
}
|
||||
|
||||
private void toWorkerTaskResult(KStream<String, Executor> stream) {
|
||||
KStream<String, WorkerTaskResult> workerTaskResultKStream = stream
|
||||
.flatMapValues(
|
||||
(readOnlyKey, value) -> value.getWorkerTaskResults(),
|
||||
Named.as("ToWorkerTaskResult.flapMap")
|
||||
);
|
||||
|
||||
toWorkerTaskResultSend(workerTaskResultKStream, "HandleWorkerTaskResult");
|
||||
}
|
||||
|
||||
private void toWorkerTaskResultSend(KStream<String, WorkerTaskResult> stream, String name) {
|
||||
KStream<String, WorkerTaskResult> workerTaskResultKStream = stream
|
||||
.transformValues(
|
||||
() -> new DeduplicationTransformer<>(
|
||||
name,
|
||||
WORKERTASK_DEDUPLICATION_STATE_STORE_NAME,
|
||||
(key, value) -> value.getTaskRun().getExecutionId() + "-" + value.getTaskRun().getId(),
|
||||
(key, value) -> value.getTaskRun().getState().getCurrent().name()
|
||||
),
|
||||
Named.as(name + ".deduplication"),
|
||||
WORKERTASK_DEDUPLICATION_STATE_STORE_NAME
|
||||
)
|
||||
.filter((key, value) -> value != null, Named.as(name + ".notNullFilter"))
|
||||
.selectKey(
|
||||
(key, value) -> value.getTaskRun().getId(),
|
||||
Named.as(name + ".selectKey")
|
||||
);
|
||||
|
||||
KafkaStreamSourceService.logIfEnabled(
|
||||
log,
|
||||
workerTaskResultKStream,
|
||||
(key, value) -> executorService.log(log, false, value),
|
||||
name
|
||||
)
|
||||
.to(
|
||||
kafkaAdminService.getTopicName(WorkerTaskResult.class),
|
||||
Produced.with(Serdes.String(), JsonSerde.of(WorkerTaskResult.class)).withName(name + ".toWorkerTaskResult")
|
||||
);
|
||||
}
|
||||
|
||||
private void purgeWorkerRunning(KStream<String, WorkerTaskResult> workerTaskResultKStream) {
|
||||
workerTaskResultKStream
|
||||
.filter((key, value) -> value.getTaskRun().getState().isTerminated(), Named.as("PurgeWorkerRunning.filterTerminated"))
|
||||
.mapValues((readOnlyKey, value) -> (WorkerTaskRunning)null, Named.as("PurgeWorkerRunning.toNull"))
|
||||
.to(
|
||||
kafkaAdminService.getTopicName(WorkerTaskRunning.class),
|
||||
Produced.with(Serdes.String(), JsonSerde.of(WorkerTaskRunning.class)).withName("PurgeWorkerRunning.toWorkerTaskRunning")
|
||||
);
|
||||
}
|
||||
|
||||
private void toExecution(KStream<String, Executor> stream, String name) {
|
||||
KStream<String, Executor> streamFrom = stream
|
||||
.filter((key, value) -> value.isExecutionUpdated(), Named.as(name + "ToExecution.haveFrom"))
|
||||
.transformValues(
|
||||
ExecutorAddHeaderTransformer::new,
|
||||
Named.as(name + "ToExecution.addHeaders")
|
||||
);
|
||||
|
||||
// send execution
|
||||
KStream<String, Executor> executionKStream = streamFrom
|
||||
.filter((key, value) -> value.getException() == null, Named.as(name + "ToExecutionExecution.notException"));
|
||||
|
||||
toExecutionSend(executionKStream, name + "ToExecutionExecution");
|
||||
|
||||
// send exception
|
||||
KStream<String, Pair<Executor, Execution.FailedExecutionWithLog>> failedStream = streamFrom
|
||||
.filter((key, value) -> value.getException() != null, Named.as(name + "ToExecutionException.isException"))
|
||||
.mapValues(
|
||||
e -> Pair.of(e, e.getExecution().failedExecutionFromExecutor(e.getException())),
|
||||
Named.as(name + "ToExecutionException.mapToFailedExecutionWithLog")
|
||||
);
|
||||
|
||||
failedStream
|
||||
.flatMapValues(e -> e.getRight().getLogs(), Named.as(name + "ToExecutionException.flatmapLogs"))
|
||||
.selectKey((key, value) -> (String)null, Named.as(name + "ToExecutionException.removeKey"))
|
||||
.to(
|
||||
kafkaAdminService.getTopicName(LogEntry.class),
|
||||
Produced.with(Serdes.String(), JsonSerde.of(LogEntry.class)).withName(name + "ToExecutionException.toLogEntry")
|
||||
);
|
||||
|
||||
KStream<String, Executor> executorFailedKStream = failedStream
|
||||
.mapValues(
|
||||
e -> e.getLeft().withExecution(e.getRight().getExecution(), "failedExecutionFromExecutor"),
|
||||
Named.as(name + "ToExecutionException.mapToExecutor")
|
||||
);
|
||||
|
||||
toExecutionSend(executorFailedKStream, name + "ToExecutionException");
|
||||
}
|
||||
|
||||
private void toExecutionSend(KStream<String, Executor> stream, String from) {
|
||||
stream = KafkaStreamSourceService.logIfEnabled(
|
||||
log,
|
||||
stream,
|
||||
(key, value) -> executorService.log(log, false, value),
|
||||
from
|
||||
);
|
||||
|
||||
stream
|
||||
.transformValues(
|
||||
() -> new StateStoreTransformer<>(EXECUTOR_STATE_STORE_NAME, Executor::serialize),
|
||||
Named.as(from + ".store"),
|
||||
EXECUTOR_STATE_STORE_NAME
|
||||
)
|
||||
.mapValues((readOnlyKey, value) -> value.getExecution(), Named.as(from + ".mapToExecution"))
|
||||
.to(
|
||||
kafkaAdminService.getTopicName(Execution.class),
|
||||
Produced.with(Serdes.String(), JsonSerde.of(Execution.class)).withName(from + ".toExecution")
|
||||
);
|
||||
}
|
||||
|
||||
@NoArgsConstructor
|
||||
@Getter
|
||||
public static class WorkerTaskResultState {
|
||||
Map<String, WorkerTaskResult> results = new HashMap<>();
|
||||
}
|
||||
|
||||
@AllArgsConstructor
|
||||
@Getter
|
||||
public static class WorkerTaskRunningWithWorkerTaskRunning {
|
||||
WorkerInstance workerInstance;
|
||||
WorkerTaskRunning workerTaskRunning;
|
||||
}
|
||||
|
||||
@NoArgsConstructor
|
||||
@Getter
|
||||
public static class WorkerTaskRunningState {
|
||||
Map<String, WorkerTaskRunning> workerTaskRunnings = new HashMap<>();
|
||||
}
|
||||
}
|
||||
@@ -1,70 +0,0 @@
|
||||
package io.kestra.runner.kafka.executors;
|
||||
|
||||
import io.kestra.core.models.flows.Flow;
|
||||
import io.kestra.core.models.templates.Template;
|
||||
import io.kestra.runner.kafka.KafkaFlowExecutor;
|
||||
import io.kestra.runner.kafka.KafkaQueueEnabled;
|
||||
import io.kestra.runner.kafka.KafkaTemplateExecutor;
|
||||
import io.kestra.runner.kafka.serializers.JsonSerde;
|
||||
import io.kestra.runner.kafka.services.KafkaAdminService;
|
||||
import io.kestra.runner.kafka.services.KafkaStreamsBuilder;
|
||||
import io.kestra.runner.kafka.streams.GlobalInMemoryStateProcessor;
|
||||
import jakarta.inject.Inject;
|
||||
import jakarta.inject.Singleton;
|
||||
import lombok.extern.slf4j.Slf4j;
|
||||
import org.apache.kafka.common.serialization.Serdes;
|
||||
import org.apache.kafka.streams.StreamsBuilder;
|
||||
import org.apache.kafka.streams.kstream.Consumed;
|
||||
import org.apache.kafka.streams.state.Stores;
|
||||
|
||||
@KafkaQueueEnabled
|
||||
@Singleton
|
||||
@Slf4j
|
||||
public class ExecutorStore implements KafkaExecutorInterface {
|
||||
public static final String FLOW_STATE_STORE_NAME = "flow";
|
||||
public static final String TEMPLATE_STATE_STORE_NAME = "template";
|
||||
|
||||
@Inject
|
||||
private KafkaAdminService kafkaAdminService;
|
||||
|
||||
@Inject
|
||||
private KafkaFlowExecutor kafkaFlowExecutor;
|
||||
|
||||
@Inject
|
||||
private KafkaTemplateExecutor kafkaTemplateExecutor;
|
||||
|
||||
public StreamsBuilder topology() {
|
||||
StreamsBuilder builder = new KafkaStreamsBuilder();
|
||||
|
||||
builder.addGlobalStore(
|
||||
Stores.keyValueStoreBuilder(
|
||||
Stores.persistentKeyValueStore(FLOW_STATE_STORE_NAME),
|
||||
Serdes.String(),
|
||||
JsonSerde.of(Flow.class)
|
||||
),
|
||||
kafkaAdminService.getTopicName(Flow.class),
|
||||
Consumed.with(Serdes.String(), JsonSerde.of(Flow.class)).withName("GlobalStore.Flow"),
|
||||
() -> new GlobalInMemoryStateProcessor<>(
|
||||
FLOW_STATE_STORE_NAME,
|
||||
flows -> kafkaFlowExecutor.setFlows(flows),
|
||||
store -> kafkaFlowExecutor.setStore(store)
|
||||
)
|
||||
);
|
||||
|
||||
builder.addGlobalStore(
|
||||
Stores.keyValueStoreBuilder(
|
||||
Stores.persistentKeyValueStore(TEMPLATE_STATE_STORE_NAME),
|
||||
Serdes.String(),
|
||||
JsonSerde.of(Template.class)
|
||||
),
|
||||
kafkaAdminService.getTopicName(Template.class),
|
||||
Consumed.with(Serdes.String(), JsonSerde.of(Template.class)).withName("GlobalStore.Template"),
|
||||
() -> new GlobalInMemoryStateProcessor<>(
|
||||
TEMPLATE_STATE_STORE_NAME,
|
||||
templates -> kafkaTemplateExecutor.setTemplates(templates)
|
||||
)
|
||||
);
|
||||
|
||||
return builder;
|
||||
}
|
||||
}
|
||||
@@ -1,74 +0,0 @@
|
||||
package io.kestra.runner.kafka.executors;
|
||||
|
||||
import io.kestra.core.models.triggers.Trigger;
|
||||
import io.kestra.core.queues.QueueService;
|
||||
import io.kestra.core.runners.Executor;
|
||||
import io.kestra.core.services.ConditionService;
|
||||
import io.kestra.runner.kafka.KafkaQueueEnabled;
|
||||
import io.kestra.runner.kafka.serializers.JsonSerde;
|
||||
import io.kestra.runner.kafka.services.KafkaAdminService;
|
||||
import io.kestra.runner.kafka.services.KafkaStreamSourceService;
|
||||
import io.kestra.runner.kafka.services.KafkaStreamsBuilder;
|
||||
import jakarta.inject.Inject;
|
||||
import jakarta.inject.Singleton;
|
||||
import lombok.extern.slf4j.Slf4j;
|
||||
import org.apache.kafka.common.serialization.Serdes;
|
||||
import org.apache.kafka.streams.StreamsBuilder;
|
||||
import org.apache.kafka.streams.kstream.GlobalKTable;
|
||||
import org.apache.kafka.streams.kstream.KStream;
|
||||
import org.apache.kafka.streams.kstream.Named;
|
||||
import org.apache.kafka.streams.kstream.Produced;
|
||||
|
||||
@KafkaQueueEnabled
|
||||
@Singleton
|
||||
@Slf4j
|
||||
public class ExecutorTriggerCleaner implements KafkaExecutorInterface {
|
||||
@Inject
|
||||
private KafkaAdminService kafkaAdminService;
|
||||
|
||||
@Inject
|
||||
private KafkaStreamSourceService kafkaStreamSourceService;
|
||||
|
||||
@Inject
|
||||
private QueueService queueService;
|
||||
|
||||
@Inject
|
||||
private ConditionService conditionService;
|
||||
|
||||
public StreamsBuilder topology() {
|
||||
StreamsBuilder builder = new KafkaStreamsBuilder();
|
||||
|
||||
KStream<String, Executor> executorKStream = kafkaStreamSourceService.executorKStream(builder);
|
||||
|
||||
KStream<String, Executor> executionWithFlowKStream = kafkaStreamSourceService.executorWithFlow(executorKStream, false);
|
||||
|
||||
GlobalKTable<String, Trigger> triggerGlobalKTable = kafkaStreamSourceService.triggerGlobalKTable(builder);
|
||||
|
||||
executionWithFlowKStream
|
||||
.filter(
|
||||
(key, value) -> value.getExecution().getTrigger() != null,
|
||||
Named.as("TriggerCleaner.hasTriggerFilter")
|
||||
)
|
||||
.filter(
|
||||
(key, value) -> value.getExecution().isDeleted() || conditionService.isTerminatedWithListeners(value.getFlow(), value.getExecution()),
|
||||
Named.as("TriggerCleaner.terminatedFilter")
|
||||
)
|
||||
.join(
|
||||
triggerGlobalKTable,
|
||||
(key, executionWithFlow) -> Trigger.uid(executionWithFlow.getExecution()),
|
||||
(execution, trigger) -> trigger.resetExecution(),
|
||||
Named.as("TriggerCleaner.join")
|
||||
)
|
||||
.selectKey(
|
||||
(key, value) -> queueService.key(value),
|
||||
Named.as("TriggerCleaner.selectKey")
|
||||
)
|
||||
.to(
|
||||
kafkaAdminService.getTopicName(Trigger.class),
|
||||
Produced.with(Serdes.String(), JsonSerde.of(Trigger.class))
|
||||
.withName("To.Trigger")
|
||||
);
|
||||
|
||||
return builder;
|
||||
}
|
||||
}
|
||||
@@ -1,140 +0,0 @@
|
||||
package io.kestra.runner.kafka.executors;
|
||||
|
||||
import io.kestra.core.runners.ExecutorService;
|
||||
import io.kestra.core.runners.WorkerInstance;
|
||||
import io.kestra.core.runners.WorkerTask;
|
||||
import io.kestra.core.runners.WorkerTaskRunning;
|
||||
import io.kestra.runner.kafka.KafkaQueueEnabled;
|
||||
import io.kestra.runner.kafka.serializers.JsonSerde;
|
||||
import io.kestra.runner.kafka.services.KafkaAdminService;
|
||||
import io.kestra.runner.kafka.services.KafkaStreamSourceService;
|
||||
import io.kestra.runner.kafka.services.KafkaStreamsBuilder;
|
||||
import io.kestra.runner.kafka.streams.GlobalStateProcessor;
|
||||
import io.kestra.runner.kafka.streams.WorkerInstanceTransformer;
|
||||
import jakarta.inject.Inject;
|
||||
import jakarta.inject.Singleton;
|
||||
import lombok.extern.slf4j.Slf4j;
|
||||
import org.apache.kafka.common.serialization.Serdes;
|
||||
import org.apache.kafka.common.utils.Bytes;
|
||||
import org.apache.kafka.streams.KeyValue;
|
||||
import org.apache.kafka.streams.StreamsBuilder;
|
||||
import org.apache.kafka.streams.kstream.*;
|
||||
import org.apache.kafka.streams.state.KeyValueStore;
|
||||
import org.apache.kafka.streams.state.Stores;
|
||||
|
||||
@KafkaQueueEnabled
|
||||
@Singleton
|
||||
@Slf4j
|
||||
public class ExecutorWorkerRunning implements KafkaExecutorInterface {
|
||||
public static final String WORKERINSTANCE_STATE_STORE_NAME = "worker_instance";
|
||||
public static final String TOPIC_EXECUTOR_WORKERINSTANCE = "executorworkerinstance";
|
||||
public static final String WORKER_RUNNING_STATE_STORE_NAME = "worker_running";
|
||||
|
||||
@Inject
|
||||
private KafkaAdminService kafkaAdminService;
|
||||
|
||||
@Inject
|
||||
private ExecutorService executorService;
|
||||
|
||||
public StreamsBuilder topology() {
|
||||
StreamsBuilder builder = new KafkaStreamsBuilder();
|
||||
|
||||
builder.addGlobalStore(
|
||||
Stores.keyValueStoreBuilder(
|
||||
Stores.persistentKeyValueStore(WORKERINSTANCE_STATE_STORE_NAME),
|
||||
Serdes.String(),
|
||||
JsonSerde.of(WorkerInstance.class)
|
||||
),
|
||||
kafkaAdminService.getTopicName(KafkaStreamSourceService.TOPIC_EXECUTOR_WORKERINSTANCE),
|
||||
Consumed.with(Serdes.String(), JsonSerde.of(WorkerInstance.class)).withName("GlobalStore.ExecutorWorkerInstance"),
|
||||
() -> new GlobalStateProcessor<>(WORKERINSTANCE_STATE_STORE_NAME)
|
||||
);
|
||||
|
||||
// only used as state store
|
||||
builder
|
||||
.globalTable(
|
||||
kafkaAdminService.getTopicName(WorkerTaskRunning.class),
|
||||
Consumed.with(Serdes.String(), JsonSerde.of(WorkerTaskRunning.class)).withName("GlobalKTable.WorkerTaskRunning"),
|
||||
Materialized.<String, WorkerTaskRunning, KeyValueStore<Bytes, byte[]>>as(WORKER_RUNNING_STATE_STORE_NAME)
|
||||
.withKeySerde(Serdes.String())
|
||||
.withValueSerde(JsonSerde.of(WorkerTaskRunning.class))
|
||||
);
|
||||
|
||||
KStream<String, WorkerInstance> workerInstanceKStream = builder
|
||||
.stream(
|
||||
kafkaAdminService.getTopicName(WorkerInstance.class),
|
||||
Consumed.with(Serdes.String(), JsonSerde.of(WorkerInstance.class)).withName("KStream.WorkerInstance")
|
||||
);
|
||||
|
||||
workerInstanceKStream
|
||||
.to(
|
||||
kafkaAdminService.getTopicName(TOPIC_EXECUTOR_WORKERINSTANCE),
|
||||
Produced.with(Serdes.String(), JsonSerde.of(WorkerInstance.class)).withName("DetectNewWorker.toExecutorWorkerInstance")
|
||||
);
|
||||
|
||||
KStream<String, WorkerInstanceTransformer.Result> stream = workerInstanceKStream
|
||||
.transformValues(
|
||||
WorkerInstanceTransformer::new,
|
||||
Named.as("DetectNewWorker.workerInstanceTransformer")
|
||||
)
|
||||
.flatMapValues((readOnlyKey, value) -> value, Named.as("DetectNewWorker.flapMapList"));
|
||||
|
||||
// we resend the worker task from evicted worker
|
||||
KStream<String, WorkerTask> resultWorkerTask = stream
|
||||
.flatMapValues(
|
||||
(readOnlyKey, value) -> value.getWorkerTasksToSend(),
|
||||
Named.as("DetectNewWorkerTask.flapMapWorkerTaskToSend")
|
||||
);
|
||||
|
||||
// and remove from running since already sent
|
||||
resultWorkerTask
|
||||
.map((key, value) -> KeyValue.pair(value.getTaskRun().getId(), (WorkerTaskRunning)null), Named.as("DetectNewWorkerTask.workerTaskRunningToNull"))
|
||||
.to(
|
||||
kafkaAdminService.getTopicName(WorkerTaskRunning.class),
|
||||
Produced.with(Serdes.String(), JsonSerde.of(WorkerTaskRunning.class)).withName("DetectNewWorker.toWorkerTaskRunning")
|
||||
);
|
||||
|
||||
KafkaStreamSourceService.logIfEnabled(
|
||||
log,
|
||||
resultWorkerTask,
|
||||
(key, value) -> executorService.log(log, false, value),
|
||||
"DetectNewWorkerTask"
|
||||
)
|
||||
.to(
|
||||
kafkaAdminService.getTopicName(WorkerTask.class),
|
||||
Produced.with(Serdes.String(), JsonSerde.of(WorkerTask.class)).withName("DetectNewWorkerTask.toWorkerTask")
|
||||
);
|
||||
|
||||
// we resend the WorkerInstance update
|
||||
KStream<String, WorkerInstance> updatedStream = KafkaStreamSourceService.logIfEnabled(
|
||||
log,
|
||||
stream,
|
||||
(key, value) -> log.debug(
|
||||
"Instance updated: {}",
|
||||
value
|
||||
),
|
||||
"DetectNewWorkerInstance"
|
||||
)
|
||||
.map(
|
||||
(key, value) -> value.getWorkerInstanceUpdated(),
|
||||
Named.as("DetectNewWorkerInstance.mapInstance")
|
||||
);
|
||||
|
||||
// cleanup executor workerinstance state store
|
||||
updatedStream
|
||||
.filter((key, value) -> value != null, Named.as("DetectNewWorkerInstance.filterNotNull"))
|
||||
.to(
|
||||
kafkaAdminService.getTopicName(TOPIC_EXECUTOR_WORKERINSTANCE),
|
||||
Produced.with(Serdes.String(), JsonSerde.of(WorkerInstance.class)).withName("DetectNewWorkerInstance.toExecutorWorkerInstance")
|
||||
);
|
||||
|
||||
updatedStream
|
||||
.to(
|
||||
kafkaAdminService.getTopicName(WorkerInstance.class),
|
||||
Produced.with(Serdes.String(), JsonSerde.of(WorkerInstance.class)).withName("DetectNewWorkerInstance.toWorkerInstance")
|
||||
);
|
||||
|
||||
|
||||
return builder;
|
||||
}
|
||||
}
|
||||
@@ -1,15 +0,0 @@
|
||||
package io.kestra.runner.kafka.executors;
|
||||
|
||||
import io.kestra.runner.kafka.KafkaQueueEnabled;
|
||||
import io.kestra.runner.kafka.services.KafkaStreamService;
|
||||
import io.micronaut.context.ApplicationContext;
|
||||
import org.apache.kafka.streams.StreamsBuilder;
|
||||
|
||||
@KafkaQueueEnabled
|
||||
public interface KafkaExecutorInterface {
|
||||
StreamsBuilder topology();
|
||||
|
||||
default void onCreated(ApplicationContext applicationContext, KafkaStreamService.Stream stream) {
|
||||
// no op
|
||||
}
|
||||
}
|
||||
@@ -1,52 +0,0 @@
|
||||
package io.kestra.runner.kafka.serializers;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import org.apache.kafka.common.errors.SerializationException;
|
||||
import org.apache.kafka.common.serialization.Deserializer;
|
||||
import io.kestra.core.serializers.JacksonMapper;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
|
||||
public class JsonDeserializer<T> implements Deserializer<T> {
|
||||
private static final ObjectMapper mapper = JacksonMapper.ofJson(false);
|
||||
|
||||
private Class<T> cls;
|
||||
private boolean strict;
|
||||
|
||||
|
||||
public JsonDeserializer(Class<T> cls) {
|
||||
super();
|
||||
|
||||
this.cls = cls;
|
||||
this.strict = true;
|
||||
}
|
||||
|
||||
public JsonDeserializer(Class<T> cls, boolean strict) {
|
||||
super();
|
||||
|
||||
this.cls = cls;
|
||||
this.strict = strict;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void configure(Map<String, ?> settings, boolean isKey) {
|
||||
}
|
||||
|
||||
@Override
|
||||
public T deserialize(String topic, byte[] bytes) {
|
||||
if (null == bytes) {
|
||||
return null;
|
||||
}
|
||||
|
||||
try {
|
||||
return mapper.readValue(bytes, this.cls);
|
||||
} catch (IOException e) {
|
||||
if (strict) {
|
||||
throw new SerializationException(e);
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -1,48 +0,0 @@
|
||||
package io.kestra.runner.kafka.serializers;
|
||||
|
||||
import org.apache.kafka.common.serialization.Deserializer;
|
||||
import org.apache.kafka.common.serialization.Serde;
|
||||
import org.apache.kafka.common.serialization.Serializer;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
public class JsonSerde<T> implements Serde<T> {
|
||||
private final Serializer<T> serializer;
|
||||
private final Deserializer<T> deserializer;
|
||||
|
||||
private JsonSerde(Class<T> cls, boolean strict) {
|
||||
this.deserializer = new JsonDeserializer<>(cls, strict);
|
||||
this.serializer = new JsonSerializer<>();
|
||||
}
|
||||
|
||||
public static <T> JsonSerde<T> of(Class<T> cls) {
|
||||
return new JsonSerde<>(cls, true);
|
||||
}
|
||||
|
||||
public static <T> JsonSerde<T> of(Class<T> cls, boolean strict) {
|
||||
return new JsonSerde<>(cls, strict);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void configure(Map<String, ?> settings, boolean isKey) {
|
||||
this.serializer.configure(settings, isKey);
|
||||
this.deserializer.configure(settings, isKey);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
this.deserializer.close();
|
||||
this.serializer.close();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Serializer<T> serializer() {
|
||||
return this.serializer;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Deserializer<T> deserializer() {
|
||||
return this.deserializer;
|
||||
}
|
||||
}
|
||||
@@ -1,37 +0,0 @@
|
||||
package io.kestra.runner.kafka.serializers;
|
||||
|
||||
import com.fasterxml.jackson.core.JsonProcessingException;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import org.apache.kafka.common.errors.SerializationException;
|
||||
import org.apache.kafka.common.serialization.Serializer;
|
||||
import io.kestra.core.serializers.JacksonMapper;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
public class JsonSerializer<T> extends JacksonMapper implements Serializer<T> {
|
||||
private static final ObjectMapper mapper = JacksonMapper.ofJson(false);
|
||||
|
||||
public JsonSerializer() {
|
||||
super();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void configure(Map<String, ?> settings, boolean isKey) {
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] serialize(String topic, T message) {
|
||||
if (null == message) {
|
||||
return null;
|
||||
}
|
||||
try {
|
||||
return mapper.writeValueAsBytes(message);
|
||||
} catch (JsonProcessingException e) {
|
||||
throw new SerializationException(e);
|
||||
}
|
||||
}
|
||||
|
||||
public byte[] serialize(T message) {
|
||||
return this.serialize("", message);
|
||||
}
|
||||
}
|
||||
@@ -1,184 +0,0 @@
|
||||
package io.kestra.runner.kafka.services;
|
||||
|
||||
import io.micrometer.core.instrument.Tag;
|
||||
import io.micrometer.core.instrument.binder.kafka.KafkaClientMetrics;
|
||||
import io.micronaut.context.annotation.Value;
|
||||
import lombok.extern.slf4j.Slf4j;
|
||||
import org.apache.kafka.clients.CommonClientConfigs;
|
||||
import org.apache.kafka.clients.admin.AdminClient;
|
||||
import org.apache.kafka.clients.admin.ConfigEntry;
|
||||
import org.apache.kafka.clients.admin.NewTopic;
|
||||
import org.apache.kafka.common.config.ConfigResource;
|
||||
import org.apache.kafka.common.errors.TimeoutException;
|
||||
import org.apache.kafka.common.errors.TopicExistsException;
|
||||
import io.kestra.core.metrics.MetricRegistry;
|
||||
import io.kestra.runner.kafka.configs.ClientConfig;
|
||||
import io.kestra.runner.kafka.configs.TopicDefaultsConfig;
|
||||
import io.kestra.runner.kafka.configs.TopicsConfig;
|
||||
|
||||
import java.util.*;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.stream.Collectors;
|
||||
import javax.annotation.PreDestroy;
|
||||
import jakarta.inject.Inject;
|
||||
import jakarta.inject.Singleton;
|
||||
|
||||
@Singleton
|
||||
@Slf4j
|
||||
public class KafkaAdminService implements AutoCloseable {
|
||||
@Inject
|
||||
private TopicDefaultsConfig topicDefaultsConfig;
|
||||
|
||||
@Inject
|
||||
private List<TopicsConfig> topicsConfig;
|
||||
|
||||
@Inject
|
||||
private ClientConfig clientConfig;
|
||||
|
||||
@Inject
|
||||
private MetricRegistry metricRegistry;
|
||||
|
||||
private AdminClient adminClient;
|
||||
|
||||
private KafkaClientMetrics kafkaClientMetrics;
|
||||
|
||||
@Value("${kestra.server.metrics.kafka.admin:true}")
|
||||
protected Boolean metricsEnabled;
|
||||
|
||||
public AdminClient of() {
|
||||
if (this.adminClient == null) {
|
||||
Properties properties = new Properties();
|
||||
properties.putAll(clientConfig.getProperties());
|
||||
|
||||
properties.put(CommonClientConfigs.CLIENT_ID_CONFIG, "default");
|
||||
|
||||
adminClient = AdminClient.create(properties);
|
||||
|
||||
if (metricsEnabled) {
|
||||
kafkaClientMetrics = new KafkaClientMetrics(
|
||||
adminClient,
|
||||
List.of(
|
||||
Tag.of("client_type", "admin"),
|
||||
Tag.of("client_class_id", (String) properties.get(CommonClientConfigs.CLIENT_ID_CONFIG))
|
||||
)
|
||||
);
|
||||
metricRegistry.bind(kafkaClientMetrics);
|
||||
}
|
||||
}
|
||||
|
||||
return adminClient;
|
||||
}
|
||||
|
||||
@PreDestroy
|
||||
@Override
|
||||
public void close() {
|
||||
if (adminClient != null) {
|
||||
adminClient.close();
|
||||
}
|
||||
|
||||
if (kafkaClientMetrics != null) {
|
||||
kafkaClientMetrics.close();
|
||||
}
|
||||
}
|
||||
|
||||
private TopicsConfig getTopicConfig(Class<?> cls) {
|
||||
return this.topicsConfig
|
||||
.stream()
|
||||
.filter(r -> r.getCls() == cls)
|
||||
.findFirst()
|
||||
.orElseThrow(() -> new NoSuchElementException("Invalid class '" + cls.getName() + "'"));
|
||||
}
|
||||
|
||||
private TopicsConfig getTopicConfig(String key) {
|
||||
return this.topicsConfig
|
||||
.stream()
|
||||
.filter(r -> r.getKey().equals(key))
|
||||
.findFirst()
|
||||
.orElseThrow(() -> new NoSuchElementException("Invalid key '" + key + "'"));
|
||||
}
|
||||
|
||||
public void createIfNotExist(String key) {
|
||||
this.createIfNotExist(this.getTopicConfig(key));
|
||||
}
|
||||
|
||||
public void createIfNotExist(Class<?> cls) {
|
||||
this.createIfNotExist(this.getTopicConfig(cls));
|
||||
}
|
||||
|
||||
@SuppressWarnings("deprecation")
|
||||
public void createIfNotExist(TopicsConfig topicConfig) {
|
||||
NewTopic newTopic = new NewTopic(
|
||||
topicConfig.getName(),
|
||||
topicConfig.getPartitions() != null ? topicConfig.getPartitions() : topicDefaultsConfig.getPartitions(),
|
||||
topicConfig.getReplicationFactor() != null ? topicConfig.getReplicationFactor() : topicDefaultsConfig.getReplicationFactor()
|
||||
);
|
||||
|
||||
Map<String, String> properties = new HashMap<>();
|
||||
|
||||
if (topicDefaultsConfig.getProperties() != null) {
|
||||
properties.putAll(topicDefaultsConfig.getProperties());
|
||||
}
|
||||
|
||||
if (topicConfig.getProperties() != null) {
|
||||
properties.putAll(topicConfig.getProperties());
|
||||
}
|
||||
|
||||
newTopic.configs(properties);
|
||||
|
||||
try {
|
||||
this.of().createTopics(Collections.singletonList(newTopic)).all().get();
|
||||
log.info("Topic '{}' created", newTopic.name());
|
||||
} catch (ExecutionException | InterruptedException | TimeoutException e) {
|
||||
if (e.getCause() instanceof TopicExistsException) {
|
||||
try {
|
||||
adminClient
|
||||
.alterConfigs(new HashMap<>() {{
|
||||
put(
|
||||
new ConfigResource(ConfigResource.Type.TOPIC, newTopic.name()),
|
||||
new org.apache.kafka.clients.admin.Config(
|
||||
newTopic.configs()
|
||||
.entrySet()
|
||||
.stream()
|
||||
.map(config -> new ConfigEntry(config.getKey(), config.getValue()))
|
||||
.collect(Collectors.toList())
|
||||
)
|
||||
);
|
||||
}}).all().get();
|
||||
|
||||
log.info("Topic Config '{}' updated", newTopic.name());
|
||||
} catch (ExecutionException | InterruptedException exception) {
|
||||
if (!(exception.getCause() instanceof TopicExistsException)) {
|
||||
log.warn("Unable to update topic '{}'", newTopic.name(), exception);
|
||||
}
|
||||
}
|
||||
} else {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void delete(String key) {
|
||||
this.delete(this.getTopicConfig(key));
|
||||
}
|
||||
|
||||
public void delete(Class<?> cls) {
|
||||
this.delete(this.getTopicConfig(cls));
|
||||
}
|
||||
|
||||
public void delete(TopicsConfig topicConfig) {
|
||||
try {
|
||||
this.of().deleteTopics(Collections.singletonList(topicConfig.getName())).all().get();
|
||||
log.info("Topic '{}' deleted", topicConfig.getName());
|
||||
} catch (ExecutionException | InterruptedException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
public String getTopicName(Class<?> cls) {
|
||||
return this.getTopicConfig(cls).getName();
|
||||
}
|
||||
|
||||
public String getTopicName(String key) {
|
||||
return this.getTopicConfig(key).getName();
|
||||
}
|
||||
}
|
||||
@@ -1,21 +0,0 @@
|
||||
package io.kestra.runner.kafka.services;
|
||||
|
||||
import com.google.common.base.CaseFormat;
|
||||
import io.micronaut.context.annotation.Value;
|
||||
import lombok.extern.slf4j.Slf4j;
|
||||
|
||||
import jakarta.inject.Singleton;
|
||||
|
||||
@Singleton
|
||||
@Slf4j
|
||||
public class KafkaConfigService {
|
||||
@Value("${kestra.kafka.defaults.consumer-prefix:kestra_}")
|
||||
private String consumerPrefix;
|
||||
|
||||
public String getConsumerGroupName(Class<?> group) {
|
||||
return this.consumerPrefix +
|
||||
CaseFormat.UPPER_CAMEL.to(CaseFormat.LOWER_UNDERSCORE,
|
||||
group.getSimpleName().replace("Kafka", "")
|
||||
);
|
||||
}
|
||||
}
|
||||
@@ -1,185 +0,0 @@
|
||||
package io.kestra.runner.kafka.services;
|
||||
|
||||
import io.kestra.core.metrics.MetricRegistry;
|
||||
import io.kestra.runner.kafka.ConsumerInterceptor;
|
||||
import io.kestra.runner.kafka.configs.ClientConfig;
|
||||
import io.kestra.runner.kafka.configs.ConsumerDefaultsConfig;
|
||||
import io.micrometer.core.instrument.Tag;
|
||||
import io.micrometer.core.instrument.binder.kafka.KafkaClientMetrics;
|
||||
import io.micronaut.context.ApplicationContext;
|
||||
import io.micronaut.context.annotation.Value;
|
||||
import lombok.extern.slf4j.Slf4j;
|
||||
import org.apache.kafka.clients.CommonClientConfigs;
|
||||
import org.apache.kafka.clients.consumer.*;
|
||||
import org.apache.kafka.common.TopicPartition;
|
||||
import org.apache.kafka.common.serialization.Serde;
|
||||
import org.apache.kafka.common.serialization.StringDeserializer;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.time.Duration;
|
||||
import java.util.*;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.StreamSupport;
|
||||
import jakarta.inject.Inject;
|
||||
import jakarta.inject.Singleton;
|
||||
|
||||
@Singleton
|
||||
@Slf4j
|
||||
public class KafkaConsumerService {
|
||||
@Inject
|
||||
private ClientConfig clientConfig;
|
||||
|
||||
@Inject
|
||||
private ApplicationContext applicationContext;
|
||||
|
||||
@Inject
|
||||
private ConsumerDefaultsConfig consumerConfig;
|
||||
|
||||
@Inject
|
||||
private KafkaConfigService kafkaConfigService;
|
||||
|
||||
@Inject
|
||||
private MetricRegistry metricRegistry;
|
||||
|
||||
@Value("${kestra.server.metrics.kafka.consumer:true}")
|
||||
protected Boolean metricsEnabled;
|
||||
|
||||
public <V> org.apache.kafka.clients.consumer.Consumer<String, V> of(Class<?> clientId, Serde<V> serde, Class<?> group) {
|
||||
return of(clientId, serde, null, group);
|
||||
}
|
||||
|
||||
public <V> org.apache.kafka.clients.consumer.Consumer<String, V> of(
|
||||
Class<?> clientId,
|
||||
Serde<V> serde,
|
||||
ConsumerRebalanceListener consumerRebalanceListener,
|
||||
Class<?> group
|
||||
) {
|
||||
Properties props = new Properties();
|
||||
props.putAll(clientConfig.getProperties());
|
||||
|
||||
if (this.consumerConfig.getProperties() != null) {
|
||||
props.putAll(consumerConfig.getProperties());
|
||||
}
|
||||
|
||||
props.put(CommonClientConfigs.CLIENT_ID_CONFIG, clientId.getName());
|
||||
props.put(KafkaStreamService.APPLICATION_CONTEXT_CONFIG, applicationContext);
|
||||
|
||||
if (group != null) {
|
||||
props.put(ConsumerConfig.GROUP_ID_CONFIG, kafkaConfigService.getConsumerGroupName(group));
|
||||
} else {
|
||||
props.remove(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG);
|
||||
}
|
||||
|
||||
// interceptor
|
||||
if (clientConfig.getLoggers() != null) {
|
||||
props.put(
|
||||
ConsumerConfig.INTERCEPTOR_CLASSES_CONFIG,
|
||||
ConsumerInterceptor.class.getName()
|
||||
);
|
||||
}
|
||||
|
||||
return new Consumer<>(props, serde, metricsEnabled ? metricRegistry : null, consumerRebalanceListener);
|
||||
}
|
||||
|
||||
public static <T> Map<TopicPartition, OffsetAndMetadata> maxOffsets(ConsumerRecords<String, T> records) {
|
||||
return KafkaConsumerService.maxOffsets(
|
||||
StreamSupport
|
||||
.stream(records.spliterator(), false)
|
||||
.collect(Collectors.toList())
|
||||
);
|
||||
}
|
||||
|
||||
public static <T> Map<TopicPartition, OffsetAndMetadata> maxOffsets(List<ConsumerRecord<String, T>> records) {
|
||||
Map<TopicPartition, OffsetAndMetadata> results = new HashMap<>();
|
||||
|
||||
for (ConsumerRecord<String, T> record: records) {
|
||||
TopicPartition topicPartition = new TopicPartition(record.topic(), record.partition());
|
||||
results.compute(topicPartition, (current, offsetAndMetadata) -> {
|
||||
if (offsetAndMetadata == null || record.offset() + 1 > offsetAndMetadata.offset()) {
|
||||
return new OffsetAndMetadata(record.offset() + 1);
|
||||
} else {
|
||||
return offsetAndMetadata;
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
return results;
|
||||
}
|
||||
|
||||
public static class Consumer<V> extends KafkaConsumer<String, V> {
|
||||
protected Logger logger = LoggerFactory.getLogger(KafkaConsumerService.class);
|
||||
private KafkaClientMetrics metrics;
|
||||
private final ConsumerRebalanceListener consumerRebalanceListener;
|
||||
|
||||
private Consumer(Properties properties, Serde<V> valueSerde, MetricRegistry meterRegistry, ConsumerRebalanceListener consumerRebalanceListener) {
|
||||
super(properties, new StringDeserializer(), valueSerde.deserializer());
|
||||
|
||||
if (meterRegistry != null) {
|
||||
metrics = new KafkaClientMetrics(
|
||||
this,
|
||||
List.of(
|
||||
Tag.of("client_type", "consumer"),
|
||||
Tag.of("client_class_id", (String) properties.get(CommonClientConfigs.CLIENT_ID_CONFIG))
|
||||
)
|
||||
);
|
||||
meterRegistry.bind(metrics);
|
||||
}
|
||||
|
||||
this.consumerRebalanceListener = consumerRebalanceListener;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void subscribe(Collection<String> topics) {
|
||||
super.subscribe(topics, new ConsumerRebalanceListener() {
|
||||
@Override
|
||||
public void onPartitionsRevoked(Collection<TopicPartition> partitions) {
|
||||
if (consumerRebalanceListener != null) {
|
||||
consumerRebalanceListener.onPartitionsRevoked(partitions);
|
||||
}
|
||||
|
||||
if (log.isTraceEnabled()) {
|
||||
partitions.forEach(topicPartition -> logger.trace(
|
||||
"Revoke partitions for topic {}, partition {}",
|
||||
topicPartition.topic(),
|
||||
topicPartition.partition()
|
||||
));
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onPartitionsAssigned(Collection<TopicPartition> partitions) {
|
||||
if (consumerRebalanceListener != null) {
|
||||
consumerRebalanceListener.onPartitionsAssigned(partitions);
|
||||
}
|
||||
|
||||
if (log.isTraceEnabled()) {
|
||||
partitions.forEach(topicPartition -> logger.trace(
|
||||
"Switching partitions for topic {}, partition {}",
|
||||
topicPartition.topic(),
|
||||
topicPartition.partition()
|
||||
));
|
||||
}
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
if (metrics != null) {
|
||||
metrics.close();
|
||||
}
|
||||
|
||||
super.close();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close(Duration timeout) {
|
||||
if (metrics != null) {
|
||||
metrics.close();
|
||||
}
|
||||
|
||||
super.close(timeout);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -1,105 +0,0 @@
|
||||
package io.kestra.runner.kafka.services;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import io.kestra.core.metrics.MetricRegistry;
|
||||
import io.kestra.runner.kafka.ProducerInterceptor;
|
||||
import io.kestra.runner.kafka.configs.ClientConfig;
|
||||
import io.kestra.runner.kafka.configs.ProducerDefaultsConfig;
|
||||
import io.micrometer.core.instrument.Tag;
|
||||
import io.micrometer.core.instrument.binder.kafka.KafkaClientMetrics;
|
||||
import io.micronaut.context.ApplicationContext;
|
||||
import io.micronaut.context.annotation.Value;
|
||||
import org.apache.kafka.clients.CommonClientConfigs;
|
||||
import org.apache.kafka.clients.producer.KafkaProducer;
|
||||
import org.apache.kafka.clients.producer.ProducerConfig;
|
||||
import org.apache.kafka.common.serialization.Serde;
|
||||
import org.apache.kafka.common.serialization.StringSerializer;
|
||||
|
||||
import java.time.Duration;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
import jakarta.inject.Inject;
|
||||
import jakarta.inject.Singleton;
|
||||
|
||||
@Singleton
|
||||
public class KafkaProducerService {
|
||||
@Inject
|
||||
private ClientConfig clientConfig;
|
||||
|
||||
@Inject
|
||||
private ApplicationContext applicationContext;
|
||||
|
||||
@Inject
|
||||
private ProducerDefaultsConfig producerConfig;
|
||||
|
||||
@Inject
|
||||
private MetricRegistry metricRegistry;
|
||||
|
||||
@Value("${kestra.server.metrics.kafka.producer:true}")
|
||||
protected Boolean metricsEnabled;
|
||||
|
||||
public <V> KafkaProducerService.Producer<V> of(Class<?> clientId, Serde<V> serde) {
|
||||
return this.of(clientId, serde, ImmutableMap.of());
|
||||
}
|
||||
|
||||
public <V> KafkaProducerService.Producer<V> of(Class<?> clientId, Serde<V> serde, Map<String, String> properties) {
|
||||
Properties props = new Properties();
|
||||
props.putAll(clientConfig.getProperties());
|
||||
|
||||
if (producerConfig.getProperties() != null) {
|
||||
props.putAll(producerConfig.getProperties());
|
||||
}
|
||||
|
||||
props.putAll(properties);
|
||||
|
||||
props.put(CommonClientConfigs.CLIENT_ID_CONFIG, clientId.getName());
|
||||
props.put(KafkaStreamService.APPLICATION_CONTEXT_CONFIG, applicationContext);
|
||||
|
||||
if (clientConfig.getLoggers() != null) {
|
||||
props.put(
|
||||
ProducerConfig.INTERCEPTOR_CLASSES_CONFIG,
|
||||
ProducerInterceptor.class.getName()
|
||||
);
|
||||
}
|
||||
|
||||
return new Producer<>(props, serde, metricsEnabled ? metricRegistry : null);
|
||||
}
|
||||
|
||||
public static class Producer<V> extends KafkaProducer<String, V> {
|
||||
private KafkaClientMetrics metrics;
|
||||
|
||||
private Producer(Properties properties, Serde<V> valueSerde, MetricRegistry meterRegistry) {
|
||||
super(properties, new StringSerializer(), valueSerde.serializer());
|
||||
|
||||
if (metrics != null) {
|
||||
metrics = new KafkaClientMetrics(
|
||||
this,
|
||||
List.of(
|
||||
Tag.of("client_type", "producer"),
|
||||
Tag.of("client_class_id", (String) properties.get(CommonClientConfigs.CLIENT_ID_CONFIG))
|
||||
)
|
||||
);
|
||||
meterRegistry.bind(metrics);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
if (metrics != null) {
|
||||
metrics.close();
|
||||
}
|
||||
|
||||
super.close();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close(Duration timeout) {
|
||||
if (metrics != null) {
|
||||
metrics.close();
|
||||
}
|
||||
|
||||
super.close(timeout);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -1,279 +0,0 @@
|
||||
package io.kestra.runner.kafka.services;
|
||||
|
||||
import io.kestra.runner.kafka.*;
|
||||
import io.micrometer.core.instrument.Tag;
|
||||
import io.micrometer.core.instrument.binder.kafka.KafkaStreamsMetrics;
|
||||
import io.micronaut.context.ApplicationContext;
|
||||
import io.micronaut.context.annotation.Value;
|
||||
import io.micronaut.context.event.ApplicationEventPublisher;
|
||||
import lombok.extern.slf4j.Slf4j;
|
||||
import org.apache.commons.lang3.ArrayUtils;
|
||||
import org.apache.kafka.clients.CommonClientConfigs;
|
||||
import org.apache.kafka.clients.consumer.ConsumerConfig;
|
||||
import org.apache.kafka.clients.producer.ProducerConfig;
|
||||
import org.apache.kafka.common.TopicPartition;
|
||||
import org.apache.kafka.streams.KafkaStreams;
|
||||
import org.apache.kafka.streams.StreamsConfig;
|
||||
import org.apache.kafka.streams.Topology;
|
||||
import org.apache.kafka.streams.errors.StreamsException;
|
||||
import org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler;
|
||||
import org.apache.kafka.streams.processor.StateRestoreListener;
|
||||
import io.kestra.core.metrics.MetricRegistry;
|
||||
import io.kestra.runner.kafka.configs.ClientConfig;
|
||||
import io.kestra.runner.kafka.configs.StreamDefaultsConfig;
|
||||
import org.slf4j.Logger;
|
||||
|
||||
import java.io.File;
|
||||
import java.time.Duration;
|
||||
import java.util.List;
|
||||
import java.util.Properties;
|
||||
import jakarta.inject.Inject;
|
||||
import jakarta.inject.Singleton;
|
||||
import javax.validation.constraints.NotNull;
|
||||
|
||||
@Singleton
|
||||
@Slf4j
|
||||
public class KafkaStreamService {
|
||||
public static final String APPLICATION_CONTEXT_CONFIG = "application.context";
|
||||
|
||||
@Inject
|
||||
@NotNull
|
||||
private ClientConfig clientConfig;
|
||||
|
||||
@Inject
|
||||
private ApplicationContext applicationContext;
|
||||
|
||||
@Inject
|
||||
private StreamDefaultsConfig streamConfig;
|
||||
|
||||
@Inject
|
||||
private KafkaConfigService kafkaConfigService;
|
||||
|
||||
@Inject
|
||||
private ApplicationEventPublisher<KafkaStreamEndpoint.Event> eventPublisher;
|
||||
|
||||
@Inject
|
||||
private MetricRegistry metricRegistry;
|
||||
|
||||
@Value("${kestra.server.metrics.kafka.stream:true}")
|
||||
protected Boolean metricsEnabled;
|
||||
|
||||
public KafkaStreamService.Stream of(Class<?> clientId, Class<?> groupId, Topology topology) {
|
||||
return this.of(clientId, groupId, topology, new Properties());
|
||||
}
|
||||
|
||||
public KafkaStreamService.Stream of(Class<?> clientId, Class<?> groupId, Topology topology, Logger logger) {
|
||||
return this.of(clientId, groupId, topology, new Properties(), logger);
|
||||
}
|
||||
|
||||
public KafkaStreamService.Stream of(Class<?> clientId, Class<?> groupId, Topology topology, Properties properties) {
|
||||
return this.of(clientId, groupId, topology, properties, null);
|
||||
}
|
||||
|
||||
public KafkaStreamService.Stream of(Class<?> clientId, Class<?> groupId, Topology topology, Properties properties, Logger logger) {
|
||||
properties.putAll(clientConfig.getProperties());
|
||||
|
||||
if (this.streamConfig.getProperties() != null) {
|
||||
properties.putAll(streamConfig.getProperties());
|
||||
}
|
||||
|
||||
properties.put(CommonClientConfigs.CLIENT_ID_CONFIG, clientId.getName());
|
||||
properties.put(StreamsConfig.APPLICATION_ID_CONFIG, kafkaConfigService.getConsumerGroupName(groupId));
|
||||
|
||||
// hack, we send application context in order to use on exception handler
|
||||
properties.put(StreamsConfig.DEFAULT_PRODUCTION_EXCEPTION_HANDLER_CLASS_CONFIG, KafkaExecutorProductionExceptionHandler.class);
|
||||
properties.put(APPLICATION_CONTEXT_CONFIG, applicationContext);
|
||||
|
||||
properties.put(StreamsConfig.DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG, KafkaDeserializationExceptionHandler.class);
|
||||
|
||||
// interceptor
|
||||
if (clientConfig.getLoggers() != null) {
|
||||
properties.put(
|
||||
StreamsConfig.PRODUCER_PREFIX + ProducerConfig.INTERCEPTOR_CLASSES_CONFIG,
|
||||
ProducerInterceptor.class.getName()
|
||||
);
|
||||
|
||||
properties.put(
|
||||
StreamsConfig.MAIN_CONSUMER_PREFIX + ConsumerConfig.INTERCEPTOR_CLASSES_CONFIG,
|
||||
ConsumerInterceptor.class.getName()
|
||||
);
|
||||
}
|
||||
|
||||
if (properties.containsKey(StreamsConfig.STATE_DIR_CONFIG)) {
|
||||
File stateDir = new File((String) properties.get(StreamsConfig.STATE_DIR_CONFIG));
|
||||
|
||||
if (!stateDir.exists()) {
|
||||
//noinspection ResultOfMethodCallIgnored
|
||||
stateDir.mkdirs();
|
||||
}
|
||||
}
|
||||
|
||||
Stream stream = new Stream(topology, properties, metricsEnabled ? metricRegistry : null, logger);
|
||||
eventPublisher.publishEvent(new KafkaStreamEndpoint.Event(clientId.getName(), stream));
|
||||
|
||||
return stream;
|
||||
}
|
||||
|
||||
public static class Stream extends KafkaStreams {
|
||||
private final Logger logger;
|
||||
|
||||
private final MetricRegistry meterRegistry;
|
||||
|
||||
private final String[] tags;
|
||||
|
||||
private KafkaStreamsMetrics metrics;
|
||||
|
||||
private boolean hasStarted = false;
|
||||
|
||||
private Stream(Topology topology, Properties props, MetricRegistry meterRegistry, Logger logger) {
|
||||
super(topology, props);
|
||||
this.meterRegistry = meterRegistry;
|
||||
|
||||
tags = new String[]{
|
||||
"client_class_id",
|
||||
(String) props.get(CommonClientConfigs.CLIENT_ID_CONFIG)
|
||||
};
|
||||
|
||||
if (meterRegistry != null) {
|
||||
metrics = new KafkaStreamsMetrics(
|
||||
this,
|
||||
List.of(
|
||||
Tag.of("client_type", "stream"),
|
||||
Tag.of("client_class_id", (String) props.get(CommonClientConfigs.CLIENT_ID_CONFIG))
|
||||
)
|
||||
);
|
||||
meterRegistry.bind(metrics);
|
||||
}
|
||||
|
||||
this.logger = logger != null ? logger : log;
|
||||
|
||||
if (this.logger.isTraceEnabled()) {
|
||||
this.logger.trace(topology.describe().toString());
|
||||
}
|
||||
}
|
||||
|
||||
public synchronized void start(final KafkaStreams.StateListener listener) throws IllegalStateException, StreamsException {
|
||||
this.setUncaughtExceptionHandler(e -> {
|
||||
this.logger.error("Uncaught exception in Kafka Stream, closing !", e);
|
||||
return StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse.SHUTDOWN_APPLICATION;
|
||||
});
|
||||
|
||||
this.setGlobalStateRestoreListener(new StateRestoreLoggerListeners(logger));
|
||||
|
||||
this.setStateListener((newState, oldState) -> {
|
||||
meterRegistry.gauge(
|
||||
MetricRegistry.STREAMS_STATE_COUNT,
|
||||
0,
|
||||
ArrayUtils.addAll(tags, "state", oldState.name())
|
||||
);
|
||||
|
||||
meterRegistry.gauge(
|
||||
MetricRegistry.STREAMS_STATE_COUNT,
|
||||
1,
|
||||
ArrayUtils.addAll(tags, "state", newState.name())
|
||||
);
|
||||
|
||||
if (newState == State.RUNNING) {
|
||||
this.hasStarted = true;
|
||||
}
|
||||
|
||||
if (
|
||||
(newState == State.REBALANCING && this.hasStarted) ||
|
||||
newState == State.NOT_RUNNING ||
|
||||
newState == State.PENDING_SHUTDOWN
|
||||
) {
|
||||
this.logger.warn("Switching stream state from {} to {}", oldState, newState);
|
||||
} else if (
|
||||
newState == State.PENDING_ERROR ||
|
||||
newState == State.ERROR
|
||||
) {
|
||||
this.logger.error("Switching stream state from {} to {}", oldState, newState);
|
||||
} else {
|
||||
logger.info("Switching stream state from {} to {}", oldState, newState);
|
||||
}
|
||||
|
||||
if (newState == State.ERROR) {
|
||||
logger.warn("Shutdown now due to ERROR state");
|
||||
System.exit(-1);
|
||||
}
|
||||
|
||||
if (listener != null) {
|
||||
listener.onChange(newState, oldState);
|
||||
}
|
||||
});
|
||||
|
||||
super.start();
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized void start() throws IllegalStateException, StreamsException {
|
||||
this.start(null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
if (metrics != null) {
|
||||
metrics.close();
|
||||
}
|
||||
|
||||
super.close();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean close(Duration timeout) {
|
||||
if (metrics != null) {
|
||||
metrics.close();
|
||||
}
|
||||
|
||||
return super.close(timeout);
|
||||
}
|
||||
}
|
||||
|
||||
public static class StateRestoreLoggerListeners implements StateRestoreListener {
|
||||
private final Logger logger;
|
||||
|
||||
public StateRestoreLoggerListeners(Logger logger) {
|
||||
this.logger = logger;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onRestoreStart(TopicPartition topicPartition, String storeName, long startingOffset, long endingOffset) {
|
||||
if (logger.isDebugEnabled()) {
|
||||
logger.debug(
|
||||
"Starting restore topic '{}', partition '{}', store '{}' from {} to {}",
|
||||
topicPartition.topic(),
|
||||
topicPartition.partition(),
|
||||
storeName,
|
||||
startingOffset,
|
||||
endingOffset
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onBatchRestored(TopicPartition topicPartition, String storeName, long batchEndOffset, long numRestored) {
|
||||
if (logger.isTraceEnabled()) {
|
||||
logger.trace(
|
||||
"Restore batch for topic '{}', partition '{}', store '{}' at offset {} with {} records",
|
||||
topicPartition.topic(),
|
||||
topicPartition.partition(),
|
||||
storeName,
|
||||
batchEndOffset,
|
||||
numRestored
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onRestoreEnd(TopicPartition topicPartition, String storeName, long totalRestored) {
|
||||
if (logger.isDebugEnabled()) {
|
||||
logger.debug(
|
||||
"Restore ended for topic '{}', partition '{}', store '{}'",
|
||||
topicPartition.topic(),
|
||||
topicPartition.partition(),
|
||||
storeName
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -1,126 +0,0 @@
|
||||
package io.kestra.runner.kafka.services;
|
||||
|
||||
import io.kestra.core.exceptions.InternalException;
|
||||
import io.kestra.core.models.flows.Flow;
|
||||
import io.kestra.core.models.triggers.Trigger;
|
||||
import io.kestra.core.runners.Executor;
|
||||
import io.kestra.core.runners.FlowExecutorInterface;
|
||||
import io.kestra.core.services.TaskDefaultService;
|
||||
import io.kestra.core.tasks.flows.Template;
|
||||
import io.kestra.core.utils.Await;
|
||||
import io.kestra.runner.kafka.serializers.JsonSerde;
|
||||
import io.kestra.runner.kafka.streams.FlowJoinerTransformer;
|
||||
import jakarta.inject.Inject;
|
||||
import jakarta.inject.Singleton;
|
||||
import lombok.extern.slf4j.Slf4j;
|
||||
import org.apache.kafka.common.serialization.Serdes;
|
||||
import org.apache.kafka.common.utils.Bytes;
|
||||
import org.apache.kafka.streams.StreamsBuilder;
|
||||
import org.apache.kafka.streams.kstream.*;
|
||||
import org.apache.kafka.streams.state.KeyValueStore;
|
||||
import org.slf4j.Logger;
|
||||
|
||||
import java.time.Duration;
|
||||
import java.util.concurrent.TimeoutException;
|
||||
|
||||
@Singleton
|
||||
@Slf4j
|
||||
public class KafkaStreamSourceService {
|
||||
public static final String TOPIC_FLOWLAST = "flowlast";
|
||||
public static final String TOPIC_EXECUTOR_WORKERINSTANCE = "executorworkerinstance";
|
||||
|
||||
@Inject
|
||||
private KafkaAdminService kafkaAdminService;
|
||||
|
||||
@Inject
|
||||
private TaskDefaultService taskDefaultService;
|
||||
|
||||
@Inject
|
||||
private FlowExecutorInterface flowExecutorInterface;
|
||||
|
||||
@Inject
|
||||
private Template.TemplateExecutorInterface templateExecutorInterface;
|
||||
|
||||
public KStream<String, Executor> executorKStream(StreamsBuilder builder) {
|
||||
return builder
|
||||
.stream(
|
||||
kafkaAdminService.getTopicName(Executor.class),
|
||||
Consumed.with(Serdes.String(), JsonSerde.of(Executor.class)).withName("KStream.Executor")
|
||||
);
|
||||
}
|
||||
|
||||
public GlobalKTable<String, Trigger> triggerGlobalKTable(StreamsBuilder builder) {
|
||||
return builder
|
||||
.globalTable(
|
||||
kafkaAdminService.getTopicName(Trigger.class),
|
||||
Consumed.with(Serdes.String(), JsonSerde.of(Trigger.class)).withName("GlobalKTable.Trigger"),
|
||||
Materialized.<String, Trigger, KeyValueStore<Bytes, byte[]>>as("trigger")
|
||||
.withKeySerde(Serdes.String())
|
||||
.withValueSerde(JsonSerde.of(Trigger.class))
|
||||
);
|
||||
}
|
||||
|
||||
public KStream<String, Executor> executorWithFlow(KStream<String, Executor> executionKStream, boolean withDefaults) {
|
||||
return executionKStream
|
||||
.filter((key, value) -> value != null, Named.as("ExecutorWithFlow.filterNotNull"))
|
||||
.transformValues(
|
||||
() -> new FlowJoinerTransformer(this, withDefaults)
|
||||
);
|
||||
}
|
||||
|
||||
public Executor joinFlow(Executor executor, Boolean withDefaults) {
|
||||
Flow flow;
|
||||
|
||||
try {
|
||||
// pooling of new flow can be delayed on ExecutorStore, we maybe need to wait that the flow is updated
|
||||
flow = Await.until(
|
||||
() -> flowExecutorInterface.findByExecution(executor.getExecution()).orElse(null),
|
||||
Duration.ofMillis(100),
|
||||
Duration.ofMinutes(flowExecutorInterface.isReady() ? 2 : 5)
|
||||
);
|
||||
} catch (TimeoutException e) {
|
||||
// execution is failed, can't find flow, avoid recursive exception, skipped it.
|
||||
if (executor.getExecution().getState().isFailed()) {
|
||||
return executor;
|
||||
}
|
||||
|
||||
return executor.withException(
|
||||
new Exception(
|
||||
"Unable to find flow with namespace: '" + executor.getExecution().getNamespace() + "'" +
|
||||
", id: '" + executor.getExecution().getFlowId() + "', " +
|
||||
"revision '" + executor.getExecution().getFlowRevision() + "'",
|
||||
e
|
||||
),
|
||||
"joinFlow"
|
||||
);
|
||||
}
|
||||
|
||||
if (!withDefaults) {
|
||||
return executor.withFlow(flow);
|
||||
}
|
||||
|
||||
try {
|
||||
flow = Template.injectTemplate(
|
||||
flow,
|
||||
executor.getExecution(),
|
||||
(namespace, id) -> this.templateExecutorInterface.findById(namespace, id).orElse(null)
|
||||
);
|
||||
} catch (InternalException e) {
|
||||
log.debug("Failed to inject template", e);
|
||||
}
|
||||
|
||||
Flow flowWithDefaults = taskDefaultService.injectDefaults(flow, executor.getExecution());
|
||||
|
||||
return executor.withFlow(flowWithDefaults);
|
||||
}
|
||||
|
||||
public static <T> KStream<String, T> logIfEnabled(Logger log, KStream<String, T> stream, ForeachAction<String, T> action, String name) {
|
||||
if (log.isDebugEnabled()) {
|
||||
return stream
|
||||
.filter((key, value) -> value != null, Named.as(name + "Log.filterNotNull"))
|
||||
.peek(action, Named.as(name + "Log.peek"));
|
||||
} else {
|
||||
return stream;
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -1,17 +0,0 @@
|
||||
package io.kestra.runner.kafka.services;
|
||||
|
||||
import org.apache.kafka.streams.StreamsBuilder;
|
||||
import org.apache.kafka.streams.StreamsConfig;
|
||||
import org.apache.kafka.streams.Topology;
|
||||
|
||||
import java.util.Properties;
|
||||
|
||||
public class KafkaStreamsBuilder extends StreamsBuilder {
|
||||
@Override
|
||||
public synchronized Topology build() {
|
||||
Properties properties = new Properties();
|
||||
properties.put(StreamsConfig.TOPOLOGY_OPTIMIZATION_CONFIG, StreamsConfig.OPTIMIZE);
|
||||
|
||||
return super.build(properties);
|
||||
}
|
||||
}
|
||||
@@ -1,98 +0,0 @@
|
||||
package io.kestra.runner.kafka.services;
|
||||
|
||||
import com.google.common.collect.Streams;
|
||||
import lombok.extern.slf4j.Slf4j;
|
||||
import org.apache.kafka.common.errors.SerializationException;
|
||||
import org.apache.kafka.common.serialization.StringSerializer;
|
||||
import org.apache.kafka.streams.KeyValue;
|
||||
import org.apache.kafka.streams.state.KeyValueIterator;
|
||||
import org.apache.kafka.streams.state.ReadOnlyKeyValueStore;
|
||||
|
||||
import java.util.Iterator;
|
||||
import java.util.NoSuchElementException;
|
||||
import java.util.Optional;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
@Slf4j
|
||||
public class SafeKeyValueStore <K, V> {
|
||||
private final ReadOnlyKeyValueStore<K, V> store;
|
||||
private final String name;
|
||||
|
||||
public SafeKeyValueStore(ReadOnlyKeyValueStore<K, V> store, String name) {
|
||||
this.store = store;
|
||||
this.name = name;
|
||||
}
|
||||
|
||||
public Optional<V> get(K key) {
|
||||
try {
|
||||
return Optional.ofNullable(this.store.get(key));
|
||||
} catch (SerializationException e) {
|
||||
if (log.isDebugEnabled()) {
|
||||
log.debug("Exception on store {}", name, e);
|
||||
}
|
||||
return Optional.empty();
|
||||
}
|
||||
}
|
||||
|
||||
public Stream<V> all() {
|
||||
KeyValueIterator<K, V> all = this.store.all();
|
||||
|
||||
return toStream(all, kvKeyValue -> kvKeyValue.value);
|
||||
}
|
||||
|
||||
public Stream<KeyValue<K, V>> allWithKey() {
|
||||
KeyValueIterator<K, V> all = this.store.all();
|
||||
|
||||
return toStream(all, kvKeyValue -> new KeyValue<>(kvKeyValue.key, kvKeyValue.value));
|
||||
}
|
||||
|
||||
public Stream<V> prefix(String prefix) {
|
||||
KeyValueIterator<K, V> all = this.store.prefixScan(prefix, new StringSerializer());
|
||||
|
||||
return toStream(all, kvKeyValue -> kvKeyValue.value);
|
||||
}
|
||||
|
||||
public Stream<KeyValue<K, V>> prefixWithKey(String prefix) {
|
||||
KeyValueIterator<K, V> all = this.store.prefixScan(prefix, new StringSerializer());
|
||||
|
||||
return toStream(all, kvKeyValue -> new KeyValue<>(kvKeyValue.key, kvKeyValue.value));
|
||||
}
|
||||
|
||||
@SuppressWarnings("UnstableApiUsage")
|
||||
private <R> Stream<R> toStream(KeyValueIterator<K, V> all, Function<KeyValue<K, V>, R> function) {
|
||||
return Streams
|
||||
.stream(new Iterator<R>() {
|
||||
private R next;
|
||||
|
||||
@Override
|
||||
public boolean hasNext() {
|
||||
boolean seek = true;
|
||||
while (seek) {
|
||||
try {
|
||||
KeyValue<K, V> rawNext = all.next();
|
||||
|
||||
next = function.apply(rawNext);
|
||||
|
||||
return true;
|
||||
} catch (SerializationException e) {
|
||||
if (log.isTraceEnabled()) {
|
||||
log.trace("Exception on store {}", name, e);
|
||||
}
|
||||
} catch (NoSuchElementException e) {
|
||||
seek = false;
|
||||
}
|
||||
}
|
||||
|
||||
all.close();
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public R next() {
|
||||
return this.next;
|
||||
}
|
||||
})
|
||||
.onClose(all::close);
|
||||
}
|
||||
}
|
||||
@@ -1,44 +0,0 @@
|
||||
package io.kestra.runner.kafka.streams;
|
||||
|
||||
import lombok.extern.slf4j.Slf4j;
|
||||
import org.apache.kafka.streams.kstream.KeyValueMapper;
|
||||
import org.apache.kafka.streams.kstream.ValueTransformerWithKey;
|
||||
import org.apache.kafka.streams.processor.ProcessorContext;
|
||||
import org.apache.kafka.streams.state.KeyValueStore;
|
||||
|
||||
@Slf4j
|
||||
public class DeduplicationPurgeTransformer<K, V, SV> implements ValueTransformerWithKey<K, V, V> {
|
||||
private final String storeName;
|
||||
private KeyValueStore<String, SV> store;
|
||||
private final KeyValueMapper<K, V, String> storeKeyMapper;
|
||||
|
||||
public DeduplicationPurgeTransformer(
|
||||
String storeName,
|
||||
KeyValueMapper<K, V, String> storeKeyMapper
|
||||
) {
|
||||
this.storeName = storeName;
|
||||
this.storeKeyMapper = storeKeyMapper;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void init(final ProcessorContext context) {
|
||||
this.store = context.getStateStore(this.storeName);
|
||||
}
|
||||
|
||||
@Override
|
||||
public V transform(final K key, final V value) {
|
||||
if (value != null) {
|
||||
String storeKey = storeKeyMapper.apply(key, value);
|
||||
|
||||
if (storeKey != null) {
|
||||
store.delete(storeKey);
|
||||
}
|
||||
}
|
||||
|
||||
return value;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
}
|
||||
}
|
||||
@@ -1,54 +0,0 @@
|
||||
package io.kestra.runner.kafka.streams;
|
||||
|
||||
import lombok.extern.slf4j.Slf4j;
|
||||
import org.apache.kafka.streams.kstream.KeyValueMapper;
|
||||
import org.apache.kafka.streams.kstream.ValueTransformerWithKey;
|
||||
import org.apache.kafka.streams.processor.ProcessorContext;
|
||||
import org.apache.kafka.streams.state.KeyValueStore;
|
||||
|
||||
@Slf4j
|
||||
public class DeduplicationTransformer<K, V, SV> implements ValueTransformerWithKey<K, V, V> {
|
||||
private final String name;
|
||||
private final String storeName;
|
||||
private KeyValueStore<String, SV> store;
|
||||
private final KeyValueMapper<K, V, String> storeKeyMapper;
|
||||
private final KeyValueMapper<K, V, SV> storeValueMapper;
|
||||
|
||||
public DeduplicationTransformer(
|
||||
String name,
|
||||
String storeName,
|
||||
KeyValueMapper<K, V, String> storeKeyMapper,
|
||||
KeyValueMapper<K, V, SV> storeValueMapper
|
||||
) {
|
||||
this.name = name;
|
||||
this.storeName = storeName;
|
||||
this.storeKeyMapper = storeKeyMapper;
|
||||
this.storeValueMapper = storeValueMapper;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void init(final ProcessorContext context) {
|
||||
this.store = context.getStateStore(this.storeName);
|
||||
}
|
||||
|
||||
@Override
|
||||
public V transform(final K key, final V value) {
|
||||
String storeKey = storeKeyMapper.apply(key, value);
|
||||
SV currentValue = storeValueMapper.apply(key, value);
|
||||
|
||||
SV latestValue = store.get(storeKey);
|
||||
|
||||
if (latestValue != null && latestValue.equals(currentValue)) {
|
||||
log.trace("{} duplicate value for key '{}', storeKey '{}', value '{}'", name, key, storeKey, latestValue);
|
||||
return null;
|
||||
}
|
||||
|
||||
store.put(storeKey, currentValue);
|
||||
|
||||
return value;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
}
|
||||
}
|
||||
@@ -1,46 +0,0 @@
|
||||
package io.kestra.runner.kafka.streams;
|
||||
|
||||
import com.fasterxml.jackson.core.JsonProcessingException;
|
||||
import io.kestra.core.runners.Executor;
|
||||
import io.kestra.core.serializers.JacksonMapper;
|
||||
import lombok.extern.slf4j.Slf4j;
|
||||
import org.apache.kafka.streams.kstream.ValueTransformerWithKey;
|
||||
import org.apache.kafka.streams.processor.ProcessorContext;
|
||||
|
||||
import java.nio.charset.StandardCharsets;
|
||||
|
||||
@Slf4j
|
||||
public class ExecutorAddHeaderTransformer implements ValueTransformerWithKey<String, Executor, Executor> {
|
||||
private ProcessorContext context;
|
||||
|
||||
public ExecutorAddHeaderTransformer() {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void init(final ProcessorContext context) {
|
||||
this.context = context;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Executor transform(final String key, final Executor value) {
|
||||
try {
|
||||
this.context.headers().add(
|
||||
"from",
|
||||
JacksonMapper.ofJson().writeValueAsString(value.getFrom()).getBytes(StandardCharsets.UTF_8)
|
||||
);
|
||||
|
||||
this.context.headers().add(
|
||||
"offset",
|
||||
JacksonMapper.ofJson().writeValueAsString(value.getOffset()).getBytes(StandardCharsets.UTF_8)
|
||||
);
|
||||
} catch (JsonProcessingException e) {
|
||||
log.warn("Unable to add headers", e);
|
||||
}
|
||||
|
||||
return value;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
}
|
||||
}
|
||||
@@ -1,13 +0,0 @@
|
||||
package io.kestra.runner.kafka.streams;
|
||||
|
||||
import lombok.AllArgsConstructor;
|
||||
import lombok.Getter;
|
||||
import io.kestra.core.models.executions.Execution;
|
||||
import io.kestra.core.models.flows.Flow;
|
||||
|
||||
@Getter
|
||||
@AllArgsConstructor
|
||||
public class ExecutorFlowTrigger {
|
||||
Flow flowHavingTrigger;
|
||||
Execution execution;
|
||||
}
|
||||
@@ -1,52 +0,0 @@
|
||||
package io.kestra.runner.kafka.streams;
|
||||
|
||||
import io.kestra.core.models.executions.Execution;
|
||||
import io.kestra.core.models.flows.State;
|
||||
import io.kestra.core.runners.Executor;
|
||||
import lombok.extern.slf4j.Slf4j;
|
||||
import org.apache.kafka.streams.kstream.ValueTransformerWithKey;
|
||||
import org.apache.kafka.streams.processor.ProcessorContext;
|
||||
import org.apache.kafka.streams.state.KeyValueStore;
|
||||
|
||||
@Slf4j
|
||||
public class ExecutorFromExecutionTransformer implements ValueTransformerWithKey<String, Execution, Executor> {
|
||||
private ProcessorContext context;
|
||||
private final String storeName;
|
||||
private KeyValueStore<String, Executor> store;
|
||||
|
||||
public ExecutorFromExecutionTransformer(String storeName) {
|
||||
this.storeName = storeName;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void init(final ProcessorContext context) {
|
||||
this.context = context;
|
||||
this.store = context.getStateStore(this.storeName);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Executor transform(final String key, final Execution value) {
|
||||
if (value == null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
Executor executor = new Executor(
|
||||
value,
|
||||
this.context.offset()
|
||||
);
|
||||
|
||||
// restart need to be saved on state store for future join
|
||||
if (executor.getExecution().getState().getCurrent() == State.Type.RESTARTED) {
|
||||
store.put(key, executor.serialize());
|
||||
}
|
||||
|
||||
this.context.headers().remove("from");
|
||||
this.context.headers().remove("offset");
|
||||
|
||||
return executor;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
}
|
||||
}
|
||||
@@ -1,108 +0,0 @@
|
||||
package io.kestra.runner.kafka.streams;
|
||||
|
||||
import io.kestra.core.metrics.MetricRegistry;
|
||||
import io.kestra.core.models.executions.Execution;
|
||||
import io.kestra.core.runners.Executor;
|
||||
import io.kestra.core.runners.ExecutorService;
|
||||
import io.kestra.core.runners.WorkerTaskResult;
|
||||
import io.kestra.runner.kafka.services.KafkaStreamSourceService;
|
||||
import lombok.extern.slf4j.Slf4j;
|
||||
import org.apache.kafka.streams.kstream.ValueTransformerWithKey;
|
||||
import org.apache.kafka.streams.processor.ProcessorContext;
|
||||
import org.apache.kafka.streams.state.KeyValueStore;
|
||||
|
||||
@Slf4j
|
||||
public class ExecutorJoinerTransformer implements ValueTransformerWithKey<String, Executor, Executor> {
|
||||
private final String storeName;
|
||||
private final ExecutorService executorService;
|
||||
private final KafkaStreamSourceService kafkaStreamSourceService;
|
||||
private final MetricRegistry metricRegistry;
|
||||
private KeyValueStore<String, Executor> store;
|
||||
private ProcessorContext context;
|
||||
|
||||
public ExecutorJoinerTransformer(String storeName, ExecutorService executorService, KafkaStreamSourceService kafkaStreamSourceService, MetricRegistry metricRegistry) {
|
||||
this.storeName = storeName;
|
||||
this.executorService = executorService;
|
||||
this.kafkaStreamSourceService = kafkaStreamSourceService;
|
||||
this.metricRegistry = metricRegistry;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void init(final ProcessorContext context) {
|
||||
this.context = context;
|
||||
this.store = context.getStateStore(this.storeName);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Executor transform(final String key, final Executor value) {
|
||||
if (value.getExecution() != null) {
|
||||
return value;
|
||||
}
|
||||
|
||||
WorkerTaskResult workerTaskResult = value.getJoined();
|
||||
|
||||
if (log.isDebugEnabled()) {
|
||||
log.debug(
|
||||
"<< IN WorkerTaskResult [key='{}', partition='{}, offset='{}'] : {}",
|
||||
key,
|
||||
context.partition(),
|
||||
context.offset(),
|
||||
workerTaskResult.getTaskRun().toStringState()
|
||||
);
|
||||
}
|
||||
|
||||
Executor executor = this.store.get(key);
|
||||
|
||||
// already purge execution ?
|
||||
if (executor == null) {
|
||||
log.warn("Unable to find Executor with key '" + key + "' for WorkerTaskResult id '" + workerTaskResult.getTaskRun().getId() + "' '" + workerTaskResult.getTaskRun().toStringState() + "'");
|
||||
return null;
|
||||
}
|
||||
|
||||
if (!executor.getExecution().hasTaskRunJoinable(value.getJoined().getTaskRun())) {
|
||||
return executor;
|
||||
}
|
||||
|
||||
kafkaStreamSourceService.joinFlow(executor, true);
|
||||
|
||||
try {
|
||||
Execution newExecution = executorService.addDynamicTaskRun(
|
||||
executor.getExecution(),
|
||||
executor.getFlow(),
|
||||
workerTaskResult
|
||||
);
|
||||
|
||||
if (newExecution != null) {
|
||||
executor = executor.withExecution(newExecution, "addDynamicTaskRun");
|
||||
}
|
||||
|
||||
newExecution = executor.getExecution().withTaskRun(workerTaskResult.getTaskRun());
|
||||
executor = executor.withExecution(newExecution, "joinWorkerResult");
|
||||
} catch (Exception e) {
|
||||
return executor.withException(e, "joinWorkerResult");
|
||||
}
|
||||
|
||||
// send metrics on terminated
|
||||
if (workerTaskResult.getTaskRun().getState().isTerminated()) {
|
||||
metricRegistry
|
||||
.counter(
|
||||
MetricRegistry.EXECUTOR_TASKRUN_ENDED_COUNT,
|
||||
metricRegistry.tags(workerTaskResult)
|
||||
)
|
||||
.increment();
|
||||
|
||||
metricRegistry
|
||||
.timer(
|
||||
MetricRegistry.EXECUTOR_TASKRUN_ENDED_DURATION,
|
||||
metricRegistry.tags(workerTaskResult)
|
||||
)
|
||||
.record(workerTaskResult.getTaskRun().getState().getDuration());
|
||||
}
|
||||
|
||||
return executor;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
}
|
||||
}
|
||||
@@ -1,61 +0,0 @@
|
||||
package io.kestra.runner.kafka.streams;
|
||||
|
||||
import io.kestra.core.models.executions.Execution;
|
||||
import io.kestra.core.models.executions.ExecutionKilled;
|
||||
import io.kestra.core.models.flows.State;
|
||||
import io.kestra.core.runners.Executor;
|
||||
import lombok.extern.slf4j.Slf4j;
|
||||
import org.apache.kafka.streams.kstream.ValueTransformerWithKey;
|
||||
import org.apache.kafka.streams.processor.ProcessorContext;
|
||||
import org.apache.kafka.streams.state.KeyValueStore;
|
||||
|
||||
@Slf4j
|
||||
public class ExecutorKilledJoinerTransformer implements ValueTransformerWithKey<String, ExecutionKilled, Executor> {
|
||||
private final String storeName;
|
||||
private KeyValueStore<String, Executor> store;
|
||||
|
||||
public ExecutorKilledJoinerTransformer(String storeName) {
|
||||
this.storeName = storeName;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void init(final ProcessorContext context) {
|
||||
this.store = context.getStateStore(this.storeName);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Executor transform(final String key, final ExecutionKilled value) {
|
||||
if (value == null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
if (!value.getExecutionId().equals(key)) {
|
||||
throw new IllegalStateException("Invalid key for killed with key='" + key + "' and execution='" + value.getExecutionId() + "'");
|
||||
}
|
||||
|
||||
Executor executor = this.store.get(key);
|
||||
|
||||
if (executor == null) {
|
||||
log.warn("Unable to find Executor with key '" + key + "' for Killed id '" + value.getExecutionId() + "'");
|
||||
return null;
|
||||
}
|
||||
|
||||
if (executor.getExecution().getState().getCurrent() != State.Type.KILLING &&
|
||||
!executor.getExecution().getState().isTerminated()
|
||||
) {
|
||||
Execution newExecution = executor.getExecution().withState(State.Type.KILLING);
|
||||
|
||||
if (log.isDebugEnabled()) {
|
||||
log.debug("Killed << IN\n{}", newExecution.toStringState());
|
||||
}
|
||||
|
||||
return executor.withExecution(newExecution, "joinExecutionKilled");
|
||||
}
|
||||
|
||||
return executor;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
}
|
||||
}
|
||||
Some files were not shown because too many files have changed in this diff Show More
Reference in New Issue
Block a user