[jira] [Created] (KAFKA-7434) DeadLetterQueueReporter throws NPE if transform throws NPE

2018-09-24 Thread Michal Borowiecki (JIRA)
Michal Borowiecki created KAFKA-7434: Summary: DeadLetterQueueReporter throws NPE if transform throws NPE Key: KAFKA-7434 URL: https://issues.apache.org/jira/browse/KAFKA-7434 Project: Kafka

Re: [VOTE] KIP-138: Change punctuate semantics

2017-08-31 Thread Michal Borowiecki
at 8:13 AM, Michal Borowiecki < michal.borowie...@openbet.com> wrote: Thank you all! This KIP passed the vote with 3 binding and 5 non-binding +1s: +1 (binding) from Guozhang Wang, Ismael Juma and Ewen Cheslack-Postava +1 (non-binding) from Matthias J. Sax, Bill Bejeck, Eno Thereska, Arun M

[jira] [Created] (KAFKA-5677) Remove deprecated punctuate method

2017-07-29 Thread Michal Borowiecki (JIRA)
Michal Borowiecki created KAFKA-5677: Summary: Remove deprecated punctuate method Key: KAFKA-5677 URL: https://issues.apache.org/jira/browse/KAFKA-5677 Project: Kafka Issue Type: Task

Re: [DISCUSS] 2017 October release planning and release version

2017-07-21 Thread Michal Borowiecki
s mean that _downgrading_ may cause loss of functionality. > > That's > > > > OK, > > > > > in my opinion. > > > > > > > > > > Change introduced contrary to the SLO, is OK to be reported as bug. > &

Re: [DISCUSS] KIP-147: Add missing type parameters to StateStoreSupplier factories and KGroupedStream/Table methods

2017-06-24 Thread Michal Borowiecki
I think the discussion on Streams DSL refactoring will render this KIP obsolete. I'll leave is as under discussion until something is agreed and then move it to discarded. Cheers, Michał On 03/06/17 10:02, Michal Borowiecki wrote: I agree maintaining backwards-compatibility here a

Re: Re: [DISCUSS] KIP-165: Extend Interactive Queries for return latest update timestamp per key

2017-06-24 Thread Michal Borowiecki
%3A+Extend+Interactive+Queries+for+return+latest+ update+timestamp+per+key [2] https://issues.apache.org/jira/browse/KAFKA-4304 Cheers, Jeyhun -- Signature <http://www.openbet.com/> Michal Borowiecki Senior Software Engineer L4 T: +44 208 742 1600

Re: [DISCUSS] KIP-163: Lower the Minimum Required ACL Permission of OffsetFetch

2017-06-20 Thread Michal Borowiecki
2017 at 2:38 PM, Vahid S Hashemian < vahidhashem...@us.ibm.com <mailto:vahidhashem...@us.ibm.com>> wrote: Hi Michal, Thanks a lot for your feedback. Your statement about Heartbeat is fair and makes sense. I'll update the KIP accordingly. --Vahid From: Michal Bor

Re: Kafka Streams vs Spark Streaming : reduce by window

2017-06-18 Thread Michal Borowiecki
t or might not depend on wall clock time. Perhaps this is in fact what you are proposing? -Jay On Fri, Jun 16, 2017 at 2:38 AM, Michal Borowiecki mailto:michal.borowie...@openbet.com>> wrote: I wonder if it's a frequent enough use case that Kafka Streams should consider

Re: Kafka Streams vs Spark Streaming : reduce by window

2017-06-16 Thread Michal Borowiecki
o Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP on Windows Embedded & IoT Microsoft Azure Advisor Twitter : @ppatierno<http://twitter.com/ppatierno> Linkedin : paolopatierno<http://it.linkedin.com/in/paolopatierno> Blog : DevExperience<http://paolopatier

Re: [DISCUSS] KIP-150 - Kafka-Streams Cogroup

2017-06-13 Thread Michal Borowiecki
one adding aggregators after each call so we must return a ktable just to possibly not need to have created it. On Jun 13, 2017 5:20 AM, "Michal Borowiecki" wrote: Actually, just had a thought. It started with naming. Are we actually co-grouping these streams or are we co-aggregating them? A

[jira] [Commented] (KAFKA-5245) KStream builder should capture serdes

