Re: [Discuss] Moving Samza to Java 1.8 source compatibility.

2016-04-28 Thread Roger Hoover
+1 for me. We're already using Java 8 in PRD. On Thu, Apr 28, 2016 at 10:45 AM, Yi Pan wrote: > I am +1 on the JDK8 move. As Jake has elaborated, there are numerous > advantages from 1.8 source compatible code. > > As for the downside of dropping JDK7 support, obviously, bin > backward-compatib

Re: ThreadJobFactory in production

2016-03-02 Thread Roger Hoover
Jose, It would be great if you could share it. I'm interested in trying to use it as well. Thanks, Roger On Wed, Mar 2, 2016 at 2:31 PM, José Barrueta wrote: > Hi guys, > > At Stormpath, we made a custom samza 10 version merging SAMZA-41 into it, > it's working well, so we are thinking to up

Re: [DISCUSS] Moving to github/pull-request for code review and check-in

2016-02-18 Thread Roger Hoover
+1 - Thanks for bringing this up, Yi. I've done it both ways and feel pull requests are much easier. Sent from my iPhone > On Feb 18, 2016, at 4:25 PM, Navina Ramesh > wrote: > > +1 > > Haven't tried any contribution with pull requests. But sounds simpler than > attaching the patch to JIRA.

Re: ElasticsearchSystemProducer Crashes Samza Job

2016-02-16 Thread Roger Hoover
dex document in Elasticsearch: " + > itemResp.getFailureMessage()); >} else { > hasFatalError = true; > LOGGER.error("Failed to index document in Elasticsearch: " + > itemResp.getFailureMessage()); >} > > - jeremiah > > On Tue, Feb 16, 2016

Re: ElasticsearchSystemProducer Crashes Samza Job

2016-02-16 Thread Roger Hoover
Hi Jeremiah, There's currently no way to do that. I think the best way to modify the existing ElasticsearchSystemProducer would be to add a config option for a callback to let you customize this behavior. Basically, a pluggable listener ( https://github.com/apache/samza/blob/master/samza-elastic

Re: HTTP-based Elasticsearch system producer and reusable task

2016-02-10 Thread Roger Hoover
er/samza-elasticsearch/src/main/java/com/quantiply/elasticsearch/HTTPBulkLoader.java#L237-L272 > Thanks! > > -Yi > > On Tue, Feb 9, 2016 at 4:19 PM, Roger Hoover > wrote: > > > Hi Yi, > > > > It could be merged into the Samza project if there's enough interest but &

Re: HTTP-based Elasticsearch system producer and reusable task

2016-02-09 Thread Roger Hoover
options do not change, I would vote to replace the implementation w/ > HTTP-based ElasticsearchSystemProducer. > > Thanks for putting this new additions up! > > -Yi > > On Tue, Feb 9, 2016 at 10:39 AM, Roger Hoover > wrote: > > > Hi Samza folks, > > >

HTTP-based Elasticsearch system producer and reusable task

2016-02-09 Thread Roger Hoover
Hi Samza folks, For people who want to use HTTP to integrate with Elasticsearch, I wrote an HTTP-based system producer and a reusable task, including latency stats from event origin time, task processing time, and time spent talking to Elasticsearch API. https://github.com/quantiply/rico/blob/mas

Re: Samza 0.10 released

2016-01-04 Thread Roger Hoover
Excellent. Thanks for all the work by everyone to get this done. Congrats! On Mon, Jan 4, 2016 at 11:04 AM, Yi Pan wrote: > Hi, all, > > In case you missed the announcement of Samza 0.10 release before the > Christmas, please check it out here: https://blogs.apache.org/samza/ and > help to spr

Re: Executing Samza jobs natively in Kubernetes

2015-11-30 Thread Roger Hoover
Awesome. Thanks. On Sun, Nov 29, 2015 at 3:25 PM, Elias Levy wrote: > Roger, > > You are welcomed. If you want to experiment, you can use my hello samza > <https://hub.docker.com/r/elevy/hello-samza/> Docker image. > > On Sun, Nov 29, 2015 at 12:19 PM, Roger Hoov

Re: Executing Samza jobs natively in Kubernetes

2015-11-29 Thread Roger Hoover
Elias, I would also love to be able to deploy Samza on Kubernetes with dynamic task management. Thanks for sharing this. It may be a good interim solution. Roger On Sun, Nov 29, 2015 at 11:18 AM, Elias Levy wrote: > I've been exploring Samza for stream processing as well as Kubernetes as a >

