Re: kstream transform forward to different topics

2019-02-13 Thread Jan Filipiak
For now, just use the name it gets automatically, or crack the AbstractStream open with reflection ;) 307 is doing it the wrong way again, just make name accessible instead of make the users put them :face_with_rolling_eyes: On 08.02.2019 02:36, Guozhang Wang wrote: > Hi Nan, > > Glad it helps w

Re: [ANNOUNCE] New Committer: Vahid Hashemian

2019-01-15 Thread Jan Filipiak
Congratz! On 15.01.2019 23:44, Jason Gustafson wrote: > Hi All, > > The PMC for Apache Kafka has invited Vahid Hashemian as a project committer > and > we are > pleased to announce that he has accepted! > > Vahid has made numerous contributions to the Kafka community over the past > few years. He

Re: Doubts in Kafka

2019-01-08 Thread Jan Filipiak
On 08.01.2019 17:11, aruna ramachandran wrote: > I need to process single sensor messages in serial (order of messages > should not be changed)at the same time I have to process 1 sensors > messages in parallel please help me to configure the topics and partitions. > If you want to process e

Re: Graceful Shutdown always fails on multi-broker setup (Windows)

2018-05-09 Thread Jan Filipiak
lling upgrade to 1.0x as prescribed here - https://kafka.apache.org/documentation/#upgrade ? Regards, On 9 May 2018 at 11:06, Jan Filipiak wrote: Hi, this is expected. A gracefully shutdown means the broker is only shutting down when it is not the leader of any partition. Therefore you shoul

Re: Graceful Shutdown always fails on multi-broker setup (Windows)

2018-05-09 Thread Jan Filipiak
Hi, this is expected. A gracefully shutdown means the broker is only shutting down when it is not the leader of any partition. Therefore you should not be able to gracefully shut down your entire cluster. Hope that helps Best Jan On 09.05.2018 12:02, M. Manna wrote: Hello, I have follo

Re: Kafka Consumer Offsets unavailable during rebalancing

2018-02-13 Thread Jan Filipiak
I would encourage you todo so. I also think its not reasonable behavior On 13.02.2018 11:28, Wouter Bancken wrote: We have upgraded our Kafka version as an attempt to solve this issue. However, the issue is still present in Kafka 1.0.0. Can I log a bug for this in JIRA? Wouter On 5 February 2

Re: Broker won't exit...

2018-01-10 Thread Jan Filipiak
HI brokers still try todo a gracefull shutdown I supose? It would only shut down if it is not the leader of any partition anymore. Can you verify: there are other brokers alive that took over leadership? and the broker in question stepped down as a leader for all partitions? Best Jan On 10.0

Re: Consuming a state store (KTable) basics - 1.0.0

2017-12-07 Thread Jan Filipiak
atest value per key or last updated might be a different story here, in the end there is a lot of flexibility here that everyone is free to explore Best Jan Thanks, Pete On Thu, Dec 7, 2017 at 1:42 AM, Jan Filipiak wrote: Hi, you should be able to retrieve your store with https://githu

Re: Consuming a state store (KTable) basics - 1.0.0

2017-12-06 Thread Jan Filipiak
Hi, you should be able to retrieve your store with https://github.com/apache/kafka/blob/trunk/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java#L1021 This would give you access to the store from inside your current application. In your Streams application your could then expose

Re: Mirrormaker consumption slowness

2017-12-06 Thread Jan Filipiak
Hi, two questions. Is your MirrorMaker collocated with the source or the target? what are the send and receive buffer sizes on the connections that do span across WAN? Hope we can get you some help. Best jan On 06.12.2017 14:36, Xu, Zhaohui wrote: Any update on this issue? We also run into

Re: Configuration: Retention and compaction

2017-12-03 Thread Jan Filipiak
Hi the only retention time that applies for compacted topics is the delete.retention.ms The duration that tombstones for deletes will be kept in the topic during compaction. A very detail explaination on what is going on can be found here: https://kafka.apache.org/documentation/#compaction

Re: Joins in Kafka Streams and partitioning of the topics

2017-11-30 Thread Jan Filipiak
, Artur Mrozowski wrote: what if I start two instances of that application? Does the state migrate between the applications? Is it then I have to use a global table? BR Artur On Thu, Nov 30, 2017 at 7:40 PM, Jan Filipiak wrote: Hi, Haven't checked your code. But from what you describ

Re: Joins in Kafka Streams and partitioning of the topics

2017-11-30 Thread Jan Filipiak
Hi, Haven't checked your code. But from what you describe you should be fine. Upgrading the version might help here and there but should still work with 0.10 I guess. Best Jan On 30.11.2017 19:16, Artur Mrozowski wrote: Thank you Damian, it was very helpful. I have implemented my solution i

Re: Plans to extend streams?

2017-11-30 Thread Jan Filipiak
d s2 are in the same network and data transfer is fast between two, we should not go through another intermediate layer. Thanks. On Wed, Nov 29, 2017 at 4:52 PM, Jan Filipiak wrote: Hey, you making some wrong assumptions here. Kafka Streams is in no way single threaded or limited to one ph

Re: Plans to extend streams?

2017-11-29 Thread Jan Filipiak
Hey, you making some wrong assumptions here. Kafka Streams is in no way single threaded or limited to one physical instance. Having connectivity issues to your brokers is IMO a problem with the deployment and not at all with how kafka streams is designed and works. Kafka Streams moves hundreds o

Re: No. of Kafk Instances in Single machine

2017-11-06 Thread Jan Filipiak
Hi, I probably would recommend you to go for 1 instance. You can bump a few thread configs to match your hardware better. Best Jan On 06.11.2017 12:23, chidigam . wrote: Hi All, Let say, I have big machine, which having 120GB RAM, with lot of cores, and very high disk capacity. How many no

Re: [DISCUSS] KIP-213 Support non-key joining in KTable

2017-10-26 Thread Jan Filipiak
search-hadoop.com/m/Kafka/uyzND1hzF8SRzUqb?subj=Re+DISCUSS+KIP+213+Support+non+key+joining+in+KTable On Thu, Oct 26, 2017 at 9:38 PM, Jan Filipiak wrote: Hello everyone, this is the new discussion thread after the ID-clash. Best Jan __ Hello Kafka-users, I want to continue with the development of

[DISCUSS] KIP-213 Support non-key joining in KTable

2017-10-25 Thread Jan Filipiak
Hello Kafka-users, I want to continue with the development of KAFKA-3705, which allows the Streams DSL to perform KTableKTable-Joins when the KTables have a one-to-many relationship. To make sure we cover the requirements of as many users as possible and have a good solution afterwards I invit

Re: Log Compaction Not Picking up Topic

2017-10-25 Thread Jan Filipiak
Hi, unfortunatly there is nothing trivial you could do here. Without upgrading your kafkas you can only bounce the partition back and forth between brokers so they compact while its still small. With upgrading you could also just cherrypick this very commit or put a logstatement to verify.

Re: Kafka Connect Sink Connector for multiple JDBC sinks

2017-09-16 Thread Jan Filipiak
Hi, entirely depends on how you want to serialize. You should be able to get everything running on Windows anyhow. Nothing expect the broker is really extensively using OS support for operating. To answer your initial question: You would simply start multiple sinks and give each sink a diffe

Re: [DISCUSS] Streams DSL/StateStore Refactoring

2017-08-11 Thread Jan Filipiak
ts where IQ would be usefull. The math of how many people are affected by this is therefore not important. Additionally the comparission doesnt make sense as the X-group can still go with a fluent interface only the Y-people need to break their fluent interface Guozhang On Fri, Aug 4, 2017 at

Re: [DISCUSS] Streams DSL/StateStore Refactoring

2017-08-04 Thread Jan Filipiak
of that, as illustrated above, is that you need to have separate object of the KTable in order to call either "queryHandle" or "materialize" (whatever the function name is) for the specifications of materialization options. This can break the concatenation of the topology constructi

Re: [DISCUSS] Streams DSL/StateStore Refactoring

2017-08-02 Thread Jan Filipiak
gt;> wrote: Hi Jan, Thanks for taking the time to put this together, appreciated. For the benefit of others would you mind explaining a bit about your motivation? Cheers, Damian On Wed, 2 Aug 2017 at 01:40 Jan Filipiak mailto:jan.filip...@trivago.com>> w

Re: [DISCUSS] Streams DSL/StateStore Refactoring

2017-08-01 Thread Jan Filipiak
Hi all, after some further discussions, the best thing to show my Idea of how it should evolve would be a bigger mock/interface description. The goal is to reduce the store maintaining processors to only the Aggregators + and KTableSource. While having KTableSource optionally materialized. I

Re: [DISCUSS] Streams DSL/StateStore Refactoring

2017-07-24 Thread Jan Filipiak
;t have control over. The whole logic about partitioners and what else does not change. Hope this makes my points more clear. Best Jan On 19.07.2017 12:03, Damian Guy wrote: Hi Jan, Thanks for your input. Comments inline On Tue, 18 Jul 2017 at 15:21 Jan Filipiak wrote: Hi, 1. To many

Re: [DISCUSS] Streams DSL/StateStore Refactoring

2017-07-18 Thread Jan Filipiak
ed.as("store1").withLoggingEnabled(configs)); // use a custom store changelog topic configs // ... more ``` Hope it helps. Guozhang On Fri, Jul 7, 2017 at 3:42 PM, Jan Filipiak <mailto:jan.filip...@trivago.com>> wrote: It makes me want to cry. why on earth is the DS

Re: Where to run kafka-consumer-groups.sh from?

2017-07-11 Thread Jan Filipiak
Hi, very likely due to timing. What problem is it causing you exactly that you want to work around? These differences shouldn't concern you to much I guess. We use the tool across continents and don't worry about it to much. Offset Commit interval makes everything blury anyways. If you can s

Re: [DISCUSS] Streams DSL/StateStore Refactoring

2017-07-09 Thread Jan Filipiak
does? Sorry can't wrap my head round that just now heading towards 3am. The example I provided was streams.$applicationid.stores.$storename.inmemory = false streams.$applicationid.stores.$storename.cachesize = 40k for the configs. The Query Handle thing make sense hopefully. Best J

Re: [DISCUSS] Streams DSL/StateStore Refactoring

2017-07-08 Thread Jan Filipiak
- too many overload (for some method we have already more than 10( - improve custom store API -Matthias On 7/7/17 3:42 PM, Jan Filipiak wrote: It makes me want to cry. why on earth is the DSL going to expose all its implementation details now? especially being materialized or not. If w

Re: [DISCUSS] Streams DSL/StateStore Refactoring

2017-07-07 Thread Jan Filipiak
ilder to reduce the overloaded functions as well. WDYT? Guozhang On Tue, Jul 4, 2017 at 1:40 AM, Damian Guy wrote: Hi Jan, Thanks very much for the input. On Tue, 4 Jul 2017 at 08:54 Jan Filipiak wrote: Hi Damian, I do see your point of something needs to change. But I fully agree with

Re: [DISCUSS] Streams DSL/StateStore Refactoring

2017-07-04 Thread Jan Filipiak
Hi Damian, I do see your point of something needs to change. But I fully agree with Gouzhang when he says. --- But since this is a incompatibility change, and we are going to remove the compatibility annotations soon it means we only have one chance and we really have to make it right. I

Re: [DISCUSS] Streams DSL/StateStore Refactoring

2017-06-24 Thread Jan Filipiak
I am with Gouzhang here. I think all the suggestions are far to short-sighted. Especially this wired materialize(String) call is broken totally and people go nuts about how this will look. + Implementing more and better joins, not this wired one we got currently. Implementing an one to many jo

Re: [DISCUSS] Streams DSL/StateStore Refactoring

2017-06-22 Thread Jan Filipiak
Hi Eno, I am less interested in the user facing interface but more in the actual implementation. Any hints where I can follow the discussion on this? As I still want to discuss upstreaming of KAFKA-3705 with someone Best Jan On 21.06.2017 17:24, Eno Thereska wrote: (cc’ing user-list too)

Re: Reliably implementing global KeyValueStore#get

2017-06-07 Thread Jan Filipiak
23:47, Steven Schlansker wrote: I was actually considering writing my own KeyValueStore backed by e.g. a Postgres or the like. Is there some feature Connect gains me that would make it better than such an approach? thanks On Jun 7, 2017, at 2:20 PM, Jan Filipiak wrote: Hi, have you thought

Re: Reliably implementing global KeyValueStore#get

2017-06-07 Thread Jan Filipiak
Hi, have you thought about using connect to put data into a store that is more reasonable for your kind of query requirements? Best Jan On 07.06.2017 00:29, Steven Schlansker wrote: On Jun 6, 2017, at 2:52 PM, Damian Guy wrote: Steven, In practice, data shouldn't be migrating that often.

Re: [DISCUSS]: KIP-161: streams record processing exception handlers

2017-06-07 Thread Jan Filipiak
Hi Eno, On 07.06.2017 22:49, Eno Thereska wrote: Comments inline: On 5 Jun 2017, at 18:19, Jan Filipiak wrote: Hi just my few thoughts On 05.06.2017 11:44, Eno Thereska wrote: Hi there, Sorry for the late reply, I was out this past week. Looks like good progress was made with the

Re: [DISCUSS]: KIP-161: streams record processing exception handlers

2017-06-05 Thread Jan Filipiak
threshold based callback" might be questionable. But as you argue for strict "fail-fast", I want to argue that this must not always be the best pattern to apply and that the overall KIP idea is super useful from my point of view. -Matthias On 6/3/17 11:57 AM, Jan Filipiak wrote:

Re: [DISCUSS]: KIP-161: streams record processing exception handlers

2017-06-03 Thread Jan Filipiak
es not have any of the fields you expect and depend on for your processing. I expect the reason for limiting the scope is it is pretty hard to reason about correctness for anything that stops in the middle of processing an operator DAG? -Jay On Fri, Jun 2, 2017 at 4:50 AM, Jan Filipiak wrote

Re: [DISCUSS]: KIP-161: streams record processing exception handlers

2017-06-02 Thread Jan Filipiak
don't want to fail fast. On Fri, 2 Jun 2017 at 11:00 Jan Filipiak wrote: Hi 1. That greatly complicates monitoring. Fail Fast gives you that when you monitor only the lag of all your apps you are completely covered. With that sort of new application Monitoring is very much more complicat

Re: [DISCUSS]: KIP-161: streams record processing exception handlers

2017-06-02 Thread Jan Filipiak
Again, it might help the KIP if we added an end-to-end example for such user code errors. snip On Tue, May 30, 2017 at 9:24 AM, Jan Filipiak wrote: Hi Jay, Eno mentioned that he will narrow down the scope to only ConsumerRecord deseri

Re: [DISCUSS]: KIP-161: streams record processing exception handlers

2017-05-30 Thread Jan Filipiak
Hi Jay, Eno mentioned that he will narrow down the scope to only ConsumerRecord deserialisation. I am working with Database Changelogs only. I would really not like to see a dead letter queue or something similliar. how am I expected to get these back in order. Just grind to hold an call me

Re: [DISCUSS]: KIP-161: streams record processing exception handlers

2017-05-28 Thread Jan Filipiak
+1 On 26.05.2017 18:36, Damian Guy wrote: In that case, though, every access to that key is doomed to failure as the database is corrupted. So i think it should probably die in a steaming heap at that point! On Fri, 26 May 2017 at 17:33 Eno Thereska wrote: Hi Damian, I was thinking of cases

Re: [DISCUSS]: KIP-161: streams record processing exception handlers

2017-05-26 Thread Jan Filipiak
catch-all handler for all sorts of other problems (e.g., NPE exception in user code). Deserialization erros can happen either when polling or when deserialising from a state store. So that narrows down the scope of the KIP, will update it. Thanks Eno On 26 May 2017, at 11:31, Jan Filipiak wr

Re: [DISCUSS]: KIP-161: streams record processing exception handlers

2017-05-26 Thread Jan Filipiak
2) any exception during processing of this record. So whenever a processor gets the record (after some caching, etc) it starts to process it, then it fails, then it will call the handler with this record. Does that match your thinking? Thanks, Eno On 26 May 2017, at 09:51, Jan Filipiak wrote:

Re: [DISCUSS]: KIP-161: streams record processing exception handlers

2017-05-26 Thread Jan Filipiak
Hi, quick question: From the KIP it doesn't quite makes sense to me how that fits with caching. With caching the consumer record might not be at all related to some processor throwing while processing. would it not make more sense to get the ProcessorName + object object for processing and s

ThoughWorks Tech Radar: Assess Kafka Streams

2017-03-29 Thread Jan Filipiak
Regardless of how usefull you find the tech radar. Well deserved! even though we all here agree that trial or adopt is in reach https://www.thoughtworks.com/radar/platforms/kafka-streams Best Jan

Re: Implementing a non-key in Kafka Streams using the Processor API

2017-02-21 Thread Jan Filipiak
do that on the DSL but at the PAPI layer, right? Guozhang On Tue, Feb 21, 2017 at 6:05 AM, Jan Filipiak wrote: Just a little note here: if you can take all rows of the "children" table for each key into memory, you get get away by using group_by and make a list of them. With this a

Re: Implementing a non-key in Kafka Streams using the Processor API

2017-02-21 Thread Jan Filipiak
Just a little note here: if you can take all rows of the "children" table for each key into memory, you get get away by using group_by and make a list of them. With this aggregation the join is straight forward and you can use a lateral view later to get to the same result. For this you could

Re: Implementing a non-key in Kafka Streams using the Processor API

2017-02-21 Thread Jan Filipiak
Hi, yes the ticket is exactly about what you want to do. The lengthy discussion is mainly about what the key of the output KTable is. @gouzhang would you be interested in seeing what we did so far? best Jan On 21.02.2017 13:10, Frank Lyaruu wrote: I've read that JIRA (although I don't under

Re: KIP-122: Add a tool to Reset Consumer Group Offsets

2017-02-08 Thread Jan Filipiak
Hi, Just my few thoughts: does it need to be json? the old zkOffset tool had a nice format, very easy to manipulate on cli very powerfull: changes as many consumergroups/topics/partitions in one go as you want maybe allow -1 and -2 to indicate earliest and latest reset regardless of what the

Re: At Least Once semantics for Kafka Streams

2017-02-03 Thread Jan Filipiak
Hey, with a little more effort you can try to make your stream application idempotent. Maybe giving you the same results. Say you want to aggregate a KStream by some key. Instead of keeping the aggregate, you keep a Set of raw values and then do the aggregate calculations with a map(). This

Re: kafka-consumer-offset-checker complaining about NoNode for X in zk

2017-02-02 Thread Jan Filipiak
Hi, sorry and using the consumer group tool, instead of the offset checker On 02.02.2017 20:08, Jan Filipiak wrote: Hi, if its a kafka stream app, its most likely going to store its offsets in kafka rather than zookeeper. You can use the --new-consumer option to check for kafka stored

Re: kafka-consumer-offset-checker complaining about NoNode for X in zk

2017-02-02 Thread Jan Filipiak
Hi, if its a kafka stream app, its most likely going to store its offsets in kafka rather than zookeeper. You can use the --new-consumer option to check for kafka stored offsets. Best Jan On 01.02.2017 21:14, Ara Ebrahimi wrote: Hi, For a subset of our topics we get this error: $KAFKA_HO

Re: Does KafkaStreams be configured to use multiple broker clusters in the same topology

2017-02-02 Thread Jan Filipiak
Sometimes I wake up cause I dreamed that this had gone down: https://cwiki.apache.org/confluence/display/KAFKA/Hierarchical+Topics On 02.02.2017 19:07, Roger Vandusen wrote: Ah, yes, I see your point and use case, thanks for the feedback. On 2/2/17, 11:02 AM, "Damian Guy" wrote: Hi R

Re: [DISCUSS] KIP-114: KTable materialization and improved semantics

2017-01-30 Thread Jan Filipiak
t we want to be. This is indeed much broader. My guess here is that's why both API's exists and helping the users to switch back and forth might be a thing. Feel free to pitch in if I have misinterpreted something. Thanks Eno On 30 Jan 2017, at 10:22, Jan Filipiak wrote: Hi Eno,

Re: [DISCUSS] KIP-114: KTable materialization and improved semantics

2017-01-30 Thread Jan Filipiak
StateStore, and can replace the `KafkaStreams.store` function; 2) comparing those two options assuming we get rid of the misleading function name, I personally favor not adding more overloading functions as it keeps the API simpler. Guozhang On Sat, Jan 28, 2017 at 2:32 PM, Jan Filipiak wro

Re: [DISCUSS] KIP-114: KTable materialization and improved semantics

2017-01-28 Thread Jan Filipiak
x27;ll consider as part of this KIP. Thanks Eno On 28 Jan 2017, at 06:49, Jan Filipiak wrote: Hi Exactly I know it works from the Processor API, but my suggestion would prevent DSL users dealing with storenames what so ever. In general I am pro switching between DSL and Processor API easily

Re: Fwd: [DISCUSS] KIP-114: KTable materialization and improved semantics

2017-01-27 Thread Jan Filipiak
reasons, I am more in favor of overloading methods rather than introducing `materialize()`. FWIW, we already have a similar API setup for e.g. `KTable#through(topicName, stateStoreName)`. A related but slightly different question is what e.g. Jan Filipiak mentioned earlier in this thread: I think we nee

Re: Fwd: [DISCUSS] KIP-114: KTable materialization and improved semantics

2017-01-27 Thread Jan Filipiak
something like: stream.groupyByKey() .window(TimeWindow.of(5000)) .aggregate(...) .withAggValueSerde(new CustomTypeSerde()) .withStoreName("storeName); (This would also reduce JavaDoc redundancy -- maybe a personal pain point right now :)) -Matthias On 1/27/17 11

Re: Fwd: [DISCUSS] KIP-114: KTable materialization and improved semantics

2017-01-27 Thread Jan Filipiak
API setup for e.g. `KTable#through(topicName, stateStoreName)`. A related but slightly different question is what e.g. Jan Filipiak mentioned earlier in this thread: I think we need to explain more clearly why KIP-114 doesn't propose the seemingly simpler solution of always materializing tables/

Re: Fwd: [DISCUSS] KIP-114: KTable materialization and improved semantics

2017-01-27 Thread Jan Filipiak
Hi, Yeah its confusing, Why shoudn't it be querable by IQ? If you uses the ValueGetter of Filter it will apply the filter and should be completely transparent as to if another processor or IQ is accessing it? How can this new method help? I cannot see the reason for the additional materializ

Re: Kafka Logo as HighRes or Vectorgraphics

2016-12-02 Thread Jan Filipiak
Hi, I was just pointed to this. https://www.vectorlogo.zone/logos/apache_kafka/ if someone else is looking for the same thing! thanks a lot Best Jan On 01.12.2016 13:05, Jan Filipiak wrote: Hi Everyone, we want to print some big banners of the Kafka logo to decorate our offices. Can anyone

Kafka Logo as HighRes or Vectorgraphics

2016-12-01 Thread Jan Filipiak
Hi Everyone, we want to print some big banners of the Kafka logo to decorate our offices. Can anyone help me find a version of the kafka logo that would still look nice printed onto 2x4m flags? Highly appreciated! Best Jan

Re: [VOTE] Add REST Server to Apache Kafka

2016-10-26 Thread Jan Filipiak
And you also still need to find the correct broker, for each http call, wich is also hard, when programming against the http api On 26.10.2016 09:46, Jan Filipiak wrote: So happy to see this reply. I do think the same, actually makes it way harder to properly batch up records on http, as

Re: [VOTE] Add REST Server to Apache Kafka

2016-10-26 Thread Jan Filipiak
So happy to see this reply. I do think the same, actually makes it way harder to properly batch up records on http, as kafka core would need to know how to split your payload. It would help people do the wrong thing IMO best Jan On 25.10.2016 23:58, Jay Kreps wrote: -1 I think the REST serve

Re: KIP-33 Opt out from Time Based indexing

2016-09-08 Thread Jan Filipiak
if there are any other issues related to time-based index? Thanks, Jun On Mon, Sep 5, 2016 at 11:52 PM, Jan Filipiak wrote: Hi Jun, sorry for the late reply. Regarding B, my main concern was just complexity of understanding what's going on. As you can see it took me probably some 2 days

Re: KIP-33 Opt out from Time Based indexing

2016-09-05 Thread Jan Filipiak
ven my view on this above. Are there any other things that you think that having a time-based index will hurt? Thanks, Jun On Fri, Aug 26, 2016 at 3:41 PM, Jan Filipiak wrote: Hi Jun, thanks for taking the time to answer on such a detailed level. You are right Log.fetchOffsetByTimestamp works,

Re: KIP-33 Opt out from Time Based indexing

2016-08-26 Thread Jan Filipiak
ing offset based on timestamp will allow the consumer to resume the consumption without missing any messages and also not replaying too many messages. Thanks, Jun On Wed, Aug 24, 2016 at 5:05 PM, Jan Filipiak mailto:jan.filip...@trivago.com>> wrote: Hey Jun, I go and try again :)

Re: KIP-33 Opt out from Time Based indexing

2016-08-24 Thread Jan Filipiak
n the specific indexes would need to throw an exception of some kind. Just an idea. / Jun On 22.08.2016 09 :24, Jan Filipiak wrote: Hello everyone, I stumbled across KIP-33 and the time based index, while briefly checking the wiki and commits, I fai

Re: Re: KIP-33 Opt out from Time Based indexing

2016-08-24 Thread Jan Filipiak
ex i : indexes) [i.shouldRoll(segment)}? wich should already be easier. If users don't want time based indexing, just don't put the timebased index in the Set then and everything should work like a charm. RPC calls that work on the specific indexes would need to throw an exception of some

KIP-33 Opt out from Time Based indexing

2016-08-22 Thread Jan Filipiak
Hello everyone, I stumbled across KIP-33 and the time based index, while briefly checking the wiki and commits, I fail to find a way to opt out. I saw it having quite some impact on when logs are rolled and was hoping not to have to deal with all of that. Is there a disable switch I overlooked

Re: Kafka Streams

2016-03-12 Thread Jan Filipiak
Hi, I am very exited about all of this in general. Sadly I haven’t had the time to really take a deep look. One thing that is/was always a difficult topic to resolve many to many relationships in table x table x table joins is the repartitioning that has to happen at some point. From the doc

Re: Consuming "backwards"?

2015-11-09 Thread Jan Filipiak
Hi, obviously this should be build different IMHO (unless I fail to see something that prevents you from doing this). When you realize you fall behind do this: 1. remember your current 2. get the latest offset 3. fork a process to replicate from the current offset +1 to the latest one just fe

Re: log compaction scaling with ~100m messages

2015-10-08 Thread Jan Filipiak
Hi, just want to pick this up again. You can always use more partitions to reduce the number of keys handled by a single broker and parallelize the compaction. So with sufficient number of machines and the ability to partition I don’t see you running into problems. Jan On 07.10.2015 05:34,

Re: Hdfs fSshell getmerge

2015-07-24 Thread Jan Filipiak
Sorry wrong mailing list On 24.07.2015 16:44, Jan Filipiak wrote: Hello hadoop users, I have an idea about a small feature for the getmerge tool. I recently was in the need of using the new line option -nl because the files I needed to merge simply didn't had one. I was merging all the

Hdfs fSshell getmerge

2015-07-24 Thread Jan Filipiak
Hello hadoop users, I have an idea about a small feature for the getmerge tool. I recently was in the need of using the new line option -nl because the files I needed to merge simply didn't had one. I was merging all the files from one directory and unfortunately this directory also included e

Questions regarding Kafka-1477

2015-07-02 Thread Jan Filipiak
Hi, just out of curiosity and because of Eugene's email, I browsed Kafka-1477 and it talks about SSL alot. So I thought I might throw in this http://tools.ietf.org/html/rfc7568 RFC. It basically says move away from SSL now and only do TLS. The title of the ticket still mentions TLS but afterw

Re: Log compaction not working as expected

2015-06-17 Thread Jan Filipiak
d thus the last segment will never be compacted. Thanks! Shayne On Wed, Jun 17, 2015 at 5:58 AM, Jan Filipiak wrote: Hi, you might want to have a look here: http://kafka.apache.org/documentation.html#topic-config _segment.ms_ and _segment.bytes _ should allow you to control the time/size when segm

Re: Log compaction not working as expected

2015-06-17 Thread Jan Filipiak
Hi, you might want to have a look here: http://kafka.apache.org/documentation.html#topic-config _segment.ms_ and _segment.bytes _ should allow you to control the time/size when segments are rolled. Best Jan On 16.06.2015 14:05, Shayne S wrote: Some further information, and is this a bug? I

Pulling Snapshots from Kafka, Log compaction last compact offset

2015-04-30 Thread Jan Filipiak
Hello Everyone, I am quite exited about the recent example of replicating PostgresSQL Changes to Kafka. My view on the log compaction feature always had been a very sceptical one, but now with its great potential exposed to the wide public, I think its an awesome feature. Especially when pulli

Re: Producer does not recognize new brokers

2015-04-13 Thread Jan Filipiak
Hey, try to not have newlines \n in your jsonfile. I think the parser dies on those and then claims the file is empty Best Jan On 13.04.2015 12:06, Ashutosh Kumar wrote: Probably you should first try to generate proposed plan using --generate option and then edit that if needed. thanks

Re: High CPU usage of Crc32 on Kafka broker

2015-02-22 Thread Jan Filipiak
I just want to bring up that idea of no server side de/recompression again. Features like KAFKA-1499 seem to steer the project into a different direction and the fact that tickets like KAFKA-845

Re: [DISCUSSION] adding the serializer api back to the new java producer

2014-12-02 Thread Jan Filipiak
Hello Everyone, I would very much appreciate if someone could provide me a real world examplewhere it is more convenient to implement the serializers instead of just making sure to provide bytearrays. The code we came up with explicitly avoids the serializer api. I think it is common underst