Re: ThreadJobFactory in production

2016-03-02 Thread Rick Mangi
There was an interesting thread a while back from I believe the netflix guys about running ThreadJobFactory in production. > On Mar 2, 2016, at 4:20 PM, Robert Crim wrote: > > Hi, > > We're currently working on a solution that allows us to run Samza jobs on > Mesos. This seems to be going wel

Re: Allow user-specified class-loader

2016-02-11 Thread Rick Mangi
This is why we went the route of using clojure from the start and not calling out to clojure from a java task. > On Feb 10, 2016, at 9:11 PM, Andy Chambers > wrote: > > Have you figured out a workflow where you can run jobs from rhe repl during > development? signature.asc Description: Mes

Re: Zombie writers protection

2016-02-10 Thread Rick Mangi
#x27;s incrementing token as the >> password. A zombie process would auth with an old token and be denied. I >> haven't looked but i imagine that 0.9.0 auth framework isn't done on a >> partition level. >> >> On Wed, Feb 10, 2016 at 2:27 PM, Rick Mangi wrot

Re: Zombie writers protection

2016-02-10 Thread Rick Mangi
Security wouldn’t stop zombie processes from writing to kafka. I had this problem with yarn before where the container thought it was killing jobs but they never actually died, and in fact continued to write to kafka. > On Feb 10, 2016, at 4:23 PM, Jagadish Venkatraman > wrote: > > Hi John >

Re: Samza Throttling clarification

2016-02-04 Thread Rick Mangi
Can I ask why you’re using samza and kafka for this? > On Feb 4, 2016, at 7:41 PM, ramesh shanmugam > wrote: > > Hi Team, > > We have use case that destination system accept only 160 messages per > minute. > > Planning to use Samza with Kafka. Looking for out of box throttling feature > but

Re: CoordinatorStream errors

2015-12-21 Thread Rick Mangi
s may still be coming up. > I am not entirely sure if this is what is happening. > > It will be great if you can share the log. > > Thanks! > navina > > On Fri, Dec 18, 2015 at 8:53 AM, Rick Mangi wrote: > >> Hi all, >> >> I just started seeing

CoordinatorStream errors

2015-12-18 Thread Rick Mangi
Hi all, I just started seeing these errors the other day. I am heavily refactoring my code, but it works locally. I’m wondering if anyone has seen this error when deploying to yarn. This is in stderr log on my application master. Exception in thread "AMRM Callback Handler Thread" org.apache.h

Re: lineage in Samza and tracking processed events

2015-12-02 Thread Rick Mangi
f which I have to know > in advance - so I will stick to some known topology and if it changes, > "finishing" condition also changes > > On Wednesday, December 2, 2015, Rick Mangi wrote: > >> The simplest way would be to have your job send a message to another kafka

Re: lineage in Samza and tracking processed events

2015-12-01 Thread Rick Mangi
The simplest way would be to have your job send a message to another kafka topic when it’s done. Rick > On Dec 1, 2015, at 3:44 PM, Anton Polyakov wrote: > > Hi > > I am looking at Samza to process some incoming stream of trades. Processing > pipeline is a complex DAG where some nodes migh

Re: Error restoring kv store

2015-11-20 Thread Rick Mangi
So I left it running in this state for 15 minutes and then it started up. Usually the app takes about 10 seconds to initialize. > On Nov 20, 2015, at 10:23 AM, Rick Mangi wrote: > > Just started getting this error this morning. On startup the consumer trying > to reload

Error restoring kv store

2015-11-20 Thread Rick Mangi
Just started getting this error this morning. On startup the consumer trying to reload the kv-store changelog starts throwing this exception and keeps restarting and failing. 2015-11-20 10:16:19 [main] TaskStorageManager [DEBUG] Stopping consumers for stores. 2015-11-20 10:16:19 [main] Broker

Re: question on yarn.container.cpu.cores

2015-11-19 Thread Rick Mangi
I’m not sure you can do this for the purposes of consuming messages faster, but I’m interested to hear if I’m mistaken. You could spawn threads to do other work within a job though. > On Nov 18, 2015, at 11:40 PM, Chen Song wrote: > > Thanks Navina > > So theoretically I can create a threa

Re: Sporadic errors in JobRunner

2015-11-18 Thread Rick Mangi
That patch seems to have fixed the problem. > On Nov 18, 2015, at 3:43 PM, Rick Mangi wrote: > > Sorry. Just read the bug. Yes, that makes sense. I deleted a bunch of topics > and then hit this. > > >> On Nov 18, 2015, at 3:42 PM, Rick Mangi wrote: >> &g

Re: Sporadic errors in JobRunner

2015-11-18 Thread Rick Mangi
Sorry. Just read the bug. Yes, that makes sense. I deleted a bunch of topics and then hit this. > On Nov 18, 2015, at 3:42 PM, Rick Mangi wrote: > > I take that back, it happened again. Will try your patch. > > >> On Nov 18, 2015, at 3:36 PM, Rick Mangi wrote: >>

Re: Sporadic errors in JobRunner

2015-11-18 Thread Rick Mangi
I take that back, it happened again. Will try your patch. > On Nov 18, 2015, at 3:36 PM, Rick Mangi wrote: > > I seem to have solved it by only specifying a single zookeeper node in my job > config. Maybe a race condition of some sort? > > >> On Nov 18, 2015, a

Re: Sporadic errors in JobRunner

2015-11-18 Thread Rick Mangi
I will upload the patch and it would be good if you can try the > patch to see whether that solves your problem. > > -Yi > > On Tue, Nov 17, 2015 at 12:01 PM, Rick Mangi wrote: > >> Hi, getting things working on samza 0.10.0 finally :) >> >> I’m seeing the foll