Re: Sample code or tutorial for writing/reading Avro type message in Samza

2015-11-17 Thread Roger Hoover
Hi Selina, If you want to use Confluent's schema registry for Avro, then I have an example in this repo: https://github.com/theduderog/hello-samza-confluent Cheers, Roger On Tue, Nov 17, 2015 at 12:32 AM, Selina Tech wrote: > Dear All: > Do you know where I can find the tutorial or samp

Re: Checkpoint tool not working

2015-10-30 Thread Roger Hoover
I tried it once with 0.9.1 and it didn't work for me either. I didn't have time to examine it more carefully at the time. Roger On Thu, Oct 29, 2015 at 10:05 PM, Lukas Steiblys wrote: > I'm using Samza 0.9.1. > > Lukas > > On 10/29/15, Yi Pan wrote: > > Hi, Lukas, > > > > Which version of che

Snapshot metrics stop getting scheduled in an exception occurs

2015-10-26 Thread Roger Hoover
Hi Samza devs, I ran into an issue with Samza 0.9.1 where I had a serialization exception thrown in the MetricsSnapshotReporter. It's very hard to find because nothing is logged and the metrics just stop getting scheduled. Samza should catch all exceptions in that thread, log them, and suppress

Re: Samza and KStreams (KIP-28): LinkedIn's POV

2015-10-05 Thread Roger Hoover
Great. Thanks, Yi. On Mon, Oct 5, 2015 at 10:25 AM, Yi Pan wrote: > Hi, Roger, > > > On Sat, Oct 3, 2015 at 11:13 AM, Roger Hoover > wrote: > > > As previously discussed, the biggest request I > > have is being able to run Samza without YARN, under somethi

Re: Samza and KStreams (KIP-28): LinkedIn's POV

2015-10-03 Thread Roger Hoover
Hi Yi, Thank you for sharing this update and perspective. I tend to agree that for simple, stateless cases, things could be easier and hopefully KStreams may help with that. I also appreciate a lot of features that Samza already supports for operations. As previously discussed, the biggest requ

Re: New Samza blog published - http://engineering.linkedin.com/performance/benchmarking-apache-samza-12-million-messages-second-single-node

2015-08-25 Thread Roger Hoover
Thanks for sharing! Tao, did you use YARN to run 15 containers or is there a way to have them statically divide up the tasks? On Mon, Aug 24, 2015 at 12:03 PM, Ed Yakabosky < eyakabo...@linkedin.com.invalid> wrote: > Hi Samza open source, > > I want to share that Tao Feng >

Re: [Discuss/Vote] upgrade to Yarn 2.6.0

2015-08-24 Thread Roger Hoover
>>> In LinkedIn we have already moved to YARN 2.6 and is moving to YARN 2.7 > >>> now. I am not aware of any major issues in upgrading. I will let our > team > >>> member Jon Bringhurst to chime in since he did all the upgrade and may > >> have

Re: [Discuss/Vote] upgrade to Yarn 2.6.0

2015-08-24 Thread Roger Hoover
i, Roger, >>>> >>>> In LinkedIn we have already moved to YARN 2.6 and is moving to YARN 2.7 >>>> now. I am not aware of any major issues in upgrading. I will let our >> team >>>> member Jon Bringhurst to chime in since he did all the upgrade

Re: [Discuss/Vote] upgrade to Yarn 2.6.0

2015-08-19 Thread Roger Hoover
the upgrade and may have > more insights. > > @Jon, could you help to comment on this? > > Thanks! > > -Yi > > On Wed, Aug 19, 2015 at 9:12 AM, Roger Hoover > wrote: > > > We're using 2.4.0 in production. Are there any major incompatibilities >

Re: [Discuss/Vote] upgrade to Yarn 2.6.0

2015-08-19 Thread Roger Hoover
We're using 2.4.0 in production. Are there any major incompatibilities to watch out for when upgrading to 2.6.0? Thanks, Roger On Mon, Aug 17, 2015 at 4:41 PM, Yan Fang wrote: > Hi guys, > > we have been discussing upgrading to Yarn 2.6.0 (SAMZA-536 >

Re: Use one producer for both coordinator stream and users system?

2015-08-18 Thread Roger Hoover
Hi Yan, My (uneducated) guess is that the performance gains come from batching. I don't know if the new producer ever batches by destination broker. If not and it only batches by (broker,topic,partition) then I doubt that one vs two producers will affect performance as they send to different top

Re: Review Request 36815: SAMZA-741 Support for versioning with Elasticsearch Producer

