[ 
https://issues.apache.org/jira/browse/KAFKA-8410?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Matthias J. Sax updated KAFKA-8410:
-----------------------------------
    Description: 
KIP-478: 
[https://cwiki.apache.org/confluence/display/KAFKA/KIP-478+-+Strongly+typed+Processor+API]
 

Presently, it's very difficult to have confidence when adding to or modifying 
processors in the DSL. There's a lot of raw types, duck-typing, and casting 
that contribute to this problem.

The root, though, is that the generic types on `Processor<K,V>` refer only to 
the _input_ key and value types. No information is captured or verified about 
what the _output_ types of a processor are. For example, this leads to 
widespread confusion in the code base about whether a processor produces `V`s 
or `Change<V>`s. The type system actually makes matters worse, since we use 
casts to make the processors conform to declared types that are in fact wrong, 
but are never checked due to erasure.

We can start to make some headway on this tech debt by adding some types to the 
ProcessorContext that bound the `<K,V>` that may be passed to 
`context.forward`. Then, we can build on this by fully specifying the input and 
output types of the Processors, which in turn would let us eliminate the 
majority of unchecked casts in the DSL operators.

I'm not sure whether adding these generic types to the existing 
ProcessorContext and Processor interfaces, which would also affect the PAPI has 
any utility, or whether we should make this purely an internal change by 
introducing GenericProcessorContext and GenericProcessor peer interfaces for 
the DSL to use.

  was:
Presently, it's very difficult to have confidence when adding to or modifying 
processors in the DSL. There's a lot of raw types, duck-typing, and casting 
that contribute to this problem.

The root, though, is that the generic types on `Processor<K,V>` refer only to 
the _input_ key and value types. No information is captured or verified about 
what the _output_ types of a processor are. For example, this leads to 
widespread confusion in the code base about whether a processor produces `V`s 
or `Change<V>`s. The type system actually makes matters worse, since we use 
casts to make the processors conform to declared types that are in fact wrong, 
but are never checked due to erasure.

We can start to make some headway on this tech debt by adding some types to the 
ProcessorContext that bound the `<K,V>` that may be passed to 
`context.forward`. Then, we can build on this by fully specifying the input and 
output types of the Processors, which in turn would let us eliminate the 
majority of unchecked casts in the DSL operators.

I'm not sure whether adding these generic types to the existing 
ProcessorContext and Processor interfaces, which would also affect the PAPI has 
any utility, or whether we should make this purely an internal change by 
introducing GenericProcessorContext and GenericProcessor peer interfaces for 
the DSL to use.


> Strengthen the types of Processors, at least in the DSL, maybe in the PAPI as 
> well
> ----------------------------------------------------------------------------------
>
>                 Key: KAFKA-8410
>                 URL: https://issues.apache.org/jira/browse/KAFKA-8410
>             Project: Kafka
>          Issue Type: Improvement
>          Components: streams
>            Reporter: John Roesler
>            Assignee: John Roesler
>            Priority: Major
>              Labels: tech-debt
>             Fix For: 2.7.0
>
>
> KIP-478: 
> [https://cwiki.apache.org/confluence/display/KAFKA/KIP-478+-+Strongly+typed+Processor+API]
>  
> Presently, it's very difficult to have confidence when adding to or modifying 
> processors in the DSL. There's a lot of raw types, duck-typing, and casting 
> that contribute to this problem.
> The root, though, is that the generic types on `Processor<K,V>` refer only to 
> the _input_ key and value types. No information is captured or verified about 
> what the _output_ types of a processor are. For example, this leads to 
> widespread confusion in the code base about whether a processor produces `V`s 
> or `Change<V>`s. The type system actually makes matters worse, since we use 
> casts to make the processors conform to declared types that are in fact 
> wrong, but are never checked due to erasure.
> We can start to make some headway on this tech debt by adding some types to 
> the ProcessorContext that bound the `<K,V>` that may be passed to 
> `context.forward`. Then, we can build on this by fully specifying the input 
> and output types of the Processors, which in turn would let us eliminate the 
> majority of unchecked casts in the DSL operators.
> I'm not sure whether adding these generic types to the existing 
> ProcessorContext and Processor interfaces, which would also affect the PAPI 
> has any utility, or whether we should make this purely an internal change by 
> introducing GenericProcessorContext and GenericProcessor peer interfaces for 
> the DSL to use.



--
This message was sent by Atlassian Jira
(v8.20.1#820001)

Reply via email to