Sporadic errors in JobRunner

2015-11-17 Thread Rick Mangi
Hi, getting things working on samza 0.10.0 finally :) I’m seeing the following error about 1/4 of the time from run-job.sh when starting jobs: [yarnmaster01] out: 2015-11-17 14:56:00 KafkaSystemAdmin$ [INFO] Got metadata: Map(__samza_coordinator_t-key-grouper_dev -> SystemStreamMetadata [strea

Re: Samza hdfs

2015-11-13 Thread Rick Mangi
We also evaluated Spark Streaming as well as Storm before settling on Samza for our work. Our reason was the confidence and modularity that comes with having Kafka between each job in the topology. Being able to treat each job as a separate deployment makes ongoing development much easier and w

Re: Problems upgrading Job

2015-11-13 Thread Rick Mangi
Thanks a lot! > > -Yi > > On Thu, Nov 12, 2015 at 10:10 AM, Rick Mangi wrote: > >> Hi Yi, >> >> I pulled from master and built this morning. >> >> Yes, that’s the output from JobRunner. I also tried setting a job.id to >> see if this was

Re: Problems upgrading Job

2015-11-12 Thread Rick Mangi
12, 2015, at 12:59 PM, Yi Pan wrote: > > Hi, Rick, > > Did you get the fix in SAMZA-723 in your test? And could you confirm that > the errors are from JobRunner log? > > -Yi > > On Thu, Nov 12, 2015 at 8:48 AM, Rick Mangi wrote: > >> Hi, >> >

Problems upgrading Job

2015-11-12 Thread Rick Mangi
Hi, I’m trying to migrate our samza jobs to 0.10.0 snapshot (built against the latest). Everything works fine running locally (although I had to make some changes to the local grid’s kafka since the checkpointing seems to require replication_factor > 1) but when I deploy it against my productio

Re: 0.10.0 Release?

2015-11-05 Thread Rick Mangi
avina > > On Thu, Nov 5, 2015 at 1:40 PM, Rick Mangi wrote: > >> I openend https://issues.apache.org/jira/browse/SAMZA-809 >> >> Thanks Navina! >> >> >> On Nov 5, 2015, at 3:50 PM, Rick Mangi wrote: >> >> I can generate the docs but

Re: 0.10.0 Release?

2015-11-05 Thread Rick Mangi
I openend https://issues.apache.org/jira/browse/SAMZA-809 <https://issues.apache.org/jira/browse/SAMZA-809> Thanks Navina! > On Nov 5, 2015, at 3:50 PM, Rick Mangi wrote: > > I can generate the docs but the links in the left hand nav to > documentation/configurati

Re: 0.10.0 Release?

2015-11-05 Thread Rick Mangi
t; > Thanks! > Navina > > On Thu, Nov 5, 2015 at 12:41 PM, Rick Mangi wrote: > >> Beautiful. >> >> Yeah, I’m trying to build the docs right now to view the new config >> options and it’s not working :) >> >> Looking forward to the release. >

Re: 0.10.0 Release?

2015-11-05 Thread Rick Mangi
> > We are still targeting a new release at the end of this month. Please let > us know if you have any other questions. > > Thanks! > Navina > > On Thu, Nov 5, 2015 at 12:30 PM, Rick Mangi wrote: > >> Hi, >> >> I’m wondering when the 0.10.0 r

0.10.0 Release?

2015-11-05 Thread Rick Mangi
Hi, I’m wondering when the 0.10.0 release is planned for? So much has changed :) Thanks, Rick signature.asc Description: Message signed with OpenPGP using GPGMail

API for containers?

2015-11-04 Thread Rick Mangi
Hello, I’m wondering if there’s currently any API (or other way) to find out which container would handle a theoretical key and what node that container is running on. The use-case I’m thinking of is if I wanted to run a rest service on each node which could handle queries against the current

Re: Does Samza work with ResourceManager in HA?

2015-11-03 Thread Rick Mangi
Nope, we left it out. Rick > On Nov 3, 2015, at 11:04 AM, John Tipper wrote: > > Thanks Rick, much appreciated. Did you have to set > yarn.resourcemanager.hostname or can you leave this out? > > Thanks, > > John > > ________

Re: Does Samza work with ResourceManager in HA?

2015-11-03 Thread Rick Mangi
Hi John, We just got this set up last week. I haven’t fully tested the failover but it certainly works for testing out our samza jobs on a pretty large cluster. All of the server names are fqdn in our live yarn-site.xml Hope this helps, Rick yarn.resourcemanager.ha.enabled true yarn

Re: Best way to log from inside a Samza task?

2015-07-01 Thread Rick Mangi
ilarly but it seems to be doing some > formatting of the messages. > > I am trying to consume messages from one Kafka topic and produce to another. > While I can use zookeeper to see the messages in the originating topic they > never make it to the destination and I am trying to find

Re: Best way to log from inside a Samza task?

2015-06-26 Thread Rick Mangi
ote: > > I was almost there. Got it now. Thanks for your help Rick. > > > > > Cheers, > > > > > Jason > > > > > > > > > > On Friday, Jun 26, 2558 at 11:43, Rick Mangi , wrote: > Hey Jason, > > > If you

Re: Best way to log from inside a Samza task?

2015-06-26 Thread Rick Mangi
Hey Jason, If you configure log4j as described here: http://samza.apache.org/learn/documentation/0.9/jobs/logging.html Your log statements will wind up in the samza-container logs which you can get to via the application maste