2015-07-29 Thread Roger Hoover
Thanks, Yi! On Wed, Jul 29, 2015 at 12:16 PM, Yi Pan wrote: > Hi, Roger, > > I am testing the patch now. Will update the JIRA soon. > > Thanks! > > -Yi > > On Wed, Jul 29, 2015 at 12:11 PM, Roger Hoover > wrote: > > > Thank you, Dan. I think we&#

Re: Review Request 36815: SAMZA-741 Support for versioning with Elasticsearch Producer

2015-07-29 Thread Roger Hoover
Logger.info("Failed to index message in ElasticSearch.", e); > > This would also be true for other log lines added. > > On July 29th, 2015, 6:22 p.m. UTC, *Roger Hoover* wrote: > > Good idea. Thanks. > > BTW, it didn't work like this: Logger.info("Faile

Re: Review Request 36815: SAMZA-741 Support for versioning with Elasticsearch Producer

2015-07-29 Thread Roger Hoover
> Should we add a Samza specifc message, then add the whole exception? so > > it's more clear what the exception was from if the user doesn't know the > > code? `Logger.info("Failed to index message in ElasticSearch.", e);` > > > > Thi

Re: Review Request 36815: SAMZA-741 Support for versioning with Elasticsearch Producer

2015-07-29 Thread Roger Hoover
DefaultIndexRequestFactory to make it easier to subclass and customize to handle version and version_type parameters. Thanks, Roger Hoover

Re: Review Request 36815: SAMZA-741 Support for versioning with Elasticsearch Producer

2015-07-29 Thread Roger Hoover
esp.getFailureMessage()); - Roger --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/36815/#review93413 --- On July 29, 2015, 6:22 a.m., Roger Hoover wr

Re: changelog compaction problem

