From 54890ab3e3dbadde3d28a2c8c9506befebee981e Mon Sep 17 00:00:00 2001 From: Simon Vergauwen Date: Mon, 9 Oct 2023 14:52:55 +0200 Subject: [PATCH 01/16] First WIP --- .../kafka/publisher/DefaultKafkaPublisher.kt | 303 ++++++++++++ .../kafka/publisher/KafkaPublisher.kt | 183 +++++++ .../kafka/publisher/PublisherRecord.kt | 50 ++ .../kafka/publisher/PublisherScope.kt | 281 +++++++++++ .../kafka/publisher/PublisherSettings.kt | 32 +- .../nomisRev/kafka/publisher/RecordAck.kt | 48 ++ .../nomisRev/kafka/publisher/SendFlow.kt | 3 + .../nomisRev/kafka/publisher/SendOperator.kt | 30 ++ .../io/github/nomisrev/kafka/KafkaSpec.kt | 74 +-- .../kafka/publisher/KafkaPublisherSpec.kt | 449 ++++++++++++++++++ 10 files changed, 1417 insertions(+), 36 deletions(-) create mode 100644 src/main/kotlin/io/github/nomisRev/kafka/publisher/DefaultKafkaPublisher.kt create mode 100644 src/main/kotlin/io/github/nomisRev/kafka/publisher/KafkaPublisher.kt create mode 100644 src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherRecord.kt create mode 100644 src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherScope.kt create mode 100644 src/main/kotlin/io/github/nomisRev/kafka/publisher/RecordAck.kt create mode 100644 src/main/kotlin/io/github/nomisRev/kafka/publisher/SendFlow.kt create mode 100644 src/main/kotlin/io/github/nomisRev/kafka/publisher/SendOperator.kt create mode 100644 src/test/kotlin/io/github/nomisrev/kafka/publisher/KafkaPublisherSpec.kt diff --git a/src/main/kotlin/io/github/nomisRev/kafka/publisher/DefaultKafkaPublisher.kt b/src/main/kotlin/io/github/nomisRev/kafka/publisher/DefaultKafkaPublisher.kt new file mode 100644 index 00000000..52868340 --- /dev/null +++ b/src/main/kotlin/io/github/nomisRev/kafka/publisher/DefaultKafkaPublisher.kt @@ -0,0 +1,303 @@ +package io.github.nomisRev.kafka.publisher + +import io.github.nomisRev.kafka.Flow.KafkaPublisher +import io.github.nomisRev.kafka.publisher.DefaultKafkaPublisher.State.ACTIVE +import io.github.nomisRev.kafka.publisher.DefaultKafkaPublisher.State.COMPLETE +import io.github.nomisRev.kafka.publisher.DefaultKafkaPublisher.State.DONE_COLLECTING +import kotlinx.coroutines.CoroutineExceptionHandler +import kotlinx.coroutines.DelicateCoroutinesApi +import kotlinx.coroutines.Dispatchers +import kotlinx.coroutines.ExecutorCoroutineDispatcher +import kotlinx.coroutines.ExperimentalCoroutinesApi +import kotlinx.coroutines.GlobalScope +import kotlinx.coroutines.asCoroutineDispatcher +import kotlinx.coroutines.async +import kotlinx.coroutines.channels.awaitClose +import kotlinx.coroutines.channels.onFailure +import kotlinx.coroutines.channels.trySendBlocking +import kotlinx.coroutines.currentCoroutineContext +import kotlinx.coroutines.flow.Flow +import kotlinx.coroutines.flow.channelFlow +import kotlinx.coroutines.flow.flowOn +import kotlinx.coroutines.invoke +import kotlinx.coroutines.newSingleThreadContext +import kotlinx.coroutines.runBlocking +import kotlinx.coroutines.runInterruptible +import org.apache.kafka.clients.producer.Callback +import org.apache.kafka.clients.producer.Producer +import org.apache.kafka.clients.producer.ProducerRecord +import org.apache.kafka.clients.producer.internals.TransactionManager +import org.slf4j.LoggerFactory +import java.time.Duration +import java.util.concurrent.Executors +import java.util.concurrent.atomic.AtomicInteger +import java.util.concurrent.atomic.AtomicReference +import kotlin.time.toJavaDuration + + +/** + * Reactive producer that sends messages to Kafka topic partitions. The producer is thread-safe + * and can be used to send messages to multiple partitions. It is recommended that a single + * producer is shared for each message type in a client. + * + * @param outgoing message key type + * @param outgoing message value type + */ +class DefaultKafkaPublisher( + _createProducer: suspend () -> Producer, + val settings: PublisherSettings, + val onPublisherRecordDropped: suspend (ProducerRecord) -> Unit = + { log.debug("onNextDropped: {}", it) } +) : KafkaPublisher { + private val producerContext: ExecutorCoroutineDispatcher = + Executors.newScheduledThreadPool(1) { runnable -> + Thread(runnable, producerId).apply { + isDaemon = true + } + }.asCoroutineDispatcher() + + @OptIn(ExperimentalCoroutinesApi::class, DelicateCoroutinesApi::class) + val transactionContext: ExecutorCoroutineDispatcher? = + settings.transactionalId()?.let { newSingleThreadContext(it) } + + private val transactionManager: TransactionManager? = null + private val producerId: String = "reactor-kafka-sender-" + System.identityHashCode(this) + + @OptIn(DelicateCoroutinesApi::class) + val producer = GlobalScope.async(producerContext) { + withTransactionContext { + _createProducer().apply { + settings.producerListener.producerAdded(producerId, this) + if (settings.isTransactional()) { + log.info("Initializing transactions for producer {}", settings.transactionalId()) + initTransactions() + } + } + } + } + + enum class State { ACTIVE, DONE_COLLECTING, COMPLETE } + + suspend fun withTransactionContext( + block: suspend () -> A + ): A = transactionContext?.invoke { block() } ?: block() + + override fun send(records: Flow>): Flow> = + doSend(records) + + private fun doSend(records: Flow>): Flow> = + channelFlow> { + val producer = producer.await() + + val inflight = AtomicInteger(0) + val delayedThrowable = AtomicReference(null) + val state: AtomicReference = AtomicReference(ACTIVE) + val ctx = currentCoroutineContext() + val handler = ctx[CoroutineExceptionHandler] + + /* + * **If** in DONE State we transition to COMPLETE. + * We close the channelFlow so that the FlowCollector can complete. + * In case of any errors, the FlowCollector will see non-null Throwable. + */ + fun doneToComplete() { + if (state.compareAndSet(DONE_COLLECTING, COMPLETE)) { + val exception = delayedThrowable.get() + close(exception) + } + } + + /* + * **If** in ACTIVE State then we transition to DONE_COLLECTING the provided records, + * and in-case all inflight records are already acknowledged then we also transition to COMPLETE. + */ + fun activeToDoneOrCompleteIfAlreadyDone() { + if (state.compareAndSet(ACTIVE, DONE_COLLECTING)) { + if (inflight.get() == 0) { + doneToComplete() + } + } + } + + /* + * Set COMPLETE State, and close the channelFlow so that the FlowCollector can complete. + * If the channelFlow was already closed, we send the error to the `CoroutineExceptionHandler`, + * which if null is set to the default logging CoroutineExceptionHandler. + */ + fun complete(t: Throwable?) { + log.trace("Sender failed with exception", t) + state.getAndSet(COMPLETE) + val alreadyClosed = !close(t) + if (alreadyClosed) { + t?.let { handler?.handleException(ctx, t) } + } + } + + records.collect { record: ProducerRecord -> + withTransactionContext { + if (state.get() == COMPLETE) return@withTransactionContext onPublisherRecordDropped(record) + + inflight.incrementAndGet() + if (settings.isTransactional()) { + log.trace( + "Transactional send initiated for producer {} in state {} inflight {}: {}", + settings.transactionalId(), + state, + inflight, + record + ) + } + + val correlationMetadata: A = + @Suppress("UNCHECKED_CAST") + (record as PublisherRecord).correlationMetadata + + val callback = Callback { metadata, exception -> + if (settings.isTransactional()) { + log.trace( + "Transactional send completed for producer {} in state {} inflight {}: {}", + settings.transactionalId(), + state, + inflight, + record + ) + } + + /* A fatal error occurred from a parallel record */ + if (state.get() === COMPLETE) { + return@Callback + } + + if (exception != null) { + log.trace("Sender failed: ", exception) + delayedThrowable.compareAndSet(null, exception) + if (settings.stopOnError || settings.isFatal(exception)) { + return@Callback complete(exception) + } else { + trySendBlocking(RecordAck.Failed(exception, correlationMetadata!!)) + // If Channel is already closed, this exception was already seen before + // + .onFailure { close(it) } + } + } else { + trySendBlocking(RecordAck.Success(metadata, correlationMetadata!!)) + // If Channel is already closed, this exception was already seen before + .onFailure { close(it) } + } + + doneToComplete() + } + + try { + runInterruptible { producer.send(record, callback) } + } catch (e: Exception) { + callback.onCompletion(null, e) + } + } + } + + activeToDoneOrCompleteIfAlreadyDone() + awaitClose { } + }.flowOn(producerContext) + + override fun sendTransitionally(records: Flow>>): Flow>> { + TODO() + // val sink: Sinks.Many = Sinks.many().unicast().onBackpressureBuffer() + // return Flow.from(transactionRecords) + // .publishOn(settings.scheduler(), false, 1) + // .concatMapDelayError({ records -> transaction(records, sink) }, false, 1) + // .window(sink.asFlow()) + // .doOnTerminate { sink.emitComplete(EmitFailureHandler.FAIL_FAST) } + // .doOnCancel { sink.emitComplete(EmitFailureHandler.FAIL_FAST) } + } + + // TODO: Protect with lifecycle + override suspend fun withProducer(function: (Producer) -> T): T = + function(producer.await()) + +// fun createOutbound(): KafkaOutbound { +// return DefaultKafkaOutbound(this) +// } + + override fun transactionManager(): TransactionManager = + requireNotNull(transactionManager) { "Transactions are not enabled" } + + override fun close(): Unit = + if (!producer.isCompleted) Unit + else { + runBlocking(Dispatchers.IO) { + withTransactionContext { + val producer = producer.await() + producer.close( + if (settings.closeTimeout.isInfinite()) Duration.ofMillis(Long.MAX_VALUE) + else settings.closeTimeout.toJavaDuration() + ) + settings.producerListener.producerRemoved(producerId, producer) + } + } + transactionContext?.close() + producerContext.close() + } + +// private fun transaction( +// transactionRecords: Publisher?>, +// transactionBoundary: Sinks.Many +// ): Flow> { +// return transactionManager() +// .begin() +// .thenMany(send(transactionRecords)) +// .concatWith(transactionManager().commit()) +// .concatWith(Mono.fromRunnable { transactionBoundary.emitNext(this, this) }) +// .onErrorResume { e -> transactionManager().abort().then(Mono.error(e)) } +// .publishOn(settings.scheduler()) +// } + +// @Synchronized +// private fun producerProxy(producer: Producer): Producer? { +// if (this.producer == null) { +// val interfaces = arrayOf>(Producer::class.java) +// val handler = +// InvocationHandler { proxy: Any?, method: Method, args: Array -> +// if (DELEGATE_METHODS.contains(method.name)) { +// try { +// return@InvocationHandler method.invoke(producer, *args) +// } catch (e: InvocationTargetException) { +// throw e.cause!! +// } +// } else { +// throw UnsupportedOperationException("Method is not supported: $method") +// } +// } +// this.producer = Proxy.newProxyInstance( +// Producer::class.java.getClassLoader(), +// interfaces, +// handler +// ) as Producer +// } +// return this.producer +// } + +// fun onEmitFailure(signalType: SignalType?, emitResult: Sinks.EmitResult?): Boolean { +// return hasProducer.get() +// } + + companion object { + val log = LoggerFactory.getLogger(DefaultKafkaPublisher::class.java.getName()) + + /** Note: Methods added to this set should also be included in javadoc for [KafkaPublisher.doOnProducer] */ + private val DELEGATE_METHODS: Set = HashSet( + mutableListOf( + "sendOffsetsToTransaction", + "partitionsFor", + "metrics", + "flush" + ) + ) + } +} + +inline fun AtomicReference.loop(action: (V) -> Unit): Nothing { + while (true) { + action(get()) + } +} diff --git a/src/main/kotlin/io/github/nomisRev/kafka/publisher/KafkaPublisher.kt b/src/main/kotlin/io/github/nomisRev/kafka/publisher/KafkaPublisher.kt new file mode 100644 index 00000000..0ee3d448 --- /dev/null +++ b/src/main/kotlin/io/github/nomisRev/kafka/publisher/KafkaPublisher.kt @@ -0,0 +1,183 @@ +package io.github.nomisRev.kafka.Flow + +import io.github.nomisRev.kafka.publisher.DefaultKafkaPublisher +import io.github.nomisRev.kafka.publisher.PublisherRecord +import io.github.nomisRev.kafka.publisher.PublisherSettings +import io.github.nomisRev.kafka.publisher.RecordAck +import kotlinx.coroutines.flow.Flow +import org.apache.kafka.clients.producer.KafkaProducer +import org.apache.kafka.clients.producer.Producer +import org.apache.kafka.clients.producer.ProducerConfig +import org.apache.kafka.clients.producer.internals.TransactionManager + +/** + * Reactive producer that sends outgoing records to topic partitions of a Kafka + * cluster. The producer is thread-safe and can be used to publish records to + * multiple partitions. It is recommended that a single [KafkaPublisher] is shared for each record + * type in a client application. + * + * @param outgoing record key type + * @param outgoing record value type + */ +interface KafkaPublisher : AutoCloseable { + /** + * Sends a sequence of records to Kafka and returns a [Flow] of response record metadata including + * partition and offset of each record. Responses are ordered for each partition in the absence of retries, + * but responses from different partitions may be interleaved in a different order from the requests. + * Additional correlation metadata may be passed through in the [SenderRecord] that is not sent + * to Kafka, but is included in the response [Flow] to match responses to requests. + * + * + * Results are published when the send is acknowledged based on the acknowledgement mode + * configured using the option [ProducerConfig.ACKS_CONFIG]. If acks=0, records are acknowledged + * after the requests are buffered without waiting for any server acknowledgements. In this case the + * requests are not retried and the offset returned in [SenderResult] will be -1. For other ack + * modes, requests are retried up to the configured [ProducerConfig.RETRIES_CONFIG] times. If + * the request does not succeed after these attempts, the request fails and an exception indicating + * the reason for failure is returned in [SenderResult.exception]. + * [SenderOptions.stopOnError] can be configured to stop the send sequence on first failure + * or to attempt all sends even if one or more records could not be delivered. + * + * + * + * Example usage: + *
+   * `source = Flow.range(1, count)
+   * .map(i -> SenderRecord.create(topic, partition, null, key(i), message(i), i));
+   * sender.send(source, true)
+   * .doOnNext(r -> System.out.println("Message #" + r.correlationMetadata() + " metadata=" + r.recordMetadata()));
+  ` *
+  
* + * + * @param records Outbound records along with additional correlation metadata to be included in response + * @return Flow of Kafka producer response record metadata along with the corresponding request correlation metadata. + * For records that could not be sent, the response contains an exception that indicates reason for failure. + */ + fun send(records: Flow>): Flow> + + /** + * Sends records from each inner Flow of `records` within a transaction. + * Each transaction is committed if all the records are successfully delivered to Kafka + * and aborted if any of the records in that batch could not be delivered. + * + * + * Example usage: + *
+   * `sender.sendTransactionally(outboundRecords.window(10));
+  ` *
+  
* + * + * + * @param records Outbound producer records along with correlation metadata to match results returned. + * Records from each inner Flow are sent within a new transaction. + * @return Flow of Kafka producer response record metadata along with the corresponding request correlation metadata. + * Each inner Flow contains results of records sent within a transaction. + * @throws IllegalStateException if the sender was created without setting a non-empty + * {@value org.apache.kafka.clients.producer.ProducerConfig#TRANSACTIONAL_ID_CONFIG} in [SenderOptions] + */ + fun sendTransitionally(records: Flow>>): Flow>> + + /** + * Returns the [TransactionManager] instance associated with this sender, + * which may be used for fine-grained control over transaction states. Sender + * must have been created with a non-empty transactional id by setting + * {@value org.apache.kafka.clients.producer.ProducerConfig#TRANSACTIONAL_ID_CONFIG} in [SenderOptions]. + * + * + * + * **Threading model for transactional sender:** + * + * Sends may be scheduled from multiple threads with a transactional sender similar + * to non-transactional senders. But transaction control operations and offset commits on + * [TransactionManager] must be serialized and no sends may be performed + * while one of the transaction control operations is in progress. + * + * @return [TransactionManager] associated with this sender + * @throws IllegalStateException if the sender was created without setting a non-empty + * {@value org.apache.kafka.clients.producer.ProducerConfig#TRANSACTIONAL_ID_CONFIG} in [SenderOptions] + */ + fun transactionManager(): TransactionManager? + + //private fun transaction( +// transactionRecords: Publisher?>, +// transactionBoundary: Sinks.Many +//): Flux> { +// return transactionManager() +// .begin() +// .thenMany(send(transactionRecords)) +// .concatWith(transactionManager().commit()) +// .concatWith(Mono.fromRunnable { transactionBoundary.emitNext(this, this) }) +// .onErrorResume { e -> transactionManager().abort().then(Mono.error(e)) } +// .publishOn(senderOptions.scheduler()) +//} + + +// /** +// * Creates a reactive gateway for outgoing Kafka records. Outgoing sends can be chained +// * using [KafkaOutbound.send] or [KafkaSender.sendTransactionally]. +// * Like [Flow] and [Mono], subscribing to the tail [KafkaOutbound] will +// * schedule all parent sends in the declaration order. +// * +// * +// * +// * Example usage: +// *
+//   * `kafkaSender.createOutbound()
+//   * .send(Flow1)
+//   * .send(Flow2)
+//   * .send(Flow3)
+//   * .then()
+//   * .subscribe();
+//  ` *
+//  
* +// * +// * @return chainable reactive gateway for outgoing Kafka producer records +// */ +// fun createOutbound(): KafkaOutbound? + + /** + * Invokes the specified function on the Kafka [Producer] associated with this [KafkaPublisher]. + * + * Example usage: + *
+   * `sender.doOnProducer(producer -> producer.partitionsFor(topic))
+   * .doOnSuccess(partitions -> System.out.println("Partitions " + partitions));
+   *
+   *
+   * Functions that are directly supported on the reactive [KafkaPublisher] interface (eg. send)
+   * should not be invoked from `function`. The methods supported by
+   * `doOnProducer` are:
+   *
+   *  * [Producer.sendOffsetsToTransaction]
+   *  * [Producer.partitionsFor]
+   *  * [Producer.metrics]
+   *  * [Producer.flush]
+   *
+   *
+   * @param function A function that takes Kafka Producer as parameter
+   * @return Mono that completes with the value returned by `function`
+   */
+  suspend fun  withProducer(function: (Producer) -> T): T
+
+  /**
+   * Closes this sender and the underlying Kafka producer and releases all resources allocated to it.
+   */
+  override fun close()
+
+  companion object {
+    /**
+     * Creates a Kafka sender that appends records to Kafka topic partitions.
+     *
+     * @param options Configuration options of this sender. Changes made to the options
+     * after the sender is created will not be used by the sender.
+     * @param createProducer Create a custom producer other than the default.
+     * @return new instance of Kafka sender
+     */
+    fun  create(
+      options: PublisherSettings,
+      createProducer: suspend () -> Producer =
+        { KafkaProducer(options.properties(), options.keySerializer, options.valueSerializer) }
+    ): KafkaPublisher =
+      DefaultKafkaPublisher(createProducer, options)
+  }
+}
diff --git a/src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherRecord.kt b/src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherRecord.kt
new file mode 100644
index 00000000..5954a2f1
--- /dev/null
+++ b/src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherRecord.kt
@@ -0,0 +1,50 @@
+package io.github.nomisRev.kafka.publisher
+
+import org.apache.kafka.clients.producer.ProducerRecord
+import org.apache.kafka.common.header.Header
+
+
+/**
+ * Represents an outgoing record. Along with the record to send to Kafka,
+ * additional correlation metadata may also be specified to correlate
+ * [PublisherRecord] to its corresponding record.
+ *
+ * @param  Outgoing record key type
+ * @param  Outgoing record value type
+ * @param  Correlation metadata type
+ */
+class PublisherRecord
+/**
+ *  @param topic Topic to which record is sent
+ *  @param partition The partition to which the record is sent. If null, the partitioner configured
+ *  for the [KafkaSender] will be used to choose the partition.
+ *  @param timestamp The timestamp of the record. If null, the current timestamp will be assigned by the producer.
+ *  The timestamp will be overwritten by the broker if the topic is configured with
+ *  [org.apache.kafka.common.record.TimestampType.LOG_APPEND_TIME]. The actual timestamp
+ *  used will be returned in [SenderResult.recordMetadata]
+ *  @param key The key to be included in the record. May be null.
+ *  @param value The contents to be included in the record.
+ *  @param correlationMetadata Additional correlation metadata that is not sent to Kafka, but is
+ *  included in the response to match [SenderResult] to this record.
+ *  @return new sender record that can be sent to Kafka using [KafkaSender.send]
+ */
+private constructor(
+  topic: String,
+  partition: Int?,
+  timestamp: Long?,
+  key: K,
+  value: V,
+  val correlationMetadata: T,
+  headers: Iterable
? = null +) : ProducerRecord(topic, partition, timestamp, key, value, headers) { + constructor(record: ProducerRecord, correlationMetadata: T) : + this( + record.topic(), + record.partition(), + record.timestamp(), + record.key(), + record.value(), + correlationMetadata, + record.headers() + ) +} diff --git a/src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherScope.kt b/src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherScope.kt new file mode 100644 index 00000000..1534421d --- /dev/null +++ b/src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherScope.kt @@ -0,0 +1,281 @@ +package io.github.nomisRev.kafka.publisher + +import kotlinx.coroutines.CompletableDeferred +import kotlinx.coroutines.CompletableJob +import kotlinx.coroutines.CopyableThrowable +import kotlinx.coroutines.CoroutineScope +import kotlinx.coroutines.DEBUG_PROPERTY_NAME +import kotlinx.coroutines.DEBUG_PROPERTY_VALUE_AUTO +import kotlinx.coroutines.DEBUG_PROPERTY_VALUE_OFF +import kotlinx.coroutines.DEBUG_PROPERTY_VALUE_ON +import kotlinx.coroutines.Deferred +import kotlinx.coroutines.DelicateCoroutinesApi +import kotlinx.coroutines.ExecutorCoroutineDispatcher +import kotlinx.coroutines.ExperimentalCoroutinesApi +import kotlinx.coroutines.GlobalScope +import kotlinx.coroutines.Job +import kotlinx.coroutines.NonCancellable +import kotlinx.coroutines.SupervisorJob +import kotlinx.coroutines.asCoroutineDispatcher +import kotlinx.coroutines.async +import kotlinx.coroutines.awaitAll +import kotlinx.coroutines.coroutineScope +import kotlinx.coroutines.currentCoroutineContext +import kotlinx.coroutines.runInterruptible +import kotlinx.coroutines.withContext +import org.apache.kafka.clients.producer.KafkaProducer +import org.apache.kafka.clients.producer.Producer +import org.apache.kafka.clients.producer.ProducerRecord +import org.apache.kafka.clients.producer.RecordMetadata +import org.apache.kafka.common.errors.AuthenticationException +import org.apache.kafka.common.errors.ProducerFencedException +import org.slf4j.Logger +import org.slf4j.LoggerFactory +import java.time.Duration +import java.util.UUID +import java.util.concurrent.Executors +import kotlin.coroutines.CoroutineContext +import kotlin.coroutines.cancellation.CancellationException +import kotlin.time.toJavaDuration + +interface PublisherScope : CoroutineScope { + + suspend fun offer(record: ProducerRecord) + + suspend fun publish(record: ProducerRecord): RecordMetadata + + suspend fun transaction(block: suspend PublisherScope.() -> A): A + + suspend fun offer(records: Iterable>) = + records.map { offer(it) } + + suspend fun publish(record: Iterable>): List = + coroutineScope { + record.map { async { publish(it) } }.awaitAll() + } + + suspend fun publishCatching(record: ProducerRecord): Result = + runCatching { publish(record) }.onFailure { throwFatal(it) } + + suspend fun publishCatching(record: Iterable>): Result> = + runCatching { + coroutineScope { + record.map { async { publish(it) } }.awaitAll() + } + }.onFailure { throwFatal(it) } +} + +suspend fun publish( + settings: PublisherSettings, + createProducer: suspend () -> Producer = + { KafkaProducer(settings.properties(), settings.keySerializer, settings.valueSerializer) }, + block: suspend PublisherScope.() -> A +): A { + val producerId = "reactor-kafka-sender-${UUID.randomUUID()}" + val producerContext: ExecutorCoroutineDispatcher = + Executors.newScheduledThreadPool(1) { runnable -> + Thread(runnable, producerId).apply { + isDaemon = true + } + }.asCoroutineDispatcher() + + @OptIn(DelicateCoroutinesApi::class) + val producer = GlobalScope.async(producerContext) { + createProducer().apply { + settings.producerListener.producerAdded(producerId, this) + if (settings.isTransactional()) { + DefaultKafkaPublisher.log.info("Initializing transactions for producer {}", settings.transactionalId()) + initTransactions() + } + } + } + + val scope = DefaultProduceScope( + settings, + producer, + producerContext, + currentCoroutineContext(), + Job(currentCoroutineContext()[Job]) + ) + return try { + runCatching { + withContext(scope.coroutineContext) { block(scope) } + }.onSuccess { + runCatching { scope.completeAndJoin() } + .also(::println) + .getOrThrow() + }.onFailure { e -> + val error = e.checkMyScope(scope) + scope.parent.join() + throw error + }.getOrThrow() + } finally { + withContext(NonCancellable) { + listOf( + runCatching { + producer.await().close( + if (settings.closeTimeout.isInfinite()) Duration.ofMillis(Long.MAX_VALUE) + else settings.closeTimeout.toJavaDuration() + ) + }, + runCatching { settings.producerListener.producerRemoved(producerId, producer.await()) }, + runCatching { producerContext.close() } + ).throwIfErrors() + } + } +} + +private class DefaultProduceScope( + val settings: PublisherSettings, + val producer: Deferred>, + val producerContext: ExecutorCoroutineDispatcher, + currentContext: CoroutineContext, + val parent: CompletableJob +) : PublisherScope, CoroutineScope { + + override val coroutineContext: CoroutineContext = + currentContext + parent + + override suspend fun offer(record: ProducerRecord) { + val p: Producer = producer.await() + val child = Job(parent) + runInterruptible(producerContext) { + p.send(record) { _, exception -> + if (exception != null) parent.cancel(ChildCancelScope("Child failed", exception, this)) + else child.complete() + } + } + } + + override suspend fun publish(record: ProducerRecord): RecordMetadata { + val p: Producer = producer.await() + val promise = CompletableDeferred() + runInterruptible(producerContext) { + p.send(record) { metadata, exception -> + if (exception != null) promise.completeExceptionally(exception) + else promise.complete(metadata) + } + } + return promise.await() + } + + + override suspend fun transaction(block: suspend PublisherScope.() -> A): A { + val transactionScope = DefaultProduceScope( + settings, + producer, + producerContext, + currentCoroutineContext(), + Job(currentCoroutineContext()[Job]) + ) + val producer = producer.await() + withContext(producerContext) { producer.beginTransaction() } + log.debug("Begin a new transaction for producer {}", settings.transactionalId()) + return runCatching { + withContext(transactionScope.coroutineContext) { + block(transactionScope) + } + }.mapCatching { + transactionScope.completeAndJoin() + it + }.onSuccess { + withContext(producerContext) { producer.commitTransaction() } + log.debug("Commit current transaction for producer {}", settings.transactionalId()) + }.onFailure { e -> + val error = e.checkMyScope(transactionScope) + transactionScope.parent.join() + withContext(producerContext) { producer.abortTransaction() } + log.debug("Abort current transaction for producer {}", settings.transactionalId()) + throw error + }.getOrThrow() + } + + suspend fun completeAndJoin() { + val promise = CompletableDeferred() + parent.complete() + parent.invokeOnCompletion { e -> + if (e == null) promise.complete(Unit) + else promise.completeExceptionally(e.checkMyScope(this)) + } + promise.await() + } + + companion object { + val log: Logger = LoggerFactory.getLogger(DefaultProduceScope::class.java.name) + } +} + +private fun Iterable>.throwIfErrors() { + fold, Throwable?>(null) { acc, result -> + acc?.apply { + result.exceptionOrNull()?.let { addSuppressed(it) } + } ?: result.exceptionOrNull() + }?.let { throw it } +} + +@OptIn(ExperimentalCoroutinesApi::class) +private class ChildCancelScope( + message: String, + override val cause: Throwable, + @Transient @JvmField val scope: PublisherScope<*, *>, +) : CancellationException(message), CopyableThrowable { + init { + initCause(cause) + } + + /* + * In non-debug mode we don't want to have a stacktrace on every cancellation/close, parent job reference is enough. + * Stacktrace of JCE is not needed most of the time (e.g., it is not logged) and hurts performance. + */ + override fun fillInStackTrace(): Throwable = + if (DEBUG) super.fillInStackTrace() + else apply { + stackTrace = emptyArray() // Prevent Android <= 6.0 bug, #1866 + } + + /* In non-debug mode we don't copy JCE for speed as it does not have the stack trace anyway. */ + override fun createCopy(): ChildCancelScope? = + if (DEBUG) ChildCancelScope(message!!, cause, scope) + else null +} + +private fun throwFatal(t: Throwable): Unit = + when (t) { + // Fatal JVM errors + is VirtualMachineError, + is ThreadDeath, + is InterruptedException, + is LinkageError, + // Fatal KotlinX error + is CancellationException, + // Fatal kafka errors + is AuthenticationException, + is ProducerFencedException -> throw t + + else -> Unit + } + +private fun Throwable.checkMyScope(scope: DefaultProduceScope<*, *>): Throwable = + when { + this is ChildCancelScope && this.scope === scope -> cause + this is ChildCancelScope && this.scope !== scope -> throw this + else -> { + scope.parent.cancel(CancellationException("ProducerScope failed, cancelling children", this)) + this + } + } + +private val ASSERTIONS_ENABLED = ChildCancelScope::class.java.desiredAssertionStatus() + +private val DEBUG = try { + System.getProperty(DEBUG_PROPERTY_NAME) +} catch (e: SecurityException) { + null +}.let { value -> + when (value) { + DEBUG_PROPERTY_VALUE_AUTO, null -> ASSERTIONS_ENABLED + DEBUG_PROPERTY_VALUE_ON, "" -> true + DEBUG_PROPERTY_VALUE_OFF -> false + else -> error("System property '$DEBUG_PROPERTY_NAME' has unrecognized value '$value'") + } +} diff --git a/src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherSettings.kt b/src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherSettings.kt index ee962335..35ad3575 100644 --- a/src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherSettings.kt +++ b/src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherSettings.kt @@ -1,11 +1,13 @@ package io.github.nomisRev.kafka.publisher import io.github.nomisRev.kafka.NothingSerializer +import io.github.nomisRev.kafka.publisher.PublisherSettings.ProducerListener import io.github.nomisRev.kafka.receiver.isPosNonZero -import io.github.nomisRev.kafka.publisher.PublisherOptions.ProducerListener import kotlinx.coroutines.channels.Channel import org.apache.kafka.clients.producer.Producer import org.apache.kafka.clients.producer.ProducerConfig +import org.apache.kafka.common.errors.AuthenticationException +import org.apache.kafka.common.errors.ProducerFencedException import org.apache.kafka.common.serialization.Serializer import java.util.Properties import kotlin.time.Duration @@ -20,7 +22,7 @@ import kotlin.time.Duration * @param stopOnError if a send operation should be terminated when an error is encountered. If set to false, send is attempted for all records in a sequence even if send of one of the records fails with a non-fatal exception. * @param producerListener listener that is called whenever a [Producer] is added, and removed. */ -data class PublisherOptions( +data class PublisherSettings( val bootstrapServers: String, val keySerializer: Serializer, val valueSerializer: Serializer, @@ -28,6 +30,8 @@ data class PublisherOptions( val closeTimeout: Duration = Duration.INFINITE, val maxInFlight: Int = Channel.BUFFERED, val stopOnError: Boolean = true, + val isFatal: (t: Throwable) -> Boolean = + { it is AuthenticationException || it is ProducerFencedException }, val producerListener: ProducerListener = NoOpProducerListener, val properties: Properties = Properties(), ) { @@ -44,6 +48,21 @@ data class PublisherOptions( putAll(properties) } + fun transactionalId(): String? = + properties[ProducerConfig.TRANSACTIONAL_ID_CONFIG] as? String + + /** + * Senders created from this options will be transactional if a transactional id is + * configured using {@link ProducerConfig#TRANSACTIONAL_ID_CONFIG}. If transactional, + * {@link KafkaProducer#initTransactions()} is invoked on the producer to initialize + * transactions before any operations are performed on the sender. If scheduler is overridden + * using {@link #scheduler(Scheduler)}, the configured scheduler + * must be single-threaded. Otherwise, the behaviour is undefined and may result in unexpected + * exceptions. + */ + fun isTransactional(): Boolean = + !transactionalId().isNullOrBlank() + /** Called whenever a [Producer] is added or removed. */ interface ProducerListener { /** @@ -62,7 +81,7 @@ data class PublisherOptions( } } -/** Alternative constructor for [PublisherOptions] without a key */ +/** Alternative constructor for [PublisherSettings] without a key */ public fun PublisherOptions( bootstrapServers: String, valueSerializer: Serializer, @@ -70,10 +89,12 @@ public fun PublisherOptions( closeTimeout: Duration = Duration.INFINITE, maxInFlight: Int = Channel.BUFFERED, stopOnError: Boolean = true, + isFatal: (t: Throwable) -> Boolean = + { it is AuthenticationException || it is ProducerFencedException }, producerListener: ProducerListener = NoOpProducerListener, properties: Properties = Properties(), -): PublisherOptions = - PublisherOptions( +): PublisherSettings = + PublisherSettings( bootstrapServers, NothingSerializer, valueSerializer, @@ -81,6 +102,7 @@ public fun PublisherOptions( closeTimeout, maxInFlight, stopOnError, + isFatal, producerListener, properties ) diff --git a/src/main/kotlin/io/github/nomisRev/kafka/publisher/RecordAck.kt b/src/main/kotlin/io/github/nomisRev/kafka/publisher/RecordAck.kt new file mode 100644 index 00000000..27055c38 --- /dev/null +++ b/src/main/kotlin/io/github/nomisRev/kafka/publisher/RecordAck.kt @@ -0,0 +1,48 @@ +package io.github.nomisRev.kafka.publisher + +import org.apache.kafka.clients.producer.RecordMetadata + +/** + * An acknowledgment message of a published record from Kafka. + * This message was either a [Success] or [Failed] message, + * and always contains the [correlationMetadata] from the [PublisherRecord] that was not sent to Kafka, + * but enables matching this response to its corresponding request. + * + * The result returns when the record is acknowledged based on the [Acks] configuration. + * If acks is not zero, sends are retried if [ProducerConfig.RETRIES_CONFIG] is configured. + */ +sealed interface RecordAck { + /** + * Returns the correlation metadata associated with this instance to enable this + * result to be matched with the corresponding [PublisherRecord] that was sent to Kafka. + * @return correlation metadata + */ + val correlationMetadata: A + + /** + * Returns the exception associated with a send failure. This is set to null for + * successful responses. + * @return send exception from Kafka [Producer] if send did not succeed even after + * the configured retry attempts. + */ + data class Failed(val exception: Exception, override val correlationMetadata: A) : RecordAck + + /** + * Returns the record metadata returned by Kafka. May be null if send request failed. + * See [.exception] for failure reason when record metadata is null. + * @return response metadata from Kafka [Producer] + */ + data class Success(val recordMetadata: RecordMetadata, override val correlationMetadata: A) : RecordAck + + fun exceptionOrNull(): Exception? = + when (this) { + is Failed -> exception + is Success -> null + } + + fun recordMetadataOrNull(): RecordMetadata? = + when (this) { + is Failed -> null + is Success -> recordMetadata + } +} \ No newline at end of file diff --git a/src/main/kotlin/io/github/nomisRev/kafka/publisher/SendFlow.kt b/src/main/kotlin/io/github/nomisRev/kafka/publisher/SendFlow.kt new file mode 100644 index 00000000..8b6ea35e --- /dev/null +++ b/src/main/kotlin/io/github/nomisRev/kafka/publisher/SendFlow.kt @@ -0,0 +1,3 @@ +package io.github.nomisRev.kafka.publisher + + diff --git a/src/main/kotlin/io/github/nomisRev/kafka/publisher/SendOperator.kt b/src/main/kotlin/io/github/nomisRev/kafka/publisher/SendOperator.kt new file mode 100644 index 00000000..85088b9e --- /dev/null +++ b/src/main/kotlin/io/github/nomisRev/kafka/publisher/SendOperator.kt @@ -0,0 +1,30 @@ +package io.github.nomisRev.kafka.publisher + +import kotlinx.coroutines.flow.Flow +import kotlinx.coroutines.flow.FlowCollector +import kotlinx.coroutines.flow.collect +import kotlinx.coroutines.flow.flow + +interface SendOperator : Flow { + fun send(records: Flow>): SendOperator +} + +private class DefaultSendOperator( + val wrapping: Flow>, + val publisher: DefaultKafkaPublisher +) : SendOperator { + + override suspend fun collect(collector: FlowCollector): Unit = + wrapping.collect { + collector.emit(Unit) + } + + override fun send(records: Flow>): SendOperator = + DefaultSendOperator( + flow { + wrapping.collect() + publisher.send(records).collect() + }, + publisher + ) +} diff --git a/src/test/kotlin/io/github/nomisrev/kafka/KafkaSpec.kt b/src/test/kotlin/io/github/nomisrev/kafka/KafkaSpec.kt index c1649afb..1530d7bd 100644 --- a/src/test/kotlin/io/github/nomisrev/kafka/KafkaSpec.kt +++ b/src/test/kotlin/io/github/nomisrev/kafka/KafkaSpec.kt @@ -2,18 +2,23 @@ package io.github.nomisrev.kafka import io.github.nomisRev.kafka.Admin import io.github.nomisRev.kafka.AdminSettings +import io.github.nomisRev.kafka.Flow.KafkaPublisher import io.github.nomisRev.kafka.receiver.AutoOffsetReset -import io.github.nomisRev.kafka.ProducerSettings import io.github.nomisRev.kafka.createTopic import io.github.nomisRev.kafka.deleteTopic import io.github.nomisRev.kafka.describeTopic -import io.github.nomisRev.kafka.produce +import io.github.nomisRev.kafka.publisher.Acks +import io.github.nomisRev.kafka.publisher.PublisherRecord +import io.github.nomisRev.kafka.publisher.PublisherSettings import io.github.nomisRev.kafka.receiver.KafkaReceiver import io.github.nomisRev.kafka.receiver.ReceiverSettings import io.kotest.core.spec.style.StringSpec +import kotlinx.coroutines.Dispatchers +import kotlinx.coroutines.channels.Channel import kotlinx.coroutines.flow.asFlow +import kotlinx.coroutines.flow.buffer import kotlinx.coroutines.flow.collect -import kotlinx.coroutines.flow.map +import kotlinx.coroutines.withContext import org.apache.kafka.clients.admin.Admin import org.apache.kafka.clients.admin.AdminClientConfig import org.apache.kafka.clients.admin.NewTopic @@ -32,13 +37,13 @@ abstract class KafkaSpec(body: KafkaSpec.() -> Unit = {}) : StringSpec() { init { body() } - + private val transactionTimeoutInterval = 1.seconds private val consumerPollingTimeout = 1.seconds - + private val kafkaImage: DockerImageName = DockerImageName.parse("confluentinc/cp-kafka:latest") - + private val container: KafkaContainer = autoClose( KafkaContainer(kafkaImage) .withExposedPorts(9092, 9093) @@ -55,20 +60,20 @@ abstract class KafkaSpec(body: KafkaSpec.() -> Unit = {}) : StringSpec() { .withReuse(true) .also { container -> container.start() } ) - + private fun adminProperties(): Properties = Properties().apply { put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, container.bootstrapServers) put(AdminClientConfig.CLIENT_ID_CONFIG, "test-kafka-admin-client") put(AdminClientConfig.REQUEST_TIMEOUT_MS_CONFIG, "10000") put(AdminClientConfig.CONNECTIONS_MAX_IDLE_MS_CONFIG, "10000") } - + fun adminSettings(): AdminSettings = AdminSettings(container.bootstrapServers, adminProperties()) - + inline fun admin(body: Admin.() -> A): A = Admin(adminSettings()).use(body) - + fun receiverSetting(): ReceiverSettings = ReceiverSettings( bootstrapServers = container.bootstrapServers, @@ -78,22 +83,27 @@ abstract class KafkaSpec(body: KafkaSpec.() -> Unit = {}) : StringSpec() { autoOffsetReset = AutoOffsetReset.Earliest, pollTimeout = consumerPollingTimeout ) - - fun producerSettings(): ProducerSettings = - ProducerSettings( + + fun publisherSettings( + acknowledgments: Acks = Acks.One, + properties: Properties = Properties() + ): PublisherSettings = + PublisherSettings( bootstrapServers = container.bootstrapServers, - keyDeserializer = StringSerializer(), - valueDeserializer = StringSerializer(), - other = Properties().apply { - put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, container.bootstrapServers) + keySerializer = StringSerializer(), + valueSerializer = StringSerializer(), + acknowledgments = acknowledgments, + properties = properties.apply { put(ProducerConfig.MAX_BLOCK_MS_CONFIG, 10000.toString()) put(ProducerConfig.RETRY_BACKOFF_MS_CONFIG, 1000.toString()) } ) - + + val publisher = autoClose(KafkaPublisher.create(publisherSettings())) + private fun nextTopicName(): String = "topic-${UUID.randomUUID()}" - + suspend fun withTopic( topicConfig: Map = emptyMap(), partitions: Int = 1, @@ -110,30 +120,32 @@ abstract class KafkaSpec(body: KafkaSpec.() -> Unit = {}) : StringSpec() { } } } - + @JvmName("publishPairsToKafka") suspend fun publishToKafka( topic: NewTopic, messages: Iterable>, ): Unit = - messages.asFlow() - .map { (key, value) -> - ProducerRecord(topic.name(), key, value) - }.produce(producerSettings()) - .collect() - + publisher.send( + messages.mapIndexed { index, (key, value) -> + PublisherRecord(ProducerRecord(topic.name(), key, value), index) + }.asFlow() + ).buffer(Channel.UNLIMITED).collect() + suspend fun publishToKafka(messages: Iterable>): Unit = - messages.asFlow() - .produce(producerSettings()) - .collect() - + publisher.send( + messages.mapIndexed { index, producerRecord -> + PublisherRecord(producerRecord, index) + }.asFlow() + ).buffer(Channel.UNLIMITED).collect() + suspend fun KafkaReceiver.committedCount(topic: String): Long = admin { val description = requireNotNull(describeTopic(topic)) { "Topic $topic not found" } val topicPartitions = description.partitions().map { TopicPartition(topic, it.partition()) }.toSet() - + withConsumer { committed(topicPartitions) .mapNotNull { (_, offset) -> diff --git a/src/test/kotlin/io/github/nomisrev/kafka/publisher/KafkaPublisherSpec.kt b/src/test/kotlin/io/github/nomisrev/kafka/publisher/KafkaPublisherSpec.kt new file mode 100644 index 00000000..5da5a177 --- /dev/null +++ b/src/test/kotlin/io/github/nomisrev/kafka/publisher/KafkaPublisherSpec.kt @@ -0,0 +1,449 @@ +package io.github.nomisrev.kafka.publisher + +import io.github.nomisRev.kafka.publisher.Acks +import io.github.nomisRev.kafka.publisher.PublisherScope +import io.github.nomisRev.kafka.publisher.publish +import io.github.nomisRev.kafka.receiver.KafkaReceiver +import io.github.nomisrev.kafka.KafkaSpec +import io.kotest.assertions.async.shouldTimeout +import io.kotest.assertions.fail +import io.kotest.assertions.throwables.shouldThrow +import io.kotest.matchers.result.shouldBeSuccess +import io.kotest.matchers.shouldBe +import io.kotest.matchers.types.shouldBeInstanceOf +import kotlinx.coroutines.CancellationException +import kotlinx.coroutines.CompletableDeferred +import kotlinx.coroutines.CoroutineStart +import kotlinx.coroutines.CoroutineStart.UNDISPATCHED +import kotlinx.coroutines.Dispatchers +import kotlinx.coroutines.async +import kotlinx.coroutines.awaitAll +import kotlinx.coroutines.awaitCancellation +import kotlinx.coroutines.coroutineScope +import kotlinx.coroutines.delay +import kotlinx.coroutines.flow.first +import kotlinx.coroutines.flow.map +import kotlinx.coroutines.flow.take +import kotlinx.coroutines.flow.toList +import kotlinx.coroutines.launch +import org.apache.kafka.clients.admin.NewTopic +import org.apache.kafka.clients.consumer.ConsumerGroupMetadata +import org.apache.kafka.clients.consumer.OffsetAndMetadata +import org.apache.kafka.clients.producer.Callback +import org.apache.kafka.clients.producer.Producer +import org.apache.kafka.clients.producer.ProducerConfig +import org.apache.kafka.clients.producer.ProducerRecord +import org.apache.kafka.clients.producer.RecordMetadata +import org.apache.kafka.common.Metric +import org.apache.kafka.common.MetricName +import org.apache.kafka.common.PartitionInfo +import org.apache.kafka.common.TopicPartition +import java.time.Duration +import java.util.Properties +import java.util.concurrent.CompletableFuture +import java.util.concurrent.Executors +import java.util.concurrent.Future +import java.util.concurrent.ScheduledExecutorService +import java.util.concurrent.TimeUnit +import kotlin.time.Duration.Companion.seconds + +class KafkaPublisherSpec : KafkaSpec({ + + "Can receive all messages that were offered on the right partitions" { + withTopic(partitions = 4) { topic -> + val count = 3 + val records = (0..count).map { + topic.createProducerRecord(it) + } + publish(publisherSettings()) { + offer(records) + } + + KafkaReceiver(receiverSetting()) + .receive(topic.name()) + .map { record -> + Pair(record.partition(), listOf(record.value())) + .also { record.offset.acknowledge() } + } + .take(count + 1) + .toList() + .toMap() shouldBe records.groupBy({ it.partition() }) { it.value() } + } + } + + "Can receive all messages that were published on the right partitions" { + withTopic(partitions = 4) { topic -> + val count = 3 + val records = (0..count).map { + topic.createProducerRecord(it) + } + publish(publisherSettings()) { + publish(records) + } + + KafkaReceiver(receiverSetting()) + .receive(topic.name()) + .map { record -> + Pair(record.partition(), listOf(record.value())) + .also { record.offset.acknowledge() } + } + .take(count + 1) + .toList() + .toMap() shouldBe records.groupBy({ it.partition() }) { it.value() } + } + } + + "A failure in a produce block, rethrows the error" { + withTopic(partitions = 4) { topic -> + val boom = RuntimeException("Boom!") + val record = topic.createProducerRecord(0) + + shouldThrow { + publish(publisherSettings()) { + offer(record) + throw boom + } + } shouldBe boom + + KafkaReceiver(receiverSetting()) + .receive(topic.name()) + .map { + it.apply { offset.acknowledge() } + }.first().value() shouldBe record.value() + } + } + + "A failure in a produce block with a concurrent launch cancels the launch, rethrows the error" { + withTopic(partitions = 4) { topic -> + val boom = RuntimeException("Boom!") + val cancelSignal = CompletableDeferred() + shouldThrow { + publish(publisherSettings()) { + launch(start = UNDISPATCHED) { + try { + awaitCancellation() + } catch (e: CancellationException) { + cancelSignal.complete(e) + throw e + } + } + throw boom + } + } shouldBe boom + cancelSignal.await() + } + } + + "A failed offer is rethrow at the end" { + withTopic(partitions = 4) { topic -> + val boom = RuntimeException("Boom!") + val record = topic.createProducerRecord(0) + val failingProducer = publisher.withProducer { + it.stubProducer(_sendCallback = { metadata, callback -> + if (metadata.key().equals("0")) { + Executors.newScheduledThreadPool(1).schedule( + { + callback.onCompletion(null, boom) + }, + 1, + TimeUnit.SECONDS + ) + + CompletableFuture.supplyAsync { throw AssertionError("Should never be called") } + } else send(record, callback) + }) + } + + shouldThrow { + publish(publisherSettings(), { failingProducer }) { + offer(record) + } + } shouldBe boom + } + } + + "test" { + val boom = RuntimeException("Boom!") + shouldThrow { + coroutineScope { + launch { throw boom } + } + } shouldBe boom + runCatching { delay(100) }.shouldBeSuccess() + } + + "An async failure is rethrow at the end" { + withTopic(partitions = 4) { topic -> + val count = 3 + val boom = RuntimeException("Boom!") + val records = (0..count).map { + topic.createProducerRecord(it) + } + shouldThrow { + publish(publisherSettings()) { + publish(records) + launch { throw boom } + } + } shouldBe boom + + KafkaReceiver(receiverSetting()) + .receive(topic.name()) + .map { record -> + Pair(record.partition(), listOf(record.value())) + .also { record.offset.acknowledge() } + } + .take(3) + .toList() + .toMap() shouldBe records.groupBy({ it.partition() }) { it.value() } + } + } + + "A failure of a sendAwait can be caught in the block" { + withTopic(partitions = 4) { topic -> + val record = topic.createProducerRecord(0) + val record2 = topic.createProducerRecord(1) + val failingProducer = publisher.withProducer { + it.stubProducer(_sendCallback = { metadata, callback -> + if (metadata.key().equals("0")) { + callback.onCompletion(null, RuntimeException("Boom!")) + CompletableFuture.supplyAsync { throw AssertionError("Should never be called") } + } else send(record, callback) + }) + } + + publish(publisherSettings(), { failingProducer }) { + publishCatching(record) + offer(record2) + } + + KafkaReceiver(receiverSetting()) + .receive(topic.name()) + .map { + it.apply { offset.acknowledge() } + }.first().value() shouldBe record.value() + } + } + + "concurrent publishing" { + withTopic(partitions = 4) { topic -> + val count = 4 + val records = + (1..count).map { + (it + 1..it + count).map { topic.createProducerRecord(it) } + } + + publish(publisherSettings()) { + listOf( + async { offer(records[0]) }, + async { offer(records[1]) }, + async { publish(records[2]) }, + async { publish(records[3]) } + ).awaitAll() + } + + val expected = + records.flatten().groupBy({ it.partition() }) { it.value() }.mapValues { it.value.toSet() } + + KafkaReceiver(receiverSetting()) + .receive(topic.name()) + .map { record -> + record.also { record.offset.acknowledge() } + } + .take(records.flatten().size) + .toList() + .groupBy({ it.partition() }) { it.value() } + .mapValues { it.value.toSet() } shouldBe expected + } + } + + "transaction an receive all messages that were published on the right partitions" { + withTopic(partitions = 4) { topic -> + val count = 3 + val records = (0..count).map { + topic.createProducerRecord(it) + } + val settings = publisherSettings( + acknowledgments = Acks.All, + properties = Properties().apply { + put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, testCase.name.testName) + put(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "true") + } + ) + publish(settings) { + transaction { + offer(records) + } + } + + KafkaReceiver(receiverSetting()) + .receive(topic.name()) + .map { record -> + Pair(record.partition(), listOf(record.value())) + .also { record.offset.acknowledge() } + } + .take(count + 1) + .toList() + .toMap() shouldBe records.groupBy({ it.partition() }) { it.value() } + } + } + + "A failure in a transaction aborts the transaction" { + withTopic(partitions = 4) { topic -> + val count = 3 + val boom = RuntimeException("Boom!") + val records = (0..count).map { + topic.createProducerRecord(it) + } + val settings = publisherSettings( + acknowledgments = Acks.All, + properties = Properties().apply { + put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, testCase.name.testName) + put(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "true") + } + ) + shouldThrow { + publish(settings) { + transaction { + publish(records) + throw boom + } + } + } shouldBe boom + + shouldTimeout(1.seconds) { + KafkaReceiver(receiverSetting()) + .receive(topic.name()) + .take(1) + .toList() + } + } + } + + "An async failure in a transaction aborts the transaction" { + withTopic(partitions = 4) { topic -> + val count = 3 + val boom = RuntimeException("Boom!") + val records = (0..count).map { + topic.createProducerRecord(it) + } + val settings = publisherSettings( + acknowledgments = Acks.All, + properties = Properties().apply { + put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, testCase.name.testName) + put(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "true") + } + ) + shouldThrow { + publish(settings) { + transaction { + publish(records) + launch { throw boom } + } + } + } shouldBe boom + + shouldTimeout(1.seconds) { + KafkaReceiver(receiverSetting()) + .receive(topic.name()) + .take(1) + .toList() + } + } + } + + "transaction - concurrent publishing" { + withTopic(partitions = 4) { topic -> + val count = 4 + val records = + (1..count).map { + (it + 1..it + count).map { topic.createProducerRecord(it) } + } + + val settings = publisherSettings( + acknowledgments = Acks.All, + properties = Properties().apply { + put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, testCase.name.testName) + put(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "true") + } + ) + + publish(settings) { + transaction { + listOf( + async { offer(records[0]) }, + async { offer(records[1]) }, + async { publish(records[2]) }, + async { publish(records[3]) } + ).awaitAll() + } + } + + val expected = + records.flatten().groupBy({ it.partition() }) { it.value() }.mapValues { it.value.toSet() } + + KafkaReceiver(receiverSetting()) + .receive(topic.name()) + .map { record -> + record.also { record.offset.acknowledge() } + } + .take(records.flatten().size) + .toList() + .groupBy({ it.partition() }) { it.value() } + .mapValues { it.value.toSet() } shouldBe expected + } + } +}) + +fun NewTopic.createProducerRecord(index: Int, partitions: Int = 4): ProducerRecord { + val partition: Int = index % partitions + return ProducerRecord(name(), partition, "$index", "Message $index") +} + +fun Producer.stubProducer( + _sendCallback: Producer.(record: ProducerRecord, callback: Callback) -> Future = + { record, callback -> send(record, callback) }, + _send: Producer.(record: ProducerRecord) -> Future = { send(it) } +) = object : Producer { + override fun close() {} + + override fun close(timeout: Duration?) {} + + override fun initTransactions() = + this@stubProducer.initTransactions() + + override fun beginTransaction() = + this@stubProducer.beginTransaction() + + @Suppress("DEPRECATION") + @Deprecated("Deprecated in Java") + override fun sendOffsetsToTransaction( + offsets: MutableMap?, + consumerGroupId: String? + ) = this@stubProducer.sendOffsetsToTransaction(offsets, consumerGroupId) + + override fun sendOffsetsToTransaction( + offsets: MutableMap?, + groupMetadata: ConsumerGroupMetadata? + ) = this@stubProducer.sendOffsetsToTransaction(offsets, groupMetadata) + + override fun commitTransaction() = + this@stubProducer.commitTransaction() + + override fun abortTransaction() = + this@stubProducer.abortTransaction() + + override fun flush() = + this@stubProducer.flush() + + override fun partitionsFor(topic: String?): MutableList = + this@stubProducer.partitionsFor(topic) + + override fun metrics(): MutableMap = + this@stubProducer.metrics() + + override fun send(record: ProducerRecord, callback: Callback): Future = + _sendCallback.invoke(this@stubProducer, record, callback) + + override fun send(record: ProducerRecord): Future = + _send.invoke(this@stubProducer, record) +} From 83beb87d76c775d7919347b9b081f16396d4d4e5 Mon Sep 17 00:00:00 2001 From: Simon Vergauwen Date: Mon, 9 Oct 2023 14:53:54 +0200 Subject: [PATCH 02/16] Remove old code --- .../kafka/publisher/DefaultKafkaPublisher.kt | 303 ------------------ .../kafka/publisher/KafkaPublisher.kt | 183 ----------- .../kafka/publisher/PublisherRecord.kt | 50 --- .../nomisRev/kafka/publisher/RecordAck.kt | 48 --- .../nomisRev/kafka/publisher/SendFlow.kt | 3 - .../nomisRev/kafka/publisher/SendOperator.kt | 30 -- .../io/github/nomisrev/kafka/KafkaSpec.kt | 4 - 7 files changed, 621 deletions(-) delete mode 100644 src/main/kotlin/io/github/nomisRev/kafka/publisher/DefaultKafkaPublisher.kt delete mode 100644 src/main/kotlin/io/github/nomisRev/kafka/publisher/KafkaPublisher.kt delete mode 100644 src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherRecord.kt delete mode 100644 src/main/kotlin/io/github/nomisRev/kafka/publisher/RecordAck.kt delete mode 100644 src/main/kotlin/io/github/nomisRev/kafka/publisher/SendFlow.kt delete mode 100644 src/main/kotlin/io/github/nomisRev/kafka/publisher/SendOperator.kt diff --git a/src/main/kotlin/io/github/nomisRev/kafka/publisher/DefaultKafkaPublisher.kt b/src/main/kotlin/io/github/nomisRev/kafka/publisher/DefaultKafkaPublisher.kt deleted file mode 100644 index 52868340..00000000 --- a/src/main/kotlin/io/github/nomisRev/kafka/publisher/DefaultKafkaPublisher.kt +++ /dev/null @@ -1,303 +0,0 @@ -package io.github.nomisRev.kafka.publisher - -import io.github.nomisRev.kafka.Flow.KafkaPublisher -import io.github.nomisRev.kafka.publisher.DefaultKafkaPublisher.State.ACTIVE -import io.github.nomisRev.kafka.publisher.DefaultKafkaPublisher.State.COMPLETE -import io.github.nomisRev.kafka.publisher.DefaultKafkaPublisher.State.DONE_COLLECTING -import kotlinx.coroutines.CoroutineExceptionHandler -import kotlinx.coroutines.DelicateCoroutinesApi -import kotlinx.coroutines.Dispatchers -import kotlinx.coroutines.ExecutorCoroutineDispatcher -import kotlinx.coroutines.ExperimentalCoroutinesApi -import kotlinx.coroutines.GlobalScope -import kotlinx.coroutines.asCoroutineDispatcher -import kotlinx.coroutines.async -import kotlinx.coroutines.channels.awaitClose -import kotlinx.coroutines.channels.onFailure -import kotlinx.coroutines.channels.trySendBlocking -import kotlinx.coroutines.currentCoroutineContext -import kotlinx.coroutines.flow.Flow -import kotlinx.coroutines.flow.channelFlow -import kotlinx.coroutines.flow.flowOn -import kotlinx.coroutines.invoke -import kotlinx.coroutines.newSingleThreadContext -import kotlinx.coroutines.runBlocking -import kotlinx.coroutines.runInterruptible -import org.apache.kafka.clients.producer.Callback -import org.apache.kafka.clients.producer.Producer -import org.apache.kafka.clients.producer.ProducerRecord -import org.apache.kafka.clients.producer.internals.TransactionManager -import org.slf4j.LoggerFactory -import java.time.Duration -import java.util.concurrent.Executors -import java.util.concurrent.atomic.AtomicInteger -import java.util.concurrent.atomic.AtomicReference -import kotlin.time.toJavaDuration - - -/** - * Reactive producer that sends messages to Kafka topic partitions. The producer is thread-safe - * and can be used to send messages to multiple partitions. It is recommended that a single - * producer is shared for each message type in a client. - * - * @param outgoing message key type - * @param outgoing message value type - */ -class DefaultKafkaPublisher( - _createProducer: suspend () -> Producer, - val settings: PublisherSettings, - val onPublisherRecordDropped: suspend (ProducerRecord) -> Unit = - { log.debug("onNextDropped: {}", it) } -) : KafkaPublisher { - private val producerContext: ExecutorCoroutineDispatcher = - Executors.newScheduledThreadPool(1) { runnable -> - Thread(runnable, producerId).apply { - isDaemon = true - } - }.asCoroutineDispatcher() - - @OptIn(ExperimentalCoroutinesApi::class, DelicateCoroutinesApi::class) - val transactionContext: ExecutorCoroutineDispatcher? = - settings.transactionalId()?.let { newSingleThreadContext(it) } - - private val transactionManager: TransactionManager? = null - private val producerId: String = "reactor-kafka-sender-" + System.identityHashCode(this) - - @OptIn(DelicateCoroutinesApi::class) - val producer = GlobalScope.async(producerContext) { - withTransactionContext { - _createProducer().apply { - settings.producerListener.producerAdded(producerId, this) - if (settings.isTransactional()) { - log.info("Initializing transactions for producer {}", settings.transactionalId()) - initTransactions() - } - } - } - } - - enum class State { ACTIVE, DONE_COLLECTING, COMPLETE } - - suspend fun withTransactionContext( - block: suspend () -> A - ): A = transactionContext?.invoke { block() } ?: block() - - override fun send(records: Flow>): Flow> = - doSend(records) - - private fun doSend(records: Flow>): Flow> = - channelFlow> { - val producer = producer.await() - - val inflight = AtomicInteger(0) - val delayedThrowable = AtomicReference(null) - val state: AtomicReference = AtomicReference(ACTIVE) - val ctx = currentCoroutineContext() - val handler = ctx[CoroutineExceptionHandler] - - /* - * **If** in DONE State we transition to COMPLETE. - * We close the channelFlow so that the FlowCollector can complete. - * In case of any errors, the FlowCollector will see non-null Throwable. - */ - fun doneToComplete() { - if (state.compareAndSet(DONE_COLLECTING, COMPLETE)) { - val exception = delayedThrowable.get() - close(exception) - } - } - - /* - * **If** in ACTIVE State then we transition to DONE_COLLECTING the provided records, - * and in-case all inflight records are already acknowledged then we also transition to COMPLETE. - */ - fun activeToDoneOrCompleteIfAlreadyDone() { - if (state.compareAndSet(ACTIVE, DONE_COLLECTING)) { - if (inflight.get() == 0) { - doneToComplete() - } - } - } - - /* - * Set COMPLETE State, and close the channelFlow so that the FlowCollector can complete. - * If the channelFlow was already closed, we send the error to the `CoroutineExceptionHandler`, - * which if null is set to the default logging CoroutineExceptionHandler. - */ - fun complete(t: Throwable?) { - log.trace("Sender failed with exception", t) - state.getAndSet(COMPLETE) - val alreadyClosed = !close(t) - if (alreadyClosed) { - t?.let { handler?.handleException(ctx, t) } - } - } - - records.collect { record: ProducerRecord -> - withTransactionContext { - if (state.get() == COMPLETE) return@withTransactionContext onPublisherRecordDropped(record) - - inflight.incrementAndGet() - if (settings.isTransactional()) { - log.trace( - "Transactional send initiated for producer {} in state {} inflight {}: {}", - settings.transactionalId(), - state, - inflight, - record - ) - } - - val correlationMetadata: A = - @Suppress("UNCHECKED_CAST") - (record as PublisherRecord).correlationMetadata - - val callback = Callback { metadata, exception -> - if (settings.isTransactional()) { - log.trace( - "Transactional send completed for producer {} in state {} inflight {}: {}", - settings.transactionalId(), - state, - inflight, - record - ) - } - - /* A fatal error occurred from a parallel record */ - if (state.get() === COMPLETE) { - return@Callback - } - - if (exception != null) { - log.trace("Sender failed: ", exception) - delayedThrowable.compareAndSet(null, exception) - if (settings.stopOnError || settings.isFatal(exception)) { - return@Callback complete(exception) - } else { - trySendBlocking(RecordAck.Failed(exception, correlationMetadata!!)) - // If Channel is already closed, this exception was already seen before - // - .onFailure { close(it) } - } - } else { - trySendBlocking(RecordAck.Success(metadata, correlationMetadata!!)) - // If Channel is already closed, this exception was already seen before - .onFailure { close(it) } - } - - doneToComplete() - } - - try { - runInterruptible { producer.send(record, callback) } - } catch (e: Exception) { - callback.onCompletion(null, e) - } - } - } - - activeToDoneOrCompleteIfAlreadyDone() - awaitClose { } - }.flowOn(producerContext) - - override fun sendTransitionally(records: Flow>>): Flow>> { - TODO() - // val sink: Sinks.Many = Sinks.many().unicast().onBackpressureBuffer() - // return Flow.from(transactionRecords) - // .publishOn(settings.scheduler(), false, 1) - // .concatMapDelayError({ records -> transaction(records, sink) }, false, 1) - // .window(sink.asFlow()) - // .doOnTerminate { sink.emitComplete(EmitFailureHandler.FAIL_FAST) } - // .doOnCancel { sink.emitComplete(EmitFailureHandler.FAIL_FAST) } - } - - // TODO: Protect with lifecycle - override suspend fun withProducer(function: (Producer) -> T): T = - function(producer.await()) - -// fun createOutbound(): KafkaOutbound { -// return DefaultKafkaOutbound(this) -// } - - override fun transactionManager(): TransactionManager = - requireNotNull(transactionManager) { "Transactions are not enabled" } - - override fun close(): Unit = - if (!producer.isCompleted) Unit - else { - runBlocking(Dispatchers.IO) { - withTransactionContext { - val producer = producer.await() - producer.close( - if (settings.closeTimeout.isInfinite()) Duration.ofMillis(Long.MAX_VALUE) - else settings.closeTimeout.toJavaDuration() - ) - settings.producerListener.producerRemoved(producerId, producer) - } - } - transactionContext?.close() - producerContext.close() - } - -// private fun transaction( -// transactionRecords: Publisher?>, -// transactionBoundary: Sinks.Many -// ): Flow> { -// return transactionManager() -// .begin() -// .thenMany(send(transactionRecords)) -// .concatWith(transactionManager().commit()) -// .concatWith(Mono.fromRunnable { transactionBoundary.emitNext(this, this) }) -// .onErrorResume { e -> transactionManager().abort().then(Mono.error(e)) } -// .publishOn(settings.scheduler()) -// } - -// @Synchronized -// private fun producerProxy(producer: Producer): Producer? { -// if (this.producer == null) { -// val interfaces = arrayOf>(Producer::class.java) -// val handler = -// InvocationHandler { proxy: Any?, method: Method, args: Array -> -// if (DELEGATE_METHODS.contains(method.name)) { -// try { -// return@InvocationHandler method.invoke(producer, *args) -// } catch (e: InvocationTargetException) { -// throw e.cause!! -// } -// } else { -// throw UnsupportedOperationException("Method is not supported: $method") -// } -// } -// this.producer = Proxy.newProxyInstance( -// Producer::class.java.getClassLoader(), -// interfaces, -// handler -// ) as Producer -// } -// return this.producer -// } - -// fun onEmitFailure(signalType: SignalType?, emitResult: Sinks.EmitResult?): Boolean { -// return hasProducer.get() -// } - - companion object { - val log = LoggerFactory.getLogger(DefaultKafkaPublisher::class.java.getName()) - - /** Note: Methods added to this set should also be included in javadoc for [KafkaPublisher.doOnProducer] */ - private val DELEGATE_METHODS: Set = HashSet( - mutableListOf( - "sendOffsetsToTransaction", - "partitionsFor", - "metrics", - "flush" - ) - ) - } -} - -inline fun AtomicReference.loop(action: (V) -> Unit): Nothing { - while (true) { - action(get()) - } -} diff --git a/src/main/kotlin/io/github/nomisRev/kafka/publisher/KafkaPublisher.kt b/src/main/kotlin/io/github/nomisRev/kafka/publisher/KafkaPublisher.kt deleted file mode 100644 index 0ee3d448..00000000 --- a/src/main/kotlin/io/github/nomisRev/kafka/publisher/KafkaPublisher.kt +++ /dev/null @@ -1,183 +0,0 @@ -package io.github.nomisRev.kafka.Flow - -import io.github.nomisRev.kafka.publisher.DefaultKafkaPublisher -import io.github.nomisRev.kafka.publisher.PublisherRecord -import io.github.nomisRev.kafka.publisher.PublisherSettings -import io.github.nomisRev.kafka.publisher.RecordAck -import kotlinx.coroutines.flow.Flow -import org.apache.kafka.clients.producer.KafkaProducer -import org.apache.kafka.clients.producer.Producer -import org.apache.kafka.clients.producer.ProducerConfig -import org.apache.kafka.clients.producer.internals.TransactionManager - -/** - * Reactive producer that sends outgoing records to topic partitions of a Kafka - * cluster. The producer is thread-safe and can be used to publish records to - * multiple partitions. It is recommended that a single [KafkaPublisher] is shared for each record - * type in a client application. - * - * @param outgoing record key type - * @param outgoing record value type - */ -interface KafkaPublisher : AutoCloseable { - /** - * Sends a sequence of records to Kafka and returns a [Flow] of response record metadata including - * partition and offset of each record. Responses are ordered for each partition in the absence of retries, - * but responses from different partitions may be interleaved in a different order from the requests. - * Additional correlation metadata may be passed through in the [SenderRecord] that is not sent - * to Kafka, but is included in the response [Flow] to match responses to requests. - * - * - * Results are published when the send is acknowledged based on the acknowledgement mode - * configured using the option [ProducerConfig.ACKS_CONFIG]. If acks=0, records are acknowledged - * after the requests are buffered without waiting for any server acknowledgements. In this case the - * requests are not retried and the offset returned in [SenderResult] will be -1. For other ack - * modes, requests are retried up to the configured [ProducerConfig.RETRIES_CONFIG] times. If - * the request does not succeed after these attempts, the request fails and an exception indicating - * the reason for failure is returned in [SenderResult.exception]. - * [SenderOptions.stopOnError] can be configured to stop the send sequence on first failure - * or to attempt all sends even if one or more records could not be delivered. - * - * - * - * Example usage: - *
-   * `source = Flow.range(1, count)
-   * .map(i -> SenderRecord.create(topic, partition, null, key(i), message(i), i));
-   * sender.send(source, true)
-   * .doOnNext(r -> System.out.println("Message #" + r.correlationMetadata() + " metadata=" + r.recordMetadata()));
-  ` *
-  
* - * - * @param records Outbound records along with additional correlation metadata to be included in response - * @return Flow of Kafka producer response record metadata along with the corresponding request correlation metadata. - * For records that could not be sent, the response contains an exception that indicates reason for failure. - */ - fun send(records: Flow>): Flow> - - /** - * Sends records from each inner Flow of `records` within a transaction. - * Each transaction is committed if all the records are successfully delivered to Kafka - * and aborted if any of the records in that batch could not be delivered. - * - * - * Example usage: - *
-   * `sender.sendTransactionally(outboundRecords.window(10));
-  ` *
-  
* - * - * - * @param records Outbound producer records along with correlation metadata to match results returned. - * Records from each inner Flow are sent within a new transaction. - * @return Flow of Kafka producer response record metadata along with the corresponding request correlation metadata. - * Each inner Flow contains results of records sent within a transaction. - * @throws IllegalStateException if the sender was created without setting a non-empty - * {@value org.apache.kafka.clients.producer.ProducerConfig#TRANSACTIONAL_ID_CONFIG} in [SenderOptions] - */ - fun sendTransitionally(records: Flow>>): Flow>> - - /** - * Returns the [TransactionManager] instance associated with this sender, - * which may be used for fine-grained control over transaction states. Sender - * must have been created with a non-empty transactional id by setting - * {@value org.apache.kafka.clients.producer.ProducerConfig#TRANSACTIONAL_ID_CONFIG} in [SenderOptions]. - * - * - * - * **Threading model for transactional sender:** - * - * Sends may be scheduled from multiple threads with a transactional sender similar - * to non-transactional senders. But transaction control operations and offset commits on - * [TransactionManager] must be serialized and no sends may be performed - * while one of the transaction control operations is in progress. - * - * @return [TransactionManager] associated with this sender - * @throws IllegalStateException if the sender was created without setting a non-empty - * {@value org.apache.kafka.clients.producer.ProducerConfig#TRANSACTIONAL_ID_CONFIG} in [SenderOptions] - */ - fun transactionManager(): TransactionManager? - - //private fun transaction( -// transactionRecords: Publisher?>, -// transactionBoundary: Sinks.Many -//): Flux> { -// return transactionManager() -// .begin() -// .thenMany(send(transactionRecords)) -// .concatWith(transactionManager().commit()) -// .concatWith(Mono.fromRunnable { transactionBoundary.emitNext(this, this) }) -// .onErrorResume { e -> transactionManager().abort().then(Mono.error(e)) } -// .publishOn(senderOptions.scheduler()) -//} - - -// /** -// * Creates a reactive gateway for outgoing Kafka records. Outgoing sends can be chained -// * using [KafkaOutbound.send] or [KafkaSender.sendTransactionally]. -// * Like [Flow] and [Mono], subscribing to the tail [KafkaOutbound] will -// * schedule all parent sends in the declaration order. -// * -// * -// * -// * Example usage: -// *
-//   * `kafkaSender.createOutbound()
-//   * .send(Flow1)
-//   * .send(Flow2)
-//   * .send(Flow3)
-//   * .then()
-//   * .subscribe();
-//  ` *
-//  
* -// * -// * @return chainable reactive gateway for outgoing Kafka producer records -// */ -// fun createOutbound(): KafkaOutbound? - - /** - * Invokes the specified function on the Kafka [Producer] associated with this [KafkaPublisher]. - * - * Example usage: - *
-   * `sender.doOnProducer(producer -> producer.partitionsFor(topic))
-   * .doOnSuccess(partitions -> System.out.println("Partitions " + partitions));
-   *
-   *
-   * Functions that are directly supported on the reactive [KafkaPublisher] interface (eg. send)
-   * should not be invoked from `function`. The methods supported by
-   * `doOnProducer` are:
-   *
-   *  * [Producer.sendOffsetsToTransaction]
-   *  * [Producer.partitionsFor]
-   *  * [Producer.metrics]
-   *  * [Producer.flush]
-   *
-   *
-   * @param function A function that takes Kafka Producer as parameter
-   * @return Mono that completes with the value returned by `function`
-   */
-  suspend fun  withProducer(function: (Producer) -> T): T
-
-  /**
-   * Closes this sender and the underlying Kafka producer and releases all resources allocated to it.
-   */
-  override fun close()
-
-  companion object {
-    /**
-     * Creates a Kafka sender that appends records to Kafka topic partitions.
-     *
-     * @param options Configuration options of this sender. Changes made to the options
-     * after the sender is created will not be used by the sender.
-     * @param createProducer Create a custom producer other than the default.
-     * @return new instance of Kafka sender
-     */
-    fun  create(
-      options: PublisherSettings,
-      createProducer: suspend () -> Producer =
-        { KafkaProducer(options.properties(), options.keySerializer, options.valueSerializer) }
-    ): KafkaPublisher =
-      DefaultKafkaPublisher(createProducer, options)
-  }
-}
diff --git a/src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherRecord.kt b/src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherRecord.kt
deleted file mode 100644
index 5954a2f1..00000000
--- a/src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherRecord.kt
+++ /dev/null
@@ -1,50 +0,0 @@
-package io.github.nomisRev.kafka.publisher
-
-import org.apache.kafka.clients.producer.ProducerRecord
-import org.apache.kafka.common.header.Header
-
-
-/**
- * Represents an outgoing record. Along with the record to send to Kafka,
- * additional correlation metadata may also be specified to correlate
- * [PublisherRecord] to its corresponding record.
- *
- * @param  Outgoing record key type
- * @param  Outgoing record value type
- * @param  Correlation metadata type
- */
-class PublisherRecord
-/**
- *  @param topic Topic to which record is sent
- *  @param partition The partition to which the record is sent. If null, the partitioner configured
- *  for the [KafkaSender] will be used to choose the partition.
- *  @param timestamp The timestamp of the record. If null, the current timestamp will be assigned by the producer.
- *  The timestamp will be overwritten by the broker if the topic is configured with
- *  [org.apache.kafka.common.record.TimestampType.LOG_APPEND_TIME]. The actual timestamp
- *  used will be returned in [SenderResult.recordMetadata]
- *  @param key The key to be included in the record. May be null.
- *  @param value The contents to be included in the record.
- *  @param correlationMetadata Additional correlation metadata that is not sent to Kafka, but is
- *  included in the response to match [SenderResult] to this record.
- *  @return new sender record that can be sent to Kafka using [KafkaSender.send]
- */
-private constructor(
-  topic: String,
-  partition: Int?,
-  timestamp: Long?,
-  key: K,
-  value: V,
-  val correlationMetadata: T,
-  headers: Iterable
? = null -) : ProducerRecord(topic, partition, timestamp, key, value, headers) { - constructor(record: ProducerRecord, correlationMetadata: T) : - this( - record.topic(), - record.partition(), - record.timestamp(), - record.key(), - record.value(), - correlationMetadata, - record.headers() - ) -} diff --git a/src/main/kotlin/io/github/nomisRev/kafka/publisher/RecordAck.kt b/src/main/kotlin/io/github/nomisRev/kafka/publisher/RecordAck.kt deleted file mode 100644 index 27055c38..00000000 --- a/src/main/kotlin/io/github/nomisRev/kafka/publisher/RecordAck.kt +++ /dev/null @@ -1,48 +0,0 @@ -package io.github.nomisRev.kafka.publisher - -import org.apache.kafka.clients.producer.RecordMetadata - -/** - * An acknowledgment message of a published record from Kafka. - * This message was either a [Success] or [Failed] message, - * and always contains the [correlationMetadata] from the [PublisherRecord] that was not sent to Kafka, - * but enables matching this response to its corresponding request. - * - * The result returns when the record is acknowledged based on the [Acks] configuration. - * If acks is not zero, sends are retried if [ProducerConfig.RETRIES_CONFIG] is configured. - */ -sealed interface RecordAck { - /** - * Returns the correlation metadata associated with this instance to enable this - * result to be matched with the corresponding [PublisherRecord] that was sent to Kafka. - * @return correlation metadata - */ - val correlationMetadata: A - - /** - * Returns the exception associated with a send failure. This is set to null for - * successful responses. - * @return send exception from Kafka [Producer] if send did not succeed even after - * the configured retry attempts. - */ - data class Failed(val exception: Exception, override val correlationMetadata: A) : RecordAck - - /** - * Returns the record metadata returned by Kafka. May be null if send request failed. - * See [.exception] for failure reason when record metadata is null. - * @return response metadata from Kafka [Producer] - */ - data class Success(val recordMetadata: RecordMetadata, override val correlationMetadata: A) : RecordAck - - fun exceptionOrNull(): Exception? = - when (this) { - is Failed -> exception - is Success -> null - } - - fun recordMetadataOrNull(): RecordMetadata? = - when (this) { - is Failed -> null - is Success -> recordMetadata - } -} \ No newline at end of file diff --git a/src/main/kotlin/io/github/nomisRev/kafka/publisher/SendFlow.kt b/src/main/kotlin/io/github/nomisRev/kafka/publisher/SendFlow.kt deleted file mode 100644 index 8b6ea35e..00000000 --- a/src/main/kotlin/io/github/nomisRev/kafka/publisher/SendFlow.kt +++ /dev/null @@ -1,3 +0,0 @@ -package io.github.nomisRev.kafka.publisher - - diff --git a/src/main/kotlin/io/github/nomisRev/kafka/publisher/SendOperator.kt b/src/main/kotlin/io/github/nomisRev/kafka/publisher/SendOperator.kt deleted file mode 100644 index 85088b9e..00000000 --- a/src/main/kotlin/io/github/nomisRev/kafka/publisher/SendOperator.kt +++ /dev/null @@ -1,30 +0,0 @@ -package io.github.nomisRev.kafka.publisher - -import kotlinx.coroutines.flow.Flow -import kotlinx.coroutines.flow.FlowCollector -import kotlinx.coroutines.flow.collect -import kotlinx.coroutines.flow.flow - -interface SendOperator : Flow { - fun send(records: Flow>): SendOperator -} - -private class DefaultSendOperator( - val wrapping: Flow>, - val publisher: DefaultKafkaPublisher -) : SendOperator { - - override suspend fun collect(collector: FlowCollector): Unit = - wrapping.collect { - collector.emit(Unit) - } - - override fun send(records: Flow>): SendOperator = - DefaultSendOperator( - flow { - wrapping.collect() - publisher.send(records).collect() - }, - publisher - ) -} diff --git a/src/test/kotlin/io/github/nomisrev/kafka/KafkaSpec.kt b/src/test/kotlin/io/github/nomisrev/kafka/KafkaSpec.kt index 1530d7bd..ebe8f3d1 100644 --- a/src/test/kotlin/io/github/nomisrev/kafka/KafkaSpec.kt +++ b/src/test/kotlin/io/github/nomisrev/kafka/KafkaSpec.kt @@ -2,23 +2,19 @@ package io.github.nomisrev.kafka import io.github.nomisRev.kafka.Admin import io.github.nomisRev.kafka.AdminSettings -import io.github.nomisRev.kafka.Flow.KafkaPublisher import io.github.nomisRev.kafka.receiver.AutoOffsetReset import io.github.nomisRev.kafka.createTopic import io.github.nomisRev.kafka.deleteTopic import io.github.nomisRev.kafka.describeTopic import io.github.nomisRev.kafka.publisher.Acks -import io.github.nomisRev.kafka.publisher.PublisherRecord import io.github.nomisRev.kafka.publisher.PublisherSettings import io.github.nomisRev.kafka.receiver.KafkaReceiver import io.github.nomisRev.kafka.receiver.ReceiverSettings import io.kotest.core.spec.style.StringSpec -import kotlinx.coroutines.Dispatchers import kotlinx.coroutines.channels.Channel import kotlinx.coroutines.flow.asFlow import kotlinx.coroutines.flow.buffer import kotlinx.coroutines.flow.collect -import kotlinx.coroutines.withContext import org.apache.kafka.clients.admin.Admin import org.apache.kafka.clients.admin.AdminClientConfig import org.apache.kafka.clients.admin.NewTopic From 05718f6e963eb61e6ead33d70edf45d3bbfc5192 Mon Sep 17 00:00:00 2001 From: Simon Vergauwen Date: Mon, 9 Oct 2023 15:42:54 +0200 Subject: [PATCH 03/16] Fix operators --- .../kafka/publisher/PublisherScope.kt | 129 ++++++++---------- .../io/github/nomisrev/kafka/KafkaSpec.kt | 26 ++-- .../kafka/publisher/KafkaPublisherSpec.kt | 65 +++------ 3 files changed, 92 insertions(+), 128 deletions(-) diff --git a/src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherScope.kt b/src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherScope.kt index 1534421d..a7385f16 100644 --- a/src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherScope.kt +++ b/src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherScope.kt @@ -1,7 +1,7 @@ package io.github.nomisRev.kafka.publisher +import io.github.nomisRev.kafka.publisher.DefaultProduceScope.Companion.log import kotlinx.coroutines.CompletableDeferred -import kotlinx.coroutines.CompletableJob import kotlinx.coroutines.CopyableThrowable import kotlinx.coroutines.CoroutineScope import kotlinx.coroutines.DEBUG_PROPERTY_NAME @@ -15,12 +15,10 @@ import kotlinx.coroutines.ExperimentalCoroutinesApi import kotlinx.coroutines.GlobalScope import kotlinx.coroutines.Job import kotlinx.coroutines.NonCancellable -import kotlinx.coroutines.SupervisorJob import kotlinx.coroutines.asCoroutineDispatcher import kotlinx.coroutines.async import kotlinx.coroutines.awaitAll import kotlinx.coroutines.coroutineScope -import kotlinx.coroutines.currentCoroutineContext import kotlinx.coroutines.runInterruptible import kotlinx.coroutines.withContext import org.apache.kafka.clients.producer.KafkaProducer @@ -34,7 +32,6 @@ import org.slf4j.LoggerFactory import java.time.Duration import java.util.UUID import java.util.concurrent.Executors -import kotlin.coroutines.CoroutineContext import kotlin.coroutines.cancellation.CancellationException import kotlin.time.toJavaDuration @@ -84,31 +81,25 @@ suspend fun publish( createProducer().apply { settings.producerListener.producerAdded(producerId, this) if (settings.isTransactional()) { - DefaultKafkaPublisher.log.info("Initializing transactions for producer {}", settings.transactionalId()) + log.info("Initializing transactions for producer {}", settings.transactionalId()) initTransactions() } } } - - val scope = DefaultProduceScope( - settings, - producer, - producerContext, - currentCoroutineContext(), - Job(currentCoroutineContext()[Job]) - ) + val token = Token() return try { - runCatching { - withContext(scope.coroutineContext) { block(scope) } - }.onSuccess { - runCatching { scope.completeAndJoin() } - .also(::println) - .getOrThrow() - }.onFailure { e -> - val error = e.checkMyScope(scope) - scope.parent.join() - throw error - }.getOrThrow() + coroutineScope { + val scope = DefaultProduceScope( + settings, + producer, + producerContext, + token, + this + ) + block(scope) + } + } catch (e: ChildCancelScope) { + e.checkMyScope(token) } finally { withContext(NonCancellable) { listOf( @@ -125,23 +116,23 @@ suspend fun publish( } } +private class Token + private class DefaultProduceScope( val settings: PublisherSettings, val producer: Deferred>, val producerContext: ExecutorCoroutineDispatcher, - currentContext: CoroutineContext, - val parent: CompletableJob -) : PublisherScope, CoroutineScope { - - override val coroutineContext: CoroutineContext = - currentContext + parent + val token: Token, + scope: CoroutineScope +) : PublisherScope, CoroutineScope by scope { + val parent: Job = requireNotNull(coroutineContext[Job]) { "Impossible, can only be called within coroutineScope" } override suspend fun offer(record: ProducerRecord) { val p: Producer = producer.await() val child = Job(parent) runInterruptible(producerContext) { p.send(record) { _, exception -> - if (exception != null) parent.cancel(ChildCancelScope("Child failed", exception, this)) + if (exception != null) parent.cancel(ChildCancelScope("Child failed", exception, token)) else child.complete() } } @@ -161,42 +152,39 @@ private class DefaultProduceScope( override suspend fun transaction(block: suspend PublisherScope.() -> A): A { - val transactionScope = DefaultProduceScope( - settings, - producer, - producerContext, - currentCoroutineContext(), - Job(currentCoroutineContext()[Job]) - ) - val producer = producer.await() - withContext(producerContext) { producer.beginTransaction() } + val token = Token() + val p = producer.await() + withContext(producerContext) { p.beginTransaction() } log.debug("Begin a new transaction for producer {}", settings.transactionalId()) - return runCatching { - withContext(transactionScope.coroutineContext) { - block(transactionScope) + return try { + coroutineScope { + val scope = DefaultProduceScope( + settings, + producer, + producerContext, + token, + this + ) + block(scope) + }.also { + withContext(producerContext) { p.commitTransaction() } + log.debug("Commit current transaction for producer {}", settings.transactionalId()) } - }.mapCatching { - transactionScope.completeAndJoin() - it - }.onSuccess { - withContext(producerContext) { producer.commitTransaction() } - log.debug("Commit current transaction for producer {}", settings.transactionalId()) - }.onFailure { e -> - val error = e.checkMyScope(transactionScope) - transactionScope.parent.join() - withContext(producerContext) { producer.abortTransaction() } + } catch (e: Throwable) { + withContext(producerContext) { p.abortTransaction() } log.debug("Abort current transaction for producer {}", settings.transactionalId()) - throw error - }.getOrThrow() + if (e is ChildCancelScope) e.checkMyScope(token) + else throw e + } } suspend fun completeAndJoin() { val promise = CompletableDeferred() - parent.complete() - parent.invokeOnCompletion { e -> - if (e == null) promise.complete(Unit) - else promise.completeExceptionally(e.checkMyScope(this)) - } +// parent.complete() +// parent.invokeOnCompletion { e -> +// if (e == null) promise.complete(Unit) +// else promise.completeExceptionally(e.checkMyScope(this)) +// } promise.await() } @@ -217,7 +205,7 @@ private fun Iterable>.throwIfErrors() { private class ChildCancelScope( message: String, override val cause: Throwable, - @Transient @JvmField val scope: PublisherScope<*, *>, + @Transient @JvmField val token: Token, ) : CancellationException(message), CopyableThrowable { init { initCause(cause) @@ -235,8 +223,14 @@ private class ChildCancelScope( /* In non-debug mode we don't copy JCE for speed as it does not have the stack trace anyway. */ override fun createCopy(): ChildCancelScope? = - if (DEBUG) ChildCancelScope(message!!, cause, scope) + if (DEBUG) ChildCancelScope(message!!, cause, token) else null + + fun checkMyScope(scope: Token): Nothing = + when { + this.token === scope -> throw cause + else -> throw this + } } private fun throwFatal(t: Throwable): Unit = @@ -255,14 +249,11 @@ private fun throwFatal(t: Throwable): Unit = else -> Unit } -private fun Throwable.checkMyScope(scope: DefaultProduceScope<*, *>): Throwable = +private fun ChildCancelScope.checkMyScope(scope: Token): Throwable = when { - this is ChildCancelScope && this.scope === scope -> cause - this is ChildCancelScope && this.scope !== scope -> throw this - else -> { - scope.parent.cancel(CancellationException("ProducerScope failed, cancelling children", this)) - this - } + this is ChildCancelScope && this.token === scope -> cause + this is ChildCancelScope && this.token !== scope -> throw this + else -> this } private val ASSERTIONS_ENABLED = ChildCancelScope::class.java.desiredAssertionStatus() diff --git a/src/test/kotlin/io/github/nomisrev/kafka/KafkaSpec.kt b/src/test/kotlin/io/github/nomisrev/kafka/KafkaSpec.kt index ebe8f3d1..52f14fd5 100644 --- a/src/test/kotlin/io/github/nomisrev/kafka/KafkaSpec.kt +++ b/src/test/kotlin/io/github/nomisrev/kafka/KafkaSpec.kt @@ -8,16 +8,14 @@ import io.github.nomisRev.kafka.deleteTopic import io.github.nomisRev.kafka.describeTopic import io.github.nomisRev.kafka.publisher.Acks import io.github.nomisRev.kafka.publisher.PublisherSettings +import io.github.nomisRev.kafka.publisher.publish import io.github.nomisRev.kafka.receiver.KafkaReceiver import io.github.nomisRev.kafka.receiver.ReceiverSettings import io.kotest.core.spec.style.StringSpec -import kotlinx.coroutines.channels.Channel -import kotlinx.coroutines.flow.asFlow -import kotlinx.coroutines.flow.buffer -import kotlinx.coroutines.flow.collect import org.apache.kafka.clients.admin.Admin import org.apache.kafka.clients.admin.AdminClientConfig import org.apache.kafka.clients.admin.NewTopic +import org.apache.kafka.clients.producer.KafkaProducer import org.apache.kafka.clients.producer.ProducerConfig import org.apache.kafka.clients.producer.ProducerRecord import org.apache.kafka.common.TopicPartition @@ -95,7 +93,7 @@ abstract class KafkaSpec(body: KafkaSpec.() -> Unit = {}) : StringSpec() { } ) - val publisher = autoClose(KafkaPublisher.create(publisherSettings())) + val producer = autoClose(KafkaProducer(publisherSettings().properties())) private fun nextTopicName(): String = "topic-${UUID.randomUUID()}" @@ -122,18 +120,16 @@ abstract class KafkaSpec(body: KafkaSpec.() -> Unit = {}) : StringSpec() { topic: NewTopic, messages: Iterable>, ): Unit = - publisher.send( - messages.mapIndexed { index, (key, value) -> - PublisherRecord(ProducerRecord(topic.name(), key, value), index) - }.asFlow() - ).buffer(Channel.UNLIMITED).collect() + publish(publisherSettings()) { + offer(messages.map { (key, value) -> + ProducerRecord(topic.name(), key, value) + }) + } suspend fun publishToKafka(messages: Iterable>): Unit = - publisher.send( - messages.mapIndexed { index, producerRecord -> - PublisherRecord(producerRecord, index) - }.asFlow() - ).buffer(Channel.UNLIMITED).collect() + publish(publisherSettings()) { + offer(messages) + } suspend fun KafkaReceiver.committedCount(topic: String): Long = admin { diff --git a/src/test/kotlin/io/github/nomisrev/kafka/publisher/KafkaPublisherSpec.kt b/src/test/kotlin/io/github/nomisrev/kafka/publisher/KafkaPublisherSpec.kt index 5da5a177..a87fd4a4 100644 --- a/src/test/kotlin/io/github/nomisrev/kafka/publisher/KafkaPublisherSpec.kt +++ b/src/test/kotlin/io/github/nomisrev/kafka/publisher/KafkaPublisherSpec.kt @@ -1,26 +1,18 @@ package io.github.nomisrev.kafka.publisher import io.github.nomisRev.kafka.publisher.Acks -import io.github.nomisRev.kafka.publisher.PublisherScope import io.github.nomisRev.kafka.publisher.publish import io.github.nomisRev.kafka.receiver.KafkaReceiver import io.github.nomisrev.kafka.KafkaSpec import io.kotest.assertions.async.shouldTimeout -import io.kotest.assertions.fail import io.kotest.assertions.throwables.shouldThrow -import io.kotest.matchers.result.shouldBeSuccess import io.kotest.matchers.shouldBe -import io.kotest.matchers.types.shouldBeInstanceOf import kotlinx.coroutines.CancellationException import kotlinx.coroutines.CompletableDeferred -import kotlinx.coroutines.CoroutineStart import kotlinx.coroutines.CoroutineStart.UNDISPATCHED -import kotlinx.coroutines.Dispatchers import kotlinx.coroutines.async import kotlinx.coroutines.awaitAll import kotlinx.coroutines.awaitCancellation -import kotlinx.coroutines.coroutineScope -import kotlinx.coroutines.delay import kotlinx.coroutines.flow.first import kotlinx.coroutines.flow.map import kotlinx.coroutines.flow.take @@ -43,7 +35,6 @@ import java.util.Properties import java.util.concurrent.CompletableFuture import java.util.concurrent.Executors import java.util.concurrent.Future -import java.util.concurrent.ScheduledExecutorService import java.util.concurrent.TimeUnit import kotlin.time.Duration.Companion.seconds @@ -138,21 +129,19 @@ class KafkaPublisherSpec : KafkaSpec({ withTopic(partitions = 4) { topic -> val boom = RuntimeException("Boom!") val record = topic.createProducerRecord(0) - val failingProducer = publisher.withProducer { - it.stubProducer(_sendCallback = { metadata, callback -> - if (metadata.key().equals("0")) { - Executors.newScheduledThreadPool(1).schedule( - { - callback.onCompletion(null, boom) - }, - 1, - TimeUnit.SECONDS - ) - - CompletableFuture.supplyAsync { throw AssertionError("Should never be called") } - } else send(record, callback) - }) - } + val failingProducer = producer.stubProducer(_sendCallback = { metadata, callback -> + if (metadata.key().equals("0")) { + Executors.newScheduledThreadPool(1).schedule( + { + callback.onCompletion(null, boom) + }, + 1, + TimeUnit.SECONDS + ) + + CompletableFuture.supplyAsync { throw AssertionError("Should never be called") } + } else send(record, callback) + }) shouldThrow { publish(publisherSettings(), { failingProducer }) { @@ -162,16 +151,6 @@ class KafkaPublisherSpec : KafkaSpec({ } } - "test" { - val boom = RuntimeException("Boom!") - shouldThrow { - coroutineScope { - launch { throw boom } - } - } shouldBe boom - runCatching { delay(100) }.shouldBeSuccess() - } - "An async failure is rethrow at the end" { withTopic(partitions = 4) { topic -> val count = 3 @@ -192,7 +171,7 @@ class KafkaPublisherSpec : KafkaSpec({ Pair(record.partition(), listOf(record.value())) .also { record.offset.acknowledge() } } - .take(3) + .take(3 + 1) .toList() .toMap() shouldBe records.groupBy({ it.partition() }) { it.value() } } @@ -202,14 +181,12 @@ class KafkaPublisherSpec : KafkaSpec({ withTopic(partitions = 4) { topic -> val record = topic.createProducerRecord(0) val record2 = topic.createProducerRecord(1) - val failingProducer = publisher.withProducer { - it.stubProducer(_sendCallback = { metadata, callback -> - if (metadata.key().equals("0")) { - callback.onCompletion(null, RuntimeException("Boom!")) - CompletableFuture.supplyAsync { throw AssertionError("Should never be called") } - } else send(record, callback) - }) - } + val failingProducer = producer.stubProducer(_sendCallback = { metadata, callback -> + if (metadata.key().equals("0")) { + callback.onCompletion(null, RuntimeException("Boom!")) + CompletableFuture.supplyAsync { throw AssertionError("Should never be called") } + } else send(record, callback) + }) publish(publisherSettings(), { failingProducer }) { publishCatching(record) @@ -336,7 +313,7 @@ class KafkaPublisherSpec : KafkaSpec({ shouldThrow { publish(settings) { transaction { - publish(records) + offer(records) launch { throw boom } } } From 6fe3aded74ef7a465b5d1312a9287569b4f83b3c Mon Sep 17 00:00:00 2001 From: Simon Vergauwen Date: Mon, 9 Oct 2023 15:48:28 +0200 Subject: [PATCH 04/16] Clean-up code a bit --- .../kafka/publisher/PublisherScope.kt | 20 +------------------ 1 file changed, 1 insertion(+), 19 deletions(-) diff --git a/src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherScope.kt b/src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherScope.kt index a7385f16..4e47b160 100644 --- a/src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherScope.kt +++ b/src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherScope.kt @@ -173,21 +173,10 @@ private class DefaultProduceScope( } catch (e: Throwable) { withContext(producerContext) { p.abortTransaction() } log.debug("Abort current transaction for producer {}", settings.transactionalId()) - if (e is ChildCancelScope) e.checkMyScope(token) - else throw e + if (e is ChildCancelScope) e.checkMyScope(token) else throw e } } - suspend fun completeAndJoin() { - val promise = CompletableDeferred() -// parent.complete() -// parent.invokeOnCompletion { e -> -// if (e == null) promise.complete(Unit) -// else promise.completeExceptionally(e.checkMyScope(this)) -// } - promise.await() - } - companion object { val log: Logger = LoggerFactory.getLogger(DefaultProduceScope::class.java.name) } @@ -249,13 +238,6 @@ private fun throwFatal(t: Throwable): Unit = else -> Unit } -private fun ChildCancelScope.checkMyScope(scope: Token): Throwable = - when { - this is ChildCancelScope && this.token === scope -> cause - this is ChildCancelScope && this.token !== scope -> throw this - else -> this - } - private val ASSERTIONS_ENABLED = ChildCancelScope::class.java.desiredAssertionStatus() private val DEBUG = try { From 0f015d22126f33634947ad4fd12e744e1b2c4197 Mon Sep 17 00:00:00 2001 From: Simon Vergauwen Date: Mon, 9 Oct 2023 15:48:28 +0200 Subject: [PATCH 05/16] Clean-up code a bit --- .../nomisRev/kafka/publisher/Publisher.kt | 4 + .../kafka/publisher/PublisherScope.kt | 238 ++---------------- .../io/github/nomisrev/kafka/KafkaSpec.kt | 6 +- .../kafka/publisher/KafkaPublisherSpec.kt | 26 +- 4 files changed, 37 insertions(+), 237 deletions(-) create mode 100644 src/main/kotlin/io/github/nomisRev/kafka/publisher/Publisher.kt diff --git a/src/main/kotlin/io/github/nomisRev/kafka/publisher/Publisher.kt b/src/main/kotlin/io/github/nomisRev/kafka/publisher/Publisher.kt new file mode 100644 index 00000000..a8f01eab --- /dev/null +++ b/src/main/kotlin/io/github/nomisRev/kafka/publisher/Publisher.kt @@ -0,0 +1,4 @@ +package io.github.nomisRev.kafka.publisher + +class Publisher { +} \ No newline at end of file diff --git a/src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherScope.kt b/src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherScope.kt index a7385f16..f4020519 100644 --- a/src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherScope.kt +++ b/src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherScope.kt @@ -1,50 +1,34 @@ +@file:JvmMultifileClass +@file:JvmName("PublisherScope.kt") + package io.github.nomisRev.kafka.publisher -import io.github.nomisRev.kafka.publisher.DefaultProduceScope.Companion.log -import kotlinx.coroutines.CompletableDeferred -import kotlinx.coroutines.CopyableThrowable import kotlinx.coroutines.CoroutineScope -import kotlinx.coroutines.DEBUG_PROPERTY_NAME -import kotlinx.coroutines.DEBUG_PROPERTY_VALUE_AUTO -import kotlinx.coroutines.DEBUG_PROPERTY_VALUE_OFF -import kotlinx.coroutines.DEBUG_PROPERTY_VALUE_ON -import kotlinx.coroutines.Deferred -import kotlinx.coroutines.DelicateCoroutinesApi -import kotlinx.coroutines.ExecutorCoroutineDispatcher -import kotlinx.coroutines.ExperimentalCoroutinesApi -import kotlinx.coroutines.GlobalScope -import kotlinx.coroutines.Job -import kotlinx.coroutines.NonCancellable -import kotlinx.coroutines.asCoroutineDispatcher import kotlinx.coroutines.async import kotlinx.coroutines.awaitAll import kotlinx.coroutines.coroutineScope -import kotlinx.coroutines.runInterruptible -import kotlinx.coroutines.withContext -import org.apache.kafka.clients.producer.KafkaProducer -import org.apache.kafka.clients.producer.Producer import org.apache.kafka.clients.producer.ProducerRecord import org.apache.kafka.clients.producer.RecordMetadata import org.apache.kafka.common.errors.AuthenticationException import org.apache.kafka.common.errors.ProducerFencedException -import org.slf4j.Logger -import org.slf4j.LoggerFactory -import java.time.Duration -import java.util.UUID -import java.util.concurrent.Executors import kotlin.coroutines.cancellation.CancellationException -import kotlin.time.toJavaDuration -interface PublisherScope : CoroutineScope { +@DslMarker +annotation class PublisherDSL +@PublisherDSL +interface OfferScope : CoroutineScope { suspend fun offer(record: ProducerRecord) + suspend fun offer(records: Iterable>) = + records.map { offer(it) } +} - suspend fun publish(record: ProducerRecord): RecordMetadata +@PublisherDSL +interface PublishScope : OfferScope, CoroutineScope { - suspend fun transaction(block: suspend PublisherScope.() -> A): A + override suspend fun offer(record: ProducerRecord) - suspend fun offer(records: Iterable>) = - records.map { offer(it) } + suspend fun publish(record: ProducerRecord): RecordMetadata suspend fun publish(record: Iterable>): List = coroutineScope { @@ -62,175 +46,9 @@ interface PublisherScope : CoroutineScope { }.onFailure { throwFatal(it) } } -suspend fun publish( - settings: PublisherSettings, - createProducer: suspend () -> Producer = - { KafkaProducer(settings.properties(), settings.keySerializer, settings.valueSerializer) }, - block: suspend PublisherScope.() -> A -): A { - val producerId = "reactor-kafka-sender-${UUID.randomUUID()}" - val producerContext: ExecutorCoroutineDispatcher = - Executors.newScheduledThreadPool(1) { runnable -> - Thread(runnable, producerId).apply { - isDaemon = true - } - }.asCoroutineDispatcher() - - @OptIn(DelicateCoroutinesApi::class) - val producer = GlobalScope.async(producerContext) { - createProducer().apply { - settings.producerListener.producerAdded(producerId, this) - if (settings.isTransactional()) { - log.info("Initializing transactions for producer {}", settings.transactionalId()) - initTransactions() - } - } - } - val token = Token() - return try { - coroutineScope { - val scope = DefaultProduceScope( - settings, - producer, - producerContext, - token, - this - ) - block(scope) - } - } catch (e: ChildCancelScope) { - e.checkMyScope(token) - } finally { - withContext(NonCancellable) { - listOf( - runCatching { - producer.await().close( - if (settings.closeTimeout.isInfinite()) Duration.ofMillis(Long.MAX_VALUE) - else settings.closeTimeout.toJavaDuration() - ) - }, - runCatching { settings.producerListener.producerRemoved(producerId, producer.await()) }, - runCatching { producerContext.close() } - ).throwIfErrors() - } - } -} - -private class Token - -private class DefaultProduceScope( - val settings: PublisherSettings, - val producer: Deferred>, - val producerContext: ExecutorCoroutineDispatcher, - val token: Token, - scope: CoroutineScope -) : PublisherScope, CoroutineScope by scope { - val parent: Job = requireNotNull(coroutineContext[Job]) { "Impossible, can only be called within coroutineScope" } - - override suspend fun offer(record: ProducerRecord) { - val p: Producer = producer.await() - val child = Job(parent) - runInterruptible(producerContext) { - p.send(record) { _, exception -> - if (exception != null) parent.cancel(ChildCancelScope("Child failed", exception, token)) - else child.complete() - } - } - } - - override suspend fun publish(record: ProducerRecord): RecordMetadata { - val p: Producer = producer.await() - val promise = CompletableDeferred() - runInterruptible(producerContext) { - p.send(record) { metadata, exception -> - if (exception != null) promise.completeExceptionally(exception) - else promise.complete(metadata) - } - } - return promise.await() - } - - - override suspend fun transaction(block: suspend PublisherScope.() -> A): A { - val token = Token() - val p = producer.await() - withContext(producerContext) { p.beginTransaction() } - log.debug("Begin a new transaction for producer {}", settings.transactionalId()) - return try { - coroutineScope { - val scope = DefaultProduceScope( - settings, - producer, - producerContext, - token, - this - ) - block(scope) - }.also { - withContext(producerContext) { p.commitTransaction() } - log.debug("Commit current transaction for producer {}", settings.transactionalId()) - } - } catch (e: Throwable) { - withContext(producerContext) { p.abortTransaction() } - log.debug("Abort current transaction for producer {}", settings.transactionalId()) - if (e is ChildCancelScope) e.checkMyScope(token) - else throw e - } - } - - suspend fun completeAndJoin() { - val promise = CompletableDeferred() -// parent.complete() -// parent.invokeOnCompletion { e -> -// if (e == null) promise.complete(Unit) -// else promise.completeExceptionally(e.checkMyScope(this)) -// } - promise.await() - } - - companion object { - val log: Logger = LoggerFactory.getLogger(DefaultProduceScope::class.java.name) - } -} - -private fun Iterable>.throwIfErrors() { - fold, Throwable?>(null) { acc, result -> - acc?.apply { - result.exceptionOrNull()?.let { addSuppressed(it) } - } ?: result.exceptionOrNull() - }?.let { throw it } -} - -@OptIn(ExperimentalCoroutinesApi::class) -private class ChildCancelScope( - message: String, - override val cause: Throwable, - @Transient @JvmField val token: Token, -) : CancellationException(message), CopyableThrowable { - init { - initCause(cause) - } - - /* - * In non-debug mode we don't want to have a stacktrace on every cancellation/close, parent job reference is enough. - * Stacktrace of JCE is not needed most of the time (e.g., it is not logged) and hurts performance. - */ - override fun fillInStackTrace(): Throwable = - if (DEBUG) super.fillInStackTrace() - else apply { - stackTrace = emptyArray() // Prevent Android <= 6.0 bug, #1866 - } - - /* In non-debug mode we don't copy JCE for speed as it does not have the stack trace anyway. */ - override fun createCopy(): ChildCancelScope? = - if (DEBUG) ChildCancelScope(message!!, cause, token) - else null - - fun checkMyScope(scope: Token): Nothing = - when { - this.token === scope -> throw cause - else -> throw this - } +@PublisherDSL +interface TransactionalScope : PublishScope { + suspend fun transaction(block: suspend PublishScope.() -> A): A } private fun throwFatal(t: Throwable): Unit = @@ -248,25 +66,3 @@ private fun throwFatal(t: Throwable): Unit = else -> Unit } - -private fun ChildCancelScope.checkMyScope(scope: Token): Throwable = - when { - this is ChildCancelScope && this.token === scope -> cause - this is ChildCancelScope && this.token !== scope -> throw this - else -> this - } - -private val ASSERTIONS_ENABLED = ChildCancelScope::class.java.desiredAssertionStatus() - -private val DEBUG = try { - System.getProperty(DEBUG_PROPERTY_NAME) -} catch (e: SecurityException) { - null -}.let { value -> - when (value) { - DEBUG_PROPERTY_VALUE_AUTO, null -> ASSERTIONS_ENABLED - DEBUG_PROPERTY_VALUE_ON, "" -> true - DEBUG_PROPERTY_VALUE_OFF -> false - else -> error("System property '$DEBUG_PROPERTY_NAME' has unrecognized value '$value'") - } -} diff --git a/src/test/kotlin/io/github/nomisrev/kafka/KafkaSpec.kt b/src/test/kotlin/io/github/nomisrev/kafka/KafkaSpec.kt index 52f14fd5..2fb1afe6 100644 --- a/src/test/kotlin/io/github/nomisrev/kafka/KafkaSpec.kt +++ b/src/test/kotlin/io/github/nomisrev/kafka/KafkaSpec.kt @@ -8,7 +8,7 @@ import io.github.nomisRev.kafka.deleteTopic import io.github.nomisRev.kafka.describeTopic import io.github.nomisRev.kafka.publisher.Acks import io.github.nomisRev.kafka.publisher.PublisherSettings -import io.github.nomisRev.kafka.publisher.publish +import io.github.nomisRev.kafka.publisher.publishScope import io.github.nomisRev.kafka.receiver.KafkaReceiver import io.github.nomisRev.kafka.receiver.ReceiverSettings import io.kotest.core.spec.style.StringSpec @@ -120,14 +120,14 @@ abstract class KafkaSpec(body: KafkaSpec.() -> Unit = {}) : StringSpec() { topic: NewTopic, messages: Iterable>, ): Unit = - publish(publisherSettings()) { + publishScope(publisherSettings()) { offer(messages.map { (key, value) -> ProducerRecord(topic.name(), key, value) }) } suspend fun publishToKafka(messages: Iterable>): Unit = - publish(publisherSettings()) { + publishScope(publisherSettings()) { offer(messages) } diff --git a/src/test/kotlin/io/github/nomisrev/kafka/publisher/KafkaPublisherSpec.kt b/src/test/kotlin/io/github/nomisrev/kafka/publisher/KafkaPublisherSpec.kt index a87fd4a4..4f3c7951 100644 --- a/src/test/kotlin/io/github/nomisrev/kafka/publisher/KafkaPublisherSpec.kt +++ b/src/test/kotlin/io/github/nomisrev/kafka/publisher/KafkaPublisherSpec.kt @@ -1,7 +1,7 @@ package io.github.nomisrev.kafka.publisher import io.github.nomisRev.kafka.publisher.Acks -import io.github.nomisRev.kafka.publisher.publish +import io.github.nomisRev.kafka.publisher.publishScope import io.github.nomisRev.kafka.receiver.KafkaReceiver import io.github.nomisrev.kafka.KafkaSpec import io.kotest.assertions.async.shouldTimeout @@ -46,7 +46,7 @@ class KafkaPublisherSpec : KafkaSpec({ val records = (0..count).map { topic.createProducerRecord(it) } - publish(publisherSettings()) { + publishScope(publisherSettings()) { offer(records) } @@ -68,7 +68,7 @@ class KafkaPublisherSpec : KafkaSpec({ val records = (0..count).map { topic.createProducerRecord(it) } - publish(publisherSettings()) { + publishScope(publisherSettings()) { publish(records) } @@ -90,7 +90,7 @@ class KafkaPublisherSpec : KafkaSpec({ val record = topic.createProducerRecord(0) shouldThrow { - publish(publisherSettings()) { + publishScope(publisherSettings()) { offer(record) throw boom } @@ -109,7 +109,7 @@ class KafkaPublisherSpec : KafkaSpec({ val boom = RuntimeException("Boom!") val cancelSignal = CompletableDeferred() shouldThrow { - publish(publisherSettings()) { + publishScope(publisherSettings()) { launch(start = UNDISPATCHED) { try { awaitCancellation() @@ -144,7 +144,7 @@ class KafkaPublisherSpec : KafkaSpec({ }) shouldThrow { - publish(publisherSettings(), { failingProducer }) { + publishScope(publisherSettings(), { failingProducer }) { offer(record) } } shouldBe boom @@ -159,7 +159,7 @@ class KafkaPublisherSpec : KafkaSpec({ topic.createProducerRecord(it) } shouldThrow { - publish(publisherSettings()) { + publishScope(publisherSettings()) { publish(records) launch { throw boom } } @@ -188,7 +188,7 @@ class KafkaPublisherSpec : KafkaSpec({ } else send(record, callback) }) - publish(publisherSettings(), { failingProducer }) { + publishScope(publisherSettings(), { failingProducer }) { publishCatching(record) offer(record2) } @@ -209,7 +209,7 @@ class KafkaPublisherSpec : KafkaSpec({ (it + 1..it + count).map { topic.createProducerRecord(it) } } - publish(publisherSettings()) { + publishScope(publisherSettings()) { listOf( async { offer(records[0]) }, async { offer(records[1]) }, @@ -246,7 +246,7 @@ class KafkaPublisherSpec : KafkaSpec({ put(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "true") } ) - publish(settings) { + publishScope(settings) { transaction { offer(records) } @@ -279,7 +279,7 @@ class KafkaPublisherSpec : KafkaSpec({ } ) shouldThrow { - publish(settings) { + publishScope(settings) { transaction { publish(records) throw boom @@ -311,7 +311,7 @@ class KafkaPublisherSpec : KafkaSpec({ } ) shouldThrow { - publish(settings) { + publishScope(settings) { transaction { offer(records) launch { throw boom } @@ -344,7 +344,7 @@ class KafkaPublisherSpec : KafkaSpec({ } ) - publish(settings) { + publishScope(settings) { transaction { listOf( async { offer(records[0]) }, From a6bad33fd8176d1dcff956d6acf17a55ff817556 Mon Sep 17 00:00:00 2001 From: Simon Vergauwen Date: Mon, 9 Oct 2023 19:58:32 +0200 Subject: [PATCH 06/16] Rely on PublisherSetting.isFatal for checking Kafka's fatal exceptions --- .../nomisRev/kafka/publisher/Publisher.kt | 272 +++++++++++++++++- .../kafka/publisher/PublisherScope.kt | 43 +-- .../kafka/publisher/PublisherSettings.kt | 11 +- .../io/github/nomisrev/kafka/KafkaSpec.kt | 9 +- .../kafka/publisher/KafkaPublisherSpec.kt | 123 ++++++-- 5 files changed, 376 insertions(+), 82 deletions(-) diff --git a/src/main/kotlin/io/github/nomisRev/kafka/publisher/Publisher.kt b/src/main/kotlin/io/github/nomisRev/kafka/publisher/Publisher.kt index a8f01eab..b33bacbe 100644 --- a/src/main/kotlin/io/github/nomisRev/kafka/publisher/Publisher.kt +++ b/src/main/kotlin/io/github/nomisRev/kafka/publisher/Publisher.kt @@ -1,4 +1,272 @@ package io.github.nomisRev.kafka.publisher -class Publisher { -} \ No newline at end of file +import io.github.nomisRev.kafka.publisher.DefaultKafkaPublisher.Companion.log +import kotlinx.coroutines.CompletableDeferred +import kotlinx.coroutines.CopyableThrowable +import kotlinx.coroutines.CoroutineScope +import kotlinx.coroutines.DEBUG_PROPERTY_NAME +import kotlinx.coroutines.DEBUG_PROPERTY_VALUE_AUTO +import kotlinx.coroutines.DEBUG_PROPERTY_VALUE_OFF +import kotlinx.coroutines.DEBUG_PROPERTY_VALUE_ON +import kotlinx.coroutines.Deferred +import kotlinx.coroutines.DelicateCoroutinesApi +import kotlinx.coroutines.ExecutorCoroutineDispatcher +import kotlinx.coroutines.ExperimentalCoroutinesApi +import kotlinx.coroutines.GlobalScope +import kotlinx.coroutines.Job +import kotlinx.coroutines.asCoroutineDispatcher +import kotlinx.coroutines.async +import kotlinx.coroutines.awaitAll +import kotlinx.coroutines.coroutineScope +import kotlinx.coroutines.runBlocking +import kotlinx.coroutines.runInterruptible +import kotlinx.coroutines.withContext +import org.apache.kafka.clients.producer.KafkaProducer +import org.apache.kafka.clients.producer.Producer +import org.apache.kafka.clients.producer.ProducerRecord +import org.apache.kafka.clients.producer.RecordMetadata +import org.apache.kafka.common.Metric +import org.apache.kafka.common.MetricName +import org.apache.kafka.common.PartitionInfo +import org.slf4j.Logger +import org.slf4j.LoggerFactory +import java.time.Duration +import java.util.concurrent.Executors +import kotlin.coroutines.cancellation.CancellationException +import kotlin.time.toJavaDuration + +fun KafkaPublisher( + settings: PublisherSettings, + createProducer: suspend () -> Producer = + { KafkaProducer(settings.properties(), settings.keySerializer, settings.valueSerializer) } +): Publisher = + DefaultKafkaPublisher(settings, createProducer) + +interface Publisher : AutoCloseable { + suspend fun publishScope(block: suspend TransactionalScope.() -> A): A + + /** + * @see KafkaProducer.partitionsFor + */ + suspend fun partitionsFor(topic: String): List + + /** + * @see KafkaProducer.metrics + */ + suspend fun metrics(): Map + +} + +private class DefaultKafkaPublisher( + val settings: PublisherSettings, + createProducer: suspend () -> Producer +) : Publisher { + + val producerId = "reactor-kafka-sender-${System.identityHashCode(this)}" + val producerContext: ExecutorCoroutineDispatcher = + Executors.newScheduledThreadPool(1) { runnable -> + Thread(runnable, producerId).apply { + isDaemon = true + } + }.asCoroutineDispatcher() + + @OptIn(DelicateCoroutinesApi::class) + val producer = GlobalScope.async(producerContext) { + createProducer().apply { + settings.producerListener.producerAdded(producerId, this) + if (settings.isTransactional()) { + log.info("Initializing transactions for producer {}", settings.transactionalId()) + initTransactions() + } + } + } + + override suspend fun partitionsFor(topic: String): List { + val p = producer.await() + return runInterruptible(producerContext) { + p.partitionsFor(topic) + } + } + + + override suspend fun metrics(): Map { + val p = producer.await() + return runInterruptible(producerContext) { + p.metrics() + } + } + + override suspend fun publishScope(block: suspend TransactionalScope.() -> A): A { + val token = Token() + return try { + coroutineScope { + val scope = DefaultProduceScope( + settings, + producer, + producerContext, + token, + this + ) + block(scope) + } + } catch (e: ChildCancelScope) { + e.checkMyScope(token) + } + } + + override fun close() = runBlocking { + listOf( + runCatching { + producer.await().close( + if (settings.closeTimeout.isInfinite()) Duration.ofMillis(Long.MAX_VALUE) + else settings.closeTimeout.toJavaDuration() + ) + }, + runCatching { settings.producerListener.producerRemoved(producerId, producer.await()) }, + runCatching { producerContext.close() } + ).throwIfErrors() + } + + companion object { + val log: Logger = LoggerFactory.getLogger(Publisher::class.java.name) + } +} + +private fun Iterable>.throwIfErrors() { + fold, Throwable?>(null) { acc, result -> + acc?.apply { + result.exceptionOrNull()?.let { addSuppressed(it) } + } ?: result.exceptionOrNull() + }?.let { throw it } +} + +private class Token + +private class DefaultProduceScope( + val settings: PublisherSettings, + val producer: Deferred>, + val producerContext: ExecutorCoroutineDispatcher, + val token: Token, + scope: CoroutineScope +) : TransactionalScope, CoroutineScope by scope { + val parent: Job = requireNotNull(coroutineContext[Job]) { "Impossible, can only be called within coroutineScope" } + + override suspend fun offer(record: ProducerRecord) { + val p: Producer = producer.await() + val child = Job(parent) + runInterruptible(producerContext) { + p.send(record) { _, exception -> + if (exception != null) parent.cancel(ChildCancelScope("Child failed", exception, token)) + else child.complete() + } + } + } + + override suspend fun publish(record: ProducerRecord): RecordMetadata { + val p: Producer = producer.await() + val promise = CompletableDeferred() + runInterruptible(producerContext) { + p.send(record) { metadata, exception -> + if (exception != null) promise.completeExceptionally(exception) + else promise.complete(metadata) + } + } + return promise.await() + } + + override suspend fun publishCatching(record: ProducerRecord): Result = + runCatching { publish(record) }.onFailure { settings.throwFatal(it) } + + override suspend fun publishCatching(record: Iterable>): Result> = + runCatching { + coroutineScope { + record.map { async { publish(it) } }.awaitAll() + } + }.onFailure { settings.throwFatal(it) } + + override suspend fun transaction(block: suspend PublishScope.() -> A): A { + val token = Token() + val p = producer.await() + withContext(producerContext) { p.beginTransaction() } + log.debug("Begin a new transaction for producer {}", settings.transactionalId()) + return try { + coroutineScope { + val scope = DefaultProduceScope( + settings, + producer, + producerContext, + token, + this + ) + block(scope) + }.also { + withContext(producerContext) { p.commitTransaction() } + log.debug("Commit current transaction for producer {}", settings.transactionalId()) + } + } catch (e: Throwable) { + withContext(producerContext) { p.abortTransaction() } + log.debug("Abort current transaction for producer {}", settings.transactionalId()) + if (e is ChildCancelScope) e.checkMyScope(token) else throw e + } + } + + private fun PublisherSettings.throwFatal(t: Throwable): Unit = + if ( + // Fatal JVM errors + t is VirtualMachineError || + t is ThreadDeath || + t is InterruptedException || + t is LinkageError || + // Fatal KotlinX error + t is CancellationException || + // Fatal kafka errors + isFatal(t) + ) throw t else Unit +} + +@OptIn(ExperimentalCoroutinesApi::class) +private class ChildCancelScope( + message: String, + override val cause: Throwable, + @Transient @JvmField val token: Token, +) : CancellationException(message), CopyableThrowable { + init { + initCause(cause) + } + + /* + * In non-debug mode we don't want to have a stacktrace on every cancellation/close, parent job reference is enough. + * Stacktrace of JCE is not needed most of the time (e.g., it is not logged) and hurts performance. + */ + override fun fillInStackTrace(): Throwable = + if (DEBUG) super.fillInStackTrace() + else apply { + stackTrace = emptyArray() // Prevent Android <= 6.0 bug, #1866 + } + + /* In non-debug mode we don't copy JCE for speed as it does not have the stack trace anyway. */ + override fun createCopy(): ChildCancelScope? = + if (DEBUG) ChildCancelScope(message!!, cause, token) + else null + + fun checkMyScope(scope: Token): Nothing = + when { + this.token === scope -> throw cause + else -> throw this + } +} + +private val ASSERTIONS_ENABLED = ChildCancelScope::class.java.desiredAssertionStatus() + +private val DEBUG = try { + System.getProperty(DEBUG_PROPERTY_NAME) +} catch (e: SecurityException) { + null +}.let { value -> + when (value) { + DEBUG_PROPERTY_VALUE_AUTO, null -> ASSERTIONS_ENABLED + DEBUG_PROPERTY_VALUE_ON, "" -> true + DEBUG_PROPERTY_VALUE_OFF -> false + else -> error("System property '$DEBUG_PROPERTY_NAME' has unrecognized value '$value'") + } +} diff --git a/src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherScope.kt b/src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherScope.kt index f4020519..b7ee39e8 100644 --- a/src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherScope.kt +++ b/src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherScope.kt @@ -9,60 +9,31 @@ import kotlinx.coroutines.awaitAll import kotlinx.coroutines.coroutineScope import org.apache.kafka.clients.producer.ProducerRecord import org.apache.kafka.clients.producer.RecordMetadata -import org.apache.kafka.common.errors.AuthenticationException -import org.apache.kafka.common.errors.ProducerFencedException -import kotlin.coroutines.cancellation.CancellationException @DslMarker annotation class PublisherDSL @PublisherDSL -interface OfferScope : CoroutineScope { +interface PublishScope : CoroutineScope { suspend fun offer(record: ProducerRecord) - suspend fun offer(records: Iterable>) = - records.map { offer(it) } -} -@PublisherDSL -interface PublishScope : OfferScope, CoroutineScope { + suspend fun publish(record: ProducerRecord): RecordMetadata - override suspend fun offer(record: ProducerRecord) + suspend fun publishCatching(record: ProducerRecord): Result - suspend fun publish(record: ProducerRecord): RecordMetadata + suspend fun publishCatching(record: Iterable>): Result> + + suspend fun offer(records: Iterable>) = + records.map { offer(it) } suspend fun publish(record: Iterable>): List = coroutineScope { record.map { async { publish(it) } }.awaitAll() } - suspend fun publishCatching(record: ProducerRecord): Result = - runCatching { publish(record) }.onFailure { throwFatal(it) } - - suspend fun publishCatching(record: Iterable>): Result> = - runCatching { - coroutineScope { - record.map { async { publish(it) } }.awaitAll() - } - }.onFailure { throwFatal(it) } } @PublisherDSL interface TransactionalScope : PublishScope { suspend fun transaction(block: suspend PublishScope.() -> A): A } - -private fun throwFatal(t: Throwable): Unit = - when (t) { - // Fatal JVM errors - is VirtualMachineError, - is ThreadDeath, - is InterruptedException, - is LinkageError, - // Fatal KotlinX error - is CancellationException, - // Fatal kafka errors - is AuthenticationException, - is ProducerFencedException -> throw t - - else -> Unit - } diff --git a/src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherSettings.kt b/src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherSettings.kt index 35ad3575..9aaa60de 100644 --- a/src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherSettings.kt +++ b/src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherSettings.kt @@ -3,7 +3,6 @@ package io.github.nomisRev.kafka.publisher import io.github.nomisRev.kafka.NothingSerializer import io.github.nomisRev.kafka.publisher.PublisherSettings.ProducerListener import io.github.nomisRev.kafka.receiver.isPosNonZero -import kotlinx.coroutines.channels.Channel import org.apache.kafka.clients.producer.Producer import org.apache.kafka.clients.producer.ProducerConfig import org.apache.kafka.common.errors.AuthenticationException @@ -13,13 +12,13 @@ import java.util.Properties import kotlin.time.Duration /** + * Typed setting to create a [Publisher], enforces the required parameters and leaves the rest as [Properties]. + * * @param bootstrapServers list of host/port pairs to use for establishing the initial connection to the Kafka cluster. Should be comma separated. * @param keySerializer the [Serializer] to use to serialize the [Key] when sending messages to kafka. * @param valueSerializer the [Serializer] to use to serialize the [Value] when sending messages to kafka. * @param acknowledgments configuration to use. * @param closeTimeout the timeout when closing the created underlying [Producer], default [Duration.INFINITE]. - * @param maxInFlight the maximum number of in-flight records that are fetched from the outbound record publisher while acknowledgements are pending. Default [Channel.BUFFERED]. - * @param stopOnError if a send operation should be terminated when an error is encountered. If set to false, send is attempted for all records in a sequence even if send of one of the records fails with a non-fatal exception. * @param producerListener listener that is called whenever a [Producer] is added, and removed. */ data class PublisherSettings( @@ -28,8 +27,6 @@ data class PublisherSettings( val valueSerializer: Serializer, val acknowledgments: Acks = Acks.One, val closeTimeout: Duration = Duration.INFINITE, - val maxInFlight: Int = Channel.BUFFERED, - val stopOnError: Boolean = true, val isFatal: (t: Throwable) -> Boolean = { it is AuthenticationException || it is ProducerFencedException }, val producerListener: ProducerListener = NoOpProducerListener, @@ -87,8 +84,6 @@ public fun PublisherOptions( valueSerializer: Serializer, acknowledgments: Acks = Acks.One, closeTimeout: Duration = Duration.INFINITE, - maxInFlight: Int = Channel.BUFFERED, - stopOnError: Boolean = true, isFatal: (t: Throwable) -> Boolean = { it is AuthenticationException || it is ProducerFencedException }, producerListener: ProducerListener = NoOpProducerListener, @@ -100,8 +95,6 @@ public fun PublisherOptions( valueSerializer, acknowledgments, closeTimeout, - maxInFlight, - stopOnError, isFatal, producerListener, properties diff --git a/src/test/kotlin/io/github/nomisrev/kafka/KafkaSpec.kt b/src/test/kotlin/io/github/nomisrev/kafka/KafkaSpec.kt index 2fb1afe6..053b6955 100644 --- a/src/test/kotlin/io/github/nomisrev/kafka/KafkaSpec.kt +++ b/src/test/kotlin/io/github/nomisrev/kafka/KafkaSpec.kt @@ -7,8 +7,8 @@ import io.github.nomisRev.kafka.createTopic import io.github.nomisRev.kafka.deleteTopic import io.github.nomisRev.kafka.describeTopic import io.github.nomisRev.kafka.publisher.Acks +import io.github.nomisRev.kafka.publisher.KafkaPublisher import io.github.nomisRev.kafka.publisher.PublisherSettings -import io.github.nomisRev.kafka.publisher.publishScope import io.github.nomisRev.kafka.receiver.KafkaReceiver import io.github.nomisRev.kafka.receiver.ReceiverSettings import io.kotest.core.spec.style.StringSpec @@ -93,7 +93,8 @@ abstract class KafkaSpec(body: KafkaSpec.() -> Unit = {}) : StringSpec() { } ) - val producer = autoClose(KafkaProducer(publisherSettings().properties())) + val producer = KafkaProducer(publisherSettings().properties()) + val publisher = autoClose(KafkaPublisher(publisherSettings()) { producer }) private fun nextTopicName(): String = "topic-${UUID.randomUUID()}" @@ -120,14 +121,14 @@ abstract class KafkaSpec(body: KafkaSpec.() -> Unit = {}) : StringSpec() { topic: NewTopic, messages: Iterable>, ): Unit = - publishScope(publisherSettings()) { + publisher.publishScope { offer(messages.map { (key, value) -> ProducerRecord(topic.name(), key, value) }) } suspend fun publishToKafka(messages: Iterable>): Unit = - publishScope(publisherSettings()) { + publisher.publishScope { offer(messages) } diff --git a/src/test/kotlin/io/github/nomisrev/kafka/publisher/KafkaPublisherSpec.kt b/src/test/kotlin/io/github/nomisrev/kafka/publisher/KafkaPublisherSpec.kt index 4f3c7951..e248cc4a 100644 --- a/src/test/kotlin/io/github/nomisrev/kafka/publisher/KafkaPublisherSpec.kt +++ b/src/test/kotlin/io/github/nomisrev/kafka/publisher/KafkaPublisherSpec.kt @@ -1,7 +1,7 @@ package io.github.nomisrev.kafka.publisher import io.github.nomisRev.kafka.publisher.Acks -import io.github.nomisRev.kafka.publisher.publishScope +import io.github.nomisRev.kafka.publisher.KafkaPublisher import io.github.nomisRev.kafka.receiver.KafkaReceiver import io.github.nomisrev.kafka.KafkaSpec import io.kotest.assertions.async.shouldTimeout @@ -30,6 +30,8 @@ import org.apache.kafka.common.Metric import org.apache.kafka.common.MetricName import org.apache.kafka.common.PartitionInfo import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.errors.FencedInstanceIdException +import org.apache.kafka.common.errors.ProducerFencedException import java.time.Duration import java.util.Properties import java.util.concurrent.CompletableFuture @@ -46,7 +48,7 @@ class KafkaPublisherSpec : KafkaSpec({ val records = (0..count).map { topic.createProducerRecord(it) } - publishScope(publisherSettings()) { + publisher.publishScope { offer(records) } @@ -68,7 +70,7 @@ class KafkaPublisherSpec : KafkaSpec({ val records = (0..count).map { topic.createProducerRecord(it) } - publishScope(publisherSettings()) { + publisher.publishScope { publish(records) } @@ -90,7 +92,7 @@ class KafkaPublisherSpec : KafkaSpec({ val record = topic.createProducerRecord(0) shouldThrow { - publishScope(publisherSettings()) { + publisher.publishScope { offer(record) throw boom } @@ -109,7 +111,7 @@ class KafkaPublisherSpec : KafkaSpec({ val boom = RuntimeException("Boom!") val cancelSignal = CompletableDeferred() shouldThrow { - publishScope(publisherSettings()) { + publisher.publishScope { launch(start = UNDISPATCHED) { try { awaitCancellation() @@ -144,7 +146,7 @@ class KafkaPublisherSpec : KafkaSpec({ }) shouldThrow { - publishScope(publisherSettings(), { failingProducer }) { + KafkaPublisher(publisherSettings()) { failingProducer }.publishScope { offer(record) } } shouldBe boom @@ -159,7 +161,7 @@ class KafkaPublisherSpec : KafkaSpec({ topic.createProducerRecord(it) } shouldThrow { - publishScope(publisherSettings()) { + publisher.publishScope { publish(records) launch { throw boom } } @@ -188,9 +190,11 @@ class KafkaPublisherSpec : KafkaSpec({ } else send(record, callback) }) - publishScope(publisherSettings(), { failingProducer }) { - publishCatching(record) - offer(record2) + KafkaPublisher(publisherSettings()) { failingProducer }.use { + it.publishScope { + publishCatching(record) + offer(record2) + } } KafkaReceiver(receiverSetting()) @@ -209,7 +213,7 @@ class KafkaPublisherSpec : KafkaSpec({ (it + 1..it + count).map { topic.createProducerRecord(it) } } - publishScope(publisherSettings()) { + publisher.publishScope { listOf( async { offer(records[0]) }, async { offer(records[1]) }, @@ -246,9 +250,11 @@ class KafkaPublisherSpec : KafkaSpec({ put(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "true") } ) - publishScope(settings) { - transaction { - offer(records) + KafkaPublisher(settings).use { + it.publishScope { + transaction { + offer(records) + } } } @@ -279,10 +285,12 @@ class KafkaPublisherSpec : KafkaSpec({ } ) shouldThrow { - publishScope(settings) { - transaction { - publish(records) - throw boom + KafkaPublisher(settings).use { + it.publishScope { + transaction { + publish(records) + throw boom + } } } } shouldBe boom @@ -311,10 +319,12 @@ class KafkaPublisherSpec : KafkaSpec({ } ) shouldThrow { - publishScope(settings) { - transaction { - offer(records) - launch { throw boom } + KafkaPublisher(settings).use { + it.publishScope { + transaction { + offer(records) + launch { throw boom } + } } } } shouldBe boom @@ -344,14 +354,16 @@ class KafkaPublisherSpec : KafkaSpec({ } ) - publishScope(settings) { - transaction { - listOf( - async { offer(records[0]) }, - async { offer(records[1]) }, - async { publish(records[2]) }, - async { publish(records[3]) } - ).awaitAll() + KafkaPublisher(settings).use { + it.publishScope { + transaction { + listOf( + async { offer(records[0]) }, + async { offer(records[1]) }, + async { publish(records[2]) }, + async { publish(records[3]) } + ).awaitAll() + } } } @@ -369,6 +381,55 @@ class KafkaPublisherSpec : KafkaSpec({ .mapValues { it.value.toSet() } shouldBe expected } } + + "Only one transactional sender with a specific transactional id is allowed at the same time" { + withTopic(partitions = 4) { topic -> + val settings = publisherSettings( + acknowledgments = Acks.All, + properties = Properties().apply { + put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, testCase.name.testName) + put(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "true") + } + ) + val records1 = (0..4).map { topic.createProducerRecord(it) } + val publisher1 = KafkaPublisher(settings) + publisher1.publishScope { + transaction { + publish(records1) + } + } + + val records2 = (5..9).map { topic.createProducerRecord(it) } + val publisher2 = KafkaPublisher(settings) + publisher2.publishScope { + transaction { + publish(records2) + } + } + + val records3 = (10..14).map { topic.createProducerRecord(it) } + shouldThrow { + publisher1.publishScope { + transaction { + // Test that ProducerFencedException is fatal + publishCatching(records3) + } + } + } + + val expected = + (records1 + records2).groupBy({ it.partition() }) { it.value() } + + KafkaReceiver(receiverSetting()) + .receive(topic.name()) + .map { record -> + record.also { record.offset.acknowledge() } + } + .take((records1 + records2).size) + .toList() + .groupBy({ it.partition() }) { it.value() } shouldBe expected + } + } }) fun NewTopic.createProducerRecord(index: Int, partitions: Int = 4): ProducerRecord { @@ -423,4 +484,4 @@ fun Producer.stubProducer( override fun send(record: ProducerRecord): Future = _send.invoke(this@stubProducer, record) -} +} \ No newline at end of file From 1ac1587180f923dc8d5d0966e6b5a6ee6368713c Mon Sep 17 00:00:00 2001 From: Simon Vergauwen Date: Mon, 9 Oct 2023 20:02:48 +0200 Subject: [PATCH 07/16] Clean up KafkaPublisherSpec --- .../io/github/nomisrev/kafka/KafkaSpec.kt | 59 +++++++++++++++++++ .../kafka/publisher/KafkaPublisherSpec.kt | 53 +---------------- 2 files changed, 61 insertions(+), 51 deletions(-) diff --git a/src/test/kotlin/io/github/nomisrev/kafka/KafkaSpec.kt b/src/test/kotlin/io/github/nomisrev/kafka/KafkaSpec.kt index 053b6955..ca5f75ce 100644 --- a/src/test/kotlin/io/github/nomisrev/kafka/KafkaSpec.kt +++ b/src/test/kotlin/io/github/nomisrev/kafka/KafkaSpec.kt @@ -15,16 +15,26 @@ import io.kotest.core.spec.style.StringSpec import org.apache.kafka.clients.admin.Admin import org.apache.kafka.clients.admin.AdminClientConfig import org.apache.kafka.clients.admin.NewTopic +import org.apache.kafka.clients.consumer.ConsumerGroupMetadata +import org.apache.kafka.clients.consumer.OffsetAndMetadata +import org.apache.kafka.clients.producer.Callback import org.apache.kafka.clients.producer.KafkaProducer +import org.apache.kafka.clients.producer.Producer import org.apache.kafka.clients.producer.ProducerConfig import org.apache.kafka.clients.producer.ProducerRecord +import org.apache.kafka.clients.producer.RecordMetadata +import org.apache.kafka.common.Metric +import org.apache.kafka.common.MetricName +import org.apache.kafka.common.PartitionInfo import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.serialization.StringDeserializer import org.apache.kafka.common.serialization.StringSerializer import org.testcontainers.containers.KafkaContainer import org.testcontainers.utility.DockerImageName +import java.time.Duration import java.util.Properties import java.util.UUID +import java.util.concurrent.Future import kotlin.time.Duration.Companion.seconds abstract class KafkaSpec(body: KafkaSpec.() -> Unit = {}) : StringSpec() { @@ -146,4 +156,53 @@ abstract class KafkaSpec(body: KafkaSpec.() -> Unit = {}) : StringSpec() { }.sum() } } + + fun stubProducer( + _sendCallback: Producer.(record: ProducerRecord, callback: Callback) -> Future = + { record, callback -> send(record, callback) }, + _send: Producer.(record: ProducerRecord) -> Future = { send(it) } + ) = object : Producer { + override fun close() {} + + override fun close(timeout: Duration?) {} + + override fun initTransactions() = + producer.initTransactions() + + override fun beginTransaction() = + producer.beginTransaction() + + @Suppress("DEPRECATION") + @Deprecated("Deprecated in Java") + override fun sendOffsetsToTransaction( + offsets: MutableMap?, + consumerGroupId: String? + ) = producer.sendOffsetsToTransaction(offsets, consumerGroupId) + + override fun sendOffsetsToTransaction( + offsets: MutableMap?, + groupMetadata: ConsumerGroupMetadata? + ) = producer.sendOffsetsToTransaction(offsets, groupMetadata) + + override fun commitTransaction() = + producer.commitTransaction() + + override fun abortTransaction() = + producer.abortTransaction() + + override fun flush() = + producer.flush() + + override fun partitionsFor(topic: String?): MutableList = + producer.partitionsFor(topic) + + override fun metrics(): MutableMap = + producer.metrics() + + override fun send(record: ProducerRecord, callback: Callback): Future = + _sendCallback.invoke(producer, record, callback) + + override fun send(record: ProducerRecord): Future = + _send.invoke(producer, record) + } } diff --git a/src/test/kotlin/io/github/nomisrev/kafka/publisher/KafkaPublisherSpec.kt b/src/test/kotlin/io/github/nomisrev/kafka/publisher/KafkaPublisherSpec.kt index e248cc4a..d446fbab 100644 --- a/src/test/kotlin/io/github/nomisrev/kafka/publisher/KafkaPublisherSpec.kt +++ b/src/test/kotlin/io/github/nomisrev/kafka/publisher/KafkaPublisherSpec.kt @@ -131,7 +131,7 @@ class KafkaPublisherSpec : KafkaSpec({ withTopic(partitions = 4) { topic -> val boom = RuntimeException("Boom!") val record = topic.createProducerRecord(0) - val failingProducer = producer.stubProducer(_sendCallback = { metadata, callback -> + val failingProducer = stubProducer(_sendCallback = { metadata, callback -> if (metadata.key().equals("0")) { Executors.newScheduledThreadPool(1).schedule( { @@ -183,7 +183,7 @@ class KafkaPublisherSpec : KafkaSpec({ withTopic(partitions = 4) { topic -> val record = topic.createProducerRecord(0) val record2 = topic.createProducerRecord(1) - val failingProducer = producer.stubProducer(_sendCallback = { metadata, callback -> + val failingProducer = stubProducer(_sendCallback = { metadata, callback -> if (metadata.key().equals("0")) { callback.onCompletion(null, RuntimeException("Boom!")) CompletableFuture.supplyAsync { throw AssertionError("Should never be called") } @@ -436,52 +436,3 @@ fun NewTopic.createProducerRecord(index: Int, partitions: Int = 4): ProducerReco val partition: Int = index % partitions return ProducerRecord(name(), partition, "$index", "Message $index") } - -fun Producer.stubProducer( - _sendCallback: Producer.(record: ProducerRecord, callback: Callback) -> Future = - { record, callback -> send(record, callback) }, - _send: Producer.(record: ProducerRecord) -> Future = { send(it) } -) = object : Producer { - override fun close() {} - - override fun close(timeout: Duration?) {} - - override fun initTransactions() = - this@stubProducer.initTransactions() - - override fun beginTransaction() = - this@stubProducer.beginTransaction() - - @Suppress("DEPRECATION") - @Deprecated("Deprecated in Java") - override fun sendOffsetsToTransaction( - offsets: MutableMap?, - consumerGroupId: String? - ) = this@stubProducer.sendOffsetsToTransaction(offsets, consumerGroupId) - - override fun sendOffsetsToTransaction( - offsets: MutableMap?, - groupMetadata: ConsumerGroupMetadata? - ) = this@stubProducer.sendOffsetsToTransaction(offsets, groupMetadata) - - override fun commitTransaction() = - this@stubProducer.commitTransaction() - - override fun abortTransaction() = - this@stubProducer.abortTransaction() - - override fun flush() = - this@stubProducer.flush() - - override fun partitionsFor(topic: String?): MutableList = - this@stubProducer.partitionsFor(topic) - - override fun metrics(): MutableMap = - this@stubProducer.metrics() - - override fun send(record: ProducerRecord, callback: Callback): Future = - _sendCallback.invoke(this@stubProducer, record, callback) - - override fun send(record: ProducerRecord): Future = - _send.invoke(this@stubProducer, record) -} \ No newline at end of file From 8e3481f6adc86a1cb65f88c146448006cdf9690d Mon Sep 17 00:00:00 2001 From: Simon Vergauwen Date: Mon, 9 Oct 2023 20:04:50 +0200 Subject: [PATCH 08/16] Update doc --- .../nomisRev/kafka/publisher/PublisherSettings.kt | 11 ++++------- .../nomisrev/kafka/publisher/KafkaPublisherSpec.kt | 12 ------------ 2 files changed, 4 insertions(+), 19 deletions(-) diff --git a/src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherSettings.kt b/src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherSettings.kt index 9aaa60de..396a786c 100644 --- a/src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherSettings.kt +++ b/src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherSettings.kt @@ -49,13 +49,10 @@ data class PublisherSettings( properties[ProducerConfig.TRANSACTIONAL_ID_CONFIG] as? String /** - * Senders created from this options will be transactional if a transactional id is - * configured using {@link ProducerConfig#TRANSACTIONAL_ID_CONFIG}. If transactional, - * {@link KafkaProducer#initTransactions()} is invoked on the producer to initialize - * transactions before any operations are performed on the sender. If scheduler is overridden - * using {@link #scheduler(Scheduler)}, the configured scheduler - * must be single-threaded. Otherwise, the behaviour is undefined and may result in unexpected - * exceptions. + * [KafkaPublisher] created from this options will be transactional if a transactional id is + * configured using [ProducerConfig.TRANSACTIONAL_ID_CONFIG]. If transactional, + * [Producer.initTransactions] is invoked on the producer to initialize + * transactions before any operations are performed on the publisher. */ fun isTransactional(): Boolean = !transactionalId().isNullOrBlank() diff --git a/src/test/kotlin/io/github/nomisrev/kafka/publisher/KafkaPublisherSpec.kt b/src/test/kotlin/io/github/nomisrev/kafka/publisher/KafkaPublisherSpec.kt index d446fbab..2d48c669 100644 --- a/src/test/kotlin/io/github/nomisrev/kafka/publisher/KafkaPublisherSpec.kt +++ b/src/test/kotlin/io/github/nomisrev/kafka/publisher/KafkaPublisherSpec.kt @@ -19,24 +19,12 @@ import kotlinx.coroutines.flow.take import kotlinx.coroutines.flow.toList import kotlinx.coroutines.launch import org.apache.kafka.clients.admin.NewTopic -import org.apache.kafka.clients.consumer.ConsumerGroupMetadata -import org.apache.kafka.clients.consumer.OffsetAndMetadata -import org.apache.kafka.clients.producer.Callback -import org.apache.kafka.clients.producer.Producer import org.apache.kafka.clients.producer.ProducerConfig import org.apache.kafka.clients.producer.ProducerRecord -import org.apache.kafka.clients.producer.RecordMetadata -import org.apache.kafka.common.Metric -import org.apache.kafka.common.MetricName -import org.apache.kafka.common.PartitionInfo -import org.apache.kafka.common.TopicPartition -import org.apache.kafka.common.errors.FencedInstanceIdException import org.apache.kafka.common.errors.ProducerFencedException -import java.time.Duration import java.util.Properties import java.util.concurrent.CompletableFuture import java.util.concurrent.Executors -import java.util.concurrent.Future import java.util.concurrent.TimeUnit import kotlin.time.Duration.Companion.seconds From 7b46e747a55e2f9366cd242725de82df9b54d5b4 Mon Sep 17 00:00:00 2001 From: Simon Vergauwen Date: Mon, 9 Oct 2023 20:43:59 +0200 Subject: [PATCH 09/16] WIP return Deferred from offer --- .../github/nomisRev/kafka/publisher/Publisher.kt | 16 ++++++++++++---- .../nomisRev/kafka/publisher/PublisherScope.kt | 9 ++++++--- 2 files changed, 18 insertions(+), 7 deletions(-) diff --git a/src/main/kotlin/io/github/nomisRev/kafka/publisher/Publisher.kt b/src/main/kotlin/io/github/nomisRev/kafka/publisher/Publisher.kt index b33bacbe..34cc3627 100644 --- a/src/main/kotlin/io/github/nomisRev/kafka/publisher/Publisher.kt +++ b/src/main/kotlin/io/github/nomisRev/kafka/publisher/Publisher.kt @@ -151,15 +151,23 @@ private class DefaultProduceScope( ) : TransactionalScope, CoroutineScope by scope { val parent: Job = requireNotNull(coroutineContext[Job]) { "Impossible, can only be called within coroutineScope" } - override suspend fun offer(record: ProducerRecord) { + override suspend fun offer(record: ProducerRecord) : Deferred { val p: Producer = producer.await() val child = Job(parent) + val deferred = CompletableDeferred(child) runInterruptible(producerContext) { - p.send(record) { _, exception -> - if (exception != null) parent.cancel(ChildCancelScope("Child failed", exception, token)) - else child.complete() + p.send(record) { metadata, exception -> + if (exception != null) { + parent.cancel(ChildCancelScope("Child failed", exception, token)) + deferred.completeExceptionally(exception) + } + else { + child.complete() + deferred.complete(metadata) + } } } + return deferred } override suspend fun publish(record: ProducerRecord): RecordMetadata { diff --git a/src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherScope.kt b/src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherScope.kt index b7ee39e8..ad4bce5e 100644 --- a/src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherScope.kt +++ b/src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherScope.kt @@ -4,6 +4,7 @@ package io.github.nomisRev.kafka.publisher import kotlinx.coroutines.CoroutineScope +import kotlinx.coroutines.Deferred import kotlinx.coroutines.async import kotlinx.coroutines.awaitAll import kotlinx.coroutines.coroutineScope @@ -15,7 +16,7 @@ annotation class PublisherDSL @PublisherDSL interface PublishScope : CoroutineScope { - suspend fun offer(record: ProducerRecord) + suspend fun offer(record: ProducerRecord): Deferred suspend fun publish(record: ProducerRecord): RecordMetadata @@ -23,8 +24,10 @@ interface PublishScope : CoroutineScope { suspend fun publishCatching(record: Iterable>): Result> - suspend fun offer(records: Iterable>) = - records.map { offer(it) } + suspend fun offer(records: Iterable>): Deferred> { + val scope = this + return scope.async { records.map { offer(it) }.awaitAll() } + } suspend fun publish(record: Iterable>): List = coroutineScope { From d889df5beaaf05acc2798a93dfa5fc12de236054 Mon Sep 17 00:00:00 2001 From: Simon Vergauwen Date: Mon, 9 Oct 2023 22:00:25 +0200 Subject: [PATCH 10/16] Clean up tests --- .../nomisRev/kafka/publisher/Publisher.kt | 5 +- .../io/github/nomisrev/kafka/KafkaSpec.kt | 157 ++++++++++++---- .../kotlin/io/github/nomisrev/kafka/Predef.kt | 15 ++ .../kafka/publisher/KafkaPublisherSpec.kt | 167 ++++-------------- 4 files changed, 174 insertions(+), 170 deletions(-) diff --git a/src/main/kotlin/io/github/nomisRev/kafka/publisher/Publisher.kt b/src/main/kotlin/io/github/nomisRev/kafka/publisher/Publisher.kt index 34cc3627..15e26dc9 100644 --- a/src/main/kotlin/io/github/nomisRev/kafka/publisher/Publisher.kt +++ b/src/main/kotlin/io/github/nomisRev/kafka/publisher/Publisher.kt @@ -151,7 +151,7 @@ private class DefaultProduceScope( ) : TransactionalScope, CoroutineScope by scope { val parent: Job = requireNotNull(coroutineContext[Job]) { "Impossible, can only be called within coroutineScope" } - override suspend fun offer(record: ProducerRecord) : Deferred { + override suspend fun offer(record: ProducerRecord): Deferred { val p: Producer = producer.await() val child = Job(parent) val deferred = CompletableDeferred(child) @@ -160,8 +160,7 @@ private class DefaultProduceScope( if (exception != null) { parent.cancel(ChildCancelScope("Child failed", exception, token)) deferred.completeExceptionally(exception) - } - else { + } else { child.complete() deferred.complete(metadata) } diff --git a/src/test/kotlin/io/github/nomisrev/kafka/KafkaSpec.kt b/src/test/kotlin/io/github/nomisrev/kafka/KafkaSpec.kt index ca5f75ce..63cdd5a4 100644 --- a/src/test/kotlin/io/github/nomisrev/kafka/KafkaSpec.kt +++ b/src/test/kotlin/io/github/nomisrev/kafka/KafkaSpec.kt @@ -11,7 +11,14 @@ import io.github.nomisRev.kafka.publisher.KafkaPublisher import io.github.nomisRev.kafka.publisher.PublisherSettings import io.github.nomisRev.kafka.receiver.KafkaReceiver import io.github.nomisRev.kafka.receiver.ReceiverSettings +import io.kotest.assertions.assertSoftly +import io.kotest.assertions.async.shouldTimeout import io.kotest.core.spec.style.StringSpec +import io.kotest.matchers.shouldBe +import kotlinx.coroutines.flow.first +import kotlinx.coroutines.flow.map +import kotlinx.coroutines.flow.take +import kotlinx.coroutines.flow.toList import org.apache.kafka.clients.admin.Admin import org.apache.kafka.clients.admin.AdminClientConfig import org.apache.kafka.clients.admin.NewTopic @@ -34,7 +41,11 @@ import org.testcontainers.utility.DockerImageName import java.time.Duration import java.util.Properties import java.util.UUID +import java.util.concurrent.CompletableFuture +import java.util.concurrent.Executors import java.util.concurrent.Future +import java.util.concurrent.TimeUnit +import kotlin.time.Duration.Companion.milliseconds import kotlin.time.Duration.Companion.seconds abstract class KafkaSpec(body: KafkaSpec.() -> Unit = {}) : StringSpec() { @@ -44,6 +55,7 @@ abstract class KafkaSpec(body: KafkaSpec.() -> Unit = {}) : StringSpec() { private val transactionTimeoutInterval = 1.seconds private val consumerPollingTimeout = 1.seconds + val boom = RuntimeException("Boom!") private val kafkaImage: DockerImageName = DockerImageName.parse("confluentinc/cp-kafka:latest") @@ -121,6 +133,7 @@ abstract class KafkaSpec(body: KafkaSpec.() -> Unit = {}) : StringSpec() { try { action(topic) } finally { + topic.shouldBeEmpty() deleteTopic(topic.name()) } } @@ -157,52 +170,126 @@ abstract class KafkaSpec(body: KafkaSpec.() -> Unit = {}) : StringSpec() { } } - fun stubProducer( - _sendCallback: Producer.(record: ProducerRecord, callback: Callback) -> Future = - { record, callback -> send(record, callback) }, - _send: Producer.(record: ProducerRecord) -> Future = { send(it) } - ) = object : Producer { - override fun close() {} + @JvmName("shouldHaveAllRecords") + suspend fun NewTopic.shouldHaveRecords( + records: Iterable>> + ) { + val expected = + records.flatten().groupBy({ it.partition() }) { it.value() }.mapValues { it.value.toSet() } + KafkaReceiver(receiverSetting()) + .receive(name()) + .map { record -> + record.also { record.offset.acknowledge() } + } + .take(records.flatten().size) + .toList() + .groupBy({ it.partition() }) { it.value() } + .mapValues { it.value.toSet() } shouldBe expected + } - override fun close(timeout: Duration?) {} + suspend fun NewTopic.shouldHaveRecords(records: Iterable>) { + KafkaReceiver(receiverSetting()) + .receive(name()) + .map { record -> + record + .also { record.offset.acknowledge() } + } + .take(records.toList().size) + .toList() + .groupBy({ it.partition() }) { it.value() } shouldBe records.groupBy({ it.partition() }) { it.value() } + } - override fun initTransactions() = - producer.initTransactions() + suspend fun NewTopic.shouldBeEmpty() { + shouldTimeout(100.milliseconds) { + KafkaReceiver(receiverSetting()) + .receive(name()) + .take(1) + .toList() + } + } - override fun beginTransaction() = - producer.beginTransaction() + suspend fun NewTopic.shouldHaveRecord(records: ProducerRecord) { + assertSoftly { + KafkaReceiver(receiverSetting()) + .receive(name()) + .map { + it.apply { offset.acknowledge() } + }.take(1) + .map { it.value() } + .toList() shouldBe listOf(records.value()) + shouldBeEmpty() + } + } - @Suppress("DEPRECATION") - @Deprecated("Deprecated in Java") - override fun sendOffsetsToTransaction( - offsets: MutableMap?, - consumerGroupId: String? - ) = producer.sendOffsetsToTransaction(offsets, consumerGroupId) + suspend fun topicWithSingleMessage(topic: NewTopic, record: ProducerRecord) = + KafkaReceiver(receiverSetting()) + .receive(topic.name()) + .map { + it.apply { offset.acknowledge() } + }.first().value() shouldBe record.value() - override fun sendOffsetsToTransaction( - offsets: MutableMap?, - groupMetadata: ConsumerGroupMetadata? - ) = producer.sendOffsetsToTransaction(offsets, groupMetadata) + suspend fun topicShouldBeEmpty(topic: NewTopic) = + shouldTimeout(1.seconds) { + KafkaReceiver(receiverSetting()) + .receive(topic.name()) + .take(1) + .toList() + } - override fun commitTransaction() = - producer.commitTransaction() + fun stubProducer(failOnNumber: Int? = null): suspend () -> Producer = suspend { + object : Producer { + override fun close() {} - override fun abortTransaction() = - producer.abortTransaction() + override fun close(timeout: Duration?) {} - override fun flush() = - producer.flush() + override fun initTransactions() = + producer.initTransactions() - override fun partitionsFor(topic: String?): MutableList = - producer.partitionsFor(topic) + override fun beginTransaction() = + producer.beginTransaction() - override fun metrics(): MutableMap = - producer.metrics() + @Suppress("DEPRECATION") + @Deprecated("Deprecated in Java") + override fun sendOffsetsToTransaction( + offsets: MutableMap?, + consumerGroupId: String? + ) = producer.sendOffsetsToTransaction(offsets, consumerGroupId) - override fun send(record: ProducerRecord, callback: Callback): Future = - _sendCallback.invoke(producer, record, callback) + override fun sendOffsetsToTransaction( + offsets: MutableMap?, + groupMetadata: ConsumerGroupMetadata? + ) = producer.sendOffsetsToTransaction(offsets, groupMetadata) - override fun send(record: ProducerRecord): Future = - _send.invoke(producer, record) + override fun commitTransaction() = + producer.commitTransaction() + + override fun abortTransaction() = + producer.abortTransaction() + + override fun flush() = + producer.flush() + + override fun partitionsFor(topic: String?): MutableList = + producer.partitionsFor(topic) + + override fun metrics(): MutableMap = + producer.metrics() + + override fun send(record: ProducerRecord, callback: Callback): Future = + if (failOnNumber != null && record.key() == failOnNumber.toString()) { + Executors.newScheduledThreadPool(1).schedule( + { + callback.onCompletion(null, boom) + }, + 50, + TimeUnit.MILLISECONDS + ) + + CompletableFuture.supplyAsync { throw AssertionError("Should never be called") } + } else producer.send(record, callback) + + override fun send(record: ProducerRecord): Future = + producer.send(record) + } } } diff --git a/src/test/kotlin/io/github/nomisrev/kafka/Predef.kt b/src/test/kotlin/io/github/nomisrev/kafka/Predef.kt index 6b5cdc2d..3c01de3d 100644 --- a/src/test/kotlin/io/github/nomisrev/kafka/Predef.kt +++ b/src/test/kotlin/io/github/nomisrev/kafka/Predef.kt @@ -1,5 +1,9 @@ package io.github.nomisrev.kafka +import io.kotest.assertions.fail +import kotlinx.coroutines.CancellationException +import kotlinx.coroutines.CompletableDeferred +import kotlinx.coroutines.CoroutineScope import kotlinx.coroutines.flow.Flow import kotlinx.coroutines.flow.map @@ -10,4 +14,15 @@ inline fun Flow.mapIndexed( return map { value -> transform(index++, value) } +} + +suspend fun CoroutineScope.shouldCancel(block: suspend () -> Unit): CancellationException { + val cancel = CompletableDeferred() + try { + block() + fail("Expected to be cancellable, but wasn't") + } catch (e: CancellationException) { + cancel.complete(e) + } + return cancel.await() } \ No newline at end of file diff --git a/src/test/kotlin/io/github/nomisrev/kafka/publisher/KafkaPublisherSpec.kt b/src/test/kotlin/io/github/nomisrev/kafka/publisher/KafkaPublisherSpec.kt index 2d48c669..b724dfb8 100644 --- a/src/test/kotlin/io/github/nomisrev/kafka/publisher/KafkaPublisherSpec.kt +++ b/src/test/kotlin/io/github/nomisrev/kafka/publisher/KafkaPublisherSpec.kt @@ -4,7 +4,6 @@ import io.github.nomisRev.kafka.publisher.Acks import io.github.nomisRev.kafka.publisher.KafkaPublisher import io.github.nomisRev.kafka.receiver.KafkaReceiver import io.github.nomisrev.kafka.KafkaSpec -import io.kotest.assertions.async.shouldTimeout import io.kotest.assertions.throwables.shouldThrow import io.kotest.matchers.shouldBe import kotlinx.coroutines.CancellationException @@ -23,10 +22,6 @@ import org.apache.kafka.clients.producer.ProducerConfig import org.apache.kafka.clients.producer.ProducerRecord import org.apache.kafka.common.errors.ProducerFencedException import java.util.Properties -import java.util.concurrent.CompletableFuture -import java.util.concurrent.Executors -import java.util.concurrent.TimeUnit -import kotlin.time.Duration.Companion.seconds class KafkaPublisherSpec : KafkaSpec({ @@ -40,15 +35,7 @@ class KafkaPublisherSpec : KafkaSpec({ offer(records) } - KafkaReceiver(receiverSetting()) - .receive(topic.name()) - .map { record -> - Pair(record.partition(), listOf(record.value())) - .also { record.offset.acknowledge() } - } - .take(count + 1) - .toList() - .toMap() shouldBe records.groupBy({ it.partition() }) { it.value() } + topic.shouldHaveRecords(records) } } @@ -62,21 +49,12 @@ class KafkaPublisherSpec : KafkaSpec({ publish(records) } - KafkaReceiver(receiverSetting()) - .receive(topic.name()) - .map { record -> - Pair(record.partition(), listOf(record.value())) - .also { record.offset.acknowledge() } - } - .take(count + 1) - .toList() - .toMap() shouldBe records.groupBy({ it.partition() }) { it.value() } + topic.shouldHaveRecords(records) } } "A failure in a produce block, rethrows the error" { withTopic(partitions = 4) { topic -> - val boom = RuntimeException("Boom!") val record = topic.createProducerRecord(0) shouldThrow { @@ -86,17 +64,12 @@ class KafkaPublisherSpec : KafkaSpec({ } } shouldBe boom - KafkaReceiver(receiverSetting()) - .receive(topic.name()) - .map { - it.apply { offset.acknowledge() } - }.first().value() shouldBe record.value() + topic.shouldHaveRecord(record) } } "A failure in a produce block with a concurrent launch cancels the launch, rethrows the error" { withTopic(partitions = 4) { topic -> - val boom = RuntimeException("Boom!") val cancelSignal = CompletableDeferred() shouldThrow { publisher.publishScope { @@ -115,37 +88,37 @@ class KafkaPublisherSpec : KafkaSpec({ } } - "A failed offer is rethrow at the end" { + "A failed offer is rethrown at the end" { withTopic(partitions = 4) { topic -> - val boom = RuntimeException("Boom!") val record = topic.createProducerRecord(0) - val failingProducer = stubProducer(_sendCallback = { metadata, callback -> - if (metadata.key().equals("0")) { - Executors.newScheduledThreadPool(1).schedule( - { - callback.onCompletion(null, boom) - }, - 1, - TimeUnit.SECONDS - ) - - CompletableFuture.supplyAsync { throw AssertionError("Should never be called") } - } else send(record, callback) - }) shouldThrow { - KafkaPublisher(publisherSettings()) { failingProducer }.publishScope { + KafkaPublisher(publisherSettings(), stubProducer(failOnNumber = 0)).publishScope { offer(record) } } shouldBe boom } } + "A failed offer await is rethrow at the end" { + withTopic(partitions = 4) { topic -> + val record0 = topic.createProducerRecord(0) + val record1 = topic.createProducerRecord(1) + shouldThrow { + KafkaPublisher(publisherSettings(), stubProducer(failOnNumber = 1)).publishScope { + publish(record0) + offer(record1).await() + } + } shouldBe boom + + topic.shouldHaveRecord(record0) + } + } + "An async failure is rethrow at the end" { withTopic(partitions = 4) { topic -> val count = 3 - val boom = RuntimeException("Boom!") - val records = (0..count).map { + val records: List> = (0..count).map { topic.createProducerRecord(it) } shouldThrow { @@ -155,41 +128,23 @@ class KafkaPublisherSpec : KafkaSpec({ } } shouldBe boom - KafkaReceiver(receiverSetting()) - .receive(topic.name()) - .map { record -> - Pair(record.partition(), listOf(record.value())) - .also { record.offset.acknowledge() } - } - .take(3 + 1) - .toList() - .toMap() shouldBe records.groupBy({ it.partition() }) { it.value() } + topic.shouldHaveRecords(records) } } "A failure of a sendAwait can be caught in the block" { withTopic(partitions = 4) { topic -> - val record = topic.createProducerRecord(0) - val record2 = topic.createProducerRecord(1) - val failingProducer = stubProducer(_sendCallback = { metadata, callback -> - if (metadata.key().equals("0")) { - callback.onCompletion(null, RuntimeException("Boom!")) - CompletableFuture.supplyAsync { throw AssertionError("Should never be called") } - } else send(record, callback) - }) - - KafkaPublisher(publisherSettings()) { failingProducer }.use { + val record0 = topic.createProducerRecord(0) + val record1 = topic.createProducerRecord(1) + + KafkaPublisher(publisherSettings(), stubProducer(failOnNumber = 0)).use { it.publishScope { - publishCatching(record) - offer(record2) + publishCatching(record0) + offer(record1) } } - KafkaReceiver(receiverSetting()) - .receive(topic.name()) - .map { - it.apply { offset.acknowledge() } - }.first().value() shouldBe record.value() + topic.shouldHaveRecord(record1) } } @@ -210,18 +165,7 @@ class KafkaPublisherSpec : KafkaSpec({ ).awaitAll() } - val expected = - records.flatten().groupBy({ it.partition() }) { it.value() }.mapValues { it.value.toSet() } - - KafkaReceiver(receiverSetting()) - .receive(topic.name()) - .map { record -> - record.also { record.offset.acknowledge() } - } - .take(records.flatten().size) - .toList() - .groupBy({ it.partition() }) { it.value() } - .mapValues { it.value.toSet() } shouldBe expected + topic.shouldHaveRecords(records) } } @@ -246,22 +190,13 @@ class KafkaPublisherSpec : KafkaSpec({ } } - KafkaReceiver(receiverSetting()) - .receive(topic.name()) - .map { record -> - Pair(record.partition(), listOf(record.value())) - .also { record.offset.acknowledge() } - } - .take(count + 1) - .toList() - .toMap() shouldBe records.groupBy({ it.partition() }) { it.value() } + topic.shouldHaveRecords(records) } } "A failure in a transaction aborts the transaction" { withTopic(partitions = 4) { topic -> val count = 3 - val boom = RuntimeException("Boom!") val records = (0..count).map { topic.createProducerRecord(it) } @@ -283,19 +218,13 @@ class KafkaPublisherSpec : KafkaSpec({ } } shouldBe boom - shouldTimeout(1.seconds) { - KafkaReceiver(receiverSetting()) - .receive(topic.name()) - .take(1) - .toList() - } + topic.shouldBeEmpty() } } "An async failure in a transaction aborts the transaction" { withTopic(partitions = 4) { topic -> val count = 3 - val boom = RuntimeException("Boom!") val records = (0..count).map { topic.createProducerRecord(it) } @@ -317,12 +246,7 @@ class KafkaPublisherSpec : KafkaSpec({ } } shouldBe boom - shouldTimeout(1.seconds) { - KafkaReceiver(receiverSetting()) - .receive(topic.name()) - .take(1) - .toList() - } + topic.shouldBeEmpty() } } @@ -355,18 +279,7 @@ class KafkaPublisherSpec : KafkaSpec({ } } - val expected = - records.flatten().groupBy({ it.partition() }) { it.value() }.mapValues { it.value.toSet() } - - KafkaReceiver(receiverSetting()) - .receive(topic.name()) - .map { record -> - record.also { record.offset.acknowledge() } - } - .take(records.flatten().size) - .toList() - .groupBy({ it.partition() }) { it.value() } - .mapValues { it.value.toSet() } shouldBe expected + topic.shouldHaveRecords(records) } } @@ -405,17 +318,7 @@ class KafkaPublisherSpec : KafkaSpec({ } } - val expected = - (records1 + records2).groupBy({ it.partition() }) { it.value() } - - KafkaReceiver(receiverSetting()) - .receive(topic.name()) - .map { record -> - record.also { record.offset.acknowledge() } - } - .take((records1 + records2).size) - .toList() - .groupBy({ it.partition() }) { it.value() } shouldBe expected + topic.shouldHaveRecords(records1 + records2) } } }) From 3ee52464a468b096d3dace65ee6d54595d015a7c Mon Sep 17 00:00:00 2001 From: Simon Vergauwen Date: Mon, 9 Oct 2023 23:41:25 +0200 Subject: [PATCH 11/16] Document PublishScope, and TransactionalScope --- .../{Publisher.kt => KafkaPublisher.kt} | 18 ++-- .../kafka/publisher/PublisherScope.kt | 92 ++++++++++++++++--- .../kafka/publisher/PublisherSettings.kt | 2 +- .../io/github/nomisrev/kafka/KafkaSpec.kt | 10 +- .../kafka/publisher/KafkaPublisherSpec.kt | 56 ++++++----- 5 files changed, 122 insertions(+), 56 deletions(-) rename src/main/kotlin/io/github/nomisRev/kafka/publisher/{Publisher.kt => KafkaPublisher.kt} (94%) diff --git a/src/main/kotlin/io/github/nomisRev/kafka/publisher/Publisher.kt b/src/main/kotlin/io/github/nomisRev/kafka/publisher/KafkaPublisher.kt similarity index 94% rename from src/main/kotlin/io/github/nomisRev/kafka/publisher/Publisher.kt rename to src/main/kotlin/io/github/nomisRev/kafka/publisher/KafkaPublisher.kt index 15e26dc9..f34c729c 100644 --- a/src/main/kotlin/io/github/nomisRev/kafka/publisher/Publisher.kt +++ b/src/main/kotlin/io/github/nomisRev/kafka/publisher/KafkaPublisher.kt @@ -39,10 +39,10 @@ fun KafkaPublisher( settings: PublisherSettings, createProducer: suspend () -> Producer = { KafkaProducer(settings.properties(), settings.keySerializer, settings.valueSerializer) } -): Publisher = +): KafkaPublisher = DefaultKafkaPublisher(settings, createProducer) -interface Publisher : AutoCloseable { +interface KafkaPublisher : AutoCloseable { suspend fun publishScope(block: suspend TransactionalScope.() -> A): A /** @@ -60,7 +60,7 @@ interface Publisher : AutoCloseable { private class DefaultKafkaPublisher( val settings: PublisherSettings, createProducer: suspend () -> Producer -) : Publisher { +) : KafkaPublisher { val producerId = "reactor-kafka-sender-${System.identityHashCode(this)}" val producerContext: ExecutorCoroutineDispatcher = @@ -128,7 +128,7 @@ private class DefaultKafkaPublisher( } companion object { - val log: Logger = LoggerFactory.getLogger(Publisher::class.java.name) + val log: Logger = LoggerFactory.getLogger(KafkaPublisher::class.java.name) } } @@ -151,7 +151,7 @@ private class DefaultProduceScope( ) : TransactionalScope, CoroutineScope by scope { val parent: Job = requireNotNull(coroutineContext[Job]) { "Impossible, can only be called within coroutineScope" } - override suspend fun offer(record: ProducerRecord): Deferred { + override suspend fun offer(record: ProducerRecord): OfferAck { val p: Producer = producer.await() val child = Job(parent) val deferred = CompletableDeferred(child) @@ -166,7 +166,7 @@ private class DefaultProduceScope( } } } - return deferred + return OfferAck(deferred) } override suspend fun publish(record: ProducerRecord): RecordMetadata { @@ -231,6 +231,10 @@ private class DefaultProduceScope( ) throw t else Unit } +/* + * Marker type, that this is our cancellation signal. + * This allows us to check if it's our scope that is failing using [token]. + */ @OptIn(ExperimentalCoroutinesApi::class) private class ChildCancelScope( message: String, @@ -263,6 +267,8 @@ private class ChildCancelScope( } } +// Piggyback (copied) on KotlinX Coroutines DEBUG mechanism. +// https://github.com/Kotlin/kotlinx.coroutines/blob/ed0cf7aa02b1266cb81e65e61b3a97b0e041a817/kotlinx-coroutines-core/jvm/src/Debug.kt#L70 private val ASSERTIONS_ENABLED = ChildCancelScope::class.java.desiredAssertionStatus() private val DEBUG = try { diff --git a/src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherScope.kt b/src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherScope.kt index ad4bce5e..e9af4e97 100644 --- a/src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherScope.kt +++ b/src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherScope.kt @@ -1,5 +1,4 @@ -@file:JvmMultifileClass -@file:JvmName("PublisherScope.kt") +@file:JvmMultifileClass @file:JvmName("PublisherScope.kt") package io.github.nomisRev.kafka.publisher @@ -7,35 +6,102 @@ import kotlinx.coroutines.CoroutineScope import kotlinx.coroutines.Deferred import kotlinx.coroutines.async import kotlinx.coroutines.awaitAll +import kotlinx.coroutines.Job import kotlinx.coroutines.coroutineScope import org.apache.kafka.clients.producer.ProducerRecord +import org.apache.kafka.clients.producer.Producer import org.apache.kafka.clients.producer.RecordMetadata +import org.apache.kafka.clients.producer.ProducerConfig +import org.apache.kafka.common.errors.ProducerFencedException @DslMarker annotation class PublisherDSL +@JvmInline +value class OfferAck(val acknowledgement: Deferred) + +@JvmInline +value class OfferAcks(val acknowledgements: Deferred>) + +/** + * The DSL, or receiver type, of [KafkaPublisher.publishScope] and [TransactionalScope.transaction]. + * + * There are 2 main methods for sending recordings to kafka. + * - [offer], this gives you highest throughput and should generally be preferred. + * - [publish], this gives you less throughput, but waits on the delivery of the messages. + */ @PublisherDSL interface PublishScope : CoroutineScope { - suspend fun offer(record: ProducerRecord): Deferred - - suspend fun publish(record: ProducerRecord): RecordMetadata - suspend fun publishCatching(record: ProducerRecord): Result + /** + * Offer a [record] to Kafka, and immediately return. + * This methods should be prepared for highest throughput, + * if the [offer] fails it will cancel the [CoroutineScope] & [PublishScope]. + * + * **IMPORTANT:** The returned [OfferAck] is typically not awaited, + * this results in slower throughput since you'll be awaiting every message to be delivered. Use [publish] instead. + * Cancelling doesn't cancel the [offer]/[Producer.send]. + * + * @param record to be offered to kafka + */ + suspend fun offer(record: ProducerRecord): OfferAck - suspend fun publishCatching(record: Iterable>): Result> + /** + * Publisher a [record] to Kafka, and suspends until acknowledged by kafka. + * This way of sending records to kafka results in a lower throughput. + * + * **IMPORTANT:** publish is slower than offer, if you need high throughput simply use [offer]. + * Cancelling doesn't cancel the [publish]/[Producer.send]. + * + * @param record to be delivered to kafka + */ + suspend fun publish(record: ProducerRecord): RecordMetadata - suspend fun offer(records: Iterable>): Deferred> { + /** + * [offer] an [Iterable] of [ProducerRecord] + * This methods should be prepared for highest throughput, + * if one of [offer] fails it will cancel the [CoroutineScope] & [PublishScope]. + * + * **IMPORTANT:** The returned [OfferAck] is typically not awaited to maintain high throughput, + * if you need Ack + * and cancelling doesn't cancel the [Producer.send]. + */ + suspend fun offer(records: Iterable>): OfferAcks { val scope = this - return scope.async { records.map { offer(it) }.awaitAll() } + return OfferAcks(scope.async { records.map { offer(it).acknowledgement }.awaitAll() }) } - suspend fun publish(record: Iterable>): List = - coroutineScope { - record.map { async { publish(it) } }.awaitAll() - } + suspend fun publish(record: Iterable>): List = coroutineScope { + record.map { async { publish(it) } }.awaitAll() + } + /** Alias for `runCatching`, and `publish` except rethrows fatal exceptions */ + suspend fun publishCatching(record: ProducerRecord): Result + + /** + * Catch first failure of [publish], except fatal exceptions. + * Alias for `runCatching`, `publish`, and `awaitAll`, except rethrows fatal exceptions + */ + suspend fun publishCatching(record: Iterable>): Result> } +/** + * The DSL, or receiver type, of [KafkaPublisher.publishScope], + * it allows for publisher configured with [ProducerConfig.TRANSACTIONAL_ID_CONFIG], + * and [ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG] set to `true` to work transitionally. + * Only one publisher with the same [ProducerConfig.TRANSACTIONAL_ID_CONFIG] can run at a given time. + * An expired producer, still works but will throw [ProducerFencedException] if attempt to make a transaction again. + * + * In addition to [PublishScope] 2 main methods, a [TransactionalScope] also has the [transaction] method. + * + * It creates a **new** nested [PublishScope], and runs the lambda with the new scope as receiver. + * + * If the lambda succeeds, and the children of [Job] have completed, + * it automatically commits the [transaction]. + * + * In case the lambda fails, or any children of this [Job] fail, + * then it'll be rethrown from this block and the transaction will be aborted. + */ @PublisherDSL interface TransactionalScope : PublishScope { suspend fun transaction(block: suspend PublishScope.() -> A): A diff --git a/src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherSettings.kt b/src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherSettings.kt index 396a786c..56f045fc 100644 --- a/src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherSettings.kt +++ b/src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherSettings.kt @@ -12,7 +12,7 @@ import java.util.Properties import kotlin.time.Duration /** - * Typed setting to create a [Publisher], enforces the required parameters and leaves the rest as [Properties]. + * Typed setting to create a [KafkaPublisher], enforces the required parameters and leaves the rest as [Properties]. * * @param bootstrapServers list of host/port pairs to use for establishing the initial connection to the Kafka cluster. Should be comma separated. * @param keySerializer the [Serializer] to use to serialize the [Key] when sending messages to kafka. diff --git a/src/test/kotlin/io/github/nomisrev/kafka/KafkaSpec.kt b/src/test/kotlin/io/github/nomisrev/kafka/KafkaSpec.kt index 63cdd5a4..011f98b7 100644 --- a/src/test/kotlin/io/github/nomisrev/kafka/KafkaSpec.kt +++ b/src/test/kotlin/io/github/nomisrev/kafka/KafkaSpec.kt @@ -123,7 +123,7 @@ abstract class KafkaSpec(body: KafkaSpec.() -> Unit = {}) : StringSpec() { suspend fun withTopic( topicConfig: Map = emptyMap(), - partitions: Int = 1, + partitions: Int = 4, replicationFactor: Short = 1, action: suspend Admin.(NewTopic) -> A, ): A { @@ -144,11 +144,9 @@ abstract class KafkaSpec(body: KafkaSpec.() -> Unit = {}) : StringSpec() { topic: NewTopic, messages: Iterable>, ): Unit = - publisher.publishScope { - offer(messages.map { (key, value) -> - ProducerRecord(topic.name(), key, value) - }) - } + publishToKafka(messages.map { (key, value) -> + ProducerRecord(topic.name(), key, value) + }) suspend fun publishToKafka(messages: Iterable>): Unit = publisher.publishScope { diff --git a/src/test/kotlin/io/github/nomisrev/kafka/publisher/KafkaPublisherSpec.kt b/src/test/kotlin/io/github/nomisrev/kafka/publisher/KafkaPublisherSpec.kt index b724dfb8..d3909717 100644 --- a/src/test/kotlin/io/github/nomisrev/kafka/publisher/KafkaPublisherSpec.kt +++ b/src/test/kotlin/io/github/nomisrev/kafka/publisher/KafkaPublisherSpec.kt @@ -2,7 +2,6 @@ package io.github.nomisrev.kafka.publisher import io.github.nomisRev.kafka.publisher.Acks import io.github.nomisRev.kafka.publisher.KafkaPublisher -import io.github.nomisRev.kafka.receiver.KafkaReceiver import io.github.nomisrev.kafka.KafkaSpec import io.kotest.assertions.throwables.shouldThrow import io.kotest.matchers.shouldBe @@ -12,10 +11,6 @@ import kotlinx.coroutines.CoroutineStart.UNDISPATCHED import kotlinx.coroutines.async import kotlinx.coroutines.awaitAll import kotlinx.coroutines.awaitCancellation -import kotlinx.coroutines.flow.first -import kotlinx.coroutines.flow.map -import kotlinx.coroutines.flow.take -import kotlinx.coroutines.flow.toList import kotlinx.coroutines.launch import org.apache.kafka.clients.admin.NewTopic import org.apache.kafka.clients.producer.ProducerConfig @@ -25,8 +20,8 @@ import java.util.Properties class KafkaPublisherSpec : KafkaSpec({ - "Can receive all messages that were offered on the right partitions" { - withTopic(partitions = 4) { topic -> + "All offered messages are received" { + withTopic { topic -> val count = 3 val records = (0..count).map { topic.createProducerRecord(it) @@ -40,7 +35,7 @@ class KafkaPublisherSpec : KafkaSpec({ } "Can receive all messages that were published on the right partitions" { - withTopic(partitions = 4) { topic -> + withTopic { topic -> val count = 3 val records = (0..count).map { topic.createProducerRecord(it) @@ -54,11 +49,11 @@ class KafkaPublisherSpec : KafkaSpec({ } "A failure in a produce block, rethrows the error" { - withTopic(partitions = 4) { topic -> + withTopic { topic -> val record = topic.createProducerRecord(0) shouldThrow { - publisher.publishScope { + publisher.publishScope { offer(record) throw boom } @@ -69,10 +64,10 @@ class KafkaPublisherSpec : KafkaSpec({ } "A failure in a produce block with a concurrent launch cancels the launch, rethrows the error" { - withTopic(partitions = 4) { topic -> + withTopic { _ -> val cancelSignal = CompletableDeferred() shouldThrow { - publisher.publishScope { + publisher.publishScope { launch(start = UNDISPATCHED) { try { awaitCancellation() @@ -89,7 +84,7 @@ class KafkaPublisherSpec : KafkaSpec({ } "A failed offer is rethrown at the end" { - withTopic(partitions = 4) { topic -> + withTopic { topic -> val record = topic.createProducerRecord(0) shouldThrow { @@ -101,13 +96,13 @@ class KafkaPublisherSpec : KafkaSpec({ } "A failed offer await is rethrow at the end" { - withTopic(partitions = 4) { topic -> + withTopic { topic -> val record0 = topic.createProducerRecord(0) val record1 = topic.createProducerRecord(1) shouldThrow { KafkaPublisher(publisherSettings(), stubProducer(failOnNumber = 1)).publishScope { publish(record0) - offer(record1).await() + offer(record1).acknowledgement.await() } } shouldBe boom @@ -116,7 +111,7 @@ class KafkaPublisherSpec : KafkaSpec({ } "An async failure is rethrow at the end" { - withTopic(partitions = 4) { topic -> + withTopic { topic -> val count = 3 val records: List> = (0..count).map { topic.createProducerRecord(it) @@ -133,7 +128,7 @@ class KafkaPublisherSpec : KafkaSpec({ } "A failure of a sendAwait can be caught in the block" { - withTopic(partitions = 4) { topic -> + withTopic { topic -> val record0 = topic.createProducerRecord(0) val record1 = topic.createProducerRecord(1) @@ -149,11 +144,11 @@ class KafkaPublisherSpec : KafkaSpec({ } "concurrent publishing" { - withTopic(partitions = 4) { topic -> + withTopic { topic -> val count = 4 val records = - (1..count).map { - (it + 1..it + count).map { topic.createProducerRecord(it) } + (0.. + (base..base + count).map { topic.createProducerRecord(it) } } publisher.publishScope { @@ -170,7 +165,7 @@ class KafkaPublisherSpec : KafkaSpec({ } "transaction an receive all messages that were published on the right partitions" { - withTopic(partitions = 4) { topic -> + withTopic { topic -> val count = 3 val records = (0..count).map { topic.createProducerRecord(it) @@ -195,7 +190,7 @@ class KafkaPublisherSpec : KafkaSpec({ } "A failure in a transaction aborts the transaction" { - withTopic(partitions = 4) { topic -> + withTopic { topic -> val count = 3 val records = (0..count).map { topic.createProducerRecord(it) @@ -209,7 +204,7 @@ class KafkaPublisherSpec : KafkaSpec({ ) shouldThrow { KafkaPublisher(settings).use { - it.publishScope { + it.publishScope { transaction { publish(records) throw boom @@ -223,7 +218,7 @@ class KafkaPublisherSpec : KafkaSpec({ } "An async failure in a transaction aborts the transaction" { - withTopic(partitions = 4) { topic -> + withTopic { topic -> val count = 3 val records = (0..count).map { topic.createProducerRecord(it) @@ -251,11 +246,11 @@ class KafkaPublisherSpec : KafkaSpec({ } "transaction - concurrent publishing" { - withTopic(partitions = 4) { topic -> + withTopic { topic -> val count = 4 val records = - (1..count).map { - (it + 1..it + count).map { topic.createProducerRecord(it) } + (0.. + (base..base + count).map { topic.createProducerRecord(it) } } val settings = publisherSettings( @@ -283,8 +278,8 @@ class KafkaPublisherSpec : KafkaSpec({ } } - "Only one transactional sender with a specific transactional id is allowed at the same time" { - withTopic(partitions = 4) { topic -> + "Only one KafkaProducer can have transactional.id at the same time, resulting ProducerFencedException is fatal" { + withTopic { topic -> val settings = publisherSettings( acknowledgments = Acks.All, properties = Properties().apply { @@ -308,16 +303,17 @@ class KafkaPublisherSpec : KafkaSpec({ } } + // publisher1 was previous transactional.id, will result in fatal ProducerFencedException val records3 = (10..14).map { topic.createProducerRecord(it) } shouldThrow { publisher1.publishScope { transaction { - // Test that ProducerFencedException is fatal publishCatching(records3) } } } + // Due to ProducerFencedException, only records1 and records2 are received topic.shouldHaveRecords(records1 + records2) } } From ea0c56aa004d583c4be19a3494e1cb82bb210d7d Mon Sep 17 00:00:00 2001 From: Simon Vergauwen Date: Mon, 9 Oct 2023 23:49:44 +0200 Subject: [PATCH 12/16] More docs --- .../nomisRev/kafka/publisher/PublisherScope.kt | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherScope.kt b/src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherScope.kt index e9af4e97..6ca01aee 100644 --- a/src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherScope.kt +++ b/src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherScope.kt @@ -58,19 +58,18 @@ interface PublishScope : CoroutineScope { suspend fun publish(record: ProducerRecord): RecordMetadata /** - * [offer] an [Iterable] of [ProducerRecord] - * This methods should be prepared for highest throughput, - * if one of [offer] fails it will cancel the [CoroutineScope] & [PublishScope]. - * - * **IMPORTANT:** The returned [OfferAck] is typically not awaited to maintain high throughput, - * if you need Ack - * and cancelling doesn't cancel the [Producer.send]. + * Same as [offer], but for an [Iterable]] of [ProducerRecord]. + * @see offer */ suspend fun offer(records: Iterable>): OfferAcks { val scope = this return OfferAcks(scope.async { records.map { offer(it).acknowledgement }.awaitAll() }) } + /** + * Same as [publish], but for an [Iterable]] of [ProducerRecord]. + * @see publish + */ suspend fun publish(record: Iterable>): List = coroutineScope { record.map { async { publish(it) } }.awaitAll() } From cd65ab6c0480b6b88ae11847f8fdb2b2749f1254 Mon Sep 17 00:00:00 2001 From: Simon Vergauwen Date: Tue, 10 Oct 2023 00:03:53 +0200 Subject: [PATCH 13/16] Finish docs --- .../github/nomisRev/kafka/publisher/Acks.kt | 11 ++-- .../kafka/publisher/KafkaPublisher.kt | 55 ++++++++++++++++--- 2 files changed, 54 insertions(+), 12 deletions(-) diff --git a/src/main/kotlin/io/github/nomisRev/kafka/publisher/Acks.kt b/src/main/kotlin/io/github/nomisRev/kafka/publisher/Acks.kt index 74e64a99..f28c545e 100644 --- a/src/main/kotlin/io/github/nomisRev/kafka/publisher/Acks.kt +++ b/src/main/kotlin/io/github/nomisRev/kafka/publisher/Acks.kt @@ -1,18 +1,19 @@ package io.github.nomisRev.kafka.publisher +import org.apache.kafka.clients.producer.ProducerConfig + /** * The number of acknowledgments the producer requires the leader to have received before considering a request complete. * This controls the durability of records that are sent * - * **Note:** that enabling idempotence requires this config value to be 'all'. - * If conflicting configurations are set and idempotence is not explicitly enabled, idempotence is disabled. + * **NOTE:** Enabling idempotence requires this config value to be [All], otherwise [ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG] is ignored. */ public enum class Acks(public val value: String) { /** - * If set to zero then the producer will not wait for any acknowledgment from the server at all. + * Using [Zero] the producer will not wait for any acknowledgment from the server at all. * The record will be immediately added to the socket buffer and considered sent. * No guarantee can be made that the server has received the record in this case, - * and the retries configuration will not take effect (as the client won't generally know of any failures). + * and [ProducerConfig.RETRIES_CONFIG] will not take effect (as the client won't generally know of any failures). * The offset given back for each record will always be set to `-1` */ Zero("0"), @@ -26,7 +27,7 @@ public enum class Acks(public val value: String) { /** * This means the leader will wait for the full set of in-sync replicas to acknowledge the record. * This guarantees that the record will not be lost as long as at least one in-sync replica remains alive. - * This is the strongest available guarantee. This is equivalent to the `acks=-1` setting. + * This is the strongest available guarantee. This is equivalent to the [MinusOne] setting. */ All("all"), diff --git a/src/main/kotlin/io/github/nomisRev/kafka/publisher/KafkaPublisher.kt b/src/main/kotlin/io/github/nomisRev/kafka/publisher/KafkaPublisher.kt index f34c729c..bd4f6bfd 100644 --- a/src/main/kotlin/io/github/nomisRev/kafka/publisher/KafkaPublisher.kt +++ b/src/main/kotlin/io/github/nomisRev/kafka/publisher/KafkaPublisher.kt @@ -35,24 +35,65 @@ import java.util.concurrent.Executors import kotlin.coroutines.cancellation.CancellationException import kotlin.time.toJavaDuration +/** + * Constructing a [KafkaPublisher] requires [PublisherSettings]. + * Optionally you can provide a different way to how the [Producer] is created. + */ fun KafkaPublisher( settings: PublisherSettings, - createProducer: suspend () -> Producer = + configureProducer: suspend () -> Producer = { KafkaProducer(settings.properties(), settings.keySerializer, settings.valueSerializer) } ): KafkaPublisher = - DefaultKafkaPublisher(settings, createProducer) + DefaultKafkaPublisher(settings, configureProducer) +/** + * A [KafkaPublisher] wraps an [Producer], so needs to be closed by [AutoCloseable]. + * It has 1 main method, [publishScope] which creates a [PublishScope], + * and two suspending methods from the [Producer] [partitionsFor], and [metrics]. + */ interface KafkaPublisher : AutoCloseable { - suspend fun publishScope(block: suspend TransactionalScope.() -> A): A /** - * @see KafkaProducer.partitionsFor + * Create and run a [publishScope], which can [PublishScope.offer] and [PublishScope.publish] records to Kafka. + * It awaits all inflight offers to finish, and reports any errors. + * + * If the [block] fails, or one of the children of the created [CoroutineScope], + * than the exception is rethrown and the [PublishScope] gets cancelled. + * + * Just like [coroutineScope] it awaits all its children to finish. + * + * ```kotlin + * publisher.publishScope { + * offer((1..10).map { + * ProducerRecord(topic.name(), "$it", "msg-$it") + * }) + * publish((11..20).map { + * ProducerRecord(topic.name(), "$it", "msg-$it") + * }) + * transaction { + * // transaction { } compiler error: illegal to be called here + * offer((21..30).map { + * ProducerRecord(topic.name(), "$it", "msg-$it") + * }) + * publish((31..40).map { + * ProducerRecord(topic.name(), "$it", "msg-$it") + * }) + * }// Waits until all offer finished in transaction, fails if any failed + * + * // streaming + * flow(1..100) + * .onEach { delay(100.milliseconds) } + * .map { ProducerRecord(topic.name(), "$it", "msg-$it") } + * .collect { offer(it) } + * } + * ``` */ + suspend fun publishScope(block: suspend TransactionalScope.() -> A): A + + /** @see KafkaProducer.partitionsFor */ suspend fun partitionsFor(topic: String): List - /** - * @see KafkaProducer.metrics - */ + /** @see KafkaProducer.metrics */ suspend fun metrics(): Map } From 224498242802b04a7d2c6e1e9dfeebaf2d003a76 Mon Sep 17 00:00:00 2001 From: Simon Vergauwen Date: Tue, 10 Oct 2023 00:07:15 +0200 Subject: [PATCH 14/16] transaction docs --- .../kafka/publisher/PublisherScope.kt | 25 +++++++++++++++++++ 1 file changed, 25 insertions(+) diff --git a/src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherScope.kt b/src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherScope.kt index 6ca01aee..2428ce82 100644 --- a/src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherScope.kt +++ b/src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherScope.kt @@ -103,5 +103,30 @@ interface PublishScope : CoroutineScope { */ @PublisherDSL interface TransactionalScope : PublishScope { + + /** + * Create and run a [transaction], which can [PublishScope.offer] and [PublishScope.publish] records to Kafka. + * It awaits all inflight offers to finish, when successfully it commits the transaction and returns. + * In case of failure + * + * If the [block] fails, or one of the children of the created [CoroutineScope], + * then it aborts the transaction and the exception is rethrown and the [PublishScope] gets cancelled. + * + * Just like [coroutineScope] it awaits all its children to finish. + * + * ```kotlin + * publisher.publishScope { + * transaction { + * // transaction { } compiler error: illegal to be called here + * offer((21..30).map { + * ProducerRecord(topic.name(), "$it", "msg-$it") + * }) + * publish((31..40).map { + * ProducerRecord(topic.name(), "$it", "msg-$it") + * }) + * }// Waits until all offer finished in transaction, fails if any failed + * } + * ``` + */ suspend fun transaction(block: suspend PublishScope.() -> A): A } From 56ea80e48e08330a723f5c8399d32fdf9194321e Mon Sep 17 00:00:00 2001 From: Simon Vergauwen Date: Tue, 10 Oct 2023 16:25:28 +0200 Subject: [PATCH 15/16] Remove OfferAck, should use publish instead --- .../kafka/publisher/KafkaPublisher.kt | 15 ++++---------- .../kafka/publisher/PublisherScope.kt | 20 ++++--------------- .../kafka/publisher/KafkaPublisherSpec.kt | 15 -------------- 3 files changed, 8 insertions(+), 42 deletions(-) diff --git a/src/main/kotlin/io/github/nomisRev/kafka/publisher/KafkaPublisher.kt b/src/main/kotlin/io/github/nomisRev/kafka/publisher/KafkaPublisher.kt index bd4f6bfd..12af640c 100644 --- a/src/main/kotlin/io/github/nomisRev/kafka/publisher/KafkaPublisher.kt +++ b/src/main/kotlin/io/github/nomisRev/kafka/publisher/KafkaPublisher.kt @@ -192,22 +192,15 @@ private class DefaultProduceScope( ) : TransactionalScope, CoroutineScope by scope { val parent: Job = requireNotNull(coroutineContext[Job]) { "Impossible, can only be called within coroutineScope" } - override suspend fun offer(record: ProducerRecord): OfferAck { + override suspend fun offer(record: ProducerRecord) { val p: Producer = producer.await() val child = Job(parent) - val deferred = CompletableDeferred(child) runInterruptible(producerContext) { - p.send(record) { metadata, exception -> - if (exception != null) { - parent.cancel(ChildCancelScope("Child failed", exception, token)) - deferred.completeExceptionally(exception) - } else { - child.complete() - deferred.complete(metadata) - } + p.send(record) { _, exception -> + if (exception != null) parent.cancel(ChildCancelScope("Child failed", exception, token)) + else child.complete() } } - return OfferAck(deferred) } override suspend fun publish(record: ProducerRecord): RecordMetadata { diff --git a/src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherScope.kt b/src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherScope.kt index 2428ce82..a14fa040 100644 --- a/src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherScope.kt +++ b/src/main/kotlin/io/github/nomisRev/kafka/publisher/PublisherScope.kt @@ -17,12 +17,6 @@ import org.apache.kafka.common.errors.ProducerFencedException @DslMarker annotation class PublisherDSL -@JvmInline -value class OfferAck(val acknowledgement: Deferred) - -@JvmInline -value class OfferAcks(val acknowledgements: Deferred>) - /** * The DSL, or receiver type, of [KafkaPublisher.publishScope] and [TransactionalScope.transaction]. * @@ -34,17 +28,13 @@ value class OfferAcks(val acknowledgements: Deferred>) interface PublishScope : CoroutineScope { /** - * Offer a [record] to Kafka, and immediately return. + * Offer the [record] to Kafka, and immediately return. * This methods should be prepared for highest throughput, * if the [offer] fails it will cancel the [CoroutineScope] & [PublishScope]. * - * **IMPORTANT:** The returned [OfferAck] is typically not awaited, - * this results in slower throughput since you'll be awaiting every message to be delivered. Use [publish] instead. - * Cancelling doesn't cancel the [offer]/[Producer.send]. - * * @param record to be offered to kafka */ - suspend fun offer(record: ProducerRecord): OfferAck + suspend fun offer(record: ProducerRecord) /** * Publisher a [record] to Kafka, and suspends until acknowledged by kafka. @@ -61,10 +51,8 @@ interface PublishScope : CoroutineScope { * Same as [offer], but for an [Iterable]] of [ProducerRecord]. * @see offer */ - suspend fun offer(records: Iterable>): OfferAcks { - val scope = this - return OfferAcks(scope.async { records.map { offer(it).acknowledgement }.awaitAll() }) - } + suspend fun offer(records: Iterable>) = + records.map { offer(it) } /** * Same as [publish], but for an [Iterable]] of [ProducerRecord]. diff --git a/src/test/kotlin/io/github/nomisrev/kafka/publisher/KafkaPublisherSpec.kt b/src/test/kotlin/io/github/nomisrev/kafka/publisher/KafkaPublisherSpec.kt index d3909717..d127bb6f 100644 --- a/src/test/kotlin/io/github/nomisrev/kafka/publisher/KafkaPublisherSpec.kt +++ b/src/test/kotlin/io/github/nomisrev/kafka/publisher/KafkaPublisherSpec.kt @@ -95,21 +95,6 @@ class KafkaPublisherSpec : KafkaSpec({ } } - "A failed offer await is rethrow at the end" { - withTopic { topic -> - val record0 = topic.createProducerRecord(0) - val record1 = topic.createProducerRecord(1) - shouldThrow { - KafkaPublisher(publisherSettings(), stubProducer(failOnNumber = 1)).publishScope { - publish(record0) - offer(record1).acknowledgement.await() - } - } shouldBe boom - - topic.shouldHaveRecord(record0) - } - } - "An async failure is rethrow at the end" { withTopic { topic -> val count = 3 From 333887ec14c98df0c693f39f1233164d2961f8c1 Mon Sep 17 00:00:00 2001 From: Simon Vergauwen Date: Tue, 10 Oct 2023 16:55:38 +0200 Subject: [PATCH 16/16] Update README.MD, rollback OfferAcks --- README.md | 31 +++++-------- guide/example/example-readme-01.kt | 21 +++++---- guide/src/main/kotlin/main.kt | 73 ------------------------------ 3 files changed, 22 insertions(+), 103 deletions(-) delete mode 100644 guide/src/main/kotlin/main.kt diff --git a/README.md b/README.md index c296e2f4..e1b76273 100644 --- a/README.md +++ b/README.md @@ -52,19 +52,18 @@ dependencies {