Re: Crashes when running storm 2.1 topologies on multiple workers

2020-04-09 Thread Roshan Naik
Ideally you want to use multiple instances of a bolt/spout over doing internal 
multithreading. if you must use internal threading, you will need to do your 
own synchronization as well.
In general these distributed systems are designed to relieve users from doing 
their own MT and synchronization.

-roshan





On Thursday, April 9, 2020, 07:28:52 AM PDT, Ethan Li 
 wrote: 





In your case,

1. Every executor has an instance of ExecutorTransfer

https://github.com/apache/storm/blob/master/storm-client/src/jvm/org/apache/storm/executor/Executor.java#L146

2. Every ExecutorTransfer has its own serializer

https://github.com/apache/storm/blob/00f48d60e75b28e11a887baba02dc77876b2bb3d/storm-client/src/jvm/org/apache/storm/executor/ExecutorTransfer.java#L44

3. Every executor has its own outputCollector

https://github.com/apache/storm/blob/master/storm-client/src/jvm/org/apache/storm/executor/bolt/BoltExecutor.java#L146-L147

4. When outputCollector is called to emit to remote workers, it uses
ExecutorTransfer to transfer data

https://github.com/apache/storm/blob/00f48d60e75b28e11a887baba02dc77876b2bb3d/storm-client/src/jvm/org/apache/storm/executor/ExecutorTransfer.java#L66

5. which will try to serialize data

https://github.com/apache/storm/blob/00f48d60e75b28e11a887baba02dc77876b2bb3d/storm-client/src/jvm/org/apache/storm/daemon/worker/WorkerTransfer.java#L116

6. But serializer is not thread-safe

https://github.com/apache/storm/blob/00f48d60e75b28e11a887baba02dc77876b2bb3d/storm-client/src/jvm/org/apache/storm/serialization/KryoTupleSerializer.java#L33-L43


Filed a JIRA https://issues.apache.org/jira/browse/STORM-3620

Thanks for reporting this issue.

Best,
Ethan

On Thu, Apr 9, 2020 at 9:03 AM Ethan Li  wrote:

> You are right. I think output collectors are not thread-safe in storm
> 2.x.
>
>
> On Tue, Apr 7, 2020 at 7:35 AM Simon Cooper <
> simon.coo...@featurespace.co.uk> wrote:
>
>> Hi Storm devs,
>>
>> We've narrowed down the issue - multiple threads in our bolts are
>> accessing output collectors at the same time. In storm1, this was fine
>> (presumably something in clojure synchronized it, or it was thread-safe),
>> but in storm2, this causes multiple threads to try and write to the same
>> buffer at the same time, causing data corruption and really weird behaviour
>> when the corrupted data was deserialized at the other end. We've fixed this
>> by putting a mutex on all the collectors.
>>
>> I know there's been some back-and-forth around this in the past, but now,
>> output collectors are definitely not thread-safe in storm2.
>>
>> Simon
>>
>> -Original Message-
>> From: Simon Cooper 
>> Sent: 03 April 2020 12:16
>> To: dev@storm.apache.org
>> Subject: Crashes when running storm 2.1 topologies on multiple workers
>>
>> Hi Storm devs,
>>
>> We've encountered a serious problem when trying to run a storm 2.1
>> topology across multiple workers - it looks like the data is being
>> corrupted somewhere between being serialized on the sending worker and
>> being deserialized on the receiving worker. This means it's impossible for
>> us to run storm 2.1 on topologies with more than one worker!
>>
>> The relevant storm bug is STORM-3582. We've seen multiple exceptions in
>> this area, some of which point towards kryo being at fault here, maybe
>> involving custom serializers. However, I've so far been unable to reproduce
>> the issue in test cases.
>>
>> Is anyone able to give us some pointers to try and work out what may be
>> going wrong? This does seem like a very serious issue for the latest storm
>> release (unfortunately we're unable to try storm 2.0 to see when it was
>> introduced due to other issues with that release).
>>
>> Many thanks,
>> Simon Cooper
>>
>> This message, and any files/attachments transmitted together with it, is
>> intended for the use only of the person (or persons) to whom it is
>> addressed. It may contain information which is confidential and/or
>> protected by legal privilege. Accordingly, any dissemination, distribution,
>> copying or use of this message, or any part of it or anything sent together
>> with it, other than by intended recipients, may constitute a breach of
>> civil or criminal law and is hereby prohibited. Unless otherwise stated,
>> any views expressed in this message are those of the person sending it and
>> not the sender's employer. No responsibility, legal or otherwise, of
>> whatever nature, is accepted as to the accuracy of the contents of this
>> message or for the completeness of the message as received. Anyone who is
>> not the intended recipient of this message is advised to make no use of it
>> and is requested to contact Featurespace Limited as soon as possible. Any
>> recipient of this message who has knowledge or suspects that it may have
>> been the subject of unauthorised interception or alteration is also
>> requested to contact Featurespace Limited.
>>
>


Recordings of Storm 2.0 talks at the Bay Area Hadoop Meetup last week.

2019-11-05 Thread Roshan Naik

1- Raising the performance bar for stream processing with Apache Storm 2.0 - 
Roshan Naik, (Uber)
Discussion of performance improvements as well as the new 2.0 architecture for 
messaging subsystem, threading model & back pressure.
 https://www.youtube.com/watch?v=QsPzAtZXIk4


2- Apache Storm 2.0 - Features and Performance Enhancements - Kishorkumar 
Patil, (Verizon Media)https://www.youtube.com/watch?v=xNsG26uZ7sw

 A whirlwind tour of the major new features in 2.0.

-roshan

Storm 2.0 talks at Bay Area Hadoop meetup

2019-10-28 Thread Roshan Naik
If you are in the San Francisco Bay Area, please join us at the Bay Area Hadoop 
meetup tomorrow (Oct 29th 2019) featuring these talks on  Storm 2.0 :


1 - Raising the performance bar for stream processing with Apache Storm 2.0 - 
Roshan Naik (Uber),  

The effort to rearchitect Storm's core engine was born from the observation 
that there exists a significant gap between hardware capabilities and the 
performance of the best streaming engines. In this talk we’ll take a look at 
the performance and architecture of Storm's new engine which features a leaner 
threading model, a lock free messaging subsystem and a new ultra-lightweight 
Back Pressure model.


2 - Storm 2.0 - Features and Performance Enhancements - Kishor Patil  (Yahoo / 
Verizon Media)
 

For details and registration :
https://www.eventbrite.com/e/hadoop-meetup-yahoo-talks-from-uber-ebay-cloudera-verizon-media-tickets-78705479207


-roshan


Re: Steps for Updating Storm Website

2019-10-02 Thread Roshan Naik



Thanks Kishor will try it out. 




  From: Kishorkumar Patil 
Sent: Monday, September 30, 2019 8:55 PM
To: dev@storm.apache.org
Subject: Re: Steps for Updating Storm Website Roshan,

Please find documentation for adding/updating site documentation
https://github.com/apache/storm/blob/master/DEVELOPER.md#contribute-documentation
.

Essentially, it comes down to updating the github:
https://git-wip-us.apache.org/repos/asf?p=storm-site.git;a=summary

-Kishor


On Mon, Sep 30, 2019 at 6:34 PM Roshan Naik 
wrote:

> Taylor,Could you please share the steps for updating the Storm
> website?
> -roshan


Steps for Updating Storm Website

2019-09-30 Thread Roshan Naik
Taylor,    Could you please share the steps for updating the Storm website? 
-roshan

Re: Location for 2.0 blogs

2019-09-25 Thread Roshan Naik


Yes I was also thinking along the same lines of adding a blog menu next to the 
news.

Can someone share the steps involved in updating the website ?

-roshan





On Monday, September 23, 2019, 01:57:02 PM PDT, Stig Rohde Døssing 
 wrote: 





We could put it in the regular news feed, similar to the release posts, or
maybe we could add another button next to news for blog posts?

Den søn. 22. sep. 2019 kl. 12.32 skrev Roshan Naik
:

> I see that we have never published any blogs previouslyanywhere on the
> Apache Storm website.So wondering where would be a good location to publish
> the (overdue) 2.0 perf blog that I am close to completing. I guess that
> might dictate the format (html / wiki) the final version needs to be in.
> Suggestions ?
> -roshan


Location for 2.0 blogs

2019-09-22 Thread Roshan Naik
I see that we have never published any blogs previouslyanywhere on the Apache 
Storm website.So wondering where would be a good location to publish the 
(overdue) 2.0 perf blog that I am close to completing. I guess that might 
dictate the format (html / wiki) the final version needs to be in. Suggestions ?
-roshan

Re: [ANNOUNCE] New Committer/PMC Member Aaron Gresch

2019-08-10 Thread Roshan Naik
 Welcome Aaron, congratulations.
On Saturday, August 10, 2019, 06:56:39 PM PDT, Kishorkumar Patil 
 wrote:  
 
 Congratulations Aaron!

On Sat, Aug 10, 2019 at 8:14 PM Derek Dagit  wrote:

> Welcome, Aaron!
>
> On Sat, Aug 10, 2019 at 08:18:40AM -0500, Ethan Li wrote:
> >
> > Congratulations, Aaron!
> >
> > Ethan Li
> >
> > > On Aug 10, 2019, at 07:36, Alexandre Vermeerbergen <
> avermeerber...@gmail.com> wrote:
> > >
> > > This is great news (the more PMC members, the more healthly the Apache
> > > Storm project is), welcome Aaron!
> > >
> > > Alexandre Vermeerbergen
> > >
> > > Le sam. 10 août 2019 à 10:47, Stig Rohde Døssing
> > >  a écrit :
> > >>
> > >> I'm happy to announce that Aaron Gresch is now a member of the Storm
> PMC.
> > >>
> > >> Aaron has been active contributing to Storm for a while now. His
> > >> contributions have a good spread around the codebase, in particular
> having
> > >> to do with the blob store, metrics system and scheduling. He helped
> > >> implement metrics v2.
> > >>
> > >> Please join me in welcoming Aaron to the PMC.
>  

Re: [DISCUSS] Create a checklist for testing new Storm release

2019-08-09 Thread Roshan Naik
 +1  that does look like a nice list. 
On Thursday, August 1, 2019, 09:30:24 AM PDT, Ethan Li 
 wrote:  
 
 Right. I found this 
http://www.apache.org/dev/release-distribution.html#sigs-and-sums 


In the new release, we should drop md5 checksum file. 