2015-07-29 Thread Roger Hoover
You also may want to check if the cleaner thread in the broker is still alive (using jstack). I've run into this issue and used the fix mentioned in the ticket to get compaction working again. https://issues.apache.org/jira/browse/KAFKA-1641 I'd just like to mention that a possible workaround (de

Re: Review Request 36815: SAMZA-741 Support for versioning with Elasticsearch Producer

2015-07-28 Thread Roger Hoover
--- Refactored DefaultIndexRequestFactory to make it easier to subclass and customize to handle version and version_type parameters. Thanks, Roger Hoover

Re: Review Request 36815: SAMZA-741 Support for versioning with Elasticsearch Producer

2015-07-28 Thread Roger Hoover
ne149> > > > > Quick question: Is it guaranteed that there is no DeleteResponse here? > > It would be good to at least log a warn if we get an unexpected response > > here. > > Roger Hoover wrote: > It is guaranteed that you will not get a DeleteResponse back

Re: Review Request 36815: SAMZA-741 Support for versioning with Elasticsearch Producer

2015-07-28 Thread Roger Hoover
--- Refactored DefaultIndexRequestFactory to make it easier to subclass and customize to handle version and version_type parameters. Thanks, Roger Hoover

Re: Review Request 36815: SAMZA-741 Support for versioning with Elasticsearch Producer

2015-07-28 Thread Roger Hoover
it: https://reviews.apache.org/r/36815/#review93395 ------- On July 29, 2015, 5:17 a.m., Roger Hoover wrote: > > --- > This is an automatically generated e-mail. To reply, visit: > https://reviews.apache.org/r/36815/ > ---

Re: [DISCUSS] Release 0.10.0

2015-07-28 Thread Roger Hoover
Thanks, Yi. I propose that we also include SAMZA-741 for Elasticsearch versioning support with the new ES producer. I think it's very close to being merged. Roger On Tue, Jul 28, 2015 at 10:08 PM, Yi Pan wrote: > Hi, all, > > I want to start the discussion on the release schedule for 0.10.0.

Re: Review Request 36815: SAMZA-741 Support for versioning with Elasticsearch Producer

2015-07-28 Thread Roger Hoover
tored DefaultIndexRequestFactory to make it easier to subclass and customize to handle version and version_type parameters. Thanks, Roger Hoover

Re: Review Request 36815: SAMZA-741 Support for versioning with Elasticsearch Producer

2015-07-28 Thread Roger Hoover
This already gets logged and thrown in flush(), is this to see the > > exception sooner? These are the individual exceptions per document. They're not being logged in the flush() method. Only batch-level errors are saved and logged in the flush. I wasn't seeing any of the docu

Re: Review Request 36815: SAMZA-741 Support for versioning with Elasticsearch Producer

2015-07-27 Thread Roger Hoover
make it easier to subclass and customize to handle version and version_type parameters. Thanks, Roger Hoover

Re: Review Request 36815: SAMZA-741 Support for versioning with Elasticsearch Producer

2015-07-25 Thread Roger Hoover
/ Testing --- Refactored DefaultIndexRequestFactory to make it easier to subclass and customize to handle version and version_type parameters. Thanks, Roger Hoover

Review Request 36815: SAMZA-741 Support for versioning with Elasticsearch Producer

2015-07-25 Thread Roger Hoover
DefaultIndexRequestFactory to make it easier to subclass and customize to handle version and version_type parameters. Thanks, Roger Hoover

Re: Kafka broker error from samza producer

2015-07-23 Thread Roger Hoover
r to lz4. Thanks for the reply. > -Jordan > > On Thu, Jul 23, 2015 at 9:32 AM, Roger Hoover > wrote: > >> Hi Jordan, >> >> I ran into a similiar issue when using snappy compression and the new >> producer. If you disable compression or switch to lz4 or g

How to map document version to the Elasticsearch System Producer?

2015-07-23 Thread Roger Hoover
Hi Dan and Samza devs, I have a use case for which I need to set an external version on Elasticsearch documents. Versioning ( https://www.elastic.co/guide/en/elasticsearch/reference/current/docs-index_.html#index-versioning) lets you prevent duplicate messages from temporarily overwriting new ver

Re: Kafka broker error from samza producer

2015-07-23 Thread Roger Hoover
Hi Jordan, I ran into a similiar issue when using snappy compression and the new producer. If you disable compression or switch to lz4 or gzip, does the issue go away? Cheers, Roger On Wed, Jul 22, 2015 at 11:54 PM, Jordan Shaw wrote: > Hey Everyone, > I'm getting an: > "kafka.message.Inva

Re: Review Request 36473: SAMZA-733 Add metrics to Elasticsearch System Producer

2015-07-22 Thread Roger Hoover
were created and indexed. Thanks, Roger Hoover

Re: Review Request 36473: SAMZA-733 Add metrics to Elasticsearch System Producer

2015-07-21 Thread Roger Hoover
were created and indexed. Thanks, Roger Hoover

Re: Review Request 36473: SAMZA-733 Add metrics to Elasticsearch System Producer

2015-07-21 Thread Roger Hoover
to use the "def" here, not only because it has > > leff overhead, but also keep all the methods consistent for better > > readability. What do you think? > > Roger Hoover wrote: > Sounds good. I only baulked on it the first time because I'm not that >

Re: Review Request 36473: SAMZA-733 Add metrics to Elasticsearch System Producer

2015-07-21 Thread Roger Hoover
you think? Sounds good. I only baulked on it the first time because I'm not that skilled with Scala type decarations yet. :) I can make this work - Roger --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/36473/#review92429

Re: Review Request 36473: SAMZA-733 Add metrics to Elasticsearch System Producer

2015-07-20 Thread Roger Hoover
and that the metrics correctly count how many Elasticsearch documents were created and indexed. Thanks, Roger Hoover

Re: Review Request 36473: SAMZA-733 Add metrics to Elasticsearch System Producer

2015-07-14 Thread Roger Hoover
Elasticsearch producer appear in JMX and the metrics stream and that the metrics correctly count how many Elasticsearch documents were created and indexed. Thanks, Roger Hoover

Re: Review Request 36473: SAMZA-733 Add metrics to Elasticsearch System Producer

2015-07-14 Thread Roger Hoover
t; > > can this class extends MetricsHelper? This can simplifies a little. > > Roger Hoover wrote: > I don't see how it simplifies things because I have to implement all the > methods in the Scala trait. I'm having trouble getting the newGaug

Re: Review Request 36473: SAMZA-733 Add metrics to Elasticsearch System Producer

2015-07-14 Thread Roger Hoover
; > remove the space Sure thing. - Roger --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/36473/#review91670 --- On July

Re: Review Request 36473: SAMZA-733 Add metrics to Elasticsearch System Producer

2015-07-14 Thread Roger Hoover
-- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/36473/#review91670 ------- On July 14, 2015, 6:12 a.m., Roger Hoover wrote: > > --

Metrics for Elasticsearch System Producer

2015-07-13 Thread Roger Hoover
Hi all, I've started using the new Elasticsearch System Producer (many thanks, Dan!) and decided to add some metrics to it. The JIRA ticket and review request links are here: https://issues.apache.org/jira/browse/SAMZA-733 https://reviews.apache.org/r/36473/ Cheers, Roger

Review Request 36473: SAMZA-733 Add metrics to Elasticsearch System Producer

2015-07-13 Thread Roger Hoover
correctly count how many Elasticsearch documents were created and indexed. Thanks, Roger Hoover

Re: Thoughts and obesrvations on Samza

2015-07-10 Thread Roger Hoover
ager (or manual placement + restart script) that > was sticky in terms of host placement since there is already a configurable > partition movement timeout and task-by-task state reuse with a check on > state validity. > > -Jay > > On Fri, Jul 10, 2015 at 8:34 AM, Roger Hoover

Re: Thoughts and obesrvations on Samza

2015-07-10 Thread Roger Hoover
e control, > and partition assignment has the advantage of putting all that complex > stuff behind a clean api that the clients are already going to be > implementing for their consumer, so the added functionality for stream > processing beyond a consumer becomes very mino

Re: Samza job on YARN stuck Unassigned

2015-07-10 Thread Roger Hoover
Hi Krzysztof, I haven't seen that error before. It does sound like it could be a connection issue. Did you check that the YARN node has access to hdfs:///user/samza/deploy/event-log-etl-nested-0.1.0-dist.tar.gz? One way to set the AM and containers to debug is to include a log4j.xml file in you

Re: Thoughts and obesrvations on Samza

2015-07-07 Thread Roger Hoover
Metamorphosis...nice. :) This has been a great discussion. As a user of Samza who's recently integrated it into a relatively large organization, I just want to add support to a few points already made. The biggest hurdles to adoption of Samza as it currently exists that I've experienced are: 1)

