[ 
https://issues.apache.org/jira/browse/KAFKA-8410?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16850204#comment-16850204
 ] 

John Roesler commented on KAFKA-8410:
-------------------------------------

Thanks for the consideration, [~guozhang]. I've created a POC so you can see 
more details about what this thing would actually look like.

One selling point is that by adding these restrictions, I surfaced one bug in 
which a ValueGetter tries to look up an incoming-type key, not the 
outgoing-type key. If this method were ever actually used, it would result in a 
class-cast exception.

I also surfaced multiple instances in which the generic type parameters in our 
DSL builder code is simply incorrect. Common mistakes include:
1. confusing the incoming and outgoing types
2. confusing V and Change<V>
3. confusing K and Windowed<K>

These aren't properly "bugs" because these generic type parameters are erased 
before run-time, and we never attempt to use incorrectly typed values, so we 
wouldn't surface any exceptions. But it's a significant impediment to reading 
and maintaining the code if we have incorrect type information on our 
variables. Similar to having an incorrect variable name (like naming a variable 
"average" when it's actually holding the count of something).

> 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
>            Priority: Major
>              Labels: tech-debt
>
> 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
(v7.6.3#76005)

Reply via email to