Not with the scope of the current discussion.

So far, we discuss to add `RecordContext`, but the context object we use
could also provide some more metadata. I see no reason why not to expose
the node name there. We already expose TaskId vie `ProcessorContext`. We
could also add thread name. IMHO, this would be better than dictating
any prefix.

Thoughts?


-Matthias

On 6/4/17 9:03 PM, Guozhang Wang wrote:
> Matthias,
> 
> I think even with KIP-159 users would not be able to access the processor
> node name right?
> 
> Guozhang
> 
> On Thu, Jun 1, 2017 at 10:28 PM, Matthias J. Sax <matth...@confluent.io>
> wrote:
> 
>> Thanks for the KIP.
>>
>> Two comments:
>>  - I think we should include #writeAsText()
>>  - I am not sure if we should use
>>
>>> "[" + this.streamName + "]: " + mapper.apply(keyToPrint, valueToPrint)
>>
>> in case a mapper is provided. This still dictates a fixed prefix a user
>> might not want to have (what contradicts or at least limits the scope of
>> this new functionality). Considering he current discussion of KIP-159, a
>> user would be able to access the stream name within the provided mapper
>> and add it if they wish anyway, and thus, I don't think we should force
>> this format.
>>
>>
>>
>> -Matthias
>>
>>
>>
>> On 5/30/17 1:38 PM, Guozhang Wang wrote:
>>> Overall +1. One comment about the wiki itself:
>>>
>>> Could you replace the general description of "Argument KStream.print()
>> which
>>> is KStream.print(KeyValueMapper<K, V, String>)" with the actual added
>>> overloaded functions in the wiki page?
>>>
>>>
>>> Guozhang
>>>
>>> On Mon, May 22, 2017 at 12:21 AM, James Chain <james.chain1...@gmail.com
>>>
>>> wrote:
>>>
>>>> Hi All,
>>>>
>>>> I want to start this KIP to argument KStream.print().
>>>> This vote is already started.
>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>>>> 160+-+Augment+KStream.print%28%29+to+allow+users+pass+in+
>>>> extra+parameters+in+the+printed+string
>>>>
>>>> Thanks,
>>>>
>>>> James Chien
>>>>
>>>
>>>
>>>
>>
>>
> 
> 

Attachment: signature.asc
Description: OpenPGP digital signature

Reply via email to