Re: How to configure log4j separately for the AM versus containers?

2015-06-23 Thread Roger Hoover
Ah, this seems to work. I saw the YarnJob.scala was referencing __package to launch to AM itself. yarn.am.opts=-Xmx768m -XX:+UseSerialGC -Dlog4j.configuration=file://$(pwd)/__package/lib/log4j-am.xml On Tue, Jun 23, 2015 at 12:40 PM, Roger Hoover wrote: > Hi, > > I want the App Mast

How to configure log4j separately for the AM versus containers?

2015-06-23 Thread Roger Hoover
Hi, I want the App Master to log at INFO level and the container to log at ERROR. Is there a way to configure the AM to use a different log4j config file? I'm trying to setting yarn.am.opts but ran couldn't get it to work with system properties. yarn.am.opts=-Xmx768m -XX:+UseSerialGC -Dlog4j.co

Re: [VOTE] Apache Samza 0.9.1 RC0

2015-06-22 Thread Roger Hoover
ems the fix for > SAMZA-720 is pretty localized and I am OK to push it into 0.9.1. I will be > working on back porting those changes to 0.9.1 later today and fix all the > release related issues. > > Thanks! > > -Yi > > On Mon, Jun 22, 2015 at 10:30 AM, Roger Hoover &

Re: [VOTE] Apache Samza 0.9.1 RC0

2015-06-22 Thread Roger Hoover
patch. > > 4. There is no maven staging link. > > Thanks, > > Fang, Yan > yanfang...@gmail.com > > On Sun, Jun 21, 2015 at 1:53 PM, Roger Hoover > wrote: > > > Hi all, > > > > Do you think we could get this bootstrapping bug fixed before 0.9.1 >

Re: Samza hung after bootstrapping

2015-06-21 Thread Roger Hoover
A quick fix: > > 1. apply https://issues.apache.org/jira/browse/SAMZA-712 > 2. add > > sourceSets.main.scala.srcDir "src/main/java" sourceSets.main.java.srcDirs = > [] > > at line 126 of build.gradle. > > Sorry for the inconvenience. > > Thanks, > >

Re: Samza hung after bootstrapping

2015-06-21 Thread Roger Hoover
ant:scalac]^ ... I've got JDK 8 installed. Wondering that makes a difference or not. I'd appreciate any help. Thanks, Roger On Sun, Jun 21, 2015 at 1:02 PM, Roger Hoover wrote: > I think I see what's happening. > > When there are 8 tasks and I set yarn.container.count=8, th

Re: [VOTE] Apache Samza 0.9.1 RC0

2015-06-21 Thread Roger Hoover
Hi all, Do you think we could get this bootstrapping bug fixed before 0.9.1 release? It seems like a critical bug. https://issues.apache.org/jira/browse/SAMZA-720 Thanks, Roger On Sat, Jun 20, 2015 at 10:38 PM, Yan Fang wrote: > Agree. I will test it this weekend. > > Thanks, > > Fang, Yan

