Interface ProcessorContext

All Known Implementing Classes:
MockProcessorContext

public interface ProcessorContext
Processor context interface.
  • Method Details

    • applicationId

      String applicationId()
      Return the application id.
      Returns:
      the application id
    • taskId

      TaskId taskId()
      Return the task id.
      Returns:
      the task id
    • keySerde

      Serde<?> keySerde()
      Return the default key serde.
      Returns:
      the key serializer
    • valueSerde

      Serde<?> valueSerde()
      Return the default value serde.
      Returns:
      the value serializer
    • stateDir

      File stateDir()
      Return the state directory for the partition.
      Returns:
      the state directory
    • metrics

      StreamsMetrics metrics()
      Return Metrics instance.
      Returns:
      StreamsMetrics
    • register

      void register(StateStore store, StateRestoreCallback stateRestoreCallback)
      Register and possibly restores the specified storage engine.
      Parameters:
      store - the storage engine
      stateRestoreCallback - the restoration callback logic for log-backed state stores upon restart
      Throws:
      IllegalStateException - If store gets registered after initialized is already finished
      StreamsException - if the store's change log does not contain the partition
    • getStateStore

      <S extends StateStore> S getStateStore(String name)
      Get the state store given the store name.
      Type Parameters:
      S - The type or interface of the store to return
      Parameters:
      name - The store name
      Returns:
      The state store instance
      Throws:
      ClassCastException - if the return type isn't a type or interface of the actual returned store.
    • schedule

      Cancellable schedule(Duration interval, PunctuationType type, Punctuator callback)
      Schedule a periodic operation for processors. A processor may call this method during initialization or processing to schedule a periodic callback — called a punctuation — to Punctuator.punctuate(long). The type parameter controls what notion of time is used for punctuation:
      • PunctuationType.STREAM_TIME — uses "stream time", which is advanced by the processing of messages in accordance with the timestamp as extracted by the TimestampExtractor in use. The first punctuation will be triggered by the first record that is processed. NOTE: Only advanced if messages arrive
      • PunctuationType.WALL_CLOCK_TIME — uses system time (the wall-clock time), which is advanced independent of whether new messages arrive. The first punctuation will be triggered after interval has elapsed. NOTE: This is best effort only as its granularity is limited by how long an iteration of the processing loop takes to complete
      Skipping punctuations: Punctuations will not be triggered more than once at any given timestamp. This means that "missed" punctuation will be skipped. It's possible to "miss" a punctuation if:
      Parameters:
      interval - the time interval between punctuations (supported minimum is 1 millisecond)
      type - one of: PunctuationType.STREAM_TIME, PunctuationType.WALL_CLOCK_TIME
      callback - a function consuming timestamps representing the current stream or system time
      Returns:
      a handle allowing cancellation of the punctuation schedule established by this method
      Throws:
      IllegalArgumentException - if the interval is not representable in milliseconds
    • forward

      <K, V> void forward(K key, V value)
      Forward a key/value pair to all downstream processors. Used the input record's timestamp as timestamp for the output record.

      If this method is called with Punctuator.punctuate(long) the record that is sent downstream won't have any associated record metadata like topic, partition, or offset.

      Parameters:
      key - key
      value - value
    • forward

      <K, V> void forward(K key, V value, To to)
      Forward a key/value pair to the specified downstream processors. Can be used to set the timestamp of the output record.

      If this method is called with Punctuator.punctuate(long) the record that is sent downstream won't have any associated record metadata like topic, partition, or offset.

      Parameters:
      key - key
      value - value
      to - the options to use when forwarding
    • commit

      void commit()
      Request a commit.
    • topic

      String topic()
      Return the topic name of the current input record; could be null if it is not available.

      For example, if this method is invoked within a punctuation callback, or while processing a record that was forwarded by a punctuation callback, the record won't have an associated topic. Another example is KTable.transformValues(ValueTransformerWithKeySupplier, String...) (and siblings), that do not always guarantee to provide a valid topic name, as they might be executed "out-of-band" due to some internal optimizations applied by the Kafka Streams DSL.

      Returns:
      the topic name
    • partition

      int partition()
      Return the partition id of the current input record; could be -1 if it is not available.

      For example, if this method is invoked within a punctuation callback, or while processing a record that was forwarded by a punctuation callback, the record won't have an associated partition id. Another example is KTable.transformValues(ValueTransformerWithKeySupplier, String...) (and siblings), that do not always guarantee to provide a valid partition id, as they might be executed "out-of-band" due to some internal optimizations applied by the Kafka Streams DSL.

      Returns:
      the partition id
    • offset

      long offset()
      Return the offset of the current input record; could be -1 if it is not available.

      For example, if this method is invoked within a punctuation callback, or while processing a record that was forwarded by a punctuation callback, the record won't have an associated offset. Another example is KTable.transformValues(ValueTransformerWithKeySupplier, String...) (and siblings), that do not always guarantee to provide a valid offset, as they might be executed "out-of-band" due to some internal optimizations applied by the Kafka Streams DSL.

      Returns:
      the offset
    • headers

      Headers headers()
      Return the headers of the current input record; could be an empty header if it is not available.

      For example, if this method is invoked within a punctuation callback, or while processing a record that was forwarded by a punctuation callback, the record might not have any associated headers. Another example is KTable.transformValues(ValueTransformerWithKeySupplier, String...) (and siblings), that do not always guarantee to provide valid headers, as they might be executed "out-of-band" due to some internal optimizations applied by the Kafka Streams DSL.

      Returns:
      the headers
    • timestamp

      long timestamp()
      Return the current timestamp.

      If it is triggered while processing a record streamed from the source processor, timestamp is defined as the timestamp of the current input record; the timestamp is extracted from ConsumerRecord by TimestampExtractor. Note, that an upstream Processor might have set a new timestamp by calling forward(..., To.all().withTimestamp(...)). In particular, some Kafka Streams DSL operators set result record timestamps explicitly, to guarantee deterministic results.

      If it is triggered while processing a record generated not from the source processor (for example, if this method is invoked from the punctuate call), timestamp is defined as the current task's stream time, which is defined as the largest timestamp of any record processed by the task.

      Returns:
      the timestamp
    • appConfigs

      Map<String,Object> appConfigs()
      Return all the application config properties as key/value pairs.

      The config properties are defined in the StreamsConfig object and associated to the ProcessorContext.

      The type of the values is dependent on the type of the property (e.g. the value of DEFAULT_KEY_SERDE_CLASS_CONFIG will be of type Class, even if it was specified as a String to StreamsConfig(Map)).

      Returns:
      all the key/values from the StreamsConfig properties
    • appConfigsWithPrefix

      Map<String,Object> appConfigsWithPrefix(String prefix)
      Return all the application config properties with the given key prefix, as key/value pairs stripping the prefix.

      The config properties are defined in the StreamsConfig object and associated to the ProcessorContext.

      Parameters:
      prefix - the properties prefix
      Returns:
      the key/values matching the given prefix from the StreamsConfig properties.
    • currentSystemTimeMs

      long currentSystemTimeMs()
      Return the current system timestamp (also called wall-clock time) in milliseconds.

      Note: this method returns the internally cached system timestamp from the Kafka Stream runtime. Thus, it may return a different value compared to System.currentTimeMillis().

      Returns:
      the current system timestamp in milliseconds
    • currentStreamTimeMs

      long currentStreamTimeMs()
      Return the current stream-time in milliseconds.

      Stream-time is the maximum observed record timestamp so far (including the currently processed record), i.e., it can be considered a high-watermark. Stream-time is tracked on a per-task basis and is preserved across restarts and during task migration.

      Note: this method is not supported for global processors (cf. Topology.addGlobalStore(org.apache.kafka.streams.state.StoreBuilder<?>, java.lang.String, org.apache.kafka.common.serialization.Deserializer<K>, org.apache.kafka.common.serialization.Deserializer<V>, java.lang.String, java.lang.String, org.apache.kafka.streams.processor.ProcessorSupplier<K, V>) (...) and StreamsBuilder.addGlobalStore(org.apache.kafka.streams.state.StoreBuilder<?>, java.lang.String, org.apache.kafka.streams.kstream.Consumed<K, V>, org.apache.kafka.streams.processor.ProcessorSupplier<K, V>) (...), because there is no concept of stream-time for this case. Calling this method in a global processor will result in an UnsupportedOperationException.

      Returns:
      the current stream-time in milliseconds