> On Aug 1, 2019, at 4:25 AM, Stig Rohde Døssing  wrote:
> 
> The list looks good. I think we should drop the .md5 files, the ASF have
> been telling people not to use them for a while.
> 
> I don't know that everyone needs to do it, but someone should check that
> the license files are up to date, and that we're not including any
> dependencies with incompatible licenses (
> https://apache.org/legal/resolved.html). Ideally we can automate most of
> the work.
> 
> Den man. 29. jul. 2019 kl. 23.56 skrev Ethan Li :
> 
>> 
>> Per Hugo’s suggestion, we should probably document the testings that every
>> contributors/committers should do to guarantee a stable release. It also
>> helps creating a check form or email template.
>> 
>> 
>> The testings I did for 2.0.0 release are
>> 
>> 1. Verify files such as *.asc, *.md5, *.sha512
>> 2. Build Storm source code and run unit tests; create a Storm distribution.
>> 3. Set up a standalone cluster using apache-storm-xxx.zip,
>> apache-storm-xxx.tar.gz, the Storm distribution created from step 2,
>> separately
>> 4. Launch WordCountTopology and ThroughputVsLatency topology and check
>> logs, UI metrics
>> 5. Test basic UI functionalities such as jstack, heap dump, deactivate,
>> activate, rebalance, change log level, kill topology
>> 
>> 
>> Please suggest anything that should be added so that we can document a
>> base checklist that everyone can follow to test a new Storm release.
>> 
>> 
>> Thanks,
>> Ethan
  

Re: [Discuss] ARM CI for Storm

2019-06-27 Thread Roshan Naik
Thanks for volunteering. 


Sent from Yahoo Mail for iPhone


On Wednesday, June 26, 2019, 7:08 PM, Yikun Jiang  wrote:

Yes, we will definitely help to fix ARM test failures in PRs, and the
OpenLab CI will tell us which PR perhaps has ARM compatible problem, if
it's an easy fix problem, I think authors can fix it by themselves. If not,
we will help them to address it.

For the lack of ARM environment, if the developer are interested in testing
and debugging their storm patch in ARM  environment but don't have ARM
environment , we can also provide tmp ARM environment to them for testing.

Regards,
Yikun

Jiang Yikun(Kero)
Mail: yikunk...@gmail.com


Stig Rohde Døssing  于2019年6月26日周三 上午1:48写道:

> It sounds pretty low risk for us, if you're volunteering to help fix any
> ARM-specific CI failures that may crop up. Will this include helping to fix
> ARM test failures in PRs? Most people are unlikely to have an ARM
> environment they can use to test, and I'd prefer not to ask contributors to
> fix ARM test failures themselves.
>
> Den tir. 25. jun. 2019 kl. 11.08 skrev Yikun Jiang :
>
> > Sorry for late reply, actually, we have a developer team that willing to
> > work on this, and I'm the owner of storm ARM CI in the OpenLab team. We
> not
> > only want to enable the OpenLab CI, but also want to maintain the arm CI
> > job and fix the CI issue in Storm project. That means if the storm
> project
> > has some ARM compatible problem, I will very happy to fix it.
> >
> > As I mentioned before, one of the OpenLab goal is to make more open
> source
> > software to be more compatible for aarch64 platform. And the Storm
> project
> > is one of the most important one in BigData area, so we would like to
> > propose to work on aarch64 related works in Storm. we plan to start the
> > aarch64 related work from to add aarch64 build job for Storm, and our
> > initial plan as below:
> >
> > 1. We first propose enable a periodical job in OpenLab project to make
> sure
> > the storm can compile and build successfully in arm64 env.
> > 2. We hope can run the OpenLab CI in every pr to ensure that any pr after
> > Step 1 will not break the arm64 build.
> > 3. We can add more complex test cases on aarch64, like unit tests and
> > functional tests, step by step. It's a log term works.
> >
> > Of course, welcome another developers join to maintain the aarch64 CI,
> and
> > take effort on aarch64 work together.
> >
> > Thanks for your attention.
> >
> > Regards,
> > Yikun
> > 
> > Jiang Yikun(Kero)
> > Mail: yikunk...@gmail.com
> >
> >
> > Stig Rohde Døssing  于2019年6月21日周五 上午2:23写道:
> >
> > > I guess there is no interest in maintaining ARM compatibility, or at
> > least
> > > no one currently wants to take on the effort. Let's not add the Openlab
> > CI
> > > then, we can always do it later if someone expresses interest (and
> > > willingness to maintain).
> > >
> > > Den ons. 12. jun. 2019 kl. 17.11 skrev Stig Rohde Døssing <
> > > stigdoess...@gmail.com>:
> > >
> > > > Good point, let's see if there's anyone with an ARM environment.
> > > >
> > > > Den tir. 11. jun. 2019 kl. 23.07 skrev Jungtaek Lim <
> kabh...@gmail.com
> > >:
> > > >
> > > >> I guess the point is not related to open CI build. The point is
> > whether
> > > we
> > > >> really want to support ARM. I'm seeing OpenLab request for other
> > Apache
> > > >> projects as well, so I'd rather not treat their request as
> commitment
> > of
> > > >> putting efforts to make builds on ARM green.
> > > >>
> > > >> I'm a bit hesitant to add some environment on the support list
> unless
> > > >> there're enough engineers willing to work on maintaining. If the
> work
> > is
> > > >> left to existing maintainers, we will be stuck on failing tests on
> ARM
> > > CI
> > > >> but no environment to play locally.
> > > >>
> > > >> So I'd be +1 if there're at least two folks experienced with ARM
> env.
> > > >> volunteer to maintain the compatibility part. Otherwise -1 here.
> > > >>
> > > >> Thanks,
> > > >> Jungtaek Lim (HeartSaVioR)
> > > >>
> > > >> On Mon, Jun 10, 2019 at 7:27 PM Stig Rohde Døssing <
> > > >> stigdoess...@gmail.com>
> > > >> wrote:
> > > >>
> > > >> > Hi,
> > > >> >
> > > >> > An issue was opened suggesting adding ARM CI to the Storm build,
> > using
> > > >> > OpenLab https://issues.apache.org/jira/browse/STORM-3401,
> > associated
> > > PR
> > > >> > here https://github.com/apache/storm/pull/3023
> > > >> >
> > > >> > As far as I can tell, we need to allow OpenLab access to the
> Github
> > > PRs,
> > > >> > similar to how Travis has permission to access our PRs. The app
> that
> > > >> needs
> > > >> > access is https://github.com/apps/theopenlab-ci. Note that the
> app
> > > will
> > > >> > have access to the PRs only, not the repo code.
> > > >> >
> > > >> > Is anyone opposed to me asking infra to allow OpenLab access to
> our
> > > PRs?
> > > >> > I'll let this thread sit for a 

Re: [VOTE] Release Apache Storm 1.2.3 (rc2)

2019-06-25 Thread Roshan Naik
Friendly reminder on this one. Roshan


Sent from Yahoo Mail for iPhone


On Saturday, June 15, 2019, 9:15 AM, P. Taylor Goetz  wrote:

This vote is now closed and passes with 3 binding +1 votes and 1 non-binding +1 
vote.

I will release the artifacts and create a release announcement.

-Taylor

> On May 10, 2019, at 12:20 PM, P. Taylor Goetz  wrote:
> 
> This is a call to vote on releasing Apache Storm 1.2.3 (rc2)
> 
> Full list of changes in this release:
> 
> https://dist.apache.org/repos/dist/dev/storm/apache-storm-1.2.3-rc2/RELEASE_NOTES.html
> 
> The tag/commit to be voted upon is v1.2.3:
> 
> https://git-wip-us.apache.org/repos/asf?p=storm.git;a=tree;h=ac0e39d7cad48b3ec863c8a3b711d36511d4daf6;hb=4e162a47c8219546ab9639401363a8f1b5e51119
> 
> The source archive being voted upon can be found here:
> 
> https://dist.apache.org/repos/dist/dev/storm/apache-storm-1.2.3-rc2/apache-storm-1.2.3-src.tar.gz
> 
> Other release files, signatures and digests can be found here:
> 
> https://dist.apache.org/repos/dist/dev/storm/apache-storm-1.2.3-rc2/
> 
> The release artifacts are signed with the following key:
> 
> https://git-wip-us.apache.org/repos/asf?p=storm.git;a=blob_plain;f=KEYS;hb=22b832708295fa2c15c4f3c70ac0d2bc6fded4bd
> 
> The Nexus staging repository for this release is:
> 
> https://repository.apache.org/content/repositories/orgapachestorm-1081
> 
> Please vote on releasing this package as Apache Storm 1.2.3.
> 
> When voting, please list the actions taken to verify the release.
> 
> This vote will be open for at least 72 hours.
> 
> [ ] +1 Release this package as Apache Storm 1.2.3
> [ ]  0 No opinion
> [ ] -1 Do not release this package because...
> 
> Thanks to everyone who contributed to this release.
> 
> -Taylor





Re: Storm 2.0 Maven Artifacts

2019-06-03 Thread Roshan Naik
Does this mean that the artifacts were not been pushed to the mvn repo ? Or do 
we just need to wait for the artifacts to show up there .. and if so how long ? 
-roshan





 On Monday, June 3, 2019, 9:33:44 AM PDT, Stig Rohde Døssing 
 wrote: 



No, I think you are right. It doesn't look like the 2.0.0 artifacts are
present in Maven yet.

CC'ing to the dev list.

Den man. 3. jun. 2019 kl. 17.30 skrev Re'em Bensimhon :

> Hey everybody
>
> I'm trying to test out the new Storm release (very exciting :-) but I'm
> not managing to find the v2.0 artifacts for storm-client or storm-core in
> any public maven repository.
>
> https://repo.maven.apache.org/maven2/org/apache/storm/storm/
> https://mvnrepository.com/artifact/org.apache.storm/storm
>
> Am I missing something?
>
> Re'em
>


Re: [Discuss] New release managers

2019-05-31 Thread Roshan Naik
Yes, having the release process documented will help a great deal to let anyone 
to step in as release manager. 
Roshan 


Sent from Yahoo Mail for iPhone







Re: [VOTE] Release Apache Storm 1.2.3 (rc2)

2019-05-13 Thread Roshan Naik
 +1- verified binary distribution- setup local cluster - ran some perf topos 
with two workers and acking enabled- basic checking of the UI
-roshan


On Saturday, May 11, 2019, 1:21:26 PM EDT, Alexandre Vermeerbergen 
 wrote:  
 
 Oh and I forgot to mention that all my tests are made using
AdoptOpenJDK11.0.3 with OpenJ9.
It all runs very fine with this Java version & VM type.
We just had to patch our Hadoop dependencies (for HDFS & HBase Bolts)
to fix badly written Java version detection code in these
dependencies.


Le sam. 11 mai 2019 à 19:18, Alexandre Vermeerbergen
 a écrit :
>
> +1 (non binding)
>
> - Downloaded binary distribution & used it to update my Storm cloud service
> - Downloaded source distribution, compiled it and used the generated
> .jar files to update my Storm builder dependencies (storm-core.jar,
> storm-kafa-client.jar, flux*.jar)
> - Run my clusters with ~15 topologies
>
> Disclaimer : I have not yet ran test at big scale with this Storm
> 1.2.3 update, but I noticed from release notes that I was already
> running for several months with a Storm 1.2.3 pre-version updated my
> all commits that sounded useful to us (in particular the ones allowing
> to use recent Kafka brokers), so basically I'm confident to run 1.2.3
> final at scale within very short time after it'll be released.
>
> Kind regards,
> Alexandre Vermeerbergen
>
> Le ven. 10 mai 2019 à 18:28, P. Taylor Goetz  a écrit :
> >
> > This is a call to vote on releasing Apache Storm 1.2.3 (rc2)
> >
> > Full list of changes in this release:
> >
> > https://dist.apache.org/repos/dist/dev/storm/apache-storm-1.2.3-rc2/RELEASE_NOTES.html
> >
> > The tag/commit to be voted upon is v1.2.3:
> >
> > https://git-wip-us.apache.org/repos/asf?p=storm.git;a=tree;h=ac0e39d7cad48b3ec863c8a3b711d36511d4daf6;hb=4e162a47c8219546ab9639401363a8f1b5e51119
> >
> > The source archive being voted upon can be found here:
> >
> > https://dist.apache.org/repos/dist/dev/storm/apache-storm-1.2.3-rc2/apache-storm-1.2.3-src.tar.gz
> >
> > Other release files, signatures and digests can be found here:
> >
> > https://dist.apache.org/repos/dist/dev/storm/apache-storm-1.2.3-rc2/
> >
> > The release artifacts are signed with the following key:
> >
> > https://git-wip-us.apache.org/repos/asf?p=storm.git;a=blob_plain;f=KEYS;hb=22b832708295fa2c15c4f3c70ac0d2bc6fded4bd
> >
> > The Nexus staging repository for this release is:
> >
> > https://repository.apache.org/content/repositories/orgapachestorm-1081
> >
> > Please vote on releasing this package as Apache Storm 1.2.3.
> >
> > When voting, please list the actions taken to verify the release.
> >
> > This vote will be open for at least 72 hours.
> >
> > [ ] +1 Release this package as Apache Storm 1.2.3
> > [ ]  0 No opinion
> > [ ] -1 Do not release this package because...
> >
> > Thanks to everyone who contributed to this release.
> >
> > -Taylor  

Re: Storm 2.0 blogs ?

2019-05-13 Thread Roshan Naik



Taylor,
   Somewhere in the release notes it would be good to mention the following:

- Compatibility: 2.0 is backward compatible with 1.x, but they need to 
recompile the topo jars. 
- Dependency changes: Like Alexandre, others are likely to discover the hard 
way that storm-core is something else now and therefore compilation breaks.


-roshan


Re: [RESULT] [VOTE] Release Apache Storm 2.0.0 RC7

2019-05-05 Thread Roshan Naik
 Ooops... just missed it...  my +1 as well
Just finished verifying the foll:- spun up local cluster with binary dist- 
checked single a 2 worker modes. with and without ackers in each mode- ran 
SimplifiedWordCountTopo- did some quick perf verifications with 
ConstSpoutNullBolt- basic checking of the UI
-roshan


On Sunday, May 5, 2019, 6:53:52 PM EDT, P. Taylor Goetz  
wrote:  
 
 Wow. This is a special occasion…

The 2.0 release has been a very long time in the making, I can’t begin to thank 
all those who helped shepherd this branch to release quality.

To me this is an Apache project at it’s finest. We’ve seen core contributors 
come and go. And when they go, we’ve seen new contributors step up and fill the 
gap created. A big thank you to all contributors, past, present and future! 
This is a true community achievement.

This vote is now closed and passes with 6 binding +1 votes,  and no 0 or -1 
votes.

Vote tally (* indicates a binding vote):

+1: 
Derek Dagit*
Stig Rohde Døssing*
Kishorkumar Patil*
Jungtaek Lim*
Arun Mahadevan*
Ethan Li*

0:

-1:

I will release the staged artifacts and prepare a release announcement.

Congratulations again to all who contributed to this release.

-Taylor

> On Apr 29, 2019, at 6:49 PM, P. Taylor Goetz  wrote:
> 
> This is a call to vote on releasing Apache Storm 2.0.0 (rc7)
> 
> Full list of changes in this release:
> 
> https://dist.apache.org/repos/dist/dev/storm/apache-storm-2.0.0-rc7/RELEASE_NOTES.html
> 
> The tag/commit to be voted upon is v2.0.0:
> 
> https://git-wip-us.apache.org/repos/asf?p=storm.git;a=tree;h=007863edd95e838b3df414928c6fa3f28244ab49;hb=2ba95bbd1c911d4fc6363b1c4b9c4c6d86ac9aae
> 
> The source archive being voted upon can be found here:
> 
> https://dist.apache.org/repos/dist/dev/storm/apache-storm-2.0.0-rc7/apache-storm-2.0.0-src.tar.gz
> 
> Other release files, signatures and digests can be found here:
> 
> https://dist.apache.org/repos/dist/dev/storm/apache-storm-2.0.0-rc7/
> 
> The release artifacts are signed with the following key:
> 
> https://git-wip-us.apache.org/repos/asf?p=storm.git;a=blob_plain;f=KEYS;hb=22b832708295fa2c15c4f3c70ac0d2bc6fded4bd
> 
> The Nexus staging repository for this release is:
> 
> https://repository.apache.org/content/repositories/orgapachestorm-1079
> 
> Please vote on releasing this package as Apache Storm 2.0.0.
> 
> When voting, please list the actions taken to verify the release.
> 
> This vote will be open for at least 72 hours.
> 
> [ ] +1 Release this package as Apache Storm 2.0.0
> [ ]  0 No opinion
> [ ] -1 Do not release this package because...
> 
> Thanks to everyone who contributed to this release.
> 
> -Taylor
  

Re: [VOTE] Release Apache Storm 2.0.0 RC7

2019-04-30 Thread Roshan Naik
 Yes, you need to rebuild your topology jars against 2.0. 
If you have some settings to tweak perf with 1.x,  refer to 
https://github.com/apache/storm/blob/master/docs/Performance.md for the 2.x 
configs.

On Tuesday, April 30, 2019, 2:41:49 PM PDT, Alexandre Vermeerbergen 
 wrote:  
 
 Hello,

I'm eager to test Storm 2.0.0 with my complex topologies, but first of
all: do I need to rebuild all my topologies' Big Jars with Storm
2.0.0, or may I try my existing Storm 1.2.3 (recent snapshot)-based
Big Jars ?

Kind regards,
Alexandre Vermeerbergen

Le mar. 30 avr. 2019 à 00:49, P. Taylor Goetz  a écrit :
>
> This is a call to vote on releasing Apache Storm 2.0.0 (rc7)
>
> Full list of changes in this release:
>
> https://dist.apache.org/repos/dist/dev/storm/apache-storm-2.0.0-rc7/RELEASE_NOTES.html
>
> The tag/commit to be voted upon is v2.0.0:
>
> https://git-wip-us.apache.org/repos/asf?p=storm.git;a=tree;h=007863edd95e838b3df414928c6fa3f28244ab49;hb=2ba95bbd1c911d4fc6363b1c4b9c4c6d86ac9aae
>
> The source archive being voted upon can be found here:
>
> https://dist.apache.org/repos/dist/dev/storm/apache-storm-2.0.0-rc7/apache-storm-2.0.0-src.tar.gz
>
> Other release files, signatures and digests can be found here:
>
> https://dist.apache.org/repos/dist/dev/storm/apache-storm-2.0.0-rc7/
>
> The release artifacts are signed with the following key:
>
> https://git-wip-us.apache.org/repos/asf?p=storm.git;a=blob_plain;f=KEYS;hb=22b832708295fa2c15c4f3c70ac0d2bc6fded4bd
>
> The Nexus staging repository for this release is:
>
> https://repository.apache.org/content/repositories/orgapachestorm-1079
>
> Please vote on releasing this package as Apache Storm 2.0.0.
>
> When voting, please list the actions taken to verify the release.
>
> This vote will be open for at least 72 hours.
>
> [ ] +1 Release this package as Apache Storm 2.0.0
> [ ]  0 No opinion
> [ ] -1 Do not release this package because...
>
> Thanks to everyone who contributed to this release.
>
> -Taylor  

Re: [RESULT] [VOTE] Release Apache Storm 1.2.3 (rc1)

2019-04-16 Thread Roshan Naik
 Taylor,    Could you share your plans for the 1.2.3 rc  ?-roshan
On Thursday, April 11, 2019, 10:20:50 PM PDT, gita gita 
 wrote:  
 
 Sorry for constant nagging, Any update on the release.

On Thu, Mar 7, 2019 at 12:22 PM Roshan Naik 
wrote:

> It appears more likely that both releases will go out together... Due to
> some overlapping concerns getting addressed. Expecting another RC for both
> ... perhaps next week.
>
> Sent from Yahoo Mail for iPhone
>
>
> On Wednesday, March 6, 2019, 8:15 PM, gita gita 
> wrote:
>
> Hey Roshan, Thanks for the update.
>
> Any approximate timelines for the release? I see last release happened on
> May 17, 2018, any possibility of releasing 1.2.3 earlier if this is being
> blocked by 2.0 release?
>
>
>
> On Thu, Feb 28, 2019 at 4:29 AM Roshan Naik  >
> wrote:
>
> >  Gita,    Sorry for the late response. Due to some impt issues being
> > discovered just prior to the release was going to be published, the
> release
> > was held off. Fixes are in and another RC should be out expected soon. RC
> > typically takes about a week or two to get approved and published.-roshan
> >    On Friday, February 22, 2019, 10:25:39 PM PST, gita gita <
> > gita1git...@gmail.com> wrote:
> >
> >  Hey,
> >
> > Any updates, any estimated date for the release, Thanks a Ton.
> >
> > On 2019/02/03 22:39:26, "P. Taylor Goetz"  wrote:
> > > This vote is now closed and passes with 3 binding +1 votes, 2
> > non-binding +1 votes and no -1 or 0 votes.>
> > >
> > > I will push the release and announce after the 24 hour waiting period.>
> > >
> > > -Taylor>
> > >
> > > > On Jan 8, 2019, at 3:18 PM, P. Taylor Goetz 
> wrote:>
> > > > >
> > > > This is a call to vote on releasing Apache Storm 1.2.3 (rc1)>
> > > > >
> > > > Full list of changes in this release:>
> > > > >
> > > >
> >
> https://dist.apache.org/repos/dist/dev/storm/apache-storm-1.2.3-rc1/RELEASE_NOTES.html
> >
> >
> > > > >
> > > > The tag/commit to be voted upon is v1.2.3:>
> > > > >
> > > >
> >
> https://git-wip-us.apache.org/repos/asf?p=storm.git;a=tree;h=5eaaa51591f2e4dc3e31f22bcc581af4a1f39c03;hb=6ba98b215857656e0186887b5d1a6a5aceee10c4
> >
> >
> > > > >
> > > > The source archive being voted upon can be found here:>
> > > > >
> > > >
> >
> https://dist.apache.org/repos/dist/dev/storm/apache-storm-1.2.3-rc1/apache-storm-1.2.3-src.tar.gz
> >
> >
> > > > >
> > > > Other release files, signatures and digests can be found here:>
> > > > >
> > > > https://dist.apache.org/repos/dist/dev/storm/apache-storm-1.2.3-rc1/
> >
> > > > >
> > > > The release artifacts are signed with the following key:>
> > > > >
> > > >
> >
> https://git-wip-us.apache.org/repos/asf?p=storm.git;a=blob_plain;f=KEYS;hb=22b832708295fa2c15c4f3c70ac0d2bc6fded4bd
> >
> >
> > > > >
> > > > The Nexus staging repository for this release is:>
> > > > >
> > > >
> https://repository.apache.org/content/repositories/orgapachestorm-1074>
> >
> > > > >
> > > > Please vote on releasing this package as Apache Storm 1.2.3.>
> > > > >
> > > > When voting, please list the actions taken to verify the release.>
> > > > >
> > > > This vote will be open for at least 72 hours.>
> > > > >
> > > > [ ] +1 Release this package as Apache Storm 1.2.3>
> > > > [ ]  0 No opinion>
> > > > [ ] -1 Do not release this package because...>
> > > > >
> > > > Thanks to everyone who contributed to this release.>
> > > > >
> > > > -Taylor>
> > >
> > >
>
>
>
>
  

Re: [VOTE] Release Apache Storm 2.0.0 RC5

2019-04-08 Thread Roshan Naik
 ?
On Wednesday, March 27, 2019, 4:39:33 PM PDT, Roshan Naik 
 wrote:  
 
  Will there be another RC or this one is good to continue with ?



Sent from Yahoo Mail for iPhone


On Wednesday, March 27, 2019, 9:41 AM, Derek Dagit  wrote:

* Downloaded source ZIP, `mvn clean install`, all passed
* Verified signatures and checksums
* Packaged my build and ran a single-tenant (default) cluster
* Ran org.apache.storm.starter.WordCountTopology
* UI seemed OK, Visualization seemed OK, Logviewer seemed OK

+1

On Fri, Mar 22, 2019 at 3:23 PM P. Taylor Goetz  wrote:

> This is a call to vote on releasing Apache Storm 2.0.0 (rc5)
>
> Full list of changes in this release:
>
>
> https://dist.apache.org/repos/dist/dev/storm/apache-storm-2.0.0-rc5/RELEASE_NOTES.html
>
> The tag/commit to be voted upon is v2.0.0:
>
>
> https://git-wip-us.apache.org/repos/asf?p=storm.git;a=tree;h=7e0a711e4ed5315f04f9f726caff61e0f169e320;hb=b5823809bd4b438e789a36f163f318d4b161ad13
>
> The source archive being voted upon can be found here:
>
>
> https://dist.apache.org/repos/dist/dev/storm/apache-storm-2.0.0-rc5/apache-storm-2.0.0-src.tar.gz
>
> Other release files, signatures and digests can be found here:
>
> https://dist.apache.org/repos/dist/dev/storm/apache-storm-2.0.0-rc5/
>
> The release artifacts are signed with the following key:
>
>
> https://git-wip-us.apache.org/repos/asf?p=storm.git;a=blob_plain;f=KEYS;hb=22b832708295fa2c15c4f3c70ac0d2bc6fded4bd
>
> The Nexus staging repository for this release is:
>
> https://repository.apache.org/content/repositories/orgapachestorm-1076
>
> Please vote on releasing this package as Apache Storm 2.0.0.
>
> When voting, please list the actions taken to verify the release.
>
> This vote will be open for at least 72 hours.
>
> [ ] +1 Release this package as Apache Storm 2.0.0
> [ ]  0 No opinion
> [ ] -1 Do not release this package because...
>
> Thanks to everyone who contributed to this release.
>
> -Taylor


  

Re: [VOTE] Release Apache Storm 2.0.0 RC5

2019-03-27 Thread Roshan Naik
 Will there be another RC or this one is good to continue with ?



Sent from Yahoo Mail for iPhone


On Wednesday, March 27, 2019, 9:41 AM, Derek Dagit  wrote:

* Downloaded source ZIP, `mvn clean install`, all passed
* Verified signatures and checksums
* Packaged my build and ran a single-tenant (default) cluster
* Ran org.apache.storm.starter.WordCountTopology
* UI seemed OK, Visualization seemed OK, Logviewer seemed OK

+1

On Fri, Mar 22, 2019 at 3:23 PM P. Taylor Goetz  wrote:

> This is a call to vote on releasing Apache Storm 2.0.0 (rc5)
>
> Full list of changes in this release:
>
>
> https://dist.apache.org/repos/dist/dev/storm/apache-storm-2.0.0-rc5/RELEASE_NOTES.html
>
> The tag/commit to be voted upon is v2.0.0:
>
>
> https://git-wip-us.apache.org/repos/asf?p=storm.git;a=tree;h=7e0a711e4ed5315f04f9f726caff61e0f169e320;hb=b5823809bd4b438e789a36f163f318d4b161ad13
>
> The source archive being voted upon can be found here:
>
>
> https://dist.apache.org/repos/dist/dev/storm/apache-storm-2.0.0-rc5/apache-storm-2.0.0-src.tar.gz
>
> Other release files, signatures and digests can be found here:
>
> https://dist.apache.org/repos/dist/dev/storm/apache-storm-2.0.0-rc5/
>
> The release artifacts are signed with the following key:
>
>
> https://git-wip-us.apache.org/repos/asf?p=storm.git;a=blob_plain;f=KEYS;hb=22b832708295fa2c15c4f3c70ac0d2bc6fded4bd
>
> The Nexus staging repository for this release is:
>
> https://repository.apache.org/content/repositories/orgapachestorm-1076
>
> Please vote on releasing this package as Apache Storm 2.0.0.
>
> When voting, please list the actions taken to verify the release.
>
> This vote will be open for at least 72 hours.
>
> [ ] +1 Release this package as Apache Storm 2.0.0
> [ ]  0 No opinion
> [ ] -1 Do not release this package because...
>
> Thanks to everyone who contributed to this release.
>
> -Taylor





Re: [RESULT] [VOTE] Release Apache Storm 1.2.3 (rc1)

2019-03-06 Thread Roshan Naik
It appears more likely that both releases will go out together... Due to some 
overlapping concerns getting addressed. Expecting another RC for both ... 
perhaps next week. 

Sent from Yahoo Mail for iPhone


On Wednesday, March 6, 2019, 8:15 PM, gita gita  wrote:

Hey Roshan, Thanks for the update.

Any approximate timelines for the release? I see last release happened on
May 17, 2018, any possibility of releasing 1.2.3 earlier if this is being
blocked by 2.0 release?



On Thu, Feb 28, 2019 at 4:29 AM Roshan Naik 
wrote:

>  Gita,    Sorry for the late response. Due to some impt issues being
> discovered just prior to the release was going to be published, the release
> was held off. Fixes are in and another RC should be out expected soon. RC
> typically takes about a week or two to get approved and published.-roshan
>    On Friday, February 22, 2019, 10:25:39 PM PST, gita gita <
> gita1git...@gmail.com> wrote:
>
>  Hey,
>
> Any updates, any estimated date for the release, Thanks a Ton.
>
> On 2019/02/03 22:39:26, "P. Taylor Goetz"  wrote:
> > This vote is now closed and passes with 3 binding +1 votes, 2
> non-binding +1 votes and no -1 or 0 votes.>
> >
> > I will push the release and announce after the 24 hour waiting period.>
> >
> > -Taylor>
> >
> > > On Jan 8, 2019, at 3:18 PM, P. Taylor Goetz  wrote:>
> > > >
> > > This is a call to vote on releasing Apache Storm 1.2.3 (rc1)>
> > > >
> > > Full list of changes in this release:>
> > > >
> > >
> https://dist.apache.org/repos/dist/dev/storm/apache-storm-1.2.3-rc1/RELEASE_NOTES.html>
>
> > > >
> > > The tag/commit to be voted upon is v1.2.3:>
> > > >
> > >
> https://git-wip-us.apache.org/repos/asf?p=storm.git;a=tree;h=5eaaa51591f2e4dc3e31f22bcc581af4a1f39c03;hb=6ba98b215857656e0186887b5d1a6a5aceee10c4>
>
> > > >
> > > The source archive being voted upon can be found here:>
> > > >
> > >
> https://dist.apache.org/repos/dist/dev/storm/apache-storm-1.2.3-rc1/apache-storm-1.2.3-src.tar.gz>
>
> > > >
> > > Other release files, signatures and digests can be found here:>
> > > >
> > > https://dist.apache.org/repos/dist/dev/storm/apache-storm-1.2.3-rc1/>
> > > >
> > > The release artifacts are signed with the following key:>
> > > >
> > >
> https://git-wip-us.apache.org/repos/asf?p=storm.git;a=blob_plain;f=KEYS;hb=22b832708295fa2c15c4f3c70ac0d2bc6fded4bd>
>
> > > >
> > > The Nexus staging repository for this release is:>
> > > >
> > > https://repository.apache.org/content/repositories/orgapachestorm-1074>
>
> > > >
> > > Please vote on releasing this package as Apache Storm 1.2.3.>
> > > >
> > > When voting, please list the actions taken to verify the release.>
> > > >
> > > This vote will be open for at least 72 hours.>
> > > >
> > > [ ] +1 Release this package as Apache Storm 1.2.3>
> > > [ ]  0 No opinion>
> > > [ ] -1 Do not release this package because...>
> > > >
> > > Thanks to everyone who contributed to this release.>
> > > >
> > > -Taylor>
> >
> >





Re: [RESULT] [VOTE] Release Apache Storm 1.2.3 (rc1)

2019-02-27 Thread Roshan Naik
 Gita,    Sorry for the late response. Due to some impt issues being discovered 
just prior to the release was going to be published, the release was held off. 
Fixes are in and another RC should be out expected soon. RC typically takes 
about a week or two to get approved and published.-roshan
On Friday, February 22, 2019, 10:25:39 PM PST, gita gita 
 wrote:  
 
 Hey, 

Any updates, any estimated date for the release, Thanks a Ton.

On 2019/02/03 22:39:26, "P. Taylor Goetz"  wrote: 
> This vote is now closed and passes with 3 binding +1 votes, 2 non-binding +1 
> votes and no -1 or 0 votes.> 
> 
> I will push the release and announce after the 24 hour waiting period.> 
> 
> -Taylor> 
> 
> > On Jan 8, 2019, at 3:18 PM, P. Taylor Goetz  wrote:> 
> > > 
> > This is a call to vote on releasing Apache Storm 1.2.3 (rc1)> 
> > > 
> > Full list of changes in this release:> 
> > > 
> > https://dist.apache.org/repos/dist/dev/storm/apache-storm-1.2.3-rc1/RELEASE_NOTES.html>
> >  
> > > 
> > The tag/commit to be voted upon is v1.2.3:> 
> > > 
> > https://git-wip-us.apache.org/repos/asf?p=storm.git;a=tree;h=5eaaa51591f2e4dc3e31f22bcc581af4a1f39c03;hb=6ba98b215857656e0186887b5d1a6a5aceee10c4>
> >  
> > > 
> > The source archive being voted upon can be found here:> 
> > > 
> > https://dist.apache.org/repos/dist/dev/storm/apache-storm-1.2.3-rc1/apache-storm-1.2.3-src.tar.gz>
> >  
> > > 
> > Other release files, signatures and digests can be found here:> 
> > > 
> > https://dist.apache.org/repos/dist/dev/storm/apache-storm-1.2.3-rc1/> 
> > > 
> > The release artifacts are signed with the following key:> 
> > > 
> > https://git-wip-us.apache.org/repos/asf?p=storm.git;a=blob_plain;f=KEYS;hb=22b832708295fa2c15c4f3c70ac0d2bc6fded4bd>
> >  
> > > 
> > The Nexus staging repository for this release is:> 
> > > 
> > https://repository.apache.org/content/repositories/orgapachestorm-1074> 
> > > 
> > Please vote on releasing this package as Apache Storm 1.2.3.> 
> > > 
> > When voting, please list the actions taken to verify the release.> 
> > > 
> > This vote will be open for at least 72 hours.> 
> > > 
> > [ ] +1 Release this package as Apache Storm 1.2.3> 
> > [ ]  0 No opinion> 
> > [ ] -1 Do not release this package because...> 
> > > 
> > Thanks to everyone who contributed to this release.> 
> > > 
> > -Taylor> 
> 
>   

Re: [VOTE] Release Apache Storm 2.0.0 (rc4)

2019-01-31 Thread Roshan Naik
Hi Taylor,  Looks like this vote and the one for v1.2.3 has been open for quite 
a while. Any reason to not close it and move forward ?Roshan


Sent from Yahoo Mail for iPhone


On Wednesday, January 30, 2019, 9:25 AM, Hugo Louro  wrote:

+1 to continuing with release candidate.

On Wed, Jan 30, 2019 at 8:22 AM Govind Menon  wrote:

> +1 for continuing with the RC
>
> On Tue, 29 Jan 2019 at 16:49, Arun Mahadevan  wrote:
>
> > Agree with continuing with the RC if this is not a blocker.
> >
> > On Tue, 29 Jan 2019 at 12:29, Roshan Naik  >
> > wrote:
> >
> > >  Correct me if I am wrong, this seems to be a bug with a workaround but
> > > not an exploitable security hole. ? if this is not a security hole, and
> > the
> > > workaround is realistic then we should go ahead with the current RC
> > > IMO.-roshan
> > >    On Tuesday, January 29, 2019, 10:26:53 AM PST, Kishorkumar Patil <
> > > kishorvpa...@apache.org> wrote:
> > >
> > >  Aaron,
> > > Thank you for patch and suggesting the work around in the mean time.
> The
> > PR
> > > for STORM-3317 is merged into master now.
> > > Considering the work around exists for STORM-3317, I am open to either
> go
> > > ahead with current RC or create a new one.
> > >
> > > Thanks,
> > > Kishor
> > >
> > > On Tue, Jan 29, 2019 at 11:19 AM Aaron Gresch 
> wrote:
> > >
> > > > The workaround for STORM-3317 is to force your
> > > > java.security.auth.login.config file on the launcher box to remain in
> > the
> > > > same location as where it is hosted on the supervisors.
> > > >
> > > >
> > > > On Mon, Jan 28, 2019 at 10:10 AM Aaron Gresch 
> > wrote:
> > > >
> > > > >
> > > > > Not sure if it affects the release, but STORM-3317 is a new bug in
> > 2.0
> > > > > where if your launcher box has the java.security.auth.login.config
> > > file
> > > > in
> > > > > a different location than the supervisors, uploading credentials
> will
> > > not
> > > > > work.
> > > > >
> > > > > A PR is available that fixes the issue.
> > > > >
> > > > >
> > > > >
> > > > > On Tue, Jan 8, 2019 at 1:03 PM P. Taylor Goetz 
> > > > wrote:
> > > > >
> > > > >> This is a call to vote on releasing Apache Storm 2.0.0 (rc4)
> > > > >>
> > > > >> Full list of changes in this release:
> > > > >>
> > > > >>
> > > > >>
> > > >
> > >
> >
> https://dist.apache.org/repos/dist/dev/storm/apache-storm-2.0.0-rc4/RELEASE_NOTES.html
> > > > >>
> > > > >> The tag/commit to be voted upon is v2.0.0:
> > > > >>
> > > > >>
> > > > >>
> > > >
> > >
> >
> https://git-wip-us.apache.org/repos/asf?p=storm.git;a=tree;h=1eece73e8c9ed7f41d2f20f727bc7f644c499360;hb=ddee8decac57d1a4a0aa23cc76066609a2abc8d2
> > > > >>
> > > > >> The source archive being voted upon can be found here:
> > > > >>
> > > > >>
> > > > >>
> > > >
> > >
> >
> https://dist.apache.org/repos/dist/dev/storm/apache-storm-2.0.0-rc4/apache-storm-2.0.0-src.tar.gz
> > > > >>
> > > > >> Other release files, signatures and digests can be found here:
> > > > >>
> > > > >>
> > https://dist.apache.org/repos/dist/dev/storm/apache-storm-2.0.0-rc4/
> > > > >>
> > > > >> The release artifacts are signed with the following key:
> > > > >>
> > > > >>
> > > > >>
> > > >
> > >
> >
> https://git-wip-us.apache.org/repos/asf?p=storm.git;a=blob_plain;f=KEYS;hb=22b832708295fa2c15c4f3c70ac0d2bc6fded4bd
> > > > >>
> > > > >> The Nexus staging repository for this release is:
> > > > >>
> > > > >>
> > > https://repository.apache.org/content/repositories/orgapachestorm-1073
> > > > >>
> > > > >> Please vote on releasing this package as Apache Storm 2.0.0.
> > > > >>
> > > > >> When voting, please list the actions taken to verify the release.
> > > > >>
> > > > >> This vote will be open for at least 72 hours.
> > > > >>
> > > > >> [ ] +1 Release this package as Apache Storm 2.0.0
> > > > >> [ ]  0 No opinion
> > > > >> [ ] -1 Do not release this package because...
> > > > >>
> > > > >> Thanks to everyone who contributed to this release.
> > > > >>
> > > > >> -Taylor
> > > > >
> > > > >
> > > >
> > >
> >
>





Re: [VOTE] Release Apache Storm 2.0.0 (rc4)

2019-01-29 Thread Roshan Naik
 Correct me if I am wrong, this seems to be a bug with a workaround but not an 
exploitable security hole. ? if this is not a security hole, and the workaround 
is realistic then we should go ahead with the current RC IMO.-roshan
On Tuesday, January 29, 2019, 10:26:53 AM PST, Kishorkumar Patil 
 wrote:  
 
 Aaron,
Thank you for patch and suggesting the work around in the mean time. The PR
for STORM-3317 is merged into master now.
Considering the work around exists for STORM-3317, I am open to either go
ahead with current RC or create a new one.

Thanks,
Kishor

On Tue, Jan 29, 2019 at 11:19 AM Aaron Gresch  wrote:

> The workaround for STORM-3317 is to force your
> java.security.auth.login.config file on the launcher box to remain in the
> same location as where it is hosted on the supervisors.
>
>
> On Mon, Jan 28, 2019 at 10:10 AM Aaron Gresch  wrote:
>
> >
> > Not sure if it affects the release, but STORM-3317 is a new bug in 2.0
> > where if your launcher box has the java.security.auth.login.config  file
> in
> > a different location than the supervisors, uploading credentials will not
> > work.
> >
> > A PR is available that fixes the issue.
> >
> >
> >
> > On Tue, Jan 8, 2019 at 1:03 PM P. Taylor Goetz 
> wrote:
> >
> >> This is a call to vote on releasing Apache Storm 2.0.0 (rc4)
> >>
> >> Full list of changes in this release:
> >>
> >>
> >>
> https://dist.apache.org/repos/dist/dev/storm/apache-storm-2.0.0-rc4/RELEASE_NOTES.html
> >>
> >> The tag/commit to be voted upon is v2.0.0:
> >>
> >>
> >>
> https://git-wip-us.apache.org/repos/asf?p=storm.git;a=tree;h=1eece73e8c9ed7f41d2f20f727bc7f644c499360;hb=ddee8decac57d1a4a0aa23cc76066609a2abc8d2
> >>
> >> The source archive being voted upon can be found here:
> >>
> >>
> >>
> https://dist.apache.org/repos/dist/dev/storm/apache-storm-2.0.0-rc4/apache-storm-2.0.0-src.tar.gz
> >>
> >> Other release files, signatures and digests can be found here:
> >>
> >> https://dist.apache.org/repos/dist/dev/storm/apache-storm-2.0.0-rc4/
> >>
> >> The release artifacts are signed with the following key:
> >>
> >>
> >>
> https://git-wip-us.apache.org/repos/asf?p=storm.git;a=blob_plain;f=KEYS;hb=22b832708295fa2c15c4f3c70ac0d2bc6fded4bd
> >>
> >> The Nexus staging repository for this release is:
> >>
> >> https://repository.apache.org/content/repositories/orgapachestorm-1073
> >>
> >> Please vote on releasing this package as Apache Storm 2.0.0.
> >>
> >> When voting, please list the actions taken to verify the release.
> >>
> >> This vote will be open for at least 72 hours.
> >>
> >> [ ] +1 Release this package as Apache Storm 2.0.0
> >> [ ]  0 No opinion
> >> [ ] -1 Do not release this package because...
> >>
> >> Thanks to everyone who contributed to this release.
> >>
> >> -Taylor
> >
> >
>
  

Re: Storm 2.0 blogs ?

2019-01-24 Thread Roshan Naik
 Here is a snippet for the rearchitecture. Keeping it short here as follow up 
blogs will have more details:

1) New High Performance Core:Storm 2.0 introduces a new core, designed to push 
boundaries on throughput, latency and energy consumption while maintaining 
backward compatibility. It features a leaner threading model, a blazing fast 
messaging subsystem and a lightweight back pressure model.
The new engine was motivated by the observation that existing hardware remains 
capable of performing much better than what the best streaming engines deliver. 
Storm 2.0 is the first streaming engine capable of breaking the 1 microsecond 
latency barrier for transfers between two operators. It can sustain very high 
throughputs and also deliver better energy efficiency. Details on the new 
architecture and its performance will be covered in upcoming blogs.


On Wednesday, January 23, 2019, 10:01:56 AM PST, Stig Rohde Døssing 
 wrote:  
 
 We should probably also highlight in the release notes that Java 8 is now
the minimum.

Here is the blurb for storm-kafka-client:

# Kafka integration changes

## Removal of storm-kafka
The most significant change to Storm's Kafka integration since 1.x, is that
storm-kafka has been removed. The module was deprecated a while back, due
to Kafka's deprecation of the underlying client library. Users will have to
move to the storm-kafka-client module, which uses Kafka's ´kafka-clients´
library for integration.

For the most part, the migration to storm-kafka-client is straightforward.
The documentation for storm-kafka-client contains a helpful mapping between
the old and new spout configurations. If you are using any of the
storm-kafka spouts, you will need to migrate offset checkpoints to the new
spout, to avoid the new spout starting from scratch on your partitions. You
can find a helper tool to do this at
https://github.com/apache/storm/tree/master/external/storm-kafka-migration.
You should stop your topology, run the migration tool, then redeploy your
topology with the storm-kafka-client spout.

## Move to using the KafkaConsumer.assign API
Storm-kafka-client in 1.x allowed you to use Kafka's own mechanism to
manage which spout tasks were responsible for which partitions. This
mechanism was a poor fit for Storm, and was deprecated in 1.2.0. It has
been removed entirely in 2.0
https://issues.apache.org/jira/browse/STORM-2542.

The storm-kafka-client Subscription interface has also been removed. It
offered too limited control over the subscription behavior. It has been
replaced with the TopicFilter and ManualPartitioner interfaces. Unless you
were using a custom Subscription implementation, this will likely not
affect you. If you were using a custom Subscription, the storm-kafka-client
documentation describes how to customize assignment
https://github.com/apache/storm/blob/master/docs/storm-kafka-client.md#manual-partition-assigment-advanced
.

## Other highlights
* The KafkaBolt now allows you to specify a callback that will be called
when a batch is written to Kafka
https://issues.apache.org/jira/browse/STORM-3175.
* The FirstPollOffsetStrategy behavior has been made consistent between the
non-Trident and Trident spouts. It is now always the case that
EARLIEST/LATEST only take effect on topology redeploy, and not when a
worker restarts https://issues.apache.org/jira/browse/STORM-2990.
* Storm-kafka-client now has a transactional non-opaque Trident spout
https://issues.apache.org/jira/browse/STORM-2974.
* There is a new examples module for storm-kafka-client at
https://github.com/apache/storm/tree/master/examples/storm-kafka-client-examples
.
* Deprecated methods in KafkaSpoutConfig have been removed. If you are
using one of the deprecated methods, check the Javadoc for the latest 1.2.x
release, which describes the replacement for each method.

Den ons. 23. jan. 2019 kl. 15.54 skrev P. Taylor Goetz :

> If you need to format (e.g. code examples, etc.) then markdown is fine. So
> is plain text.
>
> -Taylor
>
> > On Jan 23, 2019, at 5:24 AM, Stig Rohde Døssing 
> wrote:
> >
> > I'll write something for 5 - Kafka related changes.
> >
> > We have dropped Druid support, so 13 should be only Kinesis.
> >
> > Which format should the blurbs be written in? (Markdown?)
> >
> > Den ons. 23. jan. 2019 kl. 08.57 skrev Roshan Naik
> > :
> >
> >> Like Taylor’s suggestion of collectively contributing small blurbs on
> >> features for the Release announcement. Thats the first thing people
> look on
> >> hearing a release announcement. The jira list is not very
> understandable at
> >> first glance.
> >>
> >>
> >> Based on suggestions so far and a quick scan of the jiras in release
> notes
> >> here is a draft list in no particular order. I am sure I am missing a
> few
> >> impt ones. This can be pruned or modified 

Re: Storm 2.0 blogs ?

2019-01-22 Thread Roshan Naik
 Like Taylor’s suggestion of collectively contributing small blurbs on features 
for the Release announcement. Thats the first thing people look on hearing a 
release announcement. The jira list is not very understandable at first glance.


Based on suggestions so far and a quick scan of the jiras in release notes here 
is a draft list in no particular order. I am sure I am missing a few impt ones. 
This can be pruned or modified as needed:

1- Re-architecture - [Roshan]
2- Windowing enhancements
3- SQL enhancements
4- Metrics
5- Kafka related changes 
6- Security (nimbus admin groups, delegation tokens, optional impersonation)
7- PMML (Machine Learning) support.
8- Streams API
9- Module restructuring & dependency mitigation 
10- Java porting
11- DRPC cmd line
12- Lambda support
13- New spouts: Kinesis & Druid ?
14- Changes to deployment and cli submission 
15- RAS changes
16- Trident enhancements
17- New Admin cmds to debug cluster state
18 ... others ?

Please pick the topics you can contribute blurbs for. I have put my name 
against one. It will help Taylor aggregate them and do the necessary final 
edits. 


-Roshan


 



Storm 2.0 blogs ?

