Skip to content

[NU-2021] Improved Kafka metadata caching: common cache and caching topics when schemaless topics are enabled #8116

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Merged
merged 21 commits into from
May 30, 2025
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -208,6 +208,10 @@ class SchemalessKafkaJsonTypeTests
"scenarioTypes.streaming.modelConfig.components.kafka.config.useDataSampleParamForSchemalessJsonTopicBasedKafkaSource",
ConfigValueFactory.fromAnyRef(true)
)
.withValue(
"scenarioTypes.streaming.modelConfig.components.kafka.config.kafkaAdminConfig.cacheConfig.topicsExpirationTime",
ConfigValueFactory.fromAnyRef(0)
)

lazy val defaultKafkaConfig: KafkaConfig = KafkaConfig(
kafkaProperties = Some(Map("bootstrap.servers" -> kafkaContainer.bootstrapServers)),
Expand Down
1 change: 1 addition & 0 deletions docs/Changelog.md
Original file line number Diff line number Diff line change
Expand Up @@ -189,6 +189,7 @@ description: Stay informed with detailed changelogs covering new features, impro
}
```
* [#7982](https://github.com/TouK/nussknacker/pull/7982) Mock expression added to enrichers (except decision-table) which can be used to hardcode enricher output in tests without calling external services.
* [#8116](https://github.com/TouK/nussknacker/pull/8116) Improved Kafka metadata caching: common cache and caching topics when schemaless topics are enabled
## 1.18
Expand Down
9 changes: 9 additions & 0 deletions docs/MigrationGuide.md
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,15 @@ To see the biggest differences please consult the [changelog](Changelog.md).
should be added to distinguish the components. In the new version of the component, OpenAPI `Service` should be
selected. For this reason, the existing scenario may not compile. The automatic migration for this parameter is not
easy and has to be done manually or through updates in the DB if you can read the swagger service name.
* [#8116](https://github.com/TouK/nussknacker/pull/8116) Improved Kafka metadata caching
* Dependencies for `TopicSelectionStrategy` (`SchemaRegistryClient`, and `KafkaConfig`) should be passed via constructor.
* `kafkaConfig.topicsExistenceValidationConfig.validatorConfig` and `kafkaConfig.topicsWithoutSchemaFetchTimeout` were moved to
common Kafka admin configuration `kafkaConfig.kafkaAdminConfig` - see [Kafka integration](integration/KafkaIntegration.md)
for more details. The properties were moved as follows:
* `topicsExistenceValidationConfig.validatorConfig.autoCreateFlagFetchCacheTtl` -> `kafkaAdminConfig.cacheConfig.autoCreateTopicSettingExpirationTime`
* `topicsExistenceValidationConfig.validatorConfig.topicsFetchCacheTtl` -> `kafkaAdminConfig.cacheConfig.topicsExpirationTime`
* `topicsExistenceValidationConfig.validatorConfig.adminClientTimeout` -> `kafkaAdminConfig.clientTimeout`
* `topicsWithoutSchemaFetchTimeout` -> `kafkaAdminConfig.clientTimeout`

### REST API changes

Expand Down
49 changes: 24 additions & 25 deletions docs/integration/KafkaIntegration.md

Large diffs are not rendered by default.

Original file line number Diff line number Diff line change
Expand Up @@ -24,8 +24,8 @@ class TopicSelectionStrategySpec extends KafkaAvroSpecMixin with KafkaAvroSource
private lazy val confluentClient = schemaRegistryClientFactory.create(kafkaConfig)

test("all topic strategy test") {
val strategy = new TopicsWithExistingSubjectSelectionStrategy()
strategy.getTopics(confluentClient, kafkaConfig).toList.map(_.toSet) shouldBe List(
val strategy = new TopicsWithExistingSubjectSelectionStrategy(confluentClient)
strategy.getTopics.toList.map(_.toSet) shouldBe List(
Set(
RecordTopic,
RecordTopicWithKey,
Expand All @@ -40,8 +40,9 @@ class TopicSelectionStrategySpec extends KafkaAvroSpecMixin with KafkaAvroSource
}

test("topic filtering strategy test") {
val strategy = new TopicsMatchingPatternWithExistingSubjectsSelectionStrategy(Pattern.compile(".*Record.*"))
strategy.getTopics(confluentClient, kafkaConfig).toList shouldBe List(
val strategy =
new TopicsMatchingPatternWithExistingSubjectsSelectionStrategy(Pattern.compile(".*Record.*"), confluentClient)
strategy.getTopics.toList shouldBe List(
List(ArrayOfRecordsTopic, RecordTopic, RecordTopicWithKey)
)
}
Expand All @@ -53,8 +54,8 @@ class TopicSelectionStrategySpec extends KafkaAvroSpecMixin with KafkaAvroSource
testModelConfig,
new FlinkKafkaSourceImplFactory(None)
) {
override def topicSelectionStrategy =
new TopicsMatchingPatternWithExistingSubjectsSelectionStrategy(Pattern.compile("test-.*"))
override lazy val topicSelectionStrategy =
new TopicsMatchingPatternWithExistingSubjectsSelectionStrategy(Pattern.compile("test-.*"), confluentClient)
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -141,6 +141,7 @@ abstract class FlinkWithKafkaSuite
s"config.kafkaEspProperties.${AvroSerializersRegistrar.autoRegisterRecordSchemaIdSerializationProperty}",
fromAnyRef(false)
)
.withValue("config.kafkaAdminConfig.cacheConfig.topicsExpirationTime", fromAnyRef(0))
maybeAddSchemaRegistryUrl(config)
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,32 +20,32 @@ class CachedTopicsExistenceValidatorWhenAutoCreateDisabledTest

test("should validate existing source topic") {
val topic = createUniqueSourceTopic()
val validator = new CachedTopicsExistenceValidator(defaultKafkaConfig)
val validator = CachedTopicsExistenceValidator(defaultKafkaConfig)
validator.validateTopic(topic) shouldBe Valid(topic)
}

test("should validate not existing source topic") {
val validator = new CachedTopicsExistenceValidator(defaultKafkaConfig)
val validator = CachedTopicsExistenceValidator(defaultKafkaConfig)
validator.validateTopic(notExistingSourceTopic) shouldBe Invalid(
TopicExistenceValidationException(NonEmptyList.one(notExistingSourceTopic))
)
}

test("should validate existing sink topic") {
val topic = createUniqueSinkTopic()
val validator = new CachedTopicsExistenceValidator(defaultKafkaConfig)
val validator = CachedTopicsExistenceValidator(defaultKafkaConfig)
validator.validateTopic(topic) shouldBe Valid(topic)
}

test("should validate not existing sink topic") {
val validator = new CachedTopicsExistenceValidator(defaultKafkaConfig)
val validator = CachedTopicsExistenceValidator(defaultKafkaConfig)
validator.validateTopic(notExistingSinkTopic) shouldBe Invalid(
TopicExistenceValidationException(NonEmptyList.one(notExistingSinkTopic))
)
}

test("should not validate not existing topic when validation disabled") {
val validator = new CachedTopicsExistenceValidator(
val validator = CachedTopicsExistenceValidator(
defaultKafkaConfig.copy(
topicsExistenceValidationConfig = TopicsExistenceValidationConfig(enabled = false)
)
Expand All @@ -55,7 +55,7 @@ class CachedTopicsExistenceValidatorWhenAutoCreateDisabledTest

test("should fetch topics every time when not valid using cache") {
val notExistingYetTopicName = createUniqueSourceTopicName()
val validator = new CachedTopicsExistenceValidator(defaultKafkaConfig)
val validator = CachedTopicsExistenceValidator(defaultKafkaConfig)

validator.validateTopic(notExistingYetTopicName) shouldBe Invalid(
TopicExistenceValidationException(NonEmptyList.one(notExistingYetTopicName))
Expand All @@ -75,30 +75,30 @@ class CachedTopicsExistenceValidatorWhenAutoCreateEnabledTest

test("should validate existing source topic") {
val topic = createUniqueSourceTopic()
val validator = new CachedTopicsExistenceValidator(defaultKafkaConfig)
val validator = CachedTopicsExistenceValidator(defaultKafkaConfig)
validator.validateTopic(topic) shouldBe Valid(topic)
}

test("should not validate not existing source topic") {
val validator = new CachedTopicsExistenceValidator(defaultKafkaConfig)
val validator = CachedTopicsExistenceValidator(defaultKafkaConfig)
validator.validateTopic(notExistingSourceTopic) shouldBe Invalid(
TopicExistenceValidationException(NonEmptyList.one(notExistingSourceTopic))
)
}

test("should validate existing sink topic") {
val topic = createUniqueSinkTopic()
val validator = new CachedTopicsExistenceValidator(defaultKafkaConfig)
val validator = CachedTopicsExistenceValidator(defaultKafkaConfig)
validator.validateTopic(topic) shouldBe Valid(topic)
}

test("should validate not existing sink topic") {
val validator = new CachedTopicsExistenceValidator(defaultKafkaConfig)
val validator = CachedTopicsExistenceValidator(defaultKafkaConfig)
validator.validateTopic(notExistingSinkTopic) shouldBe Valid(notExistingSinkTopic)
}

test("should not validate not existing topic when validation disabled") {
val validator = new CachedTopicsExistenceValidator(
val validator = CachedTopicsExistenceValidator(
defaultKafkaConfig.copy(
topicsExistenceValidationConfig = TopicsExistenceValidationConfig(enabled = false)
)
Expand All @@ -107,7 +107,7 @@ class CachedTopicsExistenceValidatorWhenAutoCreateEnabledTest
}

test("should use cache when validating") {
val validator = new CachedTopicsExistenceValidator(defaultKafkaConfig)
val validator = CachedTopicsExistenceValidator(defaultKafkaConfig)
validator.validateTopic(notExistingSinkTopic) shouldBe Valid(notExistingSinkTopic)
container.stop()
validator.validateTopic(notExistingSinkTopic) shouldBe Valid(notExistingSinkTopic)
Expand All @@ -134,7 +134,6 @@ abstract class BaseCachedTopicsExistenceValidatorTest(kafkaAutoCreateEnabled: Bo
// longer timeout, as container might need some time to make initial assignments etc.
topicsExistenceValidationConfig = TopicsExistenceValidationConfig(
enabled = true,
validatorConfig = CachedTopicsExistenceValidatorConfig.DefaultConfig.copy(adminClientTimeout = 30 seconds)
)
)

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -12,7 +12,7 @@ object KafkaComponentsUtils extends KafkaUtils {
topics: NonEmptyList[PreparedKafkaTopic[T]],
kafkaConfig: KafkaConfig
): Unit = {
new CachedTopicsExistenceValidator(kafkaConfig = kafkaConfig)
CachedTopicsExistenceValidator(kafkaConfig)
.validateTopics(topics.map(_.prepared))
.valueOr(err => throw err)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@ import pl.touk.nussknacker.engine.kafka._
import pl.touk.nussknacker.engine.kafka.KafkaFactory.TopicParamName
import pl.touk.nussknacker.engine.kafka.serialization.{KafkaDeserializationSchema, KafkaDeserializationSchemaFactory}
import pl.touk.nussknacker.engine.kafka.source.KafkaSourceFactory._
import pl.touk.nussknacker.engine.kafka.validator.WithCachedTopicsExistenceValidator
import pl.touk.nussknacker.engine.kafka.validator.CachedTopicsExistenceValidator

import scala.reflect.ClassTag

Expand All @@ -45,7 +45,6 @@ class KafkaSourceFactory[K: ClassTag, V: ClassTag](
protected val implProvider: KafkaSourceImplFactory[K, V]
) extends SourceFactory
with SingleInputDynamicComponent[Source]
with WithCachedTopicsExistenceValidator
with WithExplicitTypesToExtract
with UnboundedStreamComponent {

Expand Down Expand Up @@ -78,7 +77,11 @@ class KafkaSourceFactory[K: ClassTag, V: ClassTag](
)(implicit nodeId: NodeId): List[ProcessCompilationError.CustomNodeError] = {
val topics = topicNamesFrom(topicsString)
val preparedTopics = topics.map(KafkaComponentsUtils.prepareKafkaTopic(_, modelConfig)).map(_.prepared)
validateTopics(preparedTopics).swap.toList.map(_.toCustomNodeError(nodeId.id, Some(TopicParamName)))
cachedTopicsExistenceValidator
.validateTopics(preparedTopics)
.swap
.toList
.map(_.toCustomNodeError(nodeId.id, Some(TopicParamName)))
}

protected def nextSteps(context: ValidationContext, dependencies: List[NodeDependencyValue])(
Expand Down Expand Up @@ -199,7 +202,9 @@ class KafkaSourceFactory[K: ClassTag, V: ClassTag](
override def nodeDependencies: List[NodeDependency] =
List(TypedNodeDependency[MetaData], TypedNodeDependency[NodeId], OutputVariableNameDependency)

override protected val kafkaConfig: KafkaConfig = KafkaConfig.parseConfig(modelConfig.underlyingConfig)
protected val kafkaConfig: KafkaConfig = KafkaConfig.parseConfig(modelConfig.underlyingConfig)

protected lazy val cachedTopicsExistenceValidator = CachedTopicsExistenceValidator(kafkaConfig)

private def topicNamesFrom(value: String) = {
val topicsList = value.split(topicNameSeparator).map(_.trim).map(TopicName.ForSource.apply).toList
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -3,42 +3,25 @@ package pl.touk.nussknacker.engine.kafka.validator
import cats.data.{NonEmptyList, Validated}
import cats.data.Validated.{Invalid, Valid}
import com.typesafe.scalalogging.LazyLogging
import org.apache.kafka.clients.admin.{Admin, DescribeClusterOptions, DescribeConfigsOptions, ListTopicsOptions}
import org.apache.kafka.common.config.ConfigResource
import pl.touk.nussknacker.engine.api.process.TopicName
import pl.touk.nussknacker.engine.kafka.{KafkaConfig, KafkaUtils, UnspecializedTopicName}
import pl.touk.nussknacker.engine.kafka.{CachingKafkaAdminClient, KafkaConfig, KafkaUtils, UnspecializedTopicName}
import pl.touk.nussknacker.engine.kafka.UnspecializedTopicName.ToUnspecializedTopicName
import pl.touk.nussknacker.engine.kafka.validator.TopicsExistenceValidator.TopicValidationType
import pl.touk.nussknacker.engine.util.cache.SingleValueCache

import scala.jdk.CollectionConverters._
object CachedTopicsExistenceValidator {

trait WithCachedTopicsExistenceValidator extends TopicsExistenceValidator {

protected val kafkaConfig: KafkaConfig

private lazy val validator = new CachedTopicsExistenceValidator(kafkaConfig)

final override def validateTopics[T <: TopicName: TopicValidationType](
topics: NonEmptyList[T]
): Validated[TopicExistenceValidationException[T], NonEmptyList[T]] =
validator.validateTopics(topics)
def apply(kafkaConfig: KafkaConfig): CachedTopicsExistenceValidator = {
new CachedTopicsExistenceValidator(
kafkaConfig.topicsExistenceValidationConfig.enabled,
KafkaUtils.createCachingAdminClient(kafkaConfig)
)
}

}

class CachedTopicsExistenceValidator(kafkaConfig: KafkaConfig) extends TopicsExistenceValidator with LazyLogging {

private val validatorConfig = kafkaConfig.topicsExistenceValidationConfig.validatorConfig

@transient private lazy val autoCreateSettingCache = new SingleValueCache[Boolean](
expireAfterAccess = None,
expireAfterWrite = Some(validatorConfig.autoCreateFlagFetchCacheTtl)
)

@transient private lazy val topicsCache = new SingleValueCache[Set[UnspecializedTopicName]](
expireAfterAccess = None,
expireAfterWrite = Some(validatorConfig.topicsFetchCacheTtl)
)
class CachedTopicsExistenceValidator(enabled: Boolean, cachingKafkaAdminClient: CachingKafkaAdminClient)
extends TopicsExistenceValidator
with LazyLogging {

def validateTopics[T <: TopicName: TopicValidationType](
topics: NonEmptyList[T]
Expand All @@ -50,26 +33,30 @@ class CachedTopicsExistenceValidator(kafkaConfig: KafkaConfig) extends TopicsExi
}

private def validateSourceTopics[T <: TopicName: TopicValidationType](topics: NonEmptyList[T]) = {
if (kafkaConfig.topicsExistenceValidationConfig.enabled) {
if (enabled) {
doValidate(topics)
} else {
Valid(topics)
}
}

private def validateSinkTopics[T <: TopicName: TopicValidationType](topics: NonEmptyList[T]) = {
if (kafkaConfig.topicsExistenceValidationConfig.enabled && !isAutoCreateEnabled) {
if (enabled && !cachingKafkaAdminClient.getOrFetchAutoCreateTopicsSetting) {
doValidate(topics)
} else {
Valid(topics)
}
}

private def doValidate[T <: TopicName](topics: NonEmptyList[T]) = {
topicsCache.get() match {
cachingKafkaAdminClient.getCachedTopics match {
case Some(cachedTopics) if doAllExist(topics, cachedTopics).isRight =>
Valid(topics)
case Some(_) | None =>
case Some(_) =>
logger.debug("Topics do not exist, fetch topics from Kafka and validate")
fetchTopicsAndValidate(topics)
case None =>
logger.debug("Empty topics cache, fetch topics from Kafka and validate")
fetchTopicsAndValidate(topics)
}
}
Expand All @@ -86,47 +73,11 @@ class CachedTopicsExistenceValidator(kafkaConfig: KafkaConfig) extends TopicsExi
}

private def fetchTopicsAndValidate[T <: TopicName](requestedTopics: NonEmptyList[T]) = {
val existingTopics = fetchAllTopicsAndCache()
val existingTopics = cachingKafkaAdminClient.fetchFreshTopicsAndCache
doAllExist(requestedTopics, existingTopics) match {
case Right(()) => Valid(requestedTopics)
case Left(notExistingTopics) => Invalid(TopicExistenceValidationException(notExistingTopics))
}
}

private def fetchAllTopicsAndCache() = {
val existingTopics = usingAdminClient {
_.listTopics(new ListTopicsOptions().timeoutMs(validatorConfig.adminClientTimeout.toMillis.toInt))
.names()
.get()
.asScala
.toSet
.map(UnspecializedTopicName.apply)
}
topicsCache.put(existingTopics)
existingTopics
}

private def isAutoCreateEnabled: Boolean = autoCreateSettingCache.getOrCreate {
val timeout = validatorConfig.adminClientTimeout.toMillis.toInt
val randomKafkaNodeId = usingAdminClient {
_.describeCluster(new DescribeClusterOptions().timeoutMs(timeout)).nodes().get().asScala.head.id().toString
}
usingAdminClient {
_.describeConfigs(
List(new ConfigResource(ConfigResource.Type.BROKER, randomKafkaNodeId)).asJava,
new DescribeConfigsOptions().timeoutMs(validatorConfig.adminClientTimeout.toMillis.toInt)
)
.values()
.values()
.asScala
.map(_.get())
.head // we ask for config of one node, but `describeConfigs` api have `List` of nodes, so here we got single element list
.get("auto.create.topics.enable")
.value()
.toBoolean
}
}

private def usingAdminClient[T]: (Admin => T) => T = KafkaUtils.usingAdminClient[T](kafkaConfig)

}
Loading