Uploaded image for project: 'Kafka'
  1. Kafka
  2. KAFKA-10417

suppress() with cogroup() throws ClassCastException

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Critical
    • Resolution: Fixed
    • 2.6.0
    • 2.8.0, 2.7.1
    • streams
    • Important

    Description

      Streams operation - `cogroup()`, `aggregate()` followed by `suppress()` throws `ClassCastException`

      Works fine without the `suppress()`

      Code block tested -

      val stream1 = requestStreams.merge(successStreams).merge(errorStreams)
                      .groupByKey(Grouped.with(Serdes.String(), serdesConfig.notificationSerde()))
      
              val streams2 = confirmationStreams
                      .groupByKey(Grouped.with(Serdes.String(), serdesConfig.confirmationsSerde()))
      
              val cogrouped = stream1.cogroup(notificationAggregator).cogroup(streams2, confirmationsAggregator)
                      .windowedBy(TimeWindows.of(Duration.ofMinutes(notificationStreamsConfig.joinWindowMinutes.toLong())).grace(Duration.ofMinutes(notificationStreamsConfig.graceDurationMinutes.toLong())))
                      .aggregate({ null }, Materialized.`as`<String, NotificationMetric, WindowStore<Bytes, ByteArray>>("time-windowed-aggregated-stream-store")
                              .withValueSerde(serdesConfig.notificationMetricSerde()))
                       .suppress(Suppressed.untilWindowCloses(unbounded()))
                      .toStream()
      
      

      Exception thrown is:

      Caused by: java.lang.ClassCastException: class org.apache.kafka.streams.kstream.internals.PassThrough cannot be cast to class org.apache.kafka.streams.kstream.internals.KTableProcessorSupplier (org.apache.kafka.streams.kstream.internals.PassThrough and org.apache.kafka.streams.kstream.internals.KTableProcessorSupplier are in unnamed module of loader 'app')
      

      https://stackoverflow.com/questions/63459685/kgroupedstream-with-cogroup-aggregate-suppress

      Attachments

        Issue Links

          Activity

            People

              lct45 Leah Thomas
              wardhapk Wardha Perinkada Kattu
              Votes:
              0 Vote for this issue
              Watchers:
              8 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: