Re: Suppress DSL operator in stream api - 2.4.0

2019-09-25 Thread Thameem Ansari
key 1 and that closes today's > window and data in the past with key 2 won't go through > > On Wed, Sep 11, 2019, 8:45 PM Thameem Ansari wrote: > >> I tried with different timestamps in the near past but nothing coming out. >> I went thru the article from Conflu

Re: log.retention.bytes not working

2019-09-18 Thread Thameem Ansari
To reliably delete the logs you have to follow this. In Kafka server.properties set the following properties (adjust per your need) log.segment.bytes=10485760 log.retention.check.interval.ms=12 log.retention.ms=60 I believe the documentation should be clear enough to explain the priority

Re: Suppress DSL operator in stream api - 2.4.0

2019-09-11 Thread Thameem Ansari
gt; -- > Alessandro Tagliapietra > > > On Wed, Sep 11, 2019 at 8:06 PM Thameem Ansari wrote: > >> Yes I am able to see the output when I remove suppress. >> >> >>> On Sep 11, 2019, at 9:58 PM, Matthias J. Sax >> wrote: >>> >>> Ha

Re: Suppress DSL operator in stream api - 2.4.0

2019-09-11 Thread Thameem Ansari
Yes I am able to see the output when I remove suppress. > On Sep 11, 2019, at 9:58 PM, Matthias J. Sax wrote: > > Hard to say. Do you see output if you remove `suppress()` from your > topology? > > -Matthias > > > On 9/11/19 6:19 PM, Thameem Ansari wrote: >>

Re: Suppress DSL operator in stream api - 2.4.0

2019-09-11 Thread Thameem Ansari
sending data, event time cannot advance and thus emit will never > send anything downstream. > > Also see this blog post about `suppress`: > https://www.confluent.io/blog/kafka-streams-take-on-watermarks-and-triggers > > > -Matthias > > > > On 9/10/19 9:52 PM

Suppress DSL operator in stream api - 2.4.0

2019-09-10 Thread Thameem Ansari
In my streaming topology, I am using the suppress dsl operator. As per the documentation, it is supposed to output the final results after the window closes. But I noticed it's not emitting anything at all. Here is the pseudo code of my topology. .filter((key, value) -> ...) .flatMap((key, val

Re: Kafka streams (2.1.1) - org.rocksdb.RocksDBException:Too many open files

2019-07-12 Thread Thameem Ansari
s? If that is the case, do you mind sharing > your topology? It seems like you have several repartition operations and > you may be able to reduce the number of repartition operations depending on > what you are doing. > > Thanks, > Bill > >> On Thu, Jul 11, 2019 at 11:24

Re: Kafka streams (2.1.1) - org.rocksdb.RocksDBException:Too many open files

2019-07-11 Thread Thameem Ansari
ust ignore the warning. See KAFKA-5998 > <https://issues.apache.org/jira/browse/KAFKA-5998 > <https://issues.apache.org/jira/browse/KAFKA-5998>> for a long history (but > scroll to the bottom for the actual explanation) > > > On Thu, Jul 4, 2019 at 11:02

Re: Group Coordinator stuck on PrepareRebalance state.

2019-07-11 Thread Thameem Ansari
In addition to the session timeout try increasing the request timeout as well. We had similar issue and resolved it by increasing the timeouts. As per my understanding, If you have complex topology then it will take some time for kafka brokers to create the tasks and assign them to consumers. In

Re: Kafka streams (2.1.1) - org.rocksdb.RocksDBException:Too many open files

2019-07-04 Thread Thameem Ansari
processes is increased. Thanks Thameem > On Jul 4, 2019, at 9:28 AM, Thameem Ansari wrote: > > Sorry for reposting the previous message as the images didn’t come thru.. > pasting as text. > > I have changed both system and user limits. > > To completely isolate the p

Re: Kafka streams (2.1.1) - org.rocksdb.RocksDBException:Too many open files

2019-07-03 Thread Thameem Ansari
group/memory/memory.limit_in_bytes", O_RDONLY) = 59 25709 14:49:23 read(59, "9223372036854775807\n", 4096) = 20 25709 14:49:23 close(59)= 0 25732 14:49:23 open("/sys/fs/cgroup/memory/memory.limit_in_bytes", O_RDONLY) = 59 25732 14:49:23 read(59, "9223372036854775807\n", 4096) = 20 25732 14:49:23 close(59)= 0 25709 14:49:23 open("/sys/fs/cgroup/memory/memory.limit_in_bytes", O_RDONLY) = 59 25709 14:49:23 read(59, "9223372036854775807\n", 4096) = 20 But unable to come to any conclusion from these findings. Any insight is appreciated. Thanks Thameem > On Jul 4, 2019, at 9:18 AM, Thameem Ansari wrote: > > > > > But unable to come to any conclusion from these findings. Any insight is > appreciated. > > Thanks > Thameem

Re: Kafka streams (2.1.1) - org.rocksdb.RocksDBException:Too many open files

2019-07-03 Thread Thameem Ansari
Tried setting the open files to 100 and 50 but the results are same. I checked the total open files while the streaming application was busy running just before getting the “too many open files” message it was around 41756 which is same as what we have got when we set to -1. VisualVM shows tha

Re: Kafka streams (2.1.1) - org.rocksdb.RocksDBException:Too many open files

2019-07-02 Thread Thameem Ansari
As I mentioned, tried setting the OS limit to 600K & 1Million on the shell and tried to start the application on the same shell but still the problem exists. Tried rebooting the laptop and the results are same. So, need a way to find out what exactly is causing this issue when we hit close to 42

Re: Kafka streams (2.1.1) - org.rocksdb.RocksDBException:Too many open files

2019-07-02 Thread Thameem Ansari
Many places it is mentioned that closing the iterator is fixing the issue but this is true only if we use Processor APIs. But in DSL there is no iterator explicitly available and we are using wrapper methods like aggregate, map, groupBy, etc. Here is the snapshot of the issue with exact statis