2019-01-21 Thread Roshan Naik
Now that  2.0 has all the votes it needs to move forward, maybe a good time to 
think of some blogs to go with this long awaited release. 
 Some potential topics that come to mind are:
1- Overview of new features and major changes since 1.x2- Re-architecture 
(messaging, threading, back pressure)3- Micro benchmarks4- Revisit the famous 
Yahoo benchmark5- Window state persistence6- SQL enhancements7- new Metrics 
stuff8- Kafka related changes9- Security10- An area you have worked on ?11- 
Other ideas ?
Anyone interested in contributing blogs ? 
FYI: I am working on content for topics 2 & 3.

-roshan

Re: [NOTICE] Migration of storm repos to gitbox is done

2019-01-21 Thread Roshan Naik
Thanks Ethan. 


Sent from Yahoo Mail for iPhone


On Sunday, January 20, 2019, 7:33 PM, Ethan Li  
wrote:

Hi Team,

The migration to https://gitbox.apache.org/ is done 
(https://issues.apache.org/jira/browse/INFRA-17705 
). 

Thanks,

- Ethan




Re: [VOTE] Release Apache Storm 2.0.0 (rc4)

2019-01-20 Thread Roshan Naik
 +1
Sorry .. it was a false alarm. It come down to a config issue. The default 
configs in Storm 2 are tuned for best latency. 
On bumping up the transfer buffer & batch settings for high throughput :
- topology.transfer.batch.size=5k (default=1)  and - 
topology.transfer.buffer.size=20k (default=1k)
Was able to obtain a very solid  ~4 mill/sec inter worker throughput. 
FYI: Does not matter now, but i noticed that I accidentally quoted throughput 
of 1.6mill/sec but it was actually only ~350 k/sec with default transfer buffer 
settings  .. which was alarmingly low. 
On Saturday, January 19, 2019, 10:45:28 PM PST, Jungtaek Lim 
 wrote:  
 
 Hello all,

+1 (binding)

> source

- verify file (MD5, SHA)
-- source, tar.gz : OK
-- source, zip : OK

- extract file
-- source, tar.gz : OK
-- source, zip : OK

- diff-ing extracted files between tar.gz and zip : OK

- build source with JDK 8 (-Pall-tests && -Pexternals)
-- source, tar.gz : OK

- build source dist
-- source, tar.gz : OK

- build binary dist
-- source, tar.gz : OK

> binary

- verify file (MD5, SHA)
-- binary, tar.gz : OK
-- binary, zip : OK

- extract file
-- binary, tar.gz : OK
-- binary, zip : OK

- diff-ing extracted files between tar.gz and zip : OK

- launch daemons : OK

- run ExclamationTopology (local) : OK

- run RollingTopWords (remote) : OK
  - activate / deactivate / rebalance / kill : OK
  - logviewer (worker dir, daemon dir) : OK
  - change log level : OK
  - thread dump, heap dump, restart worker : OK
  - log search : OK

I don't think perf. degrade is a blocker for this release, since we are
still providing far better performance then 1.x. As Roshan is planning to
address this, that could be fixed in 2.0.x hopefully.

Thanks,
Jungtaek Lim (heartSaVioR)

2019년 1월 19일 (토) 오전 5:23, Roshan Naik 님이 작성:

>  Was verifying the performance on 2.0 and noticed that the inter worker
> messaging has fallen off a cliff  With ConstSpoutNullBoltTopo (producer
> batch size=1k, ackers=0, workers=2) it was ~3.2mill/sec down  and now its
> down to  ~1.6mill/sec. Something has impacted this recently. Will try to
> narrow down the issue by tomorrow hopefully. Single worker numbers look
> good.
> -roshan
>    On Friday, January 11, 2019, 2:46:14 AM PST, Stig Rohde Døssing <
> stigdoess...@gmail.com> wrote:
>
>  Thanks Arun,
>
> I don't have an opinion on it, just wanted to make sure that it wasn't an
> oversight.
>
> +1
>
> Set up Storm from the binary zip file
> Built storm-kafka-client-examples from the zip while pointing Maven at the
> Nexus repo
> Ran one of the example Kafka topologies with a single-node cluster against
> a local Kafka. Verified there were no errors in the logs, clicked around in
> UI.
>
> Den tor. 10. jan. 2019 kl. 22.25 skrev Arun Mahadevan :
>
> > This is for users to use the "auto credentials" mechanism (delegation
> > tokens) with HDFS/Hive/Hbase.
> >
> > We have been shipping it since 1.x (I think since 1.2.0 release) so that
> > users can just add that directory to class path rather than building it
> > separately to get the right dependencies. We could consider removing it
> > from the main binary and ship it separately but it will need changes to
> the
> > build, release and documentation and users will need to download and
> > install it separately.
> >
> >
> > Thanks,
> > Arun
> >
> > On Thu, 10 Jan 2019 at 10:28, Stig Rohde Døssing  >
> > wrote:
> >
> > > I think this was remarked on by Roshan in the last RC, but the binary
> > > distribution has become significantly larger since 1.x. It looks like
> > this
> > > is down to storm-autocreds not being added to the exclusion list in
> > > storm-dist/binary/final-package/src/main/assembly/binary.xml.
> > >
> > > Since the module isn't excluded, external/storm-autocreds contains the
> > > module jar, plus all dependency jars. Is this an accident, or do we
> want
> > to
> > > include these jars in the distribution?
> > >
> > > Den ons. 9. jan. 2019 kl. 19.48 skrev Ethan Li <
> > ethanopensou...@gmail.com
> > > >:
> > >
> > > > +1
> > > >
> > > > - Built from the src, ran all the unit tests and integration tests.
> > > > - Set up a single-node cluster and submit ThroughputVsLatency
> topology.
> > > > - Checked the UI.
> > > > They look good.
> > > >
> > > > Thanks
> > > > Ethan
> > > >
> > > > > On Jan 9, 2019, at 8:48 AM, Bobby Evans  wrote:
> > > > >
> > > > > +1 built from the git tag.  Ran all of the unit tests and ran some
> > &g

Re: [VOTE] Release Apache Storm 2.0.0 (rc4)

2019-01-18 Thread Roshan Naik
 Was verifying the performance on 2.0 and noticed that the inter worker 
messaging has fallen off a cliff  With ConstSpoutNullBoltTopo (producer 
batch size=1k, ackers=0, workers=2) it was ~3.2mill/sec down  and now its down 
to  ~1.6mill/sec. Something has impacted this recently. Will try to narrow down 
the issue by tomorrow hopefully. Single worker numbers look good.  
-roshan
On Friday, January 11, 2019, 2:46:14 AM PST, Stig Rohde Døssing 
 wrote:  
 
 Thanks Arun,

I don't have an opinion on it, just wanted to make sure that it wasn't an
oversight.

+1

Set up Storm from the binary zip file
Built storm-kafka-client-examples from the zip while pointing Maven at the
Nexus repo
Ran one of the example Kafka topologies with a single-node cluster against
a local Kafka. Verified there were no errors in the logs, clicked around in
UI.

Den tor. 10. jan. 2019 kl. 22.25 skrev Arun Mahadevan :

> This is for users to use the "auto credentials" mechanism (delegation
> tokens) with HDFS/Hive/Hbase.
>
> We have been shipping it since 1.x (I think since 1.2.0 release) so that
> users can just add that directory to class path rather than building it
> separately to get the right dependencies. We could consider removing it
> from the main binary and ship it separately but it will need changes to the
> build, release and documentation and users will need to download and
> install it separately.
>
>
> Thanks,
> Arun
>
> On Thu, 10 Jan 2019 at 10:28, Stig Rohde Døssing 
> wrote:
>
> > I think this was remarked on by Roshan in the last RC, but the binary
> > distribution has become significantly larger since 1.x. It looks like
> this
> > is down to storm-autocreds not being added to the exclusion list in
> > storm-dist/binary/final-package/src/main/assembly/binary.xml.
> >
> > Since the module isn't excluded, external/storm-autocreds contains the
> > module jar, plus all dependency jars. Is this an accident, or do we want
> to
> > include these jars in the distribution?
> >
> > Den ons. 9. jan. 2019 kl. 19.48 skrev Ethan Li <
> ethanopensou...@gmail.com
> > >:
> >
> > > +1
> > >
> > > - Built from the src, ran all the unit tests and integration tests.
> > > - Set up a single-node cluster and submit ThroughputVsLatency topology.
> > > - Checked the UI.
> > > They look good.
> > >
> > > Thanks
> > > Ethan
> > >
> > > > On Jan 9, 2019, at 8:48 AM, Bobby Evans  wrote:
> > > >
> > > > +1 built from the git tag.  Ran all of the unit tests and ran some
> > manual
> > > > tests they all passed.
> > > >
> > > > Thanks,
> > > >
> > > > Bobby
> > > >
> > > > On Tue, Jan 8, 2019 at 6:30 PM Xin Wang 
> > wrote:
> > > >
> > > >> +1
> > > >>
> > > >> Built it and ran all of the tests.  Everything passed.
> > > >>
> > > >> -Xin
> > > >>
> > > >> Kishorkumar Patil  于2019年1月9日周三 上午5:08写道:
> > > >>
> > > >>> +1
> > > >>>
> > > >>> - built from source code and deployment works.
> > > >>> -  Ran some of the tests for UI, DRPC, ThroughputVsLatency
> > > >>> -  Validated UI bugs reported in the recent past are fixed in this
> > > >> version
> > > >>>
> > > >>> -Kishor
> > > >>>
> > > >>>
> > > >>> On Tue, Jan 8, 2019 at 2:29 PM Arun Mahadevan 
> > > wrote:
> > > >>>
> > >  +1
> > > 
> > >  - Downloaded the binaries and validated signatures.
> > >  - Deployed the binaries, ran some sample topologies and checked
> the
> > > UI.
> > >  - Ran top level build using the source zip.
> > > 
> > >  Thanks,
> > >  Arun
> > > 
> > > 
> > >  On Tue, 8 Jan 2019 at 11:03, P. Taylor Goetz 
> > > >> wrote:
> > > 
> > > > This is a call to vote on releasing Apache Storm 2.0.0 (rc4)
> > > >
> > > > Full list of changes in this release:
> > > >
> > > >
> > > >
> > > 
> > > >>>
> > > >>
> > >
> >
> https://dist.apache.org/repos/dist/dev/storm/apache-storm-2.0.0-rc4/RELEASE_NOTES.html
> > > >
> > > > The tag/commit to be voted upon is v2.0.0:
> > > >
> > > >
> > > >
> > > 
> > > >>>
> > > >>
> > >
> >
> https://git-wip-us.apache.org/repos/asf?p=storm.git;a=tree;h=1eece73e8c9ed7f41d2f20f727bc7f644c499360;hb=ddee8decac57d1a4a0aa23cc76066609a2abc8d2
> > > >
> > > > The source archive being voted upon can be found here:
> > > >
> > > >
> > > >
> > > 
> > > >>>
> > > >>
> > >
> >
> https://dist.apache.org/repos/dist/dev/storm/apache-storm-2.0.0-rc4/apache-storm-2.0.0-src.tar.gz
> > > >
> > > > Other release files, signatures and digests can be found here:
> > > >
> > > >
> > https://dist.apache.org/repos/dist/dev/storm/apache-storm-2.0.0-rc4/
> > > >
> > > > The release artifacts are signed with the following key:
> > > >
> > > >
> > > >
> > > 
> > > >>>
> > > >>
> > >
> >
> https://git-wip-us.apache.org/repos/asf?p=storm.git;a=blob_plain;f=KEYS;hb=22b832708295fa2c15c4f3c70ac0d2bc6fded4bd
> > > >
> > > > The Nexus staging repository for this release is:
> > > >
> > > >
> > > >>
> > 

Re: New Storm Committer and PMC Member

2019-01-09 Thread Roshan Naik
Congratulations Govind. Roshan


Sent from Yahoo Mail for iPhone


On Wednesday, January 9, 2019, 10:47 AM, Ethan Li  
wrote:

Congratulations! Govind. Well deserved!

Ethan

> On Jan 9, 2019, at 12:40 PM, Hugo Louro  wrote:
> 
> Congratulations Govind. Very well deserved. Thank you for all your
> contributions and dedication to the Storm project.
> 
> Best,
> Hugo
> 
> On Wed, Jan 9, 2019 at 10:36 AM Bobby Evans  wrote:
> 
>> I am happy to announce that Govind Menon has just been added as the latest
>> Committer and PMC member to the Apache Storm Project.  Please join me in
>> congratulating him on this and thanking him for his contributions so far.
>> 
>> Thanks,
>> 
>> Bobby Evans
>> 






Re: Storm 2.0.0 release?

2019-01-04 Thread Roshan Naik
+1


Sent from Yahoo Mail for iPhone


On Friday, January 4, 2019, 9:02 PM, Ethan Li  wrote:

+1 on this 

Ethan Li

> On Jan 4, 2019, at 20:19, P. Taylor Goetz  wrote:
> 
> If no one objects, I’ll kick off a release candidate.
> 
> -Taylor
> 
>> On Jan 4, 2019, at 9:06 PM, saurabh mimani  wrote:
>> 
>> Hey, Any approximate date for 2.0 release given there are no blockers?
>> 
>> 
>> 
>> Best Regards
>> 
>> Saurabh Kumar Mimani
>> 
>> 
>> 
>> 
>> On Sat, Dec 22, 2018 at 1:13 AM Stig Rohde Døssing 
>> wrote:
>> 
>>> Looks good to me, the blockers list in JIRA is empty for 2.0.0.
>>> 
>>>> Den fre. 21. dec. 2018 kl. 19.52 skrev Bobby Evans :
>>>> 
>>>> I think all of the blockers are in now.  Please take a look and
>>> hopefully,
>>>> we can get a release out soon.
>>>> 
>>>> Thanks,
>>>> 
>>>> Bobby
>>>> 
>>>> 
>>>>> On Fri, Dec 14, 2018 at 9:02 AM Bobby Evans  wrote:
>>>>> 
>>>>> Sorry I was out at a conference for the past week, and have been heads
>>>> down
>>>>> on a different project for a while before that.  I'll respond to the
>>>> JIRA.
>>>>> I am happy to let it go in.
>>>>> 
>>>>> Thanks,
>>>>> 
>>>>> Bobby
>>>>> 
>>>>> On Thu, Dec 13, 2018 at 1:07 PM Stig Rohde Døssing <
>>>> stigdoess...@gmail.com
>>>>>> 
>>>>> wrote:
>>>>> 
>>>>>> I think STORM-2990/3279 is ready. Bobby had a question (
>>>>>> https://github.com/apache/storm/pull/2907#discussion_r234329136)
>>>>> regarding
>>>>>> whether Kafka offsets loop, but I wasn't sure where he was going with
>>>> it,
>>>>>> so I didn't want to merge prematurely.
>>>>>> 
>>>>>> I agree that we can postpone STORM-2720. As far as I know it's
>>> waiting
>>>>> for
>>>>>> STORM-2990 to go in, since it's going to be touching the same code.
>>>>>> 
>>>>>> Den tor. 13. dec. 2018 kl. 19.54 skrev Roshan Naik
>>>>>> :
>>>>>> 
>>>>>>> Sounds like -  https://github.com/apache/storm/pull/2913
>>>> (STORM-3290)
>>>>>> is
>>>>>>> merged -  https://github.com/apache/storm/pull/2908 (STORM-3276)
>>> is
>>>>>>> nearly complete and may need some small tweaks. -
>>>>>>> https://github.com/apache/storm/pull/2907  (STORM-2990,
>>> STORM-3279)
>>>>>>> appears ready to be committed ?
>>>>>>> 
>>>>>>> and- https://github.com/apache/storm/pull/2911   - (STORM-2720)
>>>>> seems a
>>>>>>> bit inactive and may not be critical enough to wait on.
>>>>>>> 
>>>>>>> -roshan
>>>>>>>  On Monday, November 26, 2018, 9:49:30 AM PST, Stig Rohde
>>> Døssing
>>>> <
>>>>>>> stigdoess...@gmail.com> wrote:
>>>>>>> 
>>>>>>> I would like to get at least
>>>>> https://github.com/apache/storm/pull/2913
>>>>>>> (breaking changes) and https://github.com/apache/storm/pull/2908
>>>>>>> (regression) in.
>>>>>>> 
>>>>>>> I think it would be nice to also get
>>>>>>> https://github.com/apache/storm/pull/2907 and
>>>>>>> https://github.com/apache/storm/pull/2911 in, but if we're in a
>>>> hurry
>>>>>> they
>>>>>>> could go in the next release.
>>>>>>> 
>>>>>>> Den man. 26. nov. 2018 kl. 17.37 skrev Julien Nioche <
>>>>>>> lists.digitalpeb...@gmail.com>:
>>>>>>> 
>>>>>>>> Hi devs,
>>>>>>>> 
>>>>>>>> Is there anything blocking the release of Storm 2.0? Any idea of
>>>> when
>>>>>> it
>>>>>>>> could happen?
>>>>>>>> 
>>>>>>>> Thanks
>>>>>>>> 
>>>>>>>> Julien
>>>>>>>> 
>>>>>>>> --
>>>>>>>> 
>>>>>>>> *Open Source Solutions for Text Engineering*
>>>>>>>> 
>>>>>>>> http://www.digitalpebble.com
>>>>>>>> http://digitalpebble.blogspot.com/
>>>>>>>> #digitalpebble <http://twitter.com/digitalpebble>
>>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>> 
>>>> 
>>> 




Re: Storm 2.0.0 release?

2018-12-13 Thread Roshan Naik
 Sounds like -  https://github.com/apache/storm/pull/2913 (STORM-3290) is 
merged -  https://github.com/apache/storm/pull/2908 (STORM-3276) is nearly 
complete and may need some small tweaks. - 
https://github.com/apache/storm/pull/2907  (STORM-2990, STORM-3279) appears 
ready to be committed ?

and- https://github.com/apache/storm/pull/2911   - (STORM-2720)  seems a bit 
inactive and may not be critical enough to wait on.

-roshan
On Monday, November 26, 2018, 9:49:30 AM PST, Stig Rohde Døssing 
 wrote:  
 
 I would like to get at least https://github.com/apache/storm/pull/2913
(breaking changes) and https://github.com/apache/storm/pull/2908
(regression) in.

I think it would be nice to also get
https://github.com/apache/storm/pull/2907 and
https://github.com/apache/storm/pull/2911 in, but if we're in a hurry they
could go in the next release.

Den man. 26. nov. 2018 kl. 17.37 skrev Julien Nioche <
lists.digitalpeb...@gmail.com>:

> Hi devs,
>
> Is there anything blocking the release of Storm 2.0? Any idea of when it
> could happen?
>
> Thanks
>
> Julien
>
> --
>
> *Open Source Solutions for Text Engineering*
>
> http://www.digitalpebble.com
> http://digitalpebble.blogspot.com/
> #digitalpebble 
>
  

Re: [VOTE] Release Apache Storm 2.0.0 (rc3)

2018-10-21 Thread Roshan Naik
 external/storm-autocreds  appears to have grown a lot.
In 1.2.2:   93mb,  113 filesIn 2.0.0 :  173mb,  204 files.

fyi: let me know if the formatting in my prev mail is messed up ... i can 
resend.
-roshan

On Sunday, October 21, 2018, 5:54:36 PM PDT, Roshan Naik 
 wrote:  
 
  The 2.0 compressed tar.gz is ~270 MB in comparison to 1.2.2 jar which stands 
at ~160MB. Seems like a big jump and wanted to make sure there is no unintended 
bloat occurring.
Listing of files larger than 5MB in both (ordered by size):
 23M  =>  apache-storm-1.2.2/lib/storm-core-1.2.2.jar 21M  =>  
apache-storm-1.2.2/toollib/storm-kafka-monitor-1.2.2.jar 17M  =>  
apache-storm-1.2.2/external/storm-autocreds/hive-exec-0.14.0.jar7.5M  =>  
apache-storm-1.2.2/external/storm-autocreds/hadoop-hdfs-2.6.1.jar6.1M  =>  
apache-storm-1.2.2/external/storm-autocreds/groovy-all-2.1.6.jar5.9M  =>  
apache-storm-1.2.2/external/storm-submit-tools/storm-submit-tools-1.2.2.jar


 33M  =>  apache-storm-2.0.0/external/storm-autocreds/hive-exec-2.3.3.jar 16M  
=>  **apache-storm-2.0.0/external/storm-autocreds/fastutil-6.5.6.jar 14M  =>  
**apache-storm-2.0.0/lib-worker/shaded-deps-2.0.0.jar 14M  =>  
**apache-storm-2.0.0/lib/shaded-deps-2.0.0.jar 13M  =>  
**apache-storm-2.0.0/lib/rocksdbjni-5.8.6.jar7.9M  =>  
apache-storm-2.0.0/lib-tools/storm-kafka-monitor/storm-kafka-monitor-2.0.0.jar7.7M
  =>  
**apache-storm-2.0.0/external/storm-autocreds/hive-metastore-2.3.3.jar7.5M  =>  
apache-storm-2.0.0/external/storm-autocreds/hadoop-hdfs-2.6.1.jar6.7M  =>  
apache-storm-2.0.0/external/storm-autocreds/groovy-all-2.4.4.jar5.7M  =>  
apache-storm-2.0.0/lib-tools/submit-tools/storm-submit-tools-2.0.0.jar


Observations:- Files news to 2.0 have put ** before the large files that are 
new in 2.0. Please take a look and see if any of these are of concern. - To me, 
the inclusion of hive-metastore jar seems a bit odd.- The newer Hive-exec is 
definitely much larger. May be ok.- Storm kafka monitor is much smaller now.


    On Friday, October 19, 2018, 3:40:36 PM PDT, Ethan Li 
 wrote:  
 
 +1 

Build from the source. Set up a single-node secure cluster and tested with 
word-count topology.  Viewed UI and log viewer.

Found some minor issues
 https://issues.apache.org/jira/browse/STORM-3265 
<https://issues.apache.org/jira/browse/STORM-3265> 
 https://issues.apache.org/jira/browse/STORM-3264 
<https://issues.apache.org/jira/browse/STORM-3264> 
https://issues.apache.org/jira/browse/STORM-3266 
<https://issues.apache.org/jira/browse/STORM-3266>

They are not blockers. 

Thanks
-Ethan


> On Oct 16, 2018, at 2:48 PM, P. Taylor Goetz  wrote:
> 
> This is a call to vote on releasing Apache Storm 2.0.0 (rc3)
> 
> Full list of changes in this release:
> 
> https://dist.apache.org/repos/dist/dev/storm/apache-storm-2.0.0-rc3/RELEASE_NOTES.html
> 
> The tag/commit to be voted upon is v2.0.0:
> 
> https://git-wip-us.apache.org/repos/asf?p=storm.git;a=commit;h=d2d6f40344e6cc92ab07f3a462d577ef6b61f8b1
> 
> The source archive being voted upon can be found here:
> 
> https://dist.apache.org/repos/dist/dev/storm/apache-storm-2.0.0-rc3/apache-storm-2.0.0-src.tar.gz
> 
> Other release files, signatures and digests can be found here:
> 
> https://dist.apache.org/repos/dist/dev/storm/apache-storm-2.0.0-rc3/
> 
> The release artifacts are signed with the following key:
> 
> https://git-wip-us.apache.org/repos/asf?p=storm.git;a=blob_plain;f=KEYS;hb=22b832708295fa2c15c4f3c70ac0d2bc6fded4bd
> 
> The Nexus staging repository for this release is:
> 
> https://repository.apache.org/content/repositories/orgapachestorm-1072
> 
> Please vote on releasing this package as Apache Storm 2.0.0.
> 
> When voting, please list the actions taken to verify the release.
> 
> This vote will be open for at least 72 hours.
> 
> [ ] +1 Release this package as Apache Storm 2.0.0
> [ ]  0 No opinion
> [ ] -1 Do not release this package because...
> 
> Thanks to everyone who contributed to this release.
> 
> -Taylor
    

Re: [VOTE] Release Apache Storm 2.0.0 (rc3)

2018-10-21 Thread Roshan Naik
 The 2.0 compressed tar.gz is ~270 MB in comparison to 1.2.2 jar which stands 
at ~160MB. Seems like a big jump and wanted to make sure there is no unintended 
bloat occurring.
Listing of files larger than 5MB in both (ordered by size):
 23M  =>  apache-storm-1.2.2/lib/storm-core-1.2.2.jar 21M  =>  
apache-storm-1.2.2/toollib/storm-kafka-monitor-1.2.2.jar 17M  =>  
apache-storm-1.2.2/external/storm-autocreds/hive-exec-0.14.0.jar7.5M  =>  
apache-storm-1.2.2/external/storm-autocreds/hadoop-hdfs-2.6.1.jar6.1M  =>  
apache-storm-1.2.2/external/storm-autocreds/groovy-all-2.1.6.jar5.9M  =>  
apache-storm-1.2.2/external/storm-submit-tools/storm-submit-tools-1.2.2.jar


 33M  =>  apache-storm-2.0.0/external/storm-autocreds/hive-exec-2.3.3.jar 16M  
=>  **apache-storm-2.0.0/external/storm-autocreds/fastutil-6.5.6.jar 14M  =>  
**apache-storm-2.0.0/lib-worker/shaded-deps-2.0.0.jar 14M  =>  
**apache-storm-2.0.0/lib/shaded-deps-2.0.0.jar 13M  =>  
**apache-storm-2.0.0/lib/rocksdbjni-5.8.6.jar7.9M  =>  
apache-storm-2.0.0/lib-tools/storm-kafka-monitor/storm-kafka-monitor-2.0.0.jar7.7M
  =>  
**apache-storm-2.0.0/external/storm-autocreds/hive-metastore-2.3.3.jar7.5M  =>  
apache-storm-2.0.0/external/storm-autocreds/hadoop-hdfs-2.6.1.jar6.7M  =>  
apache-storm-2.0.0/external/storm-autocreds/groovy-all-2.4.4.jar5.7M  =>  
apache-storm-2.0.0/lib-tools/submit-tools/storm-submit-tools-2.0.0.jar


Observations:- Files news to 2.0 have put ** before the large files that are 
new in 2.0. Please take a look and see if any of these are of concern. - To me, 
the inclusion of hive-metastore jar seems a bit odd.- The newer Hive-exec is 
definitely much larger. May be ok.- Storm kafka monitor is much smaller now.


On Friday, October 19, 2018, 3:40:36 PM PDT, Ethan Li 
 wrote:  
 
 +1 

Build from the source. Set up a single-node secure cluster and tested with 
word-count topology.  Viewed UI and log viewer.

Found some minor issues
 https://issues.apache.org/jira/browse/STORM-3265 
 
 https://issues.apache.org/jira/browse/STORM-3264 
 
https://issues.apache.org/jira/browse/STORM-3266 


They are not blockers. 

Thanks
-Ethan


> On Oct 16, 2018, at 2:48 PM, P. Taylor Goetz  wrote:
> 
> This is a call to vote on releasing Apache Storm 2.0.0 (rc3)
> 
> Full list of changes in this release:
> 
> https://dist.apache.org/repos/dist/dev/storm/apache-storm-2.0.0-rc3/RELEASE_NOTES.html
> 
> The tag/commit to be voted upon is v2.0.0:
> 
> https://git-wip-us.apache.org/repos/asf?p=storm.git;a=commit;h=d2d6f40344e6cc92ab07f3a462d577ef6b61f8b1
> 
> The source archive being voted upon can be found here:
> 
> https://dist.apache.org/repos/dist/dev/storm/apache-storm-2.0.0-rc3/apache-storm-2.0.0-src.tar.gz
> 
> Other release files, signatures and digests can be found here:
> 
> https://dist.apache.org/repos/dist/dev/storm/apache-storm-2.0.0-rc3/
> 
> The release artifacts are signed with the following key:
> 
> https://git-wip-us.apache.org/repos/asf?p=storm.git;a=blob_plain;f=KEYS;hb=22b832708295fa2c15c4f3c70ac0d2bc6fded4bd
> 
> The Nexus staging repository for this release is:
> 
> https://repository.apache.org/content/repositories/orgapachestorm-1072
> 
> Please vote on releasing this package as Apache Storm 2.0.0.
> 
> When voting, please list the actions taken to verify the release.
> 
> This vote will be open for at least 72 hours.
> 
> [ ] +1 Release this package as Apache Storm 2.0.0
> [ ]  0 No opinion
> [ ] -1 Do not release this package because...
> 
> Thanks to everyone who contributed to this release.
> 
> -Taylor
  

Re: Hadoop Contributor Meetup

2018-09-23 Thread Roshan Naik
I can do a talk on the rework of threading,  messaging and backpressure  for 
2.0 
Roshan

Sent from Yahoo Mail for iPhone


On Friday, September 21, 2018, 8:37 AM, Bobby Evans  wrote:

There is a Hadoop Contributor Meetup at Oath/Yahoo on Monday in Sunnyvale CA

https://www.meetup.com/Hadoop-Contributors/events/254012512/

This is short notice and we tend to be distributed very geographically as
an open source project, but I am going to be there and if anyone wants to
show up to talk about storm/stream processing that would be great.

Also happy to talk after the event.  just send me an email at
bo...@apache.org and we can schedule something.