2017-06-13 Thread Michal Borowiecki (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5245?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16047608#comment-16047608 ] Michal Borowiecki commented on KAFKA-5245: -- Just wanted to say it's gre

Re: [DISCUSS] KIP-163: Lower the Minimum Required ACL Permission of OffsetFetch

2017-06-13 Thread Michal Borowiecki
gestions are welcome! Thanks. --Vahid -- Signature <http://www.openbet.com/> Michal Borowiecki Senior Software Engineer L4 T: +44 208 742 1600 +44 203 249 8448 E: michal.borowie...@openbet.com W:

Re: Test fail reason on doc fix

2017-06-12 Thread Michal Borowiecki
olopatierno<http://it.linkedin.com/in/paolopatierno> Blog : DevExperience<http://paolopatierno.wordpress.com/> -- Signature <http://www.openbet.com/> Michal Borowiecki Senior Software Engineer L4 T: +44 208 742 1600 +44 203 249 8448

[jira] [Comment Edited] (KAFKA-5419) Console consumer --key-deserializer and --value-deserializer are always overwritten by ByteArrayDeserializer

2017-06-09 Thread Michal Borowiecki (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5419?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16044557#comment-16044557 ] Michal Borowiecki edited comment on KAFKA-5419 at 6/9/17 3:4

[jira] [Comment Edited] (KAFKA-5419) Console consumer --key-deserializer and --value-deserializer are always overwritten by ByteArrayDeserializer

2017-06-09 Thread Michal Borowiecki (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5419?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16044557#comment-16044557 ] Michal Borowiecki edited comment on KAFKA-5419 at 6/9/17 3:4

[jira] [Commented] (KAFKA-5419) Console consumer --key-deserializer and --value-deserializer are always overwritten by ByteArrayDeserializer

2017-06-09 Thread Michal Borowiecki (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5419?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16044557#comment-16044557 ] Michal Borowiecki commented on KAFKA-5419: -- I think it's a duplicate

Re: Re: synchronous request response using kafka

2017-06-09 Thread Michal Borowiecki
njay wrote: Can someone please share some thoughts whether we can do synchronous call (request response) using kafka similar to JMS Thanks Sanjay 913-221-9164 -- Signature <http://www.openbet.com/> Michal Borowiecki Senior Software Engineer L4 T: +44 208 742 1600

[jira] [Commented] (KAFKA-5246) Remove backdoor that allows any client to produce to internal topics

2017-06-09 Thread Michal Borowiecki (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5246?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16044179#comment-16044179 ] Michal Borowiecki commented on KAFKA-5246: -- Would it instead perhaps make s

Re: [DISCUSS] KIP-147: Add missing type parameters to StateStoreSupplier factories and KGroupedStream/Table methods

2017-06-03 Thread Michal Borowiecki
increasing the surface area for what is minimal benefit. It is one of those cases where i'd love to not have to maintain backward compatibility. Thanks, Damian On Fri, 2 Jun 2017 at 08:20 Michal Borowiecki mailto:michal.borowie...@openbet.com>> wrote:

Re: Sink Processor definition

2017-06-03 Thread Michal Borowiecki
Maybe the terminology is incorrect as strictly speaking a processor that writes data to anything could be considered a Sink Processor. On Sat, 3 Jun 2017 at 09:23 Michal Borowiecki mailto:michal.borowie...@openbet.com>> wrote: Hi all, Streams docs say: * *Sink Process

Sink Processor definition

2017-06-03 Thread Michal Borowiecki
to a kafka topic (directly) but only updates a state store also be considered a sink processor? I think yes. I'll submit a PR to that effect unless I hear otherwise. Cheers, Michał -- Signature <http://www.openbet.com/> Michal Borowiecki Senior Software Engineer L4 T:

Re: [DISCUSS] KIP-138: Change punctuate semantics

2017-06-02 Thread Michal Borowiecki
lternative. One minor comment: can you add `ValueTransformer#punctuate()` to the list of deprecated methods? -Matthias On 5/4/17 1:41 AM, Michal Borowiecki wrote: Further in this direction I've updated the main proposal to incorporate the Cancellable return type for ProcessorContext.s

Re: 0.11.0.0 Release Update

2017-06-02 Thread Michal Borowiecki
ded with the release notes/email for the release. I added some items to get it going. Please add to this list anything you think is worth noting. I'll plan to give another update next week just before the KIP freeze. Ismael -- Signature <http://www.openbet.com/> Michal Borowie

Re: [VOTE] KIP-164 Add unavailablePartitionCount and per-partition Unavailable metrics

2017-06-02 Thread Michal Borowiecki
+UnderMinIsrPartitionCount+and+per-partition+UnderMinIsr+metrics . Thanks, Dong -- Signature <http://www.openbet.com/> Michal Borowiecki Senior Software Engineer L4 T: +44 208 742 1600 +44 203 249 8448 E: michal.b

Re: [DISCUSS] KIP-147: Add missing type parameters to StateStoreSupplier factories and KGroupedStream/Table methods

2017-06-02 Thread Michal Borowiecki
11:11 AM, Michal Borowiecki wrote: Hi all, I've updated the KIP to reflect the proposed backwards-compatible approach: https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=69408481 Given the vast area of APIs affected, I think the PR is easier to read than the code excerpts

Re: Jira-Spam on Dev-Mailinglist

2017-05-30 Thread Michal Borowiecki
w.openbet.com/> Michal Borowiecki Senior Software Engineer L4 T: +44 208 742 1600 +44 203 249 8448 E: michal.borowie...@openbet.com W: www.openbet.com <http://www.openbet.com/> OpenBet Ltd

[jira] [Commented] (KAFKA-5155) Messages can be deleted prematurely when some producers use timestamps and some not

2017-05-27 Thread Michal Borowiecki (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5155?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16027521#comment-16027521 ] Michal Borowiecki commented on KAFKA-5155: -- Hi [~plavjanik], do you car

Re: [DISCUSS] KIP-147: Add missing type parameters to StateStoreSupplier factories and KGroupedStream/Table methods

2017-05-27 Thread Michal Borowiecki
b.com/apache/kafka/pull/2992/files Thanks, Michał On 07/05/17 10:16, Eno Thereska wrote: I like this KIP in general and I agree it’s needed. Perhaps Damian can comment on the session store issue? Thanks Eno On May 6, 2017, at 10:32 PM, Michal Borowiecki wrote: Hi Matthias, Agreed. I tried

[jira] [Commented] (KAFKA-5319) Add a tool to make cluster replica and leader balance

2017-05-24 Thread Michal Borowiecki (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5319?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16022695#comment-16022695 ] Michal Borowiecki commented on KAFKA-5319: -- [~markTC], shouldn't t

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-05-23 Thread Michal Borowiecki
and the rationale for init() and close(). Maybe I should add some examples. Thanks. Cheers, Jeyhun On Mon, May 22, 2017 at 11:02 AM, Michal Borowiecki mailto:michal.borowie...@openbet.com>> wrote: Hi Jeyhun, I'd like to understand better the premise of RichFunc

[jira] [Commented] (KAFKA-5243) Request to add row limit in ReadOnlyKeyValueStore range function

2017-05-23 Thread Michal Borowiecki (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5243?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16020911#comment-16020911 ] Michal Borowiecki commented on KAFKA-5243: -- Just a note, replacing the se

[jira] [Updated] (KAFKA-5233) Changes to punctuate semantics (KIP-138)

2017-05-23 Thread Michal Borowiecki (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5233?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michal Borowiecki updated KAFKA-5233: - Fix Version/s: (was: 0.11.0.0) 0.11.1.0 > Changes to punctu

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-05-22 Thread Michal Borowiecki
y/KAFKA/KIP-159%3A+Introducing+Rich+functions+to+Streams Cheers, Jeyhun -- Signature <http://www.openbet.com/> Michal Borowiecki Senior Software Engineer L4 T: +44 208 742 1600 +44 203 249 8448 E: mic

[jira] [Updated] (KAFKA-5233) Changes to punctuate semantics (KIP-138)

2017-05-15 Thread Michal Borowiecki (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5233?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michal Borowiecki updated KAFKA-5233: - Status: Patch Available (was: In Progress) Pull request here: https://github.com/apache

[jira] [Work started] (KAFKA-5233) Changes to punctuate semantics (KIP-138)

2017-05-15 Thread Michal Borowiecki (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5233?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-5233 started by Michal Borowiecki. > Changes to punctuate semantics (KIP-

[jira] [Commented] (KAFKA-3455) Connect custom processors with the streams DSL

2017-05-15 Thread Michal Borowiecki (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3455?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16010530#comment-16010530 ] Michal Borowiecki commented on KAFKA-3455: -- Hi [~bobbycalderwood], Can

[jira] [Updated] (KAFKA-5233) Changes to punctuate semantics (KIP-138)

2017-05-13 Thread Michal Borowiecki (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5233?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michal Borowiecki updated KAFKA-5233: - Labels: kip (was: ) > Changes to punctuate semantics (KIP-

Re: [VOTE] KIP-138: Change punctuate semantics

2017-05-13 Thread Michal Borowiecki
have enough votes, would you like to close the vote? Ismael On Thu, May 11, 2017 at 4:49 PM, Ewen Cheslack-Postava wrote: +1 (binding) -Ewen On Thu, May 11, 2017 at 7:12 AM, Ismael Juma wrote: Thanks for the KIP, Michal. +1(binding) from me. Ismael On Sat, May 6, 2017 at 6:18 PM, Michal

[jira] [Commented] (KAFKA-3514) Stream timestamp computation needs some further thoughts

2017-05-13 Thread Michal Borowiecki (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3514?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16009365#comment-16009365 ] Michal Borowiecki commented on KAFKA-3514: -- I've created KAFKA-5233

[jira] [Created] (KAFKA-5233) Changes to punctuate semantics (KIP-138)

2017-05-13 Thread Michal Borowiecki (JIRA)
Michal Borowiecki created KAFKA-5233: Summary: Changes to punctuate semantics (KIP-138) Key: KAFKA-5233 URL: https://issues.apache.org/jira/browse/KAFKA-5233 Project: Kafka Issue Type

Re: [VOTE] KIP-155 Add range scan for windowed state stores

2017-05-11 Thread Michal Borowiecki
oreIterator. All-in-all, I'd still prefer KeyValueIterator, V> as it more clearly names what's what. What do you think? Thanks, Michal On 11/05/17 07:51, Michal Borowiecki wrote: Well, another concern, apart from potential confusion, is that you won't be able to peek the ac

Re: [VOTE] KIP-155 Add range scan for windowed state stores

2017-05-10 Thread Michal Borowiecki
this make sense? Let me know if you still have concerns about this. Thank you, Xavier On Wed, May 10, 2017 at 12:25 PM Michal Borowiecki < michal.borowie...@openbet.com> wrote: > Apologies, I missed the discussion (or lack thereof) about the return > type of: > > WindowStoreItera

Re: [VOTE] KIP-155 Add range scan for windowed state stores

2017-05-10 Thread Michal Borowiecki
Apologies, I missed the discussion (or lack thereof) about the return type of: WindowStoreIterator> fetch(K from, K to, long timeFrom, long timeTo) WindowStoreIterator (as the KIP mentions) is a subclass of KeyValueIterator KeyValueIterator has the following method: /** * Peek at the nex

Re: [VOTE] KIP-138: Change punctuate semantics

2017-05-10 Thread Michal Borowiecki
May 6, 2017, at 11:01 PM, Bill Bejeck wrote: +1 Thanks, Bill On Sat, May 6, 2017 at 5:58 PM, Matthias J. Sax wrote: +1 Thanks a lot for this KIP! -Matthias On 5/6/17 10:18 AM, Michal Borowiecki wrote: Hi all, Given I'm not seeing any contentious issues remaining on the discussion t

Re: [DISCUSS]: KIP-149: Enabling key access in ValueTransformer, ValueMapper, and ValueJoiner

2017-05-09 Thread Michal Borowiecki
+1 :) On 08/05/17 23:52, Matthias J. Sax wrote: Hi, I was reading the updated KIP and I am wondering, if we should do the design a little different. Instead of distinguishing between a RichFunction and non-RichFunction at runtime level, we would use RichFunctions all the time. Thus, on the DS

[jira] [Commented] (KAFKA-5201) Compacted topic could be misused to fill up a disk but deletion policy can't retain legitimate keys

2017-05-09 Thread Michal Borowiecki (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5201?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16002311#comment-16002311 ] Michal Borowiecki commented on KAFKA-5201: -- Hi Edoardo, >From the b

Re: [DISCUSS] KIP-155 Add range scan for windowed state stores

2017-05-08 Thread Michal Borowiecki
://cwiki.apache.org/confluence/display/KAFKA/KIP+155+-+Add+range+scan+for+windowed+state+stores https://issues.apache.org/jira/browse/KAFKA-5192 Thank you, Xavier -- Signature <http://www.openbet.com/> Michal Borowiecki Senior Software Engineer L4 T: +44 208 74

Re: [DISCUSS] KIP-147: Add missing type parameters to StateStoreSupplier factories and KGroupedStream/Table methods

2017-05-07 Thread Michal Borowiecki
n the session store issue? Thanks Eno On May 6, 2017, at 10:32 PM, Michal Borowiecki wrote: Hi Matthias, Agreed. I tried your proposal and indeed it would work. However, I think to maintain full backward compatibility we would also need to deprecate Stores.create() and leave it unchanged,

Re: [DISCUSS]: KIP-149: Enabling key access in ValueTransformer, ValueMapper, and ValueJoiner

2017-05-07 Thread Michal Borowiecki
sions. Cheers, Jeyhun On Sun, May 7, 2017 at 11:46 AM Michal Borowiecki mailto:michal.borowie...@openbet.com>> wrote: Do I understanding correctly, that with the proposed pattern one could not pass a lambda expression and access the context from within it? TBH, I was hopi

Re: [DISCUSS]: KIP-149: Enabling key access in ValueTransformer, ValueMapper, and ValueJoiner

2017-05-07 Thread Michal Borowiecki
pache.org/jira/browse/KAFKA-4218 [3] https://issues.apache.org/jira/browse/KAFKA-4726 [4] https://issues.apache.org/jira/browse/KAFKA-3745 [5] https://github.com/apache/kafka/pull/2946 Cheers, Jeyhun -- -Cheers Jeyhun -- -Cheers Jeyhun -- -Cheers Jeyhun -- Signature <http://www.

Re: [DISCUSS] KIP-147: Add missing type parameters to StateStoreSupplier factories and KGroupedStream/Table methods

2017-05-06 Thread Michal Borowiecki
to have a look. But backward compatibility is a must from my point of view. -Matthias On 5/4/17 12:56 AM, Michal Borowiecki wrote: Hello, I've updated the KIP with missing information. I would especially appreciate some comments on the compatibility aspects of this as the proposed

[VOTE] KIP-138: Change punctuate semantics

2017-05-06 Thread Michal Borowiecki
ture <http://www.openbet.com/> Michal Borowiecki Senior Software Engineer L4 T: +44 208 742 1600 +44 203 249 8448 E: michal.borowie...@openbet.com W: www.openbet.com <http://www.openbet.com/>

Re: [DISCUSS] KIP-147: Add missing type parameters to StateStoreSupplier factories and KGroupedStream/Table methods

2017-05-04 Thread Michal Borowiecki
On 30/04/17 23:11, Michal Borowiecki wrote: Hi community! I have just drafted KIP-147: Add missing type parameters to StateStoreSupplier factories and KGroupedStream/Table methods <https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=69408481> Please let me know if this a

[jira] [Commented] (KAFKA-5155) Messages can be deleted prematurely when some producers use timestamps and some not

2017-05-03 Thread Michal Borowiecki (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5155?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15994575#comment-15994575 ] Michal Borowiecki commented on KAFKA-5155: -- Hi @huxi, Personally, I feel

[jira] [Comment Edited] (KAFKA-5155) Messages can be deleted prematurely when some producers use timestamps and some not

2017-05-03 Thread Michal Borowiecki (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5155?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15994575#comment-15994575 ] Michal Borowiecki edited comment on KAFKA-5155 at 5/3/17 9:4

Re: Support for Kafka Consumer over SASL_SSL

2017-05-01 Thread Michal Borowiecki
9) Nixon -- Signature <http://www.openbet.com/> Michal Borowiecki Senior Software Engineer L4 T: +44 208 742 1600 +44 203 249 8448 E: michal.borowie...@openbet.com W: www.openbet.com <htt

[jira] [Commented] (KAFKA-5144) MinTimestampTracker does not correctly add timestamps lower than the current max

2017-05-01 Thread Michal Borowiecki (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5144?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15990683#comment-15990683 ] Michal Borowiecki commented on KAFKA-5144: -- I understand now, thanks a lot

[DISCUSS] KIP-147: Add missing type parameters to StateStoreSupplier factories and KGroupedStream/Table methods

2017-04-30 Thread Michal Borowiecki
elcome. Thanks, Michal -- Signature <http://www.openbet.com/> Michal Borowiecki Senior Software Engineer L4 T: +44 208 742 1600 +44 203 249 8448 E: michal.borowie...@openbet.com W: www.openbet.com <http://

Re: [DISCUSS] KIP-138: Change punctuate semantics

2017-04-30 Thread Michal Borowiecki
)); lastPunctuationTime += interval;// I'm not sure of the merit of this vs lastPunctuationTime = ctx.timestamp(); but that's what PunctuationQueue does currently } // do some other business logic here } Looking forward to your thoughts. Cheers, Michal -- Signatur

[jira] [Commented] (KAFKA-5144) MinTimestampTracker does not correctly add timestamps lower than the current max

2017-04-30 Thread Michal Borowiecki (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5144?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15990335#comment-15990335 ] Michal Borowiecki commented on KAFKA-5144: -- Added a second [PR|h

[jira] [Commented] (KAFKA-5144) MinTimestampTracker does not correctly add timestamps lower than the current max

2017-04-30 Thread Michal Borowiecki (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5144?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15990329#comment-15990329 ] Michal Borowiecki commented on KAFKA-5144: -- [PR|https://github.com/apache/k

[jira] [Created] (KAFKA-5144) MinTimestampTracker does not correctly add timestamps lower than the current max

2017-04-30 Thread Michal Borowiecki (JIRA)
Michal Borowiecki created KAFKA-5144: Summary: MinTimestampTracker does not correctly add timestamps lower than the current max Key: KAFKA-5144 URL: https://issues.apache.org/jira/browse/KAFKA-5144

[jira] [Commented] (KAFKA-3514) Stream timestamp computation needs some further thoughts

2017-04-30 Thread Michal Borowiecki (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3514?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15990288#comment-15990288 ] Michal Borowiecki commented on KAFKA-3514: -- Agreed. That's what I mean

[jira] [Commented] (KAFKA-3514) Stream timestamp computation needs some further thoughts

2017-04-30 Thread Michal Borowiecki (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3514?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15990176#comment-15990176 ] Michal Borowiecki commented on KAFKA-3514: -- I think the description of

[jira] [Updated] (KAFKA-4593) Task migration during rebalance callback process could lead the obsoleted task's IllegalStateException

2017-04-29 Thread Michal Borowiecki (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4593?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michal Borowiecki updated KAFKA-4593: - Description: 1. Assume 2 running threads A and B, and one task t1 just for simplicity

Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface

2017-04-23 Thread Michal Borowiecki
However, if you're used to working with immutables, it can feel natural that withPartition would return a new object, so it could be more prone to mistakes. Cheers, Michal On 23/04/17 10:41, Michal Borowiecki wrote: IMHO, the ProducerRecord is anyway not immutable until send, since key

Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface

2017-04-23 Thread Michal Borowiecki
ed and regulated by the Financial Conduct Authority. -- Signature <http://www.openbet.com/> Michal Borowiecki Senior Software Engineer L4 T: +44 208 742 1600 +44 203 249 8448 E: michal.borowie...@openbet.co

Re: [VOTE] 0.10.2.1 RC3

2017-04-22 Thread Michal Borowiecki
://www.confluent.io/blog> -- Signature <http://www.openbet.com/> Michal Borowiecki Senior Software Engineer L4 T: +44 208 742 1600 +44 203 249 8448 E: michal.borowie...@openbet.com W: www.openbet

[jira] [Updated] (KAFKA-5090) Kafka Streams SessionStore.findSessions javadoc broken

2017-04-19 Thread Michal Borowiecki (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5090?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michal Borowiecki updated KAFKA-5090: - Affects Version/s: 0.10.2.1 > Kafka Streams SessionStore.findSessions javadoc bro

[jira] [Issue Comment Deleted] (KAFKA-5090) Kafka Streams SessionStore.findSessions javadoc broken

2017-04-19 Thread Michal Borowiecki (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5090?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michal Borowiecki updated KAFKA-5090: - Comment: was deleted (was: https://github.com/apache/kafka/pull/2874) > Kafka Stre

[jira] [Updated] (KAFKA-5090) Kafka Streams SessionStore.findSessions javadoc broken

2017-04-19 Thread Michal Borowiecki (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5090?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michal Borowiecki updated KAFKA-5090: - Status: Patch Available (was: Open) https://github.com/apache/kafka/pull/2874 > Ka

[jira] [Created] (KAFKA-5090) Kafka Streams SessionStore.findSessions javadoc broken

2017-04-19 Thread Michal Borowiecki (JIRA)
Michal Borowiecki created KAFKA-5090: Summary: Kafka Streams SessionStore.findSessions javadoc broken Key: KAFKA-5090 URL: https://issues.apache.org/jira/browse/KAFKA-5090 Project: Kafka

Re: [DISCUSS] KIP-138: Change punctuate semantics

2017-04-16 Thread Michal Borowiecki
ccess yet [arunmathew88]. Should I be > sending a separate mail for this? > > I thought one of the person following this thread would be able to give me > access. > > > > *From: *Michal Borowiecki > *Reply-To:

Re: 答复: Invoking KafkaConsumer#seek for the same partition

2017-04-13 Thread Michal Borowiecki
ight. I would have been thinking it that way Thank you. ________ 发件人: Michal Borowiecki 发送时间: 2017年4月13日 17:02 收件人: dev@kafka.apache.org 主题: Re: Invoking KafkaConsumer#seek for the same partition Sounds to me the comment is imprecisely phrased but was meant to indica

Re: Invoking KafkaConsumer#seek for the same partition

2017-04-13 Thread Michal Borowiecki
Sounds to me the comment is imprecisely phrased but was meant to indicate the behaviour you are describing. Perhaps instead of "the latest offset", it should say, "the offset used in the latest seek" to make it super-clear. Cheers, Michal On 13/04/17 08:28, Hu Xi wrote: Hi guys, The com

Re: [VOTE] 0.10.2.1 RC0

2017-04-12 Thread Michal Borowiecki
ger | Confluent 650.450.2760 | @gwenshap Follow us: Twitter | blog -- Signature <http://www.openbet.com/> Michal Borowiecki Senior Software Engineer L4 T: +44 208 742 1600 +44 203 249 8448 E: michal.borowie...@

Re: [DISCUSS] KIP-138: Change punctuate semantics

2017-04-07 Thread Michal Borowiecki
>>>>>>>> get record timestamps via context.timestamp(). Thus, users >> would >>>>>>>> need >>>>>>>> to >>>>>>>> track the time

Re: [DISCUSS] KIP-138: Change punctuate semantics

2017-04-04 Thread Michal Borowiecki
10 seconds event-time or 60 seconds system-time whatever comes > >> first). I don't say we should add this right away, but we might want > >> to > >> define the API in a way, that it allows extensions like this later > >> on, > >> without redesignin

[jira] [Commented] (KAFKA-4971) Why is there no difference between kafka benchmark tests on SSD and HDD?

2017-04-04 Thread Michal Borowiecki (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4971?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15955260#comment-15955260 ] Michal Borowiecki commented on KAFKA-4971: -- I'd venture a guess tha

Re: [DISCUSS] KIP-138: Change punctuate semantics

2017-04-03 Thread Michal Borowiecki
dded an additional callback to Processor that can be scheduled similarly to punctuate() but was always called at fixed, wall clock based intervals? This way you wouldn't have to give up the notion of stream time to be able to do periodic processing. On Mon, 2017-04-03 at 10:34 +0100, Michal Borowieck

[jira] [Comment Edited] (KAFKA-4971) Why is there no difference between kafka benchmark tests on SSD and HDD?

2017-04-03 Thread Michal Borowiecki (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4971?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15953608#comment-15953608 ] Michal Borowiecki edited comment on KAFKA-4971 at 4/3/17 2:5

[jira] [Commented] (KAFKA-4971) Why is there no difference between kafka benchmark tests on SSD and HDD?

2017-04-03 Thread Michal Borowiecki (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4971?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15953608#comment-15953608 ] Michal Borowiecki commented on KAFKA-4971: -- I think your question woul

[DISCUSS] KIP-138: Change punctuate semantics

2017-04-03 Thread Michal Borowiecki
Hi all, I have created a draft for KIP-138: Change punctuate semantics . Appreciating there can be different views on system-time vs event-time semantics for punctuation depending on use-case and the importa

[jira] [Commented] (KAFKA-4835) Allow users control over repartitioning

2017-03-13 Thread Michal Borowiecki (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4835?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15907523#comment-15907523 ] Michal Borowiecki commented on KAFKA-4835: -- My point above was that if

[jira] [Commented] (KAFKA-4835) Allow users control over repartitioning

2017-03-10 Thread Michal Borowiecki (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4835?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15904785#comment-15904785 ] Michal Borowiecki commented on KAFKA-4835: -- Yes, that's the case. Messa

[jira] [Commented] (KAFKA-3514) Stream timestamp computation needs some further thoughts

2017-03-06 Thread Michal Borowiecki (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3514?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15897771#comment-15897771 ] Michal Borowiecki commented on KAFKA-3514: -- Oh, I wouldn't mind th

[jira] [Commented] (KAFKA-3514) Stream timestamp computation needs some further thoughts

2017-03-06 Thread Michal Borowiecki (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3514?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15897413#comment-15897413 ] Michal Borowiecki commented on KAFKA-3514: -- Thank you for responding. Just n

[jira] [Commented] (KAFKA-3514) Stream timestamp computation needs some further thoughts

2017-03-06 Thread Michal Borowiecki (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3514?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15897371#comment-15897371 ] Michal Borowiecki commented on KAFKA-3514: -- Hi [~enothereska], I hav

[jira] [Commented] (KAFKA-4601) Avoid duplicated repartitioning in KStream DSL

2017-03-03 Thread Michal Borowiecki (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4601?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15894498#comment-15894498 ] Michal Borowiecki commented on KAFKA-4601: -- Created KAFKA-4835. >

[jira] [Created] (KAFKA-4835) Allow users control over repartitioning

2017-03-03 Thread Michal Borowiecki (JIRA)
Michal Borowiecki created KAFKA-4835: Summary: Allow users control over repartitioning Key: KAFKA-4835 URL: https://issues.apache.org/jira/browse/KAFKA-4835 Project: Kafka Issue Type

[jira] [Commented] (KAFKA-4601) Avoid duplicated repartitioning in KStream DSL

2017-02-23 Thread Michal Borowiecki (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4601?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15881030#comment-15881030 ] Michal Borowiecki commented on KAFKA-4601: -- Don't know if this belong

[jira] [Created] (KAFKA-4750) KeyValueIterator returns null values

2017-02-09 Thread Michal Borowiecki (JIRA)
Michal Borowiecki created KAFKA-4750: Summary: KeyValueIterator returns null values Key: KAFKA-4750 URL: https://issues.apache.org/jira/browse/KAFKA-4750 Project: Kafka Issue Type: Bug

[jira] [Commented] (KAFKA-3514) Stream timestamp computation needs some further thoughts

2016-11-21 Thread Michal Borowiecki (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3514?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15683056#comment-15683056 ] Michal Borowiecki commented on KAFKA-3514: -- IMO, 2) *is* a severe pro

[jira] [Commented] (KAFKA-4355) StreamThread intermittently dies with "Topic not found during partition assignment" when broker restarted

2016-11-02 Thread Michal Borowiecki (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4355?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15628442#comment-15628442 ] Michal Borowiecki commented on KAFKA-4355: -- KAFKA-4366 created for

[jira] [Created] (KAFKA-4366) KafkaStreams.close() blocks indefinitely

2016-11-02 Thread Michal Borowiecki (JIRA)
Michal Borowiecki created KAFKA-4366: Summary: KafkaStreams.close() blocks indefinitely Key: KAFKA-4366 URL: https://issues.apache.org/jira/browse/KAFKA-4366 Project: Kafka Issue Type

[jira] [Comment Edited] (KAFKA-4355) StreamThread intermittently dies with "Topic not found during partition assignment" when broker restarted

2016-10-29 Thread Michal Borowiecki (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4355?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15618009#comment-15618009 ] Michal Borowiecki edited comment on KAFKA-4355 at 10/29/16 12:1

[jira] [Commented] (KAFKA-4355) StreamThread intermittently dies with "Topic not found during partition assignment" when broker restarted

2016-10-29 Thread Michal Borowiecki (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4355?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15618009#comment-15618009 ] Michal Borowiecki commented on KAFKA-4355: -- Trying to work around this issu

[jira] [Commented] (KAFKA-4355) StreamThread intermittently dies with "Topic not found during partition assignment" when broker restarted

2016-10-28 Thread Michal Borowiecki (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4355?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15616118#comment-15616118 ] Michal Borowiecki commented on KAFKA-4355: -- Perhaps the DefultPartitionGro

[jira] [Commented] (KAFKA-4355) StreamThread intermittently dies with "Topic not found during partition assignment" when broker restarted

2016-10-28 Thread Michal Borowiecki (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4355?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15615967#comment-15615967 ] Michal Borowiecki commented on KAFKA-4355: -- My first suspect so far is

[jira] [Created] (KAFKA-4355) StreamThread intermittently dies with "Topic not found during partition assignment" when broker restarted

2016-10-28 Thread Michal Borowiecki (JIRA)
Michal Borowiecki created KAFKA-4355: Summary: StreamThread intermittently dies with "Topic not found during partition assignment" when broker restarted Key: KAFKA-4355 URL: https://issues.apach

  1   2   >