From 038934d2aaee9d65dbd5c30b82818d48e3438d41 Mon Sep 17 00:00:00 2001 From: Miljenko Brkic Date: Mon, 11 Sep 2023 11:51:33 +0100 Subject: [PATCH 01/50] CORE-16199 Multi-source event mediator implementation --- .../flow/service/FlowExecutorMediatorImpl.kt | 170 ++++++++++++++ .../db/consumer/DBCordaConsumerImpl.kt | 4 + .../kafka/consumer/CordaKafkaConsumerImpl.kt | 35 +++ .../messagebus/api/consumer/CordaConsumer.kt | 13 ++ .../mediator/MultiSourceEventMediatorImpl.kt | 215 ++++++++++++++++++ .../corda/messaging/mediator/ProcessorTask.kt | 37 +++ .../factory/EventMediatorFactoryImpl.kt | 50 ++++ .../mediator/statemanager/StateManagerImpl.kt | 34 +++ .../mediator/taskmanager/TaskManagerImpl.kt | 14 ++ .../api/mediator/config/package-info.java | 4 + .../api/mediator/factory/package-info.java | 4 + .../messaging/api/mediator/package-info.java | 4 + .../mediator/statemanager/package-info.java | 4 + .../mediator/taskmanager/package-info.java | 4 + .../api/mediator/MediatorConsumer.kt | 31 +++ .../api/mediator/MediatorProducer.kt | 22 ++ .../corda/messaging/api/mediator/Message.kt | 11 + .../api/mediator/MessageBusConsumer.kt | 37 +++ .../api/mediator/MessageBusProducer.kt | 19 ++ .../messaging/api/mediator/MessageRouter.kt | 6 + .../api/mediator/MultiSourceEventMediator.kt | 10 + .../messaging/api/mediator/ProducerReply.kt | 11 + .../mediator/config/EventMediatorConfig.kt | 27 +++ .../config/EventMediatorConfigBuilder.kt | 63 +++++ .../mediator/config/MediatorConsumerConfig.kt | 16 ++ .../mediator/config/MediatorProducerConfig.kt | 12 + .../mediator/factory/EventMediatorFactory.kt | 22 ++ .../factory/MediatorConsumerFactory.kt | 19 ++ .../factory/MediatorProducerFactory.kt | 17 ++ .../factory/MessageBusConsumerFactory.kt | 105 +++++++++ .../factory/MessageBusProducerFactory.kt | 50 ++++ .../api/mediator/statemanager/Metadata.kt | 40 ++++ .../api/mediator/statemanager/State.kt | 34 +++ .../api/mediator/statemanager/StateManager.kt | 44 ++++ .../api/mediator/taskmanager/TaskManager.kt | 10 + 35 files changed, 1198 insertions(+) create mode 100644 components/flow/flow-service/src/main/kotlin/net/corda/flow/service/FlowExecutorMediatorImpl.kt create mode 100644 libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt create mode 100644 libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/ProcessorTask.kt create mode 100644 libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/EventMediatorFactoryImpl.kt create mode 100644 libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/statemanager/StateManagerImpl.kt create mode 100644 libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/taskmanager/TaskManagerImpl.kt create mode 100644 libs/messaging/messaging/src/main/java/net/corda/messaging/api/mediator/config/package-info.java create mode 100644 libs/messaging/messaging/src/main/java/net/corda/messaging/api/mediator/factory/package-info.java create mode 100644 libs/messaging/messaging/src/main/java/net/corda/messaging/api/mediator/package-info.java create mode 100644 libs/messaging/messaging/src/main/java/net/corda/messaging/api/mediator/statemanager/package-info.java create mode 100644 libs/messaging/messaging/src/main/java/net/corda/messaging/api/mediator/taskmanager/package-info.java create mode 100644 libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MediatorConsumer.kt create mode 100644 libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MediatorProducer.kt create mode 100644 libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/Message.kt create mode 100644 libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MessageBusConsumer.kt create mode 100644 libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MessageBusProducer.kt create mode 100644 libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MessageRouter.kt create mode 100644 libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MultiSourceEventMediator.kt create mode 100644 libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/ProducerReply.kt create mode 100644 libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/config/EventMediatorConfig.kt create mode 100644 libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/config/EventMediatorConfigBuilder.kt create mode 100644 libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/config/MediatorConsumerConfig.kt create mode 100644 libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/config/MediatorProducerConfig.kt create mode 100644 libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/EventMediatorFactory.kt create mode 100644 libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MediatorConsumerFactory.kt create mode 100644 libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MediatorProducerFactory.kt create mode 100644 libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MessageBusConsumerFactory.kt create mode 100644 libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MessageBusProducerFactory.kt create mode 100644 libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/statemanager/Metadata.kt create mode 100644 libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/statemanager/State.kt create mode 100644 libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/statemanager/StateManager.kt create mode 100644 libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/taskmanager/TaskManager.kt diff --git a/components/flow/flow-service/src/main/kotlin/net/corda/flow/service/FlowExecutorMediatorImpl.kt b/components/flow/flow-service/src/main/kotlin/net/corda/flow/service/FlowExecutorMediatorImpl.kt new file mode 100644 index 00000000000..bb51e400b92 --- /dev/null +++ b/components/flow/flow-service/src/main/kotlin/net/corda/flow/service/FlowExecutorMediatorImpl.kt @@ -0,0 +1,170 @@ +package net.corda.flow.service + +import com.typesafe.config.ConfigValueFactory +import net.corda.data.flow.event.FlowEvent +import net.corda.data.flow.state.checkpoint.Checkpoint +import net.corda.flow.pipeline.factory.FlowEventProcessorFactory +import net.corda.libs.configuration.SmartConfig +import net.corda.libs.configuration.helper.getConfig +import net.corda.lifecycle.LifecycleCoordinatorFactory +import net.corda.lifecycle.LifecycleEvent +import net.corda.lifecycle.LifecycleStatus +import net.corda.lifecycle.RegistrationHandle +import net.corda.lifecycle.StartEvent +import net.corda.lifecycle.StopEvent +import net.corda.lifecycle.createCoordinator +import net.corda.messagebus.api.consumer.builder.CordaConsumerBuilder +import net.corda.messagebus.api.producer.builder.CordaProducerBuilder +import net.corda.messaging.api.mediator.MessageRouter +import net.corda.messaging.api.mediator.MultiSourceEventMediator +import net.corda.messaging.api.mediator.config.EventMediatorConfigBuilder +import net.corda.messaging.api.mediator.factory.EventMediatorFactory +import net.corda.messaging.api.mediator.factory.MessageBusConsumerFactory +import net.corda.messaging.api.mediator.factory.MessageBusProducerFactory +import net.corda.messaging.api.processor.StateAndEventProcessor +import net.corda.schema.Schemas.Flow.FLOW_EVENT_TOPIC +import net.corda.schema.configuration.ConfigKeys.FLOW_CONFIG +import net.corda.schema.configuration.ConfigKeys.MESSAGING_CONFIG +import net.corda.schema.configuration.MessagingConfig.MAX_ALLOWED_MSG_SIZE +import net.corda.schema.configuration.MessagingConfig.Subscription.PROCESSOR_TIMEOUT +import net.corda.utilities.trace +import org.osgi.service.component.annotations.Activate +import org.osgi.service.component.annotations.Reference +import org.slf4j.LoggerFactory + +@Suppress("LongParameterList") +// TODO @Component(service = [FlowExecutor::class]) +class FlowExecutorMediatorImpl ( + coordinatorFactory: LifecycleCoordinatorFactory, + private val flowEventProcessorFactory: FlowEventProcessorFactory, + private val eventMediatorFactory: EventMediatorFactory, + private val cordaConsumerBuilder: CordaConsumerBuilder, + private val cordaProducerBuilder: CordaProducerBuilder, + private val toMessagingConfig: (Map) -> SmartConfig +) : FlowExecutor { + + @Activate + constructor( + @Reference(service = LifecycleCoordinatorFactory::class) + coordinatorFactory: LifecycleCoordinatorFactory, + @Reference(service = FlowEventProcessorFactory::class) + flowEventProcessorFactory: FlowEventProcessorFactory, + @Reference(service = EventMediatorFactory::class) + eventMediatorFactory: EventMediatorFactory, + @Reference(service = CordaConsumerBuilder::class) + cordaConsumerBuilder: CordaConsumerBuilder, + @Reference(service = CordaProducerBuilder::class) + cordaProducerBuilder: CordaProducerBuilder, + ) : this( + coordinatorFactory, + flowEventProcessorFactory, + eventMediatorFactory, + cordaConsumerBuilder, + cordaProducerBuilder, + { cfg -> cfg.getConfig(MESSAGING_CONFIG) } + ) + + companion object { + private val log = LoggerFactory.getLogger(this::class.java.enclosingClass) + private const val CONSUMER_GROUP = "FlowEventConsumer" + } + + private val coordinator = coordinatorFactory.createCoordinator { event, _ -> eventHandler(event) } + private var subscriptionRegistrationHandle: RegistrationHandle? = null + private var multiSourceEventMediator: MultiSourceEventMediator? = null + + override fun onConfigChange(config: Map) { + try { + val messagingConfig = toMessagingConfig(config) + val updatedConfigs = updateConfigsWithFlowConfig(config, messagingConfig) + + // close the lifecycle registration first to prevent down being signaled + subscriptionRegistrationHandle?.close() + multiSourceEventMediator?.close() + + multiSourceEventMediator = createEventMediator( + messagingConfig, + flowEventProcessorFactory.create(updatedConfigs), + eventMediatorFactory, + ) + + subscriptionRegistrationHandle = coordinator.followStatusChangesByName( + setOf(multiSourceEventMediator!!.subscriptionName) + ) + + multiSourceEventMediator?.start() + } catch (ex: Exception) { + val reason = "Failed to configure the flow executor using '${config}'" + log.error(reason, ex) + coordinator.updateStatus(LifecycleStatus.ERROR, reason) + } + } + + override val isRunning: Boolean + get() = coordinator.isRunning + + override fun start() { + coordinator.start() + } + + override fun stop() { + coordinator.stop() + } + + private fun updateConfigsWithFlowConfig( + initialConfigs: Map, + messagingConfig: SmartConfig + ): Map { + val flowConfig = initialConfigs.getConfig(FLOW_CONFIG) + val updatedFlowConfig = flowConfig + .withValue(PROCESSOR_TIMEOUT, ConfigValueFactory.fromAnyRef(messagingConfig.getLong(PROCESSOR_TIMEOUT))) + .withValue(MAX_ALLOWED_MSG_SIZE, ConfigValueFactory.fromAnyRef(messagingConfig.getLong(MAX_ALLOWED_MSG_SIZE))) + + return initialConfigs.mapValues { + if (it.key == FLOW_CONFIG) { + updatedFlowConfig + } else { + it.value + } + } + } + + private fun eventHandler(event: LifecycleEvent) { + when (event) { + is StartEvent -> { + coordinator.updateStatus(LifecycleStatus.UP) + } + is StopEvent -> { + log.trace { "Flow executor is stopping..." } + subscriptionRegistrationHandle?.close() + multiSourceEventMediator?.close() + log.trace { "Flow executor stopped" } + } + } + } + + private fun createEventMediator( + messagingConfig: SmartConfig, + messageProcessor: StateAndEventProcessor, + eventMediatorFactory: EventMediatorFactory + ): MultiSourceEventMediator { + + val eventMediatorConfig = EventMediatorConfigBuilder() + .name("FlowEventMediator") + .messagingConfig(messagingConfig) + .consumerFactories( + MessageBusConsumerFactory(FLOW_EVENT_TOPIC, CONSUMER_GROUP, messagingConfig, cordaConsumerBuilder), + ) + .producerFactories( + MessageBusProducerFactory("messageBusProducer", messagingConfig, cordaProducerBuilder), + //RpcProducerFactory("cryptoWorkerProducer", messagingConfig, cordaRpcBuilder), + ) + .messageProcessor(messageProcessor) + .messageRouter(MessageRouter()) + .build() + + return eventMediatorFactory.createMultiSourceEventMediator( + eventMediatorConfig + ) + } +} diff --git a/libs/messaging/db-message-bus-impl/src/main/kotlin/net/corda/messagebus/db/consumer/DBCordaConsumerImpl.kt b/libs/messaging/db-message-bus-impl/src/main/kotlin/net/corda/messagebus/db/consumer/DBCordaConsumerImpl.kt index 1b6fc1fce8b..41c0cf4280e 100644 --- a/libs/messaging/db-message-bus-impl/src/main/kotlin/net/corda/messagebus/db/consumer/DBCordaConsumerImpl.kt +++ b/libs/messaging/db-message-bus-impl/src/main/kotlin/net/corda/messagebus/db/consumer/DBCordaConsumerImpl.kt @@ -203,6 +203,10 @@ internal class DBCordaConsumerImpl constructor( } } + override fun commitAsync(callback: CordaConsumer.Callback?) { + TODO("Not yet implemented") + } + override fun commitSyncOffsets(event: CordaConsumerRecord, metaData: String?) { dbAccess.writeOffsets( listOf( diff --git a/libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/consumer/CordaKafkaConsumerImpl.kt b/libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/consumer/CordaKafkaConsumerImpl.kt index b1705bd2f0e..04eb81b014c 100644 --- a/libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/consumer/CordaKafkaConsumerImpl.kt +++ b/libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/consumer/CordaKafkaConsumerImpl.kt @@ -304,6 +304,41 @@ class CordaKafkaConsumerImpl( } } + override fun commitAsync(callback: CordaConsumer.Callback?) { + var attemptCommit = true + + while (attemptCommit) { + try { + consumer.commitAsync { offsets, exception -> + callback?.onCompletion( + offsets.entries.associate { + it.key!!.toCordaTopicPartition(config.topicPrefix) to it.value.offset() + }, + exception + ) + } + attemptCommit = false + } catch (ex: Exception) { + when (ex::class.java) { + in fatalExceptions -> { + logErrorAndThrowFatalException( + "Error attempting to commitAsync offsets.", + ex + ) + } + in transientExceptions -> { + logWarningAndThrowIntermittentException("Failed to commitAsync offsets.", ex) + } + else -> { + logErrorAndThrowFatalException( + "Unexpected error attempting to commitAsync offsets.", ex + ) + } + } + } + } + } + override fun commitSyncOffsets(event: CordaConsumerRecord, metaData: String?) { val offsets = mutableMapOf() val topicPartition = TopicPartition(config.topicPrefix + event.topic, event.partition) diff --git a/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/consumer/CordaConsumer.kt b/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/consumer/CordaConsumer.kt index 6892832dc74..bb848c9377d 100644 --- a/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/consumer/CordaConsumer.kt +++ b/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/consumer/CordaConsumer.kt @@ -10,6 +10,13 @@ import java.time.Duration @Suppress("TooManyFunctions") interface CordaConsumer : AutoCloseable { + /** + * Defines the callback for async commits. If there was an exception it will be provided on this callback. + */ + fun interface Callback { + fun onCompletion(offsets: Map, exception: Exception?) + } + /** * Subscribe to given [topics]. If not null, attach the rebalance [listener] to the [Consumer]. * If a recoverable error occurs retry. If max retries is exceeded or a fatal error occurs then @@ -138,6 +145,12 @@ interface CordaConsumer : AutoCloseable { */ fun resetToLastCommittedPositions(offsetStrategy: CordaOffsetResetStrategy) + /** + * Asynchronously commit the consumer offsets. + * @throws CordaMessageAPIFatalException fatal error occurred attempting to commit offsets. + */ + fun commitAsync(callback: Callback?) + /** * Synchronously commit the consumer offset for this [event] back to the topic partition. * Record [metaData] about this commit back on the [event] topic. diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt new file mode 100644 index 00000000000..9f1e62a502b --- /dev/null +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt @@ -0,0 +1,215 @@ +package net.corda.messaging.mediator + +import net.corda.avro.serialization.CordaAvroSerializer +import net.corda.lifecycle.LifecycleCoordinatorFactory +import net.corda.lifecycle.LifecycleCoordinatorName +import net.corda.lifecycle.LifecycleStatus +import net.corda.messagebus.api.consumer.CordaConsumerRecord +import net.corda.messaging.api.exception.CordaMessageAPIFatalException +import net.corda.messaging.api.exception.CordaMessageAPIIntermittentException +import net.corda.messaging.api.mediator.MediatorConsumer +import net.corda.messaging.api.mediator.MediatorProducer +import net.corda.messaging.api.mediator.MultiSourceEventMediator +import net.corda.messaging.api.mediator.config.EventMediatorConfig +import net.corda.messaging.api.mediator.config.MediatorConsumerConfig +import net.corda.messaging.api.mediator.config.MediatorProducerConfig +import net.corda.messaging.api.mediator.statemanager.StateManager +import net.corda.messaging.api.mediator.taskmanager.TaskManager +import net.corda.messaging.api.mediator.taskmanager.TaskType +import net.corda.utilities.debug +import org.slf4j.LoggerFactory +import java.time.Duration +import java.util.UUID + +class MultiSourceEventMediatorImpl( + private val config: EventMediatorConfig, + private val cordaAvroSerializer: CordaAvroSerializer, + private val stateManager: StateManager, + private val taskManager: TaskManager, + lifecycleCoordinatorFactory: LifecycleCoordinatorFactory, +): MultiSourceEventMediator { + + private val log = LoggerFactory.getLogger("${this.javaClass.name}-${config.name}") + + private var consumers = listOf>() + private var producers = listOf() + private val pollTimeoutInNanos = Duration.ofMillis(100).toNanos() // TODO take from config + private val uniqueId = UUID.randomUUID().toString() + private val lifecycleCoordinatorName = LifecycleCoordinatorName( + "MultiSourceEventMediator--${config.name}", uniqueId + ) + private val lifecycleCoordinator = + lifecycleCoordinatorFactory.createCoordinator(lifecycleCoordinatorName) { _, _ -> } + + override val subscriptionName: LifecycleCoordinatorName + get() = lifecycleCoordinatorName + + override fun start() { + log.debug { "Starting multi-source event mediator with config: $config" } + lifecycleCoordinator.start() + taskManager.execute(TaskType.LONG_RUNNING, ::run) + } + + private fun stop() = + Thread.currentThread().interrupt() + private fun stopped() = + Thread.currentThread().isInterrupted + + /** + * This method is for closing the loop/thread externally. From inside the loop use the private [stopConsumeLoop]. + */ + override fun close() { + lifecycleCoordinator.close() + } + + private fun run() { + var attempts = 0 + + while (!stopped()) { + attempts++ + try { + createConsumers() + createProducers() + + consumers.forEach{ it.subscribe() } + lifecycleCoordinator.updateStatus(LifecycleStatus.UP) + + while (!stopped()) { + // TODO processEvents() + } + + } catch (ex: Exception) { + when (ex) { + is InterruptedException -> { + // Stopped + } + is CordaMessageAPIIntermittentException -> { + log.warn( + "${ex.message} Attempts: $attempts. Recreating consumers/producers and Retrying.", ex + ) + } + else -> { + log.error( + "${ex.message} Attempts: $attempts. Closing subscription.", ex + ) + lifecycleCoordinator.updateStatus(LifecycleStatus.ERROR, "Error: ${ex.message}") + stop() + } + } + } finally { + closeConsumersAndProducers() + } + } + closeConsumersAndProducers() + } + + private fun onSerializationError(event: ByteArray) { + log.info("Error serializing [$event]") + TODO() + } + + private fun createConsumers() { + check (config.consumerFactories.isNotEmpty()) { + "None consumer factory set in configuration" + } + consumers = config.consumerFactories.map { consumerFactory -> + consumerFactory.create( + MediatorConsumerConfig( + config.messageProcessor.keyClass, + config.messageProcessor.eventValueClass, + ::onSerializationError + ) + ) + } + } + + private fun createProducers() { + check (config.producerFactories.isNotEmpty()) { + "None producer factory set in configuration" + } + producers = config.producerFactories.map { producerFactory -> + producerFactory.create( + MediatorProducerConfig( + ::onSerializationError + ) + ) + } + } + + private fun closeConsumersAndProducers() { + consumers.forEach { it.close() } + producers.forEach { it.close() } + } + + private fun processEvents(isTaskStopped: Boolean) { + var attempts = 0 + while (!isTaskStopped) { + try { + log.debug { "Polling and processing events" } + val messages = poll(pollTimeoutInNanos) + val msgGroups = messages.groupBy { it.key } +// val states = stateManager.get( +// config.messageProcessor.stateValueClass, msgGroups.keys.mapTo(HashSet()) { it.toString() } +// ) + val processorTasks = msgGroups.map { msgGroup -> + val key = msgGroup.key.toString() + val events = msgGroup.value.map { it } + ProcessorTask( + key, + config.messageProcessor.stateValueClass, +// states[key], + events, + config.messageProcessor, + stateManager + ) + } + + processorTasks.map { processorTask -> + taskManager.execute(TaskType.SHORT_RUNNING, processorTask::run) + .thenApply { //outputEvents -> + // TODO + } + } +// val commitResults = consumers.map { consumer -> +// consumer.commitAsync() +// } + + } catch (ex: Exception) { + when (ex) { + is CordaMessageAPIIntermittentException -> { + attempts++ + // TODO handleProcessEventRetries(attempts, ex) + } + + else -> { + throw CordaMessageAPIFatalException( + "Multi-source event mediator ${config.name} failed to process messages, " + + "Fatal error occurred.", ex + ) + } + } + } + } + } + + private fun poll(pollTimeoutInNanos: Long): List> { + val maxEndTime = System.nanoTime() + pollTimeoutInNanos + return consumers.map { consumer -> + val remainingTime = (maxEndTime - System.nanoTime()).coerceAtLeast(0) + consumer.poll(Duration.ofNanos(remainingTime)) + }.flatten() + } + +// private fun generateDeadLetterRecord(event: CordaConsumerRecord, state: S?): Record<*, *> { +// val keyBytes = ByteBuffer.wrap(cordaAvroSerializer.serialize(event.key)) +// val stateBytes = +// if (state != null) ByteBuffer.wrap(cordaAvroSerializer.serialize(state)) else null +// val eventValue = event.value +// val eventBytes = +// if (eventValue != null) ByteBuffer.wrap(cordaAvroSerializer.serialize(eventValue)) else null +// return Record( +// Schemas.getDLQTopic(eventTopic), event.key, +// StateAndEventDeadLetterRecord(clock.instant(), keyBytes, stateBytes, eventBytes) +// ) +// } +} \ No newline at end of file diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/ProcessorTask.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/ProcessorTask.kt new file mode 100644 index 00000000000..a7193cb74f2 --- /dev/null +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/ProcessorTask.kt @@ -0,0 +1,37 @@ +package net.corda.messaging.mediator + +import net.corda.messagebus.api.consumer.CordaConsumerRecord +import net.corda.messaging.api.mediator.statemanager.State +import net.corda.messaging.api.mediator.statemanager.StateManager +import net.corda.messaging.api.processor.StateAndEventProcessor +import net.corda.messaging.api.records.Record +import net.corda.messaging.utils.toRecord + +class ProcessorTask( + private val key: String, + private val stateClass: Class, + private val events: Collection>, + private val processor: StateAndEventProcessor, + private val stateManager: StateManager, +) { + + private var responseEvents = emptyList>() + fun run() { + val state = stateManager.get(stateClass, setOf(key))[key] + ?: State(null, key) + var updatedState = state.state + responseEvents = events.map { event -> + val response = processor.onNext(updatedState, event.toRecord()) + response.updatedState?.let { updatedState = it } + response.responseEvents + }.flatten() + + val newState = State( + updatedState!!, + key, + state.metadata, + state.version + 1 + ) + stateManager.update(stateClass, setOf(newState)) + } +} \ No newline at end of file diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/EventMediatorFactoryImpl.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/EventMediatorFactoryImpl.kt new file mode 100644 index 00000000000..eabf177a054 --- /dev/null +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/EventMediatorFactoryImpl.kt @@ -0,0 +1,50 @@ +package net.corda.messaging.mediator.factory + +import net.corda.avro.serialization.CordaAvroSerializationFactory +import net.corda.lifecycle.LifecycleCoordinatorFactory +import net.corda.messaging.api.mediator.MultiSourceEventMediator +import net.corda.messaging.api.mediator.config.EventMediatorConfig +import net.corda.messaging.api.mediator.factory.EventMediatorFactory +import net.corda.messaging.api.mediator.statemanager.StateManager +import net.corda.messaging.api.mediator.taskmanager.TaskManager +import net.corda.messaging.mediator.MultiSourceEventMediatorImpl +import org.osgi.service.component.annotations.Activate +import org.osgi.service.component.annotations.Component +import org.osgi.service.component.annotations.Reference + +// @Suppress("LongParameterList") +@Component(service = [EventMediatorFactory::class]) +class EventMediatorFactoryImpl @Activate constructor( + @Reference(service = CordaAvroSerializationFactory::class) + private val cordaAvroSerializationFactory: CordaAvroSerializationFactory, + @Reference(service = TaskManager::class) + private val taskManager: TaskManager, + @Reference(service = StateManager::class) + private val stateManager: StateManager, + @Reference(service = LifecycleCoordinatorFactory::class) + private val lifecycleCoordinatorFactory: LifecycleCoordinatorFactory, +): EventMediatorFactory { + + override fun createMultiSourceEventMediator( + eventMediatorConfig: EventMediatorConfig, + ): MultiSourceEventMediator { + val serializer = cordaAvroSerializationFactory.createAvroSerializer { } + return MultiSourceEventMediatorImpl( + eventMediatorConfig, + serializer, + stateManager, + taskManager, + lifecycleCoordinatorFactory, + ) + } + +// val msgConfig = messagingConfig.withFallback(defaults) +// messagingConfig.getInt(BootConfig.INSTANCE_ID), +// Duration.ofMillis(messagingConfig.getLong(MessagingConfig.Subscription.POLL_TIMEOUT)), +// Duration.ofMillis(messagingConfig.getLong(MessagingConfig.Subscription.THREAD_STOP_TIMEOUT)), +// messagingConfig.getInt(MessagingConfig.Subscription.PROCESSOR_RETRIES), +// messagingConfig.getInt(MessagingConfig.Subscription.SUBSCRIBE_RETRIES), +// messagingConfig.getInt(MessagingConfig.Subscription.COMMIT_RETRIES), +// Duration.ofMillis(messagingConfig.getLong(MessagingConfig.Subscription.PROCESSOR_TIMEOUT)), +// messagingConfig +} \ No newline at end of file diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/statemanager/StateManagerImpl.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/statemanager/StateManagerImpl.kt new file mode 100644 index 00000000000..91079f0b9d6 --- /dev/null +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/statemanager/StateManagerImpl.kt @@ -0,0 +1,34 @@ +package net.corda.messaging.mediator.statemanager + +import net.corda.messaging.api.mediator.statemanager.State +import net.corda.messaging.api.mediator.statemanager.StateManager +import org.osgi.service.component.annotations.Activate +import org.osgi.service.component.annotations.Component +import java.time.Instant + +@Component(service = [StateManager::class]) +class StateManagerImpl @Activate constructor() : StateManager { + override fun create(clazz: Class, states: Set>) { + TODO("Not yet implemented") + } + + override fun get(clazz: Class, keys: Set): Map> { + TODO("Not yet implemented") + } + + override fun update(clazz: Class, states: Set>): Map> { + TODO("Not yet implemented") + } + + override fun delete(clazz: Class, keys: Set): Map> { + TODO("Not yet implemented") + } + + override fun getUpdatedBetween(clazz: Class, start: Instant, finish: Instant): Map> { + TODO("Not yet implemented") + } + + override fun close() { + TODO("Not yet implemented") + } +} \ No newline at end of file diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/taskmanager/TaskManagerImpl.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/taskmanager/TaskManagerImpl.kt new file mode 100644 index 00000000000..a86e0661998 --- /dev/null +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/taskmanager/TaskManagerImpl.kt @@ -0,0 +1,14 @@ +package net.corda.messaging.mediator.taskmanager + +import net.corda.messaging.api.mediator.taskmanager.TaskManager +import net.corda.messaging.api.mediator.taskmanager.TaskType +import org.osgi.service.component.annotations.Activate +import org.osgi.service.component.annotations.Component +import java.util.concurrent.CompletableFuture + +@Component(service = [TaskManager::class]) +class TaskManagerImpl @Activate constructor() : TaskManager { + override fun execute(type: TaskType, command: () -> T): CompletableFuture { + TODO("Not yet implemented") + } +} \ No newline at end of file diff --git a/libs/messaging/messaging/src/main/java/net/corda/messaging/api/mediator/config/package-info.java b/libs/messaging/messaging/src/main/java/net/corda/messaging/api/mediator/config/package-info.java new file mode 100644 index 00000000000..fd617f717ee --- /dev/null +++ b/libs/messaging/messaging/src/main/java/net/corda/messaging/api/mediator/config/package-info.java @@ -0,0 +1,4 @@ +@Export +package net.corda.messaging.api.mediator.config; + +import org.osgi.annotation.bundle.Export; diff --git a/libs/messaging/messaging/src/main/java/net/corda/messaging/api/mediator/factory/package-info.java b/libs/messaging/messaging/src/main/java/net/corda/messaging/api/mediator/factory/package-info.java new file mode 100644 index 00000000000..f3f5ee02007 --- /dev/null +++ b/libs/messaging/messaging/src/main/java/net/corda/messaging/api/mediator/factory/package-info.java @@ -0,0 +1,4 @@ +@Export +package net.corda.messaging.api.mediator.factory; + +import org.osgi.annotation.bundle.Export; diff --git a/libs/messaging/messaging/src/main/java/net/corda/messaging/api/mediator/package-info.java b/libs/messaging/messaging/src/main/java/net/corda/messaging/api/mediator/package-info.java new file mode 100644 index 00000000000..e50971fad9c --- /dev/null +++ b/libs/messaging/messaging/src/main/java/net/corda/messaging/api/mediator/package-info.java @@ -0,0 +1,4 @@ +@Export +package net.corda.messaging.api.mediator; + +import org.osgi.annotation.bundle.Export; diff --git a/libs/messaging/messaging/src/main/java/net/corda/messaging/api/mediator/statemanager/package-info.java b/libs/messaging/messaging/src/main/java/net/corda/messaging/api/mediator/statemanager/package-info.java new file mode 100644 index 00000000000..f83836d0d17 --- /dev/null +++ b/libs/messaging/messaging/src/main/java/net/corda/messaging/api/mediator/statemanager/package-info.java @@ -0,0 +1,4 @@ +@Export +package net.corda.messaging.api.mediator.statemanager; + +import org.osgi.annotation.bundle.Export; diff --git a/libs/messaging/messaging/src/main/java/net/corda/messaging/api/mediator/taskmanager/package-info.java b/libs/messaging/messaging/src/main/java/net/corda/messaging/api/mediator/taskmanager/package-info.java new file mode 100644 index 00000000000..ee350bad555 --- /dev/null +++ b/libs/messaging/messaging/src/main/java/net/corda/messaging/api/mediator/taskmanager/package-info.java @@ -0,0 +1,4 @@ +@Export +package net.corda.messaging.api.mediator.taskmanager; + +import org.osgi.annotation.bundle.Export; diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MediatorConsumer.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MediatorConsumer.kt new file mode 100644 index 00000000000..fff1b8793ce --- /dev/null +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MediatorConsumer.kt @@ -0,0 +1,31 @@ +package net.corda.messaging.api.mediator + +import net.corda.messagebus.api.CordaTopicPartition +import net.corda.messagebus.api.consumer.CordaConsumerRecord +import java.time.Duration +import java.util.concurrent.CompletableFuture + +/** + * Multi-source event mediator message consumer. + */ +interface MediatorConsumer : AutoCloseable { + + /** + * Subscribes to a message bus. + */ + fun subscribe() + + /** + * Poll messages from the consumer with a [timeout]. + * + * @param timeout - The maximum time to block if there are no available messages. + */ + fun poll(timeout: Duration): List> + + /** + * Asynchronously commit the consumer offsets. + * + * @return [CompletableFuture] with committed offsets. + */ + fun commitAsync(): CompletableFuture> +} diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MediatorProducer.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MediatorProducer.kt new file mode 100644 index 00000000000..71c60ed7092 --- /dev/null +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MediatorProducer.kt @@ -0,0 +1,22 @@ +package net.corda.messaging.api.mediator + +/** + * Multi-source event mediator message producer. + */ +interface MediatorProducer: AutoCloseable { + + /** + * Determines whether producer supports request-reply messaging pattern. + */ + val isRequestReply: Boolean + get() = false + + /** + * Sends message to producer's endpoint and returns reply. + * + * @param message Message + * @returns ProducerReply Holds producer's reply if producer supports request-reply messaging pattern + * (@see [isRequestReply]). + */ + fun send(message: Message): ProducerReply +} \ No newline at end of file diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/Message.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/Message.kt new file mode 100644 index 00000000000..efa5c92cdb0 --- /dev/null +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/Message.kt @@ -0,0 +1,11 @@ +package net.corda.messaging.api.mediator + +/** + * Class for storing message data and metadata. + */ +data class Message( + /** Message body (payload). */ + val body: Any, + /** Message properties (metadata). */ + val properties: Map, +) diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MessageBusConsumer.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MessageBusConsumer.kt new file mode 100644 index 00000000000..7fca09c8b02 --- /dev/null +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MessageBusConsumer.kt @@ -0,0 +1,37 @@ +package net.corda.messaging.api.mediator + +import net.corda.messagebus.api.CordaTopicPartition +import net.corda.messagebus.api.consumer.CordaConsumer +import net.corda.messagebus.api.consumer.CordaConsumerRecord +import java.time.Duration +import java.util.concurrent.CompletableFuture + +/** + * Message bus consumer that reads messages from configured topic. + */ +class MessageBusConsumer( + private val topic: String, + private val consumer: CordaConsumer, +): MediatorConsumer { + + override fun subscribe() = + consumer.subscribe(topic) + + override fun poll(timeout: Duration): List> = + consumer.poll(timeout) + + override fun commitAsync(): CompletableFuture> { + val result = CompletableFuture>() + consumer.commitAsync { offsets, exception -> + if (exception != null) { + result.completeExceptionally(exception) + } else { + result.complete(offsets) + } + } + return result + } + + override fun close() = + consumer.close() +} \ No newline at end of file diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MessageBusProducer.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MessageBusProducer.kt new file mode 100644 index 00000000000..3b0e21d50c6 --- /dev/null +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MessageBusProducer.kt @@ -0,0 +1,19 @@ +package net.corda.messaging.api.mediator + +import net.corda.messagebus.api.producer.CordaProducer + +/** + * Message bus producer that sends messages to message bus topics. + */ +class MessageBusProducer( + private val name: String, + private val producer: CordaProducer, +): MediatorProducer { + + override fun send(message: Message): ProducerReply { + TODO("Not implemented yet") + } + + override fun close() = + producer.close() +} \ No newline at end of file diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MessageRouter.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MessageRouter.kt new file mode 100644 index 00000000000..f98a73b9f6e --- /dev/null +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MessageRouter.kt @@ -0,0 +1,6 @@ +package net.corda.messaging.api.mediator + +/** + * Multi-source event mediator's message router that routes messages generated by stae and event processor to producers. + */ +class MessageRouter \ No newline at end of file diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MultiSourceEventMediator.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MultiSourceEventMediator.kt new file mode 100644 index 00000000000..1b0fe273260 --- /dev/null +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MultiSourceEventMediator.kt @@ -0,0 +1,10 @@ +package net.corda.messaging.api.mediator + +import net.corda.messaging.api.processor.StateAndEventProcessor +import net.corda.messaging.api.subscription.SubscriptionBase + +/** + * Multi-source event mediator is used to consume messages from multiple sources using [MediatorConsumer]s, + * process them using [StateAndEventProcessor] to generate output messages that are then sent to [MediatorProducer]s. + */ +interface MultiSourceEventMediator : SubscriptionBase \ No newline at end of file diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/ProducerReply.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/ProducerReply.kt new file mode 100644 index 00000000000..c1caab27822 --- /dev/null +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/ProducerReply.kt @@ -0,0 +1,11 @@ +package net.corda.messaging.api.mediator + +/** + * Class to store reply of the [MediatorProducer]. + */ +data class ProducerReply( + /** Reply message (set only if [MediatorProducer] supports request-reply messaging pattern). */ + val reply: Message?, + /** Exception (set in case of error). */ + val exception: Exception?, +) diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/config/EventMediatorConfig.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/config/EventMediatorConfig.kt new file mode 100644 index 00000000000..7efefaabde5 --- /dev/null +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/config/EventMediatorConfig.kt @@ -0,0 +1,27 @@ +package net.corda.messaging.api.mediator.config + +import net.corda.libs.configuration.SmartConfig +import net.corda.messaging.api.mediator.MessageRouter +import net.corda.messaging.api.mediator.MultiSourceEventMediator +import net.corda.messaging.api.mediator.factory.MediatorConsumerFactory +import net.corda.messaging.api.mediator.factory.MediatorProducerFactory +import net.corda.messaging.api.processor.StateAndEventProcessor + +/** + * Class to store the required params to create a [MultiSourceEventMediator]. + * + * @property name The unique name for a multi-source event mediator. + * @property messagingConfig Messaging related configuration. + * @property consumerFactories Factories for creating message consumers. + * @property producerFactories Factories for creating message producers. + * @property messageProcessor State and event processor. + * @property messageRouter Message router that routes output messages of the state and event processor to producers. + */ +data class EventMediatorConfig( + val name: String, + val messagingConfig : SmartConfig, + val consumerFactories: Collection, + val producerFactories: Collection, + val messageProcessor : StateAndEventProcessor, + val messageRouter: MessageRouter, +) \ No newline at end of file diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/config/EventMediatorConfigBuilder.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/config/EventMediatorConfigBuilder.kt new file mode 100644 index 00000000000..0ceb4f373c1 --- /dev/null +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/config/EventMediatorConfigBuilder.kt @@ -0,0 +1,63 @@ +package net.corda.messaging.api.mediator.config + +import net.corda.libs.configuration.SmartConfig +import net.corda.messaging.api.mediator.MessageRouter +import net.corda.messaging.api.mediator.MultiSourceEventMediator +import net.corda.messaging.api.mediator.factory.MediatorConsumerFactory +import net.corda.messaging.api.mediator.factory.MediatorProducerFactory +import net.corda.messaging.api.processor.StateAndEventProcessor + +/** + * Builder for creating [EventMediatorConfig]. + */ +class EventMediatorConfigBuilder { + + private var name : String? = null + private var messagingConfig : SmartConfig? = null + private var consumerFactories = emptyArray() + private var producerFactories = emptyArray() + private var messageProcessor : StateAndEventProcessor? = null + private var messageRouter: MessageRouter? = null + + /** Sets name for [MultiSourceEventMediator]. */ + fun name(name: String) = + apply { this.name = name } + + /** Sets messaging related configuration for [MultiSourceEventMediator]. */ + fun messagingConfig(messagingConfig: SmartConfig) = + apply { this.messagingConfig = messagingConfig } + + /** Sets factories for creating message consumers. */ + fun consumerFactories(vararg consumerFactories: MediatorConsumerFactory) = + apply { this.consumerFactories = arrayOf(*consumerFactories) } + + /** Sets factories for creating message producers. */ + fun producerFactories(vararg producerFactories: MediatorProducerFactory) = + apply { this.producerFactories = arrayOf(*producerFactories) } + + /** Sets state and event processor for [MultiSourceEventMediator]. */ + fun messageProcessor(messageProcessor: StateAndEventProcessor) = + apply { this.messageProcessor = messageProcessor } + + /** Sets message router. */ + fun messageRouter(messageRouter: MessageRouter) = + apply { this.messageRouter = messageRouter } + + /** Builds [EventMediatorConfig]. */ + fun build(): EventMediatorConfig { + check(name != null) { "Name not set" } + check(messagingConfig != null) { "Messaging config not set" } + check(consumerFactories.isNotEmpty()) { "At least on consumer factory has to be set" } + check(producerFactories.isNotEmpty()) { "At least on producer factory has to be set" } + check(messageProcessor != null) { "Message processor not set" } + check(messageRouter != null) { "Message router not set" } + return EventMediatorConfig( + name!!, + messagingConfig!!, + consumerFactories.asList(), + producerFactories.asList(), + messageProcessor!!, + messageRouter!! + ) + } +} \ No newline at end of file diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/config/MediatorConsumerConfig.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/config/MediatorConsumerConfig.kt new file mode 100644 index 00000000000..0a05faeaf26 --- /dev/null +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/config/MediatorConsumerConfig.kt @@ -0,0 +1,16 @@ +package net.corda.messaging.api.mediator.config + +import net.corda.messaging.api.mediator.MediatorConsumer + +/** + * Class to store configuration for [MediatorConsumer]. + * + * @property keyClass Class of the message key. + * @property valueClass Class of the message value (payload). + * @property onSerializationError Handler for serialization errors. + */ +class MediatorConsumerConfig ( + val keyClass: Class, + val valueClass: Class, + val onSerializationError: (ByteArray) -> Unit, +) \ No newline at end of file diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/config/MediatorProducerConfig.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/config/MediatorProducerConfig.kt new file mode 100644 index 00000000000..eed07aac37c --- /dev/null +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/config/MediatorProducerConfig.kt @@ -0,0 +1,12 @@ +package net.corda.messaging.api.mediator.config + +import net.corda.messaging.api.mediator.MediatorProducer + +/** + * Class to store configuration for [MediatorProducer]. + * + * @property onSerializationError Handler for serialization errors. + */ +class MediatorProducerConfig ( + val onSerializationError: (ByteArray) -> Unit, +) \ No newline at end of file diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/EventMediatorFactory.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/EventMediatorFactory.kt new file mode 100644 index 00000000000..7b1150d82a0 --- /dev/null +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/EventMediatorFactory.kt @@ -0,0 +1,22 @@ +package net.corda.messaging.api.mediator.factory + +import net.corda.messaging.api.mediator.MultiSourceEventMediator +import net.corda.messaging.api.mediator.config.EventMediatorConfig + +/** + * Factory for creating multi-source event mediator configuration. + */ +interface EventMediatorFactory { + + /** + * Creates a multi-source event mediator configuration. + * + * @param The type of the event key. + * @param The type of the state. + * @param The type of the event. + * @param eventMediatorConfig Multi-source event mediator configuration. + */ + fun createMultiSourceEventMediator( + eventMediatorConfig: EventMediatorConfig, + ): MultiSourceEventMediator +} diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MediatorConsumerFactory.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MediatorConsumerFactory.kt new file mode 100644 index 00000000000..9116d902189 --- /dev/null +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MediatorConsumerFactory.kt @@ -0,0 +1,19 @@ +package net.corda.messaging.api.mediator.factory + +import net.corda.messaging.api.mediator.MediatorConsumer +import net.corda.messaging.api.mediator.config.MediatorConsumerConfig + +/** + * Factory for creating multi-source event mediator consumers. + */ +interface MediatorConsumerFactory { + + /** + * Creates a multi-source event mediator consumer. + * + * @param The type of the message key. + * @param The type of the message value (payload). + * @param config Multi-source event mediator consumer configuration. + */ + fun create(config: MediatorConsumerConfig): MediatorConsumer +} \ No newline at end of file diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MediatorProducerFactory.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MediatorProducerFactory.kt new file mode 100644 index 00000000000..4aff78355da --- /dev/null +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MediatorProducerFactory.kt @@ -0,0 +1,17 @@ +package net.corda.messaging.api.mediator.factory + +import net.corda.messaging.api.mediator.MediatorProducer +import net.corda.messaging.api.mediator.config.MediatorProducerConfig + +/** + * Factory for creating multi-source event mediator producers. + */ +interface MediatorProducerFactory { + + /** + * Creates a multi-source event mediator producer. + * + * @param config Multi-source event mediator producer configuration. + */ + fun create(config: MediatorProducerConfig): MediatorProducer +} \ No newline at end of file diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MessageBusConsumerFactory.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MessageBusConsumerFactory.kt new file mode 100644 index 00000000000..c6aff5cbabd --- /dev/null +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MessageBusConsumerFactory.kt @@ -0,0 +1,105 @@ +package net.corda.messaging.api.mediator.factory + +import net.corda.libs.configuration.SmartConfig +import net.corda.messagebus.api.configuration.ConsumerConfig +import net.corda.messagebus.api.constants.ConsumerRoles +import net.corda.messagebus.api.consumer.CordaConsumer +import net.corda.messagebus.api.consumer.builder.CordaConsumerBuilder +import net.corda.messaging.api.mediator.MediatorConsumer +import net.corda.messaging.api.mediator.MessageBusConsumer +import net.corda.messaging.api.mediator.config.MediatorConsumerConfig +import java.util.UUID + +/** + * Factory for creating multi-source event mediator message bus consumers. + * + * @param topicName Topic name. + * @param groupName Consumer group name. + * @param messageBusConfig Message bus related configuration. + * @param cordaConsumerBuilder [CordaConsumer] builder. + */ +class MessageBusConsumerFactory( + private val topicName: String, + private val groupName: String, + private val messageBusConfig: SmartConfig, + private val cordaConsumerBuilder: CordaConsumerBuilder, +): MediatorConsumerFactory { + + override fun create(config: MediatorConsumerConfig): MediatorConsumer { + val subscriptionType = "MultiSourceSubscription" + val uniqueId = UUID.randomUUID().toString() + val clientId = "$subscriptionType--$groupName--$topicName--$uniqueId" + + val eventConsumerConfig = ConsumerConfig( + groupName, + "$clientId-eventConsumer", + ConsumerRoles.SAE_EVENT + ) + + val eventConsumer = cordaConsumerBuilder.createConsumer( + eventConsumerConfig, + messageBusConfig, + config.keyClass, + config.valueClass, + config.onSerializationError + ) + + return MessageBusConsumer( + topicName, + eventConsumer, + ) + } + +// fun createMediatorConsumerFactory( +// topicName: String, +// groupName: String, +// messageBusConfig: SmartConfig, +// cordaConsumerBuilder: CordaConsumerBuilder, +// ): MediatorConsumerFactory { +// return object : MediatorConsumerFactory { +// override fun create(config: MediatorConsumerConfig): MediatorConsumer = +// createMessageBusConsumer( +// topicName, +// groupName, +// config.keyClass, +// config.valueClass, +// config.onSerializationError, +// messageBusConfig, +// cordaConsumerBuilder) +// } +// } +// +// fun createMessageBusConsumer( +// topicName: String, +// groupName: String, +// keyClass: Class, +// valueClass: Class, +// onSerializationError: (ByteArray) -> Unit, +// messageBusConfig: SmartConfig, +// cordaConsumerBuilder: CordaConsumerBuilder, +// ): MessageBusConsumer { +// val subscriptionType = "MultiSourceSubscription" +// val uniqueId = UUID.randomUUID().toString() +// val clientId = "$subscriptionType--$groupName--$topicName--$uniqueId" +// +// val eventConsumerConfig = +// ConsumerConfig( +// groupName, +// "$clientId-eventConsumer", +// ConsumerRoles.SAE_EVENT +// ) +// val eventConsumer = cordaConsumerBuilder.createConsumer( +// eventConsumerConfig, +// messageBusConfig, +// keyClass, +// valueClass, +// onSerializationError +// ) +// +// return MessageBusConsumer( +// topicName, +// eventConsumer, +// ) +// } + +} \ No newline at end of file diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MessageBusProducerFactory.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MessageBusProducerFactory.kt new file mode 100644 index 00000000000..2b258e70544 --- /dev/null +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MessageBusProducerFactory.kt @@ -0,0 +1,50 @@ +package net.corda.messaging.api.mediator.factory + +import net.corda.libs.configuration.SmartConfig +import net.corda.messagebus.api.configuration.ProducerConfig +import net.corda.messagebus.api.constants.ProducerRoles +import net.corda.messagebus.api.producer.CordaProducer +import net.corda.messagebus.api.producer.builder.CordaProducerBuilder +import net.corda.messaging.api.mediator.MediatorProducer +import net.corda.messaging.api.mediator.MessageBusProducer +import net.corda.messaging.api.mediator.config.MediatorProducerConfig +import net.corda.schema.configuration.BootConfig +import java.util.UUID + +/** + * Factory for creating multi-source event mediator message bus producers. + * + * @param name Producer name. + * @param messageBusConfig Message bus related configuration. + * @param cordaProducerBuilder [CordaProducer] builder. + */ +class MessageBusProducerFactory( + private val name: String, + private val messageBusConfig: SmartConfig, + private val cordaProducerBuilder: CordaProducerBuilder, +): MediatorProducerFactory { + + override fun create(config: MediatorProducerConfig): MediatorProducer { + val uniqueId = UUID.randomUUID().toString() + val clientId = "$name--$uniqueId" + + val eventProducerConfig = ProducerConfig( + clientId, + messageBusConfig.getInt(BootConfig.INSTANCE_ID), + transactional = false, + ProducerRoles.SAE_PRODUCER, + throwOnSerializationError = false + ) + + val eventProducer = cordaProducerBuilder.createProducer( + eventProducerConfig, + messageBusConfig, + config.onSerializationError + ) + + return MessageBusProducer( + name, + eventProducer, + ) + } +} \ No newline at end of file diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/statemanager/Metadata.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/statemanager/Metadata.kt new file mode 100644 index 00000000000..b42f2542bf3 --- /dev/null +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/statemanager/Metadata.kt @@ -0,0 +1,40 @@ +package net.corda.messaging.api.mediator.statemanager + +/** + * Mutable map that allows only primitive types to be used as values. + */ +class Metadata( + private val map: MutableMap = mutableMapOf() +) : MutableMap by map { + + private val supportedBoxedPrimitives = listOf( + String::class.java, + Byte::class.java, + Short::class.java, + Integer::class.java, + Long::class.java, + Float::class.java, + Double::class.java, + Character::class.java + ) + + private fun isPrimitiveOrBoxedValue(value: V): Boolean { + return when { + value::class.java.isPrimitive -> true + supportedBoxedPrimitives.contains(value::class.java) -> true + else -> false + } + } + + override fun put(key: String, value: V): V? { + if (!isPrimitiveOrBoxedValue(value)) { + throw IllegalArgumentException("Only primitive types are allowed: ${value::class.simpleName}") + } + + return map.put(key, value) + } +} + +fun metadata(): Metadata = Metadata() + +fun metadata(vararg pairs: Pair): Metadata = Metadata(mutableMapOf(*pairs)) \ No newline at end of file diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/statemanager/State.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/statemanager/State.kt new file mode 100644 index 00000000000..6153f97b8c0 --- /dev/null +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/statemanager/State.kt @@ -0,0 +1,34 @@ +package net.corda.messaging.api.mediator.statemanager + +import java.time.Instant + +/** + * A state managed via the state manager. + */ +data class State( + /** + * The typed state itself. + */ + val state: S, + + /** + * Identifier for the state. + */ + val key: String, + + /** + * Arbitrary Map of primitive types that can be used to store and query extra data associated with the state + * that must generally not be part of the state itself. + */ + val metadata: Metadata = Metadata(), + + /** + * Version of the state. + */ + val version: Int = -1, + + /** + * Time when the state was last modified. + */ + val modifiedTime: Instant = Instant.now(), +) diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/statemanager/StateManager.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/statemanager/StateManager.kt new file mode 100644 index 00000000000..05496f8b85c --- /dev/null +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/statemanager/StateManager.kt @@ -0,0 +1,44 @@ +package net.corda.messaging.api.mediator.statemanager + +import java.time.Instant + +interface StateManager : AutoCloseable { + /** + * Create [states] into the underlying storage. + * Control is only returned to the caller once all [states] have been updated and replicas of the underlying + * persistent storage, if any, are synced. + */ + fun create(clazz: Class, states: Set>) + + /** + * Get all states referenced by [keys]. + * Only states that have been successfully committed and distributed within the underlying persistent + * storage are returned. + */ + fun get(clazz: Class, keys: Set): Map> + + /** + * Update [states] within the underlying storage. + * Control is only returned to the caller once all [states] have been updated and replicas of the underlying + * persistent storage, if any, are synced. + * + * @return states that could not be updated due to mismatch versions. + */ + fun update(clazz: Class, states: Set>): Map> + + /** + * Delete all states referenced by [keys] from the underlying storage. + * Control is only returned to the caller once all states have been deleted and replicas of the underlying + * persistent storage, if any, are synced. + * + * @return states that could not be deleted due to mismatch versions. + */ + fun delete(clazz: Class, keys: Set): Map> + + /** + * Retrieve all states that were last updated between [start] and [finish] times. + * + * @return states that were last updated between [start] and [finish] times. + */ + fun getUpdatedBetween(clazz: Class, start: Instant, finish: Instant): Map> +} \ No newline at end of file diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/taskmanager/TaskManager.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/taskmanager/TaskManager.kt new file mode 100644 index 00000000000..4baa467b384 --- /dev/null +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/taskmanager/TaskManager.kt @@ -0,0 +1,10 @@ +package net.corda.messaging.api.mediator.taskmanager + +import java.util.concurrent.CompletableFuture + +enum class TaskType { + SHORT_RUNNING, LONG_RUNNING +} +interface TaskManager { + fun execute(type: TaskType, command: () -> T): CompletableFuture +} \ No newline at end of file From 951a6b64c43696f20f7f329f4785c12eda6873a0 Mon Sep 17 00:00:00 2001 From: Miljenko Brkic Date: Mon, 11 Sep 2023 16:30:16 +0100 Subject: [PATCH 02/50] CORE-16199 Using the latest version of StateManager API --- .../mediator/MultiSourceEventMediatorImpl.kt | 10 ++-- .../corda/messaging/mediator/ProcessorTask.kt | 28 +++++++---- .../factory/EventMediatorFactoryImpl.kt | 7 ++- .../mediator/statemanager/StateManagerImpl.kt | 16 +++++-- .../mediator/config/EventMediatorConfig.kt | 4 +- .../api/mediator/statemanager/Metadata.kt | 46 ++++++++++-------- .../api/mediator/statemanager/State.kt | 44 +++++++++++++---- .../api/mediator/statemanager/StateManager.kt | 48 +++++++++++++++---- 8 files changed, 142 insertions(+), 61 deletions(-) diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt index 9f1e62a502b..f89e49115c5 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt @@ -1,5 +1,6 @@ package net.corda.messaging.mediator +import net.corda.avro.serialization.CordaAvroDeserializer import net.corda.avro.serialization.CordaAvroSerializer import net.corda.lifecycle.LifecycleCoordinatorFactory import net.corda.lifecycle.LifecycleCoordinatorName @@ -21,9 +22,11 @@ import org.slf4j.LoggerFactory import java.time.Duration import java.util.UUID +@Suppress("LongParameterList") class MultiSourceEventMediatorImpl( private val config: EventMediatorConfig, - private val cordaAvroSerializer: CordaAvroSerializer, + private val serializer: CordaAvroSerializer, + private val stateDeserializer: CordaAvroDeserializer, private val stateManager: StateManager, private val taskManager: TaskManager, lifecycleCoordinatorFactory: LifecycleCoordinatorFactory, @@ -156,11 +159,12 @@ class MultiSourceEventMediatorImpl( val events = msgGroup.value.map { it } ProcessorTask( key, - config.messageProcessor.stateValueClass, // states[key], events, config.messageProcessor, - stateManager + stateManager, + serializer, + stateDeserializer, ) } diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/ProcessorTask.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/ProcessorTask.kt index a7193cb74f2..0ea2a7273a3 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/ProcessorTask.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/ProcessorTask.kt @@ -1,37 +1,47 @@ package net.corda.messaging.mediator +import net.corda.avro.serialization.CordaAvroDeserializer +import net.corda.avro.serialization.CordaAvroSerializer import net.corda.messagebus.api.consumer.CordaConsumerRecord +import net.corda.messaging.api.mediator.statemanager.Metadata import net.corda.messaging.api.mediator.statemanager.State import net.corda.messaging.api.mediator.statemanager.StateManager import net.corda.messaging.api.processor.StateAndEventProcessor import net.corda.messaging.api.records.Record import net.corda.messaging.utils.toRecord +import net.corda.v5.base.exceptions.CordaRuntimeException +@Suppress("LongParameterList") class ProcessorTask( private val key: String, - private val stateClass: Class, private val events: Collection>, private val processor: StateAndEventProcessor, private val stateManager: StateManager, + private val serializer: CordaAvroSerializer, + private val stateDeserializer: CordaAvroDeserializer, ) { - private var responseEvents = emptyList>() + var responseEvents = emptyList>() + private set fun run() { - val state = stateManager.get(stateClass, setOf(key))[key] - ?: State(null, key) - var updatedState = state.state + val persistedState = stateManager.get(setOf(key))[key] + + var updatedState = persistedState?.value?.let { stateDeserializer.deserialize(it) } responseEvents = events.map { event -> val response = processor.onNext(updatedState, event.toRecord()) response.updatedState?.let { updatedState = it } response.responseEvents }.flatten() + val serializedState = serializer.serialize(updatedState!!) + ?: throw CordaRuntimeException("Cannot serialize updated state: $updatedState") + val newState = State( - updatedState!!, key, - state.metadata, - state.version + 1 + serializedState, + (persistedState?.version ?: -1) + 1, + persistedState?.metadata ?: Metadata() ) - stateManager.update(stateClass, setOf(newState)) + stateManager.update(setOf(newState)) } } \ No newline at end of file diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/EventMediatorFactoryImpl.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/EventMediatorFactoryImpl.kt index eabf177a054..4cedbea7644 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/EventMediatorFactoryImpl.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/EventMediatorFactoryImpl.kt @@ -12,7 +12,7 @@ import org.osgi.service.component.annotations.Activate import org.osgi.service.component.annotations.Component import org.osgi.service.component.annotations.Reference -// @Suppress("LongParameterList") +@Suppress("LongParameterList") @Component(service = [EventMediatorFactory::class]) class EventMediatorFactoryImpl @Activate constructor( @Reference(service = CordaAvroSerializationFactory::class) @@ -29,9 +29,14 @@ class EventMediatorFactoryImpl @Activate constructor( eventMediatorConfig: EventMediatorConfig, ): MultiSourceEventMediator { val serializer = cordaAvroSerializationFactory.createAvroSerializer { } + val stateDeserializer = cordaAvroSerializationFactory.createAvroDeserializer( + {}, + eventMediatorConfig.messageProcessor.stateValueClass + ) return MultiSourceEventMediatorImpl( eventMediatorConfig, serializer, + stateDeserializer, stateManager, taskManager, lifecycleCoordinatorFactory, diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/statemanager/StateManagerImpl.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/statemanager/StateManagerImpl.kt index 91079f0b9d6..1701e215923 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/statemanager/StateManagerImpl.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/statemanager/StateManagerImpl.kt @@ -1,5 +1,6 @@ package net.corda.messaging.mediator.statemanager +import net.corda.messaging.api.mediator.statemanager.Operation import net.corda.messaging.api.mediator.statemanager.State import net.corda.messaging.api.mediator.statemanager.StateManager import org.osgi.service.component.annotations.Activate @@ -8,27 +9,32 @@ import java.time.Instant @Component(service = [StateManager::class]) class StateManagerImpl @Activate constructor() : StateManager { - override fun create(clazz: Class, states: Set>) { + override fun create(states: Collection): Map { TODO("Not yet implemented") } - override fun get(clazz: Class, keys: Set): Map> { + override fun get(keys: Collection): Map { TODO("Not yet implemented") } - override fun update(clazz: Class, states: Set>): Map> { + override fun update(states: Collection): Map { TODO("Not yet implemented") } - override fun delete(clazz: Class, keys: Set): Map> { + override fun delete(states: Collection): Map { TODO("Not yet implemented") } - override fun getUpdatedBetween(clazz: Class, start: Instant, finish: Instant): Map> { + override fun getUpdatedBetween(start: Instant, finish: Instant): Map { + TODO("Not yet implemented") + } + + override fun find(key: String, operation: Operation, value: Any): Map { TODO("Not yet implemented") } override fun close() { TODO("Not yet implemented") } + } \ No newline at end of file diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/config/EventMediatorConfig.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/config/EventMediatorConfig.kt index 7efefaabde5..5b99f0f3cc0 100644 --- a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/config/EventMediatorConfig.kt +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/config/EventMediatorConfig.kt @@ -14,8 +14,8 @@ import net.corda.messaging.api.processor.StateAndEventProcessor * @property messagingConfig Messaging related configuration. * @property consumerFactories Factories for creating message consumers. * @property producerFactories Factories for creating message producers. - * @property messageProcessor State and event processor. - * @property messageRouter Message router that routes output messages of the state and event processor to producers. + * @property messageProcessor State and event processor. + * @property messageRouter Message router that routes output messages of the state and event processor to producers. */ data class EventMediatorConfig( val name: String, diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/statemanager/Metadata.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/statemanager/Metadata.kt index b42f2542bf3..7e0a6aae5db 100644 --- a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/statemanager/Metadata.kt +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/statemanager/Metadata.kt @@ -1,40 +1,44 @@ package net.corda.messaging.api.mediator.statemanager +/** + * Supported comparison operations on metadata values. + */ +enum class Operation { + Equals, + NotEquals, + LesserThan, + GreaterThan, +} + /** * Mutable map that allows only primitive types to be used as values. */ -class Metadata( - private val map: MutableMap = mutableMapOf() -) : MutableMap by map { +class Metadata( + private val map: MutableMap = mutableMapOf() +) : MutableMap by map { - private val supportedBoxedPrimitives = listOf( + private val supportedType = listOf( String::class.java, - Byte::class.java, - Short::class.java, - Integer::class.java, - Long::class.java, - Float::class.java, - Double::class.java, - Character::class.java + java.lang.String::class.java, + Number::class.java, + java.lang.Number::class.java, + Boolean::class.java, + java.lang.Boolean::class.java, ) - private fun isPrimitiveOrBoxedValue(value: V): Boolean { - return when { - value::class.java.isPrimitive -> true - supportedBoxedPrimitives.contains(value::class.java) -> true - else -> false - } + private fun isPrimitiveOrBoxedValue(value: Any): Boolean { + return supportedType.any { it.isAssignableFrom(value.javaClass) } } - override fun put(key: String, value: V): V? { + override fun put(key: String, value: Any): Any? { if (!isPrimitiveOrBoxedValue(value)) { - throw IllegalArgumentException("Only primitive types are allowed: ${value::class.simpleName}") + throw IllegalArgumentException("Type not supported: ${value::class}") } return map.put(key, value) } } -fun metadata(): Metadata = Metadata() +fun metadata(): Metadata = Metadata() -fun metadata(vararg pairs: Pair): Metadata = Metadata(mutableMapOf(*pairs)) \ No newline at end of file +fun metadata(vararg pairs: Pair): Metadata = Metadata(mutableMapOf(*pairs)) \ No newline at end of file diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/statemanager/State.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/statemanager/State.kt index 6153f97b8c0..ad45d743e9c 100644 --- a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/statemanager/State.kt +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/statemanager/State.kt @@ -5,30 +5,54 @@ import java.time.Instant /** * A state managed via the state manager. */ -data class State( - /** - * The typed state itself. - */ - val state: S, - +data class State( /** * Identifier for the state. */ val key: String, /** - * Arbitrary Map of primitive types that can be used to store and query extra data associated with the state - * that must generally not be part of the state itself. + * The actual value of the state. */ - val metadata: Metadata = Metadata(), + val value: ByteArray, /** * Version of the state. */ val version: Int = -1, + /** + * Arbitrary Map of primitive types that can be used to store and query data associated with the state. + */ + val metadata: Metadata = Metadata(), + /** * Time when the state was last modified. */ val modifiedTime: Instant = Instant.now(), -) +) { + override fun equals(other: Any?): Boolean { + if (this === other) return true + if (javaClass != other?.javaClass) return false + + other as State + + if (key != other.key) return false + if (!value.contentEquals(other.value)) return false + if (version != other.version) return false + if (metadata != other.metadata) return false + if (modifiedTime != other.modifiedTime) return false + + return true + } + + override fun hashCode(): Int { + var result = key.hashCode() + result = 31 * result + value.contentHashCode() + result = 31 * result + version + result = 31 * result + metadata.hashCode() + result = 31 * result + modifiedTime.hashCode() + + return result + } +} \ No newline at end of file diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/statemanager/StateManager.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/statemanager/StateManager.kt index 05496f8b85c..7444042a26b 100644 --- a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/statemanager/StateManager.kt +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/statemanager/StateManager.kt @@ -2,43 +2,71 @@ package net.corda.messaging.api.mediator.statemanager import java.time.Instant +/** + * The [StateManager] provides functions to manage states within the underlying persistent storage. + */ interface StateManager : AutoCloseable { + /** * Create [states] into the underlying storage. * Control is only returned to the caller once all [states] have been updated and replicas of the underlying * persistent storage, if any, are synced. + * + * @param states Collection of states to be persisted. + * @return States that could not be created on the persistent storage, along with the actual reason for the failures. */ - fun create(clazz: Class, states: Set>) + fun create(states: Collection): Map /** * Get all states referenced by [keys]. * Only states that have been successfully committed and distributed within the underlying persistent * storage are returned. + * + * @param keys Collection of state keys to use when querying the persistent storage. + * @return States found in the underlying persistent storage. */ - fun get(clazz: Class, keys: Set): Map> + fun get(keys: Collection): Map /** * Update [states] within the underlying storage. * Control is only returned to the caller once all [states] have been updated and replicas of the underlying * persistent storage, if any, are synced. + * The operation is transactional, either all [states] are updated or none is. * - * @return states that could not be updated due to mismatch versions. + * @param states Collection of states to be updated. + * @return States that could not be updated due to mismatch versions. */ - fun update(clazz: Class, states: Set>): Map> + fun update(states: Collection): Map /** - * Delete all states referenced by [keys] from the underlying storage. + * Delete all [states] from the underlying storage. * Control is only returned to the caller once all states have been deleted and replicas of the underlying * persistent storage, if any, are synced. + * The operation is transactional, either all [states] are deleted or none is. * - * @return states that could not be deleted due to mismatch versions. + * @param states Collection of states to be deleted. + * @return States that could not be deleted due to mismatch versions. */ - fun delete(clazz: Class, keys: Set): Map> + fun delete(states: Collection): Map /** - * Retrieve all states that were last updated between [start] and [finish] times. + * Retrieve all states that were updated for the last time between [start] (inclusive) and [finish] (inclusive). + * + * @param start Time filter lower bound (inclusive). + * @param finish Time filter upper bound (inclusive). + * @return States that were last updated between [start] and [finish] times. + */ + fun getUpdatedBetween(start: Instant, finish: Instant): Map + + /** + * Retrieve states based on custom [operation] to be executed against a single [key] within the [State.metadata]. + * Only states that have been successfully committed and distributed within the underlying persistent + * storage are returned. * - * @return states that were last updated between [start] and [finish] times. + * @param key The name of the key in the [State.metadata] to apply the comparison on. + * @param operation The comparison operation to perform (">", "=", "<", "<>", etc.). + * @param value The value to compare against. + * @return states for which the [State.metadata] has [key] for which [value] matches [operation]. */ - fun getUpdatedBetween(clazz: Class, start: Instant, finish: Instant): Map> + fun find(key: String, operation: Operation, value: Any): Map } \ No newline at end of file From 3eebf02222c9201105d0664a98ef7ba5201ef191 Mon Sep 17 00:00:00 2001 From: ben-millar Date: Thu, 7 Sep 2023 15:33:57 +0100 Subject: [PATCH 03/50] CORE-16177 Adding CordaMessage, MessageProducer, and KafkaMessageProducerImpl --- .../producer/KafkaMessageProducerImpl.kt | 213 ++++++++++++++++++ .../messagebus/api/producer/CordaMessage.kt | 26 +++ .../api/producer/MessageProducer.kt | 27 +++ .../factory/MessageProducerFactory.kt | 11 + 4 files changed, 277 insertions(+) create mode 100644 libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/producer/KafkaMessageProducerImpl.kt create mode 100644 libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/CordaMessage.kt create mode 100644 libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/MessageProducer.kt create mode 100644 libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/factory/MessageProducerFactory.kt diff --git a/libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/producer/KafkaMessageProducerImpl.kt b/libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/producer/KafkaMessageProducerImpl.kt new file mode 100644 index 00000000000..183ba75ccda --- /dev/null +++ b/libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/producer/KafkaMessageProducerImpl.kt @@ -0,0 +1,213 @@ +package net.corda.messagebus.kafka.producer + +import io.micrometer.core.instrument.binder.MeterBinder +import net.corda.messagebus.api.producer.CordaMessage +import net.corda.messagebus.api.producer.CordaProducerRecord +import net.corda.messagebus.api.producer.MessageProducer +import net.corda.messagebus.kafka.config.ResolvedProducerConfig +import net.corda.messagebus.kafka.utils.toKafkaRecord +import net.corda.messaging.api.exception.CordaMessageAPIFatalException +import net.corda.messaging.api.exception.CordaMessageAPIIntermittentException +import net.corda.messaging.api.exception.CordaMessageAPIProducerRequiresReset +import net.corda.metrics.CordaMetrics +import net.corda.tracing.TraceContext +import net.corda.tracing.addTraceContextToRecord +import net.corda.tracing.getOrCreateBatchPublishTracing +import net.corda.tracing.traceSend +import net.corda.v5.base.exceptions.CordaRuntimeException +import org.apache.kafka.clients.producer.Callback +import org.apache.kafka.clients.producer.Producer +import org.apache.kafka.common.KafkaException +import org.apache.kafka.common.errors.AuthorizationException +import org.apache.kafka.common.errors.FencedInstanceIdException +import org.apache.kafka.common.errors.InterruptException +import org.apache.kafka.common.errors.InvalidProducerEpochException +import org.apache.kafka.common.errors.ProducerFencedException +import org.apache.kafka.common.errors.TimeoutException +import org.apache.kafka.common.errors.UnsupportedForMessageFormatException +import org.apache.kafka.common.errors.UnsupportedVersionException +import org.slf4j.Logger +import org.slf4j.LoggerFactory + +class KafkaMessageProducerImpl( + private val config: ResolvedProducerConfig, + private val producer: Producer, +// private val chunkSerializerService: ChunkSerializerService, + private val producerMetricsBinder: MeterBinder, +) : MessageProducer { + private val topicPrefix = config.topicPrefix + private val clientId = config.clientId + + init { + producerMetricsBinder.bindTo(CordaMetrics.registry) + } + + private companion object { + private val log: Logger = LoggerFactory.getLogger(this::class.java.enclosingClass) +// const val asyncChunkErrorMessage = "Tried to send record which requires chunking using an asynchronous producer" + + val fatalExceptions: Set> = setOf( + AuthorizationException::class.java, + FencedInstanceIdException::class.java, + ProducerFencedException::class.java, + UnsupportedForMessageFormatException::class.java, + UnsupportedVersionException::class.java + ) + + val transientExceptions: Set> = setOf( + TimeoutException::class.java, + InterruptException::class.java, + // Failure to commit here might be due to consumer kicked from group. + // Return as intermittent to trigger retry + InvalidProducerEpochException::class.java, + // See https://cwiki.apache.org/confluence/display/KAFKA/KIP-588%3A+Allow+producers+to+recover+gracefully+from+transaction+timeouts + // This exception means the coordinator has bumped the producer epoch because of a timeout of this producer. + // There is no other producer, we are not a zombie, and so don't need to be fenced, we can simply abort and retry. + KafkaException::class.java + ) + + val apiExceptions: Set> = setOf( + CordaMessageAPIFatalException::class.java, + CordaMessageAPIIntermittentException::class.java + ) + } + + private fun toTraceKafkaCallback(callback: MessageProducer.Callback, ctx: TraceContext) : Callback { + return Callback { m, ex -> + ctx.markInScope().use { + ctx.traceTag("send.offset", m.offset().toString()) + ctx.traceTag("send.partition", m.partition().toString()) + ctx.traceTag("send.topic", m.topic()) + callback.onCompletion(null, ex) + if (ex != null) { + ctx.errorAndFinish(ex) + } else { + ctx.finish() + } + } + } + } + + override fun send(message: CordaMessage<*>, callback: MessageProducer.Callback?) { + val headers = message.getProperty("headers") + val partition = message.getPropertyOrNull("partition") + getOrCreateBatchPublishTracing(clientId).begin(listOf(headers)) + tryWithCleanupOnFailure("send single message") { + sendMessage(message, partition, callback) + } + } + + override fun sendMessages(messages: List>) { + getOrCreateBatchPublishTracing(clientId).begin(messages.map { + it.getProperty("headers") + }) + tryWithCleanupOnFailure("send multiple message") { + messages.forEach { message -> + val partition = message.getPropertyOrNull("partition") + sendMessage(message, partition, callback = null) + } + } + } + + private fun sendMessage( + message: CordaMessage<*>, + partition: Int? = null, + callback: MessageProducer.Callback? = null + ) { + // TODO: Support message chunking + sendWholeMessage(message, partition, callback) + } + + private fun sendChunkedMessage( + messages: List>, + partition: Int? = null, + callback: MessageProducer.Callback? = null + ) { + TODO("Not yet implemented") + } + + private fun sendWholeMessage( + message: CordaMessage<*>, + partition: Int? = null, + callback: MessageProducer.Callback? = null + ) { + val headers = message.getProperty("headers") + + val record = CordaProducerRecord( + topic = message.getProperty("topic"), + key = message.getProperty("key"), + value = message.payload, + headers = headers + ) + + val traceContext = traceSend(headers, "send $clientId") + traceContext.markInScope().use { + try { + producer.send( + addTraceContextToRecord(record).toKafkaRecord(topicPrefix, partition), + toTraceKafkaCallback({res, ex-> callback?.onCompletion(res, ex)}, traceContext) + ) + } catch (ex: CordaRuntimeException) { + traceContext.errorAndFinish(ex) + val msg = "Failed to send record to topic ${record.topic} with key ${record.key}\"" + if (config.throwOnSerializationError) { + log.error(msg, ex) + throw ex + } else { + log.warn(msg, ex) + } + } catch (ex: Exception) { + traceContext.errorAndFinish(ex) + throw ex + } + } + } + + private fun tryWithCleanupOnFailure( + operation: String, + block: () -> Unit + ) { + try { + block() + getOrCreateBatchPublishTracing(config.clientId).complete() + } catch (ex: Exception) { + getOrCreateBatchPublishTracing(config.clientId).abort() + handleException(ex, operation) + } + } + + private fun handleException(ex: Exception, operation: String) { + val errorString = "$operation for CordaKafkaProducer with clientId ${config.clientId}" + when (ex::class.java) { + in fatalExceptions -> throw CordaMessageAPIFatalException("FatalError occurred $errorString", ex) + in transientExceptions -> throw CordaMessageAPIIntermittentException("Error occurred $errorString", ex) + in apiExceptions -> throw ex + IllegalStateException::class.java -> { + // It's not clear whether the producer is ok to abort and continue or not in this case, so play it safe + // and let the client know to create a new one. + throw CordaMessageAPIProducerRequiresReset("Error occurred $errorString", ex) + } + else -> { + // Here we do not know what the exact cause of the exception is, but we do know Kafka has not told us we + // must close down, nor has it told us we can abort and retry. In this instance the most sensible thing + // for the client to do would be to close this producer and create a new one. + throw CordaMessageAPIProducerRequiresReset("Unexpected error occurred $errorString", ex) + } + } + } + + override fun close() { + try { + producer.close() + } catch (ex: Exception) { + log.info( + "CordaKafkaProducer failed to close producer safely. This can be observed when there are " + + "no reachable brokers. ClientId: ${config.clientId}", ex + ) + } finally { + (producerMetricsBinder as? AutoCloseable)?.close() + } + } +} + +private typealias Headers = List> \ No newline at end of file diff --git a/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/CordaMessage.kt b/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/CordaMessage.kt new file mode 100644 index 00000000000..efd9f2f7088 --- /dev/null +++ b/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/CordaMessage.kt @@ -0,0 +1,26 @@ +package net.corda.messagebus.api.producer + +@Suppress("UNCHECKED_CAST") +data class CordaMessage( + val payload: T?, + val props: MutableMap = mutableMapOf() +) { + fun addProperty(property: Pair) { + props[property.first] = property.second + } + fun getProperty(id: String) : Any { + return getPropertyOrNull(id) ?: throw NoSuchElementException("") + } + + fun getProperty(id: String) : T { + return getPropertyOrNull(id) ?: throw NoSuchElementException("") + } + + fun getPropertyOrNull(id: String) : Any? { + return props[id] + } + + fun getPropertyOrNull(id: String) : T? { + return props[id] as? T ?: throw IllegalArgumentException("") + } +} \ No newline at end of file diff --git a/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/MessageProducer.kt b/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/MessageProducer.kt new file mode 100644 index 00000000000..a648199100b --- /dev/null +++ b/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/MessageProducer.kt @@ -0,0 +1,27 @@ +package net.corda.messagebus.api.producer + +interface MessageProducer : AutoCloseable { + /** + * Defines the callback for post-send events. If there was an exception it will be provided on this callback. + */ + fun interface Callback { + fun onCompletion(response: CordaMessage<*>?, exception: Exception?) + } + + /** + * Asynchronously sends a generic [CordaMessage], and invoke the provided callback when the message has been acknowledged. + * If the developer would like this to be a blocking call, this can be achieved through the callback. + * + * @param message The message to send. + * @param callback A user-supplied callback to execute when teh record has been successfully sent or an error has occurred. + */ + fun send(message: CordaMessage<*>, callback: Callback?) + + /** + * Send a batch of [CordaMessage] instances to their respective destinations. These should all be of the same type + * (E.g. Kafka, DB, RPC). + * + * @param messages the list of [CordaMessage] to be sent. + */ + fun sendMessages(messages: List>) +} \ No newline at end of file diff --git a/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/factory/MessageProducerFactory.kt b/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/factory/MessageProducerFactory.kt new file mode 100644 index 00000000000..8eee2f6d8b1 --- /dev/null +++ b/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/factory/MessageProducerFactory.kt @@ -0,0 +1,11 @@ +package net.corda.messagebus.api.producer.factory + +import net.corda.libs.configuration.SmartConfig + +interface MessageProducerFactory { + fun createProducer( + producerConfig: SmartConfig, + targetConfig: SmartConfig, + onSerializationError: ((ByteArray) -> Unit)? = null + ) +} \ No newline at end of file From a9f0936af7de83845a669d20c3d284ab14a10e8b Mon Sep 17 00:00:00 2001 From: ben-millar Date: Thu, 7 Sep 2023 15:45:54 +0100 Subject: [PATCH 04/50] CORE-16177 Adding kdocs to CordaMessage --- .../messagebus/api/producer/CordaMessage.kt | 55 ++++++++++++++++--- 1 file changed, 47 insertions(+), 8 deletions(-) diff --git a/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/CordaMessage.kt b/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/CordaMessage.kt index efd9f2f7088..2974c597014 100644 --- a/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/CordaMessage.kt +++ b/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/CordaMessage.kt @@ -1,26 +1,65 @@ package net.corda.messagebus.api.producer +/** + * Object to encapsulate a generic producer message. + * @property payload the payload of the message. + * @property props an additional properties map. + */ @Suppress("UNCHECKED_CAST") data class CordaMessage( val payload: T?, val props: MutableMap = mutableMapOf() ) { + /** + * Adds a new property to the internal storage. + * + * @param property A key value [Pair] to insert into our additional properties map. + */ fun addProperty(property: Pair) { props[property.first] = property.second } - fun getProperty(id: String) : Any { - return getPropertyOrNull(id) ?: throw NoSuchElementException("") + + /** + * Retrieves a property from the additional properties map without casting. + * + * @param key The key of the property to retrieve. + * @return The property associated with the given key. + * @throws NoSuchElementException if no property with the given key exists. + */ + fun getProperty(key: String) : Any { + return getPropertyOrNull(key) ?: throw NoSuchElementException("") } - fun getProperty(id: String) : T { - return getPropertyOrNull(id) ?: throw NoSuchElementException("") + /** + * Retrieves a property of a specific type from the additional properties map. + * + * @param key The key of the property to retrieve. + * @return The property associated with the given key, cast to the specified type. + * @throws NoSuchElementException if no property with the given key exists. + * @throws ClassCastException if the property cannot be cast to the specified type. + */ + fun getProperty(key: String) : T { + return getPropertyOrNull(key) ?: throw NoSuchElementException("") } - fun getPropertyOrNull(id: String) : Any? { - return props[id] + /** + * Retrieves a property from the additional properties map without casting, returning null if not found. + * + * @param key The key of the property to retrieve. + * @return The property associated with the given key, or null if not found. + */ + fun getPropertyOrNull(key: String) : Any? { + return props[key] } - fun getPropertyOrNull(id: String) : T? { - return props[id] as? T ?: throw IllegalArgumentException("") + /** + * Retrieves a property of a specific type from the additional properties map, returning null if not found or if the cast fails. + * + * @param key The identifier of the property to retrieve. + * @return The property associated with the given key cast to the specified type, or null if not found or casting fails. + * @throws ClassCastException if the property cannot be cast to the specified type. + */ + fun getPropertyOrNull(key: String) : T? { + return props[key] as? T ?: throw ClassCastException("") } } \ No newline at end of file From 73dacd238b3594f0e577f4e4ab2e78698188286d Mon Sep 17 00:00:00 2001 From: ben-millar Date: Thu, 7 Sep 2023 18:04:47 +0100 Subject: [PATCH 05/50] CORE-16177 Added helper function for message conversion, fixed naming issue with templated/non-templated getProp funcs --- .../producer/KafkaMessageProducerImpl.kt | 46 +++++++++++-------- .../messagebus/api/producer/CordaMessage.kt | 4 +- .../api/producer/MessageProducer.kt | 2 +- .../factory/MessageProducerFactory.kt | 2 +- 4 files changed, 33 insertions(+), 21 deletions(-) diff --git a/libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/producer/KafkaMessageProducerImpl.kt b/libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/producer/KafkaMessageProducerImpl.kt index 183ba75ccda..e58a36bdadf 100644 --- a/libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/producer/KafkaMessageProducerImpl.kt +++ b/libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/producer/KafkaMessageProducerImpl.kt @@ -6,6 +6,7 @@ import net.corda.messagebus.api.producer.CordaProducerRecord import net.corda.messagebus.api.producer.MessageProducer import net.corda.messagebus.kafka.config.ResolvedProducerConfig import net.corda.messagebus.kafka.utils.toKafkaRecord +import net.corda.messaging.api.chunking.ChunkSerializerService import net.corda.messaging.api.exception.CordaMessageAPIFatalException import net.corda.messaging.api.exception.CordaMessageAPIIntermittentException import net.corda.messaging.api.exception.CordaMessageAPIProducerRequiresReset @@ -32,7 +33,7 @@ import org.slf4j.LoggerFactory class KafkaMessageProducerImpl( private val config: ResolvedProducerConfig, private val producer: Producer, -// private val chunkSerializerService: ChunkSerializerService, + private val chunkSerializerService: ChunkSerializerService, private val producerMetricsBinder: MeterBinder, ) : MessageProducer { private val topicPrefix = config.topicPrefix @@ -44,7 +45,7 @@ class KafkaMessageProducerImpl( private companion object { private val log: Logger = LoggerFactory.getLogger(this::class.java.enclosingClass) -// const val asyncChunkErrorMessage = "Tried to send record which requires chunking using an asynchronous producer" + const val asyncChunkErrorMessage = "Tried to send record which requires chunking using an asynchronous producer" val fatalExceptions: Set> = setOf( AuthorizationException::class.java, @@ -114,33 +115,33 @@ class KafkaMessageProducerImpl( partition: Int? = null, callback: MessageProducer.Callback? = null ) { - // TODO: Support message chunking - sendWholeMessage(message, partition, callback) + val record = message.toCordaProducerRecord() + val chunkedRecords = chunkSerializerService.generateChunkedRecords(record) + + if (chunkedRecords.isNotEmpty()) { + sendChunkedMessage(chunkedRecords, partition, callback) + } else { + sendWholeMessage(record, partition, callback) + } } private fun sendChunkedMessage( - messages: List>, + messages: List>, partition: Int? = null, callback: MessageProducer.Callback? = null ) { - TODO("Not yet implemented") + // TODO: Chunking is not supported for non-transactional calls; this needs to be figured out + val ex = CordaMessageAPIFatalException(asyncChunkErrorMessage) + callback?.onCompletion(null, ex) + throw ex } private fun sendWholeMessage( - message: CordaMessage<*>, + record: CordaProducerRecord<*, *>, partition: Int? = null, callback: MessageProducer.Callback? = null ) { - val headers = message.getProperty("headers") - - val record = CordaProducerRecord( - topic = message.getProperty("topic"), - key = message.getProperty("key"), - value = message.payload, - headers = headers - ) - - val traceContext = traceSend(headers, "send $clientId") + val traceContext = traceSend(record.headers, "send $clientId") traceContext.markInScope().use { try { producer.send( @@ -210,4 +211,13 @@ class KafkaMessageProducerImpl( } } -private typealias Headers = List> \ No newline at end of file +fun CordaMessage<*>.toCordaProducerRecord() : CordaProducerRecord<*, *> { + return CordaProducerRecord( + topic = this.getProperty("topic"), + key = this.getProperty("key"), + value = this.payload, + headers = this.getProperty("headers") + ) +} + +private typealias Headers = List> diff --git a/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/CordaMessage.kt b/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/CordaMessage.kt index 2974c597014..75d61518671 100644 --- a/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/CordaMessage.kt +++ b/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/CordaMessage.kt @@ -38,6 +38,7 @@ data class CordaMessage( * @throws NoSuchElementException if no property with the given key exists. * @throws ClassCastException if the property cannot be cast to the specified type. */ + @JvmName("getPropertyTyped") fun getProperty(key: String) : T { return getPropertyOrNull(key) ?: throw NoSuchElementException("") } @@ -59,7 +60,8 @@ data class CordaMessage( * @return The property associated with the given key cast to the specified type, or null if not found or casting fails. * @throws ClassCastException if the property cannot be cast to the specified type. */ + @JvmName("getPropertyOrNullTyped") fun getPropertyOrNull(key: String) : T? { return props[key] as? T ?: throw ClassCastException("") } -} \ No newline at end of file +} diff --git a/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/MessageProducer.kt b/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/MessageProducer.kt index a648199100b..4cde7a865e1 100644 --- a/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/MessageProducer.kt +++ b/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/MessageProducer.kt @@ -24,4 +24,4 @@ interface MessageProducer : AutoCloseable { * @param messages the list of [CordaMessage] to be sent. */ fun sendMessages(messages: List>) -} \ No newline at end of file +} diff --git a/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/factory/MessageProducerFactory.kt b/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/factory/MessageProducerFactory.kt index 8eee2f6d8b1..049b2fb0ca1 100644 --- a/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/factory/MessageProducerFactory.kt +++ b/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/factory/MessageProducerFactory.kt @@ -8,4 +8,4 @@ interface MessageProducerFactory { targetConfig: SmartConfig, onSerializationError: ((ByteArray) -> Unit)? = null ) -} \ No newline at end of file +} From 3e151935acbfd0360f1da535115e271783c91bf0 Mon Sep 17 00:00:00 2001 From: ben-millar Date: Fri, 8 Sep 2023 14:13:27 +0100 Subject: [PATCH 06/50] CORE-16177 Modified MessageProducer to return a Kotlin Deferred (coroutine) --- .../db/producer/DBMessageProducerImpl.kt | 28 +++++++++++++++++++ .../kafka-message-bus-impl/build.gradle | 1 + .../producer/KafkaMessageProducerImpl.kt | 22 +++++++++------ libs/messaging/message-bus/build.gradle | 1 + .../api/producer/MessageProducer.kt | 17 +++++++---- 5 files changed, 55 insertions(+), 14 deletions(-) create mode 100644 libs/messaging/db-message-bus-impl/src/main/kotlin/net/corda/messagebus/db/producer/DBMessageProducerImpl.kt diff --git a/libs/messaging/db-message-bus-impl/src/main/kotlin/net/corda/messagebus/db/producer/DBMessageProducerImpl.kt b/libs/messaging/db-message-bus-impl/src/main/kotlin/net/corda/messagebus/db/producer/DBMessageProducerImpl.kt new file mode 100644 index 00000000000..2bd794552f8 --- /dev/null +++ b/libs/messaging/db-message-bus-impl/src/main/kotlin/net/corda/messagebus/db/producer/DBMessageProducerImpl.kt @@ -0,0 +1,28 @@ +package net.corda.messagebus.db.producer + +import net.corda.avro.serialization.CordaAvroSerializer +import net.corda.messagebus.api.producer.CordaMessage +import net.corda.messagebus.api.producer.MessageProducer +import net.corda.messagebus.db.persistence.DBAccess +import net.corda.messagebus.db.serialization.MessageHeaderSerializer +import net.corda.messagebus.db.util.WriteOffsets + +class DBMessageProducerImpl( + private val serializer: CordaAvroSerializer, + private val dbAccess: DBAccess, + private val writeOffets: WriteOffsets, + private val headerSerializer: MessageHeaderSerializer, + private val throwOnSerializationError: Boolean = true +) : MessageProducer { + override fun send(message: CordaMessage<*>, callback: MessageProducer.Callback?) { + TODO("Not yet implemented") + } + + override fun sendMessages(messages: List>) { + TODO("Not yet implemented") + } + + override fun close() { + TODO("Not yet implemented") + } +} \ No newline at end of file diff --git a/libs/messaging/kafka-message-bus-impl/build.gradle b/libs/messaging/kafka-message-bus-impl/build.gradle index 9af965f45ee..56c9c8fb411 100644 --- a/libs/messaging/kafka-message-bus-impl/build.gradle +++ b/libs/messaging/kafka-message-bus-impl/build.gradle @@ -24,6 +24,7 @@ dependencies { implementation "net.corda:corda-base" implementation "net.corda:corda-config-schema" implementation 'org.jetbrains.kotlin:kotlin-osgi-bundle' + implementation "org.jetbrains.kotlinx:kotlinx-coroutines-core-jvm:$kotlinCoroutinesVersion" implementation "org.apache.servicemix.bundles:org.apache.servicemix.bundles.kafka-clients:$kafkaClientVersion" constraints { implementation("org.xerial.snappy:snappy-java:$xerialSnappyVersion") { diff --git a/libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/producer/KafkaMessageProducerImpl.kt b/libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/producer/KafkaMessageProducerImpl.kt index e58a36bdadf..9ef6f38e7b7 100644 --- a/libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/producer/KafkaMessageProducerImpl.kt +++ b/libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/producer/KafkaMessageProducerImpl.kt @@ -1,6 +1,7 @@ package net.corda.messagebus.kafka.producer import io.micrometer.core.instrument.binder.MeterBinder +import kotlinx.coroutines.Deferred import net.corda.messagebus.api.producer.CordaMessage import net.corda.messagebus.api.producer.CordaProducerRecord import net.corda.messagebus.api.producer.MessageProducer @@ -79,7 +80,7 @@ class KafkaMessageProducerImpl( ctx.traceTag("send.offset", m.offset().toString()) ctx.traceTag("send.partition", m.partition().toString()) ctx.traceTag("send.topic", m.topic()) - callback.onCompletion(null, ex) + callback.onCompletion(ex) if (ex != null) { ctx.errorAndFinish(ex) } else { @@ -89,25 +90,30 @@ class KafkaMessageProducerImpl( } } - override fun send(message: CordaMessage<*>, callback: MessageProducer.Callback?) { + override fun send(message: CordaMessage<*>, callback: MessageProducer.Callback?): Deferred<*>? { val headers = message.getProperty("headers") val partition = message.getPropertyOrNull("partition") + getOrCreateBatchPublishTracing(clientId).begin(listOf(headers)) tryWithCleanupOnFailure("send single message") { sendMessage(message, partition, callback) } + + return null } - override fun sendMessages(messages: List>) { + override fun sendMessages(messages: List>) : List>? { getOrCreateBatchPublishTracing(clientId).begin(messages.map { it.getProperty("headers") }) tryWithCleanupOnFailure("send multiple message") { messages.forEach { message -> val partition = message.getPropertyOrNull("partition") - sendMessage(message, partition, callback = null) + sendMessage(message, partition) } } + + return null } private fun sendMessage( @@ -130,9 +136,8 @@ class KafkaMessageProducerImpl( partition: Int? = null, callback: MessageProducer.Callback? = null ) { - // TODO: Chunking is not supported for non-transactional calls; this needs to be figured out + // TODO: Producer-level is not supported for non-transactional calls. val ex = CordaMessageAPIFatalException(asyncChunkErrorMessage) - callback?.onCompletion(null, ex) throw ex } @@ -146,7 +151,7 @@ class KafkaMessageProducerImpl( try { producer.send( addTraceContextToRecord(record).toKafkaRecord(topicPrefix, partition), - toTraceKafkaCallback({res, ex-> callback?.onCompletion(res, ex)}, traceContext) + toTraceKafkaCallback({ exception -> callback?.onCompletion(exception) }, traceContext) ) } catch (ex: CordaRuntimeException) { traceContext.errorAndFinish(ex) @@ -183,6 +188,7 @@ class KafkaMessageProducerImpl( in fatalExceptions -> throw CordaMessageAPIFatalException("FatalError occurred $errorString", ex) in transientExceptions -> throw CordaMessageAPIIntermittentException("Error occurred $errorString", ex) in apiExceptions -> throw ex + // TODO: Internally handle recoverable exceptions IllegalStateException::class.java -> { // It's not clear whether the producer is ok to abort and continue or not in this case, so play it safe // and let the client know to create a new one. @@ -211,7 +217,7 @@ class KafkaMessageProducerImpl( } } -fun CordaMessage<*>.toCordaProducerRecord() : CordaProducerRecord<*, *> { +private fun CordaMessage<*>.toCordaProducerRecord() : CordaProducerRecord<*, *> { return CordaProducerRecord( topic = this.getProperty("topic"), key = this.getProperty("key"), diff --git a/libs/messaging/message-bus/build.gradle b/libs/messaging/message-bus/build.gradle index f499ca5fddc..f884ea06a59 100644 --- a/libs/messaging/message-bus/build.gradle +++ b/libs/messaging/message-bus/build.gradle @@ -9,6 +9,7 @@ dependencies { implementation platform("net.corda:corda-api:$cordaApiVersion") implementation 'org.jetbrains.kotlin:kotlin-osgi-bundle' + implementation "org.jetbrains.kotlinx:kotlinx-coroutines-core-jvm:$kotlinCoroutinesVersion" implementation "com.typesafe:config:$typeSafeConfigVersion" implementation "net.corda:corda-base" implementation project(":libs:configuration:configuration-core") diff --git a/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/MessageProducer.kt b/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/MessageProducer.kt index 4cde7a865e1..73d282df49a 100644 --- a/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/MessageProducer.kt +++ b/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/MessageProducer.kt @@ -1,27 +1,32 @@ package net.corda.messagebus.api.producer +import kotlinx.coroutines.Deferred + interface MessageProducer : AutoCloseable { /** * Defines the callback for post-send events. If there was an exception it will be provided on this callback. */ fun interface Callback { - fun onCompletion(response: CordaMessage<*>?, exception: Exception?) + fun onCompletion(exception: Exception?) } /** * Asynchronously sends a generic [CordaMessage], and invoke the provided callback when the message has been acknowledged. * If the developer would like this to be a blocking call, this can be achieved through the callback. * - * @param message The message to send. - * @param callback A user-supplied callback to execute when teh record has been successfully sent or an error has occurred. - */ - fun send(message: CordaMessage<*>, callback: Callback?) + * @param message The [CordaMessage] to send. + * @return [Deferred] instance representing the asynchronous computation result, or null if the destination doesn't + * provide a response. + * */ + fun send(message: CordaMessage<*>, callback: Callback?) : Deferred<*>? /** * Send a batch of [CordaMessage] instances to their respective destinations. These should all be of the same type * (E.g. Kafka, DB, RPC). * * @param messages the list of [CordaMessage] to be sent. + * @return List of [Deferred] instances representing the asynchronous computation results, or null if the + * destination doesn't provide a response. */ - fun sendMessages(messages: List>) + fun sendMessages(messages: List>) : List>? } From a2346045103d65e97620905228a9d243fb45aee7 Mon Sep 17 00:00:00 2001 From: ben-millar Date: Fri, 8 Sep 2023 14:42:30 +0100 Subject: [PATCH 07/50] CORE-16177 Implemented KafkaMessageProducer factory --- .../KafkaMessageProducerFactoryImpl.kt | 97 +++++++++++++++++++ .../factory/MessageProducerFactory.kt | 15 ++- 2 files changed, 111 insertions(+), 1 deletion(-) create mode 100644 libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/producer/factory/KafkaMessageProducerFactoryImpl.kt diff --git a/libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/producer/factory/KafkaMessageProducerFactoryImpl.kt b/libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/producer/factory/KafkaMessageProducerFactoryImpl.kt new file mode 100644 index 00000000000..1974ee7bcbb --- /dev/null +++ b/libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/producer/factory/KafkaMessageProducerFactoryImpl.kt @@ -0,0 +1,97 @@ +package net.corda.messagebus.kafka.producer.factory + +import io.micrometer.core.instrument.binder.kafka.KafkaClientMetrics +import java.util.Properties +import net.corda.libs.configuration.SmartConfig +import net.corda.messagebus.api.configuration.ProducerConfig +import net.corda.messagebus.api.constants.ProducerRoles +import net.corda.messagebus.api.producer.MessageProducer +import net.corda.messagebus.api.producer.factory.MessageProducerFactory +import net.corda.messagebus.kafka.config.MessageBusConfigResolver +import net.corda.messagebus.kafka.producer.KafkaMessageProducerImpl +import net.corda.messagebus.kafka.serialization.CordaAvroSerializerImpl +import net.corda.messagebus.kafka.utils.KafkaRetryUtils.executeKafkaActionWithRetry +import net.corda.messaging.api.chunking.MessagingChunkFactory +import net.corda.schema.configuration.MessagingConfig +import net.corda.schema.registry.AvroSchemaRegistry +import org.apache.kafka.clients.producer.KafkaProducer +import org.osgi.framework.FrameworkUtil +import org.osgi.framework.wiring.BundleWiring +import org.osgi.service.component.annotations.Component +import org.osgi.service.component.annotations.Reference +import org.slf4j.Logger +import org.slf4j.LoggerFactory + +@Component(service = [MessageProducerFactory::class]) +class KafkaMessageProducerFactoryImpl( + @Reference(service = AvroSchemaRegistry::class) + private val avroSchemaRegistry: AvroSchemaRegistry, + @Reference(service = MessagingChunkFactory::class) + private val messagingChunkFactory: MessagingChunkFactory, +) : MessageProducerFactory { + + companion object { + private val log: Logger = LoggerFactory.getLogger(this::class.java.enclosingClass) + } + + override fun createProducer( + producerConfig: SmartConfig, + targetConfig: SmartConfig, + onSerializationError: ((ByteArray) -> Unit)? + ): MessageProducer { + val cordaProducerConfig = producerConfig.toProducerConfig() + val configResolver = MessageBusConfigResolver(targetConfig.factory) + val (resolvedConfig, kafkaProperties) = configResolver.resolve(targetConfig, cordaProducerConfig) + + return executeKafkaActionWithRetry( + action = { + val producer = createKafkaProducer(kafkaProperties, onSerializationError) + val maxAllowedMessageSize = targetConfig.getLong(MessagingConfig.MAX_ALLOWED_MSG_SIZE) + val producerChunkService = messagingChunkFactory.createChunkSerializerService(maxAllowedMessageSize) + KafkaMessageProducerImpl( + resolvedConfig, + producer, + producerChunkService, + KafkaClientMetrics(producer) + ) + }, + errorMessage = { + "KafkaMessageProducerFactoryImpl failed to producer with clientId ${cordaProducerConfig.clientId}, " + + "with configuration: $targetConfig" + }, + log = log + ) + } + + private fun createKafkaProducer( + kafkaProperties: Properties, + onSerializationError: ((ByteArray) -> Unit)? + ) : KafkaProducer { + val contextClassLoader = Thread.currentThread().contextClassLoader + val currentBundle = FrameworkUtil.getBundle(KafkaProducer::class.java) + + return try { + if (currentBundle != null) { + Thread.currentThread().contextClassLoader = currentBundle.adapt(BundleWiring::class.java).classLoader + } + KafkaProducer( + kafkaProperties, + CordaAvroSerializerImpl(avroSchemaRegistry, onSerializationError), + CordaAvroSerializerImpl(avroSchemaRegistry, onSerializationError) + ) + } finally { + Thread.currentThread().contextClassLoader = contextClassLoader + } + } + + private fun SmartConfig.toProducerConfig(): ProducerConfig { + return ProducerConfig( + clientId = getString("clientId"), + instanceId = getInt("instanceId"), + transactional = getBoolean("transactional"), + role = ProducerRoles.valueOf(getString("role")), + throwOnSerializationError = + if (hasPath("throwOnSerializationError")) getBoolean("throwOnSerializationError") else true + ) + } +} \ No newline at end of file diff --git a/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/factory/MessageProducerFactory.kt b/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/factory/MessageProducerFactory.kt index 049b2fb0ca1..2c08cecd1fd 100644 --- a/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/factory/MessageProducerFactory.kt +++ b/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/factory/MessageProducerFactory.kt @@ -1,11 +1,24 @@ package net.corda.messagebus.api.producer.factory import net.corda.libs.configuration.SmartConfig +import net.corda.messagebus.api.producer.MessageProducer +/** + * Producer factory interface for creating message producers + */ interface MessageProducerFactory { + + /** + * Generate a message producer with given properties. + * @param producerConfig The mandatory config for setting up producers. + * @param targetConfig Configuration for connecting to the producer target (message bus, rpc endpoint, etc.) + * @param onSerializationError a callback to execute when serialization fails. + * @return [MessageProducer] capable of publishing records of any type to some external system. + * @throws CordaMessageAPIFatalException if producer cannot be created. + */ fun createProducer( producerConfig: SmartConfig, targetConfig: SmartConfig, onSerializationError: ((ByteArray) -> Unit)? = null - ) + ): MessageProducer } From ccfbb72a6c9a08d5930087e5591386c8a3db9f90 Mon Sep 17 00:00:00 2001 From: ben-millar Date: Mon, 11 Sep 2023 14:54:11 +0100 Subject: [PATCH 08/50] CORE-16177 Adding CordaMessage unit tests --- .../producer/KafkaMessageProducerImpl.kt | 3 +- .../KafkaMessageProducerFactoryImpl.kt | 2 +- .../messagebus/api/producer/CordaMessage.kt | 14 +- .../api/producer/CordaMessageTest.kt | 126 ++++++++++++++++++ 4 files changed, 138 insertions(+), 7 deletions(-) create mode 100644 libs/messaging/message-bus/src/test/kotlin/net/corda/messagebus/api/producer/CordaMessageTest.kt diff --git a/libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/producer/KafkaMessageProducerImpl.kt b/libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/producer/KafkaMessageProducerImpl.kt index 9ef6f38e7b7..9885046bc2d 100644 --- a/libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/producer/KafkaMessageProducerImpl.kt +++ b/libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/producer/KafkaMessageProducerImpl.kt @@ -131,12 +131,13 @@ class KafkaMessageProducerImpl( } } + // TODO: Producer-level is not supported for non-transactional calls. + @Suppress("UNUSED_PARAMETER") private fun sendChunkedMessage( messages: List>, partition: Int? = null, callback: MessageProducer.Callback? = null ) { - // TODO: Producer-level is not supported for non-transactional calls. val ex = CordaMessageAPIFatalException(asyncChunkErrorMessage) throw ex } diff --git a/libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/producer/factory/KafkaMessageProducerFactoryImpl.kt b/libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/producer/factory/KafkaMessageProducerFactoryImpl.kt index 1974ee7bcbb..7f53876e9cf 100644 --- a/libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/producer/factory/KafkaMessageProducerFactoryImpl.kt +++ b/libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/producer/factory/KafkaMessageProducerFactoryImpl.kt @@ -94,4 +94,4 @@ class KafkaMessageProducerFactoryImpl( if (hasPath("throwOnSerializationError")) getBoolean("throwOnSerializationError") else true ) } -} \ No newline at end of file +} diff --git a/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/CordaMessage.kt b/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/CordaMessage.kt index 75d61518671..8b249436313 100644 --- a/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/CordaMessage.kt +++ b/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/CordaMessage.kt @@ -27,7 +27,7 @@ data class CordaMessage( * @throws NoSuchElementException if no property with the given key exists. */ fun getProperty(key: String) : Any { - return getPropertyOrNull(key) ?: throw NoSuchElementException("") + return getPropertyOrNull(key) ?: throw NoSuchElementException("No property found with the key: '$key'") } /** @@ -39,8 +39,9 @@ data class CordaMessage( * @throws ClassCastException if the property cannot be cast to the specified type. */ @JvmName("getPropertyTyped") - fun getProperty(key: String) : T { - return getPropertyOrNull(key) ?: throw NoSuchElementException("") + inline fun getProperty(key: String) : T { + return (getProperty(key) as? T) + ?: throw ClassCastException("Property '$key' could not be cast to type: '${T::class.java}'.") } /** @@ -61,7 +62,10 @@ data class CordaMessage( * @throws ClassCastException if the property cannot be cast to the specified type. */ @JvmName("getPropertyOrNullTyped") - fun getPropertyOrNull(key: String) : T? { - return props[key] as? T ?: throw ClassCastException("") + inline fun getPropertyOrNull(key: String) : T? { + return (props[key] as? T) + ?: throw ClassCastException( + "Property '$key' could not be cast to type: '${T::class.java}'. Ensure the property is not null." + ) } } diff --git a/libs/messaging/message-bus/src/test/kotlin/net/corda/messagebus/api/producer/CordaMessageTest.kt b/libs/messaging/message-bus/src/test/kotlin/net/corda/messagebus/api/producer/CordaMessageTest.kt new file mode 100644 index 00000000000..d22638dc647 --- /dev/null +++ b/libs/messaging/message-bus/src/test/kotlin/net/corda/messagebus/api/producer/CordaMessageTest.kt @@ -0,0 +1,126 @@ +package net.corda.messagebus.api.producer + +import org.junit.jupiter.api.Assertions.assertEquals +import org.junit.jupiter.api.Assertions.assertNull +import org.junit.jupiter.api.Assertions.assertThrows +import org.junit.jupiter.api.Test + +class CordaMessageTest { + private val payload: String = "payload" + private val topicKey: String = "topic" + private val topicValue: String = "topic" + private val partitionKey: String = "partition" + private val partitionValue: Long = 1L + + private val defaultMessage: CordaMessage = CordaMessage(payload, mutableMapOf( + topicKey to topicValue, + partitionKey to partitionValue + )) + + @Test + fun `Test add property (string)`() { + val message: CordaMessage = CordaMessage(payload) + message.addProperty(Pair(topicKey, topicValue)) + + assertEquals(message.props, mutableMapOf(topicKey to topicValue)) + } + + @Test + fun `Test add property (long)`() { + val message: CordaMessage = CordaMessage(payload) + message.addProperty(Pair(partitionKey, partitionValue)) + + assertEquals(message.props, mutableMapOf(partitionKey to partitionValue)) + } + + @Test + fun `Test create message with props`() { + val message: CordaMessage = CordaMessage(payload, mutableMapOf( + topicKey to topicValue, + partitionKey to partitionValue + )) + + assertEquals(message.props, mutableMapOf(topicKey to topicValue, partitionKey to partitionValue)) + } + + @Test + fun `Test get property, non-typed`() { + val property = defaultMessage.getProperty(topicKey) + assertEquals(property, topicValue) + } + + @Test + fun `Test get property that doesn't exist, non-typed`() { + val ex = assertThrows(NoSuchElementException::class.java) { + defaultMessage.getProperty("hello world") + } + + assertEquals(ex.message, "No property found with the key: 'hello world'") + } + + @Test + fun `Test get property, typed`() { + val property = defaultMessage.getProperty(topicKey) + assertEquals(property, topicValue) + assertEquals(property::class, String::class) + } + + @Test + fun `Test get property that doesn't exist, typed`() { + val ex = assertThrows(NoSuchElementException::class.java) { + defaultMessage.getProperty("hello world") + } + + assertEquals(ex.message, "No property found with the key: 'hello world'") + } + + @Test + fun `Test get existing typed property, passing in the wrong type`() { + val ex = assertThrows(ClassCastException::class.java) { + defaultMessage.getProperty(topicKey) + } + + assertEquals(ex.message, "Property 'topic' could not be cast to type: 'class java.lang.Long'.") + } + + @Test + fun `Test get existing nullable property, non-typed`() { + val property = defaultMessage.getPropertyOrNull(topicKey) + assertEquals(property, topicValue) + } + + @Test + fun `Test get existing nullable property, typed`() { + val property = defaultMessage.getPropertyOrNull(topicKey) + assertEquals(property, topicValue) + assertEquals(property!!::class, String::class) + } + + @Test + fun `Test get nullable property that doesn't exist, non-typed`() { + val property = defaultMessage.getPropertyOrNull("hello world") + assertNull(property) + } + + @Test + fun `Test get nullable property that doesn't exist, typed`() { + val ex = assertThrows(ClassCastException::class.java) { + defaultMessage.getPropertyOrNull("hello world") + } + + assertEquals(ex.message, "Property 'hello world' could not be cast to type: 'class java.lang.String'. " + + "Ensure the property is not null." + ) + } + + @Test + fun `Test get existing nullable typed property, passing in the wrong type`() { + val ex = assertThrows(ClassCastException::class.java) { + defaultMessage.getPropertyOrNull(topicKey) + } + + assertEquals(ex.message, "Property 'topic' could not be cast to type: 'class java.lang.Long'. " + + "Ensure the property is not null." + ) + } +} From d4dba855d398e636d0dd0a4b90a6c25dc8b2fed4 Mon Sep 17 00:00:00 2001 From: ben-millar Date: Mon, 11 Sep 2023 16:50:40 +0100 Subject: [PATCH 09/50] CORE-16177 Adding KafkaMessageProducerImpl unit tests --- .../producer/KafkaMessageProducerImpl.kt | 1 + .../producer/KafkaMessageProducerImplTest.kt | 150 ++++++++++++++++++ .../messagebus/api/producer/CordaMessage.kt | 5 +- .../api/producer/CordaMessageTest.kt | 13 +- 4 files changed, 157 insertions(+), 12 deletions(-) create mode 100644 libs/messaging/kafka-message-bus-impl/src/test/kotlin/net/corda/messagebus/kafka/producer/KafkaMessageProducerImplTest.kt diff --git a/libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/producer/KafkaMessageProducerImpl.kt b/libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/producer/KafkaMessageProducerImpl.kt index 9885046bc2d..48ae9343c18 100644 --- a/libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/producer/KafkaMessageProducerImpl.kt +++ b/libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/producer/KafkaMessageProducerImpl.kt @@ -139,6 +139,7 @@ class KafkaMessageProducerImpl( callback: MessageProducer.Callback? = null ) { val ex = CordaMessageAPIFatalException(asyncChunkErrorMessage) + callback?.onCompletion(ex) throw ex } diff --git a/libs/messaging/kafka-message-bus-impl/src/test/kotlin/net/corda/messagebus/kafka/producer/KafkaMessageProducerImplTest.kt b/libs/messaging/kafka-message-bus-impl/src/test/kotlin/net/corda/messagebus/kafka/producer/KafkaMessageProducerImplTest.kt new file mode 100644 index 00000000000..7a8032f5d27 --- /dev/null +++ b/libs/messaging/kafka-message-bus-impl/src/test/kotlin/net/corda/messagebus/kafka/producer/KafkaMessageProducerImplTest.kt @@ -0,0 +1,150 @@ +package net.corda.messaging.kafka.subscription.net.corda.messagebus.kafka.producer + +import io.micrometer.core.instrument.binder.kafka.KafkaClientMetrics +import net.corda.messagebus.api.producer.CordaMessage +import net.corda.messagebus.api.producer.CordaProducerRecord +import net.corda.messagebus.api.producer.MessageProducer +import net.corda.messagebus.kafka.config.ResolvedProducerConfig +import net.corda.messagebus.kafka.producer.KafkaMessageProducerImpl +import net.corda.messaging.api.chunking.ChunkSerializerService +import net.corda.messaging.api.exception.CordaMessageAPIFatalException +import net.corda.messaging.api.exception.CordaMessageAPIIntermittentException +import org.apache.kafka.clients.producer.MockProducer +import org.apache.kafka.clients.producer.ProducerRecord +import org.apache.kafka.common.KafkaException +import org.junit.jupiter.api.BeforeEach +import org.junit.jupiter.api.Test +import org.junit.jupiter.api.assertThrows +import org.mockito.Mockito.doThrow +import org.mockito.Mockito.times +import org.mockito.kotlin.any +import org.mockito.kotlin.anyOrNull +import org.mockito.kotlin.atLeastOnce +import org.mockito.kotlin.eq +import org.mockito.kotlin.mock +import org.mockito.kotlin.verify +import org.mockito.kotlin.whenever + +class KafkaMessageProducerImplTest { + private lateinit var kafkaProducer: MockProducer + private lateinit var messageProducer: KafkaMessageProducerImpl + + private val config = ResolvedProducerConfig("clientId", true, "prefix", false) + private val chunkSerializerService: ChunkSerializerService = mock() + private val metricsBinder: KafkaClientMetrics = mock() + private val defaultHeaders: List> = emptyList() + private val messageProps: MutableMap = mutableMapOf( + "topic" to "topic", + "key" to "key", + "headers" to defaultHeaders + ) + private val message: CordaMessage = CordaMessage("value", messageProps) + + + @BeforeEach + fun setup() { + kafkaProducer = mock() + messageProducer = KafkaMessageProducerImpl(config, kafkaProducer, chunkSerializerService, metricsBinder) + } + + @Test + fun testSend() { + val callback = mock() + messageProducer.send(message, callback) + + val expected = ProducerRecord( + config.topicPrefix + message.getProperty("topic"), + message.getProperty("key"), + message.payload + ) + + verify(kafkaProducer).send(eq(expected), any()) + } + + @Test + fun testSendWithPartition() { + val callback = mock() + val messageWithPartition = message.copy().apply { + this.addProperty(Pair("partition", 1)) + } + messageProducer.send(messageWithPartition, callback) + + val expected = ProducerRecord( + config.topicPrefix + messageWithPartition.getProperty("topic"), + 1, + messageWithPartition.getProperty("key"), + messageWithPartition.payload + ) + + verify(kafkaProducer).send(eq(expected), any()) + } + + @Test + fun testSendWithError() { + val callback = mock() + val record = ProducerRecord( + config.topicPrefix + message.getProperty("topic"), + message.getProperty("key"), + message.payload + ) + + doThrow(KafkaException("")).whenever(kafkaProducer).send(eq(record), any()) + assertThrows { messageProducer.send(message, callback) } + } + + @Test + fun testSendMessages() { + val messages = listOf(message, message, message) + val expectedPublishedMessages = + messages.map { + ProducerRecord( + config.topicPrefix + it.getProperty("topic"), + it.getProperty("key"), + it.payload + ) + } + + messageProducer.sendMessages(messages) + verify(kafkaProducer, times(3)).send(any(), anyOrNull()) + expectedPublishedMessages.forEach { verify(kafkaProducer, atLeastOnce()).send(eq(it), any()) } + } + + @Test + fun testSendMessagesWithError() { + val record = ProducerRecord( + config.topicPrefix + message.getProperty("topic"), + message.getProperty("key"), + message.payload + ) + + doThrow(KafkaException("")).whenever(kafkaProducer).send(eq(record), any()) + assertThrows { + messageProducer.sendMessages(listOf(message, message, message)) + } + } + + // TODO: Producer-level chunking is not yet implemented for non-transactional kafka producers + @Test + fun `Trying to send chunked message throws a fatal exception and executes callback`() { + val callback = mock() + val record = CordaProducerRecord( + config.topicPrefix + message.getProperty("topic"), + message.getProperty("key"), + message.payload + ) + + whenever(chunkSerializerService.generateChunkedRecords(any())).thenReturn(listOf(record, record)) + assertThrows { + messageProducer.send(message, callback) + } + + verify(callback, org.mockito.kotlin.times(1)).onCompletion(any()) + } + + @Test + fun testClose() { + messageProducer.close() + verify(kafkaProducer, times(1)).close() + verify(metricsBinder, times(1)).close() + } +} \ No newline at end of file diff --git a/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/CordaMessage.kt b/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/CordaMessage.kt index 8b249436313..a478713ed69 100644 --- a/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/CordaMessage.kt +++ b/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/CordaMessage.kt @@ -63,9 +63,10 @@ data class CordaMessage( */ @JvmName("getPropertyOrNullTyped") inline fun getPropertyOrNull(key: String) : T? { - return (props[key] as? T) + val value = props[key] ?: return null + return (value as? T) ?: throw ClassCastException( - "Property '$key' could not be cast to type: '${T::class.java}'. Ensure the property is not null." + "Property '$key' could not be cast to type: '${T::class.java}'." ) } } diff --git a/libs/messaging/message-bus/src/test/kotlin/net/corda/messagebus/api/producer/CordaMessageTest.kt b/libs/messaging/message-bus/src/test/kotlin/net/corda/messagebus/api/producer/CordaMessageTest.kt index d22638dc647..b53790abef2 100644 --- a/libs/messaging/message-bus/src/test/kotlin/net/corda/messagebus/api/producer/CordaMessageTest.kt +++ b/libs/messaging/message-bus/src/test/kotlin/net/corda/messagebus/api/producer/CordaMessageTest.kt @@ -104,13 +104,8 @@ class CordaMessageTest { @Test fun `Test get nullable property that doesn't exist, typed`() { - val ex = assertThrows(ClassCastException::class.java) { - defaultMessage.getPropertyOrNull("hello world") - } - - assertEquals(ex.message, "Property 'hello world' could not be cast to type: 'class java.lang.String'. " + - "Ensure the property is not null." - ) + val property = defaultMessage.getPropertyOrNull("hello world") + assertNull(property) } @Test @@ -119,8 +114,6 @@ class CordaMessageTest { defaultMessage.getPropertyOrNull(topicKey) } - assertEquals(ex.message, "Property 'topic' could not be cast to type: 'class java.lang.Long'. " + - "Ensure the property is not null." - ) + assertEquals(ex.message, "Property 'topic' could not be cast to type: 'class java.lang.Long'.") } } From 0b2b8697e45c4d597f36646e89fc47a077e48ffe Mon Sep 17 00:00:00 2001 From: ben-millar Date: Mon, 11 Sep 2023 17:03:24 +0100 Subject: [PATCH 10/50] CORE-16177 Removing DBMessageProducerImpl --- .../db/producer/DBMessageProducerImpl.kt | 28 ------------------- 1 file changed, 28 deletions(-) delete mode 100644 libs/messaging/db-message-bus-impl/src/main/kotlin/net/corda/messagebus/db/producer/DBMessageProducerImpl.kt diff --git a/libs/messaging/db-message-bus-impl/src/main/kotlin/net/corda/messagebus/db/producer/DBMessageProducerImpl.kt b/libs/messaging/db-message-bus-impl/src/main/kotlin/net/corda/messagebus/db/producer/DBMessageProducerImpl.kt deleted file mode 100644 index 2bd794552f8..00000000000 --- a/libs/messaging/db-message-bus-impl/src/main/kotlin/net/corda/messagebus/db/producer/DBMessageProducerImpl.kt +++ /dev/null @@ -1,28 +0,0 @@ -package net.corda.messagebus.db.producer - -import net.corda.avro.serialization.CordaAvroSerializer -import net.corda.messagebus.api.producer.CordaMessage -import net.corda.messagebus.api.producer.MessageProducer -import net.corda.messagebus.db.persistence.DBAccess -import net.corda.messagebus.db.serialization.MessageHeaderSerializer -import net.corda.messagebus.db.util.WriteOffsets - -class DBMessageProducerImpl( - private val serializer: CordaAvroSerializer, - private val dbAccess: DBAccess, - private val writeOffets: WriteOffsets, - private val headerSerializer: MessageHeaderSerializer, - private val throwOnSerializationError: Boolean = true -) : MessageProducer { - override fun send(message: CordaMessage<*>, callback: MessageProducer.Callback?) { - TODO("Not yet implemented") - } - - override fun sendMessages(messages: List>) { - TODO("Not yet implemented") - } - - override fun close() { - TODO("Not yet implemented") - } -} \ No newline at end of file From 3dea3df93022683341daf692ce31e8628b438f29 Mon Sep 17 00:00:00 2001 From: ben-millar Date: Tue, 12 Sep 2023 12:49:05 +0100 Subject: [PATCH 11/50] CORE-16177 Include the kotlinx.coroutine JAR within message0-bus JAR --- libs/messaging/message-bus/build.gradle | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/libs/messaging/message-bus/build.gradle b/libs/messaging/message-bus/build.gradle index f884ea06a59..d313b644130 100644 --- a/libs/messaging/message-bus/build.gradle +++ b/libs/messaging/message-bus/build.gradle @@ -15,4 +15,15 @@ dependencies { implementation project(":libs:configuration:configuration-core") } +// include coroutines bundle in this jar as it doesn't have OSGi metadata +tasks.named('jar', Jar) { + bundle { + bnd '''\ +-conditionalpackage: kotlinx.coroutines* +Export-Package: kotlinx.coroutines* +Import-Package: !android*, * +''' + } +} + description 'Message Bus API' From 0817b6a5bd8173ffdc2b2ee5960d16acbe91db60 Mon Sep 17 00:00:00 2001 From: ben-millar Date: Tue, 12 Sep 2023 14:26:46 +0100 Subject: [PATCH 12/50] CORE-16177 Addressing PR comments --- .../kafka/producer/CordaKafkaProducerImpl.kt | 21 +- .../producer/KafkaMessageProducerImpl.kt | 231 ------------------ .../kafka/producer/MessageProducerImpl.kt | 60 +++++ .../KafkaMessageProducerFactoryImpl.kt | 97 -------- .../producer/KafkaMessageProducerImplTest.kt | 150 ------------ .../kafka/producer/MessageProducerImplTest.kt | 76 ++++++ .../messagebus/api/producer/CordaMessage.kt | 4 +- .../api/producer/MessageProducer.kt | 22 +- .../api/producer/CordaMessageTest.kt | 4 +- 9 files changed, 150 insertions(+), 515 deletions(-) delete mode 100644 libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/producer/KafkaMessageProducerImpl.kt create mode 100644 libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/producer/MessageProducerImpl.kt delete mode 100644 libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/producer/factory/KafkaMessageProducerFactoryImpl.kt delete mode 100644 libs/messaging/kafka-message-bus-impl/src/test/kotlin/net/corda/messagebus/kafka/producer/KafkaMessageProducerImplTest.kt create mode 100644 libs/messaging/kafka-message-bus-impl/src/test/kotlin/net/corda/messagebus/kafka/producer/MessageProducerImplTest.kt diff --git a/libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/producer/CordaKafkaProducerImpl.kt b/libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/producer/CordaKafkaProducerImpl.kt index b3c9abc1e8f..b561abc1e9e 100644 --- a/libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/producer/CordaKafkaProducerImpl.kt +++ b/libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/producer/CordaKafkaProducerImpl.kt @@ -82,7 +82,7 @@ class CordaKafkaProducerImpl( // There is no other producer, we are not a zombie, and so don't need to be fenced, we can simply abort and retry. KafkaException::class.java ) - val ApiExceptions: Set> = setOf( + val apiExceptions: Set> = setOf( CordaMessageAPIFatalException::class.java, CordaMessageAPIIntermittentException::class.java ) @@ -381,24 +381,19 @@ class CordaKafkaProducerImpl( private fun handleException(ex: Exception, operation: String, abortTransaction: Boolean) { val errorString = "$operation for CordaKafkaProducer with clientId ${config.clientId}" when (ex::class.java) { - in fatalExceptions -> { - throw CordaMessageAPIFatalException("FatalError occurred $errorString", ex) + in fatalExceptions -> throw CordaMessageAPIFatalException("FatalError occurred $errorString", ex) + in transientExceptions -> { + if (abortTransaction) { + abortTransaction() + } + throw CordaMessageAPIIntermittentException("Error occurred $errorString", ex) } - + in apiExceptions -> throw ex IllegalStateException::class.java -> { // It's not clear whether the producer is ok to abort and continue or not in this case, so play it safe // and let the client know to create a new one. throw CordaMessageAPIProducerRequiresReset("Error occurred $errorString", ex) } - - in transientExceptions -> { - if (abortTransaction) { - abortTransaction() - } - throw CordaMessageAPIIntermittentException("Error occurred $errorString", ex) - } - in ApiExceptions -> { throw ex } - else -> { // Here we do not know what the exact cause of the exception is, but we do know Kafka has not told us we // must close down, nor has it told us we can abort and retry. In this instance the most sensible thing diff --git a/libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/producer/KafkaMessageProducerImpl.kt b/libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/producer/KafkaMessageProducerImpl.kt deleted file mode 100644 index 48ae9343c18..00000000000 --- a/libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/producer/KafkaMessageProducerImpl.kt +++ /dev/null @@ -1,231 +0,0 @@ -package net.corda.messagebus.kafka.producer - -import io.micrometer.core.instrument.binder.MeterBinder -import kotlinx.coroutines.Deferred -import net.corda.messagebus.api.producer.CordaMessage -import net.corda.messagebus.api.producer.CordaProducerRecord -import net.corda.messagebus.api.producer.MessageProducer -import net.corda.messagebus.kafka.config.ResolvedProducerConfig -import net.corda.messagebus.kafka.utils.toKafkaRecord -import net.corda.messaging.api.chunking.ChunkSerializerService -import net.corda.messaging.api.exception.CordaMessageAPIFatalException -import net.corda.messaging.api.exception.CordaMessageAPIIntermittentException -import net.corda.messaging.api.exception.CordaMessageAPIProducerRequiresReset -import net.corda.metrics.CordaMetrics -import net.corda.tracing.TraceContext -import net.corda.tracing.addTraceContextToRecord -import net.corda.tracing.getOrCreateBatchPublishTracing -import net.corda.tracing.traceSend -import net.corda.v5.base.exceptions.CordaRuntimeException -import org.apache.kafka.clients.producer.Callback -import org.apache.kafka.clients.producer.Producer -import org.apache.kafka.common.KafkaException -import org.apache.kafka.common.errors.AuthorizationException -import org.apache.kafka.common.errors.FencedInstanceIdException -import org.apache.kafka.common.errors.InterruptException -import org.apache.kafka.common.errors.InvalidProducerEpochException -import org.apache.kafka.common.errors.ProducerFencedException -import org.apache.kafka.common.errors.TimeoutException -import org.apache.kafka.common.errors.UnsupportedForMessageFormatException -import org.apache.kafka.common.errors.UnsupportedVersionException -import org.slf4j.Logger -import org.slf4j.LoggerFactory - -class KafkaMessageProducerImpl( - private val config: ResolvedProducerConfig, - private val producer: Producer, - private val chunkSerializerService: ChunkSerializerService, - private val producerMetricsBinder: MeterBinder, -) : MessageProducer { - private val topicPrefix = config.topicPrefix - private val clientId = config.clientId - - init { - producerMetricsBinder.bindTo(CordaMetrics.registry) - } - - private companion object { - private val log: Logger = LoggerFactory.getLogger(this::class.java.enclosingClass) - const val asyncChunkErrorMessage = "Tried to send record which requires chunking using an asynchronous producer" - - val fatalExceptions: Set> = setOf( - AuthorizationException::class.java, - FencedInstanceIdException::class.java, - ProducerFencedException::class.java, - UnsupportedForMessageFormatException::class.java, - UnsupportedVersionException::class.java - ) - - val transientExceptions: Set> = setOf( - TimeoutException::class.java, - InterruptException::class.java, - // Failure to commit here might be due to consumer kicked from group. - // Return as intermittent to trigger retry - InvalidProducerEpochException::class.java, - // See https://cwiki.apache.org/confluence/display/KAFKA/KIP-588%3A+Allow+producers+to+recover+gracefully+from+transaction+timeouts - // This exception means the coordinator has bumped the producer epoch because of a timeout of this producer. - // There is no other producer, we are not a zombie, and so don't need to be fenced, we can simply abort and retry. - KafkaException::class.java - ) - - val apiExceptions: Set> = setOf( - CordaMessageAPIFatalException::class.java, - CordaMessageAPIIntermittentException::class.java - ) - } - - private fun toTraceKafkaCallback(callback: MessageProducer.Callback, ctx: TraceContext) : Callback { - return Callback { m, ex -> - ctx.markInScope().use { - ctx.traceTag("send.offset", m.offset().toString()) - ctx.traceTag("send.partition", m.partition().toString()) - ctx.traceTag("send.topic", m.topic()) - callback.onCompletion(ex) - if (ex != null) { - ctx.errorAndFinish(ex) - } else { - ctx.finish() - } - } - } - } - - override fun send(message: CordaMessage<*>, callback: MessageProducer.Callback?): Deferred<*>? { - val headers = message.getProperty("headers") - val partition = message.getPropertyOrNull("partition") - - getOrCreateBatchPublishTracing(clientId).begin(listOf(headers)) - tryWithCleanupOnFailure("send single message") { - sendMessage(message, partition, callback) - } - - return null - } - - override fun sendMessages(messages: List>) : List>? { - getOrCreateBatchPublishTracing(clientId).begin(messages.map { - it.getProperty("headers") - }) - tryWithCleanupOnFailure("send multiple message") { - messages.forEach { message -> - val partition = message.getPropertyOrNull("partition") - sendMessage(message, partition) - } - } - - return null - } - - private fun sendMessage( - message: CordaMessage<*>, - partition: Int? = null, - callback: MessageProducer.Callback? = null - ) { - val record = message.toCordaProducerRecord() - val chunkedRecords = chunkSerializerService.generateChunkedRecords(record) - - if (chunkedRecords.isNotEmpty()) { - sendChunkedMessage(chunkedRecords, partition, callback) - } else { - sendWholeMessage(record, partition, callback) - } - } - - // TODO: Producer-level is not supported for non-transactional calls. - @Suppress("UNUSED_PARAMETER") - private fun sendChunkedMessage( - messages: List>, - partition: Int? = null, - callback: MessageProducer.Callback? = null - ) { - val ex = CordaMessageAPIFatalException(asyncChunkErrorMessage) - callback?.onCompletion(ex) - throw ex - } - - private fun sendWholeMessage( - record: CordaProducerRecord<*, *>, - partition: Int? = null, - callback: MessageProducer.Callback? = null - ) { - val traceContext = traceSend(record.headers, "send $clientId") - traceContext.markInScope().use { - try { - producer.send( - addTraceContextToRecord(record).toKafkaRecord(topicPrefix, partition), - toTraceKafkaCallback({ exception -> callback?.onCompletion(exception) }, traceContext) - ) - } catch (ex: CordaRuntimeException) { - traceContext.errorAndFinish(ex) - val msg = "Failed to send record to topic ${record.topic} with key ${record.key}\"" - if (config.throwOnSerializationError) { - log.error(msg, ex) - throw ex - } else { - log.warn(msg, ex) - } - } catch (ex: Exception) { - traceContext.errorAndFinish(ex) - throw ex - } - } - } - - private fun tryWithCleanupOnFailure( - operation: String, - block: () -> Unit - ) { - try { - block() - getOrCreateBatchPublishTracing(config.clientId).complete() - } catch (ex: Exception) { - getOrCreateBatchPublishTracing(config.clientId).abort() - handleException(ex, operation) - } - } - - private fun handleException(ex: Exception, operation: String) { - val errorString = "$operation for CordaKafkaProducer with clientId ${config.clientId}" - when (ex::class.java) { - in fatalExceptions -> throw CordaMessageAPIFatalException("FatalError occurred $errorString", ex) - in transientExceptions -> throw CordaMessageAPIIntermittentException("Error occurred $errorString", ex) - in apiExceptions -> throw ex - // TODO: Internally handle recoverable exceptions - IllegalStateException::class.java -> { - // It's not clear whether the producer is ok to abort and continue or not in this case, so play it safe - // and let the client know to create a new one. - throw CordaMessageAPIProducerRequiresReset("Error occurred $errorString", ex) - } - else -> { - // Here we do not know what the exact cause of the exception is, but we do know Kafka has not told us we - // must close down, nor has it told us we can abort and retry. In this instance the most sensible thing - // for the client to do would be to close this producer and create a new one. - throw CordaMessageAPIProducerRequiresReset("Unexpected error occurred $errorString", ex) - } - } - } - - override fun close() { - try { - producer.close() - } catch (ex: Exception) { - log.info( - "CordaKafkaProducer failed to close producer safely. This can be observed when there are " + - "no reachable brokers. ClientId: ${config.clientId}", ex - ) - } finally { - (producerMetricsBinder as? AutoCloseable)?.close() - } - } -} - -private fun CordaMessage<*>.toCordaProducerRecord() : CordaProducerRecord<*, *> { - return CordaProducerRecord( - topic = this.getProperty("topic"), - key = this.getProperty("key"), - value = this.payload, - headers = this.getProperty("headers") - ) -} - -private typealias Headers = List> diff --git a/libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/producer/MessageProducerImpl.kt b/libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/producer/MessageProducerImpl.kt new file mode 100644 index 00000000000..d613db3f567 --- /dev/null +++ b/libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/producer/MessageProducerImpl.kt @@ -0,0 +1,60 @@ +package net.corda.messagebus.kafka.producer + +import io.micrometer.core.instrument.binder.MeterBinder +import kotlinx.coroutines.CompletableDeferred +import kotlinx.coroutines.Deferred +import net.corda.messagebus.api.producer.CordaMessage +import net.corda.messagebus.api.producer.CordaProducer +import net.corda.messagebus.api.producer.CordaProducerRecord +import net.corda.messagebus.api.producer.MessageProducer +import net.corda.metrics.CordaMetrics +import org.slf4j.Logger +import org.slf4j.LoggerFactory + +class MessageProducerImpl( + private val clientId: String, + private val producer: CordaProducer, + private val producerMetricsBinder: MeterBinder, +) : MessageProducer { + init { + producerMetricsBinder.bindTo(CordaMetrics.registry) + } + + private companion object { + private val log: Logger = LoggerFactory.getLogger(this::class.java.enclosingClass) + } + + override fun send(message: CordaMessage<*>): Deferred?> { + val result: CompletableDeferred> = CompletableDeferred() + producer.send(message.toCordaProducerRecord()) { ex -> + if (ex != null) { + result.completeExceptionally(ex) + } + } + return result + } + + override fun close() { + try { + producer.close() + } catch (ex: Exception) { + log.info( + "MessageProducerImpl failed to close producer safely. This can be observed when there are " + + "no reachable brokers. ClientId: $clientId", ex + ) + } finally { + (producerMetricsBinder as? AutoCloseable)?.close() + } + } +} + +private fun CordaMessage<*>.toCordaProducerRecord() : CordaProducerRecord<*, *> { + return CordaProducerRecord( + topic = this.getProperty("topic"), + key = this.getProperty("key"), + value = this.payload, + headers = this.getProperty("headers"), + ) +} + +private typealias Headers = List> diff --git a/libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/producer/factory/KafkaMessageProducerFactoryImpl.kt b/libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/producer/factory/KafkaMessageProducerFactoryImpl.kt deleted file mode 100644 index 7f53876e9cf..00000000000 --- a/libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/producer/factory/KafkaMessageProducerFactoryImpl.kt +++ /dev/null @@ -1,97 +0,0 @@ -package net.corda.messagebus.kafka.producer.factory - -import io.micrometer.core.instrument.binder.kafka.KafkaClientMetrics -import java.util.Properties -import net.corda.libs.configuration.SmartConfig -import net.corda.messagebus.api.configuration.ProducerConfig -import net.corda.messagebus.api.constants.ProducerRoles -import net.corda.messagebus.api.producer.MessageProducer -import net.corda.messagebus.api.producer.factory.MessageProducerFactory -import net.corda.messagebus.kafka.config.MessageBusConfigResolver -import net.corda.messagebus.kafka.producer.KafkaMessageProducerImpl -import net.corda.messagebus.kafka.serialization.CordaAvroSerializerImpl -import net.corda.messagebus.kafka.utils.KafkaRetryUtils.executeKafkaActionWithRetry -import net.corda.messaging.api.chunking.MessagingChunkFactory -import net.corda.schema.configuration.MessagingConfig -import net.corda.schema.registry.AvroSchemaRegistry -import org.apache.kafka.clients.producer.KafkaProducer -import org.osgi.framework.FrameworkUtil -import org.osgi.framework.wiring.BundleWiring -import org.osgi.service.component.annotations.Component -import org.osgi.service.component.annotations.Reference -import org.slf4j.Logger -import org.slf4j.LoggerFactory - -@Component(service = [MessageProducerFactory::class]) -class KafkaMessageProducerFactoryImpl( - @Reference(service = AvroSchemaRegistry::class) - private val avroSchemaRegistry: AvroSchemaRegistry, - @Reference(service = MessagingChunkFactory::class) - private val messagingChunkFactory: MessagingChunkFactory, -) : MessageProducerFactory { - - companion object { - private val log: Logger = LoggerFactory.getLogger(this::class.java.enclosingClass) - } - - override fun createProducer( - producerConfig: SmartConfig, - targetConfig: SmartConfig, - onSerializationError: ((ByteArray) -> Unit)? - ): MessageProducer { - val cordaProducerConfig = producerConfig.toProducerConfig() - val configResolver = MessageBusConfigResolver(targetConfig.factory) - val (resolvedConfig, kafkaProperties) = configResolver.resolve(targetConfig, cordaProducerConfig) - - return executeKafkaActionWithRetry( - action = { - val producer = createKafkaProducer(kafkaProperties, onSerializationError) - val maxAllowedMessageSize = targetConfig.getLong(MessagingConfig.MAX_ALLOWED_MSG_SIZE) - val producerChunkService = messagingChunkFactory.createChunkSerializerService(maxAllowedMessageSize) - KafkaMessageProducerImpl( - resolvedConfig, - producer, - producerChunkService, - KafkaClientMetrics(producer) - ) - }, - errorMessage = { - "KafkaMessageProducerFactoryImpl failed to producer with clientId ${cordaProducerConfig.clientId}, " + - "with configuration: $targetConfig" - }, - log = log - ) - } - - private fun createKafkaProducer( - kafkaProperties: Properties, - onSerializationError: ((ByteArray) -> Unit)? - ) : KafkaProducer { - val contextClassLoader = Thread.currentThread().contextClassLoader - val currentBundle = FrameworkUtil.getBundle(KafkaProducer::class.java) - - return try { - if (currentBundle != null) { - Thread.currentThread().contextClassLoader = currentBundle.adapt(BundleWiring::class.java).classLoader - } - KafkaProducer( - kafkaProperties, - CordaAvroSerializerImpl(avroSchemaRegistry, onSerializationError), - CordaAvroSerializerImpl(avroSchemaRegistry, onSerializationError) - ) - } finally { - Thread.currentThread().contextClassLoader = contextClassLoader - } - } - - private fun SmartConfig.toProducerConfig(): ProducerConfig { - return ProducerConfig( - clientId = getString("clientId"), - instanceId = getInt("instanceId"), - transactional = getBoolean("transactional"), - role = ProducerRoles.valueOf(getString("role")), - throwOnSerializationError = - if (hasPath("throwOnSerializationError")) getBoolean("throwOnSerializationError") else true - ) - } -} diff --git a/libs/messaging/kafka-message-bus-impl/src/test/kotlin/net/corda/messagebus/kafka/producer/KafkaMessageProducerImplTest.kt b/libs/messaging/kafka-message-bus-impl/src/test/kotlin/net/corda/messagebus/kafka/producer/KafkaMessageProducerImplTest.kt deleted file mode 100644 index 7a8032f5d27..00000000000 --- a/libs/messaging/kafka-message-bus-impl/src/test/kotlin/net/corda/messagebus/kafka/producer/KafkaMessageProducerImplTest.kt +++ /dev/null @@ -1,150 +0,0 @@ -package net.corda.messaging.kafka.subscription.net.corda.messagebus.kafka.producer - -import io.micrometer.core.instrument.binder.kafka.KafkaClientMetrics -import net.corda.messagebus.api.producer.CordaMessage -import net.corda.messagebus.api.producer.CordaProducerRecord -import net.corda.messagebus.api.producer.MessageProducer -import net.corda.messagebus.kafka.config.ResolvedProducerConfig -import net.corda.messagebus.kafka.producer.KafkaMessageProducerImpl -import net.corda.messaging.api.chunking.ChunkSerializerService -import net.corda.messaging.api.exception.CordaMessageAPIFatalException -import net.corda.messaging.api.exception.CordaMessageAPIIntermittentException -import org.apache.kafka.clients.producer.MockProducer -import org.apache.kafka.clients.producer.ProducerRecord -import org.apache.kafka.common.KafkaException -import org.junit.jupiter.api.BeforeEach -import org.junit.jupiter.api.Test -import org.junit.jupiter.api.assertThrows -import org.mockito.Mockito.doThrow -import org.mockito.Mockito.times -import org.mockito.kotlin.any -import org.mockito.kotlin.anyOrNull -import org.mockito.kotlin.atLeastOnce -import org.mockito.kotlin.eq -import org.mockito.kotlin.mock -import org.mockito.kotlin.verify -import org.mockito.kotlin.whenever - -class KafkaMessageProducerImplTest { - private lateinit var kafkaProducer: MockProducer - private lateinit var messageProducer: KafkaMessageProducerImpl - - private val config = ResolvedProducerConfig("clientId", true, "prefix", false) - private val chunkSerializerService: ChunkSerializerService = mock() - private val metricsBinder: KafkaClientMetrics = mock() - private val defaultHeaders: List> = emptyList() - private val messageProps: MutableMap = mutableMapOf( - "topic" to "topic", - "key" to "key", - "headers" to defaultHeaders - ) - private val message: CordaMessage = CordaMessage("value", messageProps) - - - @BeforeEach - fun setup() { - kafkaProducer = mock() - messageProducer = KafkaMessageProducerImpl(config, kafkaProducer, chunkSerializerService, metricsBinder) - } - - @Test - fun testSend() { - val callback = mock() - messageProducer.send(message, callback) - - val expected = ProducerRecord( - config.topicPrefix + message.getProperty("topic"), - message.getProperty("key"), - message.payload - ) - - verify(kafkaProducer).send(eq(expected), any()) - } - - @Test - fun testSendWithPartition() { - val callback = mock() - val messageWithPartition = message.copy().apply { - this.addProperty(Pair("partition", 1)) - } - messageProducer.send(messageWithPartition, callback) - - val expected = ProducerRecord( - config.topicPrefix + messageWithPartition.getProperty("topic"), - 1, - messageWithPartition.getProperty("key"), - messageWithPartition.payload - ) - - verify(kafkaProducer).send(eq(expected), any()) - } - - @Test - fun testSendWithError() { - val callback = mock() - val record = ProducerRecord( - config.topicPrefix + message.getProperty("topic"), - message.getProperty("key"), - message.payload - ) - - doThrow(KafkaException("")).whenever(kafkaProducer).send(eq(record), any()) - assertThrows { messageProducer.send(message, callback) } - } - - @Test - fun testSendMessages() { - val messages = listOf(message, message, message) - val expectedPublishedMessages = - messages.map { - ProducerRecord( - config.topicPrefix + it.getProperty("topic"), - it.getProperty("key"), - it.payload - ) - } - - messageProducer.sendMessages(messages) - verify(kafkaProducer, times(3)).send(any(), anyOrNull()) - expectedPublishedMessages.forEach { verify(kafkaProducer, atLeastOnce()).send(eq(it), any()) } - } - - @Test - fun testSendMessagesWithError() { - val record = ProducerRecord( - config.topicPrefix + message.getProperty("topic"), - message.getProperty("key"), - message.payload - ) - - doThrow(KafkaException("")).whenever(kafkaProducer).send(eq(record), any()) - assertThrows { - messageProducer.sendMessages(listOf(message, message, message)) - } - } - - // TODO: Producer-level chunking is not yet implemented for non-transactional kafka producers - @Test - fun `Trying to send chunked message throws a fatal exception and executes callback`() { - val callback = mock() - val record = CordaProducerRecord( - config.topicPrefix + message.getProperty("topic"), - message.getProperty("key"), - message.payload - ) - - whenever(chunkSerializerService.generateChunkedRecords(any())).thenReturn(listOf(record, record)) - assertThrows { - messageProducer.send(message, callback) - } - - verify(callback, org.mockito.kotlin.times(1)).onCompletion(any()) - } - - @Test - fun testClose() { - messageProducer.close() - verify(kafkaProducer, times(1)).close() - verify(metricsBinder, times(1)).close() - } -} \ No newline at end of file diff --git a/libs/messaging/kafka-message-bus-impl/src/test/kotlin/net/corda/messagebus/kafka/producer/MessageProducerImplTest.kt b/libs/messaging/kafka-message-bus-impl/src/test/kotlin/net/corda/messagebus/kafka/producer/MessageProducerImplTest.kt new file mode 100644 index 00000000000..863905d78d5 --- /dev/null +++ b/libs/messaging/kafka-message-bus-impl/src/test/kotlin/net/corda/messagebus/kafka/producer/MessageProducerImplTest.kt @@ -0,0 +1,76 @@ +package net.corda.messaging.kafka.subscription.net.corda.messagebus.kafka.producer + +import io.micrometer.core.instrument.binder.kafka.KafkaClientMetrics +import kotlinx.coroutines.runBlocking +import net.corda.messagebus.api.producer.CordaMessage +import net.corda.messagebus.api.producer.CordaProducer +import net.corda.messagebus.api.producer.CordaProducerRecord +import net.corda.messagebus.kafka.producer.MessageProducerImpl +import org.apache.kafka.common.KafkaException +import org.junit.jupiter.api.BeforeEach +import org.junit.jupiter.api.Test +import org.junit.jupiter.api.assertThrows +import org.mockito.Mockito.doThrow +import org.mockito.Mockito.times +import org.mockito.kotlin.any +import org.mockito.kotlin.eq +import org.mockito.kotlin.mock +import org.mockito.kotlin.verify +import org.mockito.kotlin.whenever + +class MessageProducerImplTest { + private lateinit var cordaProducer: CordaProducer + private lateinit var messageProducer: MessageProducerImpl + + private val metricsBinder: KafkaClientMetrics = mock() + private val defaultHeaders: List> = emptyList() + private val messageProps: MutableMap = mutableMapOf( + "topic" to "topic", + "key" to "key", + "headers" to defaultHeaders + ) + private val message: CordaMessage = CordaMessage("value", messageProps) + + + @BeforeEach + fun setup() { + cordaProducer = mock() + messageProducer = MessageProducerImpl("client-id", cordaProducer, metricsBinder) + } + + @Test + fun testSend() { + messageProducer.send(message) + + val expected = CordaProducerRecord( + message.getProperty("topic"), + message.getProperty("key"), + message.payload + ) + + verify(cordaProducer).send(eq(expected), any()) + } + + @Test + fun testSendWithError() { + val record = CordaProducerRecord( + message.getProperty("topic"), + message.getProperty("key"), + message.payload + ) + + doThrow(KafkaException("")).whenever(cordaProducer).send(eq(record), any()) + assertThrows { + runBlocking { + messageProducer.send(message).await() + } + } + } + + @Test + fun testClose() { + messageProducer.close() + verify(cordaProducer, times(1)).close() + verify(metricsBinder, times(1)).close() + } +} \ No newline at end of file diff --git a/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/CordaMessage.kt b/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/CordaMessage.kt index a478713ed69..6e2087907e1 100644 --- a/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/CordaMessage.kt +++ b/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/CordaMessage.kt @@ -15,8 +15,8 @@ data class CordaMessage( * * @param property A key value [Pair] to insert into our additional properties map. */ - fun addProperty(property: Pair) { - props[property.first] = property.second + fun addProperty(key: String, value: Any) { + props[key] = value } /** diff --git a/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/MessageProducer.kt b/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/MessageProducer.kt index 73d282df49a..1e9837fe2a5 100644 --- a/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/MessageProducer.kt +++ b/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/MessageProducer.kt @@ -4,29 +4,11 @@ import kotlinx.coroutines.Deferred interface MessageProducer : AutoCloseable { /** - * Defines the callback for post-send events. If there was an exception it will be provided on this callback. - */ - fun interface Callback { - fun onCompletion(exception: Exception?) - } - - /** - * Asynchronously sends a generic [CordaMessage], and invoke the provided callback when the message has been acknowledged. - * If the developer would like this to be a blocking call, this can be achieved through the callback. + * Asynchronously sends a generic [CordaMessage], and returns any result/error through a [Deferred] response. * * @param message The [CordaMessage] to send. * @return [Deferred] instance representing the asynchronous computation result, or null if the destination doesn't * provide a response. * */ - fun send(message: CordaMessage<*>, callback: Callback?) : Deferred<*>? - - /** - * Send a batch of [CordaMessage] instances to their respective destinations. These should all be of the same type - * (E.g. Kafka, DB, RPC). - * - * @param messages the list of [CordaMessage] to be sent. - * @return List of [Deferred] instances representing the asynchronous computation results, or null if the - * destination doesn't provide a response. - */ - fun sendMessages(messages: List>) : List>? + fun send(message: CordaMessage<*>) : Deferred?> } diff --git a/libs/messaging/message-bus/src/test/kotlin/net/corda/messagebus/api/producer/CordaMessageTest.kt b/libs/messaging/message-bus/src/test/kotlin/net/corda/messagebus/api/producer/CordaMessageTest.kt index b53790abef2..5a06af7b9b7 100644 --- a/libs/messaging/message-bus/src/test/kotlin/net/corda/messagebus/api/producer/CordaMessageTest.kt +++ b/libs/messaging/message-bus/src/test/kotlin/net/corda/messagebus/api/producer/CordaMessageTest.kt @@ -20,7 +20,7 @@ class CordaMessageTest { @Test fun `Test add property (string)`() { val message: CordaMessage = CordaMessage(payload) - message.addProperty(Pair(topicKey, topicValue)) + message.addProperty(topicKey, topicValue) assertEquals(message.props, mutableMapOf(topicKey to topicValue)) } @@ -28,7 +28,7 @@ class CordaMessageTest { @Test fun `Test add property (long)`() { val message: CordaMessage = CordaMessage(payload) - message.addProperty(Pair(partitionKey, partitionValue)) + message.addProperty(partitionKey, partitionValue) assertEquals(message.props, mutableMapOf(partitionKey to partitionValue)) } From a147437df8c4c0ae5280c49481589087ae6ced40 Mon Sep 17 00:00:00 2001 From: ben-millar Date: Tue, 12 Sep 2023 14:31:06 +0100 Subject: [PATCH 13/50] CORE-16177 Reverting changes to CordaKafkaProducerImpl --- .../kafka/producer/CordaKafkaProducerImpl.kt | 21 ++++++++++++------- 1 file changed, 13 insertions(+), 8 deletions(-) diff --git a/libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/producer/CordaKafkaProducerImpl.kt b/libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/producer/CordaKafkaProducerImpl.kt index b561abc1e9e..b3c9abc1e8f 100644 --- a/libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/producer/CordaKafkaProducerImpl.kt +++ b/libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/producer/CordaKafkaProducerImpl.kt @@ -82,7 +82,7 @@ class CordaKafkaProducerImpl( // There is no other producer, we are not a zombie, and so don't need to be fenced, we can simply abort and retry. KafkaException::class.java ) - val apiExceptions: Set> = setOf( + val ApiExceptions: Set> = setOf( CordaMessageAPIFatalException::class.java, CordaMessageAPIIntermittentException::class.java ) @@ -381,19 +381,24 @@ class CordaKafkaProducerImpl( private fun handleException(ex: Exception, operation: String, abortTransaction: Boolean) { val errorString = "$operation for CordaKafkaProducer with clientId ${config.clientId}" when (ex::class.java) { - in fatalExceptions -> throw CordaMessageAPIFatalException("FatalError occurred $errorString", ex) - in transientExceptions -> { - if (abortTransaction) { - abortTransaction() - } - throw CordaMessageAPIIntermittentException("Error occurred $errorString", ex) + in fatalExceptions -> { + throw CordaMessageAPIFatalException("FatalError occurred $errorString", ex) } - in apiExceptions -> throw ex + IllegalStateException::class.java -> { // It's not clear whether the producer is ok to abort and continue or not in this case, so play it safe // and let the client know to create a new one. throw CordaMessageAPIProducerRequiresReset("Error occurred $errorString", ex) } + + in transientExceptions -> { + if (abortTransaction) { + abortTransaction() + } + throw CordaMessageAPIIntermittentException("Error occurred $errorString", ex) + } + in ApiExceptions -> { throw ex } + else -> { // Here we do not know what the exact cause of the exception is, but we do know Kafka has not told us we // must close down, nor has it told us we can abort and retry. In this instance the most sensible thing From d63542184d07e9fe49e9d23d00115dcbdf37b72e Mon Sep 17 00:00:00 2001 From: ben-millar Date: Tue, 12 Sep 2023 14:32:12 +0100 Subject: [PATCH 14/50] CORE-16177 Removing MessageProducerFactory --- .../factory/MessageProducerFactory.kt | 24 ------------------- 1 file changed, 24 deletions(-) delete mode 100644 libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/factory/MessageProducerFactory.kt diff --git a/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/factory/MessageProducerFactory.kt b/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/factory/MessageProducerFactory.kt deleted file mode 100644 index 2c08cecd1fd..00000000000 --- a/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/factory/MessageProducerFactory.kt +++ /dev/null @@ -1,24 +0,0 @@ -package net.corda.messagebus.api.producer.factory - -import net.corda.libs.configuration.SmartConfig -import net.corda.messagebus.api.producer.MessageProducer - -/** - * Producer factory interface for creating message producers - */ -interface MessageProducerFactory { - - /** - * Generate a message producer with given properties. - * @param producerConfig The mandatory config for setting up producers. - * @param targetConfig Configuration for connecting to the producer target (message bus, rpc endpoint, etc.) - * @param onSerializationError a callback to execute when serialization fails. - * @return [MessageProducer] capable of publishing records of any type to some external system. - * @throws CordaMessageAPIFatalException if producer cannot be created. - */ - fun createProducer( - producerConfig: SmartConfig, - targetConfig: SmartConfig, - onSerializationError: ((ByteArray) -> Unit)? = null - ): MessageProducer -} From a6a094d4bd53cbc11374bf6d42a54a71828ba365 Mon Sep 17 00:00:00 2001 From: ben-millar Date: Tue, 12 Sep 2023 14:41:14 +0100 Subject: [PATCH 15/50] CORE-16177 Tidied up MessageProducerImpl::send() --- .../messagebus/kafka/producer/MessageProducerImpl.kt | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/producer/MessageProducerImpl.kt b/libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/producer/MessageProducerImpl.kt index d613db3f567..9b22dff5223 100644 --- a/libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/producer/MessageProducerImpl.kt +++ b/libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/producer/MessageProducerImpl.kt @@ -24,15 +24,12 @@ class MessageProducerImpl( private val log: Logger = LoggerFactory.getLogger(this::class.java.enclosingClass) } - override fun send(message: CordaMessage<*>): Deferred?> { - val result: CompletableDeferred> = CompletableDeferred() - producer.send(message.toCordaProducerRecord()) { ex -> - if (ex != null) { - result.completeExceptionally(ex) + override fun send(message: CordaMessage<*>): Deferred?> = + CompletableDeferred>().apply { + producer.send(message.toCordaProducerRecord()) { ex -> + ex?.let { completeExceptionally(ex) } } } - return result - } override fun close() { try { From 87df137560ef19e4d4047d86f51328ef219220c9 Mon Sep 17 00:00:00 2001 From: ben-millar Date: Thu, 14 Sep 2023 14:45:56 +0100 Subject: [PATCH 16/50] CORE-16177 Updating comment in CordaMessage.kt --- .../kotlin/net/corda/messagebus/api/producer/CordaMessage.kt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/CordaMessage.kt b/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/CordaMessage.kt index 6e2087907e1..300dae6a17a 100644 --- a/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/CordaMessage.kt +++ b/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/CordaMessage.kt @@ -55,7 +55,7 @@ data class CordaMessage( } /** - * Retrieves a property of a specific type from the additional properties map, returning null if not found or if the cast fails. + * Retrieves a property of a specific type from the additional properties map, returning null if not found. * * @param key The identifier of the property to retrieve. * @return The property associated with the given key cast to the specified type, or null if not found or casting fails. From da6f7b7ce65b4896bc9f024a3dd93e37e549ca62 Mon Sep 17 00:00:00 2001 From: ben-millar Date: Thu, 14 Sep 2023 17:41:23 +0100 Subject: [PATCH 17/50] CORE-16177 Ensuring 'Deferred' is completed in all cases before returning --- libs/messaging/kafka-message-bus-impl/build.gradle | 4 ++-- .../messagebus/kafka/producer/MessageProducerImpl.kt | 8 ++++++-- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/libs/messaging/kafka-message-bus-impl/build.gradle b/libs/messaging/kafka-message-bus-impl/build.gradle index 56c9c8fb411..e7fe0d596dc 100644 --- a/libs/messaging/kafka-message-bus-impl/build.gradle +++ b/libs/messaging/kafka-message-bus-impl/build.gradle @@ -8,7 +8,8 @@ description 'Kafka Message Bus Impl' dependencies { compileOnly 'org.osgi:osgi.core' compileOnly 'org.osgi:org.osgi.service.component.annotations' - + compileOnly "org.jetbrains.kotlinx:kotlinx-coroutines-core-jvm:$kotlinCoroutinesVersion" + implementation project(":libs:messaging:messaging") implementation project(":libs:messaging:message-bus") implementation project(":libs:schema-registry:schema-registry") @@ -24,7 +25,6 @@ dependencies { implementation "net.corda:corda-base" implementation "net.corda:corda-config-schema" implementation 'org.jetbrains.kotlin:kotlin-osgi-bundle' - implementation "org.jetbrains.kotlinx:kotlinx-coroutines-core-jvm:$kotlinCoroutinesVersion" implementation "org.apache.servicemix.bundles:org.apache.servicemix.bundles.kafka-clients:$kafkaClientVersion" constraints { implementation("org.xerial.snappy:snappy-java:$xerialSnappyVersion") { diff --git a/libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/producer/MessageProducerImpl.kt b/libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/producer/MessageProducerImpl.kt index 9b22dff5223..0ae4726e42f 100644 --- a/libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/producer/MessageProducerImpl.kt +++ b/libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/producer/MessageProducerImpl.kt @@ -25,9 +25,13 @@ class MessageProducerImpl( } override fun send(message: CordaMessage<*>): Deferred?> = - CompletableDeferred>().apply { + CompletableDeferred?>().apply { producer.send(message.toCordaProducerRecord()) { ex -> - ex?.let { completeExceptionally(ex) } + if (ex != null) { + completeExceptionally(ex) + } else { + complete(null) + } } } From eb544fc02f72823af9d1c170115153b71b11bbb5 Mon Sep 17 00:00:00 2001 From: ben-millar Date: Thu, 14 Sep 2023 17:49:26 +0100 Subject: [PATCH 18/50] CORE-16177 Modifying how the kotlinx.coroutine dependency is packaged --- .../workers/workers-smoketest/build.gradle | 2 +- libs/kotlin-coroutines/build.gradle | 63 +++++++++++++++++++ .../kafka-message-bus-impl/build.gradle | 2 +- libs/messaging/message-bus/build.gradle | 2 +- settings.gradle | 3 + 5 files changed, 69 insertions(+), 3 deletions(-) create mode 100644 libs/kotlin-coroutines/build.gradle diff --git a/applications/workers/workers-smoketest/build.gradle b/applications/workers/workers-smoketest/build.gradle index f22fbda10d4..e9f999ff4e6 100644 --- a/applications/workers/workers-smoketest/build.gradle +++ b/applications/workers/workers-smoketest/build.gradle @@ -54,7 +54,7 @@ kotlin { dependencies { // NO CORDA DEPENDENCIES!! implementation "org.jetbrains.kotlin:kotlin-stdlib:$kotlinVersion" - implementation "org.jetbrains.kotlinx:kotlinx-coroutines-core-jvm:$kotlinCoroutinesVersion" + implementation project(path: ':libs:kotlin-coroutines', configuration: 'bundle') // Avoid having the schema names and keys scattered across projects smokeTestImplementation "net.corda:corda-config-schema:$cordaApiVersion" diff --git a/libs/kotlin-coroutines/build.gradle b/libs/kotlin-coroutines/build.gradle new file mode 100644 index 00000000000..9d4e2168de9 --- /dev/null +++ b/libs/kotlin-coroutines/build.gradle @@ -0,0 +1,63 @@ +plugins { + id 'biz.aQute.bnd.builder' + id 'com.jfrog.artifactory' + id 'maven-publish' +} + +description 'Kotlin Coroutines' + +configurations { + bundle { + canBeDeclared = false + canBeResolved = false + } +} + +dependencies { + implementation platform("org.jetbrains.kotlinx:kotlinx-coroutines-bom:$kotlinCoroutinesVersion") + compileOnly 'org.jetbrains.kotlinx:kotlinx-coroutines-core-jvm' +} + +def jar = tasks.named('jar', Jar) { + archiveBaseName = 'corda-kotlin-coroutines' + + bundle { + bnd """\ +Bundle-Name: \${project.description} +Bundle-SymbolicName: \${project.group}.kotlin-coroutines +Bundle-Version: ${kotlinCoroutinesVersion} +Import-Package: \ + android.os;resolution:=optional,\ + sun.misc;resolution:=optional,\ + * +Export-Package: \ + kotlinx.coroutines.* +Multi-Release: true +Quasar-Ignore-Package: kotlinx.coroutines** +-fixupmessages: \ + "Export [^,]++,\\\\s++has (\\\\d++),\\\\s++private references "; restrict:=warning; is:=error,\ + "Classes found in the wrong directory"; restrict:=error; is:=warning +-includeresource: @kotlinx-coroutines-core-jvm-${kotlinCoroutinesVersion}.jar +""" + } +} + +tasks.named('sourcesJar', Jar) { + enabled = false +} + +artifacts { + bundle jar +} + +publishing { + publications { + coroutines(MavenPublication) { + from components.java + } + } +} + +artifactoryPublish { + publications 'coroutines' +} diff --git a/libs/messaging/kafka-message-bus-impl/build.gradle b/libs/messaging/kafka-message-bus-impl/build.gradle index e7fe0d596dc..075bdf52243 100644 --- a/libs/messaging/kafka-message-bus-impl/build.gradle +++ b/libs/messaging/kafka-message-bus-impl/build.gradle @@ -8,8 +8,8 @@ description 'Kafka Message Bus Impl' dependencies { compileOnly 'org.osgi:osgi.core' compileOnly 'org.osgi:org.osgi.service.component.annotations' - compileOnly "org.jetbrains.kotlinx:kotlinx-coroutines-core-jvm:$kotlinCoroutinesVersion" + implementation project(path: ':libs:kotlin-coroutines', configuration: 'bundle') implementation project(":libs:messaging:messaging") implementation project(":libs:messaging:message-bus") implementation project(":libs:schema-registry:schema-registry") diff --git a/libs/messaging/message-bus/build.gradle b/libs/messaging/message-bus/build.gradle index d313b644130..4ea803869cb 100644 --- a/libs/messaging/message-bus/build.gradle +++ b/libs/messaging/message-bus/build.gradle @@ -8,8 +8,8 @@ dependencies { compileOnly "co.paralleluniverse:quasar-osgi-annotations:$quasarVersion" implementation platform("net.corda:corda-api:$cordaApiVersion") + implementation project(path: ':libs:kotlin-coroutines', configuration: 'bundle') implementation 'org.jetbrains.kotlin:kotlin-osgi-bundle' - implementation "org.jetbrains.kotlinx:kotlinx-coroutines-core-jvm:$kotlinCoroutinesVersion" implementation "com.typesafe:config:$typeSafeConfigVersion" implementation "net.corda:corda-base" implementation project(":libs:configuration:configuration-core") diff --git a/settings.gradle b/settings.gradle index f13095c4a16..2937176707b 100644 --- a/settings.gradle +++ b/settings.gradle @@ -540,3 +540,6 @@ gradleEnterprise { } } } +include 'libs:kotlin-coroutines' +findProject(':libs:kotlin-coroutines')?.name = 'kotlin-coroutines' + From 6238bd30952fd7c5f161aa7aea4e1e55f5f79bc6 Mon Sep 17 00:00:00 2001 From: Chris Rankin Date: Sun, 17 Sep 2023 14:11:07 +0100 Subject: [PATCH 19/50] CORE-16177: Align Kotlin coroutines with other wrapped bundles. (#4657) --- .../workers/workers-smoketest/build.gradle | 2 +- .../main/groovy/corda.common-library.gradle | 23 ++++++++++++++++--- libs/kotlin-coroutines/build.gradle | 8 ++++++- 3 files changed, 28 insertions(+), 5 deletions(-) diff --git a/applications/workers/workers-smoketest/build.gradle b/applications/workers/workers-smoketest/build.gradle index e9f999ff4e6..f22fbda10d4 100644 --- a/applications/workers/workers-smoketest/build.gradle +++ b/applications/workers/workers-smoketest/build.gradle @@ -54,7 +54,7 @@ kotlin { dependencies { // NO CORDA DEPENDENCIES!! implementation "org.jetbrains.kotlin:kotlin-stdlib:$kotlinVersion" - implementation project(path: ':libs:kotlin-coroutines', configuration: 'bundle') + implementation "org.jetbrains.kotlinx:kotlinx-coroutines-core-jvm:$kotlinCoroutinesVersion" // Avoid having the schema names and keys scattered across projects smokeTestImplementation "net.corda:corda-config-schema:$cordaApiVersion" diff --git a/buildSrc/src/main/groovy/corda.common-library.gradle b/buildSrc/src/main/groovy/corda.common-library.gradle index e312488a153..62de2c5f925 100644 --- a/buildSrc/src/main/groovy/corda.common-library.gradle +++ b/buildSrc/src/main/groovy/corda.common-library.gradle @@ -22,9 +22,26 @@ configurations { configureEach { resolutionStrategy { dependencySubstitution { - substitute module('antlr:antlr') using project(':libs:antlr') - substitute module('de.javakaffee:kryo-serializers') using project(':libs:serialization:kryo-serializers') - substitute module('software.amazon.awssdk:cloudwatch') using project(':libs:awssdk') + substitute module('antlr:antlr') using variant(project(':libs:antlr')) { + attributes { + attribute(LibraryElements.LIBRARY_ELEMENTS_ATTRIBUTE, objects.named(LibraryElements, LibraryElements.JAR)) + } + } + substitute module('de.javakaffee:kryo-serializers') using variant(project(':libs:serialization:kryo-serializers')) { + attributes { + attribute(LibraryElements.LIBRARY_ELEMENTS_ATTRIBUTE, objects.named(LibraryElements, LibraryElements.JAR)) + } + } + substitute module('org.jetbrains.kotlinx:kotlinx-coroutines-core-jvm') using variant(project(':libs:kotlin-coroutines')) { + attributes { + attribute(LibraryElements.LIBRARY_ELEMENTS_ATTRIBUTE, objects.named(LibraryElements, LibraryElements.JAR)) + } + } + substitute module('software.amazon.awssdk:cloudwatch') using variant(project(':libs:awssdk')) { + attributes { + attribute(LibraryElements.LIBRARY_ELEMENTS_ATTRIBUTE, objects.named(LibraryElements, LibraryElements.JAR)) + } + } } } } diff --git a/libs/kotlin-coroutines/build.gradle b/libs/kotlin-coroutines/build.gradle index 9d4e2168de9..744d8f40507 100644 --- a/libs/kotlin-coroutines/build.gradle +++ b/libs/kotlin-coroutines/build.gradle @@ -1,3 +1,5 @@ +import static aQute.bnd.version.MavenVersion.parseMavenString + plugins { id 'biz.aQute.bnd.builder' id 'com.jfrog.artifactory' @@ -21,11 +23,15 @@ dependencies { def jar = tasks.named('jar', Jar) { archiveBaseName = 'corda-kotlin-coroutines' + ext { + bundleVersion = parseMavenString(kotlinCoroutinesVersion).OSGiVersion + } + bundle { bnd """\ Bundle-Name: \${project.description} Bundle-SymbolicName: \${project.group}.kotlin-coroutines -Bundle-Version: ${kotlinCoroutinesVersion} +Bundle-Version: \${task.bundleVersion} Import-Package: \ android.os;resolution:=optional,\ sun.misc;resolution:=optional,\ From b532ca04ad513d661b8bdd4f8875fcea5fe3716e Mon Sep 17 00:00:00 2001 From: Miljenko Brkic Date: Mon, 18 Sep 2023 13:35:58 +0100 Subject: [PATCH 20/50] CORE-16177 Refactored to address PR comments --- libs/messaging/message-bus/build.gradle | 12 ------- .../api/producer/MessageProducer.kt | 14 --------- libs/messaging/messaging-impl/build.gradle | 1 + .../messaging/mediator/MessageBusProducer.kt} | 31 +++++++------------ .../mediator/MessageBusProducerTest.kt} | 28 +++++++---------- libs/messaging/messaging/build.gradle | 12 +++++++ .../messaging/api/mediator/package-info.java | 4 +++ .../api/mediator/MediatorMessage.kt} | 20 ++++++------ .../api/mediator/MediatorProducer.kt | 22 +++++++++++++ .../api/mediator/MediatorMessageTest.kt} | 18 +++++------ 10 files changed, 81 insertions(+), 81 deletions(-) delete mode 100644 libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/MessageProducer.kt rename libs/messaging/{kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/producer/MessageProducerImpl.kt => messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MessageBusProducer.kt} (51%) rename libs/messaging/{kafka-message-bus-impl/src/test/kotlin/net/corda/messagebus/kafka/producer/MessageProducerImplTest.kt => messaging-impl/src/test/kotlin/net/corda/messaging/mediator/MessageBusProducerTest.kt} (61%) create mode 100644 libs/messaging/messaging/src/main/java/net/corda/messaging/api/mediator/package-info.java rename libs/messaging/{message-bus/src/main/kotlin/net/corda/messagebus/api/producer/CordaMessage.kt => messaging/src/main/kotlin/net/corda/messaging/api/mediator/MediatorMessage.kt} (83%) create mode 100644 libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MediatorProducer.kt rename libs/messaging/{message-bus/src/test/kotlin/net/corda/messagebus/api/producer/CordaMessageTest.kt => messaging/src/test/kotlin/net/corda/messaging/api/mediator/MediatorMessageTest.kt} (83%) diff --git a/libs/messaging/message-bus/build.gradle b/libs/messaging/message-bus/build.gradle index 4ea803869cb..f499ca5fddc 100644 --- a/libs/messaging/message-bus/build.gradle +++ b/libs/messaging/message-bus/build.gradle @@ -8,22 +8,10 @@ dependencies { compileOnly "co.paralleluniverse:quasar-osgi-annotations:$quasarVersion" implementation platform("net.corda:corda-api:$cordaApiVersion") - implementation project(path: ':libs:kotlin-coroutines', configuration: 'bundle') implementation 'org.jetbrains.kotlin:kotlin-osgi-bundle' implementation "com.typesafe:config:$typeSafeConfigVersion" implementation "net.corda:corda-base" implementation project(":libs:configuration:configuration-core") } -// include coroutines bundle in this jar as it doesn't have OSGi metadata -tasks.named('jar', Jar) { - bundle { - bnd '''\ --conditionalpackage: kotlinx.coroutines* -Export-Package: kotlinx.coroutines* -Import-Package: !android*, * -''' - } -} - description 'Message Bus API' diff --git a/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/MessageProducer.kt b/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/MessageProducer.kt deleted file mode 100644 index 1e9837fe2a5..00000000000 --- a/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/MessageProducer.kt +++ /dev/null @@ -1,14 +0,0 @@ -package net.corda.messagebus.api.producer - -import kotlinx.coroutines.Deferred - -interface MessageProducer : AutoCloseable { - /** - * Asynchronously sends a generic [CordaMessage], and returns any result/error through a [Deferred] response. - * - * @param message The [CordaMessage] to send. - * @return [Deferred] instance representing the asynchronous computation result, or null if the destination doesn't - * provide a response. - * */ - fun send(message: CordaMessage<*>) : Deferred?> -} diff --git a/libs/messaging/messaging-impl/build.gradle b/libs/messaging/messaging-impl/build.gradle index 83e169e6e6c..d313764995a 100644 --- a/libs/messaging/messaging-impl/build.gradle +++ b/libs/messaging/messaging-impl/build.gradle @@ -12,6 +12,7 @@ dependencies { implementation project(":libs:chunking:chunking-core") implementation project(":libs:crypto:cipher-suite") implementation project(":libs:crypto:crypto-core") + implementation project(":libs:kotlin-coroutines") implementation project(":libs:lifecycle:lifecycle") implementation project(":libs:messaging:messaging") implementation project(":libs:messaging:message-bus") diff --git a/libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/producer/MessageProducerImpl.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MessageBusProducer.kt similarity index 51% rename from libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/producer/MessageProducerImpl.kt rename to libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MessageBusProducer.kt index 0ae4726e42f..c95a7c262a1 100644 --- a/libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/producer/MessageProducerImpl.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MessageBusProducer.kt @@ -1,31 +1,25 @@ -package net.corda.messagebus.kafka.producer +package net.corda.messaging.mediator -import io.micrometer.core.instrument.binder.MeterBinder import kotlinx.coroutines.CompletableDeferred import kotlinx.coroutines.Deferred -import net.corda.messagebus.api.producer.CordaMessage import net.corda.messagebus.api.producer.CordaProducer import net.corda.messagebus.api.producer.CordaProducerRecord -import net.corda.messagebus.api.producer.MessageProducer -import net.corda.metrics.CordaMetrics +import net.corda.messaging.api.mediator.MediatorMessage +import net.corda.messaging.api.mediator.MediatorProducer import org.slf4j.Logger import org.slf4j.LoggerFactory -class MessageProducerImpl( - private val clientId: String, +class MessageBusProducer( + override val id: String, private val producer: CordaProducer, - private val producerMetricsBinder: MeterBinder, -) : MessageProducer { - init { - producerMetricsBinder.bindTo(CordaMetrics.registry) - } +) : MediatorProducer { private companion object { private val log: Logger = LoggerFactory.getLogger(this::class.java.enclosingClass) } - override fun send(message: CordaMessage<*>): Deferred?> = - CompletableDeferred?>().apply { + override fun send(message: MediatorMessage<*>): Deferred?> = + CompletableDeferred?>().apply { producer.send(message.toCordaProducerRecord()) { ex -> if (ex != null) { completeExceptionally(ex) @@ -40,16 +34,13 @@ class MessageProducerImpl( producer.close() } catch (ex: Exception) { log.info( - "MessageProducerImpl failed to close producer safely. This can be observed when there are " + - "no reachable brokers. ClientId: $clientId", ex + "Failed to close producer [$id] safely.", ex ) - } finally { - (producerMetricsBinder as? AutoCloseable)?.close() } } } -private fun CordaMessage<*>.toCordaProducerRecord() : CordaProducerRecord<*, *> { +private fun MediatorMessage<*>.toCordaProducerRecord() : CordaProducerRecord<*, *> { return CordaProducerRecord( topic = this.getProperty("topic"), key = this.getProperty("key"), @@ -58,4 +49,4 @@ private fun CordaMessage<*>.toCordaProducerRecord() : CordaProducerRecord<*, *> ) } -private typealias Headers = List> +private typealias Headers = List> \ No newline at end of file diff --git a/libs/messaging/kafka-message-bus-impl/src/test/kotlin/net/corda/messagebus/kafka/producer/MessageProducerImplTest.kt b/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/MessageBusProducerTest.kt similarity index 61% rename from libs/messaging/kafka-message-bus-impl/src/test/kotlin/net/corda/messagebus/kafka/producer/MessageProducerImplTest.kt rename to libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/MessageBusProducerTest.kt index 863905d78d5..eaa141e2d31 100644 --- a/libs/messaging/kafka-message-bus-impl/src/test/kotlin/net/corda/messagebus/kafka/producer/MessageProducerImplTest.kt +++ b/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/MessageBusProducerTest.kt @@ -1,12 +1,10 @@ -package net.corda.messaging.kafka.subscription.net.corda.messagebus.kafka.producer +package net.corda.messaging.mediator -import io.micrometer.core.instrument.binder.kafka.KafkaClientMetrics import kotlinx.coroutines.runBlocking -import net.corda.messagebus.api.producer.CordaMessage import net.corda.messagebus.api.producer.CordaProducer import net.corda.messagebus.api.producer.CordaProducerRecord -import net.corda.messagebus.kafka.producer.MessageProducerImpl -import org.apache.kafka.common.KafkaException +import net.corda.messaging.api.mediator.MediatorMessage +import net.corda.v5.base.exceptions.CordaRuntimeException import org.junit.jupiter.api.BeforeEach import org.junit.jupiter.api.Test import org.junit.jupiter.api.assertThrows @@ -18,29 +16,28 @@ import org.mockito.kotlin.mock import org.mockito.kotlin.verify import org.mockito.kotlin.whenever -class MessageProducerImplTest { +class MessageBusProducerTest { private lateinit var cordaProducer: CordaProducer - private lateinit var messageProducer: MessageProducerImpl + private lateinit var mediatorProducer: MessageBusProducer - private val metricsBinder: KafkaClientMetrics = mock() private val defaultHeaders: List> = emptyList() private val messageProps: MutableMap = mutableMapOf( "topic" to "topic", "key" to "key", "headers" to defaultHeaders ) - private val message: CordaMessage = CordaMessage("value", messageProps) + private val message: MediatorMessage = MediatorMessage("value", messageProps) @BeforeEach fun setup() { cordaProducer = mock() - messageProducer = MessageProducerImpl("client-id", cordaProducer, metricsBinder) + mediatorProducer = MessageBusProducer("client-id", cordaProducer) } @Test fun testSend() { - messageProducer.send(message) + mediatorProducer.send(message) val expected = CordaProducerRecord( message.getProperty("topic"), @@ -59,18 +56,17 @@ class MessageProducerImplTest { message.payload ) - doThrow(KafkaException("")).whenever(cordaProducer).send(eq(record), any()) - assertThrows { + doThrow(CordaRuntimeException("")).whenever(cordaProducer).send(eq(record), any()) + assertThrows { runBlocking { - messageProducer.send(message).await() + mediatorProducer.send(message).await() } } } @Test fun testClose() { - messageProducer.close() + mediatorProducer.close() verify(cordaProducer, times(1)).close() - verify(metricsBinder, times(1)).close() } } \ No newline at end of file diff --git a/libs/messaging/messaging/build.gradle b/libs/messaging/messaging/build.gradle index cc0a5337e00..e0cc4a00340 100644 --- a/libs/messaging/messaging/build.gradle +++ b/libs/messaging/messaging/build.gradle @@ -14,6 +14,7 @@ dependencies { implementation "net.corda:corda-base" implementation "net.corda:corda-config-schema" implementation project(":libs:chunking:chunking-core") + implementation project(":libs:kotlin-coroutines") implementation project(":libs:lifecycle:lifecycle") implementation project(":libs:messaging:message-bus") implementation project(":libs:configuration:configuration-core") @@ -23,4 +24,15 @@ dependencies { testImplementation "org.mockito.kotlin:mockito-kotlin:$mockitoKotlinVersion" } +// include coroutines bundle in this jar as it doesn't have OSGi metadata +tasks.named('jar', Jar) { + bundle { + bnd '''\ +-conditionalpackage: kotlinx.coroutines* +Export-Package: kotlinx.coroutines* +Import-Package: !android*, * +''' + } +} + description 'Messaging API' diff --git a/libs/messaging/messaging/src/main/java/net/corda/messaging/api/mediator/package-info.java b/libs/messaging/messaging/src/main/java/net/corda/messaging/api/mediator/package-info.java new file mode 100644 index 00000000000..e50971fad9c --- /dev/null +++ b/libs/messaging/messaging/src/main/java/net/corda/messaging/api/mediator/package-info.java @@ -0,0 +1,4 @@ +@Export +package net.corda.messaging.api.mediator; + +import org.osgi.annotation.bundle.Export; diff --git a/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/CordaMessage.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MediatorMessage.kt similarity index 83% rename from libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/CordaMessage.kt rename to libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MediatorMessage.kt index 300dae6a17a..b7c96a3c3f0 100644 --- a/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/producer/CordaMessage.kt +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MediatorMessage.kt @@ -1,22 +1,22 @@ -package net.corda.messagebus.api.producer +package net.corda.messaging.api.mediator /** - * Object to encapsulate a generic producer message. + * Object to encapsulate a generic mediator message. * @property payload the payload of the message. - * @property props an additional properties map. + * @property properties an additional properties map. */ -@Suppress("UNCHECKED_CAST") -data class CordaMessage( +data class MediatorMessage( val payload: T?, - val props: MutableMap = mutableMapOf() + val properties: MutableMap = mutableMapOf() ) { /** * Adds a new property to the internal storage. * - * @param property A key value [Pair] to insert into our additional properties map. + * @param key Property key. + * @param value Property value. */ fun addProperty(key: String, value: Any) { - props[key] = value + properties[key] = value } /** @@ -51,7 +51,7 @@ data class CordaMessage( * @return The property associated with the given key, or null if not found. */ fun getPropertyOrNull(key: String) : Any? { - return props[key] + return properties[key] } /** @@ -63,7 +63,7 @@ data class CordaMessage( */ @JvmName("getPropertyOrNullTyped") inline fun getPropertyOrNull(key: String) : T? { - val value = props[key] ?: return null + val value = properties[key] ?: return null return (value as? T) ?: throw ClassCastException( "Property '$key' could not be cast to type: '${T::class.java}'." diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MediatorProducer.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MediatorProducer.kt new file mode 100644 index 00000000000..a53bcfc9d71 --- /dev/null +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MediatorProducer.kt @@ -0,0 +1,22 @@ +package net.corda.messaging.api.mediator + +import kotlinx.coroutines.Deferred + +/** + * Multi-source event mediator message producer. + */ +interface MediatorProducer : AutoCloseable { + /** + * Producer's unique ID. + */ + val id: String + + /** + * Asynchronously sends a generic [MediatorMessage], and returns any result/error through a [Deferred] response. + * + * @param message The [MediatorMessage] to send. + * @return [Deferred] instance representing the asynchronous computation result, or null if the destination doesn't + * provide a response. + * */ + fun send(message: MediatorMessage<*>) : Deferred?> +} diff --git a/libs/messaging/message-bus/src/test/kotlin/net/corda/messagebus/api/producer/CordaMessageTest.kt b/libs/messaging/messaging/src/test/kotlin/net/corda/messaging/api/mediator/MediatorMessageTest.kt similarity index 83% rename from libs/messaging/message-bus/src/test/kotlin/net/corda/messagebus/api/producer/CordaMessageTest.kt rename to libs/messaging/messaging/src/test/kotlin/net/corda/messaging/api/mediator/MediatorMessageTest.kt index 5a06af7b9b7..8d8ee8d1914 100644 --- a/libs/messaging/message-bus/src/test/kotlin/net/corda/messagebus/api/producer/CordaMessageTest.kt +++ b/libs/messaging/messaging/src/test/kotlin/net/corda/messaging/api/mediator/MediatorMessageTest.kt @@ -1,46 +1,46 @@ -package net.corda.messagebus.api.producer +package net.corda.messaging.api.mediator import org.junit.jupiter.api.Assertions.assertEquals import org.junit.jupiter.api.Assertions.assertNull import org.junit.jupiter.api.Assertions.assertThrows import org.junit.jupiter.api.Test -class CordaMessageTest { +class MediatorMessageTest { private val payload: String = "payload" private val topicKey: String = "topic" private val topicValue: String = "topic" private val partitionKey: String = "partition" private val partitionValue: Long = 1L - private val defaultMessage: CordaMessage = CordaMessage(payload, mutableMapOf( + private val defaultMessage: MediatorMessage = MediatorMessage(payload, mutableMapOf( topicKey to topicValue, partitionKey to partitionValue )) @Test fun `Test add property (string)`() { - val message: CordaMessage = CordaMessage(payload) + val message: MediatorMessage = MediatorMessage(payload) message.addProperty(topicKey, topicValue) - assertEquals(message.props, mutableMapOf(topicKey to topicValue)) + assertEquals(message.properties, mutableMapOf(topicKey to topicValue)) } @Test fun `Test add property (long)`() { - val message: CordaMessage = CordaMessage(payload) + val message: MediatorMessage = MediatorMessage(payload) message.addProperty(partitionKey, partitionValue) - assertEquals(message.props, mutableMapOf(partitionKey to partitionValue)) + assertEquals(message.properties, mutableMapOf(partitionKey to partitionValue)) } @Test fun `Test create message with props`() { - val message: CordaMessage = CordaMessage(payload, mutableMapOf( + val message: MediatorMessage = MediatorMessage(payload, mutableMapOf( topicKey to topicValue, partitionKey to partitionValue )) - assertEquals(message.props, mutableMapOf(topicKey to topicValue, partitionKey to partitionValue)) + assertEquals(message.properties, mutableMapOf(topicKey to topicValue, partitionKey to partitionValue)) } @Test From 15f50d773da59badda60a9ecc5f39999710a62c5 Mon Sep 17 00:00:00 2001 From: Miljenko Brkic Date: Mon, 18 Sep 2023 15:06:43 +0100 Subject: [PATCH 21/50] CORE-16177 Fixed dependencies --- libs/messaging/messaging-impl/build.gradle | 2 +- libs/messaging/messaging/build.gradle | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/libs/messaging/messaging-impl/build.gradle b/libs/messaging/messaging-impl/build.gradle index d313764995a..fca4c1cf310 100644 --- a/libs/messaging/messaging-impl/build.gradle +++ b/libs/messaging/messaging-impl/build.gradle @@ -12,7 +12,7 @@ dependencies { implementation project(":libs:chunking:chunking-core") implementation project(":libs:crypto:cipher-suite") implementation project(":libs:crypto:crypto-core") - implementation project(":libs:kotlin-coroutines") + implementation project(path: ':libs:kotlin-coroutines', configuration: 'bundle') implementation project(":libs:lifecycle:lifecycle") implementation project(":libs:messaging:messaging") implementation project(":libs:messaging:message-bus") diff --git a/libs/messaging/messaging/build.gradle b/libs/messaging/messaging/build.gradle index e0cc4a00340..d569b059fba 100644 --- a/libs/messaging/messaging/build.gradle +++ b/libs/messaging/messaging/build.gradle @@ -14,7 +14,7 @@ dependencies { implementation "net.corda:corda-base" implementation "net.corda:corda-config-schema" implementation project(":libs:chunking:chunking-core") - implementation project(":libs:kotlin-coroutines") + implementation project(path: ':libs:kotlin-coroutines', configuration: 'bundle') implementation project(":libs:lifecycle:lifecycle") implementation project(":libs:messaging:message-bus") implementation project(":libs:configuration:configuration-core") From e9b76531a40330d3d1c3517f8ded2f68168de6ff Mon Sep 17 00:00:00 2001 From: Miljenko Brkic Date: Mon, 18 Sep 2023 15:10:48 +0100 Subject: [PATCH 22/50] CORE-16199 Implementation of MessageRouter and related classes, implementation of ProducerTask, refactoring --- .../flow/service/FlowExecutorMediatorImpl.kt | 47 +++--- .../mediator/MediatorComponentFactory.kt | 80 +++++++++ .../mediator/MediatorStateManager.kt | 77 +++++++++ .../messaging/mediator/MediatorTaskManager.kt | 153 +++++++++++++++++ .../mediator/MultiSourceEventMediatorImpl.kt | 157 ++++++++++-------- .../corda/messaging/mediator/ProcessorTask.kt | 60 +++---- .../corda/messaging/mediator/ProducerTask.kt | 29 ++++ ...=> MultiSourceEventMediatorFactoryImpl.kt} | 10 +- .../api/mediator/MediatorConsumer.kt | 5 + .../api/mediator/MediatorProducer.kt | 7 +- .../api/mediator/MessageBusConsumer.kt | 4 + .../api/mediator/MessageBusProducer.kt | 4 +- .../messaging/api/mediator/MessageRouter.kt | 23 ++- .../api/mediator/RoutingDestination.kt | 14 ++ .../mediator/config/EventMediatorConfig.kt | 22 ++- .../config/EventMediatorConfigBuilder.kt | 14 +- .../factory/MediatorProducerFinder.kt | 15 ++ .../mediator/factory/MessageRouterFactory.kt | 32 ++++ ....kt => MultiSourceEventMediatorFactory.kt} | 6 +- .../api/mediator/statemanager/State.kt | 7 +- 20 files changed, 622 insertions(+), 144 deletions(-) create mode 100644 libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MediatorComponentFactory.kt create mode 100644 libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MediatorStateManager.kt create mode 100644 libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MediatorTaskManager.kt create mode 100644 libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/ProducerTask.kt rename libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/{EventMediatorFactoryImpl.kt => MultiSourceEventMediatorFactoryImpl.kt} (89%) create mode 100644 libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/RoutingDestination.kt create mode 100644 libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MediatorProducerFinder.kt create mode 100644 libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MessageRouterFactory.kt rename libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/{EventMediatorFactory.kt => MultiSourceEventMediatorFactory.kt} (77%) diff --git a/components/flow/flow-service/src/main/kotlin/net/corda/flow/service/FlowExecutorMediatorImpl.kt b/components/flow/flow-service/src/main/kotlin/net/corda/flow/service/FlowExecutorMediatorImpl.kt index bb51e400b92..2d2c9add9d2 100644 --- a/components/flow/flow-service/src/main/kotlin/net/corda/flow/service/FlowExecutorMediatorImpl.kt +++ b/components/flow/flow-service/src/main/kotlin/net/corda/flow/service/FlowExecutorMediatorImpl.kt @@ -2,6 +2,7 @@ package net.corda.flow.service import com.typesafe.config.ConfigValueFactory import net.corda.data.flow.event.FlowEvent +import net.corda.data.flow.event.mapper.FlowMapperEvent import net.corda.data.flow.state.checkpoint.Checkpoint import net.corda.flow.pipeline.factory.FlowEventProcessorFactory import net.corda.libs.configuration.SmartConfig @@ -17,12 +18,15 @@ import net.corda.messagebus.api.consumer.builder.CordaConsumerBuilder import net.corda.messagebus.api.producer.builder.CordaProducerBuilder import net.corda.messaging.api.mediator.MessageRouter import net.corda.messaging.api.mediator.MultiSourceEventMediator +import net.corda.messaging.api.mediator.RoutingDestination.Companion.routeTo import net.corda.messaging.api.mediator.config.EventMediatorConfigBuilder -import net.corda.messaging.api.mediator.factory.EventMediatorFactory import net.corda.messaging.api.mediator.factory.MessageBusConsumerFactory import net.corda.messaging.api.mediator.factory.MessageBusProducerFactory +import net.corda.messaging.api.mediator.factory.MessageRouterFactory +import net.corda.messaging.api.mediator.factory.MultiSourceEventMediatorFactory import net.corda.messaging.api.processor.StateAndEventProcessor import net.corda.schema.Schemas.Flow.FLOW_EVENT_TOPIC +import net.corda.schema.Schemas.Flow.FLOW_MAPPER_EVENT_TOPIC import net.corda.schema.configuration.ConfigKeys.FLOW_CONFIG import net.corda.schema.configuration.ConfigKeys.MESSAGING_CONFIG import net.corda.schema.configuration.MessagingConfig.MAX_ALLOWED_MSG_SIZE @@ -37,7 +41,7 @@ import org.slf4j.LoggerFactory class FlowExecutorMediatorImpl ( coordinatorFactory: LifecycleCoordinatorFactory, private val flowEventProcessorFactory: FlowEventProcessorFactory, - private val eventMediatorFactory: EventMediatorFactory, + private val eventMediatorFactory: MultiSourceEventMediatorFactory, private val cordaConsumerBuilder: CordaConsumerBuilder, private val cordaProducerBuilder: CordaProducerBuilder, private val toMessagingConfig: (Map) -> SmartConfig @@ -49,8 +53,8 @@ class FlowExecutorMediatorImpl ( coordinatorFactory: LifecycleCoordinatorFactory, @Reference(service = FlowEventProcessorFactory::class) flowEventProcessorFactory: FlowEventProcessorFactory, - @Reference(service = EventMediatorFactory::class) - eventMediatorFactory: EventMediatorFactory, + @Reference(service = MultiSourceEventMediatorFactory::class) + eventMediatorFactory: MultiSourceEventMediatorFactory, @Reference(service = CordaConsumerBuilder::class) cordaConsumerBuilder: CordaConsumerBuilder, @Reference(service = CordaProducerBuilder::class) @@ -67,6 +71,7 @@ class FlowExecutorMediatorImpl ( companion object { private val log = LoggerFactory.getLogger(this::class.java.enclosingClass) private const val CONSUMER_GROUP = "FlowEventConsumer" + private const val MESSAGE_BUS_PRODUCER = "MessageBusProducer" } private val coordinator = coordinatorFactory.createCoordinator { event, _ -> eventHandler(event) } @@ -82,10 +87,11 @@ class FlowExecutorMediatorImpl ( subscriptionRegistrationHandle?.close() multiSourceEventMediator?.close() - multiSourceEventMediator = createEventMediator( - messagingConfig, - flowEventProcessorFactory.create(updatedConfigs), - eventMediatorFactory, + multiSourceEventMediator = eventMediatorFactory.create( + createEventMediatorConfig( + messagingConfig, + flowEventProcessorFactory.create(updatedConfigs), + ) ) subscriptionRegistrationHandle = coordinator.followStatusChangesByName( @@ -143,28 +149,31 @@ class FlowExecutorMediatorImpl ( } } - private fun createEventMediator( + private fun createEventMediatorConfig( messagingConfig: SmartConfig, messageProcessor: StateAndEventProcessor, - eventMediatorFactory: EventMediatorFactory - ): MultiSourceEventMediator { - - val eventMediatorConfig = EventMediatorConfigBuilder() + ) = EventMediatorConfigBuilder() .name("FlowEventMediator") .messagingConfig(messagingConfig) .consumerFactories( MessageBusConsumerFactory(FLOW_EVENT_TOPIC, CONSUMER_GROUP, messagingConfig, cordaConsumerBuilder), ) .producerFactories( - MessageBusProducerFactory("messageBusProducer", messagingConfig, cordaProducerBuilder), - //RpcProducerFactory("cryptoWorkerProducer", messagingConfig, cordaRpcBuilder), + MessageBusProducerFactory(MESSAGE_BUS_PRODUCER, messagingConfig, cordaProducerBuilder), + //RpcProducerFactory(CRYPTO_RPC_PRODUCER, messagingConfig, cordaRpcBuilder), ) .messageProcessor(messageProcessor) - .messageRouter(MessageRouter()) + .messageRouterFactory(createMessageRouterFactory()) .build() - return eventMediatorFactory.createMultiSourceEventMediator( - eventMediatorConfig - ) + private fun createMessageRouterFactory() = MessageRouterFactory { producerFinder -> + val messageBusProducer = producerFinder.find(MESSAGE_BUS_PRODUCER) + + MessageRouter { message -> + when (message.body) { + is FlowMapperEvent -> routeTo(messageBusProducer, FLOW_MAPPER_EVENT_TOPIC) + else -> throw IllegalStateException("No route defined for message $message") + } + } } } diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MediatorComponentFactory.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MediatorComponentFactory.kt new file mode 100644 index 00000000000..d31061ef553 --- /dev/null +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MediatorComponentFactory.kt @@ -0,0 +1,80 @@ +package net.corda.messaging.mediator + +import net.corda.messaging.api.mediator.MediatorConsumer +import net.corda.messaging.api.mediator.MediatorProducer +import net.corda.messaging.api.mediator.MessageRouter +import net.corda.messaging.api.mediator.config.MediatorConsumerConfig +import net.corda.messaging.api.mediator.config.MediatorProducerConfig +import net.corda.messaging.api.mediator.factory.MediatorConsumerFactory +import net.corda.messaging.api.mediator.factory.MediatorProducerFactory +import net.corda.messaging.api.mediator.factory.MessageRouterFactory +import net.corda.messaging.api.processor.StateAndEventProcessor + +/** + * Factory for creating various components used by [MultiSourceEventMediatorImpl] + */ +internal class MediatorComponentFactory( + private val messageProcessor: StateAndEventProcessor, + private val consumerFactories: Collection, + private val producerFactories: Collection, + private val messageRouterFactory: MessageRouterFactory, + ) { + + /** + * Creates message consumers. + * + * @param onSerializationError Function for handling serialization errors. + * @return List of created [MediatorConsumer]s. + */ + fun createConsumers( + onSerializationError: (ByteArray) -> Unit + ): List> { + check (consumerFactories.isNotEmpty()) { + "None consumer factory set in configuration" + } + return consumerFactories.map { consumerFactory -> + consumerFactory.create( + MediatorConsumerConfig( + messageProcessor.keyClass, + messageProcessor.eventValueClass, + onSerializationError + ) + ) + } + } + + /** + * Creates message producers. + * + * @param onSerializationError Function for handling serialization errors. + * @return List of created [MediatorProducer]s. + */ + fun createProducers( + onSerializationError: (ByteArray) -> Unit + ): List { + check (producerFactories.isNotEmpty()) { + "None producer factory set in configuration" + } + return producerFactories.map { producerFactory -> + producerFactory.create( + MediatorProducerConfig(onSerializationError) + ) + } + } + + /** + * Creates message router. + * + * @param producers Collection of [MediatorProducer]s. + * @return Message router. + */ + fun createRouter( + producers: Collection + ): MessageRouter { + val producersByName = producers.associateBy { it.name } + return messageRouterFactory.create { name -> + producersByName[name] + ?: throw IllegalStateException("Producer with name \"$name\" not found") + } + } +} \ No newline at end of file diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MediatorStateManager.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MediatorStateManager.kt new file mode 100644 index 00000000000..0bfc5411a33 --- /dev/null +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MediatorStateManager.kt @@ -0,0 +1,77 @@ +package net.corda.messaging.mediator + +import net.corda.avro.serialization.CordaAvroDeserializer +import net.corda.avro.serialization.CordaAvroSerializer +import net.corda.messaging.api.mediator.statemanager.Metadata +import net.corda.messaging.api.mediator.statemanager.State +import net.corda.messaging.api.mediator.statemanager.StateManager + +/** + * Helper for working with [StateManager], used by [MultiSourceEventMediatorImpl]. + */ +class MediatorStateManager( + private val stateManager: StateManager, + private val serializer: CordaAvroSerializer, + private val stateDeserializer: CordaAvroDeserializer, +) { + + /** + * Creates an updated [State] or a new one if there was no previous version. + * + * @param key Event's key. + * @param persistedState State bieing updated. + * @param newValue Updated state value. + */ + fun createOrUpdateState( + key: String, + persistedState: State?, + newValue: S?, + ) = serialize(newValue)?.let { serializedValue -> + State( + key, + serializedValue, + persistedState?.version ?: State.INITIAL_VERSION, + persistedState?.metadata ?: Metadata() + ) + } + + /** + * Persists states of [ProcessorTask] results. + * + * @param processorTaskResults [ProcessorTask] results with updated states. + * @return The latest states in case persistence failed due to conflict (state being updated by another process in + * the meanwhile). + */ + fun persistStates(processorTaskResults: Collection>): Map { + val states = processorTaskResults.mapNotNull { result -> + result.updatedState + } + val (newStates, existingStates) = states.partition { it.version == State.INITIAL_VERSION } + val invalidStates = mutableMapOf() + if (newStates.isNotEmpty()) { + invalidStates.putAll(stateManager.update(newStates)) + } + if (existingStates.isNotEmpty()) { + invalidStates.putAll(stateManager.update(existingStates)) + } + return invalidStates + } + + /** + * Serializes state value. + * + * @param value State value. + * @return Serialized state value. + */ + private fun serialize(value: S?) = + value?.let { serializer.serialize(it) } + + /** + * Deserializes state value. + * + * @param state State. + * @return Deserialized state value. + */ + fun deserializeValue(state: State?) = + state?.value?.let { stateDeserializer.deserialize(it) } +} \ No newline at end of file diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MediatorTaskManager.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MediatorTaskManager.kt new file mode 100644 index 00000000000..85db498a58d --- /dev/null +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MediatorTaskManager.kt @@ -0,0 +1,153 @@ +package net.corda.messaging.mediator + +import net.corda.messagebus.api.consumer.CordaConsumerRecord +import net.corda.messaging.api.mediator.Message +import net.corda.messaging.api.mediator.MessageRouter +import net.corda.messaging.api.mediator.statemanager.State +import net.corda.messaging.api.mediator.statemanager.StateManager +import net.corda.messaging.api.mediator.taskmanager.TaskManager +import net.corda.messaging.api.mediator.taskmanager.TaskType +import net.corda.messaging.api.processor.StateAndEventProcessor +import java.time.Instant + +/** + * Helper that creates and executes various tasks used by [MultiSourceEventMediatorImpl]. + */ +internal class MediatorTaskManager( + private val taskManager: TaskManager, + private val stateManager: MediatorStateManager, +) { + + /** + * Creates [ProcessorTask]s for given events and states. + * + * @param messageGroups Map of messages keys and related events. + * @param persistedStates Mpa of message keys and related states. + * @param messageProcessor State and event processor. + * @return Created [ProcessorTask]s. + */ + fun createMsgProcessorTasks( + messageGroups: Map>>, + persistedStates: Map, + messageProcessor:StateAndEventProcessor, + ): List> { + return messageGroups.map { msgGroup -> + val key = msgGroup.key.toString() + val events = msgGroup.value.map { it } + ProcessorTask( + key, + persistedStates[key], + events, + messageProcessor, + stateManager, + ) + } + } + + /** + * Creates [ProcessorTask]s from [ProducerTask.Result]s that have reply message set. Reply messages are + * grouped by message keys. The latest updated state from related [ProcessorTask] is used as the input state to + * state and event processor. + * + * @param producerResults List of results of [ProducerTask]s. + * @return Created [ProcessorTask]s. + */ + fun createMsgProcessorTasks( + producerResults: List>, + ): List> { + return producerResults.filter { it.hasReply() } + .groupBy { it.producerTask.processorTask.persistedState!!.key } + .map { (_, producerTaskResults) -> + val messageGroup = producerTaskResults.map { it.toCordaConsumerRecord() } + producerTaskResults.first().producerTask.processorTask.copy(events = messageGroup) + } + } + + /** + * Creates [ProcessorTask]s from [ProcessorTask.Result]s of [ProcessorTask] that failed to update state via + * [StateManager] due to conflicts. + * + * @param invalidResults [ProcessorTask.Result]s of [ProcessorTask] that failed to update state via + * [StateManager] due to conflicts. + * @param persistedStates The latest states from [StateManager]. + * @return Created [ProcessorTask]s. + */ + fun createMsgProcessorTasks( + invalidResults: List>, + persistedStates: Map, + ): List> { + return invalidResults.map { + it.processorTask.copy(persistedState = persistedStates[it.processorTask.key]!!) + } + } + + /** + * Executes given [ProcessorTask]s using [TaskManager] and waits for all to finish. + * + * @param processorTasks Tasks to execute. + * @return Result of task executions. + */ + fun executeProcessorTasks( + processorTasks: Collection> + ): List> { + return processorTasks.map { processorTask -> + taskManager.execute(TaskType.SHORT_RUNNING, processorTask::call) + }.map { + it.join() + } + } + + /** + * Creates [ProducerTask]s for given results of [ProcessorTask]s. Given [MessageRouter] is used to select producer + * for specific message. + * + * @param processorTaskResults Results of [ProducerTask]s. + * @param messageRouter Message router. + * @return Created [ProducerTask]s. + */ + fun createProducerTasks( + processorTaskResults: List>, + messageRouter: MessageRouter, + ): List> { + return processorTaskResults.map { result -> + result.outputEvents.map { event -> + val message = Message(event.value!!, emptyMap()) + ProducerTask( + message, + messageRouter, + result.processorTask + ) + } + }.flatten() + } + + /** + * Executes given [ProducerTask]s using [TaskManager] and waits for all to finish. + * + * @param producerTasks Tasks to execute. + * @return Result of task executions. + */ + fun executeProducerTasks( + producerTasks: Collection> + ): List> { + return producerTasks.map { producerTask -> + taskManager.execute(TaskType.SHORT_RUNNING, producerTask::call) + }.map { + it.join() + } + } + + /** + * Converts [ProducerTask.Result] to [CordaConsumerRecord]. + */ + private fun ProducerTask.Result.toCordaConsumerRecord() = + @Suppress("UNCHECKED_CAST") + (CordaConsumerRecord( + "", + -1, + -1, + producerTask.processorTask.events.first().key, + replyMessage!!.body as E, + Instant.now().toEpochMilli() + )) +} \ No newline at end of file diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt index f89e49115c5..a21b4f0d8cb 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt @@ -10,10 +10,9 @@ import net.corda.messaging.api.exception.CordaMessageAPIFatalException import net.corda.messaging.api.exception.CordaMessageAPIIntermittentException import net.corda.messaging.api.mediator.MediatorConsumer import net.corda.messaging.api.mediator.MediatorProducer +import net.corda.messaging.api.mediator.MessageRouter import net.corda.messaging.api.mediator.MultiSourceEventMediator import net.corda.messaging.api.mediator.config.EventMediatorConfig -import net.corda.messaging.api.mediator.config.MediatorConsumerConfig -import net.corda.messaging.api.mediator.config.MediatorProducerConfig import net.corda.messaging.api.mediator.statemanager.StateManager import net.corda.messaging.api.mediator.taskmanager.TaskManager import net.corda.messaging.api.mediator.taskmanager.TaskType @@ -36,7 +35,17 @@ class MultiSourceEventMediatorImpl( private var consumers = listOf>() private var producers = listOf() - private val pollTimeoutInNanos = Duration.ofMillis(100).toNanos() // TODO take from config + private lateinit var messageRouter: MessageRouter + private val mediatorComponentFactory = MediatorComponentFactory( + config.messageProcessor, config.consumerFactories, config.producerFactories, config.messageRouterFactory + ) + private val mediatorStateManager = MediatorStateManager( + stateManager, serializer, stateDeserializer + ) + private val mediatorTaskManager = MediatorTaskManager( + taskManager, mediatorStateManager + ) + private val pollTimeoutInNanos = config.pollTimeout.toNanos() private val uniqueId = UUID.randomUUID().toString() private val lifecycleCoordinatorName = LifecycleCoordinatorName( "MultiSourceEventMediator--${config.name}", uniqueId @@ -55,8 +64,7 @@ class MultiSourceEventMediatorImpl( private fun stop() = Thread.currentThread().interrupt() - private fun stopped() = - Thread.currentThread().isInterrupted + private val stopped get() = Thread.currentThread().isInterrupted /** * This method is for closing the loop/thread externally. From inside the loop use the private [stopConsumeLoop]. @@ -68,17 +76,18 @@ class MultiSourceEventMediatorImpl( private fun run() { var attempts = 0 - while (!stopped()) { + while (!stopped) { attempts++ try { - createConsumers() - createProducers() + consumers = mediatorComponentFactory.createConsumers(::onSerializationError) + producers = mediatorComponentFactory.createProducers(::onSerializationError) + messageRouter = mediatorComponentFactory.createRouter(producers) consumers.forEach{ it.subscribe() } lifecycleCoordinator.updateStatus(LifecycleStatus.UP) - while (!stopped()) { - // TODO processEvents() + while (!stopped) { + processEventsWithRetries() } } catch (ex: Exception) { @@ -111,78 +120,23 @@ class MultiSourceEventMediatorImpl( TODO() } - private fun createConsumers() { - check (config.consumerFactories.isNotEmpty()) { - "None consumer factory set in configuration" - } - consumers = config.consumerFactories.map { consumerFactory -> - consumerFactory.create( - MediatorConsumerConfig( - config.messageProcessor.keyClass, - config.messageProcessor.eventValueClass, - ::onSerializationError - ) - ) - } - } - - private fun createProducers() { - check (config.producerFactories.isNotEmpty()) { - "None producer factory set in configuration" - } - producers = config.producerFactories.map { producerFactory -> - producerFactory.create( - MediatorProducerConfig( - ::onSerializationError - ) - ) - } - } - private fun closeConsumersAndProducers() { consumers.forEach { it.close() } producers.forEach { it.close() } } - private fun processEvents(isTaskStopped: Boolean) { + private fun processEventsWithRetries() { var attempts = 0 - while (!isTaskStopped) { + var keepProcessing = true + while (keepProcessing && !stopped) { try { - log.debug { "Polling and processing events" } - val messages = poll(pollTimeoutInNanos) - val msgGroups = messages.groupBy { it.key } -// val states = stateManager.get( -// config.messageProcessor.stateValueClass, msgGroups.keys.mapTo(HashSet()) { it.toString() } -// ) - val processorTasks = msgGroups.map { msgGroup -> - val key = msgGroup.key.toString() - val events = msgGroup.value.map { it } - ProcessorTask( - key, -// states[key], - events, - config.messageProcessor, - stateManager, - serializer, - stateDeserializer, - ) - } - - processorTasks.map { processorTask -> - taskManager.execute(TaskType.SHORT_RUNNING, processorTask::run) - .thenApply { //outputEvents -> - // TODO - } - } -// val commitResults = consumers.map { consumer -> -// consumer.commitAsync() -// } - + processEvents() + keepProcessing = false } catch (ex: Exception) { when (ex) { is CordaMessageAPIIntermittentException -> { attempts++ - // TODO handleProcessEventRetries(attempts, ex) + handleProcessEventRetries(attempts, ex) } else -> { @@ -196,6 +150,33 @@ class MultiSourceEventMediatorImpl( } } + private fun processEvents() { + log.debug { "Polling and processing events" } + val messages = poll(pollTimeoutInNanos) + if (messages.isNotEmpty()) { + val msgGroups = messages.groupBy { it.key } + val persistedStates = stateManager.get(msgGroups.keys.map { it.toString() }) + var msgProcessorTasks = mediatorTaskManager.createMsgProcessorTasks( + msgGroups, persistedStates, config.messageProcessor + ) + do { + val processingResults = mediatorTaskManager.executeProcessorTasks(msgProcessorTasks) + val conflictingStates = mediatorStateManager.persistStates(processingResults) + val (validResults, invalidResults) = processingResults.partition { + !conflictingStates.contains(it.key) + } + val producerTasks = mediatorTaskManager.createProducerTasks(validResults, messageRouter) + val producerResults = mediatorTaskManager.executeProducerTasks(producerTasks) + msgProcessorTasks = + mediatorTaskManager.createMsgProcessorTasks(producerResults) + mediatorTaskManager.createMsgProcessorTasks( + invalidResults, + conflictingStates + ) + } while (msgProcessorTasks.isNotEmpty()) + } + commitOffsets() + } + private fun poll(pollTimeoutInNanos: Long): List> { val maxEndTime = System.nanoTime() + pollTimeoutInNanos return consumers.map { consumer -> @@ -204,6 +185,40 @@ class MultiSourceEventMediatorImpl( }.flatten() } + private fun commitOffsets() { + consumers.map { consumer -> + consumer.commitAsync() + }.map { + it.join() + } + } + + /** + * Handle retries for event processing. + * Reset [MediatorConsumer]s position and retry poll and process of event records + * Retry a max of [EventMediatorConfig.processorRetries] times. + * If [EventMediatorConfig.processorRetries] is exceeded then throw a [CordaMessageAPIIntermittentException] + */ + private fun handleProcessEventRetries( + attempts: Int, + ex: Exception + ) { + if (attempts <= config.processorRetries) { + log.warn( + "Multi-source event mediator ${config.name} failed to process records, " + + "Retrying poll and process. Attempts: $attempts." + ) + consumers.forEach { it.resetEventOffsetPosition() } + } else { + val message = "Multi-source event mediator ${config.name} failed to process records, " + + "Attempts: $attempts. Max reties exceeded." + log.warn(message, ex) + throw CordaMessageAPIIntermittentException(message, ex) + } + } + + + // private fun generateDeadLetterRecord(event: CordaConsumerRecord, state: S?): Record<*, *> { // val keyBytes = ByteBuffer.wrap(cordaAvroSerializer.serialize(event.key)) // val stateBytes = diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/ProcessorTask.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/ProcessorTask.kt index 0ea2a7273a3..a07ef10ee0b 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/ProcessorTask.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/ProcessorTask.kt @@ -1,47 +1,49 @@ package net.corda.messaging.mediator -import net.corda.avro.serialization.CordaAvroDeserializer -import net.corda.avro.serialization.CordaAvroSerializer import net.corda.messagebus.api.consumer.CordaConsumerRecord -import net.corda.messaging.api.mediator.statemanager.Metadata import net.corda.messaging.api.mediator.statemanager.State -import net.corda.messaging.api.mediator.statemanager.StateManager import net.corda.messaging.api.processor.StateAndEventProcessor import net.corda.messaging.api.records.Record import net.corda.messaging.utils.toRecord -import net.corda.v5.base.exceptions.CordaRuntimeException +import java.util.concurrent.Callable +/** + * [ProcessorTask] uses [StateAndEventProcessor] to process input events (that have the same key) and related states. + * Events are processed sequentially and updated state of the current event is used as the input state of the next + * event. Result of processing are output events and final updated state. + */ @Suppress("LongParameterList") -class ProcessorTask( - private val key: String, - private val events: Collection>, +data class ProcessorTask( + val key: String, + val persistedState: State?, + val events: Collection>, private val processor: StateAndEventProcessor, - private val stateManager: StateManager, - private val serializer: CordaAvroSerializer, - private val stateDeserializer: CordaAvroDeserializer, -) { + private val mediatorStateManager: MediatorStateManager, +): Callable> { - var responseEvents = emptyList>() - private set - fun run() { - val persistedState = stateManager.get(setOf(key))[key] + class Result( + val processorTask: ProcessorTask, + val outputEvents: List>, + val updatedState: State?, + ) { + val key get() = processorTask.key + } + + override fun call(): Result { + var stateValue = mediatorStateManager.deserializeValue(persistedState) - var updatedState = persistedState?.value?.let { stateDeserializer.deserialize(it) } - responseEvents = events.map { event -> - val response = processor.onNext(updatedState, event.toRecord()) - response.updatedState?.let { updatedState = it } + val outputEvents = events.map { event -> + val response = processor.onNext(stateValue, event.toRecord()) + response.updatedState?.let { stateValue = it } response.responseEvents }.flatten() - val serializedState = serializer.serialize(updatedState!!) - ?: throw CordaRuntimeException("Cannot serialize updated state: $updatedState") - - val newState = State( + val updatedState = mediatorStateManager.createOrUpdateState( key, - serializedState, - (persistedState?.version ?: -1) + 1, - persistedState?.metadata ?: Metadata() + persistedState, + stateValue ) - stateManager.update(setOf(newState)) + + return Result(this, outputEvents, updatedState) } -} \ No newline at end of file +} diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/ProducerTask.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/ProducerTask.kt new file mode 100644 index 00000000000..07d8ea17c69 --- /dev/null +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/ProducerTask.kt @@ -0,0 +1,29 @@ +package net.corda.messaging.mediator + +import net.corda.messaging.api.mediator.MediatorProducer +import net.corda.messaging.api.mediator.Message +import net.corda.messaging.api.mediator.MessageRouter +import java.util.concurrent.Callable + +/** + * [ProducerTask] sends a [Message] to [MediatorProducer] selected by [MessageRouter]. + */ +class ProducerTask( + private val message: Message, + private val messageRouter: MessageRouter, + val processorTask: ProcessorTask, +): Callable> { + + class Result( + val producerTask: ProducerTask, + val replyMessage: Message?, + ) { + fun hasReply() = replyMessage != null + } + + override fun call(): Result { + val destination = messageRouter.getDestination(message) + val reply = with(destination) { producer.send(message, address) } + return Result(this, reply.reply) + } +} \ No newline at end of file diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/EventMediatorFactoryImpl.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/MultiSourceEventMediatorFactoryImpl.kt similarity index 89% rename from libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/EventMediatorFactoryImpl.kt rename to libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/MultiSourceEventMediatorFactoryImpl.kt index 4cedbea7644..41ef1f2ba08 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/EventMediatorFactoryImpl.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/MultiSourceEventMediatorFactoryImpl.kt @@ -4,7 +4,7 @@ import net.corda.avro.serialization.CordaAvroSerializationFactory import net.corda.lifecycle.LifecycleCoordinatorFactory import net.corda.messaging.api.mediator.MultiSourceEventMediator import net.corda.messaging.api.mediator.config.EventMediatorConfig -import net.corda.messaging.api.mediator.factory.EventMediatorFactory +import net.corda.messaging.api.mediator.factory.MultiSourceEventMediatorFactory import net.corda.messaging.api.mediator.statemanager.StateManager import net.corda.messaging.api.mediator.taskmanager.TaskManager import net.corda.messaging.mediator.MultiSourceEventMediatorImpl @@ -13,8 +13,8 @@ import org.osgi.service.component.annotations.Component import org.osgi.service.component.annotations.Reference @Suppress("LongParameterList") -@Component(service = [EventMediatorFactory::class]) -class EventMediatorFactoryImpl @Activate constructor( +@Component(service = [MultiSourceEventMediatorFactory::class]) +class MultiSourceEventMediatorFactoryImpl @Activate constructor( @Reference(service = CordaAvroSerializationFactory::class) private val cordaAvroSerializationFactory: CordaAvroSerializationFactory, @Reference(service = TaskManager::class) @@ -23,9 +23,9 @@ class EventMediatorFactoryImpl @Activate constructor( private val stateManager: StateManager, @Reference(service = LifecycleCoordinatorFactory::class) private val lifecycleCoordinatorFactory: LifecycleCoordinatorFactory, -): EventMediatorFactory { +): MultiSourceEventMediatorFactory { - override fun createMultiSourceEventMediator( + override fun create( eventMediatorConfig: EventMediatorConfig, ): MultiSourceEventMediator { val serializer = cordaAvroSerializationFactory.createAvroSerializer { } diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MediatorConsumer.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MediatorConsumer.kt index fff1b8793ce..0171cfc4b87 100644 --- a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MediatorConsumer.kt +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MediatorConsumer.kt @@ -28,4 +28,9 @@ interface MediatorConsumer : AutoCloseable { * @return [CompletableFuture] with committed offsets. */ fun commitAsync(): CompletableFuture> + + /** + * Resets consumer's offsets to the last committed positions. + */ + fun resetEventOffsetPosition() } diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MediatorProducer.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MediatorProducer.kt index 71c60ed7092..d398105b53d 100644 --- a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MediatorProducer.kt +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MediatorProducer.kt @@ -5,6 +5,11 @@ package net.corda.messaging.api.mediator */ interface MediatorProducer: AutoCloseable { + /** + * Producer's name. Name is used for routing messages to producers, so it should be unique. + */ + val name: String + /** * Determines whether producer supports request-reply messaging pattern. */ @@ -18,5 +23,5 @@ interface MediatorProducer: AutoCloseable { * @returns ProducerReply Holds producer's reply if producer supports request-reply messaging pattern * (@see [isRequestReply]). */ - fun send(message: Message): ProducerReply + fun send(message: Message, address: String): ProducerReply } \ No newline at end of file diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MessageBusConsumer.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MessageBusConsumer.kt index 7fca09c8b02..40cce5c2fa5 100644 --- a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MessageBusConsumer.kt +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MessageBusConsumer.kt @@ -3,6 +3,7 @@ package net.corda.messaging.api.mediator import net.corda.messagebus.api.CordaTopicPartition import net.corda.messagebus.api.consumer.CordaConsumer import net.corda.messagebus.api.consumer.CordaConsumerRecord +import net.corda.messagebus.api.consumer.CordaOffsetResetStrategy import java.time.Duration import java.util.concurrent.CompletableFuture @@ -32,6 +33,9 @@ class MessageBusConsumer( return result } + override fun resetEventOffsetPosition() = + consumer.resetToLastCommittedPositions(CordaOffsetResetStrategy.EARLIEST) + override fun close() = consumer.close() } \ No newline at end of file diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MessageBusProducer.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MessageBusProducer.kt index 3b0e21d50c6..3b20892dc2c 100644 --- a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MessageBusProducer.kt +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MessageBusProducer.kt @@ -6,11 +6,11 @@ import net.corda.messagebus.api.producer.CordaProducer * Message bus producer that sends messages to message bus topics. */ class MessageBusProducer( - private val name: String, + override val name: String, private val producer: CordaProducer, ): MediatorProducer { - override fun send(message: Message): ProducerReply { + override fun send(message: Message, address: String): ProducerReply { TODO("Not implemented yet") } diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MessageRouter.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MessageRouter.kt index f98a73b9f6e..48ed65fff3d 100644 --- a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MessageRouter.kt +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MessageRouter.kt @@ -1,6 +1,25 @@ package net.corda.messaging.api.mediator /** - * Multi-source event mediator's message router that routes messages generated by stae and event processor to producers. + * Multi-source event mediator's message router that routes messages generated by state and event processor to producers. + * Example: + * + * ``` + * MessageRouter { message -> + * when (message.body) { + * is FlowEvent -> RoutingDestination(messageBusProducer, "flow.event.topic") + * else -> throw IllegalStateException("No route defined for message $message") + * } + * } + * ``` */ -class MessageRouter \ No newline at end of file +fun interface MessageRouter { + + /** + * Provides [RoutingDestination] for given [Message]. + * + * @param message Message. + * @return Destination for given message. + */ + fun getDestination(message: Message): RoutingDestination +} diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/RoutingDestination.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/RoutingDestination.kt new file mode 100644 index 00000000000..d3203ddf479 --- /dev/null +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/RoutingDestination.kt @@ -0,0 +1,14 @@ +package net.corda.messaging.api.mediator + +/** + * Routing destination encapsulate [MediatorProducer] and related data needed to send a [Message]. + */ +data class RoutingDestination( + val producer: MediatorProducer, + val address: String, +) { + companion object { + fun routeTo(producer: MediatorProducer, address: String) = + RoutingDestination(producer, address) + } +} diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/config/EventMediatorConfig.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/config/EventMediatorConfig.kt index 5b99f0f3cc0..97c75d8cc6e 100644 --- a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/config/EventMediatorConfig.kt +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/config/EventMediatorConfig.kt @@ -1,11 +1,13 @@ package net.corda.messaging.api.mediator.config import net.corda.libs.configuration.SmartConfig -import net.corda.messaging.api.mediator.MessageRouter import net.corda.messaging.api.mediator.MultiSourceEventMediator import net.corda.messaging.api.mediator.factory.MediatorConsumerFactory import net.corda.messaging.api.mediator.factory.MediatorProducerFactory +import net.corda.messaging.api.mediator.factory.MessageRouterFactory import net.corda.messaging.api.processor.StateAndEventProcessor +import net.corda.schema.configuration.MessagingConfig +import java.time.Duration /** * Class to store the required params to create a [MultiSourceEventMediator]. @@ -15,7 +17,7 @@ import net.corda.messaging.api.processor.StateAndEventProcessor * @property consumerFactories Factories for creating message consumers. * @property producerFactories Factories for creating message producers. * @property messageProcessor State and event processor. - * @property messageRouter Message router that routes output messages of the state and event processor to producers. + * @property messageRouterFactory Message router factory. */ data class EventMediatorConfig( val name: String, @@ -23,5 +25,17 @@ data class EventMediatorConfig( val consumerFactories: Collection, val producerFactories: Collection, val messageProcessor : StateAndEventProcessor, - val messageRouter: MessageRouter, -) \ No newline at end of file + val messageRouterFactory: MessageRouterFactory, +) { + /** + * Timeout for polling consumers. + */ + val pollTimeout: Duration + get() = Duration.ofMillis(messagingConfig.getLong(MessagingConfig.Subscription.POLL_TIMEOUT)) + + /** + * Maximal number of event processing retries. + */ + val processorRetries: Int + get() = messagingConfig.getInt(MessagingConfig.Subscription.PROCESSOR_RETRIES) +} \ No newline at end of file diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/config/EventMediatorConfigBuilder.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/config/EventMediatorConfigBuilder.kt index 0ceb4f373c1..9b9011f6727 100644 --- a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/config/EventMediatorConfigBuilder.kt +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/config/EventMediatorConfigBuilder.kt @@ -1,10 +1,10 @@ package net.corda.messaging.api.mediator.config import net.corda.libs.configuration.SmartConfig -import net.corda.messaging.api.mediator.MessageRouter import net.corda.messaging.api.mediator.MultiSourceEventMediator import net.corda.messaging.api.mediator.factory.MediatorConsumerFactory import net.corda.messaging.api.mediator.factory.MediatorProducerFactory +import net.corda.messaging.api.mediator.factory.MessageRouterFactory import net.corda.messaging.api.processor.StateAndEventProcessor /** @@ -17,7 +17,7 @@ class EventMediatorConfigBuilder { private var consumerFactories = emptyArray() private var producerFactories = emptyArray() private var messageProcessor : StateAndEventProcessor? = null - private var messageRouter: MessageRouter? = null + private var messageRouterFactory: MessageRouterFactory? = null /** Sets name for [MultiSourceEventMediator]. */ fun name(name: String) = @@ -40,24 +40,24 @@ class EventMediatorConfigBuilder { apply { this.messageProcessor = messageProcessor } /** Sets message router. */ - fun messageRouter(messageRouter: MessageRouter) = - apply { this.messageRouter = messageRouter } + fun messageRouterFactory(messageRouterFactory: MessageRouterFactory) = + apply { this.messageRouterFactory = messageRouterFactory } /** Builds [EventMediatorConfig]. */ fun build(): EventMediatorConfig { check(name != null) { "Name not set" } - check(messagingConfig != null) { "Messaging config not set" } + check(messagingConfig != null) { "Messaging configuration not set" } check(consumerFactories.isNotEmpty()) { "At least on consumer factory has to be set" } check(producerFactories.isNotEmpty()) { "At least on producer factory has to be set" } check(messageProcessor != null) { "Message processor not set" } - check(messageRouter != null) { "Message router not set" } + check(messageRouterFactory != null) { "Message router factory not set" } return EventMediatorConfig( name!!, messagingConfig!!, consumerFactories.asList(), producerFactories.asList(), messageProcessor!!, - messageRouter!! + messageRouterFactory!! ) } } \ No newline at end of file diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MediatorProducerFinder.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MediatorProducerFinder.kt new file mode 100644 index 00000000000..7e787cdb5ec --- /dev/null +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MediatorProducerFinder.kt @@ -0,0 +1,15 @@ +package net.corda.messaging.api.mediator.factory + +import net.corda.messaging.api.mediator.MediatorProducer + +/** + * Mediator producer finder is used to access [MediatorProducer] by its name. + */ +fun interface MediatorProducerFinder { + + /** + * @param name Producer's name. + * @return Producer found by given name. + */ + fun find(name: String): MediatorProducer +} diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MessageRouterFactory.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MessageRouterFactory.kt new file mode 100644 index 00000000000..5ec09ca15f7 --- /dev/null +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MessageRouterFactory.kt @@ -0,0 +1,32 @@ +package net.corda.messaging.api.mediator.factory + +import net.corda.messaging.api.mediator.MediatorProducer +import net.corda.messaging.api.mediator.MessageRouter + +/** + * Factory for creating [MessageRouter]s. + */ +fun interface MessageRouterFactory { + + /** + * Creates a new instance of [MessageRouter]. Provided [MediatorProducerFinder] is used to find [MediatorProducer]s + * by their names. Example: + * + * ``` + * MessageRouterFactory { producerFinder -> + * val messageBusProducer = producerFinder.find("MessageBusProducer") + * + * MessageRouter { message -> + * when (message.body) { + * is FlowEvent -> RoutingDestination(messageBusProducer, "flow.event.topic") + * else -> throw IllegalStateException("No route defined for message $message") + * } + * } + * } + * ``` + * + * @param producerFinder Producer finder + * @return created message router. + */ + fun create(producerFinder: MediatorProducerFinder): MessageRouter +} diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/EventMediatorFactory.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MultiSourceEventMediatorFactory.kt similarity index 77% rename from libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/EventMediatorFactory.kt rename to libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MultiSourceEventMediatorFactory.kt index 7b1150d82a0..92dc3e6f172 100644 --- a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/EventMediatorFactory.kt +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MultiSourceEventMediatorFactory.kt @@ -4,9 +4,9 @@ import net.corda.messaging.api.mediator.MultiSourceEventMediator import net.corda.messaging.api.mediator.config.EventMediatorConfig /** - * Factory for creating multi-source event mediator configuration. + * Factory for creating multi-source event mediator. */ -interface EventMediatorFactory { +interface MultiSourceEventMediatorFactory { /** * Creates a multi-source event mediator configuration. @@ -16,7 +16,7 @@ interface EventMediatorFactory { * @param The type of the event. * @param eventMediatorConfig Multi-source event mediator configuration. */ - fun createMultiSourceEventMediator( + fun create( eventMediatorConfig: EventMediatorConfig, ): MultiSourceEventMediator } diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/statemanager/State.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/statemanager/State.kt index ad45d743e9c..c27614e8cea 100644 --- a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/statemanager/State.kt +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/statemanager/State.kt @@ -19,7 +19,7 @@ data class State( /** * Version of the state. */ - val version: Int = -1, + val version: Int = INITIAL_VERSION, /** * Arbitrary Map of primitive types that can be used to store and query data associated with the state. @@ -31,6 +31,11 @@ data class State( */ val modifiedTime: Instant = Instant.now(), ) { + + companion object { + const val INITIAL_VERSION = -1 + } + override fun equals(other: Any?): Boolean { if (this === other) return true if (javaClass != other?.javaClass) return false From b2a7c72dfea89bae8e0518c92e1191b4d992ae22 Mon Sep 17 00:00:00 2001 From: Miljenko Brkic Date: Mon, 18 Sep 2023 15:25:36 +0100 Subject: [PATCH 23/50] CORE-16199 Refactoring --- .../net/corda/messaging}/mediator/MessageBusConsumer.kt | 3 ++- .../net/corda/messaging}/mediator/MessageBusProducer.kt | 5 ++++- .../messaging}/mediator/factory/MessageBusConsumerFactory.kt | 5 +++-- .../messaging}/mediator/factory/MessageBusProducerFactory.kt | 3 ++- 4 files changed, 11 insertions(+), 5 deletions(-) rename libs/messaging/{messaging/src/main/kotlin/net/corda/messaging/api => messaging-impl/src/main/kotlin/net/corda/messaging}/mediator/MessageBusConsumer.kt (93%) rename libs/messaging/{messaging/src/main/kotlin/net/corda/messaging/api => messaging-impl/src/main/kotlin/net/corda/messaging}/mediator/MessageBusProducer.kt (68%) rename libs/messaging/{messaging/src/main/kotlin/net/corda/messaging/api => messaging-impl/src/main/kotlin/net/corda/messaging}/mediator/factory/MessageBusConsumerFactory.kt (95%) rename libs/messaging/{messaging/src/main/kotlin/net/corda/messaging/api => messaging-impl/src/main/kotlin/net/corda/messaging}/mediator/factory/MessageBusProducerFactory.kt (93%) diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MessageBusConsumer.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MessageBusConsumer.kt similarity index 93% rename from libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MessageBusConsumer.kt rename to libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MessageBusConsumer.kt index 40cce5c2fa5..580cb6bcc9d 100644 --- a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MessageBusConsumer.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MessageBusConsumer.kt @@ -1,9 +1,10 @@ -package net.corda.messaging.api.mediator +package net.corda.messaging.mediator import net.corda.messagebus.api.CordaTopicPartition import net.corda.messagebus.api.consumer.CordaConsumer import net.corda.messagebus.api.consumer.CordaConsumerRecord import net.corda.messagebus.api.consumer.CordaOffsetResetStrategy +import net.corda.messaging.api.mediator.MediatorConsumer import java.time.Duration import java.util.concurrent.CompletableFuture diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MessageBusProducer.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MessageBusProducer.kt similarity index 68% rename from libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MessageBusProducer.kt rename to libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MessageBusProducer.kt index 3b20892dc2c..4c53425bce0 100644 --- a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MessageBusProducer.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MessageBusProducer.kt @@ -1,6 +1,9 @@ -package net.corda.messaging.api.mediator +package net.corda.messaging.mediator import net.corda.messagebus.api.producer.CordaProducer +import net.corda.messaging.api.mediator.MediatorProducer +import net.corda.messaging.api.mediator.Message +import net.corda.messaging.api.mediator.ProducerReply /** * Message bus producer that sends messages to message bus topics. diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MessageBusConsumerFactory.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/MessageBusConsumerFactory.kt similarity index 95% rename from libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MessageBusConsumerFactory.kt rename to libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/MessageBusConsumerFactory.kt index c6aff5cbabd..ea2c175ea39 100644 --- a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MessageBusConsumerFactory.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/MessageBusConsumerFactory.kt @@ -1,4 +1,4 @@ -package net.corda.messaging.api.mediator.factory +package net.corda.messaging.mediator.factory import net.corda.libs.configuration.SmartConfig import net.corda.messagebus.api.configuration.ConsumerConfig @@ -6,8 +6,9 @@ import net.corda.messagebus.api.constants.ConsumerRoles import net.corda.messagebus.api.consumer.CordaConsumer import net.corda.messagebus.api.consumer.builder.CordaConsumerBuilder import net.corda.messaging.api.mediator.MediatorConsumer -import net.corda.messaging.api.mediator.MessageBusConsumer import net.corda.messaging.api.mediator.config.MediatorConsumerConfig +import net.corda.messaging.api.mediator.factory.MediatorConsumerFactory +import net.corda.messaging.mediator.MessageBusConsumer import java.util.UUID /** diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MessageBusProducerFactory.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/MessageBusProducerFactory.kt similarity index 93% rename from libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MessageBusProducerFactory.kt rename to libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/MessageBusProducerFactory.kt index 2b258e70544..898e6f84b6f 100644 --- a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MessageBusProducerFactory.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/MessageBusProducerFactory.kt @@ -1,4 +1,4 @@ -package net.corda.messaging.api.mediator.factory +package net.corda.messaging.mediator.factory import net.corda.libs.configuration.SmartConfig import net.corda.messagebus.api.configuration.ProducerConfig @@ -8,6 +8,7 @@ import net.corda.messagebus.api.producer.builder.CordaProducerBuilder import net.corda.messaging.api.mediator.MediatorProducer import net.corda.messaging.api.mediator.MessageBusProducer import net.corda.messaging.api.mediator.config.MediatorProducerConfig +import net.corda.messaging.api.mediator.factory.MediatorProducerFactory import net.corda.schema.configuration.BootConfig import java.util.UUID From 09ed3a570f483bb2854fe10fb44244165f791794 Mon Sep 17 00:00:00 2001 From: Miljenko Brkic Date: Mon, 18 Sep 2023 16:16:16 +0100 Subject: [PATCH 24/50] CORE-16177 Reverted build file for kafka-message-bus-impl --- libs/messaging/kafka-message-bus-impl/build.gradle | 1 - 1 file changed, 1 deletion(-) diff --git a/libs/messaging/kafka-message-bus-impl/build.gradle b/libs/messaging/kafka-message-bus-impl/build.gradle index 075bdf52243..8acab0a36cd 100644 --- a/libs/messaging/kafka-message-bus-impl/build.gradle +++ b/libs/messaging/kafka-message-bus-impl/build.gradle @@ -9,7 +9,6 @@ dependencies { compileOnly 'org.osgi:osgi.core' compileOnly 'org.osgi:org.osgi.service.component.annotations' - implementation project(path: ':libs:kotlin-coroutines', configuration: 'bundle') implementation project(":libs:messaging:messaging") implementation project(":libs:messaging:message-bus") implementation project(":libs:schema-registry:schema-registry") From 153048d31083cde164ec9c8a11f0851aa08a0dd9 Mon Sep 17 00:00:00 2001 From: Miljenko Brkic Date: Mon, 18 Sep 2023 17:09:41 +0100 Subject: [PATCH 25/50] CORE-16199 Refactoring --- components/flow/flow-service/build.gradle | 2 +- .../net/corda/flow/service/FlowExecutorMediatorImpl.kt | 4 ++-- .../net/corda/messaging/mediator/MediatorTaskManager.kt | 4 ++-- .../net/corda/messaging/mediator/MessageBusProducer.kt | 4 ++-- .../kotlin/net/corda/messaging/mediator/ProducerTask.kt | 8 ++++---- .../mediator/factory/MessageBusProducerFactory.kt | 2 +- .../api/mediator/{Message.kt => MediatorMessage.kt} | 2 +- .../net/corda/messaging/api/mediator/MediatorProducer.kt | 2 +- .../net/corda/messaging/api/mediator/MessageRouter.kt | 4 ++-- .../net/corda/messaging/api/mediator/ProducerReply.kt | 2 +- .../corda/messaging/api/mediator/RoutingDestination.kt | 2 +- 11 files changed, 18 insertions(+), 18 deletions(-) rename libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/{Message.kt => MediatorMessage.kt} (89%) diff --git a/components/flow/flow-service/build.gradle b/components/flow/flow-service/build.gradle index d13f5d4188d..a6566a4a577 100644 --- a/components/flow/flow-service/build.gradle +++ b/components/flow/flow-service/build.gradle @@ -34,7 +34,7 @@ dependencies { implementation project(":libs:lifecycle:lifecycle") implementation project(":libs:membership:membership-common") implementation project(":libs:metrics") - implementation project(":libs:messaging:messaging") + implementation project(":libs:messaging:messaging-impl") implementation project(':libs:platform-info') implementation project(":libs:sandbox") implementation project(':libs:serialization:serialization-amqp') diff --git a/components/flow/flow-service/src/main/kotlin/net/corda/flow/service/FlowExecutorMediatorImpl.kt b/components/flow/flow-service/src/main/kotlin/net/corda/flow/service/FlowExecutorMediatorImpl.kt index 2d2c9add9d2..f2bea3f0daf 100644 --- a/components/flow/flow-service/src/main/kotlin/net/corda/flow/service/FlowExecutorMediatorImpl.kt +++ b/components/flow/flow-service/src/main/kotlin/net/corda/flow/service/FlowExecutorMediatorImpl.kt @@ -20,11 +20,11 @@ import net.corda.messaging.api.mediator.MessageRouter import net.corda.messaging.api.mediator.MultiSourceEventMediator import net.corda.messaging.api.mediator.RoutingDestination.Companion.routeTo import net.corda.messaging.api.mediator.config.EventMediatorConfigBuilder -import net.corda.messaging.api.mediator.factory.MessageBusConsumerFactory -import net.corda.messaging.api.mediator.factory.MessageBusProducerFactory import net.corda.messaging.api.mediator.factory.MessageRouterFactory import net.corda.messaging.api.mediator.factory.MultiSourceEventMediatorFactory import net.corda.messaging.api.processor.StateAndEventProcessor +import net.corda.messaging.mediator.factory.MessageBusConsumerFactory +import net.corda.messaging.mediator.factory.MessageBusProducerFactory import net.corda.schema.Schemas.Flow.FLOW_EVENT_TOPIC import net.corda.schema.Schemas.Flow.FLOW_MAPPER_EVENT_TOPIC import net.corda.schema.configuration.ConfigKeys.FLOW_CONFIG diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MediatorTaskManager.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MediatorTaskManager.kt index 85db498a58d..441a1b6b85c 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MediatorTaskManager.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MediatorTaskManager.kt @@ -1,7 +1,7 @@ package net.corda.messaging.mediator import net.corda.messagebus.api.consumer.CordaConsumerRecord -import net.corda.messaging.api.mediator.Message +import net.corda.messaging.api.mediator.MediatorMessage import net.corda.messaging.api.mediator.MessageRouter import net.corda.messaging.api.mediator.statemanager.State import net.corda.messaging.api.mediator.statemanager.StateManager @@ -111,7 +111,7 @@ internal class MediatorTaskManager( ): List> { return processorTaskResults.map { result -> result.outputEvents.map { event -> - val message = Message(event.value!!, emptyMap()) + val message = MediatorMessage(event.value!!, emptyMap()) ProducerTask( message, messageRouter, diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MessageBusProducer.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MessageBusProducer.kt index 4c53425bce0..692f60be892 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MessageBusProducer.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MessageBusProducer.kt @@ -1,8 +1,8 @@ package net.corda.messaging.mediator import net.corda.messagebus.api.producer.CordaProducer +import net.corda.messaging.api.mediator.MediatorMessage import net.corda.messaging.api.mediator.MediatorProducer -import net.corda.messaging.api.mediator.Message import net.corda.messaging.api.mediator.ProducerReply /** @@ -13,7 +13,7 @@ class MessageBusProducer( private val producer: CordaProducer, ): MediatorProducer { - override fun send(message: Message, address: String): ProducerReply { + override fun send(message: MediatorMessage, address: String): ProducerReply { TODO("Not implemented yet") } diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/ProducerTask.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/ProducerTask.kt index 07d8ea17c69..245a7ffd644 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/ProducerTask.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/ProducerTask.kt @@ -1,22 +1,22 @@ package net.corda.messaging.mediator +import net.corda.messaging.api.mediator.MediatorMessage import net.corda.messaging.api.mediator.MediatorProducer -import net.corda.messaging.api.mediator.Message import net.corda.messaging.api.mediator.MessageRouter import java.util.concurrent.Callable /** - * [ProducerTask] sends a [Message] to [MediatorProducer] selected by [MessageRouter]. + * [ProducerTask] sends a [MediatorMessage] to [MediatorProducer] selected by [MessageRouter]. */ class ProducerTask( - private val message: Message, + private val message: MediatorMessage, private val messageRouter: MessageRouter, val processorTask: ProcessorTask, ): Callable> { class Result( val producerTask: ProducerTask, - val replyMessage: Message?, + val replyMessage: MediatorMessage?, ) { fun hasReply() = replyMessage != null } diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/MessageBusProducerFactory.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/MessageBusProducerFactory.kt index 898e6f84b6f..dfc57b6aa1a 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/MessageBusProducerFactory.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/MessageBusProducerFactory.kt @@ -6,9 +6,9 @@ import net.corda.messagebus.api.constants.ProducerRoles import net.corda.messagebus.api.producer.CordaProducer import net.corda.messagebus.api.producer.builder.CordaProducerBuilder import net.corda.messaging.api.mediator.MediatorProducer -import net.corda.messaging.api.mediator.MessageBusProducer import net.corda.messaging.api.mediator.config.MediatorProducerConfig import net.corda.messaging.api.mediator.factory.MediatorProducerFactory +import net.corda.messaging.mediator.MessageBusProducer import net.corda.schema.configuration.BootConfig import java.util.UUID diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/Message.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MediatorMessage.kt similarity index 89% rename from libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/Message.kt rename to libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MediatorMessage.kt index efa5c92cdb0..1ee7ea44dfb 100644 --- a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/Message.kt +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MediatorMessage.kt @@ -3,7 +3,7 @@ package net.corda.messaging.api.mediator /** * Class for storing message data and metadata. */ -data class Message( +data class MediatorMessage( /** Message body (payload). */ val body: Any, /** Message properties (metadata). */ diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MediatorProducer.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MediatorProducer.kt index d398105b53d..946a899f4f5 100644 --- a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MediatorProducer.kt +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MediatorProducer.kt @@ -23,5 +23,5 @@ interface MediatorProducer: AutoCloseable { * @returns ProducerReply Holds producer's reply if producer supports request-reply messaging pattern * (@see [isRequestReply]). */ - fun send(message: Message, address: String): ProducerReply + fun send(message: MediatorMessage, address: String): ProducerReply } \ No newline at end of file diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MessageRouter.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MessageRouter.kt index 48ed65fff3d..b7ceb70968d 100644 --- a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MessageRouter.kt +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MessageRouter.kt @@ -16,10 +16,10 @@ package net.corda.messaging.api.mediator fun interface MessageRouter { /** - * Provides [RoutingDestination] for given [Message]. + * Provides [RoutingDestination] for given [MediatorMessage]. * * @param message Message. * @return Destination for given message. */ - fun getDestination(message: Message): RoutingDestination + fun getDestination(message: MediatorMessage): RoutingDestination } diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/ProducerReply.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/ProducerReply.kt index c1caab27822..c65e9dadb8d 100644 --- a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/ProducerReply.kt +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/ProducerReply.kt @@ -5,7 +5,7 @@ package net.corda.messaging.api.mediator */ data class ProducerReply( /** Reply message (set only if [MediatorProducer] supports request-reply messaging pattern). */ - val reply: Message?, + val reply: MediatorMessage?, /** Exception (set in case of error). */ val exception: Exception?, ) diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/RoutingDestination.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/RoutingDestination.kt index d3203ddf479..be141df26ca 100644 --- a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/RoutingDestination.kt +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/RoutingDestination.kt @@ -1,7 +1,7 @@ package net.corda.messaging.api.mediator /** - * Routing destination encapsulate [MediatorProducer] and related data needed to send a [Message]. + * Routing destination encapsulate [MediatorProducer] and related data needed to send a [MediatorMessage]. */ data class RoutingDestination( val producer: MediatorProducer, From a8511b1bd51630511f5e2a4ff77b90d753663db8 Mon Sep 17 00:00:00 2001 From: Miljenko Brkic Date: Mon, 18 Sep 2023 17:35:33 +0100 Subject: [PATCH 26/50] CORE-16199 Refactoring --- .../net/corda/messaging/mediator/MessageBusProducer.kt | 2 +- .../kotlin/net/corda/messaging/mediator/ProducerTask.kt | 2 +- .../net/corda/messaging/api/mediator/MediatorProducer.kt | 3 ++- .../net/corda/messaging/api/mediator/RoutingDestination.kt | 6 +++--- 4 files changed, 7 insertions(+), 6 deletions(-) diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MessageBusProducer.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MessageBusProducer.kt index 692f60be892..97bb509f27f 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MessageBusProducer.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MessageBusProducer.kt @@ -13,7 +13,7 @@ class MessageBusProducer( private val producer: CordaProducer, ): MediatorProducer { - override fun send(message: MediatorMessage, address: String): ProducerReply { + override fun send(message: MediatorMessage, endpoint: String): ProducerReply { TODO("Not implemented yet") } diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/ProducerTask.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/ProducerTask.kt index 245a7ffd644..909e7d2a950 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/ProducerTask.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/ProducerTask.kt @@ -23,7 +23,7 @@ class ProducerTask( override fun call(): Result { val destination = messageRouter.getDestination(message) - val reply = with(destination) { producer.send(message, address) } + val reply = with(destination) { producer.send(message, endpoint) } return Result(this, reply.reply) } } \ No newline at end of file diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MediatorProducer.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MediatorProducer.kt index 946a899f4f5..9b919d5cddf 100644 --- a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MediatorProducer.kt +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MediatorProducer.kt @@ -20,8 +20,9 @@ interface MediatorProducer: AutoCloseable { * Sends message to producer's endpoint and returns reply. * * @param message Message + * @param endpoint Endpoint where messages is sent to. * @returns ProducerReply Holds producer's reply if producer supports request-reply messaging pattern * (@see [isRequestReply]). */ - fun send(message: MediatorMessage, address: String): ProducerReply + fun send(message: MediatorMessage, endpoint: String): ProducerReply } \ No newline at end of file diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/RoutingDestination.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/RoutingDestination.kt index be141df26ca..cdc0ca8dade 100644 --- a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/RoutingDestination.kt +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/RoutingDestination.kt @@ -5,10 +5,10 @@ package net.corda.messaging.api.mediator */ data class RoutingDestination( val producer: MediatorProducer, - val address: String, + val endpoint: String, ) { companion object { - fun routeTo(producer: MediatorProducer, address: String) = - RoutingDestination(producer, address) + fun routeTo(producer: MediatorProducer, endpoint: String) = + RoutingDestination(producer, endpoint) } } From a102c6911a1190a354a585f7282bebe6e0cf4fce Mon Sep 17 00:00:00 2001 From: Miljenko Brkic Date: Mon, 18 Sep 2023 17:37:17 +0100 Subject: [PATCH 27/50] CORE-16199 Refactoring --- .../corda/messaging/mediator/MultiSourceEventMediatorImpl.kt | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt index a21b4f0d8cb..a47e1a8a5ae 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt @@ -24,8 +24,8 @@ import java.util.UUID @Suppress("LongParameterList") class MultiSourceEventMediatorImpl( private val config: EventMediatorConfig, - private val serializer: CordaAvroSerializer, - private val stateDeserializer: CordaAvroDeserializer, + serializer: CordaAvroSerializer, + stateDeserializer: CordaAvroDeserializer, private val stateManager: StateManager, private val taskManager: TaskManager, lifecycleCoordinatorFactory: LifecycleCoordinatorFactory, From f38f2a26c1bef3c7f89b59d1a36cfab000cc574f Mon Sep 17 00:00:00 2001 From: Miljenko Brkic Date: Tue, 19 Sep 2023 15:01:52 +0100 Subject: [PATCH 28/50] CORE-16199 Refactoring --- components/flow/flow-service/build.gradle | 2 +- .../flow/service/FlowExecutorMediatorImpl.kt | 32 ++++++++++--------- .../mediator/MediatorComponentFactory.kt | 8 ++--- .../messaging/mediator/MediatorTaskManager.kt | 17 ++++++---- .../messaging/mediator/MessageBusProducer.kt | 8 ++--- .../mediator/MultiSourceEventMediatorImpl.kt | 2 +- .../corda/messaging/mediator/ProducerTask.kt | 12 ++++--- .../MediatorConsumerFactoryFactoryImpl.kt | 28 ++++++++++++++++ .../MediatorProducerFactoryFactoryImpl.kt | 26 +++++++++++++++ .../api/mediator/MediatorProducer.kt | 3 +- .../messaging/api/mediator/MessageRouter.kt | 4 +-- .../messaging/api/mediator/ProducerReply.kt | 11 ------- .../factory/MediatorConsumerFactoryFactory.kt | 21 ++++++++++++ .../factory/MediatorProducerFactoryFactory.kt | 20 ++++++++++++ .../factory/MediatorProducerFinder.kt | 8 ++--- .../mediator/factory/MessageRouterFactory.kt | 4 +-- 16 files changed, 150 insertions(+), 56 deletions(-) create mode 100644 libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/MediatorConsumerFactoryFactoryImpl.kt create mode 100644 libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/MediatorProducerFactoryFactoryImpl.kt delete mode 100644 libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/ProducerReply.kt create mode 100644 libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MediatorConsumerFactoryFactory.kt create mode 100644 libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MediatorProducerFactoryFactory.kt diff --git a/components/flow/flow-service/build.gradle b/components/flow/flow-service/build.gradle index a6566a4a577..d13f5d4188d 100644 --- a/components/flow/flow-service/build.gradle +++ b/components/flow/flow-service/build.gradle @@ -34,7 +34,7 @@ dependencies { implementation project(":libs:lifecycle:lifecycle") implementation project(":libs:membership:membership-common") implementation project(":libs:metrics") - implementation project(":libs:messaging:messaging-impl") + implementation project(":libs:messaging:messaging") implementation project(':libs:platform-info') implementation project(":libs:sandbox") implementation project(':libs:serialization:serialization-amqp') diff --git a/components/flow/flow-service/src/main/kotlin/net/corda/flow/service/FlowExecutorMediatorImpl.kt b/components/flow/flow-service/src/main/kotlin/net/corda/flow/service/FlowExecutorMediatorImpl.kt index f2bea3f0daf..e15e57d11b3 100644 --- a/components/flow/flow-service/src/main/kotlin/net/corda/flow/service/FlowExecutorMediatorImpl.kt +++ b/components/flow/flow-service/src/main/kotlin/net/corda/flow/service/FlowExecutorMediatorImpl.kt @@ -14,17 +14,15 @@ import net.corda.lifecycle.RegistrationHandle import net.corda.lifecycle.StartEvent import net.corda.lifecycle.StopEvent import net.corda.lifecycle.createCoordinator -import net.corda.messagebus.api.consumer.builder.CordaConsumerBuilder -import net.corda.messagebus.api.producer.builder.CordaProducerBuilder import net.corda.messaging.api.mediator.MessageRouter import net.corda.messaging.api.mediator.MultiSourceEventMediator import net.corda.messaging.api.mediator.RoutingDestination.Companion.routeTo import net.corda.messaging.api.mediator.config.EventMediatorConfigBuilder +import net.corda.messaging.api.mediator.factory.MediatorConsumerFactoryFactory +import net.corda.messaging.api.mediator.factory.MediatorProducerFactoryFactory import net.corda.messaging.api.mediator.factory.MessageRouterFactory import net.corda.messaging.api.mediator.factory.MultiSourceEventMediatorFactory import net.corda.messaging.api.processor.StateAndEventProcessor -import net.corda.messaging.mediator.factory.MessageBusConsumerFactory -import net.corda.messaging.mediator.factory.MessageBusProducerFactory import net.corda.schema.Schemas.Flow.FLOW_EVENT_TOPIC import net.corda.schema.Schemas.Flow.FLOW_MAPPER_EVENT_TOPIC import net.corda.schema.configuration.ConfigKeys.FLOW_CONFIG @@ -42,8 +40,8 @@ class FlowExecutorMediatorImpl ( coordinatorFactory: LifecycleCoordinatorFactory, private val flowEventProcessorFactory: FlowEventProcessorFactory, private val eventMediatorFactory: MultiSourceEventMediatorFactory, - private val cordaConsumerBuilder: CordaConsumerBuilder, - private val cordaProducerBuilder: CordaProducerBuilder, + private val mediatorConsumerFactoryFactory: MediatorConsumerFactoryFactory, + private val mediatorProducerFactoryFactory: MediatorProducerFactoryFactory, private val toMessagingConfig: (Map) -> SmartConfig ) : FlowExecutor { @@ -55,16 +53,16 @@ class FlowExecutorMediatorImpl ( flowEventProcessorFactory: FlowEventProcessorFactory, @Reference(service = MultiSourceEventMediatorFactory::class) eventMediatorFactory: MultiSourceEventMediatorFactory, - @Reference(service = CordaConsumerBuilder::class) - cordaConsumerBuilder: CordaConsumerBuilder, - @Reference(service = CordaProducerBuilder::class) - cordaProducerBuilder: CordaProducerBuilder, + @Reference(service = MediatorConsumerFactoryFactory::class) + mediatorConsumerFactoryFactory: MediatorConsumerFactoryFactory, + @Reference(service = MediatorProducerFactoryFactory::class) + mediatorProducerFactoryFactory: MediatorProducerFactoryFactory, ) : this( coordinatorFactory, flowEventProcessorFactory, eventMediatorFactory, - cordaConsumerBuilder, - cordaProducerBuilder, + mediatorConsumerFactoryFactory, + mediatorProducerFactoryFactory, { cfg -> cfg.getConfig(MESSAGING_CONFIG) } ) @@ -156,10 +154,14 @@ class FlowExecutorMediatorImpl ( .name("FlowEventMediator") .messagingConfig(messagingConfig) .consumerFactories( - MessageBusConsumerFactory(FLOW_EVENT_TOPIC, CONSUMER_GROUP, messagingConfig, cordaConsumerBuilder), + mediatorConsumerFactoryFactory.createMessageBusProducerFactory( + FLOW_EVENT_TOPIC, CONSUMER_GROUP, messagingConfig + ), ) .producerFactories( - MessageBusProducerFactory(MESSAGE_BUS_PRODUCER, messagingConfig, cordaProducerBuilder), + mediatorProducerFactoryFactory.createMessageBusProducerFactory( + MESSAGE_BUS_PRODUCER, messagingConfig + ), //RpcProducerFactory(CRYPTO_RPC_PRODUCER, messagingConfig, cordaRpcBuilder), ) .messageProcessor(messageProcessor) @@ -170,7 +172,7 @@ class FlowExecutorMediatorImpl ( val messageBusProducer = producerFinder.find(MESSAGE_BUS_PRODUCER) MessageRouter { message -> - when (message.body) { + when (message.payload) { is FlowMapperEvent -> routeTo(messageBusProducer, FLOW_MAPPER_EVENT_TOPIC) else -> throw IllegalStateException("No route defined for message $message") } diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MediatorComponentFactory.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MediatorComponentFactory.kt index d31061ef553..c1dfdfc6099 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MediatorComponentFactory.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MediatorComponentFactory.kt @@ -71,10 +71,10 @@ internal class MediatorComponentFactory( fun createRouter( producers: Collection ): MessageRouter { - val producersByName = producers.associateBy { it.name } - return messageRouterFactory.create { name -> - producersByName[name] - ?: throw IllegalStateException("Producer with name \"$name\" not found") + val producersById = producers.associateBy { it.id } + return messageRouterFactory.create { id -> + producersById[id] + ?: throw IllegalStateException("Producer with ID \"$id\" not found") } } } \ No newline at end of file diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MediatorTaskManager.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MediatorTaskManager.kt index 441a1b6b85c..ad472dbb60a 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MediatorTaskManager.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MediatorTaskManager.kt @@ -1,5 +1,6 @@ package net.corda.messaging.mediator +import kotlinx.coroutines.runBlocking import net.corda.messagebus.api.consumer.CordaConsumerRecord import net.corda.messaging.api.mediator.MediatorMessage import net.corda.messaging.api.mediator.MessageRouter @@ -111,7 +112,7 @@ internal class MediatorTaskManager( ): List> { return processorTaskResults.map { result -> result.outputEvents.map { event -> - val message = MediatorMessage(event.value!!, emptyMap()) + val message = MediatorMessage(event.value!!) ProducerTask( message, messageRouter, @@ -130,10 +131,13 @@ internal class MediatorTaskManager( fun executeProducerTasks( producerTasks: Collection> ): List> { - return producerTasks.map { producerTask -> - taskManager.execute(TaskType.SHORT_RUNNING, producerTask::call) - }.map { - it.join() +// return producerTasks.map { producerTask -> +// taskManager.execute(TaskType.SHORT_RUNNING, producerTask::call) +// }.map { +// it.join() +// } + return runBlocking { + producerTasks.map { it.call() } } } @@ -141,13 +145,12 @@ internal class MediatorTaskManager( * Converts [ProducerTask.Result] to [CordaConsumerRecord]. */ private fun ProducerTask.Result.toCordaConsumerRecord() = - @Suppress("UNCHECKED_CAST") (CordaConsumerRecord( "", -1, -1, producerTask.processorTask.events.first().key, - replyMessage!!.body as E, + replyMessage!!.payload, Instant.now().toEpochMilli() )) } \ No newline at end of file diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MessageBusProducer.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MessageBusProducer.kt index c95a7c262a1..02664cdfcc4 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MessageBusProducer.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MessageBusProducer.kt @@ -18,9 +18,9 @@ class MessageBusProducer( private val log: Logger = LoggerFactory.getLogger(this::class.java.enclosingClass) } - override fun send(message: MediatorMessage<*>): Deferred?> = + override fun send(message: MediatorMessage<*>, endpoint: String): Deferred?> = CompletableDeferred?>().apply { - producer.send(message.toCordaProducerRecord()) { ex -> + producer.send(message.toCordaProducerRecord(endpoint)) { ex -> if (ex != null) { completeExceptionally(ex) } else { @@ -40,9 +40,9 @@ class MessageBusProducer( } } -private fun MediatorMessage<*>.toCordaProducerRecord() : CordaProducerRecord<*, *> { +private fun MediatorMessage<*>.toCordaProducerRecord(endpoint: String) : CordaProducerRecord<*, *> { return CordaProducerRecord( - topic = this.getProperty("topic"), + topic = endpoint, key = this.getProperty("key"), value = this.payload, headers = this.getProperty("headers"), diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt index a47e1a8a5ae..5e3eb79e2f5 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt @@ -36,7 +36,7 @@ class MultiSourceEventMediatorImpl( private var consumers = listOf>() private var producers = listOf() private lateinit var messageRouter: MessageRouter - private val mediatorComponentFactory = MediatorComponentFactory( + private val mediatorComponentFactory = MediatorComponentFactory( config.messageProcessor, config.consumerFactories, config.producerFactories, config.messageRouterFactory ) private val mediatorStateManager = MediatorStateManager( diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/ProducerTask.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/ProducerTask.kt index 909e7d2a950..ab00d23a885 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/ProducerTask.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/ProducerTask.kt @@ -1,5 +1,6 @@ package net.corda.messaging.mediator +import kotlinx.coroutines.runBlocking import net.corda.messaging.api.mediator.MediatorMessage import net.corda.messaging.api.mediator.MediatorProducer import net.corda.messaging.api.mediator.MessageRouter @@ -9,21 +10,24 @@ import java.util.concurrent.Callable * [ProducerTask] sends a [MediatorMessage] to [MediatorProducer] selected by [MessageRouter]. */ class ProducerTask( - private val message: MediatorMessage, + private val message: MediatorMessage, private val messageRouter: MessageRouter, val processorTask: ProcessorTask, ): Callable> { class Result( val producerTask: ProducerTask, - val replyMessage: MediatorMessage?, + val replyMessage: MediatorMessage?, ) { fun hasReply() = replyMessage != null } override fun call(): Result { val destination = messageRouter.getDestination(message) - val reply = with(destination) { producer.send(message, endpoint) } - return Result(this, reply.reply) + @Suppress("UNCHECKED_CAST") + val reply = runBlocking { + with(destination) { producer.send(message, endpoint).await() } + } as MediatorMessage? + return Result(this, reply) } } \ No newline at end of file diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/MediatorConsumerFactoryFactoryImpl.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/MediatorConsumerFactoryFactoryImpl.kt new file mode 100644 index 00000000000..1774f7877f7 --- /dev/null +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/MediatorConsumerFactoryFactoryImpl.kt @@ -0,0 +1,28 @@ +package net.corda.messaging.mediator.factory + +import net.corda.libs.configuration.SmartConfig +import net.corda.messagebus.api.consumer.builder.CordaConsumerBuilder +import net.corda.messaging.api.mediator.factory.MediatorConsumerFactoryFactory +import org.osgi.service.component.annotations.Activate +import org.osgi.service.component.annotations.Component +import org.osgi.service.component.annotations.Reference + +/** + * Factory for creating multi-source event mediator consumers. + */ +@Component(service = [MediatorConsumerFactoryFactory::class]) +class MediatorConsumerFactoryFactoryImpl @Activate constructor( + @Reference(service = CordaConsumerBuilder::class) + private val cordaConsumerBuilder: CordaConsumerBuilder, +): MediatorConsumerFactoryFactory { + override fun createMessageBusProducerFactory( + topicName: String, + groupName: String, + messageBusConfig: SmartConfig + ) = MessageBusConsumerFactory( + topicName, + groupName, + messageBusConfig, + cordaConsumerBuilder, + ) +} \ No newline at end of file diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/MediatorProducerFactoryFactoryImpl.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/MediatorProducerFactoryFactoryImpl.kt new file mode 100644 index 00000000000..cce092169ae --- /dev/null +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/MediatorProducerFactoryFactoryImpl.kt @@ -0,0 +1,26 @@ +package net.corda.messaging.mediator.factory + +import net.corda.libs.configuration.SmartConfig +import net.corda.messagebus.api.producer.builder.CordaProducerBuilder +import net.corda.messaging.api.mediator.factory.MediatorProducerFactoryFactory +import org.osgi.service.component.annotations.Activate +import org.osgi.service.component.annotations.Component +import org.osgi.service.component.annotations.Reference + +/** + * Factory for creating multi-source event mediator producers. + */ +@Component(service = [MediatorProducerFactoryFactory::class]) +class MediatorProducerFactoryFactoryImpl @Activate constructor( + @Reference(service = CordaProducerBuilder::class) + private val cordaProducerBuilder: CordaProducerBuilder, +): MediatorProducerFactoryFactory { + override fun createMessageBusProducerFactory( + id: String, + messageBusConfig: SmartConfig, + ) = MessageBusProducerFactory( + id, + messageBusConfig, + cordaProducerBuilder, + ) +} \ No newline at end of file diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MediatorProducer.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MediatorProducer.kt index a53bcfc9d71..c333271e008 100644 --- a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MediatorProducer.kt +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MediatorProducer.kt @@ -15,8 +15,9 @@ interface MediatorProducer : AutoCloseable { * Asynchronously sends a generic [MediatorMessage], and returns any result/error through a [Deferred] response. * * @param message The [MediatorMessage] to send. + * @param endpoint Endpoint to which the message is sent to. * @return [Deferred] instance representing the asynchronous computation result, or null if the destination doesn't * provide a response. * */ - fun send(message: MediatorMessage<*>) : Deferred?> + fun send(message: MediatorMessage<*>, endpoint: String) : Deferred?> } diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MessageRouter.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MessageRouter.kt index b7ceb70968d..c9fd04fa21b 100644 --- a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MessageRouter.kt +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MessageRouter.kt @@ -6,7 +6,7 @@ package net.corda.messaging.api.mediator * * ``` * MessageRouter { message -> - * when (message.body) { + * when (message.payload) { * is FlowEvent -> RoutingDestination(messageBusProducer, "flow.event.topic") * else -> throw IllegalStateException("No route defined for message $message") * } @@ -21,5 +21,5 @@ fun interface MessageRouter { * @param message Message. * @return Destination for given message. */ - fun getDestination(message: MediatorMessage): RoutingDestination + fun getDestination(message: MediatorMessage): RoutingDestination } diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/ProducerReply.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/ProducerReply.kt deleted file mode 100644 index c65e9dadb8d..00000000000 --- a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/ProducerReply.kt +++ /dev/null @@ -1,11 +0,0 @@ -package net.corda.messaging.api.mediator - -/** - * Class to store reply of the [MediatorProducer]. - */ -data class ProducerReply( - /** Reply message (set only if [MediatorProducer] supports request-reply messaging pattern). */ - val reply: MediatorMessage?, - /** Exception (set in case of error). */ - val exception: Exception?, -) diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MediatorConsumerFactoryFactory.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MediatorConsumerFactoryFactory.kt new file mode 100644 index 00000000000..37593b1d5ce --- /dev/null +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MediatorConsumerFactoryFactory.kt @@ -0,0 +1,21 @@ +package net.corda.messaging.api.mediator.factory + +import net.corda.libs.configuration.SmartConfig + +/** + * Factory for creating multi-source event mediator consumer factories. + */ +interface MediatorConsumerFactoryFactory { + /** + * Creates a message bus consumer factory. + * + * @param topicName Topic name. + * @param groupName Consumer group name. + * @param messageBusConfig Message bus related configuration. + */ + fun createMessageBusProducerFactory( + topicName: String, + groupName: String, + messageBusConfig: SmartConfig, + ) : MediatorConsumerFactory +} \ No newline at end of file diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MediatorProducerFactoryFactory.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MediatorProducerFactoryFactory.kt new file mode 100644 index 00000000000..90ab9457692 --- /dev/null +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MediatorProducerFactoryFactory.kt @@ -0,0 +1,20 @@ +package net.corda.messaging.api.mediator.factory + +import net.corda.libs.configuration.SmartConfig + +/** + * Factory for creating multi-source event mediator producer factories. + */ +interface MediatorProducerFactoryFactory { + /** + * Creates a message bus producer factory. + * + * + * @param id Producer ID. + * @param messageBusConfig Message bus related configuration. + */ + fun createMessageBusProducerFactory( + id: String, + messageBusConfig: SmartConfig, + ) : MediatorProducerFactory +} \ No newline at end of file diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MediatorProducerFinder.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MediatorProducerFinder.kt index 7e787cdb5ec..a961096c97c 100644 --- a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MediatorProducerFinder.kt +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MediatorProducerFinder.kt @@ -3,13 +3,13 @@ package net.corda.messaging.api.mediator.factory import net.corda.messaging.api.mediator.MediatorProducer /** - * Mediator producer finder is used to access [MediatorProducer] by its name. + * Mediator producer finder is used to access [MediatorProducer] by its ID. */ fun interface MediatorProducerFinder { /** - * @param name Producer's name. - * @return Producer found by given name. + * @param id Producer's ID. + * @return Producer found by given ID. */ - fun find(name: String): MediatorProducer + fun find(id: String): MediatorProducer } diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MessageRouterFactory.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MessageRouterFactory.kt index 5ec09ca15f7..c78fc85946f 100644 --- a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MessageRouterFactory.kt +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MessageRouterFactory.kt @@ -10,14 +10,14 @@ fun interface MessageRouterFactory { /** * Creates a new instance of [MessageRouter]. Provided [MediatorProducerFinder] is used to find [MediatorProducer]s - * by their names. Example: + * by their IDs. Example: * * ``` * MessageRouterFactory { producerFinder -> * val messageBusProducer = producerFinder.find("MessageBusProducer") * * MessageRouter { message -> - * when (message.body) { + * when (message.payload) { * is FlowEvent -> RoutingDestination(messageBusProducer, "flow.event.topic") * else -> throw IllegalStateException("No route defined for message $message") * } From 4a3171019b32197d60b5082f27a6ede9f82031e5 Mon Sep 17 00:00:00 2001 From: Miljenko Brkic Date: Wed, 20 Sep 2023 12:23:01 +0100 Subject: [PATCH 29/50] CORE-16199 Refactoring, using Deferred instead of CompletableFuture in MessageBusConsumer, unit tests for message bus cosumer/producer and mediator factories --- .../flow/service/FlowExecutorMediatorImpl.kt | 2 +- .../db/consumer/DBCordaConsumerImpl.kt | 14 +++- .../kafka/consumer/CordaKafkaConsumerImpl.kt | 40 ++-------- .../consumer/CordaKafkaConsumerImplTest.kt | 33 ++++++++ .../messagebus/api/consumer/CordaConsumer.kt | 2 +- .../messaging/mediator/MessageBusConsumer.kt | 20 ++--- .../mediator/MultiSourceEventMediatorImpl.kt | 11 ++- .../MediatorConsumerFactoryFactoryImpl.kt | 2 +- .../factory/MessageBusConsumerFactory.kt | 53 ------------ .../factory/MessageBusProducerFactory.kt | 8 +- .../MultiSourceEventMediatorFactoryImpl.kt | 1 + .../mediator/MessageBusConsumerTest.kt | 80 +++++++++++++++++++ .../mediator/MessageBusProducerTest.kt | 13 +-- .../MediatorConsumerFactoryFactoryTest.kt | 33 ++++++++ .../MediatorProducerFactoryFactoryTest.kt | 31 +++++++ .../factory/MessageBusConsumerFactoryTest.kt | 45 +++++++++++ .../factory/MessageBusProducerFactoryTest.kt | 39 +++++++++ .../MultiSourceEventMediatorFactoryTest.kt | 51 ++++++++++++ .../api/mediator/MediatorConsumer.kt | 4 +- .../factory/MediatorConsumerFactoryFactory.kt | 2 +- 20 files changed, 368 insertions(+), 116 deletions(-) create mode 100644 libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/MessageBusConsumerTest.kt create mode 100644 libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/factory/MediatorConsumerFactoryFactoryTest.kt create mode 100644 libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/factory/MediatorProducerFactoryFactoryTest.kt create mode 100644 libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/factory/MessageBusConsumerFactoryTest.kt create mode 100644 libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/factory/MessageBusProducerFactoryTest.kt create mode 100644 libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/factory/MultiSourceEventMediatorFactoryTest.kt diff --git a/components/flow/flow-service/src/main/kotlin/net/corda/flow/service/FlowExecutorMediatorImpl.kt b/components/flow/flow-service/src/main/kotlin/net/corda/flow/service/FlowExecutorMediatorImpl.kt index e15e57d11b3..f9d2ded2eec 100644 --- a/components/flow/flow-service/src/main/kotlin/net/corda/flow/service/FlowExecutorMediatorImpl.kt +++ b/components/flow/flow-service/src/main/kotlin/net/corda/flow/service/FlowExecutorMediatorImpl.kt @@ -154,7 +154,7 @@ class FlowExecutorMediatorImpl ( .name("FlowEventMediator") .messagingConfig(messagingConfig) .consumerFactories( - mediatorConsumerFactoryFactory.createMessageBusProducerFactory( + mediatorConsumerFactoryFactory.createMessageBusConsumerFactory( FLOW_EVENT_TOPIC, CONSUMER_GROUP, messagingConfig ), ) diff --git a/libs/messaging/db-message-bus-impl/src/main/kotlin/net/corda/messagebus/db/consumer/DBCordaConsumerImpl.kt b/libs/messaging/db-message-bus-impl/src/main/kotlin/net/corda/messagebus/db/consumer/DBCordaConsumerImpl.kt index 41c0cf4280e..20ca064cd71 100644 --- a/libs/messaging/db-message-bus-impl/src/main/kotlin/net/corda/messagebus/db/consumer/DBCordaConsumerImpl.kt +++ b/libs/messaging/db-message-bus-impl/src/main/kotlin/net/corda/messagebus/db/consumer/DBCordaConsumerImpl.kt @@ -203,8 +203,18 @@ internal class DBCordaConsumerImpl constructor( } } - override fun commitAsync(callback: CordaConsumer.Callback?) { - TODO("Not yet implemented") + override fun commitAsyncOffsets(callback: CordaConsumer.Callback?) { + dbAccess.writeOffsets( + lastReadOffset.map { (cordaTopicPartition, offset) -> + CommittedPositionEntry( + cordaTopicPartition.topic, + groupId, + cordaTopicPartition.partition, + offset, + ATOMIC_TRANSACTION, + ) + } + ) } override fun commitSyncOffsets(event: CordaConsumerRecord, metaData: String?) { diff --git a/libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/consumer/CordaKafkaConsumerImpl.kt b/libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/consumer/CordaKafkaConsumerImpl.kt index 04eb81b014c..326562f43d4 100644 --- a/libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/consumer/CordaKafkaConsumerImpl.kt +++ b/libs/messaging/kafka-message-bus-impl/src/main/kotlin/net/corda/messagebus/kafka/consumer/CordaKafkaConsumerImpl.kt @@ -304,38 +304,14 @@ class CordaKafkaConsumerImpl( } } - override fun commitAsync(callback: CordaConsumer.Callback?) { - var attemptCommit = true - - while (attemptCommit) { - try { - consumer.commitAsync { offsets, exception -> - callback?.onCompletion( - offsets.entries.associate { - it.key!!.toCordaTopicPartition(config.topicPrefix) to it.value.offset() - }, - exception - ) - } - attemptCommit = false - } catch (ex: Exception) { - when (ex::class.java) { - in fatalExceptions -> { - logErrorAndThrowFatalException( - "Error attempting to commitAsync offsets.", - ex - ) - } - in transientExceptions -> { - logWarningAndThrowIntermittentException("Failed to commitAsync offsets.", ex) - } - else -> { - logErrorAndThrowFatalException( - "Unexpected error attempting to commitAsync offsets.", ex - ) - } - } - } + override fun commitAsyncOffsets(callback: CordaConsumer.Callback?) { + consumer.commitAsync { offsets, exception -> + callback?.onCompletion( + offsets.entries.associate { + it.key!!.toCordaTopicPartition(config.topicPrefix) to it.value.offset() + }, + exception + ) } } diff --git a/libs/messaging/kafka-message-bus-impl/src/test/kotlin/net/corda/messagebus/kafka/consumer/CordaKafkaConsumerImplTest.kt b/libs/messaging/kafka-message-bus-impl/src/test/kotlin/net/corda/messagebus/kafka/consumer/CordaKafkaConsumerImplTest.kt index 82042556f37..e18012b335b 100644 --- a/libs/messaging/kafka-message-bus-impl/src/test/kotlin/net/corda/messagebus/kafka/consumer/CordaKafkaConsumerImplTest.kt +++ b/libs/messaging/kafka-message-bus-impl/src/test/kotlin/net/corda/messagebus/kafka/consumer/CordaKafkaConsumerImplTest.kt @@ -4,6 +4,7 @@ import io.micrometer.core.instrument.binder.kafka.KafkaClientMetrics import net.corda.data.chunking.Chunk import net.corda.data.chunking.ChunkKey import net.corda.messagebus.api.CordaTopicPartition +import net.corda.messagebus.api.consumer.CordaConsumer import net.corda.messagebus.api.consumer.CordaConsumerRebalanceListener import net.corda.messagebus.api.consumer.CordaConsumerRecord import net.corda.messagebus.api.consumer.CordaOffsetResetStrategy @@ -24,6 +25,7 @@ import org.apache.kafka.clients.consumer.Consumer import org.apache.kafka.clients.consumer.ConsumerRebalanceListener import org.apache.kafka.clients.consumer.MockConsumer import org.apache.kafka.clients.consumer.OffsetAndMetadata +import org.apache.kafka.clients.consumer.OffsetCommitCallback import org.apache.kafka.common.KafkaException import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.errors.AuthenticationException @@ -43,6 +45,7 @@ import org.mockito.kotlin.anyOrNull import org.mockito.kotlin.doAnswer import org.mockito.kotlin.doReturn import org.mockito.kotlin.doThrow +import org.mockito.kotlin.eq import org.mockito.kotlin.mock import org.mockito.kotlin.times import org.mockito.kotlin.verify @@ -176,6 +179,36 @@ class CordaKafkaConsumerImplTest { verify(metricsBinder, times(1)).close() } + @Test + fun testCommitAsyncOffsets() { + val callback = mock() + assertThat(consumer.committed(setOf(partition))).isEmpty() + + cordaKafkaConsumer.poll(Duration.ZERO) + cordaKafkaConsumer.commitAsyncOffsets(callback) + + val committedPositionAfterPoll = consumer.committed(setOf(partition)) + assertThat(committedPositionAfterPoll.values.first().offset()).isEqualTo(numberOfRecords) + } + + @Test + fun testCommitAsyncOffsetsException() { + consumer = mock() + cordaKafkaConsumer = createConsumer(consumer) + val exception = CommitFailedException() + doAnswer { + val callback = it.arguments[0] as OffsetCommitCallback + callback.onComplete(mock(), exception) + null + }.whenever(consumer).commitAsync(any()) + val callback = mock() + + cordaKafkaConsumer.commitAsyncOffsets(callback) + + verify(consumer, times(1)).commitAsync(any()) + verify(callback, times(1)).onCompletion(any(), eq(exception)) + } + @Test fun testCommitOffsets() { val consumerRecord = CordaConsumerRecord(eventTopic, 1, 5L, "", "value", 0) diff --git a/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/consumer/CordaConsumer.kt b/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/consumer/CordaConsumer.kt index bb848c9377d..c2a3aea38f7 100644 --- a/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/consumer/CordaConsumer.kt +++ b/libs/messaging/message-bus/src/main/kotlin/net/corda/messagebus/api/consumer/CordaConsumer.kt @@ -149,7 +149,7 @@ interface CordaConsumer : AutoCloseable { * Asynchronously commit the consumer offsets. * @throws CordaMessageAPIFatalException fatal error occurred attempting to commit offsets. */ - fun commitAsync(callback: Callback?) + fun commitAsyncOffsets(callback: Callback?) /** * Synchronously commit the consumer offset for this [event] back to the topic partition. diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MessageBusConsumer.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MessageBusConsumer.kt index 580cb6bcc9d..675d876bf3b 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MessageBusConsumer.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MessageBusConsumer.kt @@ -1,12 +1,13 @@ package net.corda.messaging.mediator +import kotlinx.coroutines.CompletableDeferred +import kotlinx.coroutines.Deferred import net.corda.messagebus.api.CordaTopicPartition import net.corda.messagebus.api.consumer.CordaConsumer import net.corda.messagebus.api.consumer.CordaConsumerRecord import net.corda.messagebus.api.consumer.CordaOffsetResetStrategy import net.corda.messaging.api.mediator.MediatorConsumer import java.time.Duration -import java.util.concurrent.CompletableFuture /** * Message bus consumer that reads messages from configured topic. @@ -22,17 +23,16 @@ class MessageBusConsumer( override fun poll(timeout: Duration): List> = consumer.poll(timeout) - override fun commitAsync(): CompletableFuture> { - val result = CompletableFuture>() - consumer.commitAsync { offsets, exception -> - if (exception != null) { - result.completeExceptionally(exception) - } else { - result.complete(offsets) + override fun commitAsyncOffsets(): Deferred> = + CompletableDeferred>().apply { + consumer.commitAsyncOffsets { offsets, exception -> + if (exception != null) { + completeExceptionally(exception) + } else { + complete(offsets) + } } } - return result - } override fun resetEventOffsetPosition() = consumer.resetToLastCommittedPositions(CordaOffsetResetStrategy.EARLIEST) diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt index 5e3eb79e2f5..e7c0a217d6b 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt @@ -1,5 +1,6 @@ package net.corda.messaging.mediator +import kotlinx.coroutines.runBlocking import net.corda.avro.serialization.CordaAvroDeserializer import net.corda.avro.serialization.CordaAvroSerializer import net.corda.lifecycle.LifecycleCoordinatorFactory @@ -186,10 +187,12 @@ class MultiSourceEventMediatorImpl( } private fun commitOffsets() { - consumers.map { consumer -> - consumer.commitAsync() - }.map { - it.join() + runBlocking { + consumers.map { consumer -> + consumer.commitAsyncOffsets() + }.map { + it.await() + } } } diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/MediatorConsumerFactoryFactoryImpl.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/MediatorConsumerFactoryFactoryImpl.kt index 1774f7877f7..4b581bab371 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/MediatorConsumerFactoryFactoryImpl.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/MediatorConsumerFactoryFactoryImpl.kt @@ -15,7 +15,7 @@ class MediatorConsumerFactoryFactoryImpl @Activate constructor( @Reference(service = CordaConsumerBuilder::class) private val cordaConsumerBuilder: CordaConsumerBuilder, ): MediatorConsumerFactoryFactory { - override fun createMessageBusProducerFactory( + override fun createMessageBusConsumerFactory( topicName: String, groupName: String, messageBusConfig: SmartConfig diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/MessageBusConsumerFactory.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/MessageBusConsumerFactory.kt index ea2c175ea39..e5519e805b4 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/MessageBusConsumerFactory.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/MessageBusConsumerFactory.kt @@ -50,57 +50,4 @@ class MessageBusConsumerFactory( eventConsumer, ) } - -// fun createMediatorConsumerFactory( -// topicName: String, -// groupName: String, -// messageBusConfig: SmartConfig, -// cordaConsumerBuilder: CordaConsumerBuilder, -// ): MediatorConsumerFactory { -// return object : MediatorConsumerFactory { -// override fun create(config: MediatorConsumerConfig): MediatorConsumer = -// createMessageBusConsumer( -// topicName, -// groupName, -// config.keyClass, -// config.valueClass, -// config.onSerializationError, -// messageBusConfig, -// cordaConsumerBuilder) -// } -// } -// -// fun createMessageBusConsumer( -// topicName: String, -// groupName: String, -// keyClass: Class, -// valueClass: Class, -// onSerializationError: (ByteArray) -> Unit, -// messageBusConfig: SmartConfig, -// cordaConsumerBuilder: CordaConsumerBuilder, -// ): MessageBusConsumer { -// val subscriptionType = "MultiSourceSubscription" -// val uniqueId = UUID.randomUUID().toString() -// val clientId = "$subscriptionType--$groupName--$topicName--$uniqueId" -// -// val eventConsumerConfig = -// ConsumerConfig( -// groupName, -// "$clientId-eventConsumer", -// ConsumerRoles.SAE_EVENT -// ) -// val eventConsumer = cordaConsumerBuilder.createConsumer( -// eventConsumerConfig, -// messageBusConfig, -// keyClass, -// valueClass, -// onSerializationError -// ) -// -// return MessageBusConsumer( -// topicName, -// eventConsumer, -// ) -// } - } \ No newline at end of file diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/MessageBusProducerFactory.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/MessageBusProducerFactory.kt index dfc57b6aa1a..93f8dcd3593 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/MessageBusProducerFactory.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/MessageBusProducerFactory.kt @@ -15,19 +15,19 @@ import java.util.UUID /** * Factory for creating multi-source event mediator message bus producers. * - * @param name Producer name. + * @param id Producer's unique ID. * @param messageBusConfig Message bus related configuration. * @param cordaProducerBuilder [CordaProducer] builder. */ class MessageBusProducerFactory( - private val name: String, + private val id: String, private val messageBusConfig: SmartConfig, private val cordaProducerBuilder: CordaProducerBuilder, ): MediatorProducerFactory { override fun create(config: MediatorProducerConfig): MediatorProducer { val uniqueId = UUID.randomUUID().toString() - val clientId = "$name--$uniqueId" + val clientId = "$id--$uniqueId" val eventProducerConfig = ProducerConfig( clientId, @@ -44,7 +44,7 @@ class MessageBusProducerFactory( ) return MessageBusProducer( - name, + id, eventProducer, ) } diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/MultiSourceEventMediatorFactoryImpl.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/MultiSourceEventMediatorFactoryImpl.kt index 41ef1f2ba08..37902f10040 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/MultiSourceEventMediatorFactoryImpl.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/MultiSourceEventMediatorFactoryImpl.kt @@ -43,6 +43,7 @@ class MultiSourceEventMediatorFactoryImpl @Activate constructor( ) } + // TODO // val msgConfig = messagingConfig.withFallback(defaults) // messagingConfig.getInt(BootConfig.INSTANCE_ID), // Duration.ofMillis(messagingConfig.getLong(MessagingConfig.Subscription.POLL_TIMEOUT)), diff --git a/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/MessageBusConsumerTest.kt b/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/MessageBusConsumerTest.kt new file mode 100644 index 00000000000..c6920dc6469 --- /dev/null +++ b/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/MessageBusConsumerTest.kt @@ -0,0 +1,80 @@ +package net.corda.messaging.mediator + +import kotlinx.coroutines.runBlocking +import net.corda.messagebus.api.consumer.CordaConsumer +import net.corda.messaging.api.mediator.MediatorMessage +import net.corda.v5.base.exceptions.CordaRuntimeException +import org.junit.jupiter.api.BeforeEach +import org.junit.jupiter.api.Test +import org.junit.jupiter.api.assertThrows +import org.mockito.Mockito.doReturn +import org.mockito.Mockito.times +import org.mockito.kotlin.any +import org.mockito.kotlin.eq +import org.mockito.kotlin.mock +import org.mockito.kotlin.verify +import org.mockito.kotlin.whenever +import java.time.Duration + +class MessageBusConsumerTest { + companion object { + private const val TOPIC = "topic" + } + + private lateinit var cordaConsumer: CordaConsumer + private lateinit var mediatorConsumer: MessageBusConsumer + + private val defaultHeaders: List> = emptyList() + private val messageProps: MutableMap = mutableMapOf( + "topic" to "topic", + "key" to "key", + "headers" to defaultHeaders + ) + private val message: MediatorMessage = MediatorMessage("value", messageProps) + + + @BeforeEach + fun setup() { + cordaConsumer = mock() + mediatorConsumer = MessageBusConsumer(TOPIC, cordaConsumer) + } + + @Test + fun testSubscribe() { + mediatorConsumer.subscribe() + + verify(cordaConsumer).subscribe(eq(TOPIC)) + } + + @Test + fun testPoll() { + val timeout = Duration.ofMillis(123) + mediatorConsumer.poll(timeout) + + verify(cordaConsumer).poll(eq(timeout)) + } + + @Test + fun testCommitAsyncOffsets() { + mediatorConsumer.commitAsyncOffsets() + + verify(cordaConsumer).commitAsyncOffsets(any()) + } + + @Test + fun testCommitAsyncOffsetsWithError() { + doReturn(CordaRuntimeException("")).whenever(cordaConsumer).commitAsyncOffsets(any()) + + assertThrows { + runBlocking { + mediatorConsumer.commitAsyncOffsets().await() + } + } + } + + @Test + fun testClose() { + mediatorConsumer.close() + verify(cordaConsumer, times(1)).close() + } +} \ No newline at end of file diff --git a/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/MessageBusProducerTest.kt b/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/MessageBusProducerTest.kt index eaa141e2d31..1e129b0dbc8 100644 --- a/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/MessageBusProducerTest.kt +++ b/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/MessageBusProducerTest.kt @@ -17,12 +17,15 @@ import org.mockito.kotlin.verify import org.mockito.kotlin.whenever class MessageBusProducerTest { + companion object { + private const val TOPIC = "topic" + } + private lateinit var cordaProducer: CordaProducer private lateinit var mediatorProducer: MessageBusProducer private val defaultHeaders: List> = emptyList() private val messageProps: MutableMap = mutableMapOf( - "topic" to "topic", "key" to "key", "headers" to defaultHeaders ) @@ -37,10 +40,10 @@ class MessageBusProducerTest { @Test fun testSend() { - mediatorProducer.send(message) + mediatorProducer.send(message, TOPIC) val expected = CordaProducerRecord( - message.getProperty("topic"), + TOPIC, message.getProperty("key"), message.payload ) @@ -51,7 +54,7 @@ class MessageBusProducerTest { @Test fun testSendWithError() { val record = CordaProducerRecord( - message.getProperty("topic"), + TOPIC, message.getProperty("key"), message.payload ) @@ -59,7 +62,7 @@ class MessageBusProducerTest { doThrow(CordaRuntimeException("")).whenever(cordaProducer).send(eq(record), any()) assertThrows { runBlocking { - mediatorProducer.send(message).await() + mediatorProducer.send(message, TOPIC).await() } } } diff --git a/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/factory/MediatorConsumerFactoryFactoryTest.kt b/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/factory/MediatorConsumerFactoryFactoryTest.kt new file mode 100644 index 00000000000..3cff6f517fb --- /dev/null +++ b/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/factory/MediatorConsumerFactoryFactoryTest.kt @@ -0,0 +1,33 @@ +package net.corda.messaging.mediator.factory + +import net.corda.libs.configuration.SmartConfig +import net.corda.messagebus.api.consumer.builder.CordaConsumerBuilder +import net.corda.messaging.api.mediator.factory.MediatorConsumerFactory +import org.junit.jupiter.api.Assertions +import org.junit.jupiter.api.BeforeEach +import org.junit.jupiter.api.Test +import org.mockito.kotlin.mock + +class MediatorConsumerFactoryFactoryTest { + private lateinit var mediatorConsumerFactoryFactory: MediatorConsumerFactoryFactoryImpl + private val cordaConsumerBuilder = mock() + private val messageBusConfig = mock() + + @BeforeEach + fun beforeEach() { + mediatorConsumerFactoryFactory = MediatorConsumerFactoryFactoryImpl( + cordaConsumerBuilder, + ) + } + + @Test + fun testCreateMessageBusConsumerFactory() { + val messageBusConsumerFactory = mediatorConsumerFactoryFactory.createMessageBusConsumerFactory( + "topic", + "consumerGroup", + messageBusConfig, + ) + + Assertions.assertNotNull(messageBusConsumerFactory) + } +} \ No newline at end of file diff --git a/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/factory/MediatorProducerFactoryFactoryTest.kt b/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/factory/MediatorProducerFactoryFactoryTest.kt new file mode 100644 index 00000000000..532ab6a795d --- /dev/null +++ b/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/factory/MediatorProducerFactoryFactoryTest.kt @@ -0,0 +1,31 @@ +package net.corda.messaging.mediator.factory + +import net.corda.libs.configuration.SmartConfig +import net.corda.messagebus.api.producer.builder.CordaProducerBuilder +import org.junit.jupiter.api.Assertions +import org.junit.jupiter.api.BeforeEach +import org.junit.jupiter.api.Test +import org.mockito.kotlin.mock + +class MediatorProducerFactoryFactoryTest { + private lateinit var mediatorProducerFactoryFactory: MediatorProducerFactoryFactoryImpl + private val cordaProducerBuilder = mock() + private val messageBusConfig = mock() + + @BeforeEach + fun beforeEach() { + mediatorProducerFactoryFactory = MediatorProducerFactoryFactoryImpl( + cordaProducerBuilder, + ) + } + + @Test + fun testCreateMessageBusProducerFactory() { + val messageBusProducerFactory = mediatorProducerFactoryFactory.createMessageBusProducerFactory( + "MessageBusProducer1", + messageBusConfig, + ) + + Assertions.assertNotNull(messageBusProducerFactory) + } +} \ No newline at end of file diff --git a/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/factory/MessageBusConsumerFactoryTest.kt b/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/factory/MessageBusConsumerFactoryTest.kt new file mode 100644 index 00000000000..350293a6bf2 --- /dev/null +++ b/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/factory/MessageBusConsumerFactoryTest.kt @@ -0,0 +1,45 @@ +package net.corda.messaging.mediator.factory + +import net.corda.libs.configuration.SmartConfig +import net.corda.messagebus.api.consumer.CordaConsumer +import net.corda.messagebus.api.consumer.builder.CordaConsumerBuilder +import net.corda.messaging.api.mediator.config.MediatorConsumerConfig +import org.junit.jupiter.api.Assertions +import org.junit.jupiter.api.BeforeEach +import org.junit.jupiter.api.Test +import org.mockito.kotlin.any +import org.mockito.kotlin.anyOrNull +import org.mockito.kotlin.doReturn +import org.mockito.kotlin.mock + +class MessageBusConsumerFactoryTest { + private lateinit var messageBusConsumerFactory: MessageBusConsumerFactory + private val cordaConsumerBuilder = mock() + private val cordaConsumer = mock>() + private val messageBusConfig = mock() + + @BeforeEach + fun beforeEach() { + doReturn(cordaConsumer).`when`(cordaConsumerBuilder).createConsumer( + any(), any(), any>(), any>(), any(), anyOrNull() + ) + messageBusConsumerFactory = MessageBusConsumerFactory( + "topic", + "group", + messageBusConfig, + cordaConsumerBuilder, + ) + } + + @Test + fun testCreateMessageBusConsumer() { + val config = MediatorConsumerConfig( + Any::class.java, + Any::class.java, + ) {} + + val messageBusConsumer = messageBusConsumerFactory.create(config) + + Assertions.assertNotNull(messageBusConsumer) + } +} \ No newline at end of file diff --git a/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/factory/MessageBusProducerFactoryTest.kt b/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/factory/MessageBusProducerFactoryTest.kt new file mode 100644 index 00000000000..c23d955bd80 --- /dev/null +++ b/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/factory/MessageBusProducerFactoryTest.kt @@ -0,0 +1,39 @@ +package net.corda.messaging.mediator.factory + +import net.corda.libs.configuration.SmartConfig +import net.corda.messagebus.api.producer.CordaProducer +import net.corda.messagebus.api.producer.builder.CordaProducerBuilder +import net.corda.messaging.api.mediator.config.MediatorProducerConfig +import org.junit.jupiter.api.Assertions +import org.junit.jupiter.api.BeforeEach +import org.junit.jupiter.api.Test +import org.mockito.kotlin.any +import org.mockito.kotlin.anyOrNull +import org.mockito.kotlin.doReturn +import org.mockito.kotlin.mock + +class MessageBusProducerFactoryTest { + private lateinit var messageBusProducerFactory: MessageBusProducerFactory + private val cordaProducerBuilder = mock() + private val cordaProducer = mock() + private val messageBusConfig = mock() + + @BeforeEach + fun beforeEach() { + doReturn(cordaProducer).`when`(cordaProducerBuilder).createProducer( + any(), any(), anyOrNull() + ) + messageBusProducerFactory = MessageBusProducerFactory( + "MessageBusProducer1", + messageBusConfig, + cordaProducerBuilder, + ) + } + + @Test + fun testCreateMessageBusProducer() { + val config = MediatorProducerConfig {} + val messageBusProducer = messageBusProducerFactory.create(config) + Assertions.assertNotNull(messageBusProducer) + } +} \ No newline at end of file diff --git a/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/factory/MultiSourceEventMediatorFactoryTest.kt b/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/factory/MultiSourceEventMediatorFactoryTest.kt new file mode 100644 index 00000000000..278bfa91df4 --- /dev/null +++ b/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/factory/MultiSourceEventMediatorFactoryTest.kt @@ -0,0 +1,51 @@ +package net.corda.messaging.mediator.factory + +import net.corda.avro.serialization.CordaAvroDeserializer +import net.corda.avro.serialization.CordaAvroSerializationFactory +import net.corda.avro.serialization.CordaAvroSerializer +import net.corda.lifecycle.LifecycleCoordinatorFactory +import net.corda.messaging.api.mediator.config.EventMediatorConfig +import net.corda.messaging.api.mediator.factory.MessageRouterFactory +import net.corda.messaging.api.mediator.statemanager.StateManager +import net.corda.messaging.api.mediator.taskmanager.TaskManager +import net.corda.messaging.api.processor.StateAndEventProcessor +import org.junit.jupiter.api.Assertions +import org.junit.jupiter.api.BeforeEach +import org.junit.jupiter.api.Test +import org.mockito.kotlin.any +import org.mockito.kotlin.anyOrNull +import org.mockito.kotlin.doReturn +import org.mockito.kotlin.mock + +class MultiSourceEventMediatorFactoryTest { + private lateinit var multiSourceEventMediatorFactory: MultiSourceEventMediatorFactoryImpl + private val cordaAvroSerializationFactory = mock() + private val serializer = mock>() + private val stateDeserializer = mock>() + + @BeforeEach + fun beforeEach() { + doReturn(serializer).`when`(cordaAvroSerializationFactory).createAvroSerializer(anyOrNull()) + doReturn(stateDeserializer).`when`(cordaAvroSerializationFactory).createAvroDeserializer(any(), any>()) + multiSourceEventMediatorFactory = MultiSourceEventMediatorFactoryImpl( + cordaAvroSerializationFactory, + mock(), + mock(), + mock(), + ) + } + + @Test + fun testMultiSourceEventMediator() { + val messageProcessor = mock>() + doReturn(Any::class.java).`when`(messageProcessor).stateValueClass + val messageRouterFactory = mock() + val config = mock>() + doReturn(messageProcessor).`when`(config).messageProcessor + doReturn(messageRouterFactory).`when`(config).messageRouterFactory + + val messageBusProducer = multiSourceEventMediatorFactory.create(config) + + Assertions.assertNotNull(messageBusProducer) + } +} \ No newline at end of file diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MediatorConsumer.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MediatorConsumer.kt index 0171cfc4b87..ba921c86d61 100644 --- a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MediatorConsumer.kt +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MediatorConsumer.kt @@ -1,9 +1,9 @@ package net.corda.messaging.api.mediator +import kotlinx.coroutines.Deferred import net.corda.messagebus.api.CordaTopicPartition import net.corda.messagebus.api.consumer.CordaConsumerRecord import java.time.Duration -import java.util.concurrent.CompletableFuture /** * Multi-source event mediator message consumer. @@ -27,7 +27,7 @@ interface MediatorConsumer : AutoCloseable { * * @return [CompletableFuture] with committed offsets. */ - fun commitAsync(): CompletableFuture> + fun commitAsyncOffsets(): Deferred> /** * Resets consumer's offsets to the last committed positions. diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MediatorConsumerFactoryFactory.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MediatorConsumerFactoryFactory.kt index 37593b1d5ce..724b6efbde9 100644 --- a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MediatorConsumerFactoryFactory.kt +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MediatorConsumerFactoryFactory.kt @@ -13,7 +13,7 @@ interface MediatorConsumerFactoryFactory { * @param groupName Consumer group name. * @param messageBusConfig Message bus related configuration. */ - fun createMessageBusProducerFactory( + fun createMessageBusConsumerFactory( topicName: String, groupName: String, messageBusConfig: SmartConfig, From 0280425b79aea21818ec7159ddf63378e6a69086 Mon Sep 17 00:00:00 2001 From: Miljenko Brkic Date: Fri, 22 Sep 2023 10:35:25 +0100 Subject: [PATCH 30/50] CORE-16199 Refactoring --- .../corda/flow/service/FlowExecutorImpl.kt | 3 +- .../flow/service/FlowExecutorMediatorImpl.kt | 2 + .../mediator/MultiSourceEventMediatorImpl.kt | 2 +- .../mediator/statemanager/StateManagerImpl.kt | 18 +++- .../mediator/taskmanager/TaskManagerImpl.kt | 35 +++++- .../messaging/mediator/ProcessorTaskTest.kt | 102 ++++++++++++++++++ .../MediatorConsumerFactoryFactoryTest.kt | 1 - 7 files changed, 154 insertions(+), 9 deletions(-) create mode 100644 libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/ProcessorTaskTest.kt diff --git a/components/flow/flow-service/src/main/kotlin/net/corda/flow/service/FlowExecutorImpl.kt b/components/flow/flow-service/src/main/kotlin/net/corda/flow/service/FlowExecutorImpl.kt index 39268ab27b6..6298318a398 100644 --- a/components/flow/flow-service/src/main/kotlin/net/corda/flow/service/FlowExecutorImpl.kt +++ b/components/flow/flow-service/src/main/kotlin/net/corda/flow/service/FlowExecutorImpl.kt @@ -23,12 +23,11 @@ import net.corda.schema.configuration.MessagingConfig.MAX_ALLOWED_MSG_SIZE import net.corda.schema.configuration.MessagingConfig.Subscription.PROCESSOR_TIMEOUT import net.corda.utilities.trace import org.osgi.service.component.annotations.Activate -import org.osgi.service.component.annotations.Component import org.osgi.service.component.annotations.Reference import org.slf4j.LoggerFactory @Suppress("LongParameterList") -@Component(service = [FlowExecutor::class]) +// TODO @Component(service = [FlowExecutor::class]) class FlowExecutorImpl constructor( coordinatorFactory: LifecycleCoordinatorFactory, private val subscriptionFactory: SubscriptionFactory, diff --git a/components/flow/flow-service/src/main/kotlin/net/corda/flow/service/FlowExecutorMediatorImpl.kt b/components/flow/flow-service/src/main/kotlin/net/corda/flow/service/FlowExecutorMediatorImpl.kt index f9d2ded2eec..c87733be2ed 100644 --- a/components/flow/flow-service/src/main/kotlin/net/corda/flow/service/FlowExecutorMediatorImpl.kt +++ b/components/flow/flow-service/src/main/kotlin/net/corda/flow/service/FlowExecutorMediatorImpl.kt @@ -31,11 +31,13 @@ import net.corda.schema.configuration.MessagingConfig.MAX_ALLOWED_MSG_SIZE import net.corda.schema.configuration.MessagingConfig.Subscription.PROCESSOR_TIMEOUT import net.corda.utilities.trace import org.osgi.service.component.annotations.Activate +import org.osgi.service.component.annotations.Component import org.osgi.service.component.annotations.Reference import org.slf4j.LoggerFactory @Suppress("LongParameterList") // TODO @Component(service = [FlowExecutor::class]) +@Component(service = [FlowExecutor::class]) class FlowExecutorMediatorImpl ( coordinatorFactory: LifecycleCoordinatorFactory, private val flowEventProcessorFactory: FlowEventProcessorFactory, diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt index e7c0a217d6b..2d7a381b78b 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt @@ -174,8 +174,8 @@ class MultiSourceEventMediatorImpl( conflictingStates ) } while (msgProcessorTasks.isNotEmpty()) + commitOffsets() } - commitOffsets() } private fun poll(pollTimeoutInNanos: Long): List> { diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/statemanager/StateManagerImpl.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/statemanager/StateManagerImpl.kt index 1701e215923..844b5a04811 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/statemanager/StateManagerImpl.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/statemanager/StateManagerImpl.kt @@ -9,16 +9,28 @@ import java.time.Instant @Component(service = [StateManager::class]) class StateManagerImpl @Activate constructor() : StateManager { + private val storage = mutableMapOf() + override fun create(states: Collection): Map { - TODO("Not yet implemented") + return states.mapNotNull { + storage.putIfAbsent(it.key, it) + }.associate { it.key to RuntimeException("State already exists [$it]") } } override fun get(keys: Collection): Map { - TODO("Not yet implemented") + return keys.mapNotNull { storage[it] }.associateBy { it.key } } override fun update(states: Collection): Map { - TODO("Not yet implemented") + return states.mapNotNull { + val existingState = storage[it.key] + if (existingState?.version == it.version) { + storage[it.key] = it + null + } else { + it + } + }.associateBy { it.key } } override fun delete(states: Collection): Map { diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/taskmanager/TaskManagerImpl.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/taskmanager/TaskManagerImpl.kt index a86e0661998..a0cca134420 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/taskmanager/TaskManagerImpl.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/taskmanager/TaskManagerImpl.kt @@ -4,11 +4,42 @@ import net.corda.messaging.api.mediator.taskmanager.TaskManager import net.corda.messaging.api.mediator.taskmanager.TaskType import org.osgi.service.component.annotations.Activate import org.osgi.service.component.annotations.Component +import java.util.UUID import java.util.concurrent.CompletableFuture +import java.util.concurrent.Executors +import kotlin.concurrent.thread + @Component(service = [TaskManager::class]) class TaskManagerImpl @Activate constructor() : TaskManager { - override fun execute(type: TaskType, command: () -> T): CompletableFuture { - TODO("Not yet implemented") + private var executorService = Executors.newSingleThreadExecutor() + + override fun execute(type: TaskType, command: () -> T) = + when (type) { + TaskType.SHORT_RUNNING -> executeShortRunning(command) + TaskType.LONG_RUNNING -> executeLongRunning(command) + } + + private fun executeShortRunning(command: () -> T): CompletableFuture { + val result = CompletableFuture() + executorService.execute { + result.complete(command()) + } + return result + } + + private fun executeLongRunning(command: () -> T): CompletableFuture { + val uniqueId = UUID.randomUUID() + val result = CompletableFuture() + thread( + start = true, + isDaemon = true, + contextClassLoader = null, + name = "Task Manager - $uniqueId", + priority = -1, + ) { + result.complete(command()) + } + return result } } \ No newline at end of file diff --git a/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/ProcessorTaskTest.kt b/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/ProcessorTaskTest.kt new file mode 100644 index 00000000000..65e5eb1e90a --- /dev/null +++ b/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/ProcessorTaskTest.kt @@ -0,0 +1,102 @@ +package net.corda.messaging.mediator + +import kotlinx.coroutines.runBlocking +import net.corda.messagebus.api.consumer.CordaConsumerRecord +import net.corda.messagebus.api.producer.CordaProducer +import net.corda.messagebus.api.producer.CordaProducerRecord +import net.corda.messaging.api.mediator.MediatorMessage +import net.corda.messaging.api.mediator.statemanager.State +import net.corda.messaging.api.processor.StateAndEventProcessor +import net.corda.v5.base.exceptions.CordaRuntimeException +import org.junit.jupiter.api.BeforeEach +import org.junit.jupiter.api.Test +import org.junit.jupiter.api.assertThrows +import org.mockito.Mockito.doThrow +import org.mockito.Mockito.times +import org.mockito.kotlin.any +import org.mockito.kotlin.eq +import org.mockito.kotlin.mock +import org.mockito.kotlin.verify +import org.mockito.kotlin.whenever +import java.time.Instant + +class ProcessorTaskTest { + companion object { + private const val TOPIC = "topic" + } + + private lateinit var cordaProducer: CordaProducer + private lateinit var mediatorProducer: MessageBusProducer + + private val defaultHeaders: List> = emptyList() + private val messageProps: MutableMap = mutableMapOf( + "key" to "key", + "headers" to defaultHeaders + ) + private val message: MediatorMessage = MediatorMessage("value", messageProps) + + + @BeforeEach + fun setup() { + cordaProducer = mock() + mediatorProducer = MessageBusProducer("client-id", cordaProducer) + } + + @Test + fun `successfully processes messages without initial state`() { + val key = "key" + val persistedSate: State? = null + val events = listOf("event1", "event2", "event3") + .map { + CordaConsumerRecord( + topic = "", + partition = -1, + offset = -1, + key = key, + value = it, + timestamp = Instant.now().toEpochMilli() + ) + } + val processor = mock>() + val mediatorStateManager = mock>() + val task = ProcessorTask( + key, + persistedSate, + events, + processor, + mediatorStateManager + ) + + val result = task.call() + + val expected = CordaProducerRecord( + TOPIC, + message.getProperty("key"), + message.payload + ) + + verify(cordaProducer).send(eq(expected), any()) + } + + @Test + fun testSendWithError() { + val record = CordaProducerRecord( + TOPIC, + message.getProperty("key"), + message.payload + ) + + doThrow(CordaRuntimeException("")).whenever(cordaProducer).send(eq(record), any()) + assertThrows { + runBlocking { + mediatorProducer.send(message, TOPIC).await() + } + } + } + + @Test + fun testClose() { + mediatorProducer.close() + verify(cordaProducer, times(1)).close() + } +} \ No newline at end of file diff --git a/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/factory/MediatorConsumerFactoryFactoryTest.kt b/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/factory/MediatorConsumerFactoryFactoryTest.kt index 3cff6f517fb..8a856f3d11a 100644 --- a/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/factory/MediatorConsumerFactoryFactoryTest.kt +++ b/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/factory/MediatorConsumerFactoryFactoryTest.kt @@ -2,7 +2,6 @@ package net.corda.messaging.mediator.factory import net.corda.libs.configuration.SmartConfig import net.corda.messagebus.api.consumer.builder.CordaConsumerBuilder -import net.corda.messaging.api.mediator.factory.MediatorConsumerFactory import org.junit.jupiter.api.Assertions import org.junit.jupiter.api.BeforeEach import org.junit.jupiter.api.Test From e91df9daa27b6361560cd4d5593fa0902027030f Mon Sep 17 00:00:00 2001 From: Miljenko Brkic Date: Fri, 22 Sep 2023 11:42:01 +0100 Subject: [PATCH 31/50] CORE-16199 Renaming MediatorProducer to MessagingClient --- .../flow/service/FlowExecutorMediatorImpl.kt | 26 +++---- .../{ProducerTask.kt => ClientTask.kt} | 12 +-- .../mediator/MediatorComponentFactory.kt | 36 ++++----- .../messaging/mediator/MediatorTaskManager.kt | 44 +++++------ .../messaging/mediator/MessageBusClient.kt | 8 +- .../messaging/mediator/MessageBusProducer.kt | 52 ------------- .../mediator/MultiSourceEventMediatorImpl.kt | 20 ++--- ...rFactory.kt => MessageBusClientFactory.kt} | 20 ++--- ...t => MessagingClientFactoryFactoryImpl.kt} | 14 ++-- .../mediator/MessageBusProducerTest.kt | 75 ------------------- ...Test.kt => MessageBusClientFactoryTest.kt} | 14 ++-- ...t => MessagingClientFactoryFactoryTest.kt} | 14 ++-- .../MultiSourceEventMediatorFactoryTest.kt | 6 +- .../api/mediator/MediatorProducer.kt | 23 ------ .../messaging/api/mediator/MessageRouter.kt | 6 +- .../messaging/api/mediator/MessagingClient.kt | 3 +- .../api/mediator/MultiSourceEventMediator.kt | 2 +- .../api/mediator/RoutingDestination.kt | 8 +- .../mediator/config/EventMediatorConfig.kt | 6 +- .../config/EventMediatorConfigBuilder.kt | 14 ++-- ...ucerConfig.kt => MessagingClientConfig.kt} | 6 +- .../factory/MediatorProducerFactory.kt | 17 ----- .../factory/MediatorProducerFactoryFactory.kt | 20 ----- .../factory/MediatorProducerFinder.kt | 15 ---- .../mediator/factory/MessageRouterFactory.kt | 14 ++-- .../factory/MessagingClientFactory.kt | 17 +++++ .../factory/MessagingClientFactoryFactory.kt | 20 +++++ .../mediator/factory/MessagingClientFinder.kt | 15 ++++ 28 files changed, 189 insertions(+), 338 deletions(-) rename libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/{ProducerTask.kt => ClientTask.kt} (68%) delete mode 100644 libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MessageBusProducer.kt rename libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/{MessageBusProducerFactory.kt => MessageBusClientFactory.kt} (73%) rename libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/{MediatorProducerFactoryFactoryImpl.kt => MessagingClientFactoryFactoryImpl.kt} (60%) delete mode 100644 libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/MessageBusProducerTest.kt rename libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/factory/{MessageBusProducerFactoryTest.kt => MessageBusClientFactoryTest.kt} (71%) rename libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/factory/{MediatorProducerFactoryFactoryTest.kt => MessagingClientFactoryFactoryTest.kt} (55%) delete mode 100644 libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MediatorProducer.kt rename libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/config/{MediatorProducerConfig.kt => MessagingClientConfig.kt} (55%) delete mode 100644 libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MediatorProducerFactory.kt delete mode 100644 libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MediatorProducerFactoryFactory.kt delete mode 100644 libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MediatorProducerFinder.kt create mode 100644 libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MessagingClientFactory.kt create mode 100644 libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MessagingClientFactoryFactory.kt create mode 100644 libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MessagingClientFinder.kt diff --git a/components/flow/flow-service/src/main/kotlin/net/corda/flow/service/FlowExecutorMediatorImpl.kt b/components/flow/flow-service/src/main/kotlin/net/corda/flow/service/FlowExecutorMediatorImpl.kt index c87733be2ed..9bb4ca2cf7a 100644 --- a/components/flow/flow-service/src/main/kotlin/net/corda/flow/service/FlowExecutorMediatorImpl.kt +++ b/components/flow/flow-service/src/main/kotlin/net/corda/flow/service/FlowExecutorMediatorImpl.kt @@ -19,8 +19,8 @@ import net.corda.messaging.api.mediator.MultiSourceEventMediator import net.corda.messaging.api.mediator.RoutingDestination.Companion.routeTo import net.corda.messaging.api.mediator.config.EventMediatorConfigBuilder import net.corda.messaging.api.mediator.factory.MediatorConsumerFactoryFactory -import net.corda.messaging.api.mediator.factory.MediatorProducerFactoryFactory import net.corda.messaging.api.mediator.factory.MessageRouterFactory +import net.corda.messaging.api.mediator.factory.MessagingClientFactoryFactory import net.corda.messaging.api.mediator.factory.MultiSourceEventMediatorFactory import net.corda.messaging.api.processor.StateAndEventProcessor import net.corda.schema.Schemas.Flow.FLOW_EVENT_TOPIC @@ -43,7 +43,7 @@ class FlowExecutorMediatorImpl ( private val flowEventProcessorFactory: FlowEventProcessorFactory, private val eventMediatorFactory: MultiSourceEventMediatorFactory, private val mediatorConsumerFactoryFactory: MediatorConsumerFactoryFactory, - private val mediatorProducerFactoryFactory: MediatorProducerFactoryFactory, + private val messagingClientFactoryFactory: MessagingClientFactoryFactory, private val toMessagingConfig: (Map) -> SmartConfig ) : FlowExecutor { @@ -57,21 +57,21 @@ class FlowExecutorMediatorImpl ( eventMediatorFactory: MultiSourceEventMediatorFactory, @Reference(service = MediatorConsumerFactoryFactory::class) mediatorConsumerFactoryFactory: MediatorConsumerFactoryFactory, - @Reference(service = MediatorProducerFactoryFactory::class) - mediatorProducerFactoryFactory: MediatorProducerFactoryFactory, + @Reference(service = MessagingClientFactoryFactory::class) + messagingClientFactoryFactory: MessagingClientFactoryFactory, ) : this( coordinatorFactory, flowEventProcessorFactory, eventMediatorFactory, mediatorConsumerFactoryFactory, - mediatorProducerFactoryFactory, + messagingClientFactoryFactory, { cfg -> cfg.getConfig(MESSAGING_CONFIG) } ) companion object { private val log = LoggerFactory.getLogger(this::class.java.enclosingClass) private const val CONSUMER_GROUP = "FlowEventConsumer" - private const val MESSAGE_BUS_PRODUCER = "MessageBusProducer" + private const val MESSAGE_BUS_CLIENT = "MessageBusClient" } private val coordinator = coordinatorFactory.createCoordinator { event, _ -> eventHandler(event) } @@ -160,22 +160,22 @@ class FlowExecutorMediatorImpl ( FLOW_EVENT_TOPIC, CONSUMER_GROUP, messagingConfig ), ) - .producerFactories( - mediatorProducerFactoryFactory.createMessageBusProducerFactory( - MESSAGE_BUS_PRODUCER, messagingConfig + .clientFactories( + messagingClientFactoryFactory.createMessageBusClientFactory( + MESSAGE_BUS_CLIENT, messagingConfig ), - //RpcProducerFactory(CRYPTO_RPC_PRODUCER, messagingConfig, cordaRpcBuilder), + //RpcClientFactory(CRYPTO_RPC_CLIENT, messagingConfig, cordaRpcBuilder), ) .messageProcessor(messageProcessor) .messageRouterFactory(createMessageRouterFactory()) .build() - private fun createMessageRouterFactory() = MessageRouterFactory { producerFinder -> - val messageBusProducer = producerFinder.find(MESSAGE_BUS_PRODUCER) + private fun createMessageRouterFactory() = MessageRouterFactory { clientFinder -> + val messageBusClient = clientFinder.find(MESSAGE_BUS_CLIENT) MessageRouter { message -> when (message.payload) { - is FlowMapperEvent -> routeTo(messageBusProducer, FLOW_MAPPER_EVENT_TOPIC) + is FlowMapperEvent -> routeTo(messageBusClient, FLOW_MAPPER_EVENT_TOPIC) else -> throw IllegalStateException("No route defined for message $message") } } diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/ProducerTask.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/ClientTask.kt similarity index 68% rename from libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/ProducerTask.kt rename to libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/ClientTask.kt index ab00d23a885..176cfe9adbc 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/ProducerTask.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/ClientTask.kt @@ -2,21 +2,21 @@ package net.corda.messaging.mediator import kotlinx.coroutines.runBlocking import net.corda.messaging.api.mediator.MediatorMessage -import net.corda.messaging.api.mediator.MediatorProducer import net.corda.messaging.api.mediator.MessageRouter +import net.corda.messaging.api.mediator.MessagingClient import java.util.concurrent.Callable /** - * [ProducerTask] sends a [MediatorMessage] to [MediatorProducer] selected by [MessageRouter]. + * [ClientTask] sends a [MediatorMessage] to [MessagingClient] selected by [MessageRouter]. */ -class ProducerTask( +class ClientTask( private val message: MediatorMessage, private val messageRouter: MessageRouter, val processorTask: ProcessorTask, -): Callable> { +): Callable> { class Result( - val producerTask: ProducerTask, + val clientTask: ClientTask, val replyMessage: MediatorMessage?, ) { fun hasReply() = replyMessage != null @@ -26,7 +26,7 @@ class ProducerTask( val destination = messageRouter.getDestination(message) @Suppress("UNCHECKED_CAST") val reply = runBlocking { - with(destination) { producer.send(message, endpoint).await() } + with(destination) { client.send(message, endpoint).await() } } as MediatorMessage? return Result(this, reply) } diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MediatorComponentFactory.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MediatorComponentFactory.kt index c1dfdfc6099..a36965d34fd 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MediatorComponentFactory.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MediatorComponentFactory.kt @@ -1,13 +1,13 @@ package net.corda.messaging.mediator import net.corda.messaging.api.mediator.MediatorConsumer -import net.corda.messaging.api.mediator.MediatorProducer import net.corda.messaging.api.mediator.MessageRouter +import net.corda.messaging.api.mediator.MessagingClient import net.corda.messaging.api.mediator.config.MediatorConsumerConfig -import net.corda.messaging.api.mediator.config.MediatorProducerConfig +import net.corda.messaging.api.mediator.config.MessagingClientConfig import net.corda.messaging.api.mediator.factory.MediatorConsumerFactory -import net.corda.messaging.api.mediator.factory.MediatorProducerFactory import net.corda.messaging.api.mediator.factory.MessageRouterFactory +import net.corda.messaging.api.mediator.factory.MessagingClientFactory import net.corda.messaging.api.processor.StateAndEventProcessor /** @@ -16,7 +16,7 @@ import net.corda.messaging.api.processor.StateAndEventProcessor internal class MediatorComponentFactory( private val messageProcessor: StateAndEventProcessor, private val consumerFactories: Collection, - private val producerFactories: Collection, + private val clientFactories: Collection, private val messageRouterFactory: MessageRouterFactory, ) { @@ -44,20 +44,20 @@ internal class MediatorComponentFactory( } /** - * Creates message producers. + * Creates messaging clients. * * @param onSerializationError Function for handling serialization errors. - * @return List of created [MediatorProducer]s. + * @return List of created [MessagingClient]s. */ - fun createProducers( + fun createClients( onSerializationError: (ByteArray) -> Unit - ): List { - check (producerFactories.isNotEmpty()) { - "None producer factory set in configuration" + ): List { + check (clientFactories.isNotEmpty()) { + "None client factory set in configuration" } - return producerFactories.map { producerFactory -> - producerFactory.create( - MediatorProducerConfig(onSerializationError) + return clientFactories.map { clientFactory -> + clientFactory.create( + MessagingClientConfig(onSerializationError) ) } } @@ -65,16 +65,16 @@ internal class MediatorComponentFactory( /** * Creates message router. * - * @param producers Collection of [MediatorProducer]s. + * @param clients Collection of [MessagingClient]s. * @return Message router. */ fun createRouter( - producers: Collection + clients: Collection ): MessageRouter { - val producersById = producers.associateBy { it.id } + val clientsById = clients.associateBy { it.id } return messageRouterFactory.create { id -> - producersById[id] - ?: throw IllegalStateException("Producer with ID \"$id\" not found") + clientsById[id] + ?: throw IllegalStateException("Messaging client with ID \"$id\" not found") } } } \ No newline at end of file diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MediatorTaskManager.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MediatorTaskManager.kt index ad472dbb60a..412e4e2739c 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MediatorTaskManager.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MediatorTaskManager.kt @@ -46,21 +46,21 @@ internal class MediatorTaskManager( } /** - * Creates [ProcessorTask]s from [ProducerTask.Result]s that have reply message set. Reply messages are + * Creates [ProcessorTask]s from [ClientTask.Result]s that have reply message set. Reply messages are * grouped by message keys. The latest updated state from related [ProcessorTask] is used as the input state to * state and event processor. * - * @param producerResults List of results of [ProducerTask]s. + * @param clientResults List of results of [ClientTask]s. * @return Created [ProcessorTask]s. */ fun createMsgProcessorTasks( - producerResults: List>, + clientResults: List>, ): List> { - return producerResults.filter { it.hasReply() } - .groupBy { it.producerTask.processorTask.persistedState!!.key } - .map { (_, producerTaskResults) -> - val messageGroup = producerTaskResults.map { it.toCordaConsumerRecord() } - producerTaskResults.first().producerTask.processorTask.copy(events = messageGroup) + return clientResults.filter { it.hasReply() } + .groupBy { it.clientTask.processorTask.persistedState!!.key } + .map { (_, clientTaskResults) -> + val messageGroup = clientTaskResults.map { it.toCordaConsumerRecord() } + clientTaskResults.first().clientTask.processorTask.copy(events = messageGroup) } } @@ -99,21 +99,21 @@ internal class MediatorTaskManager( } /** - * Creates [ProducerTask]s for given results of [ProcessorTask]s. Given [MessageRouter] is used to select producer - * for specific message. + * Creates [ClientTask]s for given results of [ProcessorTask]s. Given [MessageRouter] is used to select messaging + * client for specific message. * - * @param processorTaskResults Results of [ProducerTask]s. + * @param processorTaskResults Results of [ClientTask]s. * @param messageRouter Message router. - * @return Created [ProducerTask]s. + * @return Created [ClientTask]s. */ fun createProducerTasks( processorTaskResults: List>, messageRouter: MessageRouter, - ): List> { + ): List> { return processorTaskResults.map { result -> result.outputEvents.map { event -> val message = MediatorMessage(event.value!!) - ProducerTask( + ClientTask( message, messageRouter, result.processorTask @@ -123,33 +123,33 @@ internal class MediatorTaskManager( } /** - * Executes given [ProducerTask]s using [TaskManager] and waits for all to finish. + * Executes given [ClientTask]s using [TaskManager] and waits for all to finish. * - * @param producerTasks Tasks to execute. + * @param clientTasks Tasks to execute. * @return Result of task executions. */ fun executeProducerTasks( - producerTasks: Collection> - ): List> { + clientTasks: Collection> + ): List> { // return producerTasks.map { producerTask -> // taskManager.execute(TaskType.SHORT_RUNNING, producerTask::call) // }.map { // it.join() // } return runBlocking { - producerTasks.map { it.call() } + clientTasks.map { it.call() } } } /** - * Converts [ProducerTask.Result] to [CordaConsumerRecord]. + * Converts [ClientTask.Result] to [CordaConsumerRecord]. */ - private fun ProducerTask.Result.toCordaConsumerRecord() = + private fun ClientTask.Result.toCordaConsumerRecord() = (CordaConsumerRecord( "", -1, -1, - producerTask.processorTask.events.first().key, + clientTask.processorTask.events.first().key, replyMessage!!.payload, Instant.now().toEpochMilli() )) diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MessageBusClient.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MessageBusClient.kt index 1ad0afc7ca5..c81a4d80957 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MessageBusClient.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MessageBusClient.kt @@ -18,9 +18,9 @@ class MessageBusClient( private val log: Logger = LoggerFactory.getLogger(this::class.java.enclosingClass) } - override fun send(message: MediatorMessage<*>): Deferred?> = + override fun send(message: MediatorMessage<*>, endpoint: String): Deferred?> = CompletableDeferred?>().apply { - producer.send(message.toCordaProducerRecord()) { ex -> + producer.send(message.toCordaProducerRecord(endpoint)) { ex -> if (ex != null) { completeExceptionally(ex) } else { @@ -40,9 +40,9 @@ class MessageBusClient( } } -private fun MediatorMessage<*>.toCordaProducerRecord() : CordaProducerRecord<*, *> { +private fun MediatorMessage<*>.toCordaProducerRecord(endpoint: String) : CordaProducerRecord<*, *> { return CordaProducerRecord( - topic = this.getProperty("topic"), + topic = endpoint, key = this.getProperty("key"), value = this.payload, headers = this.getProperty("headers"), diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MessageBusProducer.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MessageBusProducer.kt deleted file mode 100644 index 02664cdfcc4..00000000000 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MessageBusProducer.kt +++ /dev/null @@ -1,52 +0,0 @@ -package net.corda.messaging.mediator - -import kotlinx.coroutines.CompletableDeferred -import kotlinx.coroutines.Deferred -import net.corda.messagebus.api.producer.CordaProducer -import net.corda.messagebus.api.producer.CordaProducerRecord -import net.corda.messaging.api.mediator.MediatorMessage -import net.corda.messaging.api.mediator.MediatorProducer -import org.slf4j.Logger -import org.slf4j.LoggerFactory - -class MessageBusProducer( - override val id: String, - private val producer: CordaProducer, -) : MediatorProducer { - - private companion object { - private val log: Logger = LoggerFactory.getLogger(this::class.java.enclosingClass) - } - - override fun send(message: MediatorMessage<*>, endpoint: String): Deferred?> = - CompletableDeferred?>().apply { - producer.send(message.toCordaProducerRecord(endpoint)) { ex -> - if (ex != null) { - completeExceptionally(ex) - } else { - complete(null) - } - } - } - - override fun close() { - try { - producer.close() - } catch (ex: Exception) { - log.info( - "Failed to close producer [$id] safely.", ex - ) - } - } -} - -private fun MediatorMessage<*>.toCordaProducerRecord(endpoint: String) : CordaProducerRecord<*, *> { - return CordaProducerRecord( - topic = endpoint, - key = this.getProperty("key"), - value = this.payload, - headers = this.getProperty("headers"), - ) -} - -private typealias Headers = List> \ No newline at end of file diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt index 2d7a381b78b..0b918fef552 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt @@ -10,8 +10,8 @@ import net.corda.messagebus.api.consumer.CordaConsumerRecord import net.corda.messaging.api.exception.CordaMessageAPIFatalException import net.corda.messaging.api.exception.CordaMessageAPIIntermittentException import net.corda.messaging.api.mediator.MediatorConsumer -import net.corda.messaging.api.mediator.MediatorProducer import net.corda.messaging.api.mediator.MessageRouter +import net.corda.messaging.api.mediator.MessagingClient import net.corda.messaging.api.mediator.MultiSourceEventMediator import net.corda.messaging.api.mediator.config.EventMediatorConfig import net.corda.messaging.api.mediator.statemanager.StateManager @@ -35,10 +35,10 @@ class MultiSourceEventMediatorImpl( private val log = LoggerFactory.getLogger("${this.javaClass.name}-${config.name}") private var consumers = listOf>() - private var producers = listOf() + private var clients = listOf() private lateinit var messageRouter: MessageRouter private val mediatorComponentFactory = MediatorComponentFactory( - config.messageProcessor, config.consumerFactories, config.producerFactories, config.messageRouterFactory + config.messageProcessor, config.consumerFactories, config.clientFactories, config.messageRouterFactory ) private val mediatorStateManager = MediatorStateManager( stateManager, serializer, stateDeserializer @@ -81,8 +81,8 @@ class MultiSourceEventMediatorImpl( attempts++ try { consumers = mediatorComponentFactory.createConsumers(::onSerializationError) - producers = mediatorComponentFactory.createProducers(::onSerializationError) - messageRouter = mediatorComponentFactory.createRouter(producers) + clients = mediatorComponentFactory.createClients(::onSerializationError) + messageRouter = mediatorComponentFactory.createRouter(clients) consumers.forEach{ it.subscribe() } lifecycleCoordinator.updateStatus(LifecycleStatus.UP) @@ -98,7 +98,7 @@ class MultiSourceEventMediatorImpl( } is CordaMessageAPIIntermittentException -> { log.warn( - "${ex.message} Attempts: $attempts. Recreating consumers/producers and Retrying.", ex + "${ex.message} Attempts: $attempts. Recreating consumers/clients and Retrying.", ex ) } else -> { @@ -123,7 +123,7 @@ class MultiSourceEventMediatorImpl( private fun closeConsumersAndProducers() { consumers.forEach { it.close() } - producers.forEach { it.close() } + clients.forEach { it.close() } } private fun processEventsWithRetries() { @@ -166,10 +166,10 @@ class MultiSourceEventMediatorImpl( val (validResults, invalidResults) = processingResults.partition { !conflictingStates.contains(it.key) } - val producerTasks = mediatorTaskManager.createProducerTasks(validResults, messageRouter) - val producerResults = mediatorTaskManager.executeProducerTasks(producerTasks) + val clientTasks = mediatorTaskManager.createProducerTasks(validResults, messageRouter) + val clientResults = mediatorTaskManager.executeProducerTasks(clientTasks) msgProcessorTasks = - mediatorTaskManager.createMsgProcessorTasks(producerResults) + mediatorTaskManager.createMsgProcessorTasks( + mediatorTaskManager.createMsgProcessorTasks(clientResults) + mediatorTaskManager.createMsgProcessorTasks( invalidResults, conflictingStates ) diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/MessageBusProducerFactory.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/MessageBusClientFactory.kt similarity index 73% rename from libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/MessageBusProducerFactory.kt rename to libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/MessageBusClientFactory.kt index 93f8dcd3593..01ca9c87616 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/MessageBusProducerFactory.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/MessageBusClientFactory.kt @@ -5,27 +5,27 @@ import net.corda.messagebus.api.configuration.ProducerConfig import net.corda.messagebus.api.constants.ProducerRoles import net.corda.messagebus.api.producer.CordaProducer import net.corda.messagebus.api.producer.builder.CordaProducerBuilder -import net.corda.messaging.api.mediator.MediatorProducer -import net.corda.messaging.api.mediator.config.MediatorProducerConfig -import net.corda.messaging.api.mediator.factory.MediatorProducerFactory -import net.corda.messaging.mediator.MessageBusProducer +import net.corda.messaging.api.mediator.MessagingClient +import net.corda.messaging.api.mediator.config.MessagingClientConfig +import net.corda.messaging.api.mediator.factory.MessagingClientFactory +import net.corda.messaging.mediator.MessageBusClient import net.corda.schema.configuration.BootConfig import java.util.UUID /** - * Factory for creating multi-source event mediator message bus producers. + * Factory for creating multi-source event mediator message bus messaging clients. * - * @param id Producer's unique ID. + * @param id Messaging client's unique ID. * @param messageBusConfig Message bus related configuration. * @param cordaProducerBuilder [CordaProducer] builder. */ -class MessageBusProducerFactory( +class MessageBusClientFactory( private val id: String, private val messageBusConfig: SmartConfig, private val cordaProducerBuilder: CordaProducerBuilder, -): MediatorProducerFactory { +): MessagingClientFactory { - override fun create(config: MediatorProducerConfig): MediatorProducer { + override fun create(config: MessagingClientConfig): MessagingClient { val uniqueId = UUID.randomUUID().toString() val clientId = "$id--$uniqueId" @@ -43,7 +43,7 @@ class MessageBusProducerFactory( config.onSerializationError ) - return MessageBusProducer( + return MessageBusClient( id, eventProducer, ) diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/MediatorProducerFactoryFactoryImpl.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/MessagingClientFactoryFactoryImpl.kt similarity index 60% rename from libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/MediatorProducerFactoryFactoryImpl.kt rename to libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/MessagingClientFactoryFactoryImpl.kt index cce092169ae..540e01d7240 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/MediatorProducerFactoryFactoryImpl.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/MessagingClientFactoryFactoryImpl.kt @@ -2,23 +2,23 @@ package net.corda.messaging.mediator.factory import net.corda.libs.configuration.SmartConfig import net.corda.messagebus.api.producer.builder.CordaProducerBuilder -import net.corda.messaging.api.mediator.factory.MediatorProducerFactoryFactory +import net.corda.messaging.api.mediator.factory.MessagingClientFactoryFactory import org.osgi.service.component.annotations.Activate import org.osgi.service.component.annotations.Component import org.osgi.service.component.annotations.Reference /** - * Factory for creating multi-source event mediator producers. + * Factory for creating multi-source event mediator messaging clients. */ -@Component(service = [MediatorProducerFactoryFactory::class]) -class MediatorProducerFactoryFactoryImpl @Activate constructor( +@Component(service = [MessagingClientFactoryFactory::class]) +class MessagingClientFactoryFactoryImpl @Activate constructor( @Reference(service = CordaProducerBuilder::class) private val cordaProducerBuilder: CordaProducerBuilder, -): MediatorProducerFactoryFactory { - override fun createMessageBusProducerFactory( +): MessagingClientFactoryFactory { + override fun createMessageBusClientFactory( id: String, messageBusConfig: SmartConfig, - ) = MessageBusProducerFactory( + ) = MessageBusClientFactory( id, messageBusConfig, cordaProducerBuilder, diff --git a/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/MessageBusProducerTest.kt b/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/MessageBusProducerTest.kt deleted file mode 100644 index 1e129b0dbc8..00000000000 --- a/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/MessageBusProducerTest.kt +++ /dev/null @@ -1,75 +0,0 @@ -package net.corda.messaging.mediator - -import kotlinx.coroutines.runBlocking -import net.corda.messagebus.api.producer.CordaProducer -import net.corda.messagebus.api.producer.CordaProducerRecord -import net.corda.messaging.api.mediator.MediatorMessage -import net.corda.v5.base.exceptions.CordaRuntimeException -import org.junit.jupiter.api.BeforeEach -import org.junit.jupiter.api.Test -import org.junit.jupiter.api.assertThrows -import org.mockito.Mockito.doThrow -import org.mockito.Mockito.times -import org.mockito.kotlin.any -import org.mockito.kotlin.eq -import org.mockito.kotlin.mock -import org.mockito.kotlin.verify -import org.mockito.kotlin.whenever - -class MessageBusProducerTest { - companion object { - private const val TOPIC = "topic" - } - - private lateinit var cordaProducer: CordaProducer - private lateinit var mediatorProducer: MessageBusProducer - - private val defaultHeaders: List> = emptyList() - private val messageProps: MutableMap = mutableMapOf( - "key" to "key", - "headers" to defaultHeaders - ) - private val message: MediatorMessage = MediatorMessage("value", messageProps) - - - @BeforeEach - fun setup() { - cordaProducer = mock() - mediatorProducer = MessageBusProducer("client-id", cordaProducer) - } - - @Test - fun testSend() { - mediatorProducer.send(message, TOPIC) - - val expected = CordaProducerRecord( - TOPIC, - message.getProperty("key"), - message.payload - ) - - verify(cordaProducer).send(eq(expected), any()) - } - - @Test - fun testSendWithError() { - val record = CordaProducerRecord( - TOPIC, - message.getProperty("key"), - message.payload - ) - - doThrow(CordaRuntimeException("")).whenever(cordaProducer).send(eq(record), any()) - assertThrows { - runBlocking { - mediatorProducer.send(message, TOPIC).await() - } - } - } - - @Test - fun testClose() { - mediatorProducer.close() - verify(cordaProducer, times(1)).close() - } -} \ No newline at end of file diff --git a/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/factory/MessageBusProducerFactoryTest.kt b/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/factory/MessageBusClientFactoryTest.kt similarity index 71% rename from libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/factory/MessageBusProducerFactoryTest.kt rename to libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/factory/MessageBusClientFactoryTest.kt index c23d955bd80..8a19b1a0a5c 100644 --- a/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/factory/MessageBusProducerFactoryTest.kt +++ b/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/factory/MessageBusClientFactoryTest.kt @@ -3,7 +3,7 @@ package net.corda.messaging.mediator.factory import net.corda.libs.configuration.SmartConfig import net.corda.messagebus.api.producer.CordaProducer import net.corda.messagebus.api.producer.builder.CordaProducerBuilder -import net.corda.messaging.api.mediator.config.MediatorProducerConfig +import net.corda.messaging.api.mediator.config.MessagingClientConfig import org.junit.jupiter.api.Assertions import org.junit.jupiter.api.BeforeEach import org.junit.jupiter.api.Test @@ -12,8 +12,8 @@ import org.mockito.kotlin.anyOrNull import org.mockito.kotlin.doReturn import org.mockito.kotlin.mock -class MessageBusProducerFactoryTest { - private lateinit var messageBusProducerFactory: MessageBusProducerFactory +class MessageBusClientFactoryTest { + private lateinit var messageBusClientFactory: MessageBusClientFactory private val cordaProducerBuilder = mock() private val cordaProducer = mock() private val messageBusConfig = mock() @@ -23,8 +23,8 @@ class MessageBusProducerFactoryTest { doReturn(cordaProducer).`when`(cordaProducerBuilder).createProducer( any(), any(), anyOrNull() ) - messageBusProducerFactory = MessageBusProducerFactory( - "MessageBusProducer1", + messageBusClientFactory = MessageBusClientFactory( + "MessageBusClient1", messageBusConfig, cordaProducerBuilder, ) @@ -32,8 +32,8 @@ class MessageBusProducerFactoryTest { @Test fun testCreateMessageBusProducer() { - val config = MediatorProducerConfig {} - val messageBusProducer = messageBusProducerFactory.create(config) + val config = MessagingClientConfig {} + val messageBusProducer = messageBusClientFactory.create(config) Assertions.assertNotNull(messageBusProducer) } } \ No newline at end of file diff --git a/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/factory/MediatorProducerFactoryFactoryTest.kt b/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/factory/MessagingClientFactoryFactoryTest.kt similarity index 55% rename from libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/factory/MediatorProducerFactoryFactoryTest.kt rename to libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/factory/MessagingClientFactoryFactoryTest.kt index 532ab6a795d..e4a68753e9e 100644 --- a/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/factory/MediatorProducerFactoryFactoryTest.kt +++ b/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/factory/MessagingClientFactoryFactoryTest.kt @@ -7,25 +7,25 @@ import org.junit.jupiter.api.BeforeEach import org.junit.jupiter.api.Test import org.mockito.kotlin.mock -class MediatorProducerFactoryFactoryTest { - private lateinit var mediatorProducerFactoryFactory: MediatorProducerFactoryFactoryImpl +class MessagingClientFactoryFactoryTest { + private lateinit var messagingClientFactoryFactory: MessagingClientFactoryFactoryImpl private val cordaProducerBuilder = mock() private val messageBusConfig = mock() @BeforeEach fun beforeEach() { - mediatorProducerFactoryFactory = MediatorProducerFactoryFactoryImpl( + messagingClientFactoryFactory = MessagingClientFactoryFactoryImpl( cordaProducerBuilder, ) } @Test - fun testCreateMessageBusProducerFactory() { - val messageBusProducerFactory = mediatorProducerFactoryFactory.createMessageBusProducerFactory( - "MessageBusProducer1", + fun testCreateMessageBusClientFactory() { + val messageBusClientFactory = messagingClientFactoryFactory.createMessageBusClientFactory( + "MessageBusClient1", messageBusConfig, ) - Assertions.assertNotNull(messageBusProducerFactory) + Assertions.assertNotNull(messageBusClientFactory) } } \ No newline at end of file diff --git a/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/factory/MultiSourceEventMediatorFactoryTest.kt b/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/factory/MultiSourceEventMediatorFactoryTest.kt index 278bfa91df4..61943e8014a 100644 --- a/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/factory/MultiSourceEventMediatorFactoryTest.kt +++ b/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/factory/MultiSourceEventMediatorFactoryTest.kt @@ -36,7 +36,7 @@ class MultiSourceEventMediatorFactoryTest { } @Test - fun testMultiSourceEventMediator() { + fun testCreateMultiSourceEventMediator() { val messageProcessor = mock>() doReturn(Any::class.java).`when`(messageProcessor).stateValueClass val messageRouterFactory = mock() @@ -44,8 +44,8 @@ class MultiSourceEventMediatorFactoryTest { doReturn(messageProcessor).`when`(config).messageProcessor doReturn(messageRouterFactory).`when`(config).messageRouterFactory - val messageBusProducer = multiSourceEventMediatorFactory.create(config) + val mediator = multiSourceEventMediatorFactory.create(config) - Assertions.assertNotNull(messageBusProducer) + Assertions.assertNotNull(mediator) } } \ No newline at end of file diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MediatorProducer.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MediatorProducer.kt deleted file mode 100644 index c333271e008..00000000000 --- a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MediatorProducer.kt +++ /dev/null @@ -1,23 +0,0 @@ -package net.corda.messaging.api.mediator - -import kotlinx.coroutines.Deferred - -/** - * Multi-source event mediator message producer. - */ -interface MediatorProducer : AutoCloseable { - /** - * Producer's unique ID. - */ - val id: String - - /** - * Asynchronously sends a generic [MediatorMessage], and returns any result/error through a [Deferred] response. - * - * @param message The [MediatorMessage] to send. - * @param endpoint Endpoint to which the message is sent to. - * @return [Deferred] instance representing the asynchronous computation result, or null if the destination doesn't - * provide a response. - * */ - fun send(message: MediatorMessage<*>, endpoint: String) : Deferred?> -} diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MessageRouter.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MessageRouter.kt index c9fd04fa21b..014cfdf1785 100644 --- a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MessageRouter.kt +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MessageRouter.kt @@ -1,13 +1,13 @@ package net.corda.messaging.api.mediator /** - * Multi-source event mediator's message router that routes messages generated by state and event processor to producers. - * Example: + * Multi-source event mediator's message router that routes messages generated by state and event processor to messaging + * clients. Example: * * ``` * MessageRouter { message -> * when (message.payload) { - * is FlowEvent -> RoutingDestination(messageBusProducer, "flow.event.topic") + * is FlowEvent -> RoutingDestination(messageBusClient, "flow.event.topic") * else -> throw IllegalStateException("No route defined for message $message") * } * } diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MessagingClient.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MessagingClient.kt index 92f7ce9b0e5..a4d13cbed4e 100644 --- a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MessagingClient.kt +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MessagingClient.kt @@ -15,8 +15,9 @@ interface MessagingClient : AutoCloseable { * Asynchronously sends a generic [MediatorMessage], and returns any result/error through a [Deferred] response. * * @param message The [MediatorMessage] to send. + * @param endpoint Endpoint to which the message is sent to. * @return [Deferred] instance representing the asynchronous computation result, or null if the destination doesn't * provide a response. * */ - fun send(message: MediatorMessage<*>) : Deferred?> + fun send(message: MediatorMessage<*>, endpoint: String) : Deferred?> } diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MultiSourceEventMediator.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MultiSourceEventMediator.kt index 1b0fe273260..411c6e1c864 100644 --- a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MultiSourceEventMediator.kt +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MultiSourceEventMediator.kt @@ -5,6 +5,6 @@ import net.corda.messaging.api.subscription.SubscriptionBase /** * Multi-source event mediator is used to consume messages from multiple sources using [MediatorConsumer]s, - * process them using [StateAndEventProcessor] to generate output messages that are then sent to [MediatorProducer]s. + * process them using [StateAndEventProcessor] to generate output messages that are then sent to [MessagingClient]s. */ interface MultiSourceEventMediator : SubscriptionBase \ No newline at end of file diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/RoutingDestination.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/RoutingDestination.kt index cdc0ca8dade..d7868fbb4f0 100644 --- a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/RoutingDestination.kt +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/RoutingDestination.kt @@ -1,14 +1,14 @@ package net.corda.messaging.api.mediator /** - * Routing destination encapsulate [MediatorProducer] and related data needed to send a [MediatorMessage]. + * Routing destination encapsulate [MessagingClient] and related data needed to send a [MediatorMessage]. */ data class RoutingDestination( - val producer: MediatorProducer, + val client: MessagingClient, val endpoint: String, ) { companion object { - fun routeTo(producer: MediatorProducer, endpoint: String) = - RoutingDestination(producer, endpoint) + fun routeTo(client: MessagingClient, endpoint: String) = + RoutingDestination(client, endpoint) } } diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/config/EventMediatorConfig.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/config/EventMediatorConfig.kt index 97c75d8cc6e..7c2fca77c6e 100644 --- a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/config/EventMediatorConfig.kt +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/config/EventMediatorConfig.kt @@ -3,8 +3,8 @@ package net.corda.messaging.api.mediator.config import net.corda.libs.configuration.SmartConfig import net.corda.messaging.api.mediator.MultiSourceEventMediator import net.corda.messaging.api.mediator.factory.MediatorConsumerFactory -import net.corda.messaging.api.mediator.factory.MediatorProducerFactory import net.corda.messaging.api.mediator.factory.MessageRouterFactory +import net.corda.messaging.api.mediator.factory.MessagingClientFactory import net.corda.messaging.api.processor.StateAndEventProcessor import net.corda.schema.configuration.MessagingConfig import java.time.Duration @@ -15,7 +15,7 @@ import java.time.Duration * @property name The unique name for a multi-source event mediator. * @property messagingConfig Messaging related configuration. * @property consumerFactories Factories for creating message consumers. - * @property producerFactories Factories for creating message producers. + * @property clientFactories Factories for creating messaging clients. * @property messageProcessor State and event processor. * @property messageRouterFactory Message router factory. */ @@ -23,7 +23,7 @@ data class EventMediatorConfig( val name: String, val messagingConfig : SmartConfig, val consumerFactories: Collection, - val producerFactories: Collection, + val clientFactories: Collection, val messageProcessor : StateAndEventProcessor, val messageRouterFactory: MessageRouterFactory, ) { diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/config/EventMediatorConfigBuilder.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/config/EventMediatorConfigBuilder.kt index 9b9011f6727..02acda5368c 100644 --- a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/config/EventMediatorConfigBuilder.kt +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/config/EventMediatorConfigBuilder.kt @@ -3,8 +3,8 @@ package net.corda.messaging.api.mediator.config import net.corda.libs.configuration.SmartConfig import net.corda.messaging.api.mediator.MultiSourceEventMediator import net.corda.messaging.api.mediator.factory.MediatorConsumerFactory -import net.corda.messaging.api.mediator.factory.MediatorProducerFactory import net.corda.messaging.api.mediator.factory.MessageRouterFactory +import net.corda.messaging.api.mediator.factory.MessagingClientFactory import net.corda.messaging.api.processor.StateAndEventProcessor /** @@ -15,7 +15,7 @@ class EventMediatorConfigBuilder { private var name : String? = null private var messagingConfig : SmartConfig? = null private var consumerFactories = emptyArray() - private var producerFactories = emptyArray() + private var clientFactories = emptyArray() private var messageProcessor : StateAndEventProcessor? = null private var messageRouterFactory: MessageRouterFactory? = null @@ -31,9 +31,9 @@ class EventMediatorConfigBuilder { fun consumerFactories(vararg consumerFactories: MediatorConsumerFactory) = apply { this.consumerFactories = arrayOf(*consumerFactories) } - /** Sets factories for creating message producers. */ - fun producerFactories(vararg producerFactories: MediatorProducerFactory) = - apply { this.producerFactories = arrayOf(*producerFactories) } + /** Sets factories for creating messaging clients. */ + fun clientFactories(vararg clientFactories: MessagingClientFactory) = + apply { this.clientFactories = arrayOf(*clientFactories) } /** Sets state and event processor for [MultiSourceEventMediator]. */ fun messageProcessor(messageProcessor: StateAndEventProcessor) = @@ -48,14 +48,14 @@ class EventMediatorConfigBuilder { check(name != null) { "Name not set" } check(messagingConfig != null) { "Messaging configuration not set" } check(consumerFactories.isNotEmpty()) { "At least on consumer factory has to be set" } - check(producerFactories.isNotEmpty()) { "At least on producer factory has to be set" } + check(clientFactories.isNotEmpty()) { "At least on messaging client factory has to be set" } check(messageProcessor != null) { "Message processor not set" } check(messageRouterFactory != null) { "Message router factory not set" } return EventMediatorConfig( name!!, messagingConfig!!, consumerFactories.asList(), - producerFactories.asList(), + clientFactories.asList(), messageProcessor!!, messageRouterFactory!! ) diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/config/MediatorProducerConfig.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/config/MessagingClientConfig.kt similarity index 55% rename from libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/config/MediatorProducerConfig.kt rename to libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/config/MessagingClientConfig.kt index eed07aac37c..f0e1a3456ed 100644 --- a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/config/MediatorProducerConfig.kt +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/config/MessagingClientConfig.kt @@ -1,12 +1,12 @@ package net.corda.messaging.api.mediator.config -import net.corda.messaging.api.mediator.MediatorProducer +import net.corda.messaging.api.mediator.MessagingClient /** - * Class to store configuration for [MediatorProducer]. + * Class to store configuration for [MessagingClient]. * * @property onSerializationError Handler for serialization errors. */ -class MediatorProducerConfig ( +class MessagingClientConfig ( val onSerializationError: (ByteArray) -> Unit, ) \ No newline at end of file diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MediatorProducerFactory.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MediatorProducerFactory.kt deleted file mode 100644 index 4aff78355da..00000000000 --- a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MediatorProducerFactory.kt +++ /dev/null @@ -1,17 +0,0 @@ -package net.corda.messaging.api.mediator.factory - -import net.corda.messaging.api.mediator.MediatorProducer -import net.corda.messaging.api.mediator.config.MediatorProducerConfig - -/** - * Factory for creating multi-source event mediator producers. - */ -interface MediatorProducerFactory { - - /** - * Creates a multi-source event mediator producer. - * - * @param config Multi-source event mediator producer configuration. - */ - fun create(config: MediatorProducerConfig): MediatorProducer -} \ No newline at end of file diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MediatorProducerFactoryFactory.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MediatorProducerFactoryFactory.kt deleted file mode 100644 index 90ab9457692..00000000000 --- a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MediatorProducerFactoryFactory.kt +++ /dev/null @@ -1,20 +0,0 @@ -package net.corda.messaging.api.mediator.factory - -import net.corda.libs.configuration.SmartConfig - -/** - * Factory for creating multi-source event mediator producer factories. - */ -interface MediatorProducerFactoryFactory { - /** - * Creates a message bus producer factory. - * - * - * @param id Producer ID. - * @param messageBusConfig Message bus related configuration. - */ - fun createMessageBusProducerFactory( - id: String, - messageBusConfig: SmartConfig, - ) : MediatorProducerFactory -} \ No newline at end of file diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MediatorProducerFinder.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MediatorProducerFinder.kt deleted file mode 100644 index a961096c97c..00000000000 --- a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MediatorProducerFinder.kt +++ /dev/null @@ -1,15 +0,0 @@ -package net.corda.messaging.api.mediator.factory - -import net.corda.messaging.api.mediator.MediatorProducer - -/** - * Mediator producer finder is used to access [MediatorProducer] by its ID. - */ -fun interface MediatorProducerFinder { - - /** - * @param id Producer's ID. - * @return Producer found by given ID. - */ - fun find(id: String): MediatorProducer -} diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MessageRouterFactory.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MessageRouterFactory.kt index c78fc85946f..2e8426eb39c 100644 --- a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MessageRouterFactory.kt +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MessageRouterFactory.kt @@ -1,7 +1,7 @@ package net.corda.messaging.api.mediator.factory -import net.corda.messaging.api.mediator.MediatorProducer import net.corda.messaging.api.mediator.MessageRouter +import net.corda.messaging.api.mediator.MessagingClient /** * Factory for creating [MessageRouter]s. @@ -9,24 +9,24 @@ import net.corda.messaging.api.mediator.MessageRouter fun interface MessageRouterFactory { /** - * Creates a new instance of [MessageRouter]. Provided [MediatorProducerFinder] is used to find [MediatorProducer]s + * Creates a new instance of [MessageRouter]. Provided [MessagingClientFinder] is used to find [MessagingClient]s * by their IDs. Example: * * ``` - * MessageRouterFactory { producerFinder -> - * val messageBusProducer = producerFinder.find("MessageBusProducer") + * MessageRouterFactory { clientFinder -> + * val messageBusClient = clientFinder.find("MessageBusClient") * * MessageRouter { message -> * when (message.payload) { - * is FlowEvent -> RoutingDestination(messageBusProducer, "flow.event.topic") + * is FlowEvent -> RoutingDestination(messageBusClient, "flow.event.topic") * else -> throw IllegalStateException("No route defined for message $message") * } * } * } * ``` * - * @param producerFinder Producer finder + * @param clientFinder Messaging client finder * @return created message router. */ - fun create(producerFinder: MediatorProducerFinder): MessageRouter + fun create(clientFinder: MessagingClientFinder): MessageRouter } diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MessagingClientFactory.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MessagingClientFactory.kt new file mode 100644 index 00000000000..f83598bfae2 --- /dev/null +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MessagingClientFactory.kt @@ -0,0 +1,17 @@ +package net.corda.messaging.api.mediator.factory + +import net.corda.messaging.api.mediator.MessagingClient +import net.corda.messaging.api.mediator.config.MessagingClientConfig + +/** + * Factory for creating multi-source event mediator messaging clients. + */ +interface MessagingClientFactory { + + /** + * Creates a multi-source event mediator messaging client. + * + * @param config Multi-source event mediator messaging client configuration. + */ + fun create(config: MessagingClientConfig): MessagingClient +} \ No newline at end of file diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MessagingClientFactoryFactory.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MessagingClientFactoryFactory.kt new file mode 100644 index 00000000000..3bcba302d5a --- /dev/null +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MessagingClientFactoryFactory.kt @@ -0,0 +1,20 @@ +package net.corda.messaging.api.mediator.factory + +import net.corda.libs.configuration.SmartConfig + +/** + * Factory for creating multi-source event mediator messaging client factories. + */ +interface MessagingClientFactoryFactory { + /** + * Creates a message bus messaging client factory. + * + * + * @param id Messaging client ID. + * @param messageBusConfig Message bus related configuration. + */ + fun createMessageBusClientFactory( + id: String, + messageBusConfig: SmartConfig, + ) : MessagingClientFactory +} \ No newline at end of file diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MessagingClientFinder.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MessagingClientFinder.kt new file mode 100644 index 00000000000..aecfa05eaee --- /dev/null +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/factory/MessagingClientFinder.kt @@ -0,0 +1,15 @@ +package net.corda.messaging.api.mediator.factory + +import net.corda.messaging.api.mediator.MessagingClient + +/** + * Messaging client finder is used to access [MessagingClient] by its ID. + */ +fun interface MessagingClientFinder { + + /** + * @param id Messaging client's ID. + * @return Messaging client found by given ID. + */ + fun find(id: String): MessagingClient +} From dc0695bc341930b038f4c19024f3b71a5de6f93b Mon Sep 17 00:00:00 2001 From: Miljenko Brkic Date: Fri, 22 Sep 2023 20:36:26 +0100 Subject: [PATCH 32/50] CORE-16199 Fixed compile errors after merge with latest from 5.1 --- .../corda/messaging/mediator/MultiSourceEventMediatorImpl.kt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt index 0b918fef552..ba2768d66c1 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt @@ -189,7 +189,7 @@ class MultiSourceEventMediatorImpl( private fun commitOffsets() { runBlocking { consumers.map { consumer -> - consumer.commitAsyncOffsets() + consumer.asyncCommitOffsets() }.map { it.await() } From cb5f5e6c913901f1d7f210af2703e9d10bb972e1 Mon Sep 17 00:00:00 2001 From: Miljenko Brkic Date: Wed, 27 Sep 2023 09:29:27 +0100 Subject: [PATCH 33/50] CORE-16199 Fixed bug with persisting states --- .../mediator/MediatorStateManager.kt | 20 +++++++++++++------ .../messaging/mediator/MediatorTaskManager.kt | 4 ++-- 2 files changed, 16 insertions(+), 8 deletions(-) diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MediatorStateManager.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MediatorStateManager.kt index 0bfc5411a33..bec5818b609 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MediatorStateManager.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MediatorStateManager.kt @@ -42,19 +42,27 @@ class MediatorStateManager( * @return The latest states in case persistence failed due to conflict (state being updated by another process in * the meanwhile). */ - fun persistStates(processorTaskResults: Collection>): Map { + fun persistStates(processorTaskResults: Collection>): Map { val states = processorTaskResults.mapNotNull { result -> result.updatedState } - val (newStates, existingStates) = states.partition { it.version == State.INITIAL_VERSION } - val invalidStates = mutableMapOf() + val (newStates, existingStates) = states.partition { state -> + state.version == State.INITIAL_VERSION + } + val latestValuesForFailedStates = mutableMapOf() if (newStates.isNotEmpty()) { - invalidStates.putAll(stateManager.update(newStates)) + val failedStatesKeys = stateManager.create(newStates).keys + if (failedStatesKeys.isNotEmpty()) { + val latestStatesValues = stateManager.get(failedStatesKeys) + latestValuesForFailedStates.putAll(failedStatesKeys.associateWith { key -> + latestStatesValues[key] + }) + } } if (existingStates.isNotEmpty()) { - invalidStates.putAll(stateManager.update(existingStates)) + latestValuesForFailedStates.putAll(stateManager.update(existingStates)) } - return invalidStates + return latestValuesForFailedStates } /** diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MediatorTaskManager.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MediatorTaskManager.kt index 412e4e2739c..671467ae52c 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MediatorTaskManager.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MediatorTaskManager.kt @@ -75,10 +75,10 @@ internal class MediatorTaskManager( */ fun createMsgProcessorTasks( invalidResults: List>, - persistedStates: Map, + persistedStates: Map, ): List> { return invalidResults.map { - it.processorTask.copy(persistedState = persistedStates[it.processorTask.key]!!) + it.processorTask.copy(persistedState = persistedStates[it.processorTask.key]) } } From f89d6e5142b3de17eccbcae1a8f6474986e04848 Mon Sep 17 00:00:00 2001 From: Miljenko Brkic Date: Wed, 27 Sep 2023 09:29:48 +0100 Subject: [PATCH 34/50] CORE-16199 Removed unnecessary build task --- libs/messaging/messaging/build.gradle | 11 ----------- 1 file changed, 11 deletions(-) diff --git a/libs/messaging/messaging/build.gradle b/libs/messaging/messaging/build.gradle index d569b059fba..18c0c5d1ae5 100644 --- a/libs/messaging/messaging/build.gradle +++ b/libs/messaging/messaging/build.gradle @@ -24,15 +24,4 @@ dependencies { testImplementation "org.mockito.kotlin:mockito-kotlin:$mockitoKotlinVersion" } -// include coroutines bundle in this jar as it doesn't have OSGi metadata -tasks.named('jar', Jar) { - bundle { - bnd '''\ --conditionalpackage: kotlinx.coroutines* -Export-Package: kotlinx.coroutines* -Import-Package: !android*, * -''' - } -} - description 'Messaging API' From 1000241991d5ea9d84ed9ba87f0845e260d16f36 Mon Sep 17 00:00:00 2001 From: Miljenko Brkic Date: Wed, 27 Sep 2023 09:31:11 +0100 Subject: [PATCH 35/50] CORE-16199 Added missing events to flow engine router and enabled FlowExecutorMediatorImpl --- .../flow/service/FlowExecutorMediatorImpl.kt | 46 +++++++++++++++++-- .../mediator/MultiSourceEventMediatorImpl.kt | 6 +-- 2 files changed, 45 insertions(+), 7 deletions(-) diff --git a/components/flow/flow-service/src/main/kotlin/net/corda/flow/service/FlowExecutorMediatorImpl.kt b/components/flow/flow-service/src/main/kotlin/net/corda/flow/service/FlowExecutorMediatorImpl.kt index 9bb4ca2cf7a..fabaeb8de64 100644 --- a/components/flow/flow-service/src/main/kotlin/net/corda/flow/service/FlowExecutorMediatorImpl.kt +++ b/components/flow/flow-service/src/main/kotlin/net/corda/flow/service/FlowExecutorMediatorImpl.kt @@ -1,10 +1,17 @@ package net.corda.flow.service import com.typesafe.config.ConfigValueFactory +import net.corda.avro.serialization.CordaAvroSerializationFactory +import net.corda.data.crypto.wire.ops.flow.FlowOpsRequest import net.corda.data.flow.event.FlowEvent import net.corda.data.flow.event.mapper.FlowMapperEvent +import net.corda.data.flow.output.FlowStatus import net.corda.data.flow.state.checkpoint.Checkpoint +import net.corda.data.ledger.persistence.LedgerPersistenceRequest +import net.corda.data.persistence.EntityRequest +import net.corda.data.uniqueness.UniquenessCheckRequestAvro import net.corda.flow.pipeline.factory.FlowEventProcessorFactory +import net.corda.ledger.utxo.verification.TransactionVerificationRequest import net.corda.libs.configuration.SmartConfig import net.corda.libs.configuration.helper.getConfig import net.corda.lifecycle.LifecycleCoordinatorFactory @@ -14,6 +21,7 @@ import net.corda.lifecycle.RegistrationHandle import net.corda.lifecycle.StartEvent import net.corda.lifecycle.StopEvent import net.corda.lifecycle.createCoordinator +import net.corda.messaging.api.mediator.MediatorMessage import net.corda.messaging.api.mediator.MessageRouter import net.corda.messaging.api.mediator.MultiSourceEventMediator import net.corda.messaging.api.mediator.RoutingDestination.Companion.routeTo @@ -23,8 +31,14 @@ import net.corda.messaging.api.mediator.factory.MessageRouterFactory import net.corda.messaging.api.mediator.factory.MessagingClientFactoryFactory import net.corda.messaging.api.mediator.factory.MultiSourceEventMediatorFactory import net.corda.messaging.api.processor.StateAndEventProcessor +import net.corda.schema.Schemas.Crypto.FLOW_OPS_MESSAGE_TOPIC import net.corda.schema.Schemas.Flow.FLOW_EVENT_TOPIC import net.corda.schema.Schemas.Flow.FLOW_MAPPER_EVENT_TOPIC +import net.corda.schema.Schemas.Flow.FLOW_STATUS_TOPIC +import net.corda.schema.Schemas.Persistence.PERSISTENCE_ENTITY_PROCESSOR_TOPIC +import net.corda.schema.Schemas.Persistence.PERSISTENCE_LEDGER_PROCESSOR_TOPIC +import net.corda.schema.Schemas.UniquenessChecker.UNIQUENESS_CHECK_TOPIC +import net.corda.schema.Schemas.Verification.VERIFICATION_LEDGER_PROCESSOR_TOPIC import net.corda.schema.configuration.ConfigKeys.FLOW_CONFIG import net.corda.schema.configuration.ConfigKeys.MESSAGING_CONFIG import net.corda.schema.configuration.MessagingConfig.MAX_ALLOWED_MSG_SIZE @@ -44,7 +58,8 @@ class FlowExecutorMediatorImpl ( private val eventMediatorFactory: MultiSourceEventMediatorFactory, private val mediatorConsumerFactoryFactory: MediatorConsumerFactoryFactory, private val messagingClientFactoryFactory: MessagingClientFactoryFactory, - private val toMessagingConfig: (Map) -> SmartConfig + cordaAvroSerializationFactory: CordaAvroSerializationFactory, + private val toMessagingConfig: (Map) -> SmartConfig, ) : FlowExecutor { @Activate @@ -59,12 +74,15 @@ class FlowExecutorMediatorImpl ( mediatorConsumerFactoryFactory: MediatorConsumerFactoryFactory, @Reference(service = MessagingClientFactoryFactory::class) messagingClientFactoryFactory: MessagingClientFactoryFactory, + @Reference(service = CordaAvroSerializationFactory::class) + cordaAvroSerializationFactory: CordaAvroSerializationFactory, ) : this( coordinatorFactory, flowEventProcessorFactory, eventMediatorFactory, mediatorConsumerFactoryFactory, messagingClientFactoryFactory, + cordaAvroSerializationFactory, { cfg -> cfg.getConfig(MESSAGING_CONFIG) } ) @@ -77,6 +95,7 @@ class FlowExecutorMediatorImpl ( private val coordinator = coordinatorFactory.createCoordinator { event, _ -> eventHandler(event) } private var subscriptionRegistrationHandle: RegistrationHandle? = null private var multiSourceEventMediator: MultiSourceEventMediator? = null + private val deserializer = cordaAvroSerializationFactory.createAvroDeserializer({}, Any::class.java) override fun onConfigChange(config: Map) { try { @@ -174,10 +193,31 @@ class FlowExecutorMediatorImpl ( val messageBusClient = clientFinder.find(MESSAGE_BUS_CLIENT) MessageRouter { message -> - when (message.payload) { + when (val event = message.event()) { + is EntityRequest -> routeTo(messageBusClient, PERSISTENCE_ENTITY_PROCESSOR_TOPIC) is FlowMapperEvent -> routeTo(messageBusClient, FLOW_MAPPER_EVENT_TOPIC) - else -> throw IllegalStateException("No route defined for message $message") + is FlowOpsRequest -> routeTo(messageBusClient, FLOW_OPS_MESSAGE_TOPIC) + is FlowStatus -> routeTo(messageBusClient, FLOW_STATUS_TOPIC) + is LedgerPersistenceRequest -> routeTo(messageBusClient, PERSISTENCE_LEDGER_PROCESSOR_TOPIC) + is TransactionVerificationRequest -> routeTo(messageBusClient, VERIFICATION_LEDGER_PROCESSOR_TOPIC) + is UniquenessCheckRequestAvro -> routeTo(messageBusClient, UNIQUENESS_CHECK_TOPIC) + else -> { + val eventType = event?.let { it::class.java } + throw IllegalStateException("No route defined for event type [$eventType]") + } } } } + + /** + * Deserialized message payload if it is a [ByteArray] (seems to be the case for external events). + */ + private fun MediatorMessage.event(): Any? { + val event = payload + return if (event is ByteArray) { + deserializer.deserialize(event) + } else { + event + } + } } diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt index ba2768d66c1..a406329a44f 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt @@ -169,10 +169,8 @@ class MultiSourceEventMediatorImpl( val clientTasks = mediatorTaskManager.createProducerTasks(validResults, messageRouter) val clientResults = mediatorTaskManager.executeProducerTasks(clientTasks) msgProcessorTasks = - mediatorTaskManager.createMsgProcessorTasks(clientResults) + mediatorTaskManager.createMsgProcessorTasks( - invalidResults, - conflictingStates - ) + mediatorTaskManager.createMsgProcessorTasks(clientResults) + + mediatorTaskManager.createMsgProcessorTasks(invalidResults, conflictingStates) } while (msgProcessorTasks.isNotEmpty()) commitOffsets() } From 7a20c15fba8546d96c6dd70ea6404aa64d140883 Mon Sep 17 00:00:00 2001 From: Miljenko Brkic Date: Wed, 27 Sep 2023 12:05:57 +0100 Subject: [PATCH 36/50] CORE-16199 Refactoring --- .../MultiSourceEventMediatorFactoryImpl.kt | 12 ----------- .../mediator/MessageBusClientTest.kt | 20 +++++++++++-------- .../messaging/mediator/ProcessorTaskTest.kt | 4 ++-- 3 files changed, 14 insertions(+), 22 deletions(-) diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/MultiSourceEventMediatorFactoryImpl.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/MultiSourceEventMediatorFactoryImpl.kt index 37902f10040..ee39653cfb1 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/MultiSourceEventMediatorFactoryImpl.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/MultiSourceEventMediatorFactoryImpl.kt @@ -12,7 +12,6 @@ import org.osgi.service.component.annotations.Activate import org.osgi.service.component.annotations.Component import org.osgi.service.component.annotations.Reference -@Suppress("LongParameterList") @Component(service = [MultiSourceEventMediatorFactory::class]) class MultiSourceEventMediatorFactoryImpl @Activate constructor( @Reference(service = CordaAvroSerializationFactory::class) @@ -42,15 +41,4 @@ class MultiSourceEventMediatorFactoryImpl @Activate constructor( lifecycleCoordinatorFactory, ) } - - // TODO -// val msgConfig = messagingConfig.withFallback(defaults) -// messagingConfig.getInt(BootConfig.INSTANCE_ID), -// Duration.ofMillis(messagingConfig.getLong(MessagingConfig.Subscription.POLL_TIMEOUT)), -// Duration.ofMillis(messagingConfig.getLong(MessagingConfig.Subscription.THREAD_STOP_TIMEOUT)), -// messagingConfig.getInt(MessagingConfig.Subscription.PROCESSOR_RETRIES), -// messagingConfig.getInt(MessagingConfig.Subscription.SUBSCRIBE_RETRIES), -// messagingConfig.getInt(MessagingConfig.Subscription.COMMIT_RETRIES), -// Duration.ofMillis(messagingConfig.getLong(MessagingConfig.Subscription.PROCESSOR_TIMEOUT)), -// messagingConfig } \ No newline at end of file diff --git a/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/MessageBusClientTest.kt b/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/MessageBusClientTest.kt index 1eb072a1169..f8d4c35fdb0 100644 --- a/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/MessageBusClientTest.kt +++ b/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/MessageBusClientTest.kt @@ -17,13 +17,17 @@ import org.mockito.kotlin.verify import org.mockito.kotlin.whenever class MessageBusClientTest { + private companion object { + const val MSG_PROP_KEY = "key" + const val TEST_ENDPOINT = "topic" + } + private lateinit var cordaProducer: CordaProducer private lateinit var messageBusClient: MessageBusClient private val defaultHeaders: List> = emptyList() private val messageProps: MutableMap = mutableMapOf( - "topic" to "topic", - "key" to "key", + MSG_PROP_KEY to "key", "headers" to defaultHeaders ) private val message: MediatorMessage = MediatorMessage("value", messageProps) @@ -37,11 +41,11 @@ class MessageBusClientTest { @Test fun testSend() { - messageBusClient.send(message) + messageBusClient.send(message, TEST_ENDPOINT) val expected = CordaProducerRecord( - message.getProperty("topic"), - message.getProperty("key"), + TEST_ENDPOINT, + message.getProperty(MSG_PROP_KEY), message.payload ) @@ -51,15 +55,15 @@ class MessageBusClientTest { @Test fun testSendWithError() { val record = CordaProducerRecord( - message.getProperty("topic"), - message.getProperty("key"), + TEST_ENDPOINT, + message.getProperty(MSG_PROP_KEY), message.payload ) doThrow(CordaRuntimeException("")).whenever(cordaProducer).send(eq(record), any()) assertThrows { runBlocking { - messageBusClient.send(message).await() + messageBusClient.send(message, TEST_ENDPOINT).await() } } } diff --git a/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/ProcessorTaskTest.kt b/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/ProcessorTaskTest.kt index 65e5eb1e90a..e1ea2d59869 100644 --- a/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/ProcessorTaskTest.kt +++ b/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/ProcessorTaskTest.kt @@ -26,7 +26,7 @@ class ProcessorTaskTest { } private lateinit var cordaProducer: CordaProducer - private lateinit var mediatorProducer: MessageBusProducer + private lateinit var mediatorProducer: MessageBusClient private val defaultHeaders: List> = emptyList() private val messageProps: MutableMap = mutableMapOf( @@ -39,7 +39,7 @@ class ProcessorTaskTest { @BeforeEach fun setup() { cordaProducer = mock() - mediatorProducer = MessageBusProducer("client-id", cordaProducer) + mediatorProducer = MessageBusClient("client-id", cordaProducer) } @Test From 71d0b560af768668603a7343f9e2a9a387926060 Mon Sep 17 00:00:00 2001 From: Miljenko Brkic Date: Thu, 28 Sep 2023 09:39:21 +0100 Subject: [PATCH 37/50] CORE-16199 Removed package api/mediator/statemanager, using interfaces of StateManager --- .../mediator/MediatorStateManager.kt | 10 +-- .../messaging/mediator/MediatorTaskManager.kt | 3 +- .../mediator/MultiSourceEventMediatorImpl.kt | 2 +- .../corda/messaging/mediator/ProcessorTask.kt | 2 +- .../api/mediator/statemanager/Metadata.kt | 44 ------------ .../api/mediator/statemanager/State.kt | 63 ---------------- .../api/mediator/statemanager/StateManager.kt | 72 ------------------- 7 files changed, 8 insertions(+), 188 deletions(-) delete mode 100644 libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/statemanager/Metadata.kt delete mode 100644 libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/statemanager/State.kt delete mode 100644 libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/statemanager/StateManager.kt diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MediatorStateManager.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MediatorStateManager.kt index bec5818b609..33ff7b98d49 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MediatorStateManager.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MediatorStateManager.kt @@ -2,9 +2,9 @@ package net.corda.messaging.mediator import net.corda.avro.serialization.CordaAvroDeserializer import net.corda.avro.serialization.CordaAvroSerializer -import net.corda.messaging.api.mediator.statemanager.Metadata -import net.corda.messaging.api.mediator.statemanager.State -import net.corda.messaging.api.mediator.statemanager.StateManager +import net.corda.libs.statemanager.api.Metadata +import net.corda.libs.statemanager.api.State +import net.corda.libs.statemanager.api.StateManager /** * Helper for working with [StateManager], used by [MultiSourceEventMediatorImpl]. @@ -30,7 +30,7 @@ class MediatorStateManager( State( key, serializedValue, - persistedState?.version ?: State.INITIAL_VERSION, + persistedState?.version ?: State.VERSION_INITIAL_VALUE, persistedState?.metadata ?: Metadata() ) } @@ -47,7 +47,7 @@ class MediatorStateManager( result.updatedState } val (newStates, existingStates) = states.partition { state -> - state.version == State.INITIAL_VERSION + state.version == State.VERSION_INITIAL_VALUE } val latestValuesForFailedStates = mutableMapOf() if (newStates.isNotEmpty()) { diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MediatorTaskManager.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MediatorTaskManager.kt index 671467ae52c..8874e48fa98 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MediatorTaskManager.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MediatorTaskManager.kt @@ -1,11 +1,10 @@ package net.corda.messaging.mediator import kotlinx.coroutines.runBlocking +import net.corda.libs.statemanager.api.State import net.corda.messagebus.api.consumer.CordaConsumerRecord import net.corda.messaging.api.mediator.MediatorMessage import net.corda.messaging.api.mediator.MessageRouter -import net.corda.messaging.api.mediator.statemanager.State -import net.corda.messaging.api.mediator.statemanager.StateManager import net.corda.messaging.api.mediator.taskmanager.TaskManager import net.corda.messaging.api.mediator.taskmanager.TaskType import net.corda.messaging.api.processor.StateAndEventProcessor diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt index a406329a44f..a28f80d7a76 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt @@ -3,6 +3,7 @@ package net.corda.messaging.mediator import kotlinx.coroutines.runBlocking import net.corda.avro.serialization.CordaAvroDeserializer import net.corda.avro.serialization.CordaAvroSerializer +import net.corda.libs.statemanager.api.StateManager import net.corda.lifecycle.LifecycleCoordinatorFactory import net.corda.lifecycle.LifecycleCoordinatorName import net.corda.lifecycle.LifecycleStatus @@ -14,7 +15,6 @@ import net.corda.messaging.api.mediator.MessageRouter import net.corda.messaging.api.mediator.MessagingClient import net.corda.messaging.api.mediator.MultiSourceEventMediator import net.corda.messaging.api.mediator.config.EventMediatorConfig -import net.corda.messaging.api.mediator.statemanager.StateManager import net.corda.messaging.api.mediator.taskmanager.TaskManager import net.corda.messaging.api.mediator.taskmanager.TaskType import net.corda.utilities.debug diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/ProcessorTask.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/ProcessorTask.kt index a07ef10ee0b..06638162b0f 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/ProcessorTask.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/ProcessorTask.kt @@ -1,7 +1,7 @@ package net.corda.messaging.mediator +import net.corda.libs.statemanager.api.State import net.corda.messagebus.api.consumer.CordaConsumerRecord -import net.corda.messaging.api.mediator.statemanager.State import net.corda.messaging.api.processor.StateAndEventProcessor import net.corda.messaging.api.records.Record import net.corda.messaging.utils.toRecord diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/statemanager/Metadata.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/statemanager/Metadata.kt deleted file mode 100644 index 7e0a6aae5db..00000000000 --- a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/statemanager/Metadata.kt +++ /dev/null @@ -1,44 +0,0 @@ -package net.corda.messaging.api.mediator.statemanager - -/** - * Supported comparison operations on metadata values. - */ -enum class Operation { - Equals, - NotEquals, - LesserThan, - GreaterThan, -} - -/** - * Mutable map that allows only primitive types to be used as values. - */ -class Metadata( - private val map: MutableMap = mutableMapOf() -) : MutableMap by map { - - private val supportedType = listOf( - String::class.java, - java.lang.String::class.java, - Number::class.java, - java.lang.Number::class.java, - Boolean::class.java, - java.lang.Boolean::class.java, - ) - - private fun isPrimitiveOrBoxedValue(value: Any): Boolean { - return supportedType.any { it.isAssignableFrom(value.javaClass) } - } - - override fun put(key: String, value: Any): Any? { - if (!isPrimitiveOrBoxedValue(value)) { - throw IllegalArgumentException("Type not supported: ${value::class}") - } - - return map.put(key, value) - } -} - -fun metadata(): Metadata = Metadata() - -fun metadata(vararg pairs: Pair): Metadata = Metadata(mutableMapOf(*pairs)) \ No newline at end of file diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/statemanager/State.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/statemanager/State.kt deleted file mode 100644 index c27614e8cea..00000000000 --- a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/statemanager/State.kt +++ /dev/null @@ -1,63 +0,0 @@ -package net.corda.messaging.api.mediator.statemanager - -import java.time.Instant - -/** - * A state managed via the state manager. - */ -data class State( - /** - * Identifier for the state. - */ - val key: String, - - /** - * The actual value of the state. - */ - val value: ByteArray, - - /** - * Version of the state. - */ - val version: Int = INITIAL_VERSION, - - /** - * Arbitrary Map of primitive types that can be used to store and query data associated with the state. - */ - val metadata: Metadata = Metadata(), - - /** - * Time when the state was last modified. - */ - val modifiedTime: Instant = Instant.now(), -) { - - companion object { - const val INITIAL_VERSION = -1 - } - - override fun equals(other: Any?): Boolean { - if (this === other) return true - if (javaClass != other?.javaClass) return false - - other as State - - if (key != other.key) return false - if (!value.contentEquals(other.value)) return false - if (version != other.version) return false - if (metadata != other.metadata) return false - if (modifiedTime != other.modifiedTime) return false - - return true - } - - override fun hashCode(): Int { - var result = key.hashCode() - result = 31 * result + value.contentHashCode() - result = 31 * result + version - result = 31 * result + metadata.hashCode() - result = 31 * result + modifiedTime.hashCode() - - return result - } -} \ No newline at end of file diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/statemanager/StateManager.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/statemanager/StateManager.kt deleted file mode 100644 index 7444042a26b..00000000000 --- a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/statemanager/StateManager.kt +++ /dev/null @@ -1,72 +0,0 @@ -package net.corda.messaging.api.mediator.statemanager - -import java.time.Instant - -/** - * The [StateManager] provides functions to manage states within the underlying persistent storage. - */ -interface StateManager : AutoCloseable { - - /** - * Create [states] into the underlying storage. - * Control is only returned to the caller once all [states] have been updated and replicas of the underlying - * persistent storage, if any, are synced. - * - * @param states Collection of states to be persisted. - * @return States that could not be created on the persistent storage, along with the actual reason for the failures. - */ - fun create(states: Collection): Map - - /** - * Get all states referenced by [keys]. - * Only states that have been successfully committed and distributed within the underlying persistent - * storage are returned. - * - * @param keys Collection of state keys to use when querying the persistent storage. - * @return States found in the underlying persistent storage. - */ - fun get(keys: Collection): Map - - /** - * Update [states] within the underlying storage. - * Control is only returned to the caller once all [states] have been updated and replicas of the underlying - * persistent storage, if any, are synced. - * The operation is transactional, either all [states] are updated or none is. - * - * @param states Collection of states to be updated. - * @return States that could not be updated due to mismatch versions. - */ - fun update(states: Collection): Map - - /** - * Delete all [states] from the underlying storage. - * Control is only returned to the caller once all states have been deleted and replicas of the underlying - * persistent storage, if any, are synced. - * The operation is transactional, either all [states] are deleted or none is. - * - * @param states Collection of states to be deleted. - * @return States that could not be deleted due to mismatch versions. - */ - fun delete(states: Collection): Map - - /** - * Retrieve all states that were updated for the last time between [start] (inclusive) and [finish] (inclusive). - * - * @param start Time filter lower bound (inclusive). - * @param finish Time filter upper bound (inclusive). - * @return States that were last updated between [start] and [finish] times. - */ - fun getUpdatedBetween(start: Instant, finish: Instant): Map - - /** - * Retrieve states based on custom [operation] to be executed against a single [key] within the [State.metadata]. - * Only states that have been successfully committed and distributed within the underlying persistent - * storage are returned. - * - * @param key The name of the key in the [State.metadata] to apply the comparison on. - * @param operation The comparison operation to perform (">", "=", "<", "<>", etc.). - * @param value The value to compare against. - * @return states for which the [State.metadata] has [key] for which [value] matches [operation]. - */ - fun find(key: String, operation: Operation, value: Any): Map -} \ No newline at end of file From f2d1e874b2a0e727481170620a32398403c5be28 Mon Sep 17 00:00:00 2001 From: Miljenko Brkic Date: Thu, 28 Sep 2023 12:27:17 +0100 Subject: [PATCH 38/50] CORE-16199 Refactoring --- .../corda/messaging/mediator/ClientTask.kt | 13 +++++--- .../messaging/mediator/MessageBusClient.kt | 8 ++--- .../mediator/MultiSourceEventMediatorImpl.kt | 33 ++++++++++--------- .../corda/messaging/mediator/ProcessorTask.kt | 12 +++---- ...rStateManager.kt => StateManagerHelper.kt} | 2 +- ...torTaskManager.kt => TaskManagerHelper.kt} | 4 +-- .../{ => factory}/MediatorComponentFactory.kt | 8 ++--- .../messaging/mediator/ProcessorTaskTest.kt | 8 ++--- .../mediator/statemanager/package-info.java | 4 --- .../messaging/api/mediator/MessagingClient.kt | 8 +++-- 10 files changed, 54 insertions(+), 46 deletions(-) rename libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/{MediatorStateManager.kt => StateManagerHelper.kt} (98%) rename libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/{MediatorTaskManager.kt => TaskManagerHelper.kt} (97%) rename libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/{ => factory}/MediatorComponentFactory.kt (94%) delete mode 100644 libs/messaging/messaging/src/main/java/net/corda/messaging/api/mediator/statemanager/package-info.java diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/ClientTask.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/ClientTask.kt index 176cfe9adbc..670e69ec16f 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/ClientTask.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/ClientTask.kt @@ -4,18 +4,19 @@ import kotlinx.coroutines.runBlocking import net.corda.messaging.api.mediator.MediatorMessage import net.corda.messaging.api.mediator.MessageRouter import net.corda.messaging.api.mediator.MessagingClient +import net.corda.messaging.api.mediator.MessagingClient.Companion.MSG_PROP_ENDPOINT import java.util.concurrent.Callable /** * [ClientTask] sends a [MediatorMessage] to [MessagingClient] selected by [MessageRouter]. */ -class ClientTask( +class ClientTask( private val message: MediatorMessage, private val messageRouter: MessageRouter, val processorTask: ProcessorTask, -): Callable> { +) : Callable> { - class Result( + class Result( val clientTask: ClientTask, val replyMessage: MediatorMessage?, ) { @@ -24,9 +25,13 @@ class ClientTask( override fun call(): Result { val destination = messageRouter.getDestination(message) + @Suppress("UNCHECKED_CAST") val reply = runBlocking { - with(destination) { client.send(message, endpoint).await() } + with(destination) { + message.addProperty(MSG_PROP_ENDPOINT, endpoint) + client.send(message).await() + } } as MediatorMessage? return Result(this, reply) } diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MessageBusClient.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MessageBusClient.kt index c81a4d80957..1ad0afc7ca5 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MessageBusClient.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MessageBusClient.kt @@ -18,9 +18,9 @@ class MessageBusClient( private val log: Logger = LoggerFactory.getLogger(this::class.java.enclosingClass) } - override fun send(message: MediatorMessage<*>, endpoint: String): Deferred?> = + override fun send(message: MediatorMessage<*>): Deferred?> = CompletableDeferred?>().apply { - producer.send(message.toCordaProducerRecord(endpoint)) { ex -> + producer.send(message.toCordaProducerRecord()) { ex -> if (ex != null) { completeExceptionally(ex) } else { @@ -40,9 +40,9 @@ class MessageBusClient( } } -private fun MediatorMessage<*>.toCordaProducerRecord(endpoint: String) : CordaProducerRecord<*, *> { +private fun MediatorMessage<*>.toCordaProducerRecord() : CordaProducerRecord<*, *> { return CordaProducerRecord( - topic = endpoint, + topic = this.getProperty("topic"), key = this.getProperty("key"), value = this.payload, headers = this.getProperty("headers"), diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt index a28f80d7a76..9634ab748f6 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt @@ -17,6 +17,7 @@ import net.corda.messaging.api.mediator.MultiSourceEventMediator import net.corda.messaging.api.mediator.config.EventMediatorConfig import net.corda.messaging.api.mediator.taskmanager.TaskManager import net.corda.messaging.api.mediator.taskmanager.TaskType +import net.corda.messaging.mediator.factory.MediatorComponentFactory import net.corda.utilities.debug import org.slf4j.LoggerFactory import java.time.Duration @@ -30,7 +31,7 @@ class MultiSourceEventMediatorImpl( private val stateManager: StateManager, private val taskManager: TaskManager, lifecycleCoordinatorFactory: LifecycleCoordinatorFactory, -): MultiSourceEventMediator { +) : MultiSourceEventMediator { private val log = LoggerFactory.getLogger("${this.javaClass.name}-${config.name}") @@ -40,11 +41,11 @@ class MultiSourceEventMediatorImpl( private val mediatorComponentFactory = MediatorComponentFactory( config.messageProcessor, config.consumerFactories, config.clientFactories, config.messageRouterFactory ) - private val mediatorStateManager = MediatorStateManager( + private val stateManagerHelper = StateManagerHelper( stateManager, serializer, stateDeserializer ) - private val mediatorTaskManager = MediatorTaskManager( - taskManager, mediatorStateManager + private val taskManagerHelper = TaskManagerHelper( + taskManager, stateManagerHelper ) private val pollTimeoutInNanos = config.pollTimeout.toNanos() private val uniqueId = UUID.randomUUID().toString() @@ -65,6 +66,7 @@ class MultiSourceEventMediatorImpl( private fun stop() = Thread.currentThread().interrupt() + private val stopped get() = Thread.currentThread().isInterrupted /** @@ -84,7 +86,7 @@ class MultiSourceEventMediatorImpl( clients = mediatorComponentFactory.createClients(::onSerializationError) messageRouter = mediatorComponentFactory.createRouter(clients) - consumers.forEach{ it.subscribe() } + consumers.forEach { it.subscribe() } lifecycleCoordinator.updateStatus(LifecycleStatus.UP) while (!stopped) { @@ -96,11 +98,13 @@ class MultiSourceEventMediatorImpl( is InterruptedException -> { // Stopped } + is CordaMessageAPIIntermittentException -> { log.warn( "${ex.message} Attempts: $attempts. Recreating consumers/clients and Retrying.", ex ) } + else -> { log.error( "${ex.message} Attempts: $attempts. Closing subscription.", ex @@ -143,7 +147,7 @@ class MultiSourceEventMediatorImpl( else -> { throw CordaMessageAPIFatalException( "Multi-source event mediator ${config.name} failed to process messages, " + - "Fatal error occurred.", ex + "Fatal error occurred.", ex ) } } @@ -157,26 +161,26 @@ class MultiSourceEventMediatorImpl( if (messages.isNotEmpty()) { val msgGroups = messages.groupBy { it.key } val persistedStates = stateManager.get(msgGroups.keys.map { it.toString() }) - var msgProcessorTasks = mediatorTaskManager.createMsgProcessorTasks( + var msgProcessorTasks = taskManagerHelper.createMsgProcessorTasks( msgGroups, persistedStates, config.messageProcessor ) do { - val processingResults = mediatorTaskManager.executeProcessorTasks(msgProcessorTasks) - val conflictingStates = mediatorStateManager.persistStates(processingResults) + val processingResults = taskManagerHelper.executeProcessorTasks(msgProcessorTasks) + val conflictingStates = stateManagerHelper.persistStates(processingResults) val (validResults, invalidResults) = processingResults.partition { !conflictingStates.contains(it.key) } - val clientTasks = mediatorTaskManager.createProducerTasks(validResults, messageRouter) - val clientResults = mediatorTaskManager.executeProducerTasks(clientTasks) + val clientTasks = taskManagerHelper.createProducerTasks(validResults, messageRouter) + val clientResults = taskManagerHelper.executeProducerTasks(clientTasks) msgProcessorTasks = - mediatorTaskManager.createMsgProcessorTasks(clientResults) + - mediatorTaskManager.createMsgProcessorTasks(invalidResults, conflictingStates) + taskManagerHelper.createMsgProcessorTasks(clientResults) + + taskManagerHelper.createMsgProcessorTasks(invalidResults, conflictingStates) } while (msgProcessorTasks.isNotEmpty()) commitOffsets() } } - private fun poll(pollTimeoutInNanos: Long): List> { + private fun poll(pollTimeoutInNanos: Long): List> { val maxEndTime = System.nanoTime() + pollTimeoutInNanos return consumers.map { consumer -> val remainingTime = (maxEndTime - System.nanoTime()).coerceAtLeast(0) @@ -219,7 +223,6 @@ class MultiSourceEventMediatorImpl( } - // private fun generateDeadLetterRecord(event: CordaConsumerRecord, state: S?): Record<*, *> { // val keyBytes = ByteBuffer.wrap(cordaAvroSerializer.serialize(event.key)) // val stateBytes = diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/ProcessorTask.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/ProcessorTask.kt index 06638162b0f..dc7dc7702f5 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/ProcessorTask.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/ProcessorTask.kt @@ -13,15 +13,15 @@ import java.util.concurrent.Callable * event. Result of processing are output events and final updated state. */ @Suppress("LongParameterList") -data class ProcessorTask( +data class ProcessorTask( val key: String, val persistedState: State?, val events: Collection>, private val processor: StateAndEventProcessor, - private val mediatorStateManager: MediatorStateManager, -): Callable> { + private val stateManagerHelper: StateManagerHelper, +) : Callable> { - class Result( + class Result( val processorTask: ProcessorTask, val outputEvents: List>, val updatedState: State?, @@ -30,7 +30,7 @@ data class ProcessorTask( } override fun call(): Result { - var stateValue = mediatorStateManager.deserializeValue(persistedState) + var stateValue = stateManagerHelper.deserializeValue(persistedState) val outputEvents = events.map { event -> val response = processor.onNext(stateValue, event.toRecord()) @@ -38,7 +38,7 @@ data class ProcessorTask( response.responseEvents }.flatten() - val updatedState = mediatorStateManager.createOrUpdateState( + val updatedState = stateManagerHelper.createOrUpdateState( key, persistedState, stateValue diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MediatorStateManager.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/StateManagerHelper.kt similarity index 98% rename from libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MediatorStateManager.kt rename to libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/StateManagerHelper.kt index 33ff7b98d49..6cf20409e1f 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MediatorStateManager.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/StateManagerHelper.kt @@ -9,7 +9,7 @@ import net.corda.libs.statemanager.api.StateManager /** * Helper for working with [StateManager], used by [MultiSourceEventMediatorImpl]. */ -class MediatorStateManager( +class StateManagerHelper( private val stateManager: StateManager, private val serializer: CordaAvroSerializer, private val stateDeserializer: CordaAvroDeserializer, diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MediatorTaskManager.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/TaskManagerHelper.kt similarity index 97% rename from libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MediatorTaskManager.kt rename to libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/TaskManagerHelper.kt index 8874e48fa98..75fdd064487 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MediatorTaskManager.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/TaskManagerHelper.kt @@ -13,9 +13,9 @@ import java.time.Instant /** * Helper that creates and executes various tasks used by [MultiSourceEventMediatorImpl]. */ -internal class MediatorTaskManager( +internal class TaskManagerHelper( private val taskManager: TaskManager, - private val stateManager: MediatorStateManager, + private val stateManager: StateManagerHelper, ) { /** diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MediatorComponentFactory.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/MediatorComponentFactory.kt similarity index 94% rename from libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MediatorComponentFactory.kt rename to libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/MediatorComponentFactory.kt index a36965d34fd..20f5e57cb53 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MediatorComponentFactory.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/MediatorComponentFactory.kt @@ -1,4 +1,4 @@ -package net.corda.messaging.mediator +package net.corda.messaging.mediator.factory import net.corda.messaging.api.mediator.MediatorConsumer import net.corda.messaging.api.mediator.MessageRouter @@ -18,7 +18,7 @@ internal class MediatorComponentFactory( private val consumerFactories: Collection, private val clientFactories: Collection, private val messageRouterFactory: MessageRouterFactory, - ) { +) { /** * Creates message consumers. @@ -29,7 +29,7 @@ internal class MediatorComponentFactory( fun createConsumers( onSerializationError: (ByteArray) -> Unit ): List> { - check (consumerFactories.isNotEmpty()) { + check(consumerFactories.isNotEmpty()) { "None consumer factory set in configuration" } return consumerFactories.map { consumerFactory -> @@ -52,7 +52,7 @@ internal class MediatorComponentFactory( fun createClients( onSerializationError: (ByteArray) -> Unit ): List { - check (clientFactories.isNotEmpty()) { + check(clientFactories.isNotEmpty()) { "None client factory set in configuration" } return clientFactories.map { clientFactory -> diff --git a/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/ProcessorTaskTest.kt b/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/ProcessorTaskTest.kt index e1ea2d59869..8cbc6f5049f 100644 --- a/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/ProcessorTaskTest.kt +++ b/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/ProcessorTaskTest.kt @@ -1,12 +1,12 @@ package net.corda.messaging.mediator import kotlinx.coroutines.runBlocking +import net.corda.libs.statemanager.api.State import net.corda.messagebus.api.consumer.CordaConsumerRecord import net.corda.messagebus.api.producer.CordaProducer import net.corda.messagebus.api.producer.CordaProducerRecord import net.corda.messaging.api.mediator.MediatorMessage -import net.corda.messaging.api.mediator.statemanager.State -import net.corda.messaging.api.processor.StateAndEventProcessor + import net.corda.messaging.api.processor.StateAndEventProcessor import net.corda.v5.base.exceptions.CordaRuntimeException import org.junit.jupiter.api.BeforeEach import org.junit.jupiter.api.Test @@ -58,13 +58,13 @@ class ProcessorTaskTest { ) } val processor = mock>() - val mediatorStateManager = mock>() + val stateManagerHelper = mock>() val task = ProcessorTask( key, persistedSate, events, processor, - mediatorStateManager + stateManagerHelper ) val result = task.call() diff --git a/libs/messaging/messaging/src/main/java/net/corda/messaging/api/mediator/statemanager/package-info.java b/libs/messaging/messaging/src/main/java/net/corda/messaging/api/mediator/statemanager/package-info.java deleted file mode 100644 index f83836d0d17..00000000000 --- a/libs/messaging/messaging/src/main/java/net/corda/messaging/api/mediator/statemanager/package-info.java +++ /dev/null @@ -1,4 +0,0 @@ -@Export -package net.corda.messaging.api.mediator.statemanager; - -import org.osgi.annotation.bundle.Export; diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MessagingClient.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MessagingClient.kt index a4d13cbed4e..9cfafa982c2 100644 --- a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MessagingClient.kt +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MessagingClient.kt @@ -6,6 +6,11 @@ import kotlinx.coroutines.Deferred * Multi-source event mediator messaging client. */ interface MessagingClient : AutoCloseable { + companion object { + /** Name of the property for specifying the endpoint string */ + const val MSG_PROP_ENDPOINT = "clientEndpoint" + } + /** * Messaging client's unique ID. */ @@ -15,9 +20,8 @@ interface MessagingClient : AutoCloseable { * Asynchronously sends a generic [MediatorMessage], and returns any result/error through a [Deferred] response. * * @param message The [MediatorMessage] to send. - * @param endpoint Endpoint to which the message is sent to. * @return [Deferred] instance representing the asynchronous computation result, or null if the destination doesn't * provide a response. * */ - fun send(message: MediatorMessage<*>, endpoint: String) : Deferred?> + fun send(message: MediatorMessage<*>): Deferred?> } From 31f88c41ee90d8ed09b70a16906f65970d35f33a Mon Sep 17 00:00:00 2001 From: Miljenko Brkic Date: Fri, 29 Sep 2023 09:18:48 +0100 Subject: [PATCH 39/50] CORE-16199 Refactored MessageBusClient --- .../messaging/mediator/MessageBusClient.kt | 3 ++- .../mediator/MessageBusClientTest.kt | 11 ++++++---- .../messaging/mediator/ProcessorTaskTest.kt | 21 ++++--------------- 3 files changed, 13 insertions(+), 22 deletions(-) diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MessageBusClient.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MessageBusClient.kt index 1ad0afc7ca5..5ce70e25c10 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MessageBusClient.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MessageBusClient.kt @@ -6,6 +6,7 @@ import net.corda.messagebus.api.producer.CordaProducer import net.corda.messagebus.api.producer.CordaProducerRecord import net.corda.messaging.api.mediator.MediatorMessage import net.corda.messaging.api.mediator.MessagingClient +import net.corda.messaging.api.mediator.MessagingClient.Companion.MSG_PROP_ENDPOINT import org.slf4j.Logger import org.slf4j.LoggerFactory @@ -42,7 +43,7 @@ class MessageBusClient( private fun MediatorMessage<*>.toCordaProducerRecord() : CordaProducerRecord<*, *> { return CordaProducerRecord( - topic = this.getProperty("topic"), + topic = this.getProperty(MSG_PROP_ENDPOINT), key = this.getProperty("key"), value = this.payload, headers = this.getProperty("headers"), diff --git a/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/MessageBusClientTest.kt b/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/MessageBusClientTest.kt index f8d4c35fdb0..95645460fab 100644 --- a/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/MessageBusClientTest.kt +++ b/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/MessageBusClientTest.kt @@ -4,6 +4,7 @@ import kotlinx.coroutines.runBlocking import net.corda.messagebus.api.producer.CordaProducer import net.corda.messagebus.api.producer.CordaProducerRecord import net.corda.messaging.api.mediator.MediatorMessage +import net.corda.messaging.api.mediator.MessagingClient.Companion.MSG_PROP_ENDPOINT import net.corda.v5.base.exceptions.CordaRuntimeException import org.junit.jupiter.api.BeforeEach import org.junit.jupiter.api.Test @@ -20,6 +21,7 @@ class MessageBusClientTest { private companion object { const val MSG_PROP_KEY = "key" const val TEST_ENDPOINT = "topic" + const val TEST_KEY = "key" } private lateinit var cordaProducer: CordaProducer @@ -27,7 +29,8 @@ class MessageBusClientTest { private val defaultHeaders: List> = emptyList() private val messageProps: MutableMap = mutableMapOf( - MSG_PROP_KEY to "key", + MSG_PROP_ENDPOINT to TEST_ENDPOINT, + MSG_PROP_KEY to TEST_KEY, "headers" to defaultHeaders ) private val message: MediatorMessage = MediatorMessage("value", messageProps) @@ -41,7 +44,7 @@ class MessageBusClientTest { @Test fun testSend() { - messageBusClient.send(message, TEST_ENDPOINT) + messageBusClient.send(message) val expected = CordaProducerRecord( TEST_ENDPOINT, @@ -56,14 +59,14 @@ class MessageBusClientTest { fun testSendWithError() { val record = CordaProducerRecord( TEST_ENDPOINT, - message.getProperty(MSG_PROP_KEY), + TEST_KEY, message.payload ) doThrow(CordaRuntimeException("")).whenever(cordaProducer).send(eq(record), any()) assertThrows { runBlocking { - messageBusClient.send(message, TEST_ENDPOINT).await() + messageBusClient.send(message).await() } } } diff --git a/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/ProcessorTaskTest.kt b/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/ProcessorTaskTest.kt index 8cbc6f5049f..5b9e9b25e3d 100644 --- a/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/ProcessorTaskTest.kt +++ b/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/ProcessorTaskTest.kt @@ -1,24 +1,9 @@ package net.corda.messaging.mediator -import kotlinx.coroutines.runBlocking -import net.corda.libs.statemanager.api.State -import net.corda.messagebus.api.consumer.CordaConsumerRecord import net.corda.messagebus.api.producer.CordaProducer -import net.corda.messagebus.api.producer.CordaProducerRecord import net.corda.messaging.api.mediator.MediatorMessage - import net.corda.messaging.api.processor.StateAndEventProcessor -import net.corda.v5.base.exceptions.CordaRuntimeException import org.junit.jupiter.api.BeforeEach -import org.junit.jupiter.api.Test -import org.junit.jupiter.api.assertThrows -import org.mockito.Mockito.doThrow -import org.mockito.Mockito.times -import org.mockito.kotlin.any -import org.mockito.kotlin.eq import org.mockito.kotlin.mock -import org.mockito.kotlin.verify -import org.mockito.kotlin.whenever -import java.time.Instant class ProcessorTaskTest { companion object { @@ -41,7 +26,7 @@ class ProcessorTaskTest { cordaProducer = mock() mediatorProducer = MessageBusClient("client-id", cordaProducer) } - +/* @Test fun `successfully processes messages without initial state`() { val key = "key" @@ -89,7 +74,7 @@ class ProcessorTaskTest { doThrow(CordaRuntimeException("")).whenever(cordaProducer).send(eq(record), any()) assertThrows { runBlocking { - mediatorProducer.send(message, TOPIC).await() + mediatorProducer.send(message).await() } } } @@ -99,4 +84,6 @@ class ProcessorTaskTest { mediatorProducer.close() verify(cordaProducer, times(1)).close() } + + */ } \ No newline at end of file From 8db1cb42c7f051b5557297472862a3385fbd95f3 Mon Sep 17 00:00:00 2001 From: Miljenko Brkic Date: Fri, 29 Sep 2023 10:55:14 +0100 Subject: [PATCH 40/50] CORE-16199 Made polling consumers async --- .../messaging/mediator/MessageBusConsumer.kt | 10 ++++++++-- .../mediator/MultiSourceEventMediatorImpl.kt | 16 ++++++++-------- .../messaging/api/mediator/MediatorConsumer.kt | 2 +- 3 files changed, 17 insertions(+), 11 deletions(-) diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MessageBusConsumer.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MessageBusConsumer.kt index e04b80c0c6c..04b4258ed70 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MessageBusConsumer.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MessageBusConsumer.kt @@ -20,8 +20,14 @@ class MessageBusConsumer( override fun subscribe() = consumer.subscribe(topic) - override fun poll(timeout: Duration): List> = - consumer.poll(timeout) + override fun poll(timeout: Duration): Deferred>> = + CompletableDeferred>>().apply { + try { + complete(consumer.poll(timeout)) + } catch (throwable: Throwable) { + completeExceptionally(throwable) + } + } override fun asyncCommitOffsets(): Deferred> = CompletableDeferred>().apply { diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt index 9634ab748f6..34aee016da5 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt @@ -20,7 +20,6 @@ import net.corda.messaging.api.mediator.taskmanager.TaskType import net.corda.messaging.mediator.factory.MediatorComponentFactory import net.corda.utilities.debug import org.slf4j.LoggerFactory -import java.time.Duration import java.util.UUID @Suppress("LongParameterList") @@ -47,7 +46,6 @@ class MultiSourceEventMediatorImpl( private val taskManagerHelper = TaskManagerHelper( taskManager, stateManagerHelper ) - private val pollTimeoutInNanos = config.pollTimeout.toNanos() private val uniqueId = UUID.randomUUID().toString() private val lifecycleCoordinatorName = LifecycleCoordinatorName( "MultiSourceEventMediator--${config.name}", uniqueId @@ -157,7 +155,7 @@ class MultiSourceEventMediatorImpl( private fun processEvents() { log.debug { "Polling and processing events" } - val messages = poll(pollTimeoutInNanos) + val messages = pollConsumers() if (messages.isNotEmpty()) { val msgGroups = messages.groupBy { it.key } val persistedStates = stateManager.get(msgGroups.keys.map { it.toString() }) @@ -180,11 +178,13 @@ class MultiSourceEventMediatorImpl( } } - private fun poll(pollTimeoutInNanos: Long): List> { - val maxEndTime = System.nanoTime() + pollTimeoutInNanos - return consumers.map { consumer -> - val remainingTime = (maxEndTime - System.nanoTime()).coerceAtLeast(0) - consumer.poll(Duration.ofNanos(remainingTime)) + private fun pollConsumers(): List> { + return runBlocking { + consumers.map { consumer -> + consumer.poll(config.pollTimeout) + }.map { + it.await() + } }.flatten() } diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MediatorConsumer.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MediatorConsumer.kt index 68204b1aa1f..79e53453d2a 100644 --- a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MediatorConsumer.kt +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MediatorConsumer.kt @@ -20,7 +20,7 @@ interface MediatorConsumer : AutoCloseable { * * @param timeout - The maximum time to block if there are no available messages. */ - fun poll(timeout: Duration): List> + fun poll(timeout: Duration): Deferred>> /** * Asynchronously commit the consumer offsets. This function should be called only after `poll` was called. From 5b47423ee64776b8c2e3923433fea5ae04624e7f Mon Sep 17 00:00:00 2001 From: Miljenko Brkic Date: Fri, 29 Sep 2023 12:11:22 +0100 Subject: [PATCH 41/50] CORE-16199 Added unit test testPollWithError --- .../messaging/mediator/MessageBusConsumerTest.kt | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/MessageBusConsumerTest.kt b/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/MessageBusConsumerTest.kt index 7dae5ce28f9..a4de4df2323 100644 --- a/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/MessageBusConsumerTest.kt +++ b/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/MessageBusConsumerTest.kt @@ -55,6 +55,18 @@ class MessageBusConsumerTest { verify(cordaConsumer).poll(eq(timeout)) } + @Test + fun testPollWithError() { + val timeout = Duration.ofMillis(123) + doThrow(CordaRuntimeException("")).whenever(cordaConsumer).poll(any()) + + assertThrows { + runBlocking { + mediatorConsumer.poll(timeout).await() + } + } + } + @Test fun testCommitAsyncOffsets() { mediatorConsumer.asyncCommitOffsets() From 71668688d62e71adc4d47b9b42a082c8e80bbf18 Mon Sep 17 00:00:00 2001 From: Miljenko Brkic Date: Tue, 3 Oct 2023 09:33:38 +0100 Subject: [PATCH 42/50] CORE-16199 Added factories for creating Multi-Source Event Mediators for Flow Engine and Flow Mapper --- .../messaging/mediator/package-info.java | 4 + .../service/messaging/package-info.java | 4 + .../FlowMapperEventMediatorFactory.kt | 23 ++++ .../FlowMapperEventMediatorFactoryImpl.kt | 86 +++++++++++++ .../FlowMapperEventMediatorFactoryImplTest.kt | 45 +++++++ .../flow/messaging/mediator/package-info.java | 4 + .../mediator/FlowEventMediatorFactory.kt | 23 ++++ .../mediator/FlowEventMediatorFactoryImpl.kt | 118 ++++++++++++++++++ .../flow/service/FlowExecutorMediatorImpl.kt | 116 ++--------------- .../FlowEventMediatorFactoryImplTest.kt | 51 ++++++++ .../corda/messaging/mediator/ProcessorTask.kt | 6 +- .../messaging/mediator/TaskManagerHelper.kt | 25 ++-- 12 files changed, 380 insertions(+), 125 deletions(-) create mode 100644 components/flow/flow-mapper-service/src/main/java/net/corda/session/mapper/service/messaging/mediator/package-info.java create mode 100644 components/flow/flow-mapper-service/src/main/java/net/corda/session/mapper/service/messaging/package-info.java create mode 100644 components/flow/flow-mapper-service/src/main/kotlin/net/corda/session/mapper/messaging/mediator/FlowMapperEventMediatorFactory.kt create mode 100644 components/flow/flow-mapper-service/src/main/kotlin/net/corda/session/mapper/messaging/mediator/FlowMapperEventMediatorFactoryImpl.kt create mode 100644 components/flow/flow-mapper-service/src/test/kotlin/net/corda/session/mapper/service/messaging/mediator/FlowMapperEventMediatorFactoryImplTest.kt create mode 100644 components/flow/flow-service/src/main/java/net/corda/flow/messaging/mediator/package-info.java create mode 100644 components/flow/flow-service/src/main/kotlin/net/corda/flow/messaging/mediator/FlowEventMediatorFactory.kt create mode 100644 components/flow/flow-service/src/main/kotlin/net/corda/flow/messaging/mediator/FlowEventMediatorFactoryImpl.kt create mode 100644 components/flow/flow-service/src/test/kotlin/net/corda/flow/messaging/FlowEventMediatorFactoryImplTest.kt diff --git a/components/flow/flow-mapper-service/src/main/java/net/corda/session/mapper/service/messaging/mediator/package-info.java b/components/flow/flow-mapper-service/src/main/java/net/corda/session/mapper/service/messaging/mediator/package-info.java new file mode 100644 index 00000000000..8d183e14ce3 --- /dev/null +++ b/components/flow/flow-mapper-service/src/main/java/net/corda/session/mapper/service/messaging/mediator/package-info.java @@ -0,0 +1,4 @@ +@Export +package net.corda.session.mapper.service.messaging.mediator; + +import org.osgi.annotation.bundle.Export; diff --git a/components/flow/flow-mapper-service/src/main/java/net/corda/session/mapper/service/messaging/package-info.java b/components/flow/flow-mapper-service/src/main/java/net/corda/session/mapper/service/messaging/package-info.java new file mode 100644 index 00000000000..2ad66720cc3 --- /dev/null +++ b/components/flow/flow-mapper-service/src/main/java/net/corda/session/mapper/service/messaging/package-info.java @@ -0,0 +1,4 @@ +@Export +package net.corda.session.mapper.service.messaging; + +import org.osgi.annotation.bundle.Export; diff --git a/components/flow/flow-mapper-service/src/main/kotlin/net/corda/session/mapper/messaging/mediator/FlowMapperEventMediatorFactory.kt b/components/flow/flow-mapper-service/src/main/kotlin/net/corda/session/mapper/messaging/mediator/FlowMapperEventMediatorFactory.kt new file mode 100644 index 00000000000..89e76f48d08 --- /dev/null +++ b/components/flow/flow-mapper-service/src/main/kotlin/net/corda/session/mapper/messaging/mediator/FlowMapperEventMediatorFactory.kt @@ -0,0 +1,23 @@ +package net.corda.session.mapper.messaging.mediator + +import net.corda.data.flow.event.mapper.FlowMapperEvent +import net.corda.data.flow.state.mapper.FlowMapperState +import net.corda.libs.configuration.SmartConfig +import net.corda.messaging.api.mediator.MultiSourceEventMediator + +/** + * Creates a Multi-Source Event Mediator for FlowMapper. + */ +interface FlowMapperEventMediatorFactory { + /** + * Creates a Multi-Source Event Mediator for FlowMapper. + * + * @param flowConfig Flow configuration. + * @param messagingConfig Messaging configuration. + */ + fun create( + flowConfig: SmartConfig, + messagingConfig: SmartConfig, + ): MultiSourceEventMediator + +} \ No newline at end of file diff --git a/components/flow/flow-mapper-service/src/main/kotlin/net/corda/session/mapper/messaging/mediator/FlowMapperEventMediatorFactoryImpl.kt b/components/flow/flow-mapper-service/src/main/kotlin/net/corda/session/mapper/messaging/mediator/FlowMapperEventMediatorFactoryImpl.kt new file mode 100644 index 00000000000..22c7937428f --- /dev/null +++ b/components/flow/flow-mapper-service/src/main/kotlin/net/corda/session/mapper/messaging/mediator/FlowMapperEventMediatorFactoryImpl.kt @@ -0,0 +1,86 @@ +package net.corda.session.mapper.messaging.mediator + +import net.corda.data.flow.event.FlowEvent +import net.corda.data.flow.event.mapper.FlowMapperEvent +import net.corda.data.flow.state.mapper.FlowMapperState +import net.corda.data.p2p.app.AppMessage +import net.corda.flow.mapper.factory.FlowMapperEventExecutorFactory +import net.corda.libs.configuration.SmartConfig +import net.corda.messaging.api.mediator.MessageRouter +import net.corda.messaging.api.mediator.RoutingDestination.Companion.routeTo +import net.corda.messaging.api.mediator.config.EventMediatorConfigBuilder +import net.corda.messaging.api.mediator.factory.MediatorConsumerFactoryFactory +import net.corda.messaging.api.mediator.factory.MessageRouterFactory +import net.corda.messaging.api.mediator.factory.MessagingClientFactoryFactory +import net.corda.messaging.api.mediator.factory.MultiSourceEventMediatorFactory +import net.corda.messaging.api.processor.StateAndEventProcessor +import net.corda.schema.Schemas.Flow.FLOW_EVENT_TOPIC +import net.corda.schema.Schemas.Flow.FLOW_MAPPER_EVENT_TOPIC +import net.corda.schema.Schemas.P2P.P2P_OUT_TOPIC +import net.corda.session.mapper.service.executor.FlowMapperMessageProcessor +import org.osgi.service.component.annotations.Activate +import org.osgi.service.component.annotations.Component +import org.osgi.service.component.annotations.Reference + +@Component(service = [FlowMapperEventMediatorFactory::class]) +class FlowMapperEventMediatorFactoryImpl @Activate constructor( + @Reference(service = FlowMapperEventExecutorFactory::class) + private val flowMapperEventExecutorFactory: FlowMapperEventExecutorFactory, + @Reference(service = MediatorConsumerFactoryFactory::class) + private val mediatorConsumerFactoryFactory: MediatorConsumerFactoryFactory, + @Reference(service = MessagingClientFactoryFactory::class) + private val messagingClientFactoryFactory: MessagingClientFactoryFactory, + @Reference(service = MultiSourceEventMediatorFactory::class) + private val eventMediatorFactory: MultiSourceEventMediatorFactory, +) : FlowMapperEventMediatorFactory { + companion object { + private const val CONSUMER_GROUP = "FlowMapperConsumer" + private const val MESSAGE_BUS_CLIENT = "MessageBusClient" + } + + override fun create( + flowConfig: SmartConfig, + messagingConfig: SmartConfig, + ) = eventMediatorFactory.create( + createEventMediatorConfig( + messagingConfig, + FlowMapperMessageProcessor(flowMapperEventExecutorFactory, flowConfig), + ) + ) + + private fun createEventMediatorConfig( + messagingConfig: SmartConfig, + messageProcessor: StateAndEventProcessor, + ) = EventMediatorConfigBuilder() + .name("FlowMapperEventMediator") + .messagingConfig(messagingConfig) + .consumerFactories( + mediatorConsumerFactoryFactory.createMessageBusConsumerFactory( + FLOW_MAPPER_EVENT_TOPIC, CONSUMER_GROUP, messagingConfig + ), + ) + .clientFactories( + messagingClientFactoryFactory.createMessageBusClientFactory( + MESSAGE_BUS_CLIENT, messagingConfig + ), + ) + .messageProcessor(messageProcessor) + .messageRouterFactory(createMessageRouterFactory()) + .build() + + private fun createMessageRouterFactory() = MessageRouterFactory { clientFinder -> + val messageBusClient = clientFinder.find(MESSAGE_BUS_CLIENT) + + MessageRouter { message -> + when (val event = message.payload) { + is AppMessage -> routeTo(messageBusClient, P2P_OUT_TOPIC) + is FlowEvent -> routeTo(messageBusClient, FLOW_EVENT_TOPIC) + is FlowMapperEvent -> routeTo(messageBusClient, FLOW_MAPPER_EVENT_TOPIC) + else -> { + val eventType = event?.let { it::class.java } + throw IllegalStateException("No route defined for event type [$eventType]") + } + } + } + } +} \ No newline at end of file diff --git a/components/flow/flow-mapper-service/src/test/kotlin/net/corda/session/mapper/service/messaging/mediator/FlowMapperEventMediatorFactoryImplTest.kt b/components/flow/flow-mapper-service/src/test/kotlin/net/corda/session/mapper/service/messaging/mediator/FlowMapperEventMediatorFactoryImplTest.kt new file mode 100644 index 00000000000..b8b3b751c93 --- /dev/null +++ b/components/flow/flow-mapper-service/src/test/kotlin/net/corda/session/mapper/service/messaging/mediator/FlowMapperEventMediatorFactoryImplTest.kt @@ -0,0 +1,45 @@ +package net.corda.session.mapper.service.messaging.mediator + +import net.corda.data.flow.event.mapper.FlowMapperEvent +import net.corda.data.flow.state.mapper.FlowMapperState +import net.corda.flow.mapper.factory.FlowMapperEventExecutorFactory +import net.corda.messaging.api.mediator.config.EventMediatorConfig +import net.corda.messaging.api.mediator.factory.MediatorConsumerFactoryFactory +import net.corda.messaging.api.mediator.factory.MessagingClientFactoryFactory +import net.corda.messaging.api.mediator.factory.MultiSourceEventMediatorFactory +import net.corda.session.mapper.messaging.mediator.FlowMapperEventMediatorFactory +import net.corda.session.mapper.messaging.mediator.FlowMapperEventMediatorFactoryImpl +import org.junit.jupiter.api.Assertions.assertNotNull +import org.junit.jupiter.api.BeforeEach +import org.junit.jupiter.api.Test +import org.mockito.Mockito.`when` +import org.mockito.kotlin.any +import org.mockito.kotlin.mock + +class FlowMapperEventMediatorFactoryImplTest { + private lateinit var flowMapperEventMediatorFactory: FlowMapperEventMediatorFactory + private val flowMapperEventExecutorFactory = mock() + private val mediatorConsumerFactoryFactory = mock() + private val messagingClientFactoryFactory = mock() + private val multiSourceEventMediatorFactory = mock() + + @BeforeEach + fun beforeEach() { + `when`(multiSourceEventMediatorFactory.create(any>())) + .thenReturn(mock()) + + flowMapperEventMediatorFactory = FlowMapperEventMediatorFactoryImpl( + flowMapperEventExecutorFactory, + mediatorConsumerFactoryFactory, + messagingClientFactoryFactory, + multiSourceEventMediatorFactory, + ) + } + + @Test + fun `successfully creates event mediator`() { + val mediator = flowMapperEventMediatorFactory.create(mock(), mock()) + + assertNotNull(mediator) + } +} \ No newline at end of file diff --git a/components/flow/flow-service/src/main/java/net/corda/flow/messaging/mediator/package-info.java b/components/flow/flow-service/src/main/java/net/corda/flow/messaging/mediator/package-info.java new file mode 100644 index 00000000000..16aec2513b5 --- /dev/null +++ b/components/flow/flow-service/src/main/java/net/corda/flow/messaging/mediator/package-info.java @@ -0,0 +1,4 @@ +@Export +package net.corda.flow.messaging.mediator; + +import org.osgi.annotation.bundle.Export; \ No newline at end of file diff --git a/components/flow/flow-service/src/main/kotlin/net/corda/flow/messaging/mediator/FlowEventMediatorFactory.kt b/components/flow/flow-service/src/main/kotlin/net/corda/flow/messaging/mediator/FlowEventMediatorFactory.kt new file mode 100644 index 00000000000..f918d87b2e3 --- /dev/null +++ b/components/flow/flow-service/src/main/kotlin/net/corda/flow/messaging/mediator/FlowEventMediatorFactory.kt @@ -0,0 +1,23 @@ +package net.corda.flow.messaging.mediator + +import net.corda.data.flow.event.FlowEvent +import net.corda.data.flow.state.checkpoint.Checkpoint +import net.corda.libs.configuration.SmartConfig +import net.corda.messaging.api.mediator.MultiSourceEventMediator + +/** + * Creates a Multi-Source Event Mediator for flow engine. + */ +interface FlowEventMediatorFactory { + /** + * Creates a Multi-Source Event Mediator for flow engine. + * + * @param configs Map of configurations. + * @param messagingConfig Messaging configuration. + */ + fun create( + configs: Map, + messagingConfig: SmartConfig, + ): MultiSourceEventMediator + +} \ No newline at end of file diff --git a/components/flow/flow-service/src/main/kotlin/net/corda/flow/messaging/mediator/FlowEventMediatorFactoryImpl.kt b/components/flow/flow-service/src/main/kotlin/net/corda/flow/messaging/mediator/FlowEventMediatorFactoryImpl.kt new file mode 100644 index 00000000000..c6c5c1435fc --- /dev/null +++ b/components/flow/flow-service/src/main/kotlin/net/corda/flow/messaging/mediator/FlowEventMediatorFactoryImpl.kt @@ -0,0 +1,118 @@ +package net.corda.flow.messaging.mediator + +import net.corda.avro.serialization.CordaAvroSerializationFactory +import net.corda.data.crypto.wire.ops.flow.FlowOpsRequest +import net.corda.data.flow.event.FlowEvent +import net.corda.data.flow.event.mapper.FlowMapperEvent +import net.corda.data.flow.output.FlowStatus +import net.corda.data.flow.state.checkpoint.Checkpoint +import net.corda.data.ledger.persistence.LedgerPersistenceRequest +import net.corda.data.persistence.EntityRequest +import net.corda.data.uniqueness.UniquenessCheckRequestAvro +import net.corda.flow.pipeline.factory.FlowEventProcessorFactory +import net.corda.ledger.utxo.verification.TransactionVerificationRequest +import net.corda.libs.configuration.SmartConfig +import net.corda.messaging.api.mediator.MediatorMessage +import net.corda.messaging.api.mediator.MessageRouter +import net.corda.messaging.api.mediator.RoutingDestination.Companion.routeTo +import net.corda.messaging.api.mediator.config.EventMediatorConfigBuilder +import net.corda.messaging.api.mediator.factory.MediatorConsumerFactoryFactory +import net.corda.messaging.api.mediator.factory.MessageRouterFactory +import net.corda.messaging.api.mediator.factory.MessagingClientFactoryFactory +import net.corda.messaging.api.mediator.factory.MultiSourceEventMediatorFactory +import net.corda.messaging.api.processor.StateAndEventProcessor +import net.corda.schema.Schemas.Crypto.FLOW_OPS_MESSAGE_TOPIC +import net.corda.schema.Schemas.Flow.FLOW_EVENT_TOPIC +import net.corda.schema.Schemas.Flow.FLOW_MAPPER_EVENT_TOPIC +import net.corda.schema.Schemas.Flow.FLOW_STATUS_TOPIC +import net.corda.schema.Schemas.Persistence.PERSISTENCE_ENTITY_PROCESSOR_TOPIC +import net.corda.schema.Schemas.Persistence.PERSISTENCE_LEDGER_PROCESSOR_TOPIC +import net.corda.schema.Schemas.UniquenessChecker.UNIQUENESS_CHECK_TOPIC +import net.corda.schema.Schemas.Verification.VERIFICATION_LEDGER_PROCESSOR_TOPIC +import org.osgi.service.component.annotations.Activate +import org.osgi.service.component.annotations.Component +import org.osgi.service.component.annotations.Reference + +@Component(service = [FlowEventMediatorFactory::class]) +class FlowEventMediatorFactoryImpl @Activate constructor( + @Reference(service = FlowEventProcessorFactory::class) + private val flowEventProcessorFactory: FlowEventProcessorFactory, + @Reference(service = MediatorConsumerFactoryFactory::class) + private val mediatorConsumerFactoryFactory: MediatorConsumerFactoryFactory, + @Reference(service = MessagingClientFactoryFactory::class) + private val messagingClientFactoryFactory: MessagingClientFactoryFactory, + @Reference(service = MultiSourceEventMediatorFactory::class) + private val eventMediatorFactory: MultiSourceEventMediatorFactory, + @Reference(service = CordaAvroSerializationFactory::class) + cordaAvroSerializationFactory: CordaAvroSerializationFactory, +) : FlowEventMediatorFactory { + companion object { + private const val CONSUMER_GROUP = "FlowEventConsumer" + private const val MESSAGE_BUS_CLIENT = "MessageBusClient" + } + + private val deserializer = cordaAvroSerializationFactory.createAvroDeserializer({}, Any::class.java) + + override fun create( + configs: Map, + messagingConfig: SmartConfig, + ) = eventMediatorFactory.create( + createEventMediatorConfig( + messagingConfig, + flowEventProcessorFactory.create(configs), + ) + ) + + private fun createEventMediatorConfig( + messagingConfig: SmartConfig, + messageProcessor: StateAndEventProcessor, + ) = EventMediatorConfigBuilder() + .name("FlowEventMediator") + .messagingConfig(messagingConfig) + .consumerFactories( + mediatorConsumerFactoryFactory.createMessageBusConsumerFactory( + FLOW_EVENT_TOPIC, CONSUMER_GROUP, messagingConfig + ), + ) + .clientFactories( + messagingClientFactoryFactory.createMessageBusClientFactory( + MESSAGE_BUS_CLIENT, messagingConfig + ), + ) + .messageProcessor(messageProcessor) + .messageRouterFactory(createMessageRouterFactory()) + .build() + + private fun createMessageRouterFactory() = MessageRouterFactory { clientFinder -> + val messageBusClient = clientFinder.find(MESSAGE_BUS_CLIENT) + + MessageRouter { message -> + when (val event = message.event()) { + // TODO Route external events to RPC client after CORE-16181 is done + is EntityRequest -> routeTo(messageBusClient, PERSISTENCE_ENTITY_PROCESSOR_TOPIC) + is FlowMapperEvent -> routeTo(messageBusClient, FLOW_MAPPER_EVENT_TOPIC) + is FlowOpsRequest -> routeTo(messageBusClient, FLOW_OPS_MESSAGE_TOPIC) + is FlowStatus -> routeTo(messageBusClient, FLOW_STATUS_TOPIC) + is LedgerPersistenceRequest -> routeTo(messageBusClient, PERSISTENCE_LEDGER_PROCESSOR_TOPIC) + is TransactionVerificationRequest -> routeTo(messageBusClient, VERIFICATION_LEDGER_PROCESSOR_TOPIC) + is UniquenessCheckRequestAvro -> routeTo(messageBusClient, UNIQUENESS_CHECK_TOPIC) + else -> { + val eventType = event?.let { it::class.java } + throw IllegalStateException("No route defined for event type [$eventType]") + } + } + } + } + + /** + * Deserialized message payload if it is a [ByteArray] (seems to be the case for external events). + */ + private fun MediatorMessage.event(): Any? { + val event = payload + return if (event is ByteArray) { + deserializer.deserialize(event) + } else { + event + } + } +} \ No newline at end of file diff --git a/components/flow/flow-service/src/main/kotlin/net/corda/flow/service/FlowExecutorMediatorImpl.kt b/components/flow/flow-service/src/main/kotlin/net/corda/flow/service/FlowExecutorMediatorImpl.kt index fabaeb8de64..3427fd40d40 100644 --- a/components/flow/flow-service/src/main/kotlin/net/corda/flow/service/FlowExecutorMediatorImpl.kt +++ b/components/flow/flow-service/src/main/kotlin/net/corda/flow/service/FlowExecutorMediatorImpl.kt @@ -1,17 +1,9 @@ package net.corda.flow.service import com.typesafe.config.ConfigValueFactory -import net.corda.avro.serialization.CordaAvroSerializationFactory -import net.corda.data.crypto.wire.ops.flow.FlowOpsRequest import net.corda.data.flow.event.FlowEvent -import net.corda.data.flow.event.mapper.FlowMapperEvent -import net.corda.data.flow.output.FlowStatus import net.corda.data.flow.state.checkpoint.Checkpoint -import net.corda.data.ledger.persistence.LedgerPersistenceRequest -import net.corda.data.persistence.EntityRequest -import net.corda.data.uniqueness.UniquenessCheckRequestAvro -import net.corda.flow.pipeline.factory.FlowEventProcessorFactory -import net.corda.ledger.utxo.verification.TransactionVerificationRequest +import net.corda.flow.messaging.mediator.FlowEventMediatorFactory import net.corda.libs.configuration.SmartConfig import net.corda.libs.configuration.helper.getConfig import net.corda.lifecycle.LifecycleCoordinatorFactory @@ -21,24 +13,7 @@ import net.corda.lifecycle.RegistrationHandle import net.corda.lifecycle.StartEvent import net.corda.lifecycle.StopEvent import net.corda.lifecycle.createCoordinator -import net.corda.messaging.api.mediator.MediatorMessage -import net.corda.messaging.api.mediator.MessageRouter import net.corda.messaging.api.mediator.MultiSourceEventMediator -import net.corda.messaging.api.mediator.RoutingDestination.Companion.routeTo -import net.corda.messaging.api.mediator.config.EventMediatorConfigBuilder -import net.corda.messaging.api.mediator.factory.MediatorConsumerFactoryFactory -import net.corda.messaging.api.mediator.factory.MessageRouterFactory -import net.corda.messaging.api.mediator.factory.MessagingClientFactoryFactory -import net.corda.messaging.api.mediator.factory.MultiSourceEventMediatorFactory -import net.corda.messaging.api.processor.StateAndEventProcessor -import net.corda.schema.Schemas.Crypto.FLOW_OPS_MESSAGE_TOPIC -import net.corda.schema.Schemas.Flow.FLOW_EVENT_TOPIC -import net.corda.schema.Schemas.Flow.FLOW_MAPPER_EVENT_TOPIC -import net.corda.schema.Schemas.Flow.FLOW_STATUS_TOPIC -import net.corda.schema.Schemas.Persistence.PERSISTENCE_ENTITY_PROCESSOR_TOPIC -import net.corda.schema.Schemas.Persistence.PERSISTENCE_LEDGER_PROCESSOR_TOPIC -import net.corda.schema.Schemas.UniquenessChecker.UNIQUENESS_CHECK_TOPIC -import net.corda.schema.Schemas.Verification.VERIFICATION_LEDGER_PROCESSOR_TOPIC import net.corda.schema.configuration.ConfigKeys.FLOW_CONFIG import net.corda.schema.configuration.ConfigKeys.MESSAGING_CONFIG import net.corda.schema.configuration.MessagingConfig.MAX_ALLOWED_MSG_SIZE @@ -49,16 +24,11 @@ import org.osgi.service.component.annotations.Component import org.osgi.service.component.annotations.Reference import org.slf4j.LoggerFactory -@Suppress("LongParameterList") // TODO @Component(service = [FlowExecutor::class]) @Component(service = [FlowExecutor::class]) class FlowExecutorMediatorImpl ( coordinatorFactory: LifecycleCoordinatorFactory, - private val flowEventProcessorFactory: FlowEventProcessorFactory, - private val eventMediatorFactory: MultiSourceEventMediatorFactory, - private val mediatorConsumerFactoryFactory: MediatorConsumerFactoryFactory, - private val messagingClientFactoryFactory: MessagingClientFactoryFactory, - cordaAvroSerializationFactory: CordaAvroSerializationFactory, + private val flowEventMediatorFactory: FlowEventMediatorFactory, private val toMessagingConfig: (Map) -> SmartConfig, ) : FlowExecutor { @@ -66,36 +36,21 @@ class FlowExecutorMediatorImpl ( constructor( @Reference(service = LifecycleCoordinatorFactory::class) coordinatorFactory: LifecycleCoordinatorFactory, - @Reference(service = FlowEventProcessorFactory::class) - flowEventProcessorFactory: FlowEventProcessorFactory, - @Reference(service = MultiSourceEventMediatorFactory::class) - eventMediatorFactory: MultiSourceEventMediatorFactory, - @Reference(service = MediatorConsumerFactoryFactory::class) - mediatorConsumerFactoryFactory: MediatorConsumerFactoryFactory, - @Reference(service = MessagingClientFactoryFactory::class) - messagingClientFactoryFactory: MessagingClientFactoryFactory, - @Reference(service = CordaAvroSerializationFactory::class) - cordaAvroSerializationFactory: CordaAvroSerializationFactory, + @Reference(service = FlowEventMediatorFactory::class) + flowEventMediatorFactory: FlowEventMediatorFactory, ) : this( coordinatorFactory, - flowEventProcessorFactory, - eventMediatorFactory, - mediatorConsumerFactoryFactory, - messagingClientFactoryFactory, - cordaAvroSerializationFactory, + flowEventMediatorFactory, { cfg -> cfg.getConfig(MESSAGING_CONFIG) } ) companion object { private val log = LoggerFactory.getLogger(this::class.java.enclosingClass) - private const val CONSUMER_GROUP = "FlowEventConsumer" - private const val MESSAGE_BUS_CLIENT = "MessageBusClient" } private val coordinator = coordinatorFactory.createCoordinator { event, _ -> eventHandler(event) } private var subscriptionRegistrationHandle: RegistrationHandle? = null private var multiSourceEventMediator: MultiSourceEventMediator? = null - private val deserializer = cordaAvroSerializationFactory.createAvroDeserializer({}, Any::class.java) override fun onConfigChange(config: Map) { try { @@ -106,11 +61,9 @@ class FlowExecutorMediatorImpl ( subscriptionRegistrationHandle?.close() multiSourceEventMediator?.close() - multiSourceEventMediator = eventMediatorFactory.create( - createEventMediatorConfig( - messagingConfig, - flowEventProcessorFactory.create(updatedConfigs), - ) + multiSourceEventMediator = flowEventMediatorFactory.create( + updatedConfigs, + messagingConfig, ) subscriptionRegistrationHandle = coordinator.followStatusChangesByName( @@ -167,57 +120,4 @@ class FlowExecutorMediatorImpl ( } } } - - private fun createEventMediatorConfig( - messagingConfig: SmartConfig, - messageProcessor: StateAndEventProcessor, - ) = EventMediatorConfigBuilder() - .name("FlowEventMediator") - .messagingConfig(messagingConfig) - .consumerFactories( - mediatorConsumerFactoryFactory.createMessageBusConsumerFactory( - FLOW_EVENT_TOPIC, CONSUMER_GROUP, messagingConfig - ), - ) - .clientFactories( - messagingClientFactoryFactory.createMessageBusClientFactory( - MESSAGE_BUS_CLIENT, messagingConfig - ), - //RpcClientFactory(CRYPTO_RPC_CLIENT, messagingConfig, cordaRpcBuilder), - ) - .messageProcessor(messageProcessor) - .messageRouterFactory(createMessageRouterFactory()) - .build() - - private fun createMessageRouterFactory() = MessageRouterFactory { clientFinder -> - val messageBusClient = clientFinder.find(MESSAGE_BUS_CLIENT) - - MessageRouter { message -> - when (val event = message.event()) { - is EntityRequest -> routeTo(messageBusClient, PERSISTENCE_ENTITY_PROCESSOR_TOPIC) - is FlowMapperEvent -> routeTo(messageBusClient, FLOW_MAPPER_EVENT_TOPIC) - is FlowOpsRequest -> routeTo(messageBusClient, FLOW_OPS_MESSAGE_TOPIC) - is FlowStatus -> routeTo(messageBusClient, FLOW_STATUS_TOPIC) - is LedgerPersistenceRequest -> routeTo(messageBusClient, PERSISTENCE_LEDGER_PROCESSOR_TOPIC) - is TransactionVerificationRequest -> routeTo(messageBusClient, VERIFICATION_LEDGER_PROCESSOR_TOPIC) - is UniquenessCheckRequestAvro -> routeTo(messageBusClient, UNIQUENESS_CHECK_TOPIC) - else -> { - val eventType = event?.let { it::class.java } - throw IllegalStateException("No route defined for event type [$eventType]") - } - } - } - } - - /** - * Deserialized message payload if it is a [ByteArray] (seems to be the case for external events). - */ - private fun MediatorMessage.event(): Any? { - val event = payload - return if (event is ByteArray) { - deserializer.deserialize(event) - } else { - event - } - } } diff --git a/components/flow/flow-service/src/test/kotlin/net/corda/flow/messaging/FlowEventMediatorFactoryImplTest.kt b/components/flow/flow-service/src/test/kotlin/net/corda/flow/messaging/FlowEventMediatorFactoryImplTest.kt new file mode 100644 index 00000000000..4f5af4bfb6d --- /dev/null +++ b/components/flow/flow-service/src/test/kotlin/net/corda/flow/messaging/FlowEventMediatorFactoryImplTest.kt @@ -0,0 +1,51 @@ +package net.corda.flow.messaging + +import net.corda.avro.serialization.CordaAvroSerializationFactory +import net.corda.data.flow.event.FlowEvent +import net.corda.data.flow.state.checkpoint.Checkpoint +import net.corda.flow.messaging.mediator.FlowEventMediatorFactory +import net.corda.flow.messaging.mediator.FlowEventMediatorFactoryImpl +import net.corda.flow.pipeline.factory.FlowEventProcessorFactory +import net.corda.messaging.api.mediator.config.EventMediatorConfig +import net.corda.messaging.api.mediator.factory.MediatorConsumerFactoryFactory +import net.corda.messaging.api.mediator.factory.MessagingClientFactoryFactory +import net.corda.messaging.api.mediator.factory.MultiSourceEventMediatorFactory +import org.junit.jupiter.api.Assertions.assertNotNull +import org.junit.jupiter.api.BeforeEach +import org.junit.jupiter.api.Test +import org.mockito.Mockito.`when` +import org.mockito.kotlin.any +import org.mockito.kotlin.mock + +class FlowEventMediatorFactoryImplTest { + private lateinit var flowEventMediatorFactory: FlowEventMediatorFactory + private val flowEventProcessorFactory = mock() + private val mediatorConsumerFactoryFactory = mock() + private val messagingClientFactoryFactory = mock() + private val multiSourceEventMediatorFactory = mock() + private val cordaAvroSerializationFactory = mock() + + @BeforeEach + fun beforeEach() { + `when`(flowEventProcessorFactory.create(any())) + .thenReturn(mock()) + + `when`(multiSourceEventMediatorFactory.create(any>())) + .thenReturn(mock()) + + flowEventMediatorFactory = FlowEventMediatorFactoryImpl( + flowEventProcessorFactory, + mediatorConsumerFactoryFactory, + messagingClientFactoryFactory, + multiSourceEventMediatorFactory, + cordaAvroSerializationFactory, + ) + } + + @Test + fun `successfully creates event mediator`() { + val mediator = flowEventMediatorFactory.create(mock(), mock()) + + assertNotNull(mediator) + } +} \ No newline at end of file diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/ProcessorTask.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/ProcessorTask.kt index dc7dc7702f5..d5d77b37b89 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/ProcessorTask.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/ProcessorTask.kt @@ -1,10 +1,8 @@ package net.corda.messaging.mediator import net.corda.libs.statemanager.api.State -import net.corda.messagebus.api.consumer.CordaConsumerRecord import net.corda.messaging.api.processor.StateAndEventProcessor import net.corda.messaging.api.records.Record -import net.corda.messaging.utils.toRecord import java.util.concurrent.Callable /** @@ -16,7 +14,7 @@ import java.util.concurrent.Callable data class ProcessorTask( val key: String, val persistedState: State?, - val events: Collection>, + val events: Collection>, private val processor: StateAndEventProcessor, private val stateManagerHelper: StateManagerHelper, ) : Callable> { @@ -33,7 +31,7 @@ data class ProcessorTask( var stateValue = stateManagerHelper.deserializeValue(persistedState) val outputEvents = events.map { event -> - val response = processor.onNext(stateValue, event.toRecord()) + val response = processor.onNext(stateValue, event) response.updatedState?.let { stateValue = it } response.responseEvents }.flatten() diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/TaskManagerHelper.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/TaskManagerHelper.kt index 75fdd064487..0ccf68eb797 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/TaskManagerHelper.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/TaskManagerHelper.kt @@ -2,13 +2,15 @@ package net.corda.messaging.mediator import kotlinx.coroutines.runBlocking import net.corda.libs.statemanager.api.State +import net.corda.libs.statemanager.api.StateManager import net.corda.messagebus.api.consumer.CordaConsumerRecord import net.corda.messaging.api.mediator.MediatorMessage import net.corda.messaging.api.mediator.MessageRouter import net.corda.messaging.api.mediator.taskmanager.TaskManager import net.corda.messaging.api.mediator.taskmanager.TaskType import net.corda.messaging.api.processor.StateAndEventProcessor -import java.time.Instant +import net.corda.messaging.api.records.Record +import net.corda.messaging.utils.toRecord /** * Helper that creates and executes various tasks used by [MultiSourceEventMediatorImpl]. @@ -33,7 +35,7 @@ internal class TaskManagerHelper( ): List> { return messageGroups.map { msgGroup -> val key = msgGroup.key.toString() - val events = msgGroup.value.map { it } + val events = msgGroup.value.map { it.toRecord() } ProcessorTask( key, persistedStates[key], @@ -58,7 +60,7 @@ internal class TaskManagerHelper( return clientResults.filter { it.hasReply() } .groupBy { it.clientTask.processorTask.persistedState!!.key } .map { (_, clientTaskResults) -> - val messageGroup = clientTaskResults.map { it.toCordaConsumerRecord() } + val messageGroup = clientTaskResults.map { it.toRecord() } clientTaskResults.first().clientTask.processorTask.copy(events = messageGroup) } } @@ -141,15 +143,12 @@ internal class TaskManagerHelper( } /** - * Converts [ClientTask.Result] to [CordaConsumerRecord]. + * Converts [ClientTask.Result] to [Record]. */ - private fun ClientTask.Result.toCordaConsumerRecord() = - (CordaConsumerRecord( - "", - -1, - -1, - clientTask.processorTask.events.first().key, - replyMessage!!.payload, - Instant.now().toEpochMilli() - )) + private fun ClientTask.Result.toRecord() = + Record( + "", + clientTask.processorTask.events.first().key, + replyMessage!!.payload, + ) } \ No newline at end of file From cd0581b10ed4282d135464a11ac3006537061570 Mon Sep 17 00:00:00 2001 From: Miljenko Brkic Date: Tue, 3 Oct 2023 09:53:33 +0100 Subject: [PATCH 43/50] CORE-16199 Fixed compile errors after merge with latest from 5.1 --- .../corda/messaging/mediator/MultiSourceEventMediatorImpl.kt | 4 ++-- .../mediator/factory/MultiSourceEventMediatorFactoryImpl.kt | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt index 34aee016da5..eea5e51d7ec 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt @@ -25,7 +25,7 @@ import java.util.UUID @Suppress("LongParameterList") class MultiSourceEventMediatorImpl( private val config: EventMediatorConfig, - serializer: CordaAvroSerializer, + stateSerializer: CordaAvroSerializer, stateDeserializer: CordaAvroDeserializer, private val stateManager: StateManager, private val taskManager: TaskManager, @@ -41,7 +41,7 @@ class MultiSourceEventMediatorImpl( config.messageProcessor, config.consumerFactories, config.clientFactories, config.messageRouterFactory ) private val stateManagerHelper = StateManagerHelper( - stateManager, serializer, stateDeserializer + stateManager, stateSerializer, stateDeserializer ) private val taskManagerHelper = TaskManagerHelper( taskManager, stateManagerHelper diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/MultiSourceEventMediatorFactoryImpl.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/MultiSourceEventMediatorFactoryImpl.kt index 7c56edfb06c..fb160de51a5 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/MultiSourceEventMediatorFactoryImpl.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/factory/MultiSourceEventMediatorFactoryImpl.kt @@ -27,14 +27,14 @@ class MultiSourceEventMediatorFactoryImpl @Activate constructor( override fun create( eventMediatorConfig: EventMediatorConfig, ): MultiSourceEventMediator { - val serializer = cordaAvroSerializationFactory.createAvroSerializer { } + val stateSerializer = cordaAvroSerializationFactory.createAvroSerializer { } val stateDeserializer = cordaAvroSerializationFactory.createAvroDeserializer( {}, eventMediatorConfig.messageProcessor.stateValueClass ) return MultiSourceEventMediatorImpl( eventMediatorConfig, - serializer, + stateSerializer, stateDeserializer, stateManager, taskManager, From a5a387d7b0a1ce14530057830a890be13443bb46 Mon Sep 17 00:00:00 2001 From: Miljenko Brkic Date: Mon, 9 Oct 2023 14:16:00 +0100 Subject: [PATCH 44/50] CORE-16199 Fixed event processing bugs --- .../db/consumer/DBCordaConsumerImpl.kt | 11 +++++--- .../messaging/mediator/MessageBusClient.kt | 8 +++--- .../mediator/MultiSourceEventMediatorImpl.kt | 2 +- .../corda/messaging/mediator/ProcessorTask.kt | 6 ++--- .../messaging/mediator/StateManagerHelper.kt | 13 +++++---- .../messaging/mediator/TaskManagerHelper.kt | 27 ++++++++++++++----- .../mediator/statemanager/StateManagerImpl.kt | 3 ++- .../messaging/api/mediator/MessagingClient.kt | 3 +++ 8 files changed, 48 insertions(+), 25 deletions(-) diff --git a/libs/messaging/db-message-bus-impl/src/main/kotlin/net/corda/messagebus/db/consumer/DBCordaConsumerImpl.kt b/libs/messaging/db-message-bus-impl/src/main/kotlin/net/corda/messagebus/db/consumer/DBCordaConsumerImpl.kt index 18ccaed6d0b..937b1415199 100644 --- a/libs/messaging/db-message-bus-impl/src/main/kotlin/net/corda/messagebus/db/consumer/DBCordaConsumerImpl.kt +++ b/libs/messaging/db-message-bus-impl/src/main/kotlin/net/corda/messagebus/db/consumer/DBCordaConsumerImpl.kt @@ -204,8 +204,9 @@ internal class DBCordaConsumerImpl constructor( } override fun asyncCommitOffsets(callback: CordaConsumer.Callback?) { - dbAccess.writeOffsets( - lastReadOffset.map { (cordaTopicPartition, offset) -> + try { + dbAccess.writeOffsets( + lastReadOffset.map { (cordaTopicPartition, offset) -> CommittedPositionEntry( cordaTopicPartition.topic, groupId, @@ -214,7 +215,11 @@ internal class DBCordaConsumerImpl constructor( ATOMIC_TRANSACTION, ) } - ) + ) + callback?.onCompletion(lastReadOffset, null) + } catch(e: Exception) { + callback?.onCompletion(emptyMap(), e) + } } override fun syncCommitOffsets(event: CordaConsumerRecord, metaData: String?) { diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MessageBusClient.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MessageBusClient.kt index 5ce70e25c10..092bf193452 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MessageBusClient.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MessageBusClient.kt @@ -7,6 +7,7 @@ import net.corda.messagebus.api.producer.CordaProducerRecord import net.corda.messaging.api.mediator.MediatorMessage import net.corda.messaging.api.mediator.MessagingClient import net.corda.messaging.api.mediator.MessagingClient.Companion.MSG_PROP_ENDPOINT +import net.corda.messaging.api.mediator.MessagingClient.Companion.MSG_PROP_KEY import org.slf4j.Logger import org.slf4j.LoggerFactory @@ -44,10 +45,11 @@ class MessageBusClient( private fun MediatorMessage<*>.toCordaProducerRecord() : CordaProducerRecord<*, *> { return CordaProducerRecord( topic = this.getProperty(MSG_PROP_ENDPOINT), - key = this.getProperty("key"), + key = this.getProperty(MSG_PROP_KEY), value = this.payload, - headers = this.getProperty("headers"), + headers = this.properties.toHeaders(), ) } -private typealias Headers = List> \ No newline at end of file +private fun Map.toHeaders() = + map { (key, value) -> (key to value.toString())} \ No newline at end of file diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt index 770d67e06b8..9b7522c133b 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt @@ -163,7 +163,7 @@ class MultiSourceEventMediatorImpl( val processingResults = taskManagerHelper.executeProcessorTasks(msgProcessorTasks) val conflictingStates = stateManagerHelper.persistStates(processingResults) val (successResults, failResults) = processingResults.partition { - !conflictingStates.contains(it.key) + !conflictingStates.contains(it.key.toString()) } val clientTasks = taskManagerHelper.createClientTasks(successResults, messageRouter) val clientResults = taskManagerHelper.executeClientTasks(clientTasks) diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/ProcessorTask.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/ProcessorTask.kt index d5d77b37b89..db7016b23a6 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/ProcessorTask.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/ProcessorTask.kt @@ -12,7 +12,7 @@ import java.util.concurrent.Callable */ @Suppress("LongParameterList") data class ProcessorTask( - val key: String, + val key: K, val persistedState: State?, val events: Collection>, private val processor: StateAndEventProcessor, @@ -32,12 +32,12 @@ data class ProcessorTask( val outputEvents = events.map { event -> val response = processor.onNext(stateValue, event) - response.updatedState?.let { stateValue = it } + stateValue = response.updatedState response.responseEvents }.flatten() val updatedState = stateManagerHelper.createOrUpdateState( - key, + key.toString(), persistedState, stateValue ) diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/StateManagerHelper.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/StateManagerHelper.kt index 464719439d2..3d43146696b 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/StateManagerHelper.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/StateManagerHelper.kt @@ -43,14 +43,12 @@ class StateManagerHelper( * the meantime). */ fun persistStates(processorTaskResults: Collection>): Map { - val states = processorTaskResults.mapNotNull { result -> - result.updatedState - } - val (newStates, existingStates) = states.partition { state -> - state.version == State.VERSION_INITIAL_VALUE + val (newStateTasks, existingStateTasks) = processorTaskResults.partition { result -> + result.processorTask.persistedState == null } val latestValuesForFailedStates = mutableMapOf() - if (newStates.isNotEmpty()) { + if (newStateTasks.isNotEmpty()) { + val newStates = newStateTasks.mapNotNull { it.updatedState } val failedStatesKeys = stateManager.create(newStates).keys if (failedStatesKeys.isNotEmpty()) { val latestStatesValues = stateManager.get(failedStatesKeys) @@ -59,7 +57,8 @@ class StateManagerHelper( }) } } - if (existingStates.isNotEmpty()) { + if (existingStateTasks.isNotEmpty()) { + val existingStates = existingStateTasks.mapNotNull { it.updatedState } latestValuesForFailedStates.putAll(stateManager.update(existingStates)) } return latestValuesForFailedStates diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/TaskManagerHelper.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/TaskManagerHelper.kt index 5b2b0e92a65..8c69492c6b3 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/TaskManagerHelper.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/TaskManagerHelper.kt @@ -6,6 +6,7 @@ import net.corda.libs.statemanager.api.StateManager import net.corda.messagebus.api.consumer.CordaConsumerRecord import net.corda.messaging.api.mediator.MediatorMessage import net.corda.messaging.api.mediator.MessageRouter +import net.corda.messaging.api.mediator.MessagingClient.Companion.MSG_PROP_KEY import net.corda.messaging.api.mediator.taskmanager.TaskManager import net.corda.messaging.api.mediator.taskmanager.TaskType import net.corda.messaging.api.processor.StateAndEventProcessor @@ -30,7 +31,7 @@ internal class TaskManagerHelper( * Creates [ProcessorTask]s for given events and states. * * @param messageGroups Map of messages keys and related events. - * @param persistedStates Mpa of message keys and related states. + * @param persistedStates Map of message keys and related states. * @param messageProcessor State and event processor. * @return Created [ProcessorTask]s. */ @@ -40,11 +41,11 @@ internal class TaskManagerHelper( messageProcessor: StateAndEventProcessor, ): List> { return messageGroups.map { msgGroup -> - val key = msgGroup.key.toString() + val key = msgGroup.key val events = msgGroup.value.map { it.toRecord() } ProcessorTask( key, - persistedStates[key], + persistedStates[key.toString()], events, messageProcessor, stateManagerHelper, @@ -68,8 +69,9 @@ internal class TaskManagerHelper( .map { (_, clientTaskResults) -> val groupedEvents = clientTaskResults.map { it.toRecord() } with(clientTaskResults.first()) { + val persistedState = processorTaskResult.updatedState!! processorTask.copy( - persistedState = processorTaskResult.updatedState, + persistedState = persistedState.copy(version = persistedState.version + 1), events = groupedEvents ) } @@ -90,7 +92,7 @@ internal class TaskManagerHelper( persistedStates: Map, ): List> { return invalidResults.map { - it.processorTask.copy(persistedState = persistedStates[it.processorTask.key]) + it.processorTask.copy(persistedState = persistedStates[it.processorTask.key.toString()]) } } @@ -124,9 +126,8 @@ internal class TaskManagerHelper( ): List> { return processorTaskResults.map { result -> result.outputEvents.map { event -> - val message = MediatorMessage(event.value!!) ClientTask( - message, + event.toMessage(), messageRouter, result, ) @@ -157,4 +158,16 @@ internal class TaskManagerHelper( processorTask.events.first().key, replyMessage!!.payload, ) + + /** + * Converts [Record] to [MediatorMessage]. + */ + private fun Record<*, *>.toMessage() = + MediatorMessage( + value!!, + headers.toMessageProperties().also { it[MSG_PROP_KEY] = key }, + ) + + private fun List>.toMessageProperties() = + associateTo(mutableMapOf()) { (key, value) -> key to (value as Any) } } \ No newline at end of file diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/statemanager/StateManagerImpl.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/statemanager/StateManagerImpl.kt index 75849e6890f..3210bff48fe 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/statemanager/StateManagerImpl.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/statemanager/StateManagerImpl.kt @@ -26,7 +26,8 @@ class StateManagerImpl @Activate constructor() : StateManager { return states.mapNotNull { val existingState = storage[it.key] if (existingState?.version == it.version) { - storage[it.key] = it + val updatedState = it.copy(version = it.version + 1) + storage[it.key] = updatedState null } else { it diff --git a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MessagingClient.kt b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MessagingClient.kt index 9cfafa982c2..71ea0f32f24 100644 --- a/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MessagingClient.kt +++ b/libs/messaging/messaging/src/main/kotlin/net/corda/messaging/api/mediator/MessagingClient.kt @@ -9,6 +9,9 @@ interface MessagingClient : AutoCloseable { companion object { /** Name of the property for specifying the endpoint string */ const val MSG_PROP_ENDPOINT = "clientEndpoint" + + /** Name of the property for specifying the message key */ + const val MSG_PROP_KEY = "key" } /** From 28225ea3187b31a9c004e415cc4e501f11659e0f Mon Sep 17 00:00:00 2001 From: Miljenko Brkic Date: Tue, 10 Oct 2023 00:33:22 +0100 Subject: [PATCH 45/50] CORE-16199 Fixed unit tests. --- .../messaging/mediator/MessageBusClient.kt | 4 ++-- .../messaging/mediator/MessageBusClientTest.kt | 17 +++++++++++------ .../mediator/MessageBusConsumerTest.kt | 15 ++++----------- .../mediator/StateManagerHelperTest.kt | 17 ++++++++++------- .../messaging/mediator/TaskManagerHelperTest.kt | 17 ++++++++++------- 5 files changed, 37 insertions(+), 33 deletions(-) diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MessageBusClient.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MessageBusClient.kt index 092bf193452..66338f569c0 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MessageBusClient.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MessageBusClient.kt @@ -42,7 +42,7 @@ class MessageBusClient( } } -private fun MediatorMessage<*>.toCordaProducerRecord() : CordaProducerRecord<*, *> { +private fun MediatorMessage<*>.toCordaProducerRecord(): CordaProducerRecord<*, *> { return CordaProducerRecord( topic = this.getProperty(MSG_PROP_ENDPOINT), key = this.getProperty(MSG_PROP_KEY), @@ -52,4 +52,4 @@ private fun MediatorMessage<*>.toCordaProducerRecord() : CordaProducerRecord<*, } private fun Map.toHeaders() = - map { (key, value) -> (key to value.toString())} \ No newline at end of file + map { (key, value) -> (key to value.toString()) } \ No newline at end of file diff --git a/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/MessageBusClientTest.kt b/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/MessageBusClientTest.kt index b4fe76c052f..09ae5ee8869 100644 --- a/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/MessageBusClientTest.kt +++ b/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/MessageBusClientTest.kt @@ -9,7 +9,6 @@ import net.corda.v5.base.exceptions.CordaRuntimeException import org.junit.jupiter.api.BeforeEach import org.junit.jupiter.api.Test import org.junit.jupiter.api.assertThrows -import org.mockito.Mockito.doThrow import org.mockito.Mockito.times import org.mockito.kotlin.any import org.mockito.kotlin.eq @@ -27,11 +26,9 @@ class MessageBusClientTest { private lateinit var cordaProducer: CordaProducer private lateinit var messageBusClient: MessageBusClient - private val defaultHeaders: List> = emptyList() private val messageProps: MutableMap = mutableMapOf( MSG_PROP_ENDPOINT to TEST_ENDPOINT, MSG_PROP_KEY to TEST_KEY, - "headers" to defaultHeaders ) private val message: MediatorMessage = MediatorMessage("value", messageProps) @@ -49,7 +46,8 @@ class MessageBusClientTest { val expected = CordaProducerRecord( TEST_ENDPOINT, TEST_KEY, - message.payload + message.payload, + messageProps.toHeaders(), ) verify(cordaProducer).send(eq(expected), any()) @@ -60,10 +58,14 @@ class MessageBusClientTest { val record = CordaProducerRecord( TEST_ENDPOINT, TEST_KEY, - message.payload + message.payload, + messageProps.toHeaders(), ) - doThrow(CordaRuntimeException("")).whenever(cordaProducer).send(eq(record), any()) + whenever(cordaProducer.send(eq(record), any())).thenAnswer { invocation -> + val callback = invocation.getArgument(1) + callback.onCompletion(CordaRuntimeException("")) + } assertThrows { runBlocking { messageBusClient.send(message).await() @@ -76,4 +78,7 @@ class MessageBusClientTest { messageBusClient.close() verify(cordaProducer, times(1)).close() } + + private fun Map.toHeaders() = + map { (key, value) -> (key to value.toString()) } } \ No newline at end of file diff --git a/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/MessageBusConsumerTest.kt b/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/MessageBusConsumerTest.kt index a4de4df2323..d4d2a466cf4 100644 --- a/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/MessageBusConsumerTest.kt +++ b/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/MessageBusConsumerTest.kt @@ -2,7 +2,6 @@ package net.corda.messaging.mediator import kotlinx.coroutines.runBlocking import net.corda.messagebus.api.consumer.CordaConsumer -import net.corda.messaging.api.mediator.MediatorMessage import net.corda.v5.base.exceptions.CordaRuntimeException import org.junit.jupiter.api.BeforeEach import org.junit.jupiter.api.Test @@ -25,15 +24,6 @@ class MessageBusConsumerTest { private lateinit var cordaConsumer: CordaConsumer private lateinit var mediatorConsumer: MessageBusConsumer - private val defaultHeaders: List> = emptyList() - private val messageProps: MutableMap = mutableMapOf( - "topic" to "topic", - "key" to "key", - "headers" to defaultHeaders - ) - private val message: MediatorMessage = MediatorMessage("value", messageProps) - - @BeforeEach fun setup() { cordaConsumer = mock() @@ -76,7 +66,10 @@ class MessageBusConsumerTest { @Test fun testCommitAsyncOffsetsWithError() { - doThrow(CordaRuntimeException("")).whenever(cordaConsumer).asyncCommitOffsets(any()) + whenever(cordaConsumer.asyncCommitOffsets(any())).thenAnswer { invocation -> + val callback = invocation.getArgument(0) + callback.onCompletion(mock(), CordaRuntimeException("")) + } assertThrows { runBlocking { diff --git a/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/StateManagerHelperTest.kt b/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/StateManagerHelperTest.kt index 90614d791b2..5610a805d5e 100644 --- a/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/StateManagerHelperTest.kt +++ b/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/StateManagerHelperTest.kt @@ -104,23 +104,26 @@ class StateManagerHelperTest { stateDeserializer, ) val states = listOf( - State("1", "1".toByteArray(), 2), - State("2", "2".toByteArray(), State.VERSION_INITIAL_VALUE), - State("3", "3".toByteArray(), 3), + mock() to State("1", "1".toByteArray(), 2), + null to State("2", "2".toByteArray(), State.VERSION_INITIAL_VALUE), + mock() to State("3", "3".toByteArray(), State.VERSION_INITIAL_VALUE), ) stateManagerHelper.persistStates( - states.map { state -> - ProcessorTask.Result(mock(), mock(), state) + states.map { (persistedState, updatedState) -> + val task = ProcessorTask( + updatedState.key, persistedState, mock(), mock(), mock() + ) + ProcessorTask.Result(task, mock(), updatedState) } ) verify(stateManager).create(newStatesCaptor.capture()) val capturedNewStates = newStatesCaptor.firstValue - assertEquals(listOf(states[1]), capturedNewStates) + assertEquals(listOf(states[1]).map { it.second }, capturedNewStates) verify(stateManager).update(updatedStatesCaptor.capture()) val capturedUpdatedStates = updatedStatesCaptor.firstValue - assertEquals(listOf(states[0], states[2]), capturedUpdatedStates) + assertEquals(listOf(states[0], states[2]).map { it.second }, capturedUpdatedStates) } @Test diff --git a/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/TaskManagerHelperTest.kt b/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/TaskManagerHelperTest.kt index 4eeacf89759..34748842f98 100644 --- a/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/TaskManagerHelperTest.kt +++ b/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/TaskManagerHelperTest.kt @@ -5,15 +5,16 @@ import net.corda.libs.statemanager.api.State import net.corda.messagebus.api.consumer.CordaConsumerRecord import net.corda.messaging.api.mediator.MediatorMessage import net.corda.messaging.api.mediator.MessageRouter +import net.corda.messaging.api.mediator.MessagingClient.Companion.MSG_PROP_KEY import net.corda.messaging.api.mediator.taskmanager.TaskManager import net.corda.messaging.api.processor.StateAndEventProcessor +import net.corda.messaging.api.records.Record import org.junit.jupiter.api.Assertions.assertEquals import org.junit.jupiter.api.Test -import org.mockito.kotlin.mock -import net.corda.messaging.api.records.Record import org.mockito.Mockito.`when` import org.mockito.kotlin.any import org.mockito.kotlin.argumentCaptor +import org.mockito.kotlin.mock import org.mockito.kotlin.times import org.mockito.kotlin.verify @@ -67,7 +68,9 @@ class TaskManagerHelperTest { @Test fun `successfully creates message processor tasks from client tasks results`() { - val updateState = mock() + val updateState = State( + KEY2, ByteArray(0), version = 1, mock(), mock() + ) fun clientTaskResult( key: String, @@ -101,7 +104,7 @@ class TaskManagerHelperTest { val expectedProcessorTasks = listOf( ProcessorTask( KEY2, - updateState, + updateState.copy(version = updateState.version + 1), listOf(replyMessage.payload!!).toRecords(KEY2), messageProcessor, stateManagerHelper @@ -195,17 +198,17 @@ class TaskManagerHelperTest { val expectedClientTasks = listOf( ClientTask( - MediatorMessage(EVENT1), + MediatorMessage(EVENT1, mutableMapOf(MSG_PROP_KEY to KEY1)), messageRouter, processorTaskResult1, ), ClientTask( - MediatorMessage(EVENT2), + MediatorMessage(EVENT2, mutableMapOf(MSG_PROP_KEY to KEY2)), messageRouter, processorTaskResult2, ), ClientTask( - MediatorMessage(EVENT3), + MediatorMessage(EVENT3, mutableMapOf(MSG_PROP_KEY to KEY2)), messageRouter, processorTaskResult2, ), From d829ea547cb6fcb29f8e024721be7570a0db5bee Mon Sep 17 00:00:00 2001 From: Miljenko Brkic Date: Tue, 10 Oct 2023 08:05:34 +0100 Subject: [PATCH 46/50] CORE-16199 Fixed unit tests. --- .../net/corda/messaging/mediator/ClientTask.kt | 9 ++++++--- .../corda/messaging/mediator/TaskManagerHelper.kt | 13 ++++--------- .../net/corda/messaging/mediator/ClientTaskTest.kt | 4 +--- .../messaging/mediator/TaskManagerHelperTest.kt | 11 ++++++----- 4 files changed, 17 insertions(+), 20 deletions(-) diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/ClientTask.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/ClientTask.kt index 0b62d338dc0..c93138ee1a3 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/ClientTask.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/ClientTask.kt @@ -1,9 +1,11 @@ package net.corda.messaging.mediator +import kotlinx.coroutines.runBlocking import net.corda.messaging.api.mediator.MediatorMessage import net.corda.messaging.api.mediator.MessageRouter import net.corda.messaging.api.mediator.MessagingClient import net.corda.messaging.api.mediator.MessagingClient.Companion.MSG_PROP_ENDPOINT +import java.util.concurrent.Callable /** * [ClientTask] sends a [MediatorMessage] to [MessagingClient] selected by [MessageRouter]. @@ -12,7 +14,7 @@ data class ClientTask( private val message: MediatorMessage, private val messageRouter: MessageRouter, val processorTaskResult: ProcessorTask.Result, -) : TaskManagerHelper.SuspendCallable> { +) : Callable> { class Result( val clientTask: ClientTask, @@ -24,15 +26,16 @@ data class ClientTask( val processorTaskResult get() = clientTask.processorTaskResult } - override suspend fun call(): Result { + override fun call(): Result { val destination = messageRouter.getDestination(message) @Suppress("UNCHECKED_CAST") - val reply = + val reply = runBlocking { with(destination) { message.addProperty(MSG_PROP_ENDPOINT, endpoint) client.send(message).await() as MediatorMessage? } + } return Result(this, reply) } } \ No newline at end of file diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/TaskManagerHelper.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/TaskManagerHelper.kt index 8c69492c6b3..bc822ef7285 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/TaskManagerHelper.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/TaskManagerHelper.kt @@ -1,6 +1,5 @@ package net.corda.messaging.mediator -import kotlinx.coroutines.runBlocking import net.corda.libs.statemanager.api.State import net.corda.libs.statemanager.api.StateManager import net.corda.messagebus.api.consumer.CordaConsumerRecord @@ -21,12 +20,6 @@ internal class TaskManagerHelper( private val stateManagerHelper: StateManagerHelper, ) { - /** Same as [Callable] but with suspend function call. */ - fun interface SuspendCallable { - @Throws(Exception::class) - suspend fun call(): V - } - /** * Creates [ProcessorTask]s for given events and states. * @@ -144,8 +137,10 @@ internal class TaskManagerHelper( fun executeClientTasks( clientTasks: Collection> ): List> { - return runBlocking { - clientTasks.map { it.call() } + return clientTasks.map { clientTask -> + taskManager.execute(TaskType.SHORT_RUNNING, clientTask::call) + }.map { + it.join() } } diff --git a/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/ClientTaskTest.kt b/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/ClientTaskTest.kt index 99e741dedd3..d28ef8b4b45 100644 --- a/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/ClientTaskTest.kt +++ b/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/ClientTaskTest.kt @@ -56,9 +56,7 @@ class ClientTaskTest { mock(), ) - val result = runBlocking { - task.call() - } + val result = task.call() assertNotNull(result) verify(messageRouter).getDestination(message) diff --git a/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/TaskManagerHelperTest.kt b/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/TaskManagerHelperTest.kt index 34748842f98..5a664098766 100644 --- a/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/TaskManagerHelperTest.kt +++ b/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/TaskManagerHelperTest.kt @@ -1,6 +1,5 @@ package net.corda.messaging.mediator -import kotlinx.coroutines.runBlocking import net.corda.libs.statemanager.api.State import net.corda.messagebus.api.consumer.CordaConsumerRecord import net.corda.messaging.api.mediator.MediatorMessage @@ -221,14 +220,16 @@ class TaskManagerHelperTest { val clientTask1 = mock>() val clientTask2 = mock>() + `when`(taskManager.execute(any(), any<() -> ClientTask.Result>())).thenReturn(mock()) + taskManagerHelper.executeClientTasks( listOf(clientTask1, clientTask2) ) - runBlocking { - verify(clientTask1).call() - verify(clientTask1).call() - } + val commandCaptor = argumentCaptor<() -> ClientTask.Result>() + verify(taskManager, times(2)).execute(any(), commandCaptor.capture()) + assertEquals(clientTask1::call, commandCaptor.firstValue) + assertEquals(clientTask2::call, commandCaptor.secondValue) } private fun List.toCordaConsumerRecords(key: String) = From 4a360569630111d88114307c2f20a368c9f7ef30 Mon Sep 17 00:00:00 2001 From: Miljenko Brkic Date: Tue, 10 Oct 2023 12:08:51 +0100 Subject: [PATCH 47/50] CORE-16199 Added logging to analyse flaky test --- .../mediator/MultiSourceEventMediatorImpl.kt | 12 ++++++++++++ .../mediator/MultiSourceEventMediatorImplTest.kt | 15 +++++++++++++++ 2 files changed, 27 insertions(+) diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt index 9b7522c133b..4f2e38d352b 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImpl.kt @@ -152,26 +152,38 @@ class MultiSourceEventMediatorImpl( private fun processEvents() { log.debug { "Polling and processing events" } + log.info("Polling consumers") val messages = pollConsumers() + log.info("Polling consumers done") if (messages.isNotEmpty()) { val msgGroups = messages.groupBy { it.key } + log.info("Retrieving states from StateManager") val persistedStates = stateManager.get(msgGroups.keys.map { it.toString() }) + log.info("Creating message processor tasks") var msgProcessorTasks = taskManagerHelper.createMessageProcessorTasks( msgGroups, persistedStates, config.messageProcessor ) do { + log.info("Executing processor tasks") val processingResults = taskManagerHelper.executeProcessorTasks(msgProcessorTasks) + log.info("Persisting states") val conflictingStates = stateManagerHelper.persistStates(processingResults) + log.info("Splitting successful/failed states") val (successResults, failResults) = processingResults.partition { !conflictingStates.contains(it.key.toString()) } + log.info("Creating client tasks") val clientTasks = taskManagerHelper.createClientTasks(successResults, messageRouter) + log.info("Executing client tasks") val clientResults = taskManagerHelper.executeClientTasks(clientTasks) + log.info("Generating new tasks") msgProcessorTasks = taskManagerHelper.createMessageProcessorTasks(clientResults) + taskManagerHelper.createMessageProcessorTasks(failResults, conflictingStates) } while (msgProcessorTasks.isNotEmpty()) + log.info("Committing offsets") commitOffsets() + log.info("Committing offsets done") } } diff --git a/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImplTest.kt b/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImplTest.kt index 277167cf19b..a837691fc7e 100644 --- a/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImplTest.kt +++ b/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImplTest.kt @@ -34,11 +34,13 @@ import org.mockito.kotlin.mock import org.mockito.kotlin.times import org.mockito.kotlin.verify import org.mockito.kotlin.whenever +import org.slf4j.LoggerFactory import java.time.Duration import java.util.concurrent.CompletableFuture class MultiSourceEventMediatorImplTest { companion object { + private val log = LoggerFactory.getLogger(this::class.java.enclosingClass) private const val TEST_TIMEOUT_SECONDS = 20L const val KEY1 = "key1" const val KEY2 = "key2" @@ -141,26 +143,39 @@ class MultiSourceEventMediatorImplTest { whenever(consumer.poll(any())).thenAnswer { CompletableDeferred( if (batchNumber < eventBatches.size) { + log.info("Polling mock for batch number [$batchNumber]") eventBatches[batchNumber++] } else { + log.info("Polling mock, all batches consumed") Thread.sleep(10) emptyList() } ) } + log.info("Starting mediator") mediator.start() waitWhile(Duration.ofSeconds(TEST_TIMEOUT_SECONDS)) { batchNumber < eventBatches.size } + log.info("Closing mediator") mediator.close() + log.info("verify(mediatorConsumerFactory)") verify(mediatorConsumerFactory).create(any>()) + log.info("verify(messagingClientFactory)") verify(messagingClientFactory).create(any()) + log.info("verify(messageRouterFactory)") verify(messageRouterFactory).create(any()) + log.info("verify(messageProcessor)") verify(messageProcessor, times(events.size)).onNext(anyOrNull(), any()) + log.info("verify(stateManager).get") verify(stateManager, times(eventBatches.size)).get(any()) + log.info("verify(stateManager).create") verify(stateManager, times(eventBatches.size)).create(any()) + log.info("verify(consumer).poll") verify(consumer, atLeast(eventBatches.size)).poll(any()) + log.info("verify(consumer).asyncCommitOffsets") verify(consumer, times(eventBatches.size)).asyncCommitOffsets() + log.info("verify(messagingClient)") verify(messagingClient, times(events.size)).send(any()) } From 19087942bc4742fea8c396f6e49c17ddf461554e Mon Sep 17 00:00:00 2001 From: Miljenko Brkic Date: Tue, 10 Oct 2023 13:14:59 +0100 Subject: [PATCH 48/50] CORE-16199 Added logging to analyse flaky test --- .../messaging/mediator/MultiSourceEventMediatorImplTest.kt | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImplTest.kt b/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImplTest.kt index 6c3a2f4612c..c6a09e62d00 100644 --- a/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImplTest.kt +++ b/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImplTest.kt @@ -121,8 +121,7 @@ class MultiSourceEventMediatorImplTest { ) } - //@Test - // TODO Test temporarily disabled as it seems to be flaky + @Test fun `mediator processes multiples events by key`() { val events = (1..6).map { "event$it" } val eventBatches = listOf( From f291084427466a4e4fc4c7fb42b5a9ad581e4a61 Mon Sep 17 00:00:00 2001 From: Miljenko Brkic Date: Tue, 10 Oct 2023 13:26:34 +0100 Subject: [PATCH 49/50] CORE-16199 Added logging to analyse flaky test --- .../messaging/mediator/MultiSourceEventMediatorImplTest.kt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImplTest.kt b/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImplTest.kt index c6a09e62d00..a837691fc7e 100644 --- a/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImplTest.kt +++ b/libs/messaging/messaging-impl/src/test/kotlin/net/corda/messaging/mediator/MultiSourceEventMediatorImplTest.kt @@ -26,7 +26,7 @@ import net.corda.messaging.api.processor.StateAndEventProcessor import net.corda.messaging.api.records.Record import net.corda.test.util.waitWhile import org.junit.jupiter.api.BeforeEach -// import org.junit.jupiter.api.Test +import org.junit.jupiter.api.Test import org.mockito.kotlin.any import org.mockito.kotlin.anyOrNull import org.mockito.kotlin.atLeast From 33334424e30df818b28c2b2f35e5a52aee57462b Mon Sep 17 00:00:00 2001 From: Miljenko Brkic Date: Tue, 10 Oct 2023 23:08:29 +0100 Subject: [PATCH 50/50] CORE-16199 Added routing for TokenPoolCacheEvent --- .../mediator/FlowEventMediatorFactoryImpl.kt | 3 +++ .../messaging/mediator/taskmanager/TaskManagerImpl.kt | 11 ++++++++--- 2 files changed, 11 insertions(+), 3 deletions(-) diff --git a/components/flow/flow-service/src/main/kotlin/net/corda/flow/messaging/mediator/FlowEventMediatorFactoryImpl.kt b/components/flow/flow-service/src/main/kotlin/net/corda/flow/messaging/mediator/FlowEventMediatorFactoryImpl.kt index 6b90dd93361..5408e207373 100644 --- a/components/flow/flow-service/src/main/kotlin/net/corda/flow/messaging/mediator/FlowEventMediatorFactoryImpl.kt +++ b/components/flow/flow-service/src/main/kotlin/net/corda/flow/messaging/mediator/FlowEventMediatorFactoryImpl.kt @@ -7,6 +7,7 @@ import net.corda.data.flow.event.mapper.FlowMapperEvent import net.corda.data.flow.output.FlowStatus import net.corda.data.flow.state.checkpoint.Checkpoint import net.corda.data.ledger.persistence.LedgerPersistenceRequest +import net.corda.data.ledger.utxo.token.selection.event.TokenPoolCacheEvent import net.corda.data.persistence.EntityRequest import net.corda.data.uniqueness.UniquenessCheckRequestAvro import net.corda.flow.pipeline.factory.FlowEventProcessorFactory @@ -27,6 +28,7 @@ import net.corda.schema.Schemas.Flow.FLOW_MAPPER_EVENT_TOPIC import net.corda.schema.Schemas.Flow.FLOW_STATUS_TOPIC import net.corda.schema.Schemas.Persistence.PERSISTENCE_ENTITY_PROCESSOR_TOPIC import net.corda.schema.Schemas.Persistence.PERSISTENCE_LEDGER_PROCESSOR_TOPIC +import net.corda.schema.Schemas.Services.TOKEN_CACHE_EVENT import net.corda.schema.Schemas.UniquenessChecker.UNIQUENESS_CHECK_TOPIC import net.corda.schema.Schemas.Verification.VERIFICATION_LEDGER_PROCESSOR_TOPIC import org.osgi.service.component.annotations.Activate @@ -94,6 +96,7 @@ class FlowEventMediatorFactoryImpl @Activate constructor( is FlowOpsRequest -> routeTo(messageBusClient, FLOW_OPS_MESSAGE_TOPIC) is FlowStatus -> routeTo(messageBusClient, FLOW_STATUS_TOPIC) is LedgerPersistenceRequest -> routeTo(messageBusClient, PERSISTENCE_LEDGER_PROCESSOR_TOPIC) + is TokenPoolCacheEvent -> routeTo(messageBusClient, TOKEN_CACHE_EVENT) is TransactionVerificationRequest -> routeTo(messageBusClient, VERIFICATION_LEDGER_PROCESSOR_TOPIC) is UniquenessCheckRequestAvro -> routeTo(messageBusClient, UNIQUENESS_CHECK_TOPIC) else -> { diff --git a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/taskmanager/TaskManagerImpl.kt b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/taskmanager/TaskManagerImpl.kt index dec4a07876a..98a1d8a6c75 100644 --- a/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/taskmanager/TaskManagerImpl.kt +++ b/libs/messaging/messaging-impl/src/main/kotlin/net/corda/messaging/mediator/taskmanager/TaskManagerImpl.kt @@ -21,11 +21,16 @@ class TaskManagerImpl @Activate constructor() : TaskManager { } private fun executeShortRunning(command: () -> T): CompletableFuture { - val result = CompletableFuture() + val resultFuture = CompletableFuture() executorService.execute { - result.complete(command()) + try { + val result = command() + resultFuture.complete(result) + } catch (t: Throwable) { + resultFuture.completeExceptionally(t) + } } - return result + return resultFuture } private fun executeLongRunning(command: () -> T): CompletableFuture {