Re: Samza hung after bootstrapping

2015-06-21 Thread Roger Hoover
't LinkedIn use partitioned bootstrapped topics? Thanks, Roger On Sun, Jun 21, 2015 at 12:22 PM, Roger Hoover wrote: > Hi Yan, > > I've uploaded a file with TRACE level logging here: > http://filebin.ca/261yhsTZcZQZ/samza-container-0.log.gz > > I really appreciate y

Re: Samza hung after bootstrapping

2015-06-21 Thread Roger Hoover
; process. If you can paste the trace level log, it will be more helpful > because many logs in chooser are trace level. > > Thanks, > > Fang, Yan > yanfang...@gmail.com > > On Thu, Jun 18, 2015 at 5:20 PM, Roger Hoover > wrote: > > > I need some help. I hav

Re: Samza hung after bootstrapping

2015-06-20 Thread Roger Hoover
e trace level. > > Thanks, > > Fang, Yan > yanfang...@gmail.com > > On Thu, Jun 18, 2015 at 5:20 PM, Roger Hoover > wrote: > >> I need some help. I have a job which bootstraps one stream and then is >> supposed to read from two. When I run it on our YARN c

Samza hung after bootstrapping

2015-06-18 Thread Roger Hoover
I need some help. I have a job which bootstraps one stream and then is supposed to read from two. When I run it on our YARN cluster with a single container, it works correctly. When I tried it with 5 containers, it gets hung after consuming the bootstrap topic. I ran it with the grid script on

Re: Yarn scheduling

2015-05-30 Thread Roger Hoover
d above, maxing out a host at a time does sound odd to me. > > Garry > > -Original Message- > From: Roger Hoover [mailto:roger.hoo...@gmail.com] > Sent: 29 May 2015 23:09 > To: dev@samza.apache.org > Subject: Yarn scheduling > > Hi, > > I notice that when YARN sche

Yarn scheduling

2015-05-29 Thread Roger Hoover
Hi, I notice that when YARN schedules my jobs, it loads up one machine completely before scheduling on the next. I'm using Capacity Scheduler with a default config. Is there a way to make it "round-robin" among the available machines? Thanks, Roger

Re: Samza job throughput much lower than Kafka throughput

2015-05-21 Thread Roger Hoover
Oops. Sent too soon. I mean: producer.batch.size=262144 producer.linger.ms=5 producer.compression.type=lz4 On Thu, May 21, 2015 at 9:00 AM, Roger Hoover wrote: > Hi George, > > You might also try tweaking the producer settings. > > producer.ba

Re: Samza job throughput much lower than Kafka throughput

2015-05-21 Thread Roger Hoover
Hi George, You might also try tweaking the producer settings. producer.batch.size=262144 producer.linger.ms=5 producer.compression.type: lz4 On Wed, May 20, 2015 at 9:30 PM, Guozhang Wang wrote: > Hi George, > > Is there any reason you need to set the following configs? > > sys

Re: Errors and hung job on broker shutdown

2015-04-29 Thread Roger Hoover
any progress" ), I think the code is stuck in > >> producer.close > >> < > >> > https://github.com/apache/samza/blob/master/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemProducer.scala#L143 > >> >,

Re: Errors and hung job on broker shutdown

2015-04-28 Thread Roger Hoover
log from the broker as well. > > On Tue, Apr 28, 2015 at 3:31 PM, Roger Hoover > wrote: > > > Hi, > > > > I need some help figuring out what's going on. > > > > I'm running Kafka 0.8.2.1 and Samza 0.9.0 on YARN. All the topics have > > r

Errors and hung job on broker shutdown

2015-04-28 Thread Roger Hoover
Hi, I need some help figuring out what's going on. I'm running Kafka 0.8.2.1 and Samza 0.9.0 on YARN. All the topics have replication factor of 2. I'm bouncing the Kafka broker using SIGTERM (with controlled.shutdown.enable=true). I see the Samza job log this message and then hang (does not ex

Re: How to configure the Resource Manager endpoint for YARN?

2015-04-15 Thread Roger Hoover
Turns out that HADOOP_CONF_DIR is the right env var (YARN_CONF_DIR did not work). I had just messed up the directory path. Doh! Sent from my iPhone > On Apr 15, 2015, at 9:41 AM, Roger Hoover wrote: > > I'll try that. Thanks, Chris. > >> On Wed, Apr 15, 2015 at 9

Re: How to configure the Resource Manager endpoint for YARN?