Thanks,

Bobby





Re: Is the powered-by file in the Storm repository necessary?

2018-09-16 Thread Roshan Naik
 Having it in the site is sufficient IMO as well. 
On Saturday, September 15, 2018, 4:16:05 AM PDT, Stig Rohde Døssing 
 wrote:  
 
 Hi,

We have a powered-by.md file in the Storm repository's /docs directory.
There's also a powered-by file in the storm-site repo root, as well as in
each release directory.

The storm-site root powered-by is the one linked to by the Storm site, and
I don't believe it's being updated based on the Storm repo powered-by. The
Storm repo powered-by ends up in the release-specific documentation
instead, e.g.
https://storm.apache.org/releases/2.0.0-SNAPSHOT/Powered-By.html.

I think we might as well treat Powered-By.md the same as the
getting-help.md file, which seems to only be present in the storm-site
repo. I'd like to delete powered-by from the Storm repo, so we only have
the one in the storm-site root.

Any opinions?
  

Re: Regarding releasing Apache Storm 2.0.0

2018-09-14 Thread Roshan Naik
 Happy to see consensus in moving fwd with 2.0 soon. 
I will try to get a minor patch (STORM-3205) within 24 hours ... as it seems 
like it has potential to deliver a decent perf boost and energy savings.
One thing I am hoping we can address before releasing Storm 2 is... to fix the 
naming of the storm-client.jar.  Its such a core jar really, it should have 
been really called storm-core or something like that... but unfortunately we 
already have another jar with that name.  Retaining the 'client' name for this 
new jar would be confusing and give wrong impressions to users and any new devs 
IMO.
-roshan

On Thursday, September 13, 2018, 2:12:40 PM PDT, Govind Menon 
 wrote:  
 
 STORM-3217 and STORM-3221 have been fixed - +1 from me for 2.0 RC.

On Wed, Sep 12, 2018 at 10:01 AM Govind Menon  wrote:

> Hi all,
>
> There are some regressions that I introduced as part of STORM-1311 which
> I'm working on as part of https://issues.apache.org/jira/browse/STORM-3217
> and https://issues.apache.org/jira/browse/STORM-3221. These should be
> fixed before a 2.x release
>
> I have code working on the Yahoo internal branch and should have PRs up
> for them in community soon.
>
> I apologize for slowing things up.
>
> Thanks,
> Govind.
>
> On Tue, Sep 11, 2018 at 3:31 PM Arun Mahadevan  wrote:
>
>> +1 for releasing 2.0.
>>
>> May be the RC can be cut once critical patches are merged.
>>
>> On Tue, 11 Sep 2018 at 10:28, Stig Rohde Døssing 
>> wrote:
>>
>> > +1 to cut an RC.
>> >
>> > Here are a couple of PRs that could maybe go in
>> >
>> > https://github.com/apache/storm/pull/2719
>> > https://github.com/apache/storm/pull/2800 (this one requires some
>> changes,
>> > but we should be able to fix it pretty quickly)
>> > also would like to get https://github.com/apache/storm/pull/2805
>> reviewed,
>> > it might change some public methods.
>> >
>> > Other than that, we should try to remove as much deprecated code as we
>> can
>> > before release
>> >
>> > https://issues.apache.org/jira/browse/STORM-2947
>> >
>> > Den man. 10. sep. 2018 kl. 21.59 skrev Alexandre Vermeerbergen <
>> > avermeerber...@gmail.com>:
>> >
>> > > +1 for an Storm 2.0 as soon as possible, let's jump into the future :)
>> > > Le lun. 10 sept. 2018 à 21:50, Kishorkumar Patil
>> > >  a écrit :
>> > > >
>> > > > Looking into all issues reported under epic
>> > > > https://issues.apache.org/jira/browse/STORM-2714 are
>> resolved/closed.
>> > I
>> > > > don't see any open issues/blockers at this point for going ahead
>> with
>> > 2.x
>> > > > release.
>> > > >
>> > > > I am +1 to 2.0 release.
>> > > >
>> > > > Regards,
>> > > > -Kishor
>> > > >
>> > > > On Mon, Sep 10, 2018 at 2:24 PM, P. Taylor Goetz > >
>> > > wrote:
>> > > >
>> > > > > I agree, and looking through the JIRAs against 2.0, I would say a
>> > > majority
>> > > > > of the ones marked critical are not critical.
>> > > > >
>> > > > > I’m +1 on moving forward with a 2.0 release, but will give others
>> > time
>> > > to
>> > > > > respond with any JIRAs they think should be included.
>> > > > >
>> > > > > > p.s. I don't want to create branch-2.x or branch-2.0.x until
>> > > absolutely
>> > > > > > necessary, I don't see any major features with pull requests up
>> but
>> > > if
>> > > > > you
>> > > > > > do run across one please send something out before merging it
>> in,
>> > so
>> > > we
>> > > > > can
>> > > > > > set up the branches properly at that time.
>> > > > >
>> > > > >
>> > > > > Agree. We can always branch off the release tag/commit.
>> > > > >
>> > > > > -Taylor
>> > > > >
>> > > > >
>> > > > > > On Sep 10, 2018, at 12:25 PM, Bobby Evans 
>> > wrote:
>> > > > > >
>> > > > > > It has been nearly a month since this was originally sent out,
>> and
>> > > this
>> > > > > is
>> > > > > > not the first of these kinds of emails to go out about a 2.0.0
>> > > release.
>> > > > > I
>> > > > > > think we have made a lot of really good progress on getting
>> ready
>> > > for a
>> > > > > 2.0
>> > > > > > release, and I really would like to see it happen before another
>> > > month
>> > > > > > passes.
>> > > > > >
>> > > > > > We have a 2.0 based deploy in some of our staging clusters,
>> > currently
>> > > > > > following the master branch with a little that is Yahoo
>> specific on
>> > > top.
>> > > > > We
>> > > > > > would like to start pushing towards production with it soon.
>> > > > > >
>> > > > > > There are a few issues that we are aware of.
>> > > > > >
>> > > > > >
>> > >
>> https://issues.apache.org/jira/issues/?jql=project%20%3D%20STORM%20AND%
>> > > > > 20affectedVersion%20in%20(2.0.0)%20AND%20resolution%20%3D%
>> > > > > 20Unresolved%20ORDER%20BY%20priority%20DESC
>> > > > > >
>> > > > > > There are no blockers still open, and only 4 issues listed as
>> > > critical.
>> > > > > >
>> > > > > > If others have any open issues that feel need to be addressed
>> prior
>> > > to a
>> > > > > > 2.0.0 release please respond to this with the JIRA number.  I
>> would
>> > > like

Re: Question regarding Storm

2018-08-02 Thread Roshan Naik
 Perhaps you can try some things depending on how feasible they it is for your 
case:
1- Try to design and deploy your topology to maximize communication within a 
worker process instead fo across worker processes. 2- Alternatively you can 
spin up multiple single worker instances of you toplo3- Do you need ACKing 
enabled ? if not disable acking as it has a perf hit.4- Play with the batch 
size a bit. ()5- Try the upcoming 2.0 stuff from the master branch. That should 
have much better perf out of the box. Take a look at suggestions in 
Performance.md for tuning it. Pay attention to the new names for some of the 
tunables like batch size.6- Just in case u have not already done ... make sure 
debugging or tracing is disabled and log level is at warn.

For measuring the raw speed of the messaging between spouts and bolts (within 
and across process), I tend to use the ConstSpout*java topologies from here 
https://github.com/apache/storm/tree/master/examples/storm-perf/src/main/java/org/apache/storm/perf
 You may want to change the message size used in them to suit your needs.







On Thursday, August 2, 2018, 11:48:38 AM PDT, Jonathan Kim 
 wrote:  
 
 Hi,

I’m trying to optimize storm to process at least a million messages per
second with each message ranging in size from 1kb to 5mb. I’ve been looking
at other benchmarks and the highest I’ve seen was 43 mb per second which is
awfully slow. Is it possible to achieve this task without spinning up
hundreds of nodes to avoid spending too much on hardware to support it? Or
is storm  

Re: New Committer/PMC Member: Ethan Li

2018-04-16 Thread Roshan Naik
Congratulations Ethan. 


Sent from Yahoo Mail for iPhone


On Monday, April 16, 2018, 9:23 AM, Bobby Evans  wrote:

Please Join with me in welcoming Ethan Li as the newest Apache Storm
committer and PMC member.

Great work!





Re: New Committer/PMC Member: Roshan Naik

2018-04-07 Thread Roshan Naik
Thanks to everyone here and on the PMC list ! -Roshan 


Sent from Yahoo Mail for iPhone


On Friday, April 6, 2018, 3:41 PM, Raghav Kumar Gautam <rag...@apache.org> 
wrote:

Congratulations Roshan !!!


On 2018/04/06 10:59:07, Satish Duggana <s...@gmail.com> wrote:
> Congratulations Roshan!!>
>
>
>
> On Fri, Apr 6, 2018 at 10:06 AM, Erik Weathers <>
> eweath...@groupon.com.invalid> wrote:>
>
> > Congrats Roshan!!>
> >>
> > On Thu, Apr 5, 2018 at 8:26 PM Ethan Li <et...@gmail.com> wrote:>
> >>
> > > Congratulations! Roshan>
> > >>
> > > Ethan Li>
> > >>
> > > > On Apr 5, 2018, at 21:40, Jungtaek Lim <ka...@gmail.com> wrote:>
> > > >>
> > > > Congrats Roshan!>
> > > >>
> > > > 2018년 4월 6일 (금) 오전 11:39, P. Taylor Goetz <pt...@gmail.com>님이 작성:>
> > > >>
> > > >> Please join me in congratulating and welcoming Roshan Naik as the>
> > latest>
> > > >> Apache Storm committer and PMC member.>
> > > >>>
> > > >> Welcome Roshan!>
> > > >>>
> > > >> -Taylor>
> > > >>>
> > > >>>
> > >>
> >>
>




Re: [DISCUSS] Plan on Storm 2.0.0

2018-02-19 Thread Roshan Naik
Would be good to callout which of issues listed in Storm 2.0 epic are 
considered “blockers” vs “very desirable” vs “good to have”

-roshan


On 2/19/18, 3:29 PM, "Jungtaek Lim"  wrote:

Hi devs,

We've just released Storm 1.2.1 (not officially announced but vote passed
anyway) and we're in agreement that no further minor version on 1.x version
line, so it's time to focus on Storm 2.0.0 so that we can bring far long
undetermined milestone to reality.

Here is the epic issue for Storm 2.0.0:
https://issues.apache.org/jira/browse/STORM-2714

We have no restriction to file issues and submit patches so there're more
changes available outside of epic issue, but once we reach consensus to
bring Storm 2.0.0 fairly sooner, I propose we (team, or individual) add all
the issues to epic which we target to include to Storm 2.0.0, so that we
can track all the remaining items from the epic issue, and decide to
postpone some non-blocker dragging items to out of Storm 2.0.0. Makes sense?

I have some backlog issues for myself which would need somewhat huge
efforts so not sure they can be included to Storm 2.0.0. I'll add some to
epic if I think I can find time to do. If you have items which are planned
to be included to Storm 2.0.0, please add them to epic issue.

I hope that we can release Storm 2.0.0 in 1Q (1 month and several days
left): even if it doesn't happen, we could release beta version of Storm
2.0.0 in 1Q and we say "feature freeze" and concentrate on stabilizing the
release and making Storm 2.0.0 happen sooner than later. If you have items
for Storm 2.0.0 which requires more than 1 month to be finished, I think it
would be worth to share the items and discuss.

Please add your voice on anything about Storm 2.0.0 plan. It would be much
appreciated if someone puts efforts on testing and stabilizing the current
master branch (it would take much time and efforts and more hands are
definitely better).

Thanks,
Jungtaek Lim (HeartSaVioR)




Re: [DISCUSS] Decouple Storm core and connectors

2018-01-31 Thread Roshan Naik
I was thinking if the any connector is released more frequently, their quality 
would be more mature and typically have lower impact on a Storm release 
(compared to now) … if we decide to bundle them in Storm as well.
-roshan


On 1/31/18, 4:02 PM, "P. Taylor Goetz" <ptgo...@gmail.com> wrote:

I think we all agree that releasing connectors as part of a Storm release 
hinders the frequency of the release cycle for both Storm proper, as well as 
connectors.

If that’s the case, then the question is how to proceed.

-Taylor

