Class Grouped<K,​V>

    • Field Summary

      Fields 
      Modifier and Type Field Description
      protected org.apache.kafka.common.serialization.Serde<K> keySerde  
      protected java.lang.String name  
      protected org.apache.kafka.common.serialization.Serde<V> valueSerde  
    • Constructor Summary

      Constructors 
      Modifier Constructor Description
      protected Grouped​(Grouped<K,​V> grouped)  
    • Method Summary

      All Methods Static Methods Instance Methods Concrete Methods 
      Modifier and Type Method Description
      static <K,​V>
      Grouped<K,​V>
      as​(java.lang.String name)
      Create a Grouped instance with the provided name used as part of the repartition topic if required.
      static <K> Grouped keySerde​(org.apache.kafka.common.serialization.Serde<K> keySerde)
      Create a Grouped instance with the provided keySerde.
      static <V> Grouped valueSerde​(org.apache.kafka.common.serialization.Serde<V> valueSerde)
      Create a Grouped instance with the provided valueSerde.
      static <K,​V>
      Grouped<K,​V>
      with​(java.lang.String name, org.apache.kafka.common.serialization.Serde<K> keySerde, org.apache.kafka.common.serialization.Serde<V> valueSerde)
      Create a Grouped instance with the provided name, keySerde, and valueSerde.
      static <K,​V>
      Grouped<K,​V>
      with​(org.apache.kafka.common.serialization.Serde<K> keySerde, org.apache.kafka.common.serialization.Serde<V> valueSerde)
      Create a Grouped instance with the provided keySerde and valueSerde.
      Grouped<K,​V> withKeySerde​(org.apache.kafka.common.serialization.Serde<K> keySerde)
      Perform the grouping operation using the provided keySerde for serializing the key.
      Grouped<K,​V> withName​(java.lang.String name)
      Perform the grouping operation with the name for a repartition topic if required.
      Grouped<K,​V> withValueSerde​(org.apache.kafka.common.serialization.Serde<V> valueSerde)
      Perform the grouping operation using the provided valueSerde for serializing the value.
      • Methods inherited from class java.lang.Object

        clone, equals, finalize, getClass, hashCode, notify, notifyAll, toString, wait, wait, wait
    • Field Detail

      • keySerde

        protected final org.apache.kafka.common.serialization.Serde<K> keySerde
      • valueSerde

        protected final org.apache.kafka.common.serialization.Serde<V> valueSerde
      • name

        protected final java.lang.String name
    • Constructor Detail

      • Grouped

        protected Grouped​(Grouped<K,​V> grouped)
    • Method Detail

      • with

        public static <K,​V> Grouped<K,​V> with​(java.lang.String name,
                                                          org.apache.kafka.common.serialization.Serde<K> keySerde,
                                                          org.apache.kafka.common.serialization.Serde<V> valueSerde)
        Create a Grouped instance with the provided name, keySerde, and valueSerde. If the keySerde and/or the valueSerde is null the default value for the respective serde from config will be used.
        Parameters:
        name - the name used as part of the repartition topic name if required
        keySerde - the Serde used for serializing the key. If null the default key serde from config will be used
        valueSerde - the Serde used for serializing the value. If null the default value serde from config will be used
        Returns:
        a new Grouped configured with the name, keySerde, and valueSerde
        See Also:
        KStream.groupByKey(Grouped), KStream.groupBy(KeyValueMapper, Grouped), KTable.groupBy(KeyValueMapper, Grouped)
      • with

        public static <K,​V> Grouped<K,​V> with​(org.apache.kafka.common.serialization.Serde<K> keySerde,
                                                          org.apache.kafka.common.serialization.Serde<V> valueSerde)
        Create a Grouped instance with the provided keySerde and valueSerde. If the keySerde and/or the valueSerde is null the default value for the respective serde from config will be used.
        Parameters:
        keySerde - the Serde used for serializing the key. If null the default key serde from config will be used
        valueSerde - the Serde used for serializing the value. If null the default value serde from config will be used
        Returns:
        a new Grouped configured with the keySerde, and valueSerde
        See Also:
        KStream.groupByKey(Grouped), KStream.groupBy(KeyValueMapper, Grouped), KTable.groupBy(KeyValueMapper, Grouped)
      • withName

        public Grouped<K,​V> withName​(java.lang.String name)
        Perform the grouping operation with the name for a repartition topic if required. Note that Kafka Streams does not always create repartition topics for grouping operations.
        Parameters:
        name - the name used as part of the repartition topic name if required
        Returns:
        a new Grouped instance configured with the name
      • withKeySerde

        public Grouped<K,​V> withKeySerde​(org.apache.kafka.common.serialization.Serde<K> keySerde)
        Perform the grouping operation using the provided keySerde for serializing the key.
        Parameters:
        keySerde - Serde to use for serializing the key. If null the default key serde from config will be used
        Returns:
        a new Grouped instance configured with the keySerde
      • withValueSerde

        public Grouped<K,​V> withValueSerde​(org.apache.kafka.common.serialization.Serde<V> valueSerde)
        Perform the grouping operation using the provided valueSerde for serializing the value.
        Parameters:
        valueSerde - Serde to use for serializing the value. If null the default value serde from config will be used
        Returns:
        a new Grouped instance configured with the valueSerde