2015-04-15 Thread Roger Hoover
I'll try that. Thanks, Chris. On Wed, Apr 15, 2015 at 9:37 AM, Chris Riccomini wrote: > Hey Roger, > > Not sure if this makes a difference, but have you tried using: > > export YARN_CONF_DIR=... > > Instead? This is what we use. > > Cheers, > Chris > &g

How to configure the Resource Manager endpoint for YARN?

2015-04-15 Thread Roger Hoover
Hi, I'm trying to deploy a job to a small YARN cluster. How do tell the launcher script where to find the Resource Manager? I tried creating a yarn-site.xml and setting HADOOP_CONF_DIR environment variable but it doesn't find my config. 2015-04-14 22:02:45 ClientHelper [INFO] trying to connect

Re: Extra Systems and other extensions.

2015-04-15 Thread Roger Hoover
Dan, This is great. Would love to have a common ElasticSearch system producer. Cheers, Roger On Tue, Apr 14, 2015 at 1:34 PM, Dan wrote: > Thanks Jakob, I agree they'll be more maintained and tested if they're in > the main repo so that's great. > > I'll sort out Jira's and get some patches

Re: Joining Avro records

2015-04-13 Thread Roger Hoover
"blah blah") .build(); Class is here: https://github.com/Quantiply/rico/blob/master/avro-serde/src/main/java/com/quantiply/avro/Join.java Cheers, Roger On Thu, Apr 9, 2015 at 12:54 PM, Roger Hoover wrote: > Yi Pan, > > Thanks for your response. I'm thinking that I'll

Re: Joining Avro records

2015-04-09 Thread Roger Hoover
> > multiple input data. > > > > All the above solution is still experimental and please feel free to > > provide your feedback and comments on that. If we agree that this > solution > > is good and suit for a broader use case, it can be considered to be used >

Re: Joining Avro records

2015-04-09 Thread Roger Hoover
> is good and suit for a broader use case, it can be considered to be used > outside the "SQL" context as well. > > Best regards! > > -Yi > > On Thu, Apr 9, 2015 at 8:55 AM, Roger Hoover > wrote: > > > Hi Milinda and others, > > > > This is an Avr

Joining Avro records

2015-04-09 Thread Roger Hoover
Hi Milinda and others, This is an Avro question but since you guys are working on Avro support for stream SQL, I thought I'd ask you for help. If I have a two records of type A and B as below and want to join them similar to "SELECT *" in SQL to produce a record of type AB, is there an simple way

Re: Newbie questions after completing "Hello Samza" about performance and project setup

2015-04-09 Thread Roger Hoover
Hi Warren, Yes, I think Hello Samza is the template project to work from. I believe that the slow message rate that you are seeing is because it's subscribed to the the wikipedia IRC stream which may only generate a few events per second. That said, some of the example configuration for the hell

Re: How do you serve the data computed by Samza?

2015-04-02 Thread Roger Hoover
the right things", so > that even newcomers to Samza can have an easy way to ingest data into their > serving system. > > My 2 cents. > > -- > > Felix GV > Data Infrastructure Engineer > Distributed Data Systems > LinkedIn > > f...@linkedin.com > linkedin.

Re: How do you serve the data computed by Samza?

2015-04-02 Thread Roger Hoover
partitioning. If that case, push or pull is the same, yeah? Thanks, Roger On Thu, Apr 2, 2015 at 3:21 PM, Roger Hoover wrote: > Chinmay, > > Thanks for your input. > > I'm not understanding what the difference is. With the design that Felix > laid out, the co-located Kafka co

Re: How do you serve the data computed by Samza?

2015-04-02 Thread Roger Hoover
up - so theoretically - you > could get similar throughput. Downside of this approach is of course - the > storage system partitioning scheme *has to* line up with the Kafka > partitioning scheme. > > On Thu, Apr 2, 2015 at 11:41 AM, Roger Hoover > wrote: > > > Felix, >

Re: How do you serve the data computed by Samza?

2015-04-02 Thread Roger Hoover
ever point in the past it needs (i.e.: rewind) without > affecting the other replicas. Tracking offsets across many replicas in the > push model is a bit more annoying, though still doable, of course. > > -- > > Felix GV > Data Infrastructure Engineer > Distributed Data Sy

Re: How do you serve the data computed by Samza?

2015-03-31 Thread Roger Hoover
Ah, thanks for the great explanation. Any particular reason that the job(s) you described should not be Samza jobs? We're started experimenting with such jobs for Druid and Elasticsearch. For Elasticsearch, the Samza job containers join the Elasticsearch cluster as transport nodes and use the Jav