> On Jan 31, 2018, at 6:46 PM, Roshan Naik <ros...@hortonworks.com> wrote:
> 
> One thought is to …
> - do a frequent separate release
> - *and also* include the latest stuff along with each Storm release.
> 
> -roshan
> 
> 
> On 1/31/18, 10:43 AM, "generalbas@gmail.com on behalf of Stig Rohde 
Døssing" <generalbas@gmail.com on behalf of stigdoess...@gmail.com> wrote:
> 
>Hugo,
>It's not my impression that anyone is complaining that 
storm-kafka-client
>has been exceptionally buggy, or that we haven't been fixing the 
issues as
>they crop up. The problem is that we're sitting on the fixes for way 
longer
>than is reasonable, and even if we release Storm more often, users 
have to
>go out of their way to know that they should really be using the latest
>storm-kafka-client rather than the one that ships with their Storm
>installation, because the version number of storm-kafka-client happens 
to
>not mean anything regarding compatibility with Storm.
> 
>Everyone,
> 
>Most of what I've written here has already been said, but I've already
>written it so...
> 
>I really don't see the point in going through the effort of separating
>connectors out to another repository if we're just going to make the 
other
>repository the second class citizen connector graveyard.
> 
>The point to separating storm-kafka-client out is so it can get a 
release
>cycle different from Storm, so we can avoid the situation we're in now 
in
>the future. There's obviously a flaw in our process when we have to 
choose
>between breaking semantic versioning and releasing broken software.
> 
>I agree that it would be good to release Storm a little more often, 
but I
>don't think that fully addresses my concerns. Are we willing to 
increment
>Storm's major version number if a connector needs to break its API 
(e.g. as
>I want to do in https://github.com/apache/storm/pull/2300)?
> 
>I think a key observation is that Storm's core API is extremely stable.
>Storm and the connectors aren't usually tightly coupled in the sense 
that
>e.g. version 1.0.2 of storm-kafka-client would only work with Storm 
1.0.2
>and not 1.0.0, so in many cases there's no reason you wouldn't use the
>latest connector version instead of the one that happens to ship with 
the
>version of Storm you're using. I think it would be attractive if we 
could
>reduce the number of branches of connectors we need to maintain, and
>instead keep a compatibility matrix between Storm and the connector in 
each
>README, for the rare occasions when the Storm core API changes.
> 
>+1 for trying out storm-kafka-client with its own release cycle and
>branches/subrepo/whichever way we want to separate the code, but still 
part
>of the main Storm project JIRA and mailing list. Worst case we merge it
>back in after a while. We may want to think about how to do that 
before we
>separate out, just so we don't release e.g. storm-kafka-client 2.3.1 
and
>then have to merge back to Storm which is still on 2.0.0.
> 
>2018-01-31 3:36 GMT+01:00 Jungtaek Lim <kabh...@gmail.com>:
> 
>> Agreed for this topic: this is not related to current release candidate 
and
>> verifying release candidate is higher priority.
>> For me I didn't start verifying 1.1.2 / 1.0.6 RC2 because the other 
topic I
>> initiated could affect the current release. I'll post a short notice in
>> that discussion thread.
>> 
>> -Jungtaek Lim (HeartSaVioR)
>> 
>> 2018년 1월 31일 (수) 오전 10:58, P. Taylor Goetz <ptgo...@gmail.com>님이 작성:
>> 
>>> Hit send on that too soon...
>>> 
>>> This is an important discussion topic, but has no effect on the current
>>> RCs. Id recommend focusing on the current releases and come back to this
&g

Re: [DISCUSS] Decouple Storm core and connectors

2018-01-31 Thread Roshan Naik
One thought is to …
- do a frequent separate release
- *and also* include the latest stuff along with each Storm release.

-roshan


On 1/31/18, 10:43 AM, "generalbas@gmail.com on behalf of Stig Rohde 
Døssing"  wrote:

Hugo,
It's not my impression that anyone is complaining that storm-kafka-client
has been exceptionally buggy, or that we haven't been fixing the issues as
they crop up. The problem is that we're sitting on the fixes for way longer
than is reasonable, and even if we release Storm more often, users have to
go out of their way to know that they should really be using the latest
storm-kafka-client rather than the one that ships with their Storm
installation, because the version number of storm-kafka-client happens to
not mean anything regarding compatibility with Storm.

Everyone,

Most of what I've written here has already been said, but I've already
written it so...

I really don't see the point in going through the effort of separating
connectors out to another repository if we're just going to make the other
repository the second class citizen connector graveyard.

The point to separating storm-kafka-client out is so it can get a release
cycle different from Storm, so we can avoid the situation we're in now in
the future. There's obviously a flaw in our process when we have to choose
between breaking semantic versioning and releasing broken software.

I agree that it would be good to release Storm a little more often, but I
don't think that fully addresses my concerns. Are we willing to increment
Storm's major version number if a connector needs to break its API (e.g. as
I want to do in https://github.com/apache/storm/pull/2300)?

I think a key observation is that Storm's core API is extremely stable.
Storm and the connectors aren't usually tightly coupled in the sense that
e.g. version 1.0.2 of storm-kafka-client would only work with Storm 1.0.2
and not 1.0.0, so in many cases there's no reason you wouldn't use the
latest connector version instead of the one that happens to ship with the
version of Storm you're using. I think it would be attractive if we could
reduce the number of branches of connectors we need to maintain, and
instead keep a compatibility matrix between Storm and the connector in each
README, for the rare occasions when the Storm core API changes.

+1 for trying out storm-kafka-client with its own release cycle and
branches/subrepo/whichever way we want to separate the code, but still part
of the main Storm project JIRA and mailing list. Worst case we merge it
back in after a while. We may want to think about how to do that before we
separate out, just so we don't release e.g. storm-kafka-client 2.3.1 and
then have to merge back to Storm which is still on 2.0.0.

2018-01-31 3:36 GMT+01:00 Jungtaek Lim :

> Agreed for this topic: this is not related to current release candidate 
and
> verifying release candidate is higher priority.
> For me I didn't start verifying 1.1.2 / 1.0.6 RC2 because the other topic 
I
> initiated could affect the current release. I'll post a short notice in
> that discussion thread.
>
> -Jungtaek Lim (HeartSaVioR)
>
> 2018년 1월 31일 (수) 오전 10:58, P. Taylor Goetz 님이 작성:
>
> > Hit send on that too soon...
> >
> > This is an important discussion topic, but has no effect on the current
> > RCs. Id recommend focusing on the current releases and come back to this
> > after getting  releases out.
> >
> > -Taylor
> >
> > > On Jan 30, 2018, at 8:51 PM, P. Taylor Goetz 
> wrote:
> > >
> > > Also, in the interest of getting releases out, we have 3 open RC 
cycles
> > in flight.
> > >
> > > Discussion energy might be better focused on that.
> > >
> > > -Taylor
> > >
> > >> On Jan 30, 2018, at 7:52 PM, P. Taylor Goetz 
> wrote:
> > >>
> > >>
> > >>
> > >>> On Jan 30, 2018, at 7:31 PM, Harsha  wrote:
> > >>>
> > >>> Hi,
> > >>>  In general connectors are independent of Storm run-time for
> > most parts. I.e if the APIs are not changed (storm-core or trident
> haven't
> > changed in years except the package re-name). You can take the latest
> > connector and run in storm 1.0 or higher. So the users doesn't need to
> > upgrade their storm cluster just to get a latest connector upgrade. 
Which
> > they might be doing it but by making the release separate and stating 
the
> > minimum supported storm version for the connectors will help the users.
> > >>> This makes it easier for the connectors to be released  
independently
> > of the 

Re: Request for holding off commits to storm-client module

2017-07-24 Thread Roshan Naik
I am fully ok with rebasing. I don’t expect development on Storm to cease while 
the PR gets reviewed.  Its only changes to storm-client are likely to require 
to be done differently or not needed at all for the new system (like some fixes 
that went recently into Disruptor). So only for changes to storm-client module 
I am asking to see if the devs can work with me if its critical to get it in 
right away… only to avoid duplication of efforts or redundant efforts.

Will have a PR out within the next 24 hours.

-roshan


On 7/23/17, 7:57 PM, "Jungtaek Lim" <kabh...@gmail.com> wrote:

Hi Roshan,

I guess rebasing could be necessary even while in review phase, so
personally I'd rather see the PR first even it has merge conflict between
origin master branch, and have review process with changes in PR, and
address once review process is done.

If you want to hold off commits just before you submit a pull request, and
you expect to submit it in several days, I'm also OK to wait for that. I
just would like to ensure that holding doesn't stay longer.

Thanks,
Jungtaek Lim (HeartSaVioR)

2017년 7월 24일 (월) 오전 7:40, Roshan Naik <ros...@hortonworks.com>님이 작성:

> Storm Devs,
>   My PR for STORM-2306 (messaging subsystem redesign) is almost ready. Not
> surprisingly, this PR brings extensive modifications in storm-client 
module
> and touches lightly on some others.
>
> Just finished manually rebasing my local changes (to ~90 files) from an
> old version of master on to the latest master this Fri… but  shortly after
> I was done, more commits came into storm-client and need some manual
> reconciling .. so it’s a bit difficult to keep up with.
>
> So, would like to request committers and contributors looking to make
> changes in this critical module to either wait for this PR to go through 
or
> work with me to see if we can work something out if its critical.
>
> Changes related to core areas like Disruptor, Metrics, Credential updates,
> Worker, Executor, Task, Bolt/Spout output collectors/executors, ACKing etc
> are areas with very high likelihood of merge conflicts.
>
> If there are better alternative ideas happy to consider.
> Thanks for understanding,
>
> -roshan
>




Request for holding off commits to storm-client module

2017-07-23 Thread Roshan Naik
Storm Devs,
  My PR for STORM-2306 (messaging subsystem redesign) is almost ready. Not 
surprisingly, this PR brings extensive modifications in storm-client module and 
touches lightly on some others.

Just finished manually rebasing my local changes (to ~90 files) from an old 
version of master on to the latest master this Fri… but  shortly after I was 
done, more commits came into storm-client and need some manual reconciling .. 
so it’s a bit difficult to keep up with.

So, would like to request committers and contributors looking to make changes 
in this critical module to either wait for this PR to go through or work with 
me to see if we can work something out if its critical.

Changes related to core areas like Disruptor, Metrics, Credential updates, 
Worker, Executor, Task, Bolt/Spout output collectors/executors, ACKing etc are 
areas with very high likelihood of merge conflicts.

If there are better alternative ideas happy to consider.
Thanks for understanding,

-roshan


Re: Few observations related to KafkaSpout implementation (1.1.0)

2017-07-11 Thread Roshan Naik
There is a set of simple topologies in storm-perf for such benchmarking… we 
have one there that measures perf of the old KafkaSpout :
https://github.com/apache/storm/blob/1.x-branch/examples/storm-perf/src/main/java/org/apache/storm/perf/KafkaSpoutNullBoltTopo.java

We don’t have one yet for the new Kafka spout… I had written one that never got 
around to contributing back….
https://github.com/roshannaik/storm/blob/perftopos1.x/examples/storm-starter/src/jvm/org/apache/storm/starter/perf/NewKafkaSpoutNullBoltTopo.java

You can copy it into the storm-perf and use that as your starting point… the 
main method that submits the topo there needs to be made more like the ones we 
already have in official Storm.

-roshan

On 7/10/17, 10:49 PM, "chandan singh"  wrote:

Thanks @Stig for the detailed explanation. I have not yet used the
KafkaSpout; was just going through the code to understand it. I will try
the fixes too. In the topologies I run, I hardly create a scenario where
the Spout is limiting the throughput. Still, I will try to do some bench
marking by using a dummy topology and update you on my experience.

Thanks a lot again.
Chandan

On Tue, Jul 11, 2017 at 3:46 AM, Stig Døssing 
wrote:

> > It is the spout thread which
> > is polling (consuming) and iterating (processing) over the polled
> messages.
> > If we separate the consumption in another thread and push messages in a
> > queue, iterating (processing) is now concurrent and decoupled
>
> Sure, but the bookkeeping done by the spout before emitting the polled
> messages should be very lightweight, and pushing the messages (and
> returning acks/fails) through another queue system isn't free. I'm just 
not
> convinced that avoiding short duration blocks by running the consumer in a
> separate thread has any benefit, but I'd be happy to see benchmarks.
>
> The auto spout block/wait when no messages are emitted makes absolute 
sense
> > but a different scenario, less likely in a mostly loaded topology.
>
> Keep in mind that the consumer blocking on poll is also less likely in a
> loaded topology, because the block only happens if either Kafka has no 
more
> new messages, or the consumer prefetching fails to fetch messages quickly
> enough.
>
> 1.2 Agreed again. I had taken the spout recommendation about non-blocking
> > calls seriously, taking into account its critical role
>
> I think it's a good recommendation, but like I said I understand it to be
> warning about blocking for long periods. Besides, the user can configure
> how long of a block in poll they will accept for the KafkaSpout, so if the
> current default timeout (200ms) is causing someone issues, it can always 
be
> lowered.
>
> I am especially thinking of the failure case
> > which, for some reason, is recurrent;  the same message which does not
> get
> > acked.
>
> This case is something the user needs to deal with regardless of how we
> implement the spout. If a message is repeatedly failing, the choice will 
be
> between repeatedly retrying that message until it finally succeeds, or
> dropping that message and moving on. Either option is something the user
> can configure via the RetryService.
>
> We cannot commit unless
> > that failed message is finally acked after few retries. Due to which, we
> > don't even process any new set of records.
> >
> This behavior is caused by a bug in 1.1.0, which has since been fixed. 
Once
> the failed message is fetched and retried, the spout should proceed past
> the failed message and emit new tuples. The only reasons the spout should
> stop emitting new tuples are if it hits the cap on uncommitted offsets
> (configurable, see
> https://github.com/apache/storm/blob/master/external/
> storm-kafka-client/src/main/java/org/apache/storm/kafka/
> spout/KafkaSpoutConfig.java#L47),
> or if the failed message is so far back in the message stream that the
> spout seeks back to the failed message, and then spends a long time
> fetching and discarding already emitted tuples while catching up to where
> it left off.
>
> There are bugs in the max uncommitted offsets mechanism right now that can
> cause a spout to stop retrying tuples (some fixed here
> https://issues.apache.org/jira/browse/STORM-2343, some still pending
> https://issues.apache.org/jira/browse/STORM-2549, we're working to fix
> this). I think the behavior you're seeing is caused by part of STORM-2343,
> which is not fixed in 1.1.0. Basically in 1.1.0 when there was a failed
> tuple, the spout would always seek to the committed offset for that
> partition in order to get the failed tuple. This caused some issues
> relating to being unable to 

Re: [Discussion]: Storm Improvemement Proposal (SIP) to discuss changes

2017-06-09 Thread Roshan Naik
l as mentioned by others in earlier mails. It
would be very useful for new contributors and others who are looking out
for a feature design and decisions taken etc.

Whenever a minor feature is added to a connector it may not need a
separate
SIP but the existing README can be updated with details for users. It
can
be discussed and decided apropos whether a SIP is really required for
any
enhancement which is not really big.


On Sat, Jun 10, 2017 at 5:13 AM, Roshan Naik <ros...@hortonworks.com>
wrote:

If I am looking at the Kafka site correctly, I see that Kafka has a
total
of 167 KIPs so far.
So I assume that minor new features would not be parrt of the SIP ?

Unlike Kafka, since Storm has a number of connectors (that keep
growing),
I am speculating the SIP process might get somewhat unwieldy if it were
to
track little changes in each of the connectors.

Also thinking that a SIP may not be needed to justify a new connector,
but
useful if we are replacing an old connector with a new one.

-roshan



On 6/9/17, 3:19 PM, "Harsha" <st...@harsha.io> wrote:

Hi Bobby,
In general, a KIP is required for adding New features,
config
changes or backward-incompatible changes. Don't require
adding a KIP for bug-fixes. Devs who wants to add any
features will write up a wiki which has JIRA link, mailing
list discussion link and outline the Motivation, Public
interface changes and protocol changes etc ..a good example
here is
https://cwiki.apache.org/confluence/display/KAFKA/KIP-
48+Delegation+token+support+for+Kafka.
They can start the discussion thread once its ready and once everyone
agrees its in a good shape, a Vote thread starts . Once there are
required votes are in one can start the PR process and get it merged
in.
Each release we can collect what features/fixes especially
to
public interfaces that went in and roll it out in release
notes. This will give a better idea for the users on what
changed and added from previous version.
We can only enforce this to new feature/config/backward
incompatible change. Having this go through the discussion
phase will give us the early feedback and potentially caught
any issues before the implementation.
Thanks,
Harsha

On Fri, Jun 9, 2017 at 2:24 PM Bobby Evans <ev...@yahoo-inc.com.invalid

wrote:

Can you please explain how KIP currently works and how you would
like to see something similar in storm?
If we make the process more formal we will probably have less
people
contributing, but we will probably have better overall patches. It
is a balancing act and having never used KIP I would like to
understand it better before going all in on it.
- Bobby


On Friday, June 9, 2017, 4:09:38 PM CDT, Stig Døssing
<generalbas@gmail.com> wrote:

This sounds like a good idea. KIPs seem to work well for Kafka.
It's
easy
for discussions to get lost or just not seen on the mailing list.

2017-06-09 21:36 GMT+02:00 Harsha <st...@harsha.io>:

Hi All,
We’ve seen good adoption of KIP approach in Kafka
community
and would like to see we adopt the similar approach for
storm
as well.
Its hard to keep track of proposed changes and mailing list
threads to
know what all changes that are coming into and what
design/backward
incompatible changes being approved. It will be good to have
this
documented and go through discussion then Vote phase to get them
approved before we merge the PRs. This will keep everyone
informed of
what changes happened even if they are not following the mailing
list
they can go to wiki to see the list of changes went into a
release.
Community overall will be well informed of the changes as well.
Would
like to hear your thoughts.

Thanks,
Harsha











Re: [Discussion]: Storm Improvemement Proposal (SIP) to discuss changes

2017-06-09 Thread Roshan Naik
If I am looking at the Kafka site correctly, I see that Kafka has a total of 
167 KIPs so far. 
So I assume that minor new features would not be parrt of the SIP ? 

Unlike Kafka, since Storm has a number of connectors (that keep growing), I am 
speculating the SIP process might get somewhat unwieldy if it were to track 
little changes in each of the connectors.

Also thinking that a SIP may not be needed to justify a new connector, but 
useful if we are replacing an old connector with a new one.

-roshan



On 6/9/17, 3:19 PM, "Harsha"  wrote:

Hi Bobby,
   In general, a KIP is required for adding New features, config
   changes or backward-incompatible changes. Don't require
   adding a KIP for bug-fixes.  Devs who wants to add any
   features will write up a wiki which has JIRA link, mailing
   list discussion link and outline the Motivation, Public
   interface changes and protocol changes etc ..a good example
   here is
   
https://cwiki.apache.org/confluence/display/KAFKA/KIP-48+Delegation+token+support+for+Kafka.
 
They can start the discussion thread once its ready and once everyone
agrees its in a good shape, a Vote thread starts . Once there are
required votes are in one can start the PR process and get it merged in. 
   Each release we can collect what features/fixes especially to
   public interfaces that went in and roll it out in release
   notes. This will give a better idea for the users on what
   changed and added from previous version.
 We can only enforce this to new feature/config/backward
 incompatible change. Having this go through the discussion
 phase will give us the early feedback and potentially caught
 any issues before the implementation.
Thanks,
Harsha

On Fri, Jun 9, 2017 at 2:24 PM Bobby Evans 
wrote:

Can you please explain how KIP currently works and how you would
like to see something similar in storm?
If we make the process more formal we will probably have less people
contributing, but we will probably have better overall patches.  It
is a balancing act and having never used KIP I would like to
understand it better before going all in on it.
- Bobby


On Friday, June 9, 2017, 4:09:38 PM CDT, Stig Døssing
 wrote:

This sounds like a good idea. KIPs seem to work well for Kafka. It's
easy
for discussions to get lost or just not seen on the mailing list.

2017-06-09 21:36 GMT+02:00 Harsha :

> Hi All,
>  We’ve seen good adoption of KIP approach in Kafka community
>  and would like to see we adopt the similar approach for storm
>  as well.
> Its hard to keep track of proposed changes and mailing list threads to
> know what all changes that are coming into  and what design/backward
> incompatible changes being approved.  It will be good to have this
> documented and go through discussion then Vote phase to get them
> approved before we merge the PRs. This will keep everyone informed of
> what changes happened even if they are not following the mailing list
> they can go to wiki to see the list of changes went into a release.
> Community overall will be well informed of the changes as well. Would
> like to hear your thoughts.
>
> Thanks,
> Harsha
>





Re: [DISCUSS] Storm 2.0 Roadmap

2017-06-03 Thread Roshan Naik
d backpressure to end-to-end, and changed to use snapshot
> rather
> >> than acker.
> >> May be sure, JStorm pulled many features from today's Storm, like Flux,
> >> Windowing, more shuffle groupings, log search, log level change, and so
> on.
> >>
> >> STORM-2426 <https://issues.apache.org/jira/browse/STORM-2426> is due to
> >> the
> >> limitation of Spout lifecycle (all the things are done in single
> thread),
> >> and STORM-1358 <https://issues.apache.org/jira/browse/STORM-1358
> >(JStorm's
> >> multi-thread Spout) can remedy this (despite that Spout implementation
> may
> >> need to guarantee thread-safety later). It's not a just improvement but
> >> close to design concern so would like to address sooner than other
> things
> >> in phase 2.
> >>
> >> For Storm SQL side, I've lost progress but major work would be adopting
> >> group by with windowing. It was not available from Calcite but will be
> >> available at next release (1.12.0).
> >> I've filed this to STORM-2405
> >> <https://issues.apache.org/jira/browse/STORM-2405>, but windowing &
> micro
> >> batch is not intuitive, so I would like to change the underlying API to
> >> stream API in SQL. Also filed this to STORM-2406
> >> <https://issues.apache.org/jira/browse/STORM-2406>.
> >>
> >> Just 2 cents btw, hopefully I would like to see metrics V2 sooner since
> we
> >> lost metrics even when doing normal operation like restarting worker,
> >> rebalancing, and so on. Eventually we need to fight with dynamic
> scaling,
> >> and then metrics will be broken often.
> >>
> >> Thanks,
> >> Jungtaek Lim (HeartSaVioR)
> >>
> >> 2017년 3월 24일 (금) 오전 5:05, Harsha Chintalapani <st...@harsha.io>님이 작성:
> >>
> >> Storm 2.0 migration to java in itself is a big win and would attract
> >> wider
> >> community and adoption. So my vote would be to resolve the first 3 
items
> >> to
> >> get a release out.
> >> All the other featured mentioned are great to have but shouldn't be
> >> blockers for 2.0 release.
> >>
> >> -Harsha
> >>
> >> On Thu, Mar 23, 2017 at 11:51 AM P. Taylor Goetz <ptgo...@gmail.com>
> >> wrote:
> >>
> >> With the 1.1.0 release nearing completion, I’d like to turn our
> >> attention
> >> to 2.0 and develop a plan for what features, etc. to include.
> >>
> >> The following 3 are what I feel are the minimum for a 2.0 release.
> >> These
> >> could likely be resolved relatively quickly:
> >>
> >> * Performance — I’ve not benchmarked the master branch vs. 1.0.x or
> >> 1.1.x
> >> in a while, but I feel it will be important to make sure there are no
> >> performance regressions, and would hope that we actually have a
> >> performance
> >> improvement over previous versions. To that end (e.g. if there is in
> >> fact a
> >> performance regression), the proposals that Roshan Naik put together
> >> for
> >> revising the threading and execution model (STORM-2307) and replacing
> >> Disruptor with JCTools (STORM-2306) warrant review and consideration.
> >> See
> >> also STORM-2284 which is the parent JIRA.
> >>
> >> * Finish porting Storm UI to java (STORM-1311)
> >>
> >> * Finish porting log viewer to java (STORM-1280)
> >>
> >> The following are items that are nice to have in 2.0, but I don’t feel
> >> are
> >> absolutely necessary for an initial 2.0 release:
> >>
> >> * Beam Runner (I wouldn’t tie this to 2.0, mentioning it because it’s
> >> relevant) — Initially there seemed to be a lot of interest in this, but
> >> that seems to have trailed off. I spoke with some Beam developers and
> >> there
> >> seems to be interest from that community as well. Do we want to move
> >> that
> >> effort to the Beam community, or keep it here? Moving it to the Beam
> >> community might lead to better collaboration between projects.
> >>
> >> * Bounded Spouts (needed for Beam Runner i

Re: [DISCUSS] Code Style

2017-04-25 Thread Roshan Naik
- Personally not a big fan of _ for members either from the readability 
standpoint. Also much of the existing code doesn’t follow this.
- Personally I find it is easier to write/read one-liner if/else without the 
curlys. Always wondered about the rationale behind that. 

Would suggest any such a coding convention be very minimalist as a conding 
convention’s usefulness is inversely proportional to its length (

-roshan

Would be good to keep any such coding convention as minimalistic and 

On 4/25/17, 3:50 PM, "Jungtaek Lim"  wrote:

I would like to review the style guide of other projects, like HBase, and
so.

Btw, IMHO, I don't like using underscore as prefix for fields. We're using
Java and the expression "class member variable" is from C++, and also
underscore style came from C++. We need to avoid mixing other languages'
style as well.

Thanks,
Jungtaek Lim (HeartSaVioR)

2017년 4월 26일 (수) 오전 7:10, Kyle Nusbaum 님이
작성:

> Now that most of our code is in Java, I think it might be time to revisit
> the issue of having some official and enforced code style.
> I don't have very strong feelings about most of it, but here is what I was
> thinking as a start, since I've seen this style quite a bit, and I've 
found
> it makes code pretty easy to read:
> 1. Indentation is 4 spaces per level (no tabs)
> 2. All class member variables begin with underscore3. No wildcard
> imports4. if / else / for / etc. always get curly braces.
> There are obviously tons of other things, and we can get as picky as we
> want, but I think enforcing at least these rules would go a long way to
> making the code more consistent.
>
> Thoughts?
>
> Thanks,
> -- Kyle




Recording - Storm & Kafka Meetup on April 20th 2017

2017-04-21 Thread Roshan Naik
It was a great meetup and for the benefit of those interested but unable to 
attend it, here is a link to the recording :

https://www.youtube.com/watch?v=kCRv6iEd7Ow

List of Talks:

-  Introduction –   Suresh Srinivas (Hortonworks)

-  [4m:31sec] –  Overview of  Storm 1.1 -  Hugo Louro (Hortonworks)

-  [20m] –  Rethinking the Storm 2.0 Worker  - Roshan Naik (Hortonworks)

-  [57m] –  Storm in Retail Context: Catalog data processing using 
Kafka, Storm & Microservices   -   Karthik Deivasigamani (WalMart Labs)

-  [1h: 54m:45sec] –   Schema Registry &  Streaming Analytics Manager 
(aka StreamLine)   -   Sriharsha Chintalapani (Hortonworks)




STORM-2423 for 1.1.0

2017-03-17 Thread Roshan Naik
All,
Found a critical issue with JoinBolt that would like to get a fix into 1.1.0.

https://issues.apache.org/jira/browse/STORM-2423


-roshan






Re: Too many threads per bolt instance (as seen with jconsole images)

2017-03-17 Thread Roshan Naik
Can you provide a jstack dump of all the threads in one worker ?
-roshan 


On 3/16/17, 8:40 PM, "S G" <sg.online.em...@gmail.com> wrote:

I did not say 3 thds per spout/bolt.
I said "When 60 bolts are run per node, there are 1200 total threads/node"
That gets me 20 threads/bolt.
And I am not creating any new threads inside my bolts.


On Thu, Mar 16, 2017 at 6:12 PM, Roshan Naik <ros...@hortonworks.com> wrote:

> I assume you mean ..3 thds per spout/bolt is on the high side ?
> Currently there is an executor thd, a xsfer thd and flusher thd. The plan
> (in the redesign – STORM-2284 subtask 2) is to have 1 thread per 
spout/bolt.
> Also have some (untested) thoughts on reducing the number of remaining
> threads in the worker.
> -roshan
>
>
> On 3/16/17, 5:07 PM, "S G" <sg.online.em...@gmail.com> wrote:
>
> Thanks for sharing the doc Roshan.
> It is very informative.
>
> I think number of threads per bolt is on the high side (When 60 bolts
> are
> run per node, there are 1200 total threads/node).
>
> Some of these would be essential for the worker's book-keeping but
> still it
> seems we can get a lot of performance boost if we can somehow reduce
> these
> threads and the associated context switching between them.
>
>
> On Thu, Mar 16, 2017 at 12:53 PM, Roshan Naik <ros...@hortonworks.com>
> wrote:
>
> > Typically  there are 3 threads per spout or bolt instance (unless 
the
> > spout or bolt is spawning its own threads). There are the acker
> bolts,
> > event loggers and system bolt running there too.
> >
> > Then there are several more per worker. See a summary of it in
> section 3.1
> > of this document
> >
> > https://docs.google.com/document/d/1EzeHL3d7EE-
> > RyyBEpN7CwRmWz3oqjbbKiVVAlzFp2Nc/edit?usp=sharing
> >
> >
> >
> >
> > Get Outlook for iOS<https://aka.ms/o0ukef>
> >
> >
> >
> >
> > On Thu, Mar 16, 2017 at 12:16 PM -0700, "S G" <
> sg.online.em...@gmail.com<
> > mailto:sg.online.em...@gmail.com>> wrote:
> >
> > Hi,
> >
> > I am trying to make sense of number of threads seen in JConsole.
> > It seems like a very high number of threads are launched per bolt
> thread.
> >
> >
> > [cid:ii_j0crtbtm0_15ad8874d0924ae2]
> > ​
> > Experiment 1
> > topology.workers.val=16
> > spout.parallelism.val=1
> > bolt1.parallelism.val=900
> > bolt2.parallelism.val=160
> > observation:
> > Threads seen per node in jconsole = 1200
> > Bolt threads per node = (900 + 160)/16 = 66
> > Threads per bolt = 1200/66 = 18
> >
> >
> > Experiment 2
> > topology.workers.val=16
> > spout.parallelism.val=1
> > bolt1.parallelism.val=16
> > bolt2.parallelism.val=16
> > observation:
> > Threads seen per node in jconsole = 61
> > Bolt threads per node = (16+16)/16 = 2
> > Threads per bolt = 61/2 = 30
> >
> > There are no other topologies running in my cluster.
> > There are no other spouts/bolts running in my cluster except the 
ones
> > mentioned above.
> > I am running only one worker process per machine.
> >
> >
> > So the question is how many threads per bolt are launched by storm?
> > I am not interested in the exact number, but concerned about the 
high
> > number of extra threads (18+ thread) for running a single bolt.
> >
> > Can we limit or optimize it somehow?
> > Or if all of them are required, it would be good to document them
> > somewhere.
> >
> > Thanks
> > SG
> >
> >
> >
> >
>
>
>




Re: Too many threads per bolt instance (as seen with jconsole images)

2017-03-16 Thread Roshan Naik
I assume you mean ..3 thds per spout/bolt is on the high side ?
Currently there is an executor thd, a xsfer thd and flusher thd. The plan (in 
the redesign – STORM-2284 subtask 2) is to have 1 thread per spout/bolt. 
Also have some (untested) thoughts on reducing the number of remaining threads 
in the worker. 
-roshan


On 3/16/17, 5:07 PM, "S G" <sg.online.em...@gmail.com> wrote:

Thanks for sharing the doc Roshan.
It is very informative.

I think number of threads per bolt is on the high side (When 60 bolts are
run per node, there are 1200 total threads/node).

Some of these would be essential for the worker's book-keeping but still it
seems we can get a lot of performance boost if we can somehow reduce these
threads and the associated context switching between them.


On Thu, Mar 16, 2017 at 12:53 PM, Roshan Naik <ros...@hortonworks.com>
wrote:

> Typically  there are 3 threads per spout or bolt instance (unless the
> spout or bolt is spawning its own threads). There are the acker bolts,
> event loggers and system bolt running there too.
>
> Then there are several more per worker. See a summary of it in section 3.1
> of this document
>
> https://docs.google.com/document/d/1EzeHL3d7EE-
> RyyBEpN7CwRmWz3oqjbbKiVVAlzFp2Nc/edit?usp=sharing
>
>
>
>
> Get Outlook for iOS<https://aka.ms/o0ukef>
>
>
>
>
> On Thu, Mar 16, 2017 at 12:16 PM -0700, "S G" <sg.online.em...@gmail.com<
> mailto:sg.online.em...@gmail.com>> wrote:
>
> Hi,
>
> I am trying to make sense of number of threads seen in JConsole.
> It seems like a very high number of threads are launched per bolt thread.
>
>
> [cid:ii_j0crtbtm0_15ad8874d0924ae2]
> ​
> Experiment 1
> topology.workers.val=16
> spout.parallelism.val=1
> bolt1.parallelism.val=900
> bolt2.parallelism.val=160
> observation:
> Threads seen per node in jconsole = 1200
> Bolt threads per node = (900 + 160)/16 = 66
> Threads per bolt = 1200/66 = 18
>
>
> Experiment 2
> topology.workers.val=16
> spout.parallelism.val=1
> bolt1.parallelism.val=16
> bolt2.parallelism.val=16
> observation:
> Threads seen per node in jconsole = 61
> Bolt threads per node = (16+16)/16 = 2
> Threads per bolt = 61/2 = 30
>
> There are no other topologies running in my cluster.
> There are no other spouts/bolts running in my cluster except the ones
> mentioned above.
> I am running only one worker process per machine.
>
>
> So the question is how many threads per bolt are launched by storm?
> I am not interested in the exact number, but concerned about the high
> number of extra threads (18+ thread) for running a single bolt.
>
> Can we limit or optimize it somehow?
> Or if all of them are required, it would be good to document them
> somewhere.
>
> Thanks
> SG
>
>
>
>




Re: Too many threads per bolt instance (as seen with jconsole images)

2017-03-16 Thread Roshan Naik
Typically  there are 3 threads per spout or bolt instance (unless the spout or 
bolt is spawning its own threads). There are the acker bolts, event loggers and 
system bolt running there too.

Then there are several more per worker. See a summary of it in section 3.1 of 
this document

https://docs.google.com/document/d/1EzeHL3d7EE-RyyBEpN7CwRmWz3oqjbbKiVVAlzFp2Nc/edit?usp=sharing




Get Outlook for iOS




On Thu, Mar 16, 2017 at 12:16 PM -0700, "S G" 
> wrote:

Hi,

I am trying to make sense of number of threads seen in JConsole.
It seems like a very high number of threads are launched per bolt thread.


[cid:ii_j0crtbtm0_15ad8874d0924ae2]
​
Experiment 1
topology.workers.val=16
spout.parallelism.val=1
bolt1.parallelism.val=900
bolt2.parallelism.val=160
observation:
Threads seen per node in jconsole = 1200
Bolt threads per node = (900 + 160)/16 = 66
Threads per bolt = 1200/66 = 18


Experiment 2
topology.workers.val=16
spout.parallelism.val=1
bolt1.parallelism.val=16
bolt2.parallelism.val=16
observation:
Threads seen per node in jconsole = 61
Bolt threads per node = (16+16)/16 = 2
Threads per bolt = 61/2 = 30

There are no other topologies running in my cluster.
There are no other spouts/bolts running in my cluster except the ones mentioned 
above.
I am running only one worker process per machine.


So the question is how many threads per bolt are launched by storm?
I am not interested in the exact number, but concerned about the high number of 
extra threads (18+ thread) for running a single bolt.

Can we limit or optimize it somehow?
Or if all of them are required, it would be good to document them somewhere.

Thanks
SG





Re: Debugging on Storm Topologies

2017-03-14 Thread Roshan Naik
Submit your topology using LocalCluster in the topology’s main() method.  Then 
you should be able to set breakpoints etc.
-roshan


On 3/14/17, 2:14 PM, "Vankadari, Chandra Sekhar (Contractor)" 
 wrote:

Hey,

I want to set break points in my Storm Topologies code running on my 
Desktop machine and debug it on Eclipse.
I searched for your groups in Dev and could not find required info.

Please let me know if you have any links or emails which can help me.
Appreciate your quick response here..

Regards,
Chandra




Re: [CANCELED] [VOTE] Release Apache Storm 1.1.0 (RC2)

2017-03-03 Thread Roshan Naik
As having a separate module for each of the connectors will cause a lot of 
bloat … may be good to club them into one.

-roshan


On 3/2/17, 10:39 AM, "Sree V" <sree_at_ch...@yahoo.com.INVALID> wrote:

+1separate main binaries, connectors and samples/examples.
 
Thanking you.
With Regards
Sree 

On Thursday, March 2, 2017 8:02 AM, Satish Duggana 
<satish.dugg...@gmail.com> wrote:
 

 Agree that such huge binaries may not be acceptable. We should really think
about the options proposed earlier like excluding some of the external
connectors from binary and update the documentation respectively.

Thanks,
Satish.

On Thu, Mar 2, 2017 at 4:41 PM, Jungtaek Lim <kabh...@gmail.com> wrote:

> Adding my observation to my last mail:
> I just give STORM-2249 a try (against master branch) and compare before vs
> after.
> It adds more than 300 MB, and the final archive is more than 550 MB. I
> guess it would be similar for 1.x branch.
>
> Before --
> -rw-r--r--  1 jlim  staff  3.3K  3  2 19:41 apache-storm-2.0.0-SNAPSHOT.
> pom
> -rw-r--r--  1 jlim  staff  473B  3  2 19:41 apache-storm-2.0.0-SNAPSHOT.
> pom.asc
> -rw-r--r--  1 jlim  staff  264M  3  2 19:41 apache-storm-2.0.0-SNAPSHOT.
> tar.gz
> -rw-r--r--  1 jlim  staff  473B  3  2 19:41 apache-storm-2.0.0-SNAPSHOT.
> tar.gz.asc
> -rw-r--r--  1 jlim  staff  264M  3  2 19:41 apache-storm-2.0.0-SNAPSHOT.
> zip
> -rw-r--r--  1 jlim  staff  473B  3  2 19:41 apache-storm-2.0.0-SNAPSHOT.
> zip.asc
>
> After ---
> -rw-r--r--  1 jlim  staff  3.3K  3  2 19:46 apache-storm-2.0.0-SNAPSHOT.
> pom
> -rw-r--r--  1 jlim  staff  473B  3  2 19:46 apache-storm-2.0.0-SNAPSHOT.
> pom.asc
> -rw-r--r--  1 jlim  staff  564M  3  2 19:46 apache-storm-2.0.0-SNAPSHOT.
> tar.gz
> -rw-r--r--  1 jlim  staff  473B  3  2 19:46 apache-storm-2.0.0-SNAPSHOT.
> tar.gz.asc
> -rw-r--r--  1 jlim  staff  565M  3  2 19:46 apache-storm-2.0.0-SNAPSHOT.
> zip
> -rw-r--r--  1 jlim  staff  473B  3  2 19:47 apache-storm-2.0.0-SNAPSHOT.
> zip.asc
>
> While 264M is already a bit huge for me, 564M is not that I can accept.
> (Binary dist. of Flink 1.2.0 is 127M, Spark 2.1.0 is 195M, Kafka 0.10.2 is
> 37M.)
>
> Btw, we're including source code of examples, and "mvn clean package" will
> work for every example modules.
>
> 2017년 3월 2일 (목) 오전 11:29, Jungtaek Lim <kabh...@gmail.com>님이 작성:
>
> > I guess it might be good time to think why we add all connectors to the
> > binary distribution.
> >
> > Spark and Flink don't include them to binary dist. They even moved some
> or
> > most of connectors out of repo, have been maintaining them in Apache
> Bahir.
> > (Personally this is something I'm in favor of. We have lots of 
connectors
> > and many of them are outdated - clear example is storm-elasticsearch.)
> >
> > If we are assuming online then we don't even need to think about users
> > touching binary dist. version of connectors. Users have been including
> them
> > via build tools' dependency management, or even starting 1.1.0, users 
can
> > include them via '--artifact' option.
> >
> > I also was just one of users for Storm, and I haven't use them directly.
> > How much UX gets worse when we remove connectors to binary dist? It only
> > helps some users who are not connected to the internet, and IMHO it's a
> > rare case.
> >
> > I would like to see the opposite approach, removing all connectors (or
> > just keeping storm-kafka/storm-kafka-client and some more preferred
> things)
> > and its relevant examples from binary dist.
> >
> > What do you think about it?
> >
> > - Jungtaek Lim (HeartSaVioR)
> >
> >
> > 2017년 3월 2일 (목) 오전 10:23, Roshan Naik <ros...@hortonworks.com>님이 작성:
> >
> > Once all of the shaded examples are included the size will go up 
further.
> >
> > But currently as they are not part of the tar.gz … something else is the
> > culprit for the bloat.
> >
> >
> >
> > Below is a comparative listing of 1.0.3 vs 1.1.0  binary releases .. of
> > files that are larger than 4MB.
> >
> >
> >
> >
> >
> > @Jungtaek Lim<mailto:j...@hortonworks.com> :  I am thinking, since the
> > code for the examples c

Re: [CANCELED] [VOTE] Release Apache Storm 1.1.0 (RC2)

2017-03-01 Thread Roshan Naik
Looks like the formatting  on my prev email got dropped. But observation is 
that storm-kafka-monitor  and storm-druid are the biggest new contributors.
-roshan


On 3/1/17, 5:23 PM, "Roshan Naik" <ros...@hortonworks.com> wrote:

Once all of the shaded examples are included the size will go up further.

But currently as they are not part of the tar.gz … something else is the 
culprit for the bloat.

Below is a comparative listing of 1.0.3 vs 1.1.0  binary releases .. of 
files that are larger than 4MB.


@Jungtaek Lim<mailto:j...@hortonworks.com> :  I am thinking, since the code 
for the examples can be easily viewed online … it would be valuable to have the 
executable topologies made available to the user as part of the binary release 
… rather have them figure out how to build it correctly before trying them out.

-roshan





➜  apache-storm-1.0.3 >  find . -type f -size +4096 -exec ls -lh {} \;

-rw-r--r--@ 1 roshan  staff70M Feb  7 12:33 
./examples/storm-starter/storm-starter-topologies-1.0.3.jar

-rwxr-xr-x@ 1 roshan  staff65M Feb  7 12:30 
./external/flux/flux-examples-1.0.3.jar

-rwxr-xr-x@ 1 roshan  staff   3.5M Feb  7 12:32 
./external/sql/storm-sql-core/calcite-core-1.4.0-incubating.jar

-rwxr-xr-x@ 1 roshan  staff   2.1M Feb  7 12:32 
./external/sql/storm-sql-core/guava-16.0.1.jar

-rwxr-xr-x@ 1 roshan  staff   7.3M Feb  7 12:30 
./external/storm-eventhubs/storm-eventhubs-1.0.3-jar-with-dependencies.jar

-rwxr-xr-x@ 1 roshan  staff   5.6M Feb  7 12:33 
./external/storm-jms/storm-jms-examples-1.0.3-jar-with-dependencies.jar

-rwxr-xr-x@ 1 roshan  staff   9.9M Feb  7 12:33 
./external/storm-mqtt/storm-mqtt-examples-1.0.3.jar

-rw-r--r--@ 1 roshan  staff   3.7M Nov  4 10:02 ./lib/clojure-1.7.0.jar

-rw-r--r--@ 1 roshan  staff19M Feb  7 12:26 ./lib/storm-core-1.0.3.jar

-rw-r--r--@ 1 roshan  staff   2.4M Feb  7 12:26 
./lib/storm-rename-hack-1.0.3.jar



➜  apache-storm-1.1.0 >  find . -type f -size +4096 -exec ls -lh {} \;

-rwxr-xr-x@ 1 roshan  staff   8.0M Feb 24 12:23 
./examples/storm-pmml-examples/storm-pmml-examples-1.1.0.jar

-rwxr-xr-x@ 1 roshan  staff60M Feb 24 12:20 
./examples/storm-starter/storm-starter-topologies-1.1.0.jar

-rwxr-xr-x@ 1 roshan  staff66M Feb 24 12:11 
./external/flux/flux-examples-1.1.0.jar

-rwxr-xr-x@ 1 roshan  staff   4.0M Feb 24 12:16 
./external/sql/storm-sql-core/calcite-core-1.11.0.jar

-rwxr-xr-x@ 1 roshan  staff   2.1M Feb 24 12:16 
./external/sql/storm-sql-core/guava-16.0.1.jar

-rwxr-xr-x@ 1 roshan  staff   4.0M Feb 24 12:12 
./external/sql/storm-sql-runtime/calcite-core-1.11.0.jar

-rwxr-xr-x@ 1 roshan  staff   2.1M Feb 24 12:12 
./external/sql/storm-sql-runtime/guava-16.0.1.jar

-rwxr-xr-x@ 1 roshan  staff78M Feb 24 12:18 
./external/storm-druid/storm-druid-1.1.0.jar

-rwxr-xr-x@ 1 roshan  staff   7.3M Feb 24 12:11 
./external/storm-eventhubs/storm-eventhubs-1.1.0-jar-with-dependencies.jar

-rwxr-xr-x@ 1 roshan  staff   5.6M Feb 24 12:20 
./external/storm-jms/storm-jms-examples-1.1.0-jar-with-dependencies.jar

-rwxr-xr-x@ 1 roshan  staff   6.7M Feb 24 12:18 
./external/storm-submit-tools/storm-submit-tools-1.1.0.jar

-rwxr-xr-x@ 1 roshan  staff   3.7M Nov  4 10:02 ./lib/clojure-1.7.0.jar

-rwxr-xr-x@ 1 roshan  staff20M Feb 24 12:07 ./lib/storm-core-1.1.0.jar

-rwxr-xr-x@ 1 roshan  staff   2.4M Feb 24 12:07 
./lib/storm-rename-hack-1.1.0.jar

-rwxr-xr-x@ 1 roshan  staff18M Feb 24 12:19 
./toollib/storm-kafka-monitor-1.1.0.jar













On 3/1/17, 4:43 PM, "Jungtaek Lim" <kabh...@gmail.com> wrote:



About STORM-2249, since examples are shading their dependencies, binary

dist will grow much bigger. I've left some comments regarding that.

Btw, I have another view of this. Showing example codes is more 
important

than just let users execute some topologies. That's what example modules

are for. We need to include source as well. If we need to pick one, 
source

code would be better.



STORM-2343 seems better to add to 1.1.0. I just am not enough familiar 
with

storm-kafka-client so not sure I can review that, but I'll try to. I 
feel

it's not that make release dragged. Let's add to 1.1.0 epic.



Let's make minimum merge before another RC vote. Addressing STORM-2389 
(and

maybe STORM-2343) is enough for me. Others are not that critical.



Thanks,

Jungtaek Lim (HeartSaVioR)





On Thu, Mar 2, 2017 at 7:00 AM, Hugo Da Cruz Louro 

Re: Storm 1.x-branch won't build in IntelliJ anymore

2017-03-01 Thread Roshan Naik
That workaround of running a clean mvn build on the cmd line before building in 
IntellJ does not work for me anymore. It used sometime back (a month ago?). 
No matter what I try… I cant get it to build in IntelliJ.
-roshan


On 3/1/17, 5:20 PM, "Jungtaek Lim" <kabh...@gmail.com> wrote:

The original plan was not adding anything related storm-core in 1.x version
line and move out 2.0.0 fast. (That's what Bobby proposed for JStorm merger
phases.)

We didn't make it faster (sorry I couldn't contribute for this side at
all), and we broke the plan, and it makes harder and harder to move toward
2.0.0.
What we're adding is what we need to maintain, and so as what we're
releasing. I have other project which has similar plan and it didn't reach
major release for years. I don't want to see same thing for Storm given
that merger plan was started more than 1 year.

Moving to Java codebase gets rid of hard thing what you talked about, and
let end-users being able to dig the code and becomes contributors. It's
invaluable change.

Btw I can still be able to play with 1.x version line with workaround (mvn
clean install -DskipTests). Does workaround not work any more? Or do you
see some spots which workaround doesn't work?

- Jungtaek Lim (HeartSaVioR)


On Wed, Mar 1, 2017 at 11:34 PM, Roshan Naik <ros...@hortonworks.com> wrote:

> If we leave 1.x in this state it will forever remain un-usable within
> IntelliJ ..  hard to debug, UT etc. And 1.x will live for a while. Is
> rolling back some of the non-critical offending commits that are causing
> this an option ?  i tried rolling back a bit but was unable to identify 
the
> offending commit.
> -roshan
>
>
> On 2/28/17, 9:13 PM, "Arun Iyer on behalf of Arun Mahadevan" <
> ai...@hortonworks.com on behalf of ar...@apache.org> wrote:
>
> >If we really need long time to discuss above, I'm even OK to revert
> DRPC
> >port and start 2.0.0 with webservices unported, (DRPC, UI, Logviewer)
> and
> >address them at 2.1 or other minor versions.
> >
> >Any other opinions?
> >
>
> Yes, it may be better to move to 2.0 sooner and migrate the pending
> components in 2.1. May be what we need is more rigorous testing of the
> master branch before we release 2.0.
>
> We could also consider moving to JDK 8 for the next 1.x release and it
> would make porting patches to 1.x branch much easier (unless we want to do
> a JDK upgrade only with 2.0).
>
> Thanks,
> Arun
>
> On 3/1/17, 7:40 AM, "Jungtaek Lim" <kabh...@gmail.com> wrote:
>
> >Thanks Roshan for bring this up.
> >
> >For me moving toward 2.0.0 makes more sense.
> >
> >I know master branch has similar issue (DRPC) and it also has
> >not-yet-ported things but if we are going to struggle with 1.x branch
> issue
> >again and again, Storm 2.0.0 will never come. Another recent headache
> issue
> >is JDK 7 vs JDK 8. Patch for master easily breaks for 1.x branch due
> to
> >this, and we just had to cancel another 1.1.0 RC vote.
> >
> >We might want to discuss how to handle webservice like DRPC (My
> feeling is
> >that current approach is somewhat kinda hacky.), but we can initiate
> >different thread for more details.
> >
> >If we really need long time to discuss above, I'm even OK to revert
> DRPC
> >port and start 2.0.0 with webservices unported, (DRPC, UI, Logviewer)
> and
> >address them at 2.1 or other minor versions.
> >
> >Any other opinions?
> >
> >- Jungtaek Lim (HeartSaVioR)
> >
> >2017년 3월 1일 (수) 오전 9:27, Roshan Naik <ros...@hortonworks.com>님이 작성:
> >
> >Lately (about a week and half maybe) it has not been possible to get
> the
> >1.x-branch to build inside IntelliJ. None of the modules are able to
> locate
> >the  LocalCluster class (which is a clojure class). Previously, every
> once
> >in a while, I used to get this problem in the storm-starter module
> and was
> >able to get around it by doing a mvn clean install –DskipTests on the
> cmd
> >line and then doing a full rebuild of the project in IntelliJ.
> >
> >Now the problem has become a lot more endemic (modules lik

Re: [CANCELED] [VOTE] Release Apache Storm 1.1.0 (RC2)

2017-03-01 Thread Roshan Naik
t pass reviewing today. It would 
be

> good if we could have this PR included with the release. Can anyone else

> review it as well ?

>

> Thanks,

> Hugo

>

> On Mar 1, 2017, at 9:14 AM, P. Taylor Goetz <ptgo...@gmail.com<mailto:ptgo

> e...@gmail.com>> wrote:

>

> Yeah, I don’t think the file size is a killer/blocker. It’s largely due to

> shaded examples, etc. But it’s something to keep an eye on. Our binary

> releases shouldn’t have to be that big.

>

> -Taylor

>

> On Mar 1, 2017, at 12:09 PM, Roshan Naik <ros...@hortonworks.com ros...@hortonworks.com>> wrote:

>

> Have filed Jiras so for the 3 issues mentioned. Not sure if we need a JIRA

> for the file size getting bloated by that much.

> Somebody better familiar with the matter may want to take about that?

> -roshan

>

>

> On 3/1/17, 8:13 AM, "P. Taylor Goetz" <ptgo...@gmail.com<mailto:ptgo

> e...@gmail.com>> wrote:

>

>   Thanks for bringing these up Roshan. Feel free to file JIRA tickets for

> these issues and assign the “Release Apache Storm 1.1.0” epic so they can

> be tracked for this release.

>

>   -Taylor

>

> On Mar 1, 2017, at 9:27 AM, Roshan Naik <ros...@hortonworks.com ros...@hortonworks.com>> wrote:

>

> Found these additional issues:

>

>

>

> 1- BUG: Even if topology.eventlogger.executors=0,  the event_logger bolt

> is instantiated … previously observed to cause ~10% degradation in perf

> even with logging disabled.

>

> 2- Missing Jars: The storm-*-examples jars are missing in the binary

> distro (other than a storm-pmml-examples.jar, storm-jms-examples.jar &

> flux-examples.jar).

>

> 3- Minor: HdfsSpoutTopology example has not been moved into

> storm-hdfs-examples from storm-starter

>

>

>

> Another side observation …  v1.0.3 tar.gz downloadable was 190MB.   This

> v1.1.0 tar.gz  downloadable is 297MB !! …. even though some of the example

> topologies didn’t make it.

>

>

>

> -roshan

>

>

>

>

>

>

>

>





--

Name : Jungtaek Lim

Blog : http://medium.com/@heartsavior

Twitter : http://twitter.com/heartsavior

LinkedIn : http://www.linkedin.com/in/heartsavior




Re: [CANCELED] [VOTE] Release Apache Storm 1.1.0 (RC2)

2017-03-01 Thread Roshan Naik
Have filed Jiras so for the 3 issues mentioned. Not sure if we need a JIRA for 
the file size getting bloated by that much. 
Somebody better familiar with the matter may want to take about that?
-roshan


On 3/1/17, 8:13 AM, "P. Taylor Goetz" <ptgo...@gmail.com> wrote:

Thanks for bringing these up Roshan. Feel free to file JIRA tickets for 
these issues and assign the “Release Apache Storm 1.1.0” epic so they can be 
tracked for this release.

-Taylor

> On Mar 1, 2017, at 9:27 AM, Roshan Naik <ros...@hortonworks.com> wrote:
> 
> Found these additional issues:
> 
> 
> 
> 1- BUG: Even if topology.eventlogger.executors=0,  the event_logger bolt 
is instantiated … previously observed to cause ~10% degradation in perf even 
with logging disabled.
> 
> 2- Missing Jars: The storm-*-examples jars are missing in the binary 
distro (other than a storm-pmml-examples.jar, storm-jms-examples.jar & 
flux-examples.jar).
> 
> 3- Minor: HdfsSpoutTopology example has not been moved into 
storm-hdfs-examples from storm-starter
> 
> 
> 
> Another side observation …  v1.0.3 tar.gz downloadable was 190MB.   This 
v1.1.0 tar.gz  downloadable is 297MB !! …. even though some of the example 
topologies didn’t make it.
> 
> 
> 
> -roshan






Re: [CANCELED] [VOTE] Release Apache Storm 1.1.0 (RC2)

2017-03-01 Thread Roshan Naik
Found these additional issues:



1- BUG: Even if topology.eventlogger.executors=0,  the event_logger bolt is 
instantiated … previously observed to cause ~10% degradation in perf even with 
logging disabled.

2- Missing Jars: The storm-*-examples jars are missing in the binary distro 
(other than a storm-pmml-examples.jar, storm-jms-examples.jar & 
flux-examples.jar).

3- Minor: HdfsSpoutTopology example has not been moved into storm-hdfs-examples 
from storm-starter



Another side observation …  v1.0.3 tar.gz downloadable was 190MB.   This v1.1.0 
tar.gz  downloadable is 297MB !! …. even though some of the example topologies 
didn’t make it.



-roshan


Re: Storm 1.x-branch won't build in IntelliJ anymore

2017-03-01 Thread Roshan Naik
If we leave 1.x in this state it will forever remain un-usable within IntelliJ 
..  hard to debug, UT etc. And 1.x will live for a while. Is rolling back some 
of the non-critical offending commits that are causing this an option ?  i 
tried rolling back a bit but was unable to identify the offending commit.
-roshan


On 2/28/17, 9:13 PM, "Arun Iyer on behalf of Arun Mahadevan" 
<ai...@hortonworks.com on behalf of ar...@apache.org> wrote:

>If we really need long time to discuss above, I'm even OK to revert DRPC
>port and start 2.0.0 with webservices unported, (DRPC, UI, Logviewer) and
>address them at 2.1 or other minor versions.
>
>Any other opinions?
>

Yes, it may be better to move to 2.0 sooner and migrate the pending 
components in 2.1. May be what we need is more rigorous testing of the master 
branch before we release 2.0.

We could also consider moving to JDK 8 for the next 1.x release and it 
would make porting patches to 1.x branch much easier (unless we want to do a 
JDK upgrade only with 2.0).

Thanks,
Arun

On 3/1/17, 7:40 AM, "Jungtaek Lim" <kabh...@gmail.com> wrote:

>Thanks Roshan for bring this up.
>
>For me moving toward 2.0.0 makes more sense.
>
>I know master branch has similar issue (DRPC) and it also has
>not-yet-ported things but if we are going to struggle with 1.x branch issue
>again and again, Storm 2.0.0 will never come. Another recent headache issue
>is JDK 7 vs JDK 8. Patch for master easily breaks for 1.x branch due to
>this, and we just had to cancel another 1.1.0 RC vote.
>
>We might want to discuss how to handle webservice like DRPC (My feeling is
>that current approach is somewhat kinda hacky.), but we can initiate
>different thread for more details.
>
>If we really need long time to discuss above, I'm even OK to revert DRPC
>port and start 2.0.0 with webservices unported, (DRPC, UI, Logviewer) and
>address them at 2.1 or other minor versions.
>
    >Any other opinions?
>
>- Jungtaek Lim (HeartSaVioR)
>
>2017년 3월 1일 (수) 오전 9:27, Roshan Naik <ros...@hortonworks.com>님이 작성:
>
>Lately (about a week and half maybe) it has not been possible to get the
>1.x-branch to build inside IntelliJ. None of the modules are able to locate
>the  LocalCluster class (which is a clojure class). Previously, every once
>in a while, I used to get this problem in the storm-starter module and was
>able to get around it by doing a mvn clean install –DskipTests on the cmd
>line and then doing a full rebuild of the project in IntelliJ.
>
>Now the problem has become a lot more endemic (modules like storm-sql,
>storm-*-examples, etc.). And the above workaround doesn’t help. Spent many
>futile hours trying to work around the build issue within Intellij (both
>2015 and 2016 versions and using different Clojure plugins).
>
>There seems to possible ways to move forward:
>
>
>-  Somebody here knows what magic to do work address this. And we
>can all use that.
>
>-  Bring in the java port of LocalCluster.clj from master branch to
>1.x (STORM-1281). I spend a little time to see if this was easy to do, but
>appears kind of complicated due to number of files involved and
>dependencies on prior patches. Somebody familiar with the original porting
>effort may be better person to take this up.
>
>Thoughts ?
>
>-Roshan







Storm 1.x-branch won't build in IntelliJ anymore

2017-02-28 Thread Roshan Naik
Lately (about a week and half maybe) it has not been possible to get the 
1.x-branch to build inside IntelliJ. None of the modules are able to locate the 
 LocalCluster class (which is a clojure class). Previously, every once in a 
while, I used to get this problem in the storm-starter module and was able to 
get around it by doing a mvn clean install –DskipTests on the cmd line and then 
doing a full rebuild of the project in IntelliJ.

Now the problem has become a lot more endemic (modules like storm-sql, 
storm-*-examples, etc.). And the above workaround doesn’t help. Spent many 
futile hours trying to work around the build issue within Intellij (both 2015 
and 2016 versions and using different Clojure plugins).

There seems to possible ways to move forward:


-  Somebody here knows what magic to do work address this. And we can 
all use that.

-  Bring in the java port of LocalCluster.clj from master branch to 1.x 
(STORM-1281). I spend a little time to see if this was easy to do, but appears 
kind of complicated due to number of files involved and dependencies on prior 
patches. Somebody familiar with the original porting effort may be better 
person to take this up.

Thoughts ?

-Roshan




Re: [DISCUSS] Remove CHANGELOG file

2017-02-23 Thread Roshan Naik
Sounds like a good idea to me.
-roshan

On 2/23/17, 4:41 PM, "Jungtaek Lim"  wrote:

Hi devs,

I guess we discussed about this before, but didn't move to actual work.

I'd like to propose again removing CHANGELOG file in repository, and use
JIRA issue's fixed version(s).

Maintaining CHANGELOG to file is really easy to break. I've seen several
times and most of them is about backport. CHANGELOG file between branches
are inconsistent.

Suppose we would like to backport the issue to 1.0.x which is only applied
to 2.0.0, then we should fix CHANGELOG from three branches. Easy to miss
and redundant.

I'd also like to remove Project leads / Committers / Contributors in README
(at least Contributors) since it's also easy to break.

For PMC members we're maintaining it to website and I think that's enough.
For contributors I love what other projects are doing: extract unique
contributors name from commits or JIRA issues of release version and
mention them from release announce note.

What do you think?

Thanks,
Jungtaek Lim (HeartSaVioR)




Re: [Discuss] Storm hdfs spout improvements

2017-02-13 Thread Roshan Naik


On 2/13/17, 12:14 PM, "Sachin Pasalkar" 
wrote:

>I have attached updated source code of HDFSSpout for more reference. I
>have updated respective classes (not attached)


Don¹t see any attachment. Answers are below. Better to do this discussion
on a JIRA.


On 2/13/17, 8:32 AM, "Sachin Pasalkar" 
wrote:

>Hi,
>
>I was looking at storm hdfs spout code in 1.x branch, I found below
>improvements can be made in below code.
>
>  1.  Make org.apache.storm.hdfs.spout.AbstractFileReader as public so
>that it can be used in generics.

Java generics and making a class public are unrelated to my knowledge. But
making it public sounds ok to me if its useful for "user defined² readers
Š although it doesn¹t really have that much going on in it. For future
built-in reader types it is immaterial as they can derive from it anyway
just like the existing ones. HdfsSpout class itself doesn¹t care about the
ŒAbstractFileReader¹ type. For that there is the ŒFileReader¹ interface.



>  2.  org.apache.storm.hdfs.spout.HdfsSpout requires readerType as
>String. It will be great to have class
>readerType; So we will not use Class.forName at multiple places also it
>will help in below point.

The reason it is a string, is that, for built-in readers,  we wanted to
support Œshort aliases¹ like Œtext¹ and Œseq¹ instead of FQCN..


>  3.  HdfsSpout also needs to provide outFields which are declared as
>constants in each reader(e.g.SequenceFileReader). We can have abstract
>API AbstractFileReader in which return them to user to make it generic.


These consts can¹t go into the AbstractFileReader as they are reader
specific.

They are there just for convenience.  Users can call withOutputFields() on
the spout and set it to these predefined names or anything else.


-Roshan



Re: Storm benchmarks

2016-11-04 Thread Roshan Naik
http://hortonworks.com/blog/microbenchmarking-storm-1-0-performance/


-roshan


On 11/4/16, 7:13 AM, "Paul Poulosky" 
wrote:

>IEEE Xplore Document - Benchmarking Streaming Computation Engines: Storm,
>Flink and Spark Streaming
>
>  
>|  
>|   |  
>IEEE Xplore Document - Benchmarking Streaming Computation Engines: Storm,
>Flink and Spark Streaming
> Streaming data processing has been gaining attention due to its
>application into a wide range of scenarios. To s...  |  |
>
>  |
>
> 
>
>
>Benchmarking Streaming Computation Engines at Yahoo!
>
>  
>|  
>|   
>|   
>|   ||
>
>   |
>
>  |
>|  
>||  
>Benchmarking Streaming Computation Engines at Yahoo!
> (Yahoo Storm Team in alphabetical order) Sanket Chintapalli, Derek
>Dagit, Bobby Evans, Reza Farivar, Tom Graves,...  |   |
>
>  |
>
>  |
>
> 
>
> 
>
>On Friday, November 4, 2016 6:05 AM, Dominik Safaric
> wrote:
> 
>
> 
>1- What do you mean "able to control message size"? Is it
>max-pending-spout parameter?
>
>By using for example Kafka as your source of information of the benchmark
>topology, you may produce i.e. control the size of messages in terms of
>bytes length. Why would you want to do this? Because there is a relation
>between certain performance characteristics such as throughput and
>message size. 
>
>Is there any published benchmark like this old-one here:
>
>As far up to my knowledge, no. However, we at the Web Information Systems
>research group of the Delft University of Technology are currently in the
>process of benchmarking several streaming engines (including Storm) part
>of an empirical research. If you¹d like to here more about the insight so
>far gathered, feel free to email me.
>
>On 4 Nov 2016, at 10:02, Walid Aljoby  wrote:
>Thank you Dominik. I have two more points, please.1- What do you mean
>"able to control message size"? Is it max-pending-spout parameter?2- Is
>there any published benchmark like this old-one here:
>https://github.com/stormprocessor/storm-benchmark/commit/22bd17a81020ceef7
>1ed73168ac89d3f8eaf61e2
>Best Regards,Walid
>
>  From: Dominik Safaric 
> To: Walid Aljoby 
>Cc: "u...@storm.apache.org" ;
>"dev@storm.apache.org" 
> Sent: Friday, November 4, 2016 4:53 PM
> Subject: Re: Storm benchmarks
>  
>Well, this depends onto the aspects of the measurements.
>You may for example define a topology consisting of a spout,
>transformation bolt and sink that receives byte arrays from Kafka,
>transforms them and outputs. The nice thing is that you¹d be able to
>control for the size of the messages.
>In addition, if you care about the performance in conjunction to stateful
>operations such as aggregations, your topology might look alike the for
>example WordCount topology.
>Regards,Dominik
>
>On 4 Nov 2016, at 09:50, Walid Aljoby  wrote:
>Hi Dominik,
>Many thanks for details. Actually I am looking for a set typologies for
>my test.
>Thank you again,--RegardsWalid
>
>  From: Dominik Safaric 
> To: u...@storm.apache.org; Walid Aljoby 
>Cc: "dev@storm.apache.org" 
> Sent: Friday, November 4, 2016 4:41 PM
> Subject: Re: Storm benchmarks
>  
>Hi Walid,
>You may benchmark Storm¹s performance in terms of throughput and
>end-to-end latency for example. In addition, the investigation could also
>include variances in the configurational settings, such as the
>parallelism, message size, intra-worker and inter-worker buffer size
>which some of them have a profound effect onto the performance of Storm.
>There are already a few benchmarks of Storm¹s performance such as:
>https://developer.ibm.com/streamsdev/wp-content/uploads/sites/15/2014/04/S
>treams-and-Storm-April-2014-Final.pdf
>In addition, you may want to take a look at the academic paper
>Storm@Twitter and Twitter Heron: Stream processing at scale which
>describe among others certain performance aspects of Storm that might be
>helpful to you when designing the benchmark.
>Regards,Dominik 
>
>On 4 Nov 2016, at 09:36, Walid Aljoby  wrote:
>Hi Everyone,
>Anyone please could tell what are the common benchmarks for testing Storm
>performance? 
>Thank you,--Regards
>WA
>
>
>
>   
>
>
>
>   
>
>
>
>   



Re: [DISCUSS] Feature Branch for Apache Beam Runner

2016-09-15 Thread Roshan Naik
Good idea. Will the Beam API be implemented to run on top Storm Core
primitives ?
-roshan


On 9/15/16, 2:00 PM, "P. Taylor Goetz"  wrote:

>I¹ve been tinkering with implementing an Apache Beam runner on top of
>Storm and would like to open it up so others in the community can
>contribute. To that end I¹d like to propose creating a feature branch for
>that work if there are others who are interested in getting involved. We
>did that a while back when storm-sql was originally developed.
>
>Basically, review requirements for that branch would be relaxed during
>development, with a final, strict review before merging back to one of
>our main branches.
>
>I¹d like to document what I have and future improvements in a proposal
>document, and follow that with pushing the code to the feature branch for
>group collaboration.
>
>Any thoughts? Anyone interested in contributing to such an effort?
>
>-Taylor



Re: Performance improvements in 1.0.0

2016-09-06 Thread Roshan Naik
Here is one effort on quantification the improvements Š
   http://hortonworks.com/blog/microbenchmarking-storm-1-0-performance/


-roshan


On 9/6/16, 10:17 AM, "S G"  wrote:

>Hi,
>
>On the http://storm.apache.org/2016/04/12/storm100-released.html page, I
>see a notice about performance improvements to the order of 16 times.
>
>Can someone please provide a little more details on the same? Is the
>throughput of a topology increased 16 times (that sounds a lot, hence the
>question) or is it something internal to storm? What are the JIRA issues
>corresponding to this cool performance bump?
>
>Thanks
>SG



Re: Details about the Storm Scheduler

2016-08-25 Thread Roshan Naik
Indeed. Since Bobby was pointing out that CPU consumption monitoring is 
impractical at a component level... My suggestion was merely intended as a next 
best alternative.
-roshan

From: Nathan Leung <ncle...@gmail.com<mailto:ncle...@gmail.com>>
Date: Wednesday, August 24, 2016 at 7:24 AM
To: "dev@storm.apache.org<mailto:dev@storm.apache.org>" 
<dev@storm.apache.org<mailto:dev@storm.apache.org>>, Bobby Evans 
<ev...@yahoo-inc.com<mailto:ev...@yahoo-inc.com>>
Cc: Roshan Naik <ros...@hortonworks.com<mailto:ros...@hortonworks.com>>
Subject: Re: Details about the Storm Scheduler

Also your bolt may be pending on a call to an external resource (e.g. DB) and 
thus not consuming much CPU despite a relatively high usage.

On Wed, Aug 24, 2016 at 10:09 AM, Bobby Evans 
<ev...@yahoo-inc.com.invalid<mailto:ev...@yahoo-inc.com.invalid>> wrote:
But CGroups is restricting the actual CPU usage and scheduling is taking the 
CPU usage into account so as to not overload a box.  You can use the latency to 
guess how much CPU is being used, but that only works for a single threaded 
bolt/spout.  Not all bolts/spouts are single threaded.  Think about a shell 
bolt or a shell spout.
 - Bobby

On Tuesday, August 23, 2016 5:38 PM, Roshan Naik 
<ros...@hortonworks.com<mailto:ros...@hortonworks.com>> wrote:




On 8/22/16, 6:55 AM, "Bobby Evans" 
<ev...@yahoo-inc.com.INVALID<mailto:ev...@yahoo-inc.com.INVALID>> wrote:

>Getting the CPU used for a worker is simple, but getting the CPU used for
>individual components is not so simple/almost impossible for
>multi-threaded bolts/spouts.  The current scheduling assumes that the CPU
>for all bolts/spouts is the same for the same component.  This could
>cause a hot spot in one part of the topology to cause rescheduling to
>happen in other parts of the topology needlessly.


I think that can be worked around by looking at the latency of each
component instead of CPU consumption of each component.

-roshan






[jira] [Commented] (STORM-1949) Backpressure can cause spout to stop emitting and stall topology

2016-08-23 Thread Roshan Naik (JIRA)

[ 
https://issues.apache.org/jira/browse/STORM-1949?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15433795#comment-15433795
 ] 

Roshan Naik commented on STORM-1949:


The amount of  *additional* pressure this BP mechanism adds to ZK in it current 
state really should be sufficient reason to leave it disabled by default. If we 
fix the problem I noted in the description, as per Bobby's suggestion, that 
would put even more pressure on ZK. Putting such pressure on ZK (or Nimbus) 
from any subsystem in Storm is essentially a regression in terms of scaling 
ability, which then begets future fixes (PaceMaker for instance)

> Backpressure can cause spout to stop emitting and stall topology
> 
>
> Key: STORM-1949
> URL: https://issues.apache.org/jira/browse/STORM-1949
> Project: Apache Storm
>  Issue Type: Bug
>    Reporter: Roshan Naik
>Assignee: Alessandro Bellina
> Attachments: 1.x-branch-works-perfect.png, wordcounttopo.zip
>
>
> Problem can be reproduced by this [Word count 
> topology|https://github.com/hortonworks/storm/blob/perftopos1.x/examples/storm-starter/src/jvm/org/apache/storm/starter/perf/FileReadWordCountTopo.java]
> within a IDE.
> I ran it with 1 spout instance, 2 splitter bolt instances, 2 counter bolt 
> instances.
> The problem is more easily reproduced with WC topology as it causes an 
> explosion of tuples due to splitting a sentence tuple into word tuples. As 
> the bolts have to process more tuples than the  spout is producing, spout 
> needs to operate slower.
> The amount of time it takes for the topology to stall can vary.. but 
> typically under 10 mins. 
> *My theory:*  I suspect there is a race condition in the way ZK is being 
> utilized to enable/disable back pressure. When congested (i.e pressure 
> exceeds high water mark), the bolt's worker records this congested situation 
> in ZK by creating a node. Once the congestion is reduced below the low water 
> mark, it deletes this node. 
> The spout's worker has setup a watch on the parent node, expecting a callback 
> whenever there is change in the child nodes. On receiving the callback the 
> spout's worker lists the parent node to check if there are 0 or more child 
> nodes it is essentially trying to figure out the nature of state change 
> in ZK to determine whether to throttle or not. Subsequently  it setsup 
> another watch in ZK to keep an eye on future changes.
> When there are multiple bolts, there can be rapid creation/deletion of these 
> ZK nodes. Between the time the worker receives a callback and sets up the 
> next watch.. many changes may have undergone in ZK which will go unnoticed by 
> the spout. 
> The condition that the bolts are no longer congested may not get noticed as a 
> result of this.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


Re: Details about the Storm Scheduler

2016-08-23 Thread Roshan Naik


On 8/22/16, 6:55 AM, "Bobby Evans"  wrote:

>Getting the CPU used for a worker is simple, but getting the CPU used for
>individual components is not so simple/almost impossible for
>multi-threaded bolts/spouts.  The current scheduling assumes that the CPU
>for all bolts/spouts is the same for the same component.  This could
>cause a hot spot in one part of the topology to cause rescheduling to
>happen in other parts of the topology needlessly.


I think that can be worked around by looking at the latency of each
component instead of CPU consumption of each component.

-roshan



[jira] [Updated] (STORM-1949) Backpressure can cause spout to stop emitting and stall topology

2016-08-23 Thread Roshan Naik (JIRA)

 [ 
https://issues.apache.org/jira/browse/STORM-1949?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Roshan Naik updated STORM-1949:
---
Attachment: wordcounttopo.zip

Attaching the wordcount topo that i used.

> Backpressure can cause spout to stop emitting and stall topology
> 
>
> Key: STORM-1949
> URL: https://issues.apache.org/jira/browse/STORM-1949
> Project: Apache Storm
>  Issue Type: Bug
>    Reporter: Roshan Naik
>Assignee: Alessandro Bellina
> Attachments: 1.x-branch-works-perfect.png, wordcounttopo.zip
>
>
> Problem can be reproduced by this [Word count 
> topology|https://github.com/hortonworks/storm/blob/perftopos1.x/examples/storm-starter/src/jvm/org/apache/storm/starter/perf/FileReadWordCountTopo.java]
> within a IDE.
> I ran it with 1 spout instance, 2 splitter bolt instances, 2 counter bolt 
> instances.
> The problem is more easily reproduced with WC topology as it causes an 
> explosion of tuples due to splitting a sentence tuple into word tuples. As 
> the bolts have to process more tuples than the  spout is producing, spout 
> needs to operate slower.
> The amount of time it takes for the topology to stall can vary.. but 
> typically under 10 mins. 
> *My theory:*  I suspect there is a race condition in the way ZK is being 
> utilized to enable/disable back pressure. When congested (i.e pressure 
> exceeds high water mark), the bolt's worker records this congested situation 
> in ZK by creating a node. Once the congestion is reduced below the low water 
> mark, it deletes this node. 
> The spout's worker has setup a watch on the parent node, expecting a callback 
> whenever there is change in the child nodes. On receiving the callback the 
> spout's worker lists the parent node to check if there are 0 or more child 
> nodes it is essentially trying to figure out the nature of state change 
> in ZK to determine whether to throttle or not. Subsequently  it setsup 
> another watch in ZK to keep an eye on future changes.
> When there are multiple bolts, there can be rapid creation/deletion of these 
> ZK nodes. Between the time the worker receives a callback and sets up the 
> next watch.. many changes may have undergone in ZK which will go unnoticed by 
> the spout. 
> The condition that the bolts are no longer congested may not get noticed as a 
> result of this.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (STORM-1949) Backpressure can cause spout to stop emitting and stall topology

2016-08-16 Thread Roshan Naik (JIRA)

[ 
https://issues.apache.org/jira/browse/STORM-1949?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15423916#comment-15423916
 ] 

Roshan Naik commented on STORM-1949:


With BP disabled the topo ran fine. Dont think saw any NPE during my runs.

> Backpressure can cause spout to stop emitting and stall topology
> 
>
> Key: STORM-1949
> URL: https://issues.apache.org/jira/browse/STORM-1949
> Project: Apache Storm
>  Issue Type: Bug
>    Reporter: Roshan Naik
> Attachments: 1.x-branch-works-perfect.png
>
>
> Problem can be reproduced by this [Word count 
> topology|https://github.com/hortonworks/storm/blob/perftopos1.x/examples/storm-starter/src/jvm/org/apache/storm/starter/perf/FileReadWordCountTopo.java]
> within a IDE.
> I ran it with 1 spout instance, 2 splitter bolt instances, 2 counter bolt 
> instances.
> The problem is more easily reproduced with WC topology as it causes an 
> explosion of tuples due to splitting a sentence tuple into word tuples. As 
> the bolts have to process more tuples than the  spout is producing, spout 
> needs to operate slower.
> The amount of time it takes for the topology to stall can vary.. but 
> typically under 10 mins. 
> *My theory:*  I suspect there is a race condition in the way ZK is being 
> utilized to enable/disable back pressure. When congested (i.e pressure 
> exceeds high water mark), the bolt's worker records this congested situation 
> in ZK by creating a node. Once the congestion is reduced below the low water 
> mark, it deletes this node. 
> The spout's worker has setup a watch on the parent node, expecting a callback 
> whenever there is change in the child nodes. On receiving the callback the 
> spout's worker lists the parent node to check if there are 0 or more child 
> nodes it is essentially trying to figure out the nature of state change 
> in ZK to determine whether to throttle or not. Subsequently  it setsup 
> another watch in ZK to keep an eye on future changes.
> When there are multiple bolts, there can be rapid creation/deletion of these 
> ZK nodes. Between the time the worker receives a callback and sets up the 
> next watch.. many changes may have undergone in ZK which will go unnoticed by 
> the spout. 
> The condition that the bolts are no longer congested may not get noticed as a 
> result of this.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (STORM-1910) One topology can't use hdfs spout to read from two locations

2016-07-14 Thread Roshan Naik (JIRA)

[ 
https://issues.apache.org/jira/browse/STORM-1910?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15378435#comment-15378435
 ] 

Roshan Naik commented on STORM-1910:


[~ptgoetz] should this be marked for 1.0.2 as well ?

> One topology can't use hdfs spout to read from two locations
> 
>
> Key: STORM-1910
> URL: https://issues.apache.org/jira/browse/STORM-1910
> Project: Apache Storm
>  Issue Type: Bug
>  Components: storm-hdfs
>Affects Versions: 1.0.1
>Reporter: Raghav Kumar Gautam
>Assignee: Roshan Naik
> Fix For: 2.0.0, 1.1.0
>
>
> The hdfs uri is passed using config:
> {code}
> conf.put(Configs.HDFS_URI, hdfsUri);
> {code}
> I see two problems with this approach:
> 1. If someone wants to used two hdfsUri in same or different spouts - then 
> that does not seem feasible.
> https://github.com/apache/storm/blob/d17b3b9c3cbc89d854bfb436d213d11cfd4545ec/examples/storm-starter/src/jvm/storm/starter/HdfsSpoutTopology.java#L117-L117
> https://github.com/apache/storm/blob/d17b3b9c3cbc89d854bfb436d213d11cfd4545ec/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/spout/HdfsSpout.java#L331-L331
> {code}
> if ( !conf.containsKey(Configs.SOURCE_DIR) ) {
>   LOG.error(Configs.SOURCE_DIR + " setting is required");
>   throw new RuntimeException(Configs.SOURCE_DIR + " setting is required");
> }
> this.sourceDirPath = new Path( conf.get(Configs.SOURCE_DIR).toString() );
> {code}
> 2. It does not fail fast i.e. at the time of topology submissing. We can fail 
> fast if the hdfs path is invalid or credentials/permissions are not ok. Such 
> errors at this time can only be detected at runtime by looking at the worker 
> logs.
> https://github.com/apache/storm/blob/d17b3b9c3cbc89d854bfb436d213d11cfd4545ec/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/spout/HdfsSpout.java#L297-L297



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


Re: Request for reviewing

2016-07-14 Thread Roshan Naik
STORM-1910 
-roshan


On 7/14/16, 8:59 AM, "Jungtaek Lim"  wrote:

>We seems close to make all issues for 1.0.2 epic resolved soon.
>
>Epic link: https://issues.apache.org/jira/browse/STORM-1855
>All of open issues are in progress of reviewing. Most of them got +1 and
>is
>expected to be merged in soon.
>
>Is anyone having other critical or blocker issues which need to be
>included
>to 1.0.2?
>
>2016년 7월 9일 (토) 오전 12:34, P. Taylor Goetz 님이 작성:
>
>> Thanks Juntaek, I finally got some time to review it.
>>
>> Regarding the 1.0.2 release, there are a few open/in-progress issues
>> remaining, but I think we should be in a position to cut a new RC
>>sometime
>> next week.
>>
>> -Taylor
>>
>>
>> > On Jul 8, 2016, at 7:43 AM, Jungtaek Lim  wrote:
>> >
>> > Hi devs,
>> >
>> > I've a pull request for a critical issue, and the patch modifies the
>> heart
>> > of supervisor code, so I'd like to make my patch get reviewed by more
>> > people in community.
>> >
>> > STORM-1934: Race condition between sync-supervisor and sync-processes
>> > raises several strange issues
>> > 
>> > Discussion is on this PR: https://github.com/apache/storm/pull/1528
>> >
>> > Btw, since Storm 1.0.2 has lots of bugfix marked as 'Critical' and
>> 'Major'
>> > which are reported by users several times, I'd like to see 1.0.2
>> > official release
>> > sooner.
>> >
>> >
>> 
>>https://issues.apache.org/jira/issues/?jql=project%20%3D%20Storm%20AND%20
>>%22epic%20link%22%20%3D%20STORM-1855%20order%20by%20priority
>> >
>> > There're 4 unresolved issues but pull requests are available for
>> 'Critical'
>> > issues so once we resolve them I guess we could get other issues out
>>of
>> > 1.0.2 and start working for releasing.
>> >
>> > Given that STORM-1934 has two binding +1 votes, I'd like to wait for
>> > several days, and merge if no further discussion is in progress.
>> >
>> > Thanks in advance for taking your time.
>> >
>> > Best regards,
>> > Jungtaek Lim (HeartSaVioR)
>>
>>



[jira] [Updated] (STORM-1956) Disable Backpressure by default

2016-07-08 Thread Roshan Naik (JIRA)

 [ 
https://issues.apache.org/jira/browse/STORM-1956?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Roshan Naik updated STORM-1956:
---
Priority: Blocker  (was: Major)

> Disable Backpressure by default
> ---
>
> Key: STORM-1956
> URL: https://issues.apache.org/jira/browse/STORM-1956
> Project: Apache Storm
>  Issue Type: Bug
>  Components: storm-core
>Affects Versions: 1.0.0, 1.0.1
>    Reporter: Roshan Naik
>Assignee: Roshan Naik
>Priority: Blocker
> Fix For: 1.0.2
>
>
> Some of the context on this is captured in STORM-1949 
> In short.. wait for BP mechanism to mature some more and be production ready 
> before we enable by default.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (STORM-1956) Disable Backpressure by default

2016-07-08 Thread Roshan Naik (JIRA)

 [ 
https://issues.apache.org/jira/browse/STORM-1956?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Roshan Naik updated STORM-1956:
---
Fix Version/s: 1.0.2

> Disable Backpressure by default
> ---
>
> Key: STORM-1956
> URL: https://issues.apache.org/jira/browse/STORM-1956
> Project: Apache Storm
>  Issue Type: Bug
>  Components: storm-core
>Affects Versions: 1.0.0, 1.0.1
>    Reporter: Roshan Naik
>Assignee: Roshan Naik
>Priority: Blocker
> Fix For: 1.0.2
>
>
> Some of the context on this is captured in STORM-1949 
> In short.. wait for BP mechanism to mature some more and be production ready 
> before we enable by default.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (STORM-1949) Backpressure can cause spout to stop emitting and stall topology

2016-07-08 Thread Roshan Naik (JIRA)

[ 
https://issues.apache.org/jira/browse/STORM-1949?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15368640#comment-15368640
 ] 

Roshan Naik commented on STORM-1949:



Have not worked out a concrete solution to avoiding ZK as yet. But 
[~sriharsha]'s  line of thinking is interesting ... basically see if we can use 
the internal messaging system as opposed to messaging over ZK. 

Opened STORM-1956 for disabling BP by default. 

> Backpressure can cause spout to stop emitting and stall topology
> 
>
> Key: STORM-1949
> URL: https://issues.apache.org/jira/browse/STORM-1949
> Project: Apache Storm
>  Issue Type: Bug
>    Reporter: Roshan Naik
>
> Problem can be reproduced by this [Word count 
> topology|https://github.com/hortonworks/storm/blob/perftopos1.x/examples/storm-starter/src/jvm/org/apache/storm/starter/perf/FileReadWordCountTopo.java]
> within a IDE.
> I ran it with 1 spout instance, 2 splitter bolt instances, 2 counter bolt 
> instances.
> The problem is more easily reproduced with WC topology as it causes an 
> explosion of tuples due to splitting a sentence tuple into word tuples. As 
> the bolts have to process more tuples than the  spout is producing, spout 
> needs to operate slower.
> The amount of time it takes for the topology to stall can vary.. but 
> typically under 10 mins. 
> *My theory:*  I suspect there is a race condition in the way ZK is being 
> utilized to enable/disable back pressure. When congested (i.e pressure 
> exceeds high water mark), the bolt's worker records this congested situation 
> in ZK by creating a node. Once the congestion is reduced below the low water 
> mark, it deletes this node. 
> The spout's worker has setup a watch on the parent node, expecting a callback 
> whenever there is change in the child nodes. On receiving the callback the 
> spout's worker lists the parent node to check if there are 0 or more child 
> nodes it is essentially trying to figure out the nature of state change 
> in ZK to determine whether to throttle or not. Subsequently  it setsup 
> another watch in ZK to keep an eye on future changes.
> When there are multiple bolts, there can be rapid creation/deletion of these 
> ZK nodes. Between the time the worker receives a callback and sets up the 
> next watch.. many changes may have undergone in ZK which will go unnoticed by 
> the spout. 
> The condition that the bolts are no longer congested may not get noticed as a 
> result of this.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (STORM-1956) Disable Backpressure by default

2016-07-08 Thread Roshan Naik (JIRA)
Roshan Naik created STORM-1956:
--

 Summary: Disable Backpressure by default
 Key: STORM-1956
 URL: https://issues.apache.org/jira/browse/STORM-1956
 Project: Apache Storm
  Issue Type: Bug
  Components: storm-core
Affects Versions: 1.0.0, 1.0.1
Reporter: Roshan Naik
Assignee: Roshan Naik


Some of the context on this is captured in STORM-1949 
In short.. wait for BP mechanism to mature some more and be production ready 
before we enable by default.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Comment Edited] (STORM-1949) Backpressure can cause spout to stop emitting and stall topology

2016-07-08 Thread Roshan Naik (JIRA)

[ 
https://issues.apache.org/jira/browse/STORM-1949?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15368451#comment-15368451
 ] 

Roshan Naik edited comment on STORM-1949 at 7/8/16 8:53 PM:


[~revans2] Not sure what you mean by "back on the write" .. are u saying have 
background thread that simply polls ZK every so often ?  That might fix this 
issue.

However, there is one basic issue with this BP mechanism in general. It can put 
too much load on ZK. For each enable/disable throttle signal raised by any 
worker we have all this interaction going on with ZK..

- Some worker adds/deletes ZK node 
- ZK issues callbacks to all workers with watches setup
- All those workers will list the parent node in ZK to count the number of 
children (expensive?)
- All those workers will setup another watch in ZK
 
Given that PaceMaker was introduced to take load off of ZK... this approach 
feels like a regression in terms of ability to scale. There are some other 
issues as well but thats for later.

After reviewing BP, I done feel it is sufficiently mature to be considered 
stable and ready for production. 

IMO Until we have a more solid BP mechanism we should disable it by default as 
soon as possible. I can open another jira for that.


was (Author: roshan_naik):
[~revans2] Not sure what you mean by "back on the write" .. are u saying have 
background thread that simply polls ZK every so often ?  That might fix this 
issue.

However, there is one basic issue with this BP mechanism in general. Its can 
put too much load on ZK. For each enable/disable throttle signal raised by any 
worker we have all this interaction going on with ZK..

- Some worker adds/deletes ZK node 
- ZK issues callbacks to all workers with watches setup
- All those workers will list the parent node in ZK to count the number of 
children (expensive?)
- All those workers will setup another watch in ZK
 
Given that PaceMaker was introduced to take load off of ZK... this approach 
feels like a regression. There are some other issues as well but thats for 
later.

After reviewing BP, I feel it is not mature enough to be considered stable and 
ready for production. 

IMO Until we have a more solid BP mechanism we should disable it by default as 
soon as possible. I can open another jira for that.

> Backpressure can cause spout to stop emitting and stall topology
> 
>
> Key: STORM-1949
> URL: https://issues.apache.org/jira/browse/STORM-1949
> Project: Apache Storm
>  Issue Type: Bug
>Reporter: Roshan Naik
>
> Problem can be reproduced by this [Word count 
> topology|https://github.com/hortonworks/storm/blob/perftopos1.x/examples/storm-starter/src/jvm/org/apache/storm/starter/perf/FileReadWordCountTopo.java]
> within a IDE.
> I ran it with 1 spout instance, 2 splitter bolt instances, 2 counter bolt 
> instances.
> The problem is more easily reproduced with WC topology as it causes an 
> explosion of tuples due to splitting a sentence tuple into word tuples. As 
> the bolts have to process more tuples than the  spout is producing, spout 
> needs to operate slower.
> The amount of time it takes for the topology to stall can vary.. but 
> typically under 10 mins. 
> *My theory:*  I suspect there is a race condition in the way ZK is being 
> utilized to enable/disable back pressure. When congested (i.e pressure 
> exceeds high water mark), the bolt's worker records this congested situation 
> in ZK by creating a node. Once the congestion is reduced below the low water 
> mark, it deletes this node. 
> The spout's worker has setup a watch on the parent node, expecting a callback 
> whenever there is change in the child nodes. On receiving the callback the 
> spout's worker lists the parent node to check if there are 0 or more child 
> nodes it is essentially trying to figure out the nature of state change 
> in ZK to determine whether to throttle or not. Subsequently  it setsup 
> another watch in ZK to keep an eye on future changes.
> When there are multiple bolts, there can be rapid creation/deletion of these 
> ZK nodes. Between the time the worker receives a callback and sets up the 
> next watch.. many changes may have undergone in ZK which will go unnoticed by 
> the spout. 
> The condition that the bolts are no longer congested may not get noticed as a 
> result of this.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Comment Edited] (STORM-1949) Backpressure can cause spout to stop emitting and stall topology

2016-07-08 Thread Roshan Naik (JIRA)

[ 
https://issues.apache.org/jira/browse/STORM-1949?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15368451#comment-15368451
 ] 

Roshan Naik edited comment on STORM-1949 at 7/8/16 8:50 PM:


[~revans2] Not sure what you mean by "back on the write" .. are u saying have 
background thread that simply polls ZK every so often ?  That might fix this 
issue.

However, there is one basic issue with this BP mechanism in general. Its can 
put too much load on ZK. For each enable/disable throttle signal raised by any 
worker we have all this interaction going on with ZK..

- Some worker adds/deletes ZK node 
- ZK issues callbacks to all workers with watches setup
- All those workers will list the parent node in ZK to count the number of 
children (expensive?)
- All those workers will setup another watch in ZK
 
Given that PaceMaker was introduced to take load off of ZK... this approach 
feels like a regression. There are some other issues as well but thats for 
later.

After reviewing BP, I feel it is not mature enough to be considered stable and 
ready for production. 

IMO Until we have a more solid BP mechanism we should disable it by default as 
soon as possible. I can open another jira for that.


was (Author: roshan_naik):
[~revans2] Not sure what you mean by "back on the write" .. are u saying have 
background thread that simply polls ZK every so often ?  That might fix this 
issue.

However, there is one basic issue with this BP mechanism in general. Its can 
put too much load on ZK. For each enable/disable throttle signal raised by any 
worker we have all this interaction going on with ZK..

- Some worker adds/deletes ZK node 
- ZK issues callbacks to all workers with watches setup
- All those workers will list the parent node in ZK to count the number of 
children (expensive?)
- All those workers will setup another watch in ZK
 
Given that PaceMaker was introduced to take load off of ZK... this approach 
feels like a regression. There are some other issues as well but thats for a 
different JIRA.

After reviewing BP, I feel it is not mature enough to be considered stable and 
ready for production. 

IMO Until we have a more solid BP mechanism we should disable it by default as 
soon as possible. I can open another jira for that.

> Backpressure can cause spout to stop emitting and stall topology
> 
>
> Key: STORM-1949
> URL: https://issues.apache.org/jira/browse/STORM-1949
> Project: Apache Storm
>  Issue Type: Bug
>Reporter: Roshan Naik
>
> Problem can be reproduced by this [Word count 
> topology|https://github.com/hortonworks/storm/blob/perftopos1.x/examples/storm-starter/src/jvm/org/apache/storm/starter/perf/FileReadWordCountTopo.java]
> within a IDE.
> I ran it with 1 spout instance, 2 splitter bolt instances, 2 counter bolt 
> instances.
> The problem is more easily reproduced with WC topology as it causes an 
> explosion of tuples due to splitting a sentence tuple into word tuples. As 
> the bolts have to process more tuples than the  spout is producing, spout 
> needs to operate slower.
> The amount of time it takes for the topology to stall can vary.. but 
> typically under 10 mins. 
> *My theory:*  I suspect there is a race condition in the way ZK is being 
> utilized to enable/disable back pressure. When congested (i.e pressure 
> exceeds high water mark), the bolt's worker records this congested situation 
> in ZK by creating a node. Once the congestion is reduced below the low water 
> mark, it deletes this node. 
> The spout's worker has setup a watch on the parent node, expecting a callback 
> whenever there is change in the child nodes. On receiving the callback the 
> spout's worker lists the parent node to check if there are 0 or more child 
> nodes it is essentially trying to figure out the nature of state change 
> in ZK to determine whether to throttle or not. Subsequently  it setsup 
> another watch in ZK to keep an eye on future changes.
> When there are multiple bolts, there can be rapid creation/deletion of these 
> ZK nodes. Between the time the worker receives a callback and sets up the 
> next watch.. many changes may have undergone in ZK which will go unnoticed by 
> the spout. 
> The condition that the bolts are no longer congested may not get noticed as a 
> result of this.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (STORM-1949) Backpressure can cause spout to stop emitting and stall topology

2016-07-08 Thread Roshan Naik (JIRA)

[ 
https://issues.apache.org/jira/browse/STORM-1949?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15368451#comment-15368451
 ] 

Roshan Naik commented on STORM-1949:


[~revans2] Not sure what you mean by "back on the write" .. are u saying have 
background thread that simply polls ZK every so often ?  That might fix this 
issue.

However, there is one basic issue with this BP mechanism in general. Its can 
put too much load on ZK. For each enable/disable throttle signal raised by any 
worker we have all this interaction going on with ZK..

- Some worker adds/deletes ZK node 
- ZK issues callbacks to all workers with watches setup
- All those workers will list the parent node in ZK to count the number of 
children (expensive?)
- All those workers will setup another watch in ZK
 
Given that PaceMaker was introduced to take load off of ZK... this approach 
feels like a regression. There are some other issues as well but thats for a 
different JIRA.

After reviewing BP, I feel it is not mature enough to be considered stable and 
ready for production. 

IMO Until we have a more solid BP mechanism we should disable it by default as 
soon as possible. I can open another jira for that.

> Backpressure can cause spout to stop emitting and stall topology
> 
>
> Key: STORM-1949
> URL: https://issues.apache.org/jira/browse/STORM-1949
> Project: Apache Storm
>  Issue Type: Bug
>Reporter: Roshan Naik
>
> Problem can be reproduced by this [Word count 
> topology|https://github.com/hortonworks/storm/blob/perftopos1.x/examples/storm-starter/src/jvm/org/apache/storm/starter/perf/FileReadWordCountTopo.java]
> within a IDE.
> I ran it with 1 spout instance, 2 splitter bolt instances, 2 counter bolt 
> instances.
> The problem is more easily reproduced with WC topology as it causes an 
> explosion of tuples due to splitting a sentence tuple into word tuples. As 
> the bolts have to process more tuples than the  spout is producing, spout 
> needs to operate slower.
> The amount of time it takes for the topology to stall can vary.. but 
> typically under 10 mins. 
> *My theory:*  I suspect there is a race condition in the way ZK is being 
> utilized to enable/disable back pressure. When congested (i.e pressure 
> exceeds high water mark), the bolt's worker records this congested situation 
> in ZK by creating a node. Once the congestion is reduced below the low water 
> mark, it deletes this node. 
> The spout's worker has setup a watch on the parent node, expecting a callback 
> whenever there is change in the child nodes. On receiving the callback the 
> spout's worker lists the parent node to check if there are 0 or more child 
> nodes it is essentially trying to figure out the nature of state change 
> in ZK to determine whether to throttle or not. Subsequently  it setsup 
> another watch in ZK to keep an eye on future changes.
> When there are multiple bolts, there can be rapid creation/deletion of these 
> ZK nodes. Between the time the worker receives a callback and sets up the 
> next watch.. many changes may have undergone in ZK which will go unnoticed by 
> the spout. 
> The condition that the bolts are no longer congested may not get noticed as a 
> result of this.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (STORM-1910) One topology can't use hdfs spout to read from two locations

2016-07-07 Thread Roshan Naik (JIRA)

[ 
https://issues.apache.org/jira/browse/STORM-1910?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15367096#comment-15367096
 ] 

Roshan Naik commented on STORM-1910:


WRT Pt# 2 in the description, we cannot check for valid HDFS path on the client 
side as it cannot be assumed that HDFS is configure and available on the host 
from where the topology is being submitted.



> One topology can't use hdfs spout to read from two locations
> 
>
> Key: STORM-1910
> URL: https://issues.apache.org/jira/browse/STORM-1910
> Project: Apache Storm
>  Issue Type: Bug
>  Components: storm-hdfs
>Affects Versions: 1.0.1
>Reporter: Raghav Kumar Gautam
>Assignee: Roshan Naik
> Fix For: 1.1.0
>
>
> The hdfs uri is passed using config:
> {code}
> conf.put(Configs.HDFS_URI, hdfsUri);
> {code}
> I see two problems with this approach:
> 1. If someone wants to used two hdfsUri in same or different spouts - then 
> that does not seem feasible.
> https://github.com/apache/storm/blob/d17b3b9c3cbc89d854bfb436d213d11cfd4545ec/examples/storm-starter/src/jvm/storm/starter/HdfsSpoutTopology.java#L117-L117
> https://github.com/apache/storm/blob/d17b3b9c3cbc89d854bfb436d213d11cfd4545ec/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/spout/HdfsSpout.java#L331-L331
> {code}
> if ( !conf.containsKey(Configs.SOURCE_DIR) ) {
>   LOG.error(Configs.SOURCE_DIR + " setting is required");
>   throw new RuntimeException(Configs.SOURCE_DIR + " setting is required");
> }
> this.sourceDirPath = new Path( conf.get(Configs.SOURCE_DIR).toString() );
> {code}
> 2. It does not fail fast i.e. at the time of topology submissing. We can fail 
> fast if the hdfs path is invalid or credentials/permissions are not ok. Such 
> errors at this time can only be detected at runtime by looking at the worker 
> logs.
> https://github.com/apache/storm/blob/d17b3b9c3cbc89d854bfb436d213d11cfd4545ec/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/spout/HdfsSpout.java#L297-L297



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (STORM-1949) Storm backpressure can cause spout to stop emitting and stall topology

2016-07-06 Thread Roshan Naik (JIRA)
Roshan Naik created STORM-1949:
--

 Summary: Storm backpressure can cause spout to stop emitting and 
stall topology
 Key: STORM-1949
 URL: https://issues.apache.org/jira/browse/STORM-1949
 Project: Apache Storm
  Issue Type: Bug
Reporter: Roshan Naik


Problem can be reproduced by this [Word count 
topology|https://github.com/hortonworks/storm/blob/perftopos1.x/examples/storm-starter/src/jvm/org/apache/storm/starter/perf/FileReadWordCountTopo.java]
within a IDE.
I ran it with 1 spout instance, 2 splitter bolt instances, 2 counter bolt 
instances.

The problem is more easily reproduced with WC topology as it causes an 
explosion of tuples due to splitting a sentence tuple into word tuples. As the 
bolts have to process more tuples than the  spout is producing, spout needs to 
operate slower.

The amount of time it takes for the topology to stall can vary.. but typically 
under 10 mins. 

*My theory:*  I suspect there is a race condition in the way ZK is being 
utilized to enable/disable back pressure. When congested (i.e pressure exceeds 
high water mark), the bolt's worker records this congested situation in ZK by 
creating a node. Once the congestion is reduced below the low water mark, it 
deletes this node. 
The spout's worker has setup a watch on the parent node, expecting a callback 
whenever there is change in the child nodes. On receiving the callback the 
spout's worker lists the parent node to check if there are 0 or more child 
nodes it is essentially trying to figure out the nature of state change in 
ZK to determine whether to throttle or not. Subsequently  it setsup another 
watch in ZK to keep an eye on future changes.

When there are multiple bolts, there can be rapid creation/deletion of these ZK 
nodes. Between the time the worker receives a callback and sets up the next 
watch.. many changes may have undergone in ZK which will go unnoticed by the 
spout. 

The condition that the bolts are no longer congested may not get noticed as a 
result of this.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (STORM-1949) Backpressure can cause spout to stop emitting and stall topology

2016-07-06 Thread Roshan Naik (JIRA)

 [ 
https://issues.apache.org/jira/browse/STORM-1949?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Roshan Naik updated STORM-1949:
---
Summary: Backpressure can cause spout to stop emitting and stall topology  
(was: Storm backpressure can cause spout to stop emitting and stall topology)

> Backpressure can cause spout to stop emitting and stall topology
> 
>
> Key: STORM-1949
> URL: https://issues.apache.org/jira/browse/STORM-1949
> Project: Apache Storm
>  Issue Type: Bug
>    Reporter: Roshan Naik
>
> Problem can be reproduced by this [Word count 
> topology|https://github.com/hortonworks/storm/blob/perftopos1.x/examples/storm-starter/src/jvm/org/apache/storm/starter/perf/FileReadWordCountTopo.java]
> within a IDE.
> I ran it with 1 spout instance, 2 splitter bolt instances, 2 counter bolt 
> instances.
> The problem is more easily reproduced with WC topology as it causes an 
> explosion of tuples due to splitting a sentence tuple into word tuples. As 
> the bolts have to process more tuples than the  spout is producing, spout 
> needs to operate slower.
> The amount of time it takes for the topology to stall can vary.. but 
> typically under 10 mins. 
> *My theory:*  I suspect there is a race condition in the way ZK is being 
> utilized to enable/disable back pressure. When congested (i.e pressure 
> exceeds high water mark), the bolt's worker records this congested situation 
> in ZK by creating a node. Once the congestion is reduced below the low water 
> mark, it deletes this node. 
> The spout's worker has setup a watch on the parent node, expecting a callback 
> whenever there is change in the child nodes. On receiving the callback the 
> spout's worker lists the parent node to check if there are 0 or more child 
> nodes it is essentially trying to figure out the nature of state change 
> in ZK to determine whether to throttle or not. Subsequently  it setsup 
> another watch in ZK to keep an eye on future changes.
> When there are multiple bolts, there can be rapid creation/deletion of these 
> ZK nodes. Between the time the worker receives a callback and sets up the 
> next watch.. many changes may have undergone in ZK which will go unnoticed by 
> the spout. 
> The condition that the bolts are no longer congested may not get noticed as a 
> result of this.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (STORM-1892) class org.apache.storm.hdfs.spout.TextFileReader should be public

2016-06-08 Thread Roshan Naik (JIRA)
Roshan Naik created STORM-1892:
--

 Summary: class org.apache.storm.hdfs.spout.TextFileReader should 
be public
 Key: STORM-1892
 URL: https://issues.apache.org/jira/browse/STORM-1892
 Project: Apache Storm
  Issue Type: Bug
Affects Versions: 1.0.1
Reporter: Roshan Naik
Assignee: Roshan Naik






--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


Re: STORM JIRA being spammed - CANNOT comment on JIRA (as a result of the counter spam measures)

2016-05-12 Thread Roshan Naik
Myself too. Thanks.
-Roshan 


On 5/12/16, 1:14 PM, "Bobby Evans"  wrote:

>Sorry about that you should be added as a contributor now.  Is there
>anyone else that I should add as a contributor?
>Also we have not been maintaining the commiters ACL very well either.
>Are there any commiter/PMC out there that want to be added to the
>committer/PMC roles.  There really isn't much different between the
>roles, but I am happy to adjust it if needed.
> - Bobby 
>
>On Thursday, May 12, 2016 2:57 PM, Hugo Da Cruz Louro
> wrote:
> 
>
> Hi,
>
>I understand that there is a wave of spam going on, but the measure that
>was taken, which limits active contributors to comment on JIRA is not
>very practical. I have several open JIRA issues that I am working on, or
>that I am interested in contributing to, that I cannot comment on right
>now.
>
>Can I get my access back, please.
>
>GitHub: hmcl
>Apache JIRA Id: hmclouro
>
>Thanks,
>Hugo
>
>  



[jira] [Commented] (STORM-1772) Create topologies for measuring performance

2016-05-10 Thread Roshan Naik (JIRA)

[ 
https://issues.apache.org/jira/browse/STORM-1772?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15279190#comment-15279190
 ] 

Roshan Naik commented on STORM-1772:


Hi [~mauzhang],
Yes thats it. I first observed that perf difference issue when working on 
STORM-1632, but was not able to get to the bottom of it. The storm native 
topology mentioned here : 
https://github.com/apache/storm/pull/1217#issuecomment-201074919

I can try to locate the benchmark-specific version of the topology but its a 
straightforward rewrite.

The storm native showed a  difference of ~12% when doing a A/B test (with and 
without the fix)
The benchmark specific version of the topology .. it was 25%  as noted in the 
description of STORM-1632.



IMO..  briefly ignoring the perf diff issue, it would be good to go ahead and 
see what we can incorporate from that benchmark . In this jira my goal is to 
add a few topologies for perf testing... not to create a benchmarking 
tool/framework itself. In that sense its not conflicting with STORM-642. 

*side note:* If we are adding a benchmarking framework, it would be good if it 
can run standard Storm topologies directly and not require topologies to be 
written specifically for it.

> Create topologies for measuring performance
> ---
>
> Key: STORM-1772
> URL: https://issues.apache.org/jira/browse/STORM-1772
> Project: Apache Storm
>  Issue Type: Bug
>    Reporter: Roshan Naik
>    Assignee: Roshan Naik
>
> Would be very useful to have some simple reference topologies included with 
> Storm that can be used to measure performance both by devs during development 
> (to start with) and perhaps also on a real storm cluster (subsequently). 
> To start with, the goal is to put the focus on the performance 
> characteristics of individual building blocks such as specifics bolts, 
> spouts,  grouping options, queues, etc. So, initially biased towards 
> micro-benchmarking but subsequently we could add higher level ones too.
> Although there is a storm benchmarking tool (originally written by Intel?) 
> that can be used, and i have personally used it, its better for this to be 
> integrated into Storm proper and also maintained by devs as storm evolves. 
> On a side note, in some instances I have noticed (to my surprise) that the 
> perf numbers change when the topologies written for Intel benchmark when 
> rewritten without the required wrappers so that they runs directly under 
> Storm.
> Have a few topologies in mind for measuring each of these:
> # *Queuing and Spout Emit Performance:* A topology with a Generator Spout but 
> no bolts.
> # *Queuing & Grouping performance*:   Generator Spout -> A grouping method -> 
> DevNull Bolt
> # *Hdfs Bolt:*Generator Spout ->  Hdfs Bolt
> # *Hdfs Spout:*   Hdfs Spout ->  DevNull Botl
> # *Kafka Spout:*   Kafka Spout ->  DevNull Bolt 
> # *Simple Data Movement*: Kafka Spout -> Hdfs Bolt
> Shall add these for Storm core first. Then we can have the same for Trident 
> also.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (STORM-1772) Create topologies for measuring performance

2016-05-09 Thread Roshan Naik (JIRA)
Roshan Naik created STORM-1772:
--

 Summary: Create topologies for measuring performance
 Key: STORM-1772
 URL: https://issues.apache.org/jira/browse/STORM-1772
 Project: Apache Storm
  Issue Type: Bug
Reporter: Roshan Naik


Would be very useful to have some simple reference topologies included with 
Storm that can be used to measure performance that can be used both by devs 
during development (to start with) and perhaps also on a real storm cluster 
(subsequently). 

To start with, the goal is to put the focus on the performance characteristics 
of individual building blocks such as specifics bolts, spouts,  grouping 
options, queues, etc. So, initially biased towards micro-benchmarking but 
subsequently we could add higher level ones too.

Although there is a storm benchmarking tool (originally written by Intel?) that 
can be used, and i have personally used, its better for this to be integrated 
into Storm proper and also maintained by devs as storm evolves. 

On a side note, in some instances I have noticed (to my surprise) that the perf 
numbers change when the topologies written for Intel benchmark when rewritten 
without the required wrappers so that they runs directly under Storm.

Have a few topologies in mind for measuring each of these:

# *Queuing and Spout Emit Performance:* A topology with a Generator Spout but 
no bolts.
# *Queuing & Grouping performance*:   Generator Spout -> A grouping method -> 
DevNull Bolt
# *Hdfs Bolt:*Generator Spout ->  Hdfs Bolt
# *Hdfs Spout:*   Hdfs Spout ->  DevNull Botl
# *Kafka Spout:*   Kafka Spout ->  DevNull Bolt 
# *Simple Data Movement*: Kafka Spout -> Hdfs Bolt

Shall add these for Storm core first. Then we can have the same for Trident 
also.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (STORM-1772) Create topologies for measuring performance

2016-05-09 Thread Roshan Naik (JIRA)

 [ 
https://issues.apache.org/jira/browse/STORM-1772?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Roshan Naik updated STORM-1772:
---
Description: 
Would be very useful to have some simple reference topologies included with 
Storm that can be used to measure performance both by devs during development 
(to start with) and perhaps also on a real storm cluster (subsequently). 

To start with, the goal is to put the focus on the performance characteristics 
of individual building blocks such as specifics bolts, spouts,  grouping 
options, queues, etc. So, initially biased towards micro-benchmarking but 
subsequently we could add higher level ones too.

Although there is a storm benchmarking tool (originally written by Intel?) that 
can be used, and i have personally used it, its better for this to be 
integrated into Storm proper and also maintained by devs as storm evolves. 

On a side note, in some instances I have noticed (to my surprise) that the perf 
numbers change when the topologies written for Intel benchmark when rewritten 
without the required wrappers so that they runs directly under Storm.

Have a few topologies in mind for measuring each of these:

# *Queuing and Spout Emit Performance:* A topology with a Generator Spout but 
no bolts.
# *Queuing & Grouping performance*:   Generator Spout -> A grouping method -> 
DevNull Bolt
# *Hdfs Bolt:*Generator Spout ->  Hdfs Bolt
# *Hdfs Spout:*   Hdfs Spout ->  DevNull Botl
# *Kafka Spout:*   Kafka Spout ->  DevNull Bolt 
# *Simple Data Movement*: Kafka Spout -> Hdfs Bolt

Shall add these for Storm core first. Then we can have the same for Trident 
also.

  was:
Would be very useful to have some simple reference topologies included with 
Storm that can be used to measure performance that can be used both by devs 
during development (to start with) and perhaps also on a real storm cluster 
(subsequently). 

To start with, the goal is to put the focus on the performance characteristics 
of individual building blocks such as specifics bolts, spouts,  grouping 
options, queues, etc. So, initially biased towards micro-benchmarking but 
subsequently we could add higher level ones too.

Although there is a storm benchmarking tool (originally written by Intel?) that 
can be used, and i have personally used, its better for this to be integrated 
into Storm proper and also maintained by devs as storm evolves. 

On a side note, in some instances I have noticed (to my surprise) that the perf 
numbers change when the topologies written for Intel benchmark when rewritten 
without the required wrappers so that they runs directly under Storm.

Have a few topologies in mind for measuring each of these:

# *Queuing and Spout Emit Performance:* A topology with a Generator Spout but 
no bolts.
# *Queuing & Grouping performance*:   Generator Spout -> A grouping method -> 
DevNull Bolt
# *Hdfs Bolt:*Generator Spout ->  Hdfs Bolt
# *Hdfs Spout:*   Hdfs Spout ->  DevNull Botl
# *Kafka Spout:*   Kafka Spout ->  DevNull Bolt 
# *Simple Data Movement*: Kafka Spout -> Hdfs Bolt

Shall add these for Storm core first. Then we can have the same for Trident 
also.


> Create topologies for measuring performance
> ---
>
> Key: STORM-1772
> URL: https://issues.apache.org/jira/browse/STORM-1772
> Project: Apache Storm
>  Issue Type: Bug
>Reporter: Roshan Naik
>
> Would be very useful to have some simple reference topologies included with 
> Storm that can be used to measure performance both by devs during development 
> (to start with) and perhaps also on a real storm cluster (subsequently). 
> To start with, the goal is to put the focus on the performance 
> characteristics of individual building blocks such as specifics bolts, 
> spouts,  grouping options, queues, etc. So, initially biased towards 
> micro-benchmarking but subsequently we could add higher level ones too.
> Although there is a storm benchmarking tool (originally written by Intel?) 
> that can be used, and i have personally used it, its better for this to be 
> integrated into Storm proper and also maintained by devs as storm evolves. 
> On a side note, in some instances I have noticed (to my surprise) that the 
> perf numbers change when the topologies written for Intel benchmark when 
> rewritten without the required wrappers so that they runs directly under 
> Storm.
> Have a few topologies in mind for measuring each of these:
> # *Queuing and Spout Emit Performance:* A topology with a Generator Spout but 
> no bolts.
> # *Queuing & Grouping performance*:   Generator Spout -> A grouping method -> 
> DevNull Bolt
> # *Hdfs Bolt:*Generator Spout ->  Hdfs Bolt
> # *Hdfs Spout:*   Hdfs Spout ->  DevNu

[jira] [Assigned] (STORM-1772) Create topologies for measuring performance

2016-05-09 Thread Roshan Naik (JIRA)

 [ 
https://issues.apache.org/jira/browse/STORM-1772?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Roshan Naik reassigned STORM-1772:
--

Assignee: Roshan Naik

> Create topologies for measuring performance
> ---
>
> Key: STORM-1772
> URL: https://issues.apache.org/jira/browse/STORM-1772
> Project: Apache Storm
>  Issue Type: Bug
>    Reporter: Roshan Naik
>    Assignee: Roshan Naik
>
> Would be very useful to have some simple reference topologies included with 
> Storm that can be used to measure performance both by devs during development 
> (to start with) and perhaps also on a real storm cluster (subsequently). 
> To start with, the goal is to put the focus on the performance 
> characteristics of individual building blocks such as specifics bolts, 
> spouts,  grouping options, queues, etc. So, initially biased towards 
> micro-benchmarking but subsequently we could add higher level ones too.
> Although there is a storm benchmarking tool (originally written by Intel?) 
> that can be used, and i have personally used it, its better for this to be 
> integrated into Storm proper and also maintained by devs as storm evolves. 
> On a side note, in some instances I have noticed (to my surprise) that the 
> perf numbers change when the topologies written for Intel benchmark when 
> rewritten without the required wrappers so that they runs directly under 
> Storm.
> Have a few topologies in mind for measuring each of these:
> # *Queuing and Spout Emit Performance:* A topology with a Generator Spout but 
> no bolts.
> # *Queuing & Grouping performance*:   Generator Spout -> A grouping method -> 
> DevNull Bolt
> # *Hdfs Bolt:*Generator Spout ->  Hdfs Bolt
> # *Hdfs Spout:*   Hdfs Spout ->  DevNull Botl
> # *Kafka Spout:*   Kafka Spout ->  DevNull Bolt 
> # *Simple Data Movement*: Kafka Spout -> Hdfs Bolt
> Shall add these for Storm core first. Then we can have the same for Trident 
> also.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


Re: [VOTE] Release Apache Storm 0.10.1 (rc2)

2016-05-03 Thread Roshan Naik
This is missing HdfsSpout   (STORM-1199)
  
-roshan


On 5/3/16, 3:39 PM, "Hugo Da Cruz Louro"  wrote:

>+1 (non binding)
>- Found a very minor bug in the storm-solr test topology, for which there
>is a very simple workaround and by NO means a blocker for the release.
>- unzipped .zip sources and binaries
>- mvn clean install in unzipped sources
>- created uber jar for storm-solr
>- tested storm-solr in local cluster mode and remode cluster mode
>- ran test examples
>
>On May 3, 2016, at 12:04 AM, Arun Mahadevan
>> wrote:
>
>+1 (binding)
>- Extracted binaries
>- Ran sample topologies
>- Browsed storm UI
>
>Thanks,
>Arun
>
>
>
>
>On 4/28/16, 12:01 PM, "Jungtaek Lim"
>> wrote:
>
>+1 (binding)
>
>- testing with source distribution : OK
>- unzip : OK
>- building from source dist : OK
>  - how to build: running `mvn -P all-tests clean install` on unzipped
>source dist.
>
>- testing with binary distribution (one machine) : OK
>- launch daemons : OK
>- run RollingTopWords (local) : OK
>- run RollingTopWords (remote) : OK
>  - activate / deactivate / rebalance / kill : OK
>
>Thanks,
>Jungtaek Lim
>
>2016년 4월 28일 (목) 오전 3:29, P. Taylor Goetz
>>님이 작성:
>
>This is a call to vote on releasing Apache Storm 0.10.1 (rc2)
>
>Full list of changes in this release:
>
>
>https://git-wip-us.apache.org/repos/asf?p=storm.git;a=blob_plain;f=CHANGEL
>OG.md;hb=8179921a569b6cf1d97798eed8e7b03b131bc495
>
>The tag/commit to be voted upon is v0.10.1:
>
>
>https://git-wip-us.apache.org/repos/asf?p=storm.git;a=tree;h=ddf051149f3c3
>86342937efdabdaf45694602dd1;hb=8179921a569b6cf1d97798eed8e7b03b131bc495;a=
>tree;h=26291835f22474506f0fe90b0459eab0d00bf4a9;hb=f0d3eae7395b3ee036b94b9
>22707f74868ba6190
>
>The source archive being voted upon can be found here:
>
>
>https://dist.apache.org/repos/dist/dev/storm/apache-storm-0.10.1-rc2/apach
>e-storm-0.10.1-src.tar.gz
>
>Other release files, signatures and digests can be found here:
>
>https://dist.apache.org/repos/dist/dev/storm/apache-storm-0.10.1-rc2/
>
>The release artifacts are signed with the following key:
>
>
>https://git-wip-us.apache.org/repos/asf?p=storm.git;a=blob_plain;f=KEYS;hb
>=22b832708295fa2c15c4f3c70ac0d2bc6fded4bd
>
>The Nexus staging repository for this release is:
>
>https://repository.apache.org/content/repositories/orgapachestorm-1031/
>
>Please vote on releasing this package as Apache Storm 0.10.1.
>
>When voting, please list the actions taken to verify the release.
>
>This vote will be open for at least 72 hours.
>
>[ ] +1 Release this package as Apache Storm 0.10.1
>[ ]  0 No opinion
>[ ] -1 Do not release this package because...
>
>Thanks to everyone who contributed to this release.
>
>-Taylor
>
>
>
>



Re: [jira] [Commented] (STORM-1757) Apache Beam Runner for Storm

2016-05-03 Thread Roshan Naik
Never mind my prev comment… got confused by looking at BEAM-216.

-roshan


On 5/3/16, 4:55 PM, "Roshan Naik" <ros...@hortonworks.com> wrote:

>Shouldn¹t we have a BEAM runner in Storm project instead of  Storm runner
>in Beam project ?
>-roshan
>
>On 5/3/16, 1:15 PM, "Hugo Louro (JIRA)" <j...@apache.org> wrote:
>
>>
>>[ 
>>https://issues.apache.org/jira/browse/STORM-1757?page=com.atlassian.jira.
>>p
>>lugin.system.issuetabpanels:comment-tabpanel=15269498#co
>>m
>>ment-15269498 ] 
>>
>>Hugo Louro commented on STORM-1757:
>>---
>>
>>I also would like to work on this project.
>>
>>> Apache Beam Runner for Storm
>>> 
>>>
>>> Key: STORM-1757
>>> URL: https://issues.apache.org/jira/browse/STORM-1757
>>> Project: Apache Storm
>>>  Issue Type: Brainstorming
>>>Reporter: P. Taylor Goetz
>>>Priority: Minor
>>>
>>> This is a call for interested parties to collaborate on an Apache Beam
>>>[1] runner for Storm, and express their thoughts and opinions.
>>> Given the addition of the Windowing API to Apache Storm, we should be
>>>able to map naturally to the Beam API. If not, it may be indicative of
>>>shortcomings of the Storm API that should be addressed.
>>> [1] http://beam.incubator.apache.org
>>
>>
>>
>>--
>>This message was sent by Atlassian JIRA
>>(v6.3.4#6332)
>>
>



Re: [jira] [Commented] (STORM-1757) Apache Beam Runner for Storm

2016-05-03 Thread Roshan Naik
Shouldn¹t we have a BEAM runner in Storm project instead of  Storm runner
in Beam project ?
-roshan

On 5/3/16, 1:15 PM, "Hugo Louro (JIRA)"  wrote:

>
>[ 
>https://issues.apache.org/jira/browse/STORM-1757?page=com.atlassian.jira.p
>lugin.system.issuetabpanels:comment-tabpanel=15269498#com
>ment-15269498 ] 
>
>Hugo Louro commented on STORM-1757:
>---
>
>I also would like to work on this project.
>
>> Apache Beam Runner for Storm
>> 
>>
>> Key: STORM-1757
>> URL: https://issues.apache.org/jira/browse/STORM-1757
>> Project: Apache Storm
>>  Issue Type: Brainstorming
>>Reporter: P. Taylor Goetz
>>Priority: Minor
>>
>> This is a call for interested parties to collaborate on an Apache Beam
>>[1] runner for Storm, and express their thoughts and opinions.
>> Given the addition of the Windowing API to Apache Storm, we should be
>>able to map naturally to the Beam API. If not, it may be indicative of
>>shortcomings of the Storm API that should be addressed.
>> [1] http://beam.incubator.apache.org
>
>
>
>--
>This message was sent by Atlassian JIRA
>(v6.3.4#6332)
>



Re: Thought on complete latency

2016-04-28 Thread Roshan Naik
IMO, avoiding the time variation on machines makes total sense. But I feel
that this is a tricky question.


Couple more thoughts:

1)  As per 
http://storm.apache.org/releases/current/Guaranteeing-message-processing.ht
ml

"Storm can detect when the tree of tuples is fully processed and can ack
or fail the spout tuple appropriately."
 

That seems to indicate that when the ACKer has received all the necessary
acks, then it considers the tuple fully processed. If we go by that, and
we define complete latency as the time taken to fully process a tuple,
then it is not necessary to include the time it takes for the ACK to be
delivered to spout.


2) If you include the time it takes to deliver the ACK to the spout, then
we also need to wonder if we should include the time that the spout takes
to process the ACK() call. I am unclear if the spout.ack() throws an
exception what that means to the idea of Œfully processed¹. Here you can
compute delta either immediately before OR immediately after the ACK() is
invoked on the spout


The benefit of including spout¹s ACK() processing time, is that any
optimizations/inefficiencies in the spout's ACK() implementation will be
detectable. 

I wonder if we should split this into two different metricsŠ

- ³delivery latency²  (which ends once the ACKer receives the last ACK
from a bolt) and 
- "complete latency² which includes ACK processing time by spout


 -roshan



On 4/28/16, 8:59 AM, "Jungtaek Lim"  wrote:

>Hi devs,
>
>While thinking about metrics improvements, I doubt how many users know
>that
>what 'exactly' is complete latency. In fact, it's somewhat complicated
>because additional waiting time could be added to complete latency because
>of single-thread model event loop of spout.
>
>Long running nextTuple() / ack() / fail() can affect complete latency but
>it's behind the scene. No latency information provided, and someone even
>didn't know about this characteristic. Moreover, calling nextTuple() could
>be skipped due to max spout waiting, which will make us harder to guess
>when avg. latency of nextTuple() will be provided.
>
>I think separation of threads (tuple handler to separate thread, as JStorm
>provides) would resolve the gap, but it requires our spout logic to be
>thread-safe, so I'd like to find workaround first.
>
>My sketched idea is let Acker decides end time for root tuple.
>There're two subsequent ways to decide start time for root tuple,
>
>1. when Spout about to emit ACK_INIT to Acker (in other words, keep it as
>it is)
>  - Acker sends ack / fail message to Spout with timestamp, and Spout
>calculates time delta
>  - pros. : It's most accurate way since it respects the definition of
>'complete latency'.
>  - cons. : The sync of machine time between machines are very important.
>Milliseconds of precision would be required.
>2. when Acker receives ACK_INIT from Spout
>  - Acker calculates time delta itself, and sends ack / fail message to
>Spout with time delta
>  - pros. : No requirement to sync the time between servers so strictly.
>  - cons. : It doesn't contain the latency to send / receive ACK_INIT
>between Spout and Acker.
>
>Sure we could leave it as is if we decide it doesn't hurt much.
>
>What do you think?
>
>Thanks,
>Jungtaek Lim (HeartSaVioR)



[jira] [Updated] (STORM-1580) Secure hdfs spout failed

2016-04-05 Thread Roshan Naik (JIRA)

 [ 
https://issues.apache.org/jira/browse/STORM-1580?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Roshan Naik updated STORM-1580:
---
Attachment: HdfsSpoutTopology.java

Sorry for the delayed update... kept getting pulled into other urgent things 
and took sometime to setup a kerberized cluster.

*Update:*  I modified  HdfsSpoutTopology.java  (from examples/storm-starter )  
for  kerberos and tried it on a secure cluster. It worked fine.

I am attaching the modified java file.

Your error might indicate some issue likely on the kerberos setup side. 
try these:
- kinit with the same keytab and principal on that host and verify its ok by 
running some hadoop fs -ls  commands 
- Ensure hdfs-site.xml and core-site.xml from the kerberized cluster are 
packaged as resources in your topology. A quick way to do this is to copy them 
into storm/lib and restart supervisor.


> Secure hdfs spout failed
> 
>
> Key: STORM-1580
> URL: https://issues.apache.org/jira/browse/STORM-1580
> Project: Apache Storm
>  Issue Type: Bug
>  Components: storm-hdfs
>Reporter: guoht
>  Labels: security
> Attachments: HdfsSpoutTopology.java
>
>
> Some error occured when using secure hdfs spout:
> "Login successful for user t...@example.com using keytab file 
> /home/test/test.keytab
> 2016-02-26 10:33:14 o.a.h.i.Client [WARN] Exception encountered while 
> connecting to the server : javax.security.sasl.SaslException: GSS initiate 
> failed [Caused by GSSException: No valid credentials provided (Mechanism 
> level: Failed to find any Kerberos tgt)]
> 2016-02-26 10:33:14 o.a.h.i.Client [WARN] Exception encountered while 
> connecting to the server : javax.security.sasl.SaslException: GSS initiate 
> failed [Caused by GSSException: No valid credentials provided (Mechanism 
> level: Failed to find any Kerberos tgt)]
> 2016-02-26 10:33:14 o.a.h.i.r.RetryInvocationHandler [INFO] Exception while 
> invoking getFileInfo of class ClientNamenodeProtocolTranslatorPB over 
> hnn025/192.168.137.2:8020 after 1 fail over attempts. Trying to fail over 
> immediately.
> java.io.IOException: Failed on local exception: java.io.IOException: 
> javax.security.sasl.SaslException: GSS initiate failed [Caused by 
> GSSException: No valid credentials provided (Mechanism level: Failed to find 
> any Kerberos tgt)]; Host Details : local host is: "HDD021/192.168.137.6"; 
> destination host is: "hnn025":8020;"



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (STORM-1632) Disable event logging by default

2016-03-24 Thread Roshan Naik (JIRA)

 [ 
https://issues.apache.org/jira/browse/STORM-1632?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Roshan Naik updated STORM-1632:
---
Attachment: BasicTopology.java

uploading  topology code to validate perf hit. 
To use it: .. 
1) Copy the java file into examples/storm-starter  
2) Rebuild  the  storm-starter package using mvn. 
3) Run topology as follows:

storm  jar 
/Users/rnaik/Projects/idea/storm/examples/storm-starter/target/storm-starter-1.0.0-SNAPSHOT.jar
 -c topology.eventlogger.executors=0 -c topology.max.spout.pending=2000  -c 
topology.disruptor.batch.size=1storm.starter.BasicTopology

and then again  with  {{topology.eventlogger.executors=1}}

I have set those additional 2 flags as they improved performance over the 
defaults for this topology.

I normally let it run for about 11 min  and then capture the 10 min window 
metrics from UI page.

> Disable event logging by default
> 
>
> Key: STORM-1632
> URL: https://issues.apache.org/jira/browse/STORM-1632
> Project: Apache Storm
>  Issue Type: Bug
>  Components: storm-core
>    Reporter: Roshan Naik
>    Assignee: Roshan Naik
>Priority: Blocker
> Fix For: 1.0.0
>
> Attachments: BasicTopology.java
>
>
> EventLogging has performance penalty. For a simple speed of light topology  
> with a single instances of a spout and a bolt, disabling event logging 
> delivers a 7% to 9% perf improvement (with acker count =1)
> Event logging can be enabled when there is need to do debug, but turned off 
> by default.
> **Update:** with acker=0  the observed impact was much higher... **25%** 
> faster when event loggers = 0 



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (STORM-1580) Secure hdfs spout failed

2016-03-23 Thread Roshan Naik (JIRA)

[ 
https://issues.apache.org/jira/browse/STORM-1580?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15209389#comment-15209389
 ] 

Roshan Naik commented on STORM-1580:


[~ght]  fyi.. I am beginning to take a look at this.

> Secure hdfs spout failed
> 
>
> Key: STORM-1580
> URL: https://issues.apache.org/jira/browse/STORM-1580
> Project: Apache Storm
>  Issue Type: Bug
>  Components: storm-hdfs
>Reporter: guoht
>  Labels: security
>
> Some error occured when using secure hdfs spout:
> "Login successful for user t...@example.com using keytab file 
> /home/test/test.keytab
> 2016-02-26 10:33:14 o.a.h.i.Client [WARN] Exception encountered while 
> connecting to the server : javax.security.sasl.SaslException: GSS initiate 
> failed [Caused by GSSException: No valid credentials provided (Mechanism 
> level: Failed to find any Kerberos tgt)]
> 2016-02-26 10:33:14 o.a.h.i.Client [WARN] Exception encountered while 
> connecting to the server : javax.security.sasl.SaslException: GSS initiate 
> failed [Caused by GSSException: No valid credentials provided (Mechanism 
> level: Failed to find any Kerberos tgt)]
> 2016-02-26 10:33:14 o.a.h.i.r.RetryInvocationHandler [INFO] Exception while 
> invoking getFileInfo of class ClientNamenodeProtocolTranslatorPB over 
> hnn025/192.168.137.2:8020 after 1 fail over attempts. Trying to fail over 
> immediately.
> java.io.IOException: Failed on local exception: java.io.IOException: 
> javax.security.sasl.SaslException: GSS initiate failed [Caused by 
> GSSException: No valid credentials provided (Mechanism level: Failed to find 
> any Kerberos tgt)]; Host Details : local host is: "HDD021/192.168.137.6"; 
> destination host is: "hnn025":8020;"



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


  1   2   >