Re: [VOTE] Apache Samza 0.9.0 RC0

2015-03-31 Thread Roger Hoover
>>> > > > > >>> > > +1 > > > >>> > > > > > >>> > > 1. Validated hello-samza works with 0.9.0 Maven binaries. > > > >>> > > 2. Validated release-0.9.0-rc0 tag exists and has correct > > > checksums. > > >

Re: How do you serve the data computed by Samza?

2015-03-31 Thread Roger Hoover
Hi Felix, 1,3. We're experimenting with both Druid and Elasticsearch for this. We're using Samza to enrich user activity and system performance events then index them in Druid +/or Elasticsearch depending on the use case. 2. These are internal BI/Operations applications 4. We're still getting up

Re: [VOTE] Apache Samza 0.9.0 RC0

2015-03-30 Thread Roger Hoover
27 March 2015 at 16:08, Yan Fang wrote: > >> > > > Hi guys, > >> > > > > >> > > > It has been 72 hours. We got +1 from Yi Pan. Do we extend the > voting > >> to > >> > > > this weekend ? > >>

Re: [VOTE] Apache Samza 0.9.0 RC0

2015-03-26 Thread Roger Hoover
Hi Chris + all, I created a basic job that does a join from local state with Samza 0.9.0 ( https://github.com/Quantiply/rico-playground/tree/master/join/samza). So far so good. I hoping to get some time this weekend to benchmark it on my laptop. I think I saw that 0.9.0 includes support for send

Re: Kafka partition key

2015-03-26 Thread Roger Hoover
Hi Richard, You can also partition by a key like "user_id" so that all messages for a given user would end up in the same partition. This can be useful for calculating user-specific aggregations or doing a distributed join where the local state is also partitioned on user_id. Cheers, Roger On

Re: java.lang.NoClassDefFoundError on Yarn job

2015-03-26 Thread Roger Hoover
Hi Jordi, You might be running into this issue ( https://issues.apache.org/jira/browse/SAMZA-456) which I just hit as well. You probably need to add a couple more jars to your YARN lib dir. Cheers, Roger On Thu, Mar 26, 2015 at 9:21 AM, Jordi Blasi Uribarri wrote: > Hi: > > I got samza runnin

Re: Error running integration tests

2015-03-25 Thread Roger Hoover
ring if you need to enable > SSH on your laptop? I have "remote login" enabled on my OSX laptop. > > Cheers, > Chris > > On Wed, Mar 25, 2015 at 4:29 PM, Roger Hoover > wrote: > >> Do I need to bring up sshd on my laptop or can the tests be made to no

Re: Error running integration tests

2015-03-25 Thread Roger Hoover
Do I need to bring up sshd on my laptop or can the tests be made to not ssh? On Wed, Mar 25, 2015 at 4:27 PM, Roger Hoover wrote: > Hi, > > I wanted to see if I could run the integration tests on the 0.9.0 branch > on my Mac. > > I cloned the 0.9.0 branch from the gi

Error running integration tests

2015-03-25 Thread Roger Hoover
Hi, I wanted to see if I could run the integration tests on the 0.9.0 branch on my Mac. I cloned the 0.9.0 branch from the github mirror, built everything (./gradlew clean build), and tried to run the integration tests. ./bin/integration-tests.sh /tmp/roger I get an error when the test script tr

Re: Log error deploying on YARN [Samza 0.8.0]

2015-03-24 Thread Roger Hoover
ssed some JARs that > need to be put in to the YARN NM classpath. > > Cheers, > Chris > > On Tue, Mar 24, 2015 at 2:22 PM, Roger Hoover > wrote: > > > Hi all, > > > > I'm new to YARN and trying to have YARN download the Samza job tarball ( > > h

Log error deploying on YARN [Samza 0.8.0]

2015-03-24 Thread Roger Hoover
Hi all, I'm new to YARN and trying to have YARN download the Samza job tarball ( https://samza.apache.org/learn/tutorials/0.8/run-in-multi-node-yarn.html). >From the log, it seems that the download failed. I've tested that the file is available via curl. The error message is: org/apache/samza/u

Re: Kafka topic naming conventions

2015-03-19 Thread Roger Hoover
; Although they don't mention anything about the number of partitions when > doing so, anyways maybe it helps. > > > Renato M. > > [1] https://www.mail-archive.com/users@kafka.apache.org/msg11976.html > > 2015-03-19 5:43 GMT+01:00 Roger Hoover : > > > Thanks, guys

  1   2   >