You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@samza.apache.org by Shekar Tippur <ct...@gmail.com> on 2014/08/21 19:15:55 UTC

Samza as a Caching layer

Hello,

I am new to Samza. I have just installed Hello Samza and got it working.

Here is the use case for which I am trying to use Samza:


1. Cache the contextual information which contains more information about
the hostname or IP address using Samza/Yarn/Kafka
2. Collect alert and metric events which contain either hostname or IP
address
3. Append contextual information to the alert and metric and insert to a
Kafka queue from which other subscribers read off of.

Can you please shed some light on

1. Is this feasible?
2. Am I on the right thought process
3. How do I start

I now have 1 & 2 of them working disparately. I need to integrate them.

Appreciate any input.

- Shekar

Re: Samza as a Caching layer

Posted by Shekar Tippur <ct...@gmail.com>.
Yan,
These are teething issues. I am quite stoked with the possibilities Samza
brings to table. I really appreciate all the hand holding till now.

1. I am using the master branch -
https://github.com/apache/incubator-samza-hello-samza

this is whats present on .git/config

[core]

        repositoryformatversion = 0

        filemode = true

        bare = false

        logallrefupdates = true

[remote "origin"]

        fetch = +refs/heads/*:refs/remotes/origin/*

        url = https://github.com/apache/incubator-samza-hello-samza.git

[branch "master"]

        remote = origin

        merge = refs/heads/master

2. Yes. I do see messages in Kafka - I run

$ deploy/kafka/bin/kafka-console-consumer.sh  --zookeeper localhost:2181
--topic wikipedia-raw

SLF4J: Failed to load class "org.slf4j.impl.StaticLoggerBinder".

SLF4J: Defaulting to no-operation (NOP) logger implementation

SLF4J: See http://www.slf4j.org/codes.html#StaticLoggerBinder for further
details.

{"raw":"[[The Elder Scrolls Online]] M
http://en.wikipedia.org/w/index.php?diff=606586804&oldid=606577107 * ESO
Fan * (+0)
","time":1398926301271,"source":"rc-pmtpa","channel":"#en.wikipedia"}


3. Will give you a feedback soon. Trying it now.

- Shekar

Re: Samza as a Caching layer

Posted by Shekar Tippur <ct...@gmail.com>.
Thanks a lot for the continued support. I meant to sent this email earlier.

I have the stream to table working well. Doing some performance testing and
building packages. I will fork and update github with my changes soon.

Thanks again.

- Shekar

On Tue, Sep 9, 2014 at 3:15 PM, Chris Riccomini <
criccomini@linkedin.com.invalid> wrote:

> Hey Shekar,
>
> Your understanding of the jobs is correct.
>
> The WikipediaFeedEvent class is just a convenience class so that
> StreamTask code doesn't have to deal with poking around inside the JSON
> structure. Instead, they get WikipediaFeedEvent events, which means the
> code can just call WikipediaFeedEvent.getChannel, etc.
>
> Cheers,
> Chris
>
> On 9/9/14 2:58 PM, "Shekar Tippur" <ct...@gmail.com> wrote:
>
> >More Questions ..
> >
> >Please correct me if I am wrong. As I am trying to unravel hello-samza,
> >
> >
> http://samza.incubator.apache.org/learn/tutorials/0.7.0/run-hello-samza-wi
> >thout-internet.html
> >
> >1. wikipedia-feed.properties - deploy a Samza job which listens to
> >wikipedia API, receives the feed in realtime and produces the feed to the
> >Kafka topic wikipedia-raw
> >2. wikipedia-parser - This pulls the messages from wikipedia-raw and
> >registers to an event that checks the integrity of the incoming message.
> >Why do we need to register this to an event?
> >
> >WikipediaFeedEvent event = new WikipediaFeedEvent(jsonObject);
> >
> >
> >- Shekar
> >
> >On Mon, Sep 8, 2014 at 10:54 AM, Chris Riccomini <
> >criccomini@linkedin.com.invalid> wrote:
> >
> >> Hey Shekar,
> >>
> >> Sure. If your input stream has 8 partitions and is partitioned by "ip
> >> address", then your state stream must also have 8 partitions and be
> >> partitioned by "ip address". This is to guarantee that the StreamTask
> >>that
> >> receives a message from the stream will have the state required to do
> >>the
> >> table join in its local store.
> >>
> >> Cheers,
> >> Chris
> >>
> >> On 9/8/14 10:51 AM, "Shekar Tippur" <ct...@gmail.com> wrote:
> >>
> >> >Chris -
> >> >Can you please elaborate on
> >> >
> >> >"Also note that if you take this approach, your state
> >> >must be partitioned in the same way as your input stream."
> >> >
> >> >- Shekar
> >> >
> >> >On Mon, Sep 8, 2014 at 10:29 AM, Chris Riccomini <
> >> >criccomini@linkedin.com.invalid> wrote:
> >> >
> >> >> Hey Shekar,
> >> >>
> >> >> You can either run some external DB that holds the data set, and you
> >>can
> >> >> query it from a StreamTask, or you can use Samza's state store
> >>feature
> >> >>to
> >> >> push data into a stream that you can then store in a partitioned
> >> >>key-value
> >> >> store along with your StreamTasks. There is some documentation here
> >> >>about
> >> >> the state store approach:
> >> >>
> >> >>
> >> >>
> >>
> >>
> http://samza.incubator.apache.org/learn/documentation/0.7.0/container/sta
> >> >>te
> >> >> -management.html
> >> >>
> >> >> Putting your data into a Kafka stream is going to require more up
> >>front
> >> >> effort from you, since you'll have to understand how Kafka's
> >> >>partitioning
> >> >> model works, and setup some pipeline to push the updates for your
> >>state.
> >> >> In the long run, I believe it's the better approach, though. Local
> >> >>lookups
> >> >> on a key-value store should be faster than doing remote RPC calls to
> >>a
> >> >>DB
> >> >> for every message. Also note that if you take this approach, your
> >>state
> >> >> must be partitioned in the same way as your input stream.
> >> >>
> >> >> I'm sorry I can't give you a more definitive answer. It's really
> >>about
> >> >> trade-offs.
> >> >>
> >> >>
> >> >> Cheers,
> >> >> Chris
> >> >>
> >> >> On 9/8/14 10:17 AM, "Shekar Tippur" <ct...@gmail.com> wrote:
> >> >>
> >> >> >Hello,
> >> >> >
> >> >> >I am able to read messages of of a new kafka queue now.
> >> >> >The next task is to enrich the data with more information. The data
> >> >>that
> >> >> >is
> >> >> >flowing in has ip address or host name. I have a redis cache where
> >> >>there
> >> >> >is
> >> >> >more contextual information (like owner of the alert, SLA, etc). The
> >> >>data
> >> >> >in redis does not change often.
> >> >> >Pretty much becomes a stream table join.
> >> >> >I can also dump the same data to a different kafka queue and make
> >>it a
> >> >> >stream - stream join as well.
> >> >> >
> >> >> >What do you guys recommend?
> >> >> >
> >> >> >- Shekar
> >> >> >
> >> >> >On Fri, Sep 5, 2014 at 4:08 PM, Chris Riccomini <
> >> >> >criccomini@linkedin.com.invalid> wrote:
> >> >> >
> >> >> >> Hey Guys,
> >> >> >>
> >> >> >> I don't know a whole lot about Fluentd, but if you don't want to
> >>do
> >> >>this
> >> >> >> flow:
> >> >> >>
> >> >> >>   Fluentd -> Kafka -> Samza
> >> >> >>
> >> >> >> Then the alternative is:
> >> >> >>
> >> >> >>   Fluentd -> Samza
> >> >> >>
> >> >> >> The "direct" approach (no Kafka) is going to be pretty labor
> >> >>intensive
> >> >> >>to
> >> >> >> build. You'd have to:
> >> >> >>
> >> >> >> 1. Implement a FluentdSystemConsumer for Samza.
> >> >> >> 2. Write a Flutend data output plugin, which sends to the
> >> >> >> FluentdSystemConsumer.
> >> >> >> 3. Figure out a way for the Fluentd data output plugin to
> >>"discover"
> >> >> >>where
> >> >> >> the Samza FluentdSystemConsumer is located (since SamzaContainers
> >>are
> >> >> >> deployed to dynamic hosts in YARN, and move around a lot).
> >> >> >> 4. Implement a bare-bones FluentdSystemAdmin and
> >>FluentdSystemFactory
> >> >> >> class (similar to the WikipediaSystemFactory in hello-samza).
> >> >> >> 5. Decide on some partitioning model that makes sense for Fluentd.
> >> >>Maybe
> >> >> >> one partition = one host? Not sure how Fluentd works here.
> >> >> >>
> >> >> >> My instinct is that it's going to be *far* better to use the first
> >> >> >> approach (pipe the Fluentd events into Kafka). This will give you
> >> >>all of
> >> >> >> the semantics that Kafka provides (e.g. Ordering within a
> >>partition,
> >> >> >> rewinding streams, durability, etc).
> >> >> >>
> >> >> >> Cheers,
> >> >> >> Chris
> >> >> >>
> >> >> >> On 9/5/14 1:36 PM, "Yan Fang" <ya...@gmail.com> wrote:
> >> >> >>
> >> >> >> >also was thinking of having fluentd push to Samza. But don't know
> >> >>how
> >> >> >>to
> >> >> >> >implement this. Not sure if adding a kafka layer between Samza
> >>and
> >> >> >>fluentd
> >> >> >> >is the only option.
> >> >> >> >
> >> >> >> >Do other guys have better ideas?
> >> >> >> >
> >> >> >> >Thanks,
> >> >> >> >
> >> >> >> >Fang, Yan
> >> >> >> >yanfang724@gmail.com
> >> >> >> >+1 (206) 849-4108
> >> >> >> >
> >> >> >> >
> >> >> >> >On Fri, Sep 5, 2014 at 12:09 PM, Shekar Tippur
> >><ct...@gmail.com>
> >> >> >>wrote:
> >> >> >> >
> >> >> >> >> Yan,
> >> >> >> >>
> >> >> >> >> Wont it add additional hop. It did occur to me earlier but was
> >>not
> >> >> >>sure
> >> >> >> >>if
> >> >> >> >> this is the right way to go if we have a stringent sla driven
> >> >>system
> >> >> >> >> depending on it.
> >> >> >> >>
> >> >> >> >> - Shekar
> >> >> >> >>
> >> >> >> >>
> >> >> >> >> On Fri, Sep 5, 2014 at 10:55 AM, Yan Fang
> >><ya...@gmail.com>
> >> >> >>wrote:
> >> >> >> >>
> >> >> >> >> > If you already put the events to the kafka, you can make the
> >> >>Samza
> >> >> >> >> accepts
> >> >> >> >> > the kafka topic, like the wikipedia-parse project in
> >>hello-samza
> >> >> >> >>accepts
> >> >> >> >> > the kafka topic wikipedia-raw ( see the config file ).
> >> >> >> >> >
> >> >> >> >> > Thanks,
> >> >> >> >> >
> >> >> >> >> > Fang, Yan
> >> >> >> >> > yanfang724@gmail.com
> >> >> >> >> > +1 (206) 849-4108
> >> >> >> >> >
> >> >> >> >> >
> >> >> >> >> > On Fri, Sep 5, 2014 at 8:48 AM, Shekar Tippur
> >> >><ct...@gmail.com>
> >> >> >> >>wrote:
> >> >> >> >> >
> >> >> >> >> > > Awesome .. This works. Thanks a lot.
> >> >> >> >> > >
> >> >> >> >> > > Now off to my next step.
> >> >> >> >> > > I want to point to an incoming stream of events. These
> >>events
> >> >>are
> >> >> >> >> routed
> >> >> >> >> > > via fluentd. So, fluentd acts as a routing layer where it
> >> >>pushes
> >> >> >>the
> >> >> >> >> > events
> >> >> >> >> > > to kafka. Since it is a push and not a pull, any pointers
> >>on
> >> >>how
> >> >> >>to
> >> >> >> >> push
> >> >> >> >> > it
> >> >> >> >> > > to samza? Guessing I need a listener on Samza to collect
> >>this?
> >> >> >> >> > >
> >> >> >> >> > > - Shekar
> >> >> >> >> > >
> >> >> >> >> > >
> >> >> >> >> > > On Fri, Sep 5, 2014 at 1:03 AM, Yan Fang
> >> >><ya...@gmail.com>
> >> >> >> >>wrote:
> >> >> >> >> > >
> >> >> >> >> > > > Aha, yes, we are almost there. I think I made a mistake
> >>in
> >> >>the
> >> >> >> >> previous
> >> >> >> >> > > > email.
> >> >> >> >> > > >
> >> >> >> >> > > > 1. modify the *wikipedia-parser.properties ,  NOT
> >> >> >> >> > > > *wikipedia-feed.properties
> >> >> >> >> > > > 2. run deploy/samza/bin/run-job.sh
> >> >> >> >> > > >
> >> >> >> >> > >
> >> >> >> >> >
> >> >> >> >>
> >> >> >>
> >> >>
> >>
> >>>>>>>>--config-factory=org.apache.samza.config.factories.PropertiesConfig
> >>>>>>>>Fa
> >> >>>>>>ct
> >> >> >>>>or
> >> >> >> >>y
> >> >> >> >> > > >
> >> >> >> >> > >
> >> >> >> >> >
> >> >> >> >>
> >> >> >>
> >> >>
> >>
> >>>>>>>>--config-path=file://$PWD/deploy/samza/config/*
> wikipedia-parser.pro
> >>>>>>>>pe
> >> >>>>>>rt
> >> >> >>>>ie
> >> >> >> >>s*
> >> >> >> >> > > > *(NOT *wikipedia-feed,properties*)*
> >> >> >> >> > > >
> >> >> >> >> > > > Then you should see the messages in the kafka topic,
> >> >> >> >> *wikipedia-edits*
> >> >> >> >> > > >
> >> >> >> >> > > > Thanks. Let me know if you have any luck . :)
> >> >> >> >> > > >
> >> >> >> >> > > > Cheers,
> >> >> >> >> > > >
> >> >> >> >> > > > Fang, Yan
> >> >> >> >> > > > yanfang724@gmail.com
> >> >> >> >> > > > +1 (206) 849-4108
> >> >> >> >> > > >
> >> >> >> >> > > >
> >> >> >> >> > > > On Thu, Sep 4, 2014 at 11:19 PM, Shekar Tippur
> >> >> >><ctippur@gmail.com
> >> >> >> >
> >> >> >> >> > > wrote:
> >> >> >> >> > > >
> >> >> >> >> > > > > Just tried #3. Changed the property file
> >> >> >> >>wikipedia-feed.properties
> >> >> >> >> > > > >
> >> >> >> >> > > > >
> >> >>job.factory.class=org.apache.samza.job.local.LocalJobFactory
> >> >> >> >> > > > > Ran ..
> >> >> >> >> > > > >
> >> >> >> >> > > > > deploy/samza/bin/run-job.sh
> >> >> >> >> > > > >
> >> >> >> >> > > >
> >> >> >> >> > >
> >> >> >> >> >
> >> >> >> >>
> >> >> >>
> >> >>
> >>
> >>>>>>>>--config-factory=org.apache.samza.config.factories.PropertiesConfig
> >>>>>>>>Fa
> >> >>>>>>ct
> >> >> >>>>or
> >> >> >> >>y
> >> >> >> >> > > > >
> >> >> >> >> > > > >
> >> >> >> >> > > >
> >> >> >> >> > >
> >> >> >> >> >
> >> >> >> >>
> >> >> >>
> >> >>
> >>
> >>>>>>>>--config-path=file:///home/ctippur/hello-samza/deploy/samza/config/
> >>>>>>>>wi
> >> >>>>>>ki
> >> >> >>>>pe
> >> >> >> >>dia-feed.properties
> >> >> >> >> > > > >
> >> >> >> >> > > > > I dont see any debug messages that I added to the feed
> >>or
> >> >>the
> >> >> >> >> parser
> >> >> >> >> > > > file..
> >> >> >> >> > > > > I see messages on the kafka-consumer ..
> >> >> >> >> > > > >
> >> >> >> >> > > > > However the feed job died with the below message
> >> >> >> >> > > > >
> >> >> >> >> > > > >
> >> >> >> >> > > > > Exception in thread "ThreadJob"
> >> >>java.lang.RuntimeException:
> >> >> >> >>Trying
> >> >> >> >> to
> >> >> >> >> > > > > unlisten to a channel that has no listeners in it.
> >> >> >> >> > > > >
> >> >> >> >> > > > > at
> >> >> >> >> > > > >
> >> >> >> >> > > > >
> >> >> >> >> > > >
> >> >> >> >> > >
> >> >> >> >> >
> >> >> >> >>
> >> >> >>
> >> >>
> >>
> >>>>>>>>samza.examples.wikipedia.system.WikipediaFeed.unlisten(WikipediaFee
> >>>>>>>>d.
> >> >>>>>>ja
> >> >> >>>>va
> >> >> >> >>:98)
> >> >> >> >> > > > >
> >> >> >> >> > > > > at
> >> >> >> >> > > > >
> >> >> >> >> > > > >
> >> >> >> >> > > >
> >> >> >> >> > >
> >> >> >> >> >
> >> >> >> >>
> >> >> >>
> >> >>
> >>
> >>>>>>>>samza.examples.wikipedia.system.WikipediaConsumer.stop(WikipediaCon
> >>>>>>>>su
> >> >>>>>>me
> >> >> >>>>r.
> >> >> >> >>java:72)
> >> >> >> >> > > > >
> >> >> >> >> > > > > at
> >> >> >> >> > > > >
> >> >> >> >> > > > >
> >> >> >> >> > > >
> >> >> >> >> > >
> >> >> >> >> >
> >> >> >> >>
> >> >> >>
> >> >>
> >>
> >>>>>>>>org.apache.samza.system.SystemConsumers$$anonfun$stop$2.apply(Syste
> >>>>>>>>mC
> >> >>>>>>on
> >> >> >>>>su
> >> >> >> >>mers.scala:152)
> >> >> >> >> > > > >
> >> >> >> >> > > > > at
> >> >> >> >> > > > >
> >> >> >> >> > > > >
> >> >> >> >> > > >
> >> >> >> >> > >
> >> >> >> >> >
> >> >> >> >>
> >> >> >>
> >> >>
> >>
> >>>>>>>>org.apache.samza.system.SystemConsumers$$anonfun$stop$2.apply(Syste
> >>>>>>>>mC
> >> >>>>>>on
> >> >> >>>>su
> >> >> >> >>mers.scala:152)
> >> >> >> >> > > > >
> >> >> >> >> > > > > at
> >> >> >>scala.collection.Iterator$class.foreach(Iterator.scala:727)
> >> >> >> >> > > > >
> >> >> >> >> > > > > at
> >> >> >> >>scala.collection.AbstractIterator.foreach(Iterator.scala:1157)
> >> >> >> >> > > > >
> >> >> >> >> > > > > at
> >> >> >> >> > > > >
> >> >> >> >> > >
> >> >> >> >>
> >> >> >>
> >> >>
> >>
> >>>>>>>>scala.collection.MapLike$DefaultValuesIterable.foreach(MapLike.scal
> >>>>>>>>a:
> >> >>>>>>20
> >> >> >>>>6)
> >> >> >> >> > > > >
> >> >> >> >> > > > > at
> >> >> >> >> > > >
> >> >> >> >>
> >> >>
> >>
> >>>>>>org.apache.samza.system.SystemConsumers.stop(SystemConsumers.scala:15
> >>>>>>2)
> >> >> >> >> > > > >
> >> >> >> >> > > > > at
> >> >> >> >> > > > >
> >> >> >> >> > > > >
> >> >> >> >> > > >
> >> >> >> >> > >
> >> >> >> >> >
> >> >> >> >>
> >> >> >>
> >> >>
> >>
> >>>>>>>>org.apache.samza.container.SamzaContainer.shutdownConsumers(SamzaCo
> >>>>>>>>nt
> >> >>>>>>ai
> >> >> >>>>ne
> >> >> >> >>r.scala:587)
> >> >> >> >> > > > >
> >> >> >> >> > > > > at
> >> >> >> >> > > >
> >> >> >> >>
> >> >>
> >>
> >>>>>>org.apache.samza.container.SamzaContainer.run(SamzaContainer.scala:51
> >>>>>>2)
> >> >> >> >> > > > >
> >> >> >> >> > > > >  at
> >> >> >> >> > >
> >> >>
> >>>>org.apache.samza.job.local.ThreadJob$$anon$1.run(ThreadJob.scala:42)
> >> >> >> >> > > > >
> >> >> >> >> > > > > - Shekar
> >> >> >> >> > > > >
> >> >> >> >> > > >
> >> >> >> >> > >
> >> >> >> >> >
> >> >> >> >>
> >> >> >>
> >> >> >>
> >> >>
> >> >>
> >>
> >>
>
>

Re: Samza as a Caching layer

Posted by Chris Riccomini <cr...@linkedin.com.INVALID>.
Hey Shekar,

Your understanding of the jobs is correct.

The WikipediaFeedEvent class is just a convenience class so that
StreamTask code doesn't have to deal with poking around inside the JSON
structure. Instead, they get WikipediaFeedEvent events, which means the
code can just call WikipediaFeedEvent.getChannel, etc.

Cheers,
Chris

On 9/9/14 2:58 PM, "Shekar Tippur" <ct...@gmail.com> wrote:

>More Questions ..
>
>Please correct me if I am wrong. As I am trying to unravel hello-samza,
>
>http://samza.incubator.apache.org/learn/tutorials/0.7.0/run-hello-samza-wi
>thout-internet.html
>
>1. wikipedia-feed.properties - deploy a Samza job which listens to
>wikipedia API, receives the feed in realtime and produces the feed to the
>Kafka topic wikipedia-raw
>2. wikipedia-parser - This pulls the messages from wikipedia-raw and
>registers to an event that checks the integrity of the incoming message.
>Why do we need to register this to an event?
>
>WikipediaFeedEvent event = new WikipediaFeedEvent(jsonObject);
>
>
>- Shekar
>
>On Mon, Sep 8, 2014 at 10:54 AM, Chris Riccomini <
>criccomini@linkedin.com.invalid> wrote:
>
>> Hey Shekar,
>>
>> Sure. If your input stream has 8 partitions and is partitioned by "ip
>> address", then your state stream must also have 8 partitions and be
>> partitioned by "ip address". This is to guarantee that the StreamTask
>>that
>> receives a message from the stream will have the state required to do
>>the
>> table join in its local store.
>>
>> Cheers,
>> Chris
>>
>> On 9/8/14 10:51 AM, "Shekar Tippur" <ct...@gmail.com> wrote:
>>
>> >Chris -
>> >Can you please elaborate on
>> >
>> >"Also note that if you take this approach, your state
>> >must be partitioned in the same way as your input stream."
>> >
>> >- Shekar
>> >
>> >On Mon, Sep 8, 2014 at 10:29 AM, Chris Riccomini <
>> >criccomini@linkedin.com.invalid> wrote:
>> >
>> >> Hey Shekar,
>> >>
>> >> You can either run some external DB that holds the data set, and you
>>can
>> >> query it from a StreamTask, or you can use Samza's state store
>>feature
>> >>to
>> >> push data into a stream that you can then store in a partitioned
>> >>key-value
>> >> store along with your StreamTasks. There is some documentation here
>> >>about
>> >> the state store approach:
>> >>
>> >>
>> >>
>> 
>>http://samza.incubator.apache.org/learn/documentation/0.7.0/container/sta
>> >>te
>> >> -management.html
>> >>
>> >> Putting your data into a Kafka stream is going to require more up
>>front
>> >> effort from you, since you'll have to understand how Kafka's
>> >>partitioning
>> >> model works, and setup some pipeline to push the updates for your
>>state.
>> >> In the long run, I believe it's the better approach, though. Local
>> >>lookups
>> >> on a key-value store should be faster than doing remote RPC calls to
>>a
>> >>DB
>> >> for every message. Also note that if you take this approach, your
>>state
>> >> must be partitioned in the same way as your input stream.
>> >>
>> >> I'm sorry I can't give you a more definitive answer. It's really
>>about
>> >> trade-offs.
>> >>
>> >>
>> >> Cheers,
>> >> Chris
>> >>
>> >> On 9/8/14 10:17 AM, "Shekar Tippur" <ct...@gmail.com> wrote:
>> >>
>> >> >Hello,
>> >> >
>> >> >I am able to read messages of of a new kafka queue now.
>> >> >The next task is to enrich the data with more information. The data
>> >>that
>> >> >is
>> >> >flowing in has ip address or host name. I have a redis cache where
>> >>there
>> >> >is
>> >> >more contextual information (like owner of the alert, SLA, etc). The
>> >>data
>> >> >in redis does not change often.
>> >> >Pretty much becomes a stream table join.
>> >> >I can also dump the same data to a different kafka queue and make
>>it a
>> >> >stream - stream join as well.
>> >> >
>> >> >What do you guys recommend?
>> >> >
>> >> >- Shekar
>> >> >
>> >> >On Fri, Sep 5, 2014 at 4:08 PM, Chris Riccomini <
>> >> >criccomini@linkedin.com.invalid> wrote:
>> >> >
>> >> >> Hey Guys,
>> >> >>
>> >> >> I don't know a whole lot about Fluentd, but if you don't want to
>>do
>> >>this
>> >> >> flow:
>> >> >>
>> >> >>   Fluentd -> Kafka -> Samza
>> >> >>
>> >> >> Then the alternative is:
>> >> >>
>> >> >>   Fluentd -> Samza
>> >> >>
>> >> >> The "direct" approach (no Kafka) is going to be pretty labor
>> >>intensive
>> >> >>to
>> >> >> build. You'd have to:
>> >> >>
>> >> >> 1. Implement a FluentdSystemConsumer for Samza.
>> >> >> 2. Write a Flutend data output plugin, which sends to the
>> >> >> FluentdSystemConsumer.
>> >> >> 3. Figure out a way for the Fluentd data output plugin to
>>"discover"
>> >> >>where
>> >> >> the Samza FluentdSystemConsumer is located (since SamzaContainers
>>are
>> >> >> deployed to dynamic hosts in YARN, and move around a lot).
>> >> >> 4. Implement a bare-bones FluentdSystemAdmin and
>>FluentdSystemFactory
>> >> >> class (similar to the WikipediaSystemFactory in hello-samza).
>> >> >> 5. Decide on some partitioning model that makes sense for Fluentd.
>> >>Maybe
>> >> >> one partition = one host? Not sure how Fluentd works here.
>> >> >>
>> >> >> My instinct is that it's going to be *far* better to use the first
>> >> >> approach (pipe the Fluentd events into Kafka). This will give you
>> >>all of
>> >> >> the semantics that Kafka provides (e.g. Ordering within a
>>partition,
>> >> >> rewinding streams, durability, etc).
>> >> >>
>> >> >> Cheers,
>> >> >> Chris
>> >> >>
>> >> >> On 9/5/14 1:36 PM, "Yan Fang" <ya...@gmail.com> wrote:
>> >> >>
>> >> >> >also was thinking of having fluentd push to Samza. But don't know
>> >>how
>> >> >>to
>> >> >> >implement this. Not sure if adding a kafka layer between Samza
>>and
>> >> >>fluentd
>> >> >> >is the only option.
>> >> >> >
>> >> >> >Do other guys have better ideas?
>> >> >> >
>> >> >> >Thanks,
>> >> >> >
>> >> >> >Fang, Yan
>> >> >> >yanfang724@gmail.com
>> >> >> >+1 (206) 849-4108
>> >> >> >
>> >> >> >
>> >> >> >On Fri, Sep 5, 2014 at 12:09 PM, Shekar Tippur
>><ct...@gmail.com>
>> >> >>wrote:
>> >> >> >
>> >> >> >> Yan,
>> >> >> >>
>> >> >> >> Wont it add additional hop. It did occur to me earlier but was
>>not
>> >> >>sure
>> >> >> >>if
>> >> >> >> this is the right way to go if we have a stringent sla driven
>> >>system
>> >> >> >> depending on it.
>> >> >> >>
>> >> >> >> - Shekar
>> >> >> >>
>> >> >> >>
>> >> >> >> On Fri, Sep 5, 2014 at 10:55 AM, Yan Fang
>><ya...@gmail.com>
>> >> >>wrote:
>> >> >> >>
>> >> >> >> > If you already put the events to the kafka, you can make the
>> >>Samza
>> >> >> >> accepts
>> >> >> >> > the kafka topic, like the wikipedia-parse project in
>>hello-samza
>> >> >> >>accepts
>> >> >> >> > the kafka topic wikipedia-raw ( see the config file ).
>> >> >> >> >
>> >> >> >> > Thanks,
>> >> >> >> >
>> >> >> >> > Fang, Yan
>> >> >> >> > yanfang724@gmail.com
>> >> >> >> > +1 (206) 849-4108
>> >> >> >> >
>> >> >> >> >
>> >> >> >> > On Fri, Sep 5, 2014 at 8:48 AM, Shekar Tippur
>> >><ct...@gmail.com>
>> >> >> >>wrote:
>> >> >> >> >
>> >> >> >> > > Awesome .. This works. Thanks a lot.
>> >> >> >> > >
>> >> >> >> > > Now off to my next step.
>> >> >> >> > > I want to point to an incoming stream of events. These
>>events
>> >>are
>> >> >> >> routed
>> >> >> >> > > via fluentd. So, fluentd acts as a routing layer where it
>> >>pushes
>> >> >>the
>> >> >> >> > events
>> >> >> >> > > to kafka. Since it is a push and not a pull, any pointers
>>on
>> >>how
>> >> >>to
>> >> >> >> push
>> >> >> >> > it
>> >> >> >> > > to samza? Guessing I need a listener on Samza to collect
>>this?
>> >> >> >> > >
>> >> >> >> > > - Shekar
>> >> >> >> > >
>> >> >> >> > >
>> >> >> >> > > On Fri, Sep 5, 2014 at 1:03 AM, Yan Fang
>> >><ya...@gmail.com>
>> >> >> >>wrote:
>> >> >> >> > >
>> >> >> >> > > > Aha, yes, we are almost there. I think I made a mistake
>>in
>> >>the
>> >> >> >> previous
>> >> >> >> > > > email.
>> >> >> >> > > >
>> >> >> >> > > > 1. modify the *wikipedia-parser.properties ,  NOT
>> >> >> >> > > > *wikipedia-feed.properties
>> >> >> >> > > > 2. run deploy/samza/bin/run-job.sh
>> >> >> >> > > >
>> >> >> >> > >
>> >> >> >> >
>> >> >> >>
>> >> >>
>> >>
>> 
>>>>>>>>--config-factory=org.apache.samza.config.factories.PropertiesConfig
>>>>>>>>Fa
>> >>>>>>ct
>> >> >>>>or
>> >> >> >>y
>> >> >> >> > > >
>> >> >> >> > >
>> >> >> >> >
>> >> >> >>
>> >> >>
>> >>
>> 
>>>>>>>>--config-path=file://$PWD/deploy/samza/config/*wikipedia-parser.pro
>>>>>>>>pe
>> >>>>>>rt
>> >> >>>>ie
>> >> >> >>s*
>> >> >> >> > > > *(NOT *wikipedia-feed,properties*)*
>> >> >> >> > > >
>> >> >> >> > > > Then you should see the messages in the kafka topic,
>> >> >> >> *wikipedia-edits*
>> >> >> >> > > >
>> >> >> >> > > > Thanks. Let me know if you have any luck . :)
>> >> >> >> > > >
>> >> >> >> > > > Cheers,
>> >> >> >> > > >
>> >> >> >> > > > Fang, Yan
>> >> >> >> > > > yanfang724@gmail.com
>> >> >> >> > > > +1 (206) 849-4108
>> >> >> >> > > >
>> >> >> >> > > >
>> >> >> >> > > > On Thu, Sep 4, 2014 at 11:19 PM, Shekar Tippur
>> >> >><ctippur@gmail.com
>> >> >> >
>> >> >> >> > > wrote:
>> >> >> >> > > >
>> >> >> >> > > > > Just tried #3. Changed the property file
>> >> >> >>wikipedia-feed.properties
>> >> >> >> > > > >
>> >> >> >> > > > >
>> >>job.factory.class=org.apache.samza.job.local.LocalJobFactory
>> >> >> >> > > > > Ran ..
>> >> >> >> > > > >
>> >> >> >> > > > > deploy/samza/bin/run-job.sh
>> >> >> >> > > > >
>> >> >> >> > > >
>> >> >> >> > >
>> >> >> >> >
>> >> >> >>
>> >> >>
>> >>
>> 
>>>>>>>>--config-factory=org.apache.samza.config.factories.PropertiesConfig
>>>>>>>>Fa
>> >>>>>>ct
>> >> >>>>or
>> >> >> >>y
>> >> >> >> > > > >
>> >> >> >> > > > >
>> >> >> >> > > >
>> >> >> >> > >
>> >> >> >> >
>> >> >> >>
>> >> >>
>> >>
>> 
>>>>>>>>--config-path=file:///home/ctippur/hello-samza/deploy/samza/config/
>>>>>>>>wi
>> >>>>>>ki
>> >> >>>>pe
>> >> >> >>dia-feed.properties
>> >> >> >> > > > >
>> >> >> >> > > > > I dont see any debug messages that I added to the feed
>>or
>> >>the
>> >> >> >> parser
>> >> >> >> > > > file..
>> >> >> >> > > > > I see messages on the kafka-consumer ..
>> >> >> >> > > > >
>> >> >> >> > > > > However the feed job died with the below message
>> >> >> >> > > > >
>> >> >> >> > > > >
>> >> >> >> > > > > Exception in thread "ThreadJob"
>> >>java.lang.RuntimeException:
>> >> >> >>Trying
>> >> >> >> to
>> >> >> >> > > > > unlisten to a channel that has no listeners in it.
>> >> >> >> > > > >
>> >> >> >> > > > > at
>> >> >> >> > > > >
>> >> >> >> > > > >
>> >> >> >> > > >
>> >> >> >> > >
>> >> >> >> >
>> >> >> >>
>> >> >>
>> >>
>> 
>>>>>>>>samza.examples.wikipedia.system.WikipediaFeed.unlisten(WikipediaFee
>>>>>>>>d.
>> >>>>>>ja
>> >> >>>>va
>> >> >> >>:98)
>> >> >> >> > > > >
>> >> >> >> > > > > at
>> >> >> >> > > > >
>> >> >> >> > > > >
>> >> >> >> > > >
>> >> >> >> > >
>> >> >> >> >
>> >> >> >>
>> >> >>
>> >>
>> 
>>>>>>>>samza.examples.wikipedia.system.WikipediaConsumer.stop(WikipediaCon
>>>>>>>>su
>> >>>>>>me
>> >> >>>>r.
>> >> >> >>java:72)
>> >> >> >> > > > >
>> >> >> >> > > > > at
>> >> >> >> > > > >
>> >> >> >> > > > >
>> >> >> >> > > >
>> >> >> >> > >
>> >> >> >> >
>> >> >> >>
>> >> >>
>> >>
>> 
>>>>>>>>org.apache.samza.system.SystemConsumers$$anonfun$stop$2.apply(Syste
>>>>>>>>mC
>> >>>>>>on
>> >> >>>>su
>> >> >> >>mers.scala:152)
>> >> >> >> > > > >
>> >> >> >> > > > > at
>> >> >> >> > > > >
>> >> >> >> > > > >
>> >> >> >> > > >
>> >> >> >> > >
>> >> >> >> >
>> >> >> >>
>> >> >>
>> >>
>> 
>>>>>>>>org.apache.samza.system.SystemConsumers$$anonfun$stop$2.apply(Syste
>>>>>>>>mC
>> >>>>>>on
>> >> >>>>su
>> >> >> >>mers.scala:152)
>> >> >> >> > > > >
>> >> >> >> > > > > at
>> >> >>scala.collection.Iterator$class.foreach(Iterator.scala:727)
>> >> >> >> > > > >
>> >> >> >> > > > > at
>> >> >> >>scala.collection.AbstractIterator.foreach(Iterator.scala:1157)
>> >> >> >> > > > >
>> >> >> >> > > > > at
>> >> >> >> > > > >
>> >> >> >> > >
>> >> >> >>
>> >> >>
>> >>
>> 
>>>>>>>>scala.collection.MapLike$DefaultValuesIterable.foreach(MapLike.scal
>>>>>>>>a:
>> >>>>>>20
>> >> >>>>6)
>> >> >> >> > > > >
>> >> >> >> > > > > at
>> >> >> >> > > >
>> >> >> >>
>> >>
>> 
>>>>>>org.apache.samza.system.SystemConsumers.stop(SystemConsumers.scala:15
>>>>>>2)
>> >> >> >> > > > >
>> >> >> >> > > > > at
>> >> >> >> > > > >
>> >> >> >> > > > >
>> >> >> >> > > >
>> >> >> >> > >
>> >> >> >> >
>> >> >> >>
>> >> >>
>> >>
>> 
>>>>>>>>org.apache.samza.container.SamzaContainer.shutdownConsumers(SamzaCo
>>>>>>>>nt
>> >>>>>>ai
>> >> >>>>ne
>> >> >> >>r.scala:587)
>> >> >> >> > > > >
>> >> >> >> > > > > at
>> >> >> >> > > >
>> >> >> >>
>> >>
>> 
>>>>>>org.apache.samza.container.SamzaContainer.run(SamzaContainer.scala:51
>>>>>>2)
>> >> >> >> > > > >
>> >> >> >> > > > >  at
>> >> >> >> > >
>> >> 
>>>>org.apache.samza.job.local.ThreadJob$$anon$1.run(ThreadJob.scala:42)
>> >> >> >> > > > >
>> >> >> >> > > > > - Shekar
>> >> >> >> > > > >
>> >> >> >> > > >
>> >> >> >> > >
>> >> >> >> >
>> >> >> >>
>> >> >>
>> >> >>
>> >>
>> >>
>>
>>


Re: Samza as a Caching layer

Posted by Shekar Tippur <ct...@gmail.com>.
More Questions ..

Please correct me if I am wrong. As I am trying to unravel hello-samza,

http://samza.incubator.apache.org/learn/tutorials/0.7.0/run-hello-samza-without-internet.html

1. wikipedia-feed.properties - deploy a Samza job which listens to
wikipedia API, receives the feed in realtime and produces the feed to the
Kafka topic wikipedia-raw
2. wikipedia-parser - This pulls the messages from wikipedia-raw and
registers to an event that checks the integrity of the incoming message.
Why do we need to register this to an event?

WikipediaFeedEvent event = new WikipediaFeedEvent(jsonObject);


- Shekar

On Mon, Sep 8, 2014 at 10:54 AM, Chris Riccomini <
criccomini@linkedin.com.invalid> wrote:

> Hey Shekar,
>
> Sure. If your input stream has 8 partitions and is partitioned by "ip
> address", then your state stream must also have 8 partitions and be
> partitioned by "ip address". This is to guarantee that the StreamTask that
> receives a message from the stream will have the state required to do the
> table join in its local store.
>
> Cheers,
> Chris
>
> On 9/8/14 10:51 AM, "Shekar Tippur" <ct...@gmail.com> wrote:
>
> >Chris -
> >Can you please elaborate on
> >
> >"Also note that if you take this approach, your state
> >must be partitioned in the same way as your input stream."
> >
> >- Shekar
> >
> >On Mon, Sep 8, 2014 at 10:29 AM, Chris Riccomini <
> >criccomini@linkedin.com.invalid> wrote:
> >
> >> Hey Shekar,
> >>
> >> You can either run some external DB that holds the data set, and you can
> >> query it from a StreamTask, or you can use Samza's state store feature
> >>to
> >> push data into a stream that you can then store in a partitioned
> >>key-value
> >> store along with your StreamTasks. There is some documentation here
> >>about
> >> the state store approach:
> >>
> >>
> >>
> http://samza.incubator.apache.org/learn/documentation/0.7.0/container/sta
> >>te
> >> -management.html
> >>
> >> Putting your data into a Kafka stream is going to require more up front
> >> effort from you, since you'll have to understand how Kafka's
> >>partitioning
> >> model works, and setup some pipeline to push the updates for your state.
> >> In the long run, I believe it's the better approach, though. Local
> >>lookups
> >> on a key-value store should be faster than doing remote RPC calls to a
> >>DB
> >> for every message. Also note that if you take this approach, your state
> >> must be partitioned in the same way as your input stream.
> >>
> >> I'm sorry I can't give you a more definitive answer. It's really about
> >> trade-offs.
> >>
> >>
> >> Cheers,
> >> Chris
> >>
> >> On 9/8/14 10:17 AM, "Shekar Tippur" <ct...@gmail.com> wrote:
> >>
> >> >Hello,
> >> >
> >> >I am able to read messages of of a new kafka queue now.
> >> >The next task is to enrich the data with more information. The data
> >>that
> >> >is
> >> >flowing in has ip address or host name. I have a redis cache where
> >>there
> >> >is
> >> >more contextual information (like owner of the alert, SLA, etc). The
> >>data
> >> >in redis does not change often.
> >> >Pretty much becomes a stream table join.
> >> >I can also dump the same data to a different kafka queue and make it a
> >> >stream - stream join as well.
> >> >
> >> >What do you guys recommend?
> >> >
> >> >- Shekar
> >> >
> >> >On Fri, Sep 5, 2014 at 4:08 PM, Chris Riccomini <
> >> >criccomini@linkedin.com.invalid> wrote:
> >> >
> >> >> Hey Guys,
> >> >>
> >> >> I don't know a whole lot about Fluentd, but if you don't want to do
> >>this
> >> >> flow:
> >> >>
> >> >>   Fluentd -> Kafka -> Samza
> >> >>
> >> >> Then the alternative is:
> >> >>
> >> >>   Fluentd -> Samza
> >> >>
> >> >> The "direct" approach (no Kafka) is going to be pretty labor
> >>intensive
> >> >>to
> >> >> build. You'd have to:
> >> >>
> >> >> 1. Implement a FluentdSystemConsumer for Samza.
> >> >> 2. Write a Flutend data output plugin, which sends to the
> >> >> FluentdSystemConsumer.
> >> >> 3. Figure out a way for the Fluentd data output plugin to "discover"
> >> >>where
> >> >> the Samza FluentdSystemConsumer is located (since SamzaContainers are
> >> >> deployed to dynamic hosts in YARN, and move around a lot).
> >> >> 4. Implement a bare-bones FluentdSystemAdmin and FluentdSystemFactory
> >> >> class (similar to the WikipediaSystemFactory in hello-samza).
> >> >> 5. Decide on some partitioning model that makes sense for Fluentd.
> >>Maybe
> >> >> one partition = one host? Not sure how Fluentd works here.
> >> >>
> >> >> My instinct is that it's going to be *far* better to use the first
> >> >> approach (pipe the Fluentd events into Kafka). This will give you
> >>all of
> >> >> the semantics that Kafka provides (e.g. Ordering within a partition,
> >> >> rewinding streams, durability, etc).
> >> >>
> >> >> Cheers,
> >> >> Chris
> >> >>
> >> >> On 9/5/14 1:36 PM, "Yan Fang" <ya...@gmail.com> wrote:
> >> >>
> >> >> >also was thinking of having fluentd push to Samza. But don't know
> >>how
> >> >>to
> >> >> >implement this. Not sure if adding a kafka layer between Samza and
> >> >>fluentd
> >> >> >is the only option.
> >> >> >
> >> >> >Do other guys have better ideas?
> >> >> >
> >> >> >Thanks,
> >> >> >
> >> >> >Fang, Yan
> >> >> >yanfang724@gmail.com
> >> >> >+1 (206) 849-4108
> >> >> >
> >> >> >
> >> >> >On Fri, Sep 5, 2014 at 12:09 PM, Shekar Tippur <ct...@gmail.com>
> >> >>wrote:
> >> >> >
> >> >> >> Yan,
> >> >> >>
> >> >> >> Wont it add additional hop. It did occur to me earlier but was not
> >> >>sure
> >> >> >>if
> >> >> >> this is the right way to go if we have a stringent sla driven
> >>system
> >> >> >> depending on it.
> >> >> >>
> >> >> >> - Shekar
> >> >> >>
> >> >> >>
> >> >> >> On Fri, Sep 5, 2014 at 10:55 AM, Yan Fang <ya...@gmail.com>
> >> >>wrote:
> >> >> >>
> >> >> >> > If you already put the events to the kafka, you can make the
> >>Samza
> >> >> >> accepts
> >> >> >> > the kafka topic, like the wikipedia-parse project in hello-samza
> >> >> >>accepts
> >> >> >> > the kafka topic wikipedia-raw ( see the config file ).
> >> >> >> >
> >> >> >> > Thanks,
> >> >> >> >
> >> >> >> > Fang, Yan
> >> >> >> > yanfang724@gmail.com
> >> >> >> > +1 (206) 849-4108
> >> >> >> >
> >> >> >> >
> >> >> >> > On Fri, Sep 5, 2014 at 8:48 AM, Shekar Tippur
> >><ct...@gmail.com>
> >> >> >>wrote:
> >> >> >> >
> >> >> >> > > Awesome .. This works. Thanks a lot.
> >> >> >> > >
> >> >> >> > > Now off to my next step.
> >> >> >> > > I want to point to an incoming stream of events. These events
> >>are
> >> >> >> routed
> >> >> >> > > via fluentd. So, fluentd acts as a routing layer where it
> >>pushes
> >> >>the
> >> >> >> > events
> >> >> >> > > to kafka. Since it is a push and not a pull, any pointers on
> >>how
> >> >>to
> >> >> >> push
> >> >> >> > it
> >> >> >> > > to samza? Guessing I need a listener on Samza to collect this?
> >> >> >> > >
> >> >> >> > > - Shekar
> >> >> >> > >
> >> >> >> > >
> >> >> >> > > On Fri, Sep 5, 2014 at 1:03 AM, Yan Fang
> >><ya...@gmail.com>
> >> >> >>wrote:
> >> >> >> > >
> >> >> >> > > > Aha, yes, we are almost there. I think I made a mistake in
> >>the
> >> >> >> previous
> >> >> >> > > > email.
> >> >> >> > > >
> >> >> >> > > > 1. modify the *wikipedia-parser.properties ,  NOT
> >> >> >> > > > *wikipedia-feed.properties
> >> >> >> > > > 2. run deploy/samza/bin/run-job.sh
> >> >> >> > > >
> >> >> >> > >
> >> >> >> >
> >> >> >>
> >> >>
> >>
> >>>>>>--config-factory=org.apache.samza.config.factories.PropertiesConfigFa
> >>>>>>ct
> >> >>>>or
> >> >> >>y
> >> >> >> > > >
> >> >> >> > >
> >> >> >> >
> >> >> >>
> >> >>
> >>
> >>>>>>--config-path=file://$PWD/deploy/samza/config/*wikipedia-parser.prope
> >>>>>>rt
> >> >>>>ie
> >> >> >>s*
> >> >> >> > > > *(NOT *wikipedia-feed,properties*)*
> >> >> >> > > >
> >> >> >> > > > Then you should see the messages in the kafka topic,
> >> >> >> *wikipedia-edits*
> >> >> >> > > >
> >> >> >> > > > Thanks. Let me know if you have any luck . :)
> >> >> >> > > >
> >> >> >> > > > Cheers,
> >> >> >> > > >
> >> >> >> > > > Fang, Yan
> >> >> >> > > > yanfang724@gmail.com
> >> >> >> > > > +1 (206) 849-4108
> >> >> >> > > >
> >> >> >> > > >
> >> >> >> > > > On Thu, Sep 4, 2014 at 11:19 PM, Shekar Tippur
> >> >><ctippur@gmail.com
> >> >> >
> >> >> >> > > wrote:
> >> >> >> > > >
> >> >> >> > > > > Just tried #3. Changed the property file
> >> >> >>wikipedia-feed.properties
> >> >> >> > > > >
> >> >> >> > > > >
> >>job.factory.class=org.apache.samza.job.local.LocalJobFactory
> >> >> >> > > > > Ran ..
> >> >> >> > > > >
> >> >> >> > > > > deploy/samza/bin/run-job.sh
> >> >> >> > > > >
> >> >> >> > > >
> >> >> >> > >
> >> >> >> >
> >> >> >>
> >> >>
> >>
> >>>>>>--config-factory=org.apache.samza.config.factories.PropertiesConfigFa
> >>>>>>ct
> >> >>>>or
> >> >> >>y
> >> >> >> > > > >
> >> >> >> > > > >
> >> >> >> > > >
> >> >> >> > >
> >> >> >> >
> >> >> >>
> >> >>
> >>
> >>>>>>--config-path=file:///home/ctippur/hello-samza/deploy/samza/config/wi
> >>>>>>ki
> >> >>>>pe
> >> >> >>dia-feed.properties
> >> >> >> > > > >
> >> >> >> > > > > I dont see any debug messages that I added to the feed or
> >>the
> >> >> >> parser
> >> >> >> > > > file..
> >> >> >> > > > > I see messages on the kafka-consumer ..
> >> >> >> > > > >
> >> >> >> > > > > However the feed job died with the below message
> >> >> >> > > > >
> >> >> >> > > > >
> >> >> >> > > > > Exception in thread "ThreadJob"
> >>java.lang.RuntimeException:
> >> >> >>Trying
> >> >> >> to
> >> >> >> > > > > unlisten to a channel that has no listeners in it.
> >> >> >> > > > >
> >> >> >> > > > > at
> >> >> >> > > > >
> >> >> >> > > > >
> >> >> >> > > >
> >> >> >> > >
> >> >> >> >
> >> >> >>
> >> >>
> >>
> >>>>>>samza.examples.wikipedia.system.WikipediaFeed.unlisten(WikipediaFeed.
> >>>>>>ja
> >> >>>>va
> >> >> >>:98)
> >> >> >> > > > >
> >> >> >> > > > > at
> >> >> >> > > > >
> >> >> >> > > > >
> >> >> >> > > >
> >> >> >> > >
> >> >> >> >
> >> >> >>
> >> >>
> >>
> >>>>>>samza.examples.wikipedia.system.WikipediaConsumer.stop(WikipediaConsu
> >>>>>>me
> >> >>>>r.
> >> >> >>java:72)
> >> >> >> > > > >
> >> >> >> > > > > at
> >> >> >> > > > >
> >> >> >> > > > >
> >> >> >> > > >
> >> >> >> > >
> >> >> >> >
> >> >> >>
> >> >>
> >>
> >>>>>>org.apache.samza.system.SystemConsumers$$anonfun$stop$2.apply(SystemC
> >>>>>>on
> >> >>>>su
> >> >> >>mers.scala:152)
> >> >> >> > > > >
> >> >> >> > > > > at
> >> >> >> > > > >
> >> >> >> > > > >
> >> >> >> > > >
> >> >> >> > >
> >> >> >> >
> >> >> >>
> >> >>
> >>
> >>>>>>org.apache.samza.system.SystemConsumers$$anonfun$stop$2.apply(SystemC
> >>>>>>on
> >> >>>>su
> >> >> >>mers.scala:152)
> >> >> >> > > > >
> >> >> >> > > > > at
> >> >>scala.collection.Iterator$class.foreach(Iterator.scala:727)
> >> >> >> > > > >
> >> >> >> > > > > at
> >> >> >>scala.collection.AbstractIterator.foreach(Iterator.scala:1157)
> >> >> >> > > > >
> >> >> >> > > > > at
> >> >> >> > > > >
> >> >> >> > >
> >> >> >>
> >> >>
> >>
> >>>>>>scala.collection.MapLike$DefaultValuesIterable.foreach(MapLike.scala:
> >>>>>>20
> >> >>>>6)
> >> >> >> > > > >
> >> >> >> > > > > at
> >> >> >> > > >
> >> >> >>
> >>
> >>>>org.apache.samza.system.SystemConsumers.stop(SystemConsumers.scala:152)
> >> >> >> > > > >
> >> >> >> > > > > at
> >> >> >> > > > >
> >> >> >> > > > >
> >> >> >> > > >
> >> >> >> > >
> >> >> >> >
> >> >> >>
> >> >>
> >>
> >>>>>>org.apache.samza.container.SamzaContainer.shutdownConsumers(SamzaCont
> >>>>>>ai
> >> >>>>ne
> >> >> >>r.scala:587)
> >> >> >> > > > >
> >> >> >> > > > > at
> >> >> >> > > >
> >> >> >>
> >>
> >>>>org.apache.samza.container.SamzaContainer.run(SamzaContainer.scala:512)
> >> >> >> > > > >
> >> >> >> > > > >  at
> >> >> >> > >
> >> >>org.apache.samza.job.local.ThreadJob$$anon$1.run(ThreadJob.scala:42)
> >> >> >> > > > >
> >> >> >> > > > > - Shekar
> >> >> >> > > > >
> >> >> >> > > >
> >> >> >> > >
> >> >> >> >
> >> >> >>
> >> >>
> >> >>
> >>
> >>
>
>

Re: Samza as a Caching layer

Posted by Chris Riccomini <cr...@linkedin.com.INVALID>.
Hey Shekar,

Sure. If your input stream has 8 partitions and is partitioned by "ip
address", then your state stream must also have 8 partitions and be
partitioned by "ip address". This is to guarantee that the StreamTask that
receives a message from the stream will have the state required to do the
table join in its local store.

Cheers,
Chris

On 9/8/14 10:51 AM, "Shekar Tippur" <ct...@gmail.com> wrote:

>Chris -
>Can you please elaborate on
>
>"Also note that if you take this approach, your state
>must be partitioned in the same way as your input stream."
>
>- Shekar
>
>On Mon, Sep 8, 2014 at 10:29 AM, Chris Riccomini <
>criccomini@linkedin.com.invalid> wrote:
>
>> Hey Shekar,
>>
>> You can either run some external DB that holds the data set, and you can
>> query it from a StreamTask, or you can use Samza's state store feature
>>to
>> push data into a stream that you can then store in a partitioned
>>key-value
>> store along with your StreamTasks. There is some documentation here
>>about
>> the state store approach:
>>
>> 
>>http://samza.incubator.apache.org/learn/documentation/0.7.0/container/sta
>>te
>> -management.html
>>
>> Putting your data into a Kafka stream is going to require more up front
>> effort from you, since you'll have to understand how Kafka's
>>partitioning
>> model works, and setup some pipeline to push the updates for your state.
>> In the long run, I believe it's the better approach, though. Local
>>lookups
>> on a key-value store should be faster than doing remote RPC calls to a
>>DB
>> for every message. Also note that if you take this approach, your state
>> must be partitioned in the same way as your input stream.
>>
>> I'm sorry I can't give you a more definitive answer. It's really about
>> trade-offs.
>>
>>
>> Cheers,
>> Chris
>>
>> On 9/8/14 10:17 AM, "Shekar Tippur" <ct...@gmail.com> wrote:
>>
>> >Hello,
>> >
>> >I am able to read messages of of a new kafka queue now.
>> >The next task is to enrich the data with more information. The data
>>that
>> >is
>> >flowing in has ip address or host name. I have a redis cache where
>>there
>> >is
>> >more contextual information (like owner of the alert, SLA, etc). The
>>data
>> >in redis does not change often.
>> >Pretty much becomes a stream table join.
>> >I can also dump the same data to a different kafka queue and make it a
>> >stream - stream join as well.
>> >
>> >What do you guys recommend?
>> >
>> >- Shekar
>> >
>> >On Fri, Sep 5, 2014 at 4:08 PM, Chris Riccomini <
>> >criccomini@linkedin.com.invalid> wrote:
>> >
>> >> Hey Guys,
>> >>
>> >> I don't know a whole lot about Fluentd, but if you don't want to do
>>this
>> >> flow:
>> >>
>> >>   Fluentd -> Kafka -> Samza
>> >>
>> >> Then the alternative is:
>> >>
>> >>   Fluentd -> Samza
>> >>
>> >> The "direct" approach (no Kafka) is going to be pretty labor
>>intensive
>> >>to
>> >> build. You'd have to:
>> >>
>> >> 1. Implement a FluentdSystemConsumer for Samza.
>> >> 2. Write a Flutend data output plugin, which sends to the
>> >> FluentdSystemConsumer.
>> >> 3. Figure out a way for the Fluentd data output plugin to "discover"
>> >>where
>> >> the Samza FluentdSystemConsumer is located (since SamzaContainers are
>> >> deployed to dynamic hosts in YARN, and move around a lot).
>> >> 4. Implement a bare-bones FluentdSystemAdmin and FluentdSystemFactory
>> >> class (similar to the WikipediaSystemFactory in hello-samza).
>> >> 5. Decide on some partitioning model that makes sense for Fluentd.
>>Maybe
>> >> one partition = one host? Not sure how Fluentd works here.
>> >>
>> >> My instinct is that it's going to be *far* better to use the first
>> >> approach (pipe the Fluentd events into Kafka). This will give you
>>all of
>> >> the semantics that Kafka provides (e.g. Ordering within a partition,
>> >> rewinding streams, durability, etc).
>> >>
>> >> Cheers,
>> >> Chris
>> >>
>> >> On 9/5/14 1:36 PM, "Yan Fang" <ya...@gmail.com> wrote:
>> >>
>> >> >also was thinking of having fluentd push to Samza. But don't know
>>how
>> >>to
>> >> >implement this. Not sure if adding a kafka layer between Samza and
>> >>fluentd
>> >> >is the only option.
>> >> >
>> >> >Do other guys have better ideas?
>> >> >
>> >> >Thanks,
>> >> >
>> >> >Fang, Yan
>> >> >yanfang724@gmail.com
>> >> >+1 (206) 849-4108
>> >> >
>> >> >
>> >> >On Fri, Sep 5, 2014 at 12:09 PM, Shekar Tippur <ct...@gmail.com>
>> >>wrote:
>> >> >
>> >> >> Yan,
>> >> >>
>> >> >> Wont it add additional hop. It did occur to me earlier but was not
>> >>sure
>> >> >>if
>> >> >> this is the right way to go if we have a stringent sla driven
>>system
>> >> >> depending on it.
>> >> >>
>> >> >> - Shekar
>> >> >>
>> >> >>
>> >> >> On Fri, Sep 5, 2014 at 10:55 AM, Yan Fang <ya...@gmail.com>
>> >>wrote:
>> >> >>
>> >> >> > If you already put the events to the kafka, you can make the
>>Samza
>> >> >> accepts
>> >> >> > the kafka topic, like the wikipedia-parse project in hello-samza
>> >> >>accepts
>> >> >> > the kafka topic wikipedia-raw ( see the config file ).
>> >> >> >
>> >> >> > Thanks,
>> >> >> >
>> >> >> > Fang, Yan
>> >> >> > yanfang724@gmail.com
>> >> >> > +1 (206) 849-4108
>> >> >> >
>> >> >> >
>> >> >> > On Fri, Sep 5, 2014 at 8:48 AM, Shekar Tippur
>><ct...@gmail.com>
>> >> >>wrote:
>> >> >> >
>> >> >> > > Awesome .. This works. Thanks a lot.
>> >> >> > >
>> >> >> > > Now off to my next step.
>> >> >> > > I want to point to an incoming stream of events. These events
>>are
>> >> >> routed
>> >> >> > > via fluentd. So, fluentd acts as a routing layer where it
>>pushes
>> >>the
>> >> >> > events
>> >> >> > > to kafka. Since it is a push and not a pull, any pointers on
>>how
>> >>to
>> >> >> push
>> >> >> > it
>> >> >> > > to samza? Guessing I need a listener on Samza to collect this?
>> >> >> > >
>> >> >> > > - Shekar
>> >> >> > >
>> >> >> > >
>> >> >> > > On Fri, Sep 5, 2014 at 1:03 AM, Yan Fang
>><ya...@gmail.com>
>> >> >>wrote:
>> >> >> > >
>> >> >> > > > Aha, yes, we are almost there. I think I made a mistake in
>>the
>> >> >> previous
>> >> >> > > > email.
>> >> >> > > >
>> >> >> > > > 1. modify the *wikipedia-parser.properties ,  NOT
>> >> >> > > > *wikipedia-feed.properties
>> >> >> > > > 2. run deploy/samza/bin/run-job.sh
>> >> >> > > >
>> >> >> > >
>> >> >> >
>> >> >>
>> >>
>> 
>>>>>>--config-factory=org.apache.samza.config.factories.PropertiesConfigFa
>>>>>>ct
>> >>>>or
>> >> >>y
>> >> >> > > >
>> >> >> > >
>> >> >> >
>> >> >>
>> >>
>> 
>>>>>>--config-path=file://$PWD/deploy/samza/config/*wikipedia-parser.prope
>>>>>>rt
>> >>>>ie
>> >> >>s*
>> >> >> > > > *(NOT *wikipedia-feed,properties*)*
>> >> >> > > >
>> >> >> > > > Then you should see the messages in the kafka topic,
>> >> >> *wikipedia-edits*
>> >> >> > > >
>> >> >> > > > Thanks. Let me know if you have any luck . :)
>> >> >> > > >
>> >> >> > > > Cheers,
>> >> >> > > >
>> >> >> > > > Fang, Yan
>> >> >> > > > yanfang724@gmail.com
>> >> >> > > > +1 (206) 849-4108
>> >> >> > > >
>> >> >> > > >
>> >> >> > > > On Thu, Sep 4, 2014 at 11:19 PM, Shekar Tippur
>> >><ctippur@gmail.com
>> >> >
>> >> >> > > wrote:
>> >> >> > > >
>> >> >> > > > > Just tried #3. Changed the property file
>> >> >>wikipedia-feed.properties
>> >> >> > > > >
>> >> >> > > > > 
>>job.factory.class=org.apache.samza.job.local.LocalJobFactory
>> >> >> > > > > Ran ..
>> >> >> > > > >
>> >> >> > > > > deploy/samza/bin/run-job.sh
>> >> >> > > > >
>> >> >> > > >
>> >> >> > >
>> >> >> >
>> >> >>
>> >>
>> 
>>>>>>--config-factory=org.apache.samza.config.factories.PropertiesConfigFa
>>>>>>ct
>> >>>>or
>> >> >>y
>> >> >> > > > >
>> >> >> > > > >
>> >> >> > > >
>> >> >> > >
>> >> >> >
>> >> >>
>> >>
>> 
>>>>>>--config-path=file:///home/ctippur/hello-samza/deploy/samza/config/wi
>>>>>>ki
>> >>>>pe
>> >> >>dia-feed.properties
>> >> >> > > > >
>> >> >> > > > > I dont see any debug messages that I added to the feed or
>>the
>> >> >> parser
>> >> >> > > > file..
>> >> >> > > > > I see messages on the kafka-consumer ..
>> >> >> > > > >
>> >> >> > > > > However the feed job died with the below message
>> >> >> > > > >
>> >> >> > > > >
>> >> >> > > > > Exception in thread "ThreadJob"
>>java.lang.RuntimeException:
>> >> >>Trying
>> >> >> to
>> >> >> > > > > unlisten to a channel that has no listeners in it.
>> >> >> > > > >
>> >> >> > > > > at
>> >> >> > > > >
>> >> >> > > > >
>> >> >> > > >
>> >> >> > >
>> >> >> >
>> >> >>
>> >>
>> 
>>>>>>samza.examples.wikipedia.system.WikipediaFeed.unlisten(WikipediaFeed.
>>>>>>ja
>> >>>>va
>> >> >>:98)
>> >> >> > > > >
>> >> >> > > > > at
>> >> >> > > > >
>> >> >> > > > >
>> >> >> > > >
>> >> >> > >
>> >> >> >
>> >> >>
>> >>
>> 
>>>>>>samza.examples.wikipedia.system.WikipediaConsumer.stop(WikipediaConsu
>>>>>>me
>> >>>>r.
>> >> >>java:72)
>> >> >> > > > >
>> >> >> > > > > at
>> >> >> > > > >
>> >> >> > > > >
>> >> >> > > >
>> >> >> > >
>> >> >> >
>> >> >>
>> >>
>> 
>>>>>>org.apache.samza.system.SystemConsumers$$anonfun$stop$2.apply(SystemC
>>>>>>on
>> >>>>su
>> >> >>mers.scala:152)
>> >> >> > > > >
>> >> >> > > > > at
>> >> >> > > > >
>> >> >> > > > >
>> >> >> > > >
>> >> >> > >
>> >> >> >
>> >> >>
>> >>
>> 
>>>>>>org.apache.samza.system.SystemConsumers$$anonfun$stop$2.apply(SystemC
>>>>>>on
>> >>>>su
>> >> >>mers.scala:152)
>> >> >> > > > >
>> >> >> > > > > at
>> >>scala.collection.Iterator$class.foreach(Iterator.scala:727)
>> >> >> > > > >
>> >> >> > > > > at
>> >> >>scala.collection.AbstractIterator.foreach(Iterator.scala:1157)
>> >> >> > > > >
>> >> >> > > > > at
>> >> >> > > > >
>> >> >> > >
>> >> >>
>> >>
>> 
>>>>>>scala.collection.MapLike$DefaultValuesIterable.foreach(MapLike.scala:
>>>>>>20
>> >>>>6)
>> >> >> > > > >
>> >> >> > > > > at
>> >> >> > > >
>> >> >>
>> 
>>>>org.apache.samza.system.SystemConsumers.stop(SystemConsumers.scala:152)
>> >> >> > > > >
>> >> >> > > > > at
>> >> >> > > > >
>> >> >> > > > >
>> >> >> > > >
>> >> >> > >
>> >> >> >
>> >> >>
>> >>
>> 
>>>>>>org.apache.samza.container.SamzaContainer.shutdownConsumers(SamzaCont
>>>>>>ai
>> >>>>ne
>> >> >>r.scala:587)
>> >> >> > > > >
>> >> >> > > > > at
>> >> >> > > >
>> >> >>
>> 
>>>>org.apache.samza.container.SamzaContainer.run(SamzaContainer.scala:512)
>> >> >> > > > >
>> >> >> > > > >  at
>> >> >> > >
>> >>org.apache.samza.job.local.ThreadJob$$anon$1.run(ThreadJob.scala:42)
>> >> >> > > > >
>> >> >> > > > > - Shekar
>> >> >> > > > >
>> >> >> > > >
>> >> >> > >
>> >> >> >
>> >> >>
>> >>
>> >>
>>
>>


Re: Samza as a Caching layer

Posted by Shekar Tippur <ct...@gmail.com>.
Chris -
Can you please elaborate on

"Also note that if you take this approach, your state
must be partitioned in the same way as your input stream."

- Shekar

On Mon, Sep 8, 2014 at 10:29 AM, Chris Riccomini <
criccomini@linkedin.com.invalid> wrote:

> Hey Shekar,
>
> You can either run some external DB that holds the data set, and you can
> query it from a StreamTask, or you can use Samza's state store feature to
> push data into a stream that you can then store in a partitioned key-value
> store along with your StreamTasks. There is some documentation here about
> the state store approach:
>
> http://samza.incubator.apache.org/learn/documentation/0.7.0/container/state
> -management.html
>
> Putting your data into a Kafka stream is going to require more up front
> effort from you, since you'll have to understand how Kafka's partitioning
> model works, and setup some pipeline to push the updates for your state.
> In the long run, I believe it's the better approach, though. Local lookups
> on a key-value store should be faster than doing remote RPC calls to a DB
> for every message. Also note that if you take this approach, your state
> must be partitioned in the same way as your input stream.
>
> I'm sorry I can't give you a more definitive answer. It's really about
> trade-offs.
>
>
> Cheers,
> Chris
>
> On 9/8/14 10:17 AM, "Shekar Tippur" <ct...@gmail.com> wrote:
>
> >Hello,
> >
> >I am able to read messages of of a new kafka queue now.
> >The next task is to enrich the data with more information. The data that
> >is
> >flowing in has ip address or host name. I have a redis cache where there
> >is
> >more contextual information (like owner of the alert, SLA, etc). The data
> >in redis does not change often.
> >Pretty much becomes a stream table join.
> >I can also dump the same data to a different kafka queue and make it a
> >stream - stream join as well.
> >
> >What do you guys recommend?
> >
> >- Shekar
> >
> >On Fri, Sep 5, 2014 at 4:08 PM, Chris Riccomini <
> >criccomini@linkedin.com.invalid> wrote:
> >
> >> Hey Guys,
> >>
> >> I don't know a whole lot about Fluentd, but if you don't want to do this
> >> flow:
> >>
> >>   Fluentd -> Kafka -> Samza
> >>
> >> Then the alternative is:
> >>
> >>   Fluentd -> Samza
> >>
> >> The "direct" approach (no Kafka) is going to be pretty labor intensive
> >>to
> >> build. You'd have to:
> >>
> >> 1. Implement a FluentdSystemConsumer for Samza.
> >> 2. Write a Flutend data output plugin, which sends to the
> >> FluentdSystemConsumer.
> >> 3. Figure out a way for the Fluentd data output plugin to "discover"
> >>where
> >> the Samza FluentdSystemConsumer is located (since SamzaContainers are
> >> deployed to dynamic hosts in YARN, and move around a lot).
> >> 4. Implement a bare-bones FluentdSystemAdmin and FluentdSystemFactory
> >> class (similar to the WikipediaSystemFactory in hello-samza).
> >> 5. Decide on some partitioning model that makes sense for Fluentd. Maybe
> >> one partition = one host? Not sure how Fluentd works here.
> >>
> >> My instinct is that it's going to be *far* better to use the first
> >> approach (pipe the Fluentd events into Kafka). This will give you all of
> >> the semantics that Kafka provides (e.g. Ordering within a partition,
> >> rewinding streams, durability, etc).
> >>
> >> Cheers,
> >> Chris
> >>
> >> On 9/5/14 1:36 PM, "Yan Fang" <ya...@gmail.com> wrote:
> >>
> >> >also was thinking of having fluentd push to Samza. But don't know how
> >>to
> >> >implement this. Not sure if adding a kafka layer between Samza and
> >>fluentd
> >> >is the only option.
> >> >
> >> >Do other guys have better ideas?
> >> >
> >> >Thanks,
> >> >
> >> >Fang, Yan
> >> >yanfang724@gmail.com
> >> >+1 (206) 849-4108
> >> >
> >> >
> >> >On Fri, Sep 5, 2014 at 12:09 PM, Shekar Tippur <ct...@gmail.com>
> >>wrote:
> >> >
> >> >> Yan,
> >> >>
> >> >> Wont it add additional hop. It did occur to me earlier but was not
> >>sure
> >> >>if
> >> >> this is the right way to go if we have a stringent sla driven system
> >> >> depending on it.
> >> >>
> >> >> - Shekar
> >> >>
> >> >>
> >> >> On Fri, Sep 5, 2014 at 10:55 AM, Yan Fang <ya...@gmail.com>
> >>wrote:
> >> >>
> >> >> > If you already put the events to the kafka, you can make the Samza
> >> >> accepts
> >> >> > the kafka topic, like the wikipedia-parse project in hello-samza
> >> >>accepts
> >> >> > the kafka topic wikipedia-raw ( see the config file ).
> >> >> >
> >> >> > Thanks,
> >> >> >
> >> >> > Fang, Yan
> >> >> > yanfang724@gmail.com
> >> >> > +1 (206) 849-4108
> >> >> >
> >> >> >
> >> >> > On Fri, Sep 5, 2014 at 8:48 AM, Shekar Tippur <ct...@gmail.com>
> >> >>wrote:
> >> >> >
> >> >> > > Awesome .. This works. Thanks a lot.
> >> >> > >
> >> >> > > Now off to my next step.
> >> >> > > I want to point to an incoming stream of events. These events are
> >> >> routed
> >> >> > > via fluentd. So, fluentd acts as a routing layer where it pushes
> >>the
> >> >> > events
> >> >> > > to kafka. Since it is a push and not a pull, any pointers on how
> >>to
> >> >> push
> >> >> > it
> >> >> > > to samza? Guessing I need a listener on Samza to collect this?
> >> >> > >
> >> >> > > - Shekar
> >> >> > >
> >> >> > >
> >> >> > > On Fri, Sep 5, 2014 at 1:03 AM, Yan Fang <ya...@gmail.com>
> >> >>wrote:
> >> >> > >
> >> >> > > > Aha, yes, we are almost there. I think I made a mistake in the
> >> >> previous
> >> >> > > > email.
> >> >> > > >
> >> >> > > > 1. modify the *wikipedia-parser.properties ,  NOT
> >> >> > > > *wikipedia-feed.properties
> >> >> > > > 2. run deploy/samza/bin/run-job.sh
> >> >> > > >
> >> >> > >
> >> >> >
> >> >>
> >>
> >>>>--config-factory=org.apache.samza.config.factories.PropertiesConfigFact
> >>>>or
> >> >>y
> >> >> > > >
> >> >> > >
> >> >> >
> >> >>
> >>
> >>>>--config-path=file://$PWD/deploy/samza/config/*wikipedia-parser.propert
> >>>>ie
> >> >>s*
> >> >> > > > *(NOT *wikipedia-feed,properties*)*
> >> >> > > >
> >> >> > > > Then you should see the messages in the kafka topic,
> >> >> *wikipedia-edits*
> >> >> > > >
> >> >> > > > Thanks. Let me know if you have any luck . :)
> >> >> > > >
> >> >> > > > Cheers,
> >> >> > > >
> >> >> > > > Fang, Yan
> >> >> > > > yanfang724@gmail.com
> >> >> > > > +1 (206) 849-4108
> >> >> > > >
> >> >> > > >
> >> >> > > > On Thu, Sep 4, 2014 at 11:19 PM, Shekar Tippur
> >><ctippur@gmail.com
> >> >
> >> >> > > wrote:
> >> >> > > >
> >> >> > > > > Just tried #3. Changed the property file
> >> >>wikipedia-feed.properties
> >> >> > > > >
> >> >> > > > > job.factory.class=org.apache.samza.job.local.LocalJobFactory
> >> >> > > > > Ran ..
> >> >> > > > >
> >> >> > > > > deploy/samza/bin/run-job.sh
> >> >> > > > >
> >> >> > > >
> >> >> > >
> >> >> >
> >> >>
> >>
> >>>>--config-factory=org.apache.samza.config.factories.PropertiesConfigFact
> >>>>or
> >> >>y
> >> >> > > > >
> >> >> > > > >
> >> >> > > >
> >> >> > >
> >> >> >
> >> >>
> >>
> >>>>--config-path=file:///home/ctippur/hello-samza/deploy/samza/config/wiki
> >>>>pe
> >> >>dia-feed.properties
> >> >> > > > >
> >> >> > > > > I dont see any debug messages that I added to the feed or the
> >> >> parser
> >> >> > > > file..
> >> >> > > > > I see messages on the kafka-consumer ..
> >> >> > > > >
> >> >> > > > > However the feed job died with the below message
> >> >> > > > >
> >> >> > > > >
> >> >> > > > > Exception in thread "ThreadJob" java.lang.RuntimeException:
> >> >>Trying
> >> >> to
> >> >> > > > > unlisten to a channel that has no listeners in it.
> >> >> > > > >
> >> >> > > > > at
> >> >> > > > >
> >> >> > > > >
> >> >> > > >
> >> >> > >
> >> >> >
> >> >>
> >>
> >>>>samza.examples.wikipedia.system.WikipediaFeed.unlisten(WikipediaFeed.ja
> >>>>va
> >> >>:98)
> >> >> > > > >
> >> >> > > > > at
> >> >> > > > >
> >> >> > > > >
> >> >> > > >
> >> >> > >
> >> >> >
> >> >>
> >>
> >>>>samza.examples.wikipedia.system.WikipediaConsumer.stop(WikipediaConsume
> >>>>r.
> >> >>java:72)
> >> >> > > > >
> >> >> > > > > at
> >> >> > > > >
> >> >> > > > >
> >> >> > > >
> >> >> > >
> >> >> >
> >> >>
> >>
> >>>>org.apache.samza.system.SystemConsumers$$anonfun$stop$2.apply(SystemCon
> >>>>su
> >> >>mers.scala:152)
> >> >> > > > >
> >> >> > > > > at
> >> >> > > > >
> >> >> > > > >
> >> >> > > >
> >> >> > >
> >> >> >
> >> >>
> >>
> >>>>org.apache.samza.system.SystemConsumers$$anonfun$stop$2.apply(SystemCon
> >>>>su
> >> >>mers.scala:152)
> >> >> > > > >
> >> >> > > > > at
> >>scala.collection.Iterator$class.foreach(Iterator.scala:727)
> >> >> > > > >
> >> >> > > > > at
> >> >>scala.collection.AbstractIterator.foreach(Iterator.scala:1157)
> >> >> > > > >
> >> >> > > > > at
> >> >> > > > >
> >> >> > >
> >> >>
> >>
> >>>>scala.collection.MapLike$DefaultValuesIterable.foreach(MapLike.scala:20
> >>>>6)
> >> >> > > > >
> >> >> > > > > at
> >> >> > > >
> >> >>
> >>org.apache.samza.system.SystemConsumers.stop(SystemConsumers.scala:152)
> >> >> > > > >
> >> >> > > > > at
> >> >> > > > >
> >> >> > > > >
> >> >> > > >
> >> >> > >
> >> >> >
> >> >>
> >>
> >>>>org.apache.samza.container.SamzaContainer.shutdownConsumers(SamzaContai
> >>>>ne
> >> >>r.scala:587)
> >> >> > > > >
> >> >> > > > > at
> >> >> > > >
> >> >>
> >>org.apache.samza.container.SamzaContainer.run(SamzaContainer.scala:512)
> >> >> > > > >
> >> >> > > > >  at
> >> >> > >
> >>org.apache.samza.job.local.ThreadJob$$anon$1.run(ThreadJob.scala:42)
> >> >> > > > >
> >> >> > > > > - Shekar
> >> >> > > > >
> >> >> > > >
> >> >> > >
> >> >> >
> >> >>
> >>
> >>
>
>

Re: Samza as a Caching layer

Posted by Chris Riccomini <cr...@linkedin.com.INVALID>.
Hey Shekar,

You can either run some external DB that holds the data set, and you can
query it from a StreamTask, or you can use Samza's state store feature to
push data into a stream that you can then store in a partitioned key-value
store along with your StreamTasks. There is some documentation here about
the state store approach:

http://samza.incubator.apache.org/learn/documentation/0.7.0/container/state
-management.html

Putting your data into a Kafka stream is going to require more up front
effort from you, since you'll have to understand how Kafka's partitioning
model works, and setup some pipeline to push the updates for your state.
In the long run, I believe it's the better approach, though. Local lookups
on a key-value store should be faster than doing remote RPC calls to a DB
for every message. Also note that if you take this approach, your state
must be partitioned in the same way as your input stream.

I'm sorry I can't give you a more definitive answer. It's really about
trade-offs.


Cheers,
Chris

On 9/8/14 10:17 AM, "Shekar Tippur" <ct...@gmail.com> wrote:

>Hello,
>
>I am able to read messages of of a new kafka queue now.
>The next task is to enrich the data with more information. The data that
>is
>flowing in has ip address or host name. I have a redis cache where there
>is
>more contextual information (like owner of the alert, SLA, etc). The data
>in redis does not change often.
>Pretty much becomes a stream table join.
>I can also dump the same data to a different kafka queue and make it a
>stream - stream join as well.
>
>What do you guys recommend?
>
>- Shekar
>
>On Fri, Sep 5, 2014 at 4:08 PM, Chris Riccomini <
>criccomini@linkedin.com.invalid> wrote:
>
>> Hey Guys,
>>
>> I don't know a whole lot about Fluentd, but if you don't want to do this
>> flow:
>>
>>   Fluentd -> Kafka -> Samza
>>
>> Then the alternative is:
>>
>>   Fluentd -> Samza
>>
>> The "direct" approach (no Kafka) is going to be pretty labor intensive
>>to
>> build. You'd have to:
>>
>> 1. Implement a FluentdSystemConsumer for Samza.
>> 2. Write a Flutend data output plugin, which sends to the
>> FluentdSystemConsumer.
>> 3. Figure out a way for the Fluentd data output plugin to "discover"
>>where
>> the Samza FluentdSystemConsumer is located (since SamzaContainers are
>> deployed to dynamic hosts in YARN, and move around a lot).
>> 4. Implement a bare-bones FluentdSystemAdmin and FluentdSystemFactory
>> class (similar to the WikipediaSystemFactory in hello-samza).
>> 5. Decide on some partitioning model that makes sense for Fluentd. Maybe
>> one partition = one host? Not sure how Fluentd works here.
>>
>> My instinct is that it's going to be *far* better to use the first
>> approach (pipe the Fluentd events into Kafka). This will give you all of
>> the semantics that Kafka provides (e.g. Ordering within a partition,
>> rewinding streams, durability, etc).
>>
>> Cheers,
>> Chris
>>
>> On 9/5/14 1:36 PM, "Yan Fang" <ya...@gmail.com> wrote:
>>
>> >also was thinking of having fluentd push to Samza. But don't know how
>>to
>> >implement this. Not sure if adding a kafka layer between Samza and
>>fluentd
>> >is the only option.
>> >
>> >Do other guys have better ideas?
>> >
>> >Thanks,
>> >
>> >Fang, Yan
>> >yanfang724@gmail.com
>> >+1 (206) 849-4108
>> >
>> >
>> >On Fri, Sep 5, 2014 at 12:09 PM, Shekar Tippur <ct...@gmail.com>
>>wrote:
>> >
>> >> Yan,
>> >>
>> >> Wont it add additional hop. It did occur to me earlier but was not
>>sure
>> >>if
>> >> this is the right way to go if we have a stringent sla driven system
>> >> depending on it.
>> >>
>> >> - Shekar
>> >>
>> >>
>> >> On Fri, Sep 5, 2014 at 10:55 AM, Yan Fang <ya...@gmail.com>
>>wrote:
>> >>
>> >> > If you already put the events to the kafka, you can make the Samza
>> >> accepts
>> >> > the kafka topic, like the wikipedia-parse project in hello-samza
>> >>accepts
>> >> > the kafka topic wikipedia-raw ( see the config file ).
>> >> >
>> >> > Thanks,
>> >> >
>> >> > Fang, Yan
>> >> > yanfang724@gmail.com
>> >> > +1 (206) 849-4108
>> >> >
>> >> >
>> >> > On Fri, Sep 5, 2014 at 8:48 AM, Shekar Tippur <ct...@gmail.com>
>> >>wrote:
>> >> >
>> >> > > Awesome .. This works. Thanks a lot.
>> >> > >
>> >> > > Now off to my next step.
>> >> > > I want to point to an incoming stream of events. These events are
>> >> routed
>> >> > > via fluentd. So, fluentd acts as a routing layer where it pushes
>>the
>> >> > events
>> >> > > to kafka. Since it is a push and not a pull, any pointers on how
>>to
>> >> push
>> >> > it
>> >> > > to samza? Guessing I need a listener on Samza to collect this?
>> >> > >
>> >> > > - Shekar
>> >> > >
>> >> > >
>> >> > > On Fri, Sep 5, 2014 at 1:03 AM, Yan Fang <ya...@gmail.com>
>> >>wrote:
>> >> > >
>> >> > > > Aha, yes, we are almost there. I think I made a mistake in the
>> >> previous
>> >> > > > email.
>> >> > > >
>> >> > > > 1. modify the *wikipedia-parser.properties ,  NOT
>> >> > > > *wikipedia-feed.properties
>> >> > > > 2. run deploy/samza/bin/run-job.sh
>> >> > > >
>> >> > >
>> >> >
>> >>
>> 
>>>>--config-factory=org.apache.samza.config.factories.PropertiesConfigFact
>>>>or
>> >>y
>> >> > > >
>> >> > >
>> >> >
>> >>
>> 
>>>>--config-path=file://$PWD/deploy/samza/config/*wikipedia-parser.propert
>>>>ie
>> >>s*
>> >> > > > *(NOT *wikipedia-feed,properties*)*
>> >> > > >
>> >> > > > Then you should see the messages in the kafka topic,
>> >> *wikipedia-edits*
>> >> > > >
>> >> > > > Thanks. Let me know if you have any luck . :)
>> >> > > >
>> >> > > > Cheers,
>> >> > > >
>> >> > > > Fang, Yan
>> >> > > > yanfang724@gmail.com
>> >> > > > +1 (206) 849-4108
>> >> > > >
>> >> > > >
>> >> > > > On Thu, Sep 4, 2014 at 11:19 PM, Shekar Tippur
>><ctippur@gmail.com
>> >
>> >> > > wrote:
>> >> > > >
>> >> > > > > Just tried #3. Changed the property file
>> >>wikipedia-feed.properties
>> >> > > > >
>> >> > > > > job.factory.class=org.apache.samza.job.local.LocalJobFactory
>> >> > > > > Ran ..
>> >> > > > >
>> >> > > > > deploy/samza/bin/run-job.sh
>> >> > > > >
>> >> > > >
>> >> > >
>> >> >
>> >>
>> 
>>>>--config-factory=org.apache.samza.config.factories.PropertiesConfigFact
>>>>or
>> >>y
>> >> > > > >
>> >> > > > >
>> >> > > >
>> >> > >
>> >> >
>> >>
>> 
>>>>--config-path=file:///home/ctippur/hello-samza/deploy/samza/config/wiki
>>>>pe
>> >>dia-feed.properties
>> >> > > > >
>> >> > > > > I dont see any debug messages that I added to the feed or the
>> >> parser
>> >> > > > file..
>> >> > > > > I see messages on the kafka-consumer ..
>> >> > > > >
>> >> > > > > However the feed job died with the below message
>> >> > > > >
>> >> > > > >
>> >> > > > > Exception in thread "ThreadJob" java.lang.RuntimeException:
>> >>Trying
>> >> to
>> >> > > > > unlisten to a channel that has no listeners in it.
>> >> > > > >
>> >> > > > > at
>> >> > > > >
>> >> > > > >
>> >> > > >
>> >> > >
>> >> >
>> >>
>> 
>>>>samza.examples.wikipedia.system.WikipediaFeed.unlisten(WikipediaFeed.ja
>>>>va
>> >>:98)
>> >> > > > >
>> >> > > > > at
>> >> > > > >
>> >> > > > >
>> >> > > >
>> >> > >
>> >> >
>> >>
>> 
>>>>samza.examples.wikipedia.system.WikipediaConsumer.stop(WikipediaConsume
>>>>r.
>> >>java:72)
>> >> > > > >
>> >> > > > > at
>> >> > > > >
>> >> > > > >
>> >> > > >
>> >> > >
>> >> >
>> >>
>> 
>>>>org.apache.samza.system.SystemConsumers$$anonfun$stop$2.apply(SystemCon
>>>>su
>> >>mers.scala:152)
>> >> > > > >
>> >> > > > > at
>> >> > > > >
>> >> > > > >
>> >> > > >
>> >> > >
>> >> >
>> >>
>> 
>>>>org.apache.samza.system.SystemConsumers$$anonfun$stop$2.apply(SystemCon
>>>>su
>> >>mers.scala:152)
>> >> > > > >
>> >> > > > > at 
>>scala.collection.Iterator$class.foreach(Iterator.scala:727)
>> >> > > > >
>> >> > > > > at
>> >>scala.collection.AbstractIterator.foreach(Iterator.scala:1157)
>> >> > > > >
>> >> > > > > at
>> >> > > > >
>> >> > >
>> >>
>> 
>>>>scala.collection.MapLike$DefaultValuesIterable.foreach(MapLike.scala:20
>>>>6)
>> >> > > > >
>> >> > > > > at
>> >> > > >
>> >> 
>>org.apache.samza.system.SystemConsumers.stop(SystemConsumers.scala:152)
>> >> > > > >
>> >> > > > > at
>> >> > > > >
>> >> > > > >
>> >> > > >
>> >> > >
>> >> >
>> >>
>> 
>>>>org.apache.samza.container.SamzaContainer.shutdownConsumers(SamzaContai
>>>>ne
>> >>r.scala:587)
>> >> > > > >
>> >> > > > > at
>> >> > > >
>> >> 
>>org.apache.samza.container.SamzaContainer.run(SamzaContainer.scala:512)
>> >> > > > >
>> >> > > > >  at
>> >> > > 
>>org.apache.samza.job.local.ThreadJob$$anon$1.run(ThreadJob.scala:42)
>> >> > > > >
>> >> > > > > - Shekar
>> >> > > > >
>> >> > > >
>> >> > >
>> >> >
>> >>
>>
>>


Re: Samza as a Caching layer

Posted by Shekar Tippur <ct...@gmail.com>.
Hello,

I am able to read messages of of a new kafka queue now.
The next task is to enrich the data with more information. The data that is
flowing in has ip address or host name. I have a redis cache where there is
more contextual information (like owner of the alert, SLA, etc). The data
in redis does not change often.
Pretty much becomes a stream table join.
I can also dump the same data to a different kafka queue and make it a
stream - stream join as well.

What do you guys recommend?

- Shekar

On Fri, Sep 5, 2014 at 4:08 PM, Chris Riccomini <
criccomini@linkedin.com.invalid> wrote:

> Hey Guys,
>
> I don't know a whole lot about Fluentd, but if you don't want to do this
> flow:
>
>   Fluentd -> Kafka -> Samza
>
> Then the alternative is:
>
>   Fluentd -> Samza
>
> The "direct" approach (no Kafka) is going to be pretty labor intensive to
> build. You'd have to:
>
> 1. Implement a FluentdSystemConsumer for Samza.
> 2. Write a Flutend data output plugin, which sends to the
> FluentdSystemConsumer.
> 3. Figure out a way for the Fluentd data output plugin to "discover" where
> the Samza FluentdSystemConsumer is located (since SamzaContainers are
> deployed to dynamic hosts in YARN, and move around a lot).
> 4. Implement a bare-bones FluentdSystemAdmin and FluentdSystemFactory
> class (similar to the WikipediaSystemFactory in hello-samza).
> 5. Decide on some partitioning model that makes sense for Fluentd. Maybe
> one partition = one host? Not sure how Fluentd works here.
>
> My instinct is that it's going to be *far* better to use the first
> approach (pipe the Fluentd events into Kafka). This will give you all of
> the semantics that Kafka provides (e.g. Ordering within a partition,
> rewinding streams, durability, etc).
>
> Cheers,
> Chris
>
> On 9/5/14 1:36 PM, "Yan Fang" <ya...@gmail.com> wrote:
>
> >also was thinking of having fluentd push to Samza. But don't know how to
> >implement this. Not sure if adding a kafka layer between Samza and fluentd
> >is the only option.
> >
> >Do other guys have better ideas?
> >
> >Thanks,
> >
> >Fang, Yan
> >yanfang724@gmail.com
> >+1 (206) 849-4108
> >
> >
> >On Fri, Sep 5, 2014 at 12:09 PM, Shekar Tippur <ct...@gmail.com> wrote:
> >
> >> Yan,
> >>
> >> Wont it add additional hop. It did occur to me earlier but was not sure
> >>if
> >> this is the right way to go if we have a stringent sla driven system
> >> depending on it.
> >>
> >> - Shekar
> >>
> >>
> >> On Fri, Sep 5, 2014 at 10:55 AM, Yan Fang <ya...@gmail.com> wrote:
> >>
> >> > If you already put the events to the kafka, you can make the Samza
> >> accepts
> >> > the kafka topic, like the wikipedia-parse project in hello-samza
> >>accepts
> >> > the kafka topic wikipedia-raw ( see the config file ).
> >> >
> >> > Thanks,
> >> >
> >> > Fang, Yan
> >> > yanfang724@gmail.com
> >> > +1 (206) 849-4108
> >> >
> >> >
> >> > On Fri, Sep 5, 2014 at 8:48 AM, Shekar Tippur <ct...@gmail.com>
> >>wrote:
> >> >
> >> > > Awesome .. This works. Thanks a lot.
> >> > >
> >> > > Now off to my next step.
> >> > > I want to point to an incoming stream of events. These events are
> >> routed
> >> > > via fluentd. So, fluentd acts as a routing layer where it pushes the
> >> > events
> >> > > to kafka. Since it is a push and not a pull, any pointers on how to
> >> push
> >> > it
> >> > > to samza? Guessing I need a listener on Samza to collect this?
> >> > >
> >> > > - Shekar
> >> > >
> >> > >
> >> > > On Fri, Sep 5, 2014 at 1:03 AM, Yan Fang <ya...@gmail.com>
> >>wrote:
> >> > >
> >> > > > Aha, yes, we are almost there. I think I made a mistake in the
> >> previous
> >> > > > email.
> >> > > >
> >> > > > 1. modify the *wikipedia-parser.properties ,  NOT
> >> > > > *wikipedia-feed.properties
> >> > > > 2. run deploy/samza/bin/run-job.sh
> >> > > >
> >> > >
> >> >
> >>
> >>--config-factory=org.apache.samza.config.factories.PropertiesConfigFactor
> >>y
> >> > > >
> >> > >
> >> >
> >>
> >>--config-path=file://$PWD/deploy/samza/config/*wikipedia-parser.propertie
> >>s*
> >> > > > *(NOT *wikipedia-feed,properties*)*
> >> > > >
> >> > > > Then you should see the messages in the kafka topic,
> >> *wikipedia-edits*
> >> > > >
> >> > > > Thanks. Let me know if you have any luck . :)
> >> > > >
> >> > > > Cheers,
> >> > > >
> >> > > > Fang, Yan
> >> > > > yanfang724@gmail.com
> >> > > > +1 (206) 849-4108
> >> > > >
> >> > > >
> >> > > > On Thu, Sep 4, 2014 at 11:19 PM, Shekar Tippur <ctippur@gmail.com
> >
> >> > > wrote:
> >> > > >
> >> > > > > Just tried #3. Changed the property file
> >>wikipedia-feed.properties
> >> > > > >
> >> > > > > job.factory.class=org.apache.samza.job.local.LocalJobFactory
> >> > > > > Ran ..
> >> > > > >
> >> > > > > deploy/samza/bin/run-job.sh
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> >>--config-factory=org.apache.samza.config.factories.PropertiesConfigFactor
> >>y
> >> > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> >>--config-path=file:///home/ctippur/hello-samza/deploy/samza/config/wikipe
> >>dia-feed.properties
> >> > > > >
> >> > > > > I dont see any debug messages that I added to the feed or the
> >> parser
> >> > > > file..
> >> > > > > I see messages on the kafka-consumer ..
> >> > > > >
> >> > > > > However the feed job died with the below message
> >> > > > >
> >> > > > >
> >> > > > > Exception in thread "ThreadJob" java.lang.RuntimeException:
> >>Trying
> >> to
> >> > > > > unlisten to a channel that has no listeners in it.
> >> > > > >
> >> > > > > at
> >> > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> >>samza.examples.wikipedia.system.WikipediaFeed.unlisten(WikipediaFeed.java
> >>:98)
> >> > > > >
> >> > > > > at
> >> > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> >>samza.examples.wikipedia.system.WikipediaConsumer.stop(WikipediaConsumer.
> >>java:72)
> >> > > > >
> >> > > > > at
> >> > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> >>org.apache.samza.system.SystemConsumers$$anonfun$stop$2.apply(SystemConsu
> >>mers.scala:152)
> >> > > > >
> >> > > > > at
> >> > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> >>org.apache.samza.system.SystemConsumers$$anonfun$stop$2.apply(SystemConsu
> >>mers.scala:152)
> >> > > > >
> >> > > > > at scala.collection.Iterator$class.foreach(Iterator.scala:727)
> >> > > > >
> >> > > > > at
> >>scala.collection.AbstractIterator.foreach(Iterator.scala:1157)
> >> > > > >
> >> > > > > at
> >> > > > >
> >> > >
> >>
> >>scala.collection.MapLike$DefaultValuesIterable.foreach(MapLike.scala:206)
> >> > > > >
> >> > > > > at
> >> > > >
> >> org.apache.samza.system.SystemConsumers.stop(SystemConsumers.scala:152)
> >> > > > >
> >> > > > > at
> >> > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> >>org.apache.samza.container.SamzaContainer.shutdownConsumers(SamzaContaine
> >>r.scala:587)
> >> > > > >
> >> > > > > at
> >> > > >
> >> org.apache.samza.container.SamzaContainer.run(SamzaContainer.scala:512)
> >> > > > >
> >> > > > >  at
> >> > > org.apache.samza.job.local.ThreadJob$$anon$1.run(ThreadJob.scala:42)
> >> > > > >
> >> > > > > - Shekar
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
>
>

Re: Samza as a Caching layer

Posted by Chris Riccomini <cr...@linkedin.com.INVALID>.
Hey Guys,

I don't know a whole lot about Fluentd, but if you don't want to do this
flow:

  Fluentd -> Kafka -> Samza

Then the alternative is:

  Fluentd -> Samza

The "direct" approach (no Kafka) is going to be pretty labor intensive to
build. You'd have to:

1. Implement a FluentdSystemConsumer for Samza.
2. Write a Flutend data output plugin, which sends to the
FluentdSystemConsumer.
3. Figure out a way for the Fluentd data output plugin to "discover" where
the Samza FluentdSystemConsumer is located (since SamzaContainers are
deployed to dynamic hosts in YARN, and move around a lot).
4. Implement a bare-bones FluentdSystemAdmin and FluentdSystemFactory
class (similar to the WikipediaSystemFactory in hello-samza).
5. Decide on some partitioning model that makes sense for Fluentd. Maybe
one partition = one host? Not sure how Fluentd works here.

My instinct is that it's going to be *far* better to use the first
approach (pipe the Fluentd events into Kafka). This will give you all of
the semantics that Kafka provides (e.g. Ordering within a partition,
rewinding streams, durability, etc).

Cheers,
Chris

On 9/5/14 1:36 PM, "Yan Fang" <ya...@gmail.com> wrote:

>also was thinking of having fluentd push to Samza. But don't know how to
>implement this. Not sure if adding a kafka layer between Samza and fluentd
>is the only option.
>
>Do other guys have better ideas?
>
>Thanks,
>
>Fang, Yan
>yanfang724@gmail.com
>+1 (206) 849-4108
>
>
>On Fri, Sep 5, 2014 at 12:09 PM, Shekar Tippur <ct...@gmail.com> wrote:
>
>> Yan,
>>
>> Wont it add additional hop. It did occur to me earlier but was not sure
>>if
>> this is the right way to go if we have a stringent sla driven system
>> depending on it.
>>
>> - Shekar
>>
>>
>> On Fri, Sep 5, 2014 at 10:55 AM, Yan Fang <ya...@gmail.com> wrote:
>>
>> > If you already put the events to the kafka, you can make the Samza
>> accepts
>> > the kafka topic, like the wikipedia-parse project in hello-samza
>>accepts
>> > the kafka topic wikipedia-raw ( see the config file ).
>> >
>> > Thanks,
>> >
>> > Fang, Yan
>> > yanfang724@gmail.com
>> > +1 (206) 849-4108
>> >
>> >
>> > On Fri, Sep 5, 2014 at 8:48 AM, Shekar Tippur <ct...@gmail.com>
>>wrote:
>> >
>> > > Awesome .. This works. Thanks a lot.
>> > >
>> > > Now off to my next step.
>> > > I want to point to an incoming stream of events. These events are
>> routed
>> > > via fluentd. So, fluentd acts as a routing layer where it pushes the
>> > events
>> > > to kafka. Since it is a push and not a pull, any pointers on how to
>> push
>> > it
>> > > to samza? Guessing I need a listener on Samza to collect this?
>> > >
>> > > - Shekar
>> > >
>> > >
>> > > On Fri, Sep 5, 2014 at 1:03 AM, Yan Fang <ya...@gmail.com>
>>wrote:
>> > >
>> > > > Aha, yes, we are almost there. I think I made a mistake in the
>> previous
>> > > > email.
>> > > >
>> > > > 1. modify the *wikipedia-parser.properties ,  NOT
>> > > > *wikipedia-feed.properties
>> > > > 2. run deploy/samza/bin/run-job.sh
>> > > >
>> > >
>> >
>> 
>>--config-factory=org.apache.samza.config.factories.PropertiesConfigFactor
>>y
>> > > >
>> > >
>> >
>> 
>>--config-path=file://$PWD/deploy/samza/config/*wikipedia-parser.propertie
>>s*
>> > > > *(NOT *wikipedia-feed,properties*)*
>> > > >
>> > > > Then you should see the messages in the kafka topic,
>> *wikipedia-edits*
>> > > >
>> > > > Thanks. Let me know if you have any luck . :)
>> > > >
>> > > > Cheers,
>> > > >
>> > > > Fang, Yan
>> > > > yanfang724@gmail.com
>> > > > +1 (206) 849-4108
>> > > >
>> > > >
>> > > > On Thu, Sep 4, 2014 at 11:19 PM, Shekar Tippur <ct...@gmail.com>
>> > > wrote:
>> > > >
>> > > > > Just tried #3. Changed the property file
>>wikipedia-feed.properties
>> > > > >
>> > > > > job.factory.class=org.apache.samza.job.local.LocalJobFactory
>> > > > > Ran ..
>> > > > >
>> > > > > deploy/samza/bin/run-job.sh
>> > > > >
>> > > >
>> > >
>> >
>> 
>>--config-factory=org.apache.samza.config.factories.PropertiesConfigFactor
>>y
>> > > > >
>> > > > >
>> > > >
>> > >
>> >
>> 
>>--config-path=file:///home/ctippur/hello-samza/deploy/samza/config/wikipe
>>dia-feed.properties
>> > > > >
>> > > > > I dont see any debug messages that I added to the feed or the
>> parser
>> > > > file..
>> > > > > I see messages on the kafka-consumer ..
>> > > > >
>> > > > > However the feed job died with the below message
>> > > > >
>> > > > >
>> > > > > Exception in thread "ThreadJob" java.lang.RuntimeException:
>>Trying
>> to
>> > > > > unlisten to a channel that has no listeners in it.
>> > > > >
>> > > > > at
>> > > > >
>> > > > >
>> > > >
>> > >
>> >
>> 
>>samza.examples.wikipedia.system.WikipediaFeed.unlisten(WikipediaFeed.java
>>:98)
>> > > > >
>> > > > > at
>> > > > >
>> > > > >
>> > > >
>> > >
>> >
>> 
>>samza.examples.wikipedia.system.WikipediaConsumer.stop(WikipediaConsumer.
>>java:72)
>> > > > >
>> > > > > at
>> > > > >
>> > > > >
>> > > >
>> > >
>> >
>> 
>>org.apache.samza.system.SystemConsumers$$anonfun$stop$2.apply(SystemConsu
>>mers.scala:152)
>> > > > >
>> > > > > at
>> > > > >
>> > > > >
>> > > >
>> > >
>> >
>> 
>>org.apache.samza.system.SystemConsumers$$anonfun$stop$2.apply(SystemConsu
>>mers.scala:152)
>> > > > >
>> > > > > at scala.collection.Iterator$class.foreach(Iterator.scala:727)
>> > > > >
>> > > > > at 
>>scala.collection.AbstractIterator.foreach(Iterator.scala:1157)
>> > > > >
>> > > > > at
>> > > > >
>> > >
>> 
>>scala.collection.MapLike$DefaultValuesIterable.foreach(MapLike.scala:206)
>> > > > >
>> > > > > at
>> > > >
>> org.apache.samza.system.SystemConsumers.stop(SystemConsumers.scala:152)
>> > > > >
>> > > > > at
>> > > > >
>> > > > >
>> > > >
>> > >
>> >
>> 
>>org.apache.samza.container.SamzaContainer.shutdownConsumers(SamzaContaine
>>r.scala:587)
>> > > > >
>> > > > > at
>> > > >
>> org.apache.samza.container.SamzaContainer.run(SamzaContainer.scala:512)
>> > > > >
>> > > > >  at
>> > > org.apache.samza.job.local.ThreadJob$$anon$1.run(ThreadJob.scala:42)
>> > > > >
>> > > > > - Shekar
>> > > > >
>> > > >
>> > >
>> >
>>


Re: Samza as a Caching layer

Posted by Yan Fang <ya...@gmail.com>.
also was thinking of having fluentd push to Samza. But don't know how to
implement this. Not sure if adding a kafka layer between Samza and fluentd
is the only option.

Do other guys have better ideas?

Thanks,

Fang, Yan
yanfang724@gmail.com
+1 (206) 849-4108


On Fri, Sep 5, 2014 at 12:09 PM, Shekar Tippur <ct...@gmail.com> wrote:

> Yan,
>
> Wont it add additional hop. It did occur to me earlier but was not sure if
> this is the right way to go if we have a stringent sla driven system
> depending on it.
>
> - Shekar
>
>
> On Fri, Sep 5, 2014 at 10:55 AM, Yan Fang <ya...@gmail.com> wrote:
>
> > If you already put the events to the kafka, you can make the Samza
> accepts
> > the kafka topic, like the wikipedia-parse project in hello-samza accepts
> > the kafka topic wikipedia-raw ( see the config file ).
> >
> > Thanks,
> >
> > Fang, Yan
> > yanfang724@gmail.com
> > +1 (206) 849-4108
> >
> >
> > On Fri, Sep 5, 2014 at 8:48 AM, Shekar Tippur <ct...@gmail.com> wrote:
> >
> > > Awesome .. This works. Thanks a lot.
> > >
> > > Now off to my next step.
> > > I want to point to an incoming stream of events. These events are
> routed
> > > via fluentd. So, fluentd acts as a routing layer where it pushes the
> > events
> > > to kafka. Since it is a push and not a pull, any pointers on how to
> push
> > it
> > > to samza? Guessing I need a listener on Samza to collect this?
> > >
> > > - Shekar
> > >
> > >
> > > On Fri, Sep 5, 2014 at 1:03 AM, Yan Fang <ya...@gmail.com> wrote:
> > >
> > > > Aha, yes, we are almost there. I think I made a mistake in the
> previous
> > > > email.
> > > >
> > > > 1. modify the *wikipedia-parser.properties ,  NOT
> > > > *wikipedia-feed.properties
> > > > 2. run deploy/samza/bin/run-job.sh
> > > >
> > >
> >
> --config-factory=org.apache.samza.config.factories.PropertiesConfigFactory
> > > >
> > >
> >
> --config-path=file://$PWD/deploy/samza/config/*wikipedia-parser.properties*
> > > > *(NOT *wikipedia-feed,properties*)*
> > > >
> > > > Then you should see the messages in the kafka topic,
> *wikipedia-edits*
> > > >
> > > > Thanks. Let me know if you have any luck . :)
> > > >
> > > > Cheers,
> > > >
> > > > Fang, Yan
> > > > yanfang724@gmail.com
> > > > +1 (206) 849-4108
> > > >
> > > >
> > > > On Thu, Sep 4, 2014 at 11:19 PM, Shekar Tippur <ct...@gmail.com>
> > > wrote:
> > > >
> > > > > Just tried #3. Changed the property file wikipedia-feed.properties
> > > > >
> > > > > job.factory.class=org.apache.samza.job.local.LocalJobFactory
> > > > > Ran ..
> > > > >
> > > > > deploy/samza/bin/run-job.sh
> > > > >
> > > >
> > >
> >
> --config-factory=org.apache.samza.config.factories.PropertiesConfigFactory
> > > > >
> > > > >
> > > >
> > >
> >
> --config-path=file:///home/ctippur/hello-samza/deploy/samza/config/wikipedia-feed.properties
> > > > >
> > > > > I dont see any debug messages that I added to the feed or the
> parser
> > > > file..
> > > > > I see messages on the kafka-consumer ..
> > > > >
> > > > > However the feed job died with the below message
> > > > >
> > > > >
> > > > > Exception in thread "ThreadJob" java.lang.RuntimeException: Trying
> to
> > > > > unlisten to a channel that has no listeners in it.
> > > > >
> > > > > at
> > > > >
> > > > >
> > > >
> > >
> >
> samza.examples.wikipedia.system.WikipediaFeed.unlisten(WikipediaFeed.java:98)
> > > > >
> > > > > at
> > > > >
> > > > >
> > > >
> > >
> >
> samza.examples.wikipedia.system.WikipediaConsumer.stop(WikipediaConsumer.java:72)
> > > > >
> > > > > at
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.samza.system.SystemConsumers$$anonfun$stop$2.apply(SystemConsumers.scala:152)
> > > > >
> > > > > at
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.samza.system.SystemConsumers$$anonfun$stop$2.apply(SystemConsumers.scala:152)
> > > > >
> > > > > at scala.collection.Iterator$class.foreach(Iterator.scala:727)
> > > > >
> > > > > at scala.collection.AbstractIterator.foreach(Iterator.scala:1157)
> > > > >
> > > > > at
> > > > >
> > >
> scala.collection.MapLike$DefaultValuesIterable.foreach(MapLike.scala:206)
> > > > >
> > > > > at
> > > >
> org.apache.samza.system.SystemConsumers.stop(SystemConsumers.scala:152)
> > > > >
> > > > > at
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.samza.container.SamzaContainer.shutdownConsumers(SamzaContainer.scala:587)
> > > > >
> > > > > at
> > > >
> org.apache.samza.container.SamzaContainer.run(SamzaContainer.scala:512)
> > > > >
> > > > >  at
> > > org.apache.samza.job.local.ThreadJob$$anon$1.run(ThreadJob.scala:42)
> > > > >
> > > > > - Shekar
> > > > >
> > > >
> > >
> >
>

Re: Samza as a Caching layer

Posted by Shekar Tippur <ct...@gmail.com>.
Yan,

Wont it add additional hop. It did occur to me earlier but was not sure if
this is the right way to go if we have a stringent sla driven system
depending on it.

- Shekar


On Fri, Sep 5, 2014 at 10:55 AM, Yan Fang <ya...@gmail.com> wrote:

> If you already put the events to the kafka, you can make the Samza accepts
> the kafka topic, like the wikipedia-parse project in hello-samza accepts
> the kafka topic wikipedia-raw ( see the config file ).
>
> Thanks,
>
> Fang, Yan
> yanfang724@gmail.com
> +1 (206) 849-4108
>
>
> On Fri, Sep 5, 2014 at 8:48 AM, Shekar Tippur <ct...@gmail.com> wrote:
>
> > Awesome .. This works. Thanks a lot.
> >
> > Now off to my next step.
> > I want to point to an incoming stream of events. These events are routed
> > via fluentd. So, fluentd acts as a routing layer where it pushes the
> events
> > to kafka. Since it is a push and not a pull, any pointers on how to push
> it
> > to samza? Guessing I need a listener on Samza to collect this?
> >
> > - Shekar
> >
> >
> > On Fri, Sep 5, 2014 at 1:03 AM, Yan Fang <ya...@gmail.com> wrote:
> >
> > > Aha, yes, we are almost there. I think I made a mistake in the previous
> > > email.
> > >
> > > 1. modify the *wikipedia-parser.properties ,  NOT
> > > *wikipedia-feed.properties
> > > 2. run deploy/samza/bin/run-job.sh
> > >
> >
> --config-factory=org.apache.samza.config.factories.PropertiesConfigFactory
> > >
> >
> --config-path=file://$PWD/deploy/samza/config/*wikipedia-parser.properties*
> > > *(NOT *wikipedia-feed,properties*)*
> > >
> > > Then you should see the messages in the kafka topic, *wikipedia-edits*
> > >
> > > Thanks. Let me know if you have any luck . :)
> > >
> > > Cheers,
> > >
> > > Fang, Yan
> > > yanfang724@gmail.com
> > > +1 (206) 849-4108
> > >
> > >
> > > On Thu, Sep 4, 2014 at 11:19 PM, Shekar Tippur <ct...@gmail.com>
> > wrote:
> > >
> > > > Just tried #3. Changed the property file wikipedia-feed.properties
> > > >
> > > > job.factory.class=org.apache.samza.job.local.LocalJobFactory
> > > > Ran ..
> > > >
> > > > deploy/samza/bin/run-job.sh
> > > >
> > >
> >
> --config-factory=org.apache.samza.config.factories.PropertiesConfigFactory
> > > >
> > > >
> > >
> >
> --config-path=file:///home/ctippur/hello-samza/deploy/samza/config/wikipedia-feed.properties
> > > >
> > > > I dont see any debug messages that I added to the feed or the parser
> > > file..
> > > > I see messages on the kafka-consumer ..
> > > >
> > > > However the feed job died with the below message
> > > >
> > > >
> > > > Exception in thread "ThreadJob" java.lang.RuntimeException: Trying to
> > > > unlisten to a channel that has no listeners in it.
> > > >
> > > > at
> > > >
> > > >
> > >
> >
> samza.examples.wikipedia.system.WikipediaFeed.unlisten(WikipediaFeed.java:98)
> > > >
> > > > at
> > > >
> > > >
> > >
> >
> samza.examples.wikipedia.system.WikipediaConsumer.stop(WikipediaConsumer.java:72)
> > > >
> > > > at
> > > >
> > > >
> > >
> >
> org.apache.samza.system.SystemConsumers$$anonfun$stop$2.apply(SystemConsumers.scala:152)
> > > >
> > > > at
> > > >
> > > >
> > >
> >
> org.apache.samza.system.SystemConsumers$$anonfun$stop$2.apply(SystemConsumers.scala:152)
> > > >
> > > > at scala.collection.Iterator$class.foreach(Iterator.scala:727)
> > > >
> > > > at scala.collection.AbstractIterator.foreach(Iterator.scala:1157)
> > > >
> > > > at
> > > >
> > scala.collection.MapLike$DefaultValuesIterable.foreach(MapLike.scala:206)
> > > >
> > > > at
> > > org.apache.samza.system.SystemConsumers.stop(SystemConsumers.scala:152)
> > > >
> > > > at
> > > >
> > > >
> > >
> >
> org.apache.samza.container.SamzaContainer.shutdownConsumers(SamzaContainer.scala:587)
> > > >
> > > > at
> > > org.apache.samza.container.SamzaContainer.run(SamzaContainer.scala:512)
> > > >
> > > >  at
> > org.apache.samza.job.local.ThreadJob$$anon$1.run(ThreadJob.scala:42)
> > > >
> > > > - Shekar
> > > >
> > >
> >
>

Re: Samza as a Caching layer

Posted by Yan Fang <ya...@gmail.com>.
If you already put the events to the kafka, you can make the Samza accepts
the kafka topic, like the wikipedia-parse project in hello-samza accepts
the kafka topic wikipedia-raw ( see the config file ).

Thanks,

Fang, Yan
yanfang724@gmail.com
+1 (206) 849-4108


On Fri, Sep 5, 2014 at 8:48 AM, Shekar Tippur <ct...@gmail.com> wrote:

> Awesome .. This works. Thanks a lot.
>
> Now off to my next step.
> I want to point to an incoming stream of events. These events are routed
> via fluentd. So, fluentd acts as a routing layer where it pushes the events
> to kafka. Since it is a push and not a pull, any pointers on how to push it
> to samza? Guessing I need a listener on Samza to collect this?
>
> - Shekar
>
>
> On Fri, Sep 5, 2014 at 1:03 AM, Yan Fang <ya...@gmail.com> wrote:
>
> > Aha, yes, we are almost there. I think I made a mistake in the previous
> > email.
> >
> > 1. modify the *wikipedia-parser.properties ,  NOT
> > *wikipedia-feed.properties
> > 2. run deploy/samza/bin/run-job.sh
> >
> --config-factory=org.apache.samza.config.factories.PropertiesConfigFactory
> >
> --config-path=file://$PWD/deploy/samza/config/*wikipedia-parser.properties*
> > *(NOT *wikipedia-feed,properties*)*
> >
> > Then you should see the messages in the kafka topic, *wikipedia-edits*
> >
> > Thanks. Let me know if you have any luck . :)
> >
> > Cheers,
> >
> > Fang, Yan
> > yanfang724@gmail.com
> > +1 (206) 849-4108
> >
> >
> > On Thu, Sep 4, 2014 at 11:19 PM, Shekar Tippur <ct...@gmail.com>
> wrote:
> >
> > > Just tried #3. Changed the property file wikipedia-feed.properties
> > >
> > > job.factory.class=org.apache.samza.job.local.LocalJobFactory
> > > Ran ..
> > >
> > > deploy/samza/bin/run-job.sh
> > >
> >
> --config-factory=org.apache.samza.config.factories.PropertiesConfigFactory
> > >
> > >
> >
> --config-path=file:///home/ctippur/hello-samza/deploy/samza/config/wikipedia-feed.properties
> > >
> > > I dont see any debug messages that I added to the feed or the parser
> > file..
> > > I see messages on the kafka-consumer ..
> > >
> > > However the feed job died with the below message
> > >
> > >
> > > Exception in thread "ThreadJob" java.lang.RuntimeException: Trying to
> > > unlisten to a channel that has no listeners in it.
> > >
> > > at
> > >
> > >
> >
> samza.examples.wikipedia.system.WikipediaFeed.unlisten(WikipediaFeed.java:98)
> > >
> > > at
> > >
> > >
> >
> samza.examples.wikipedia.system.WikipediaConsumer.stop(WikipediaConsumer.java:72)
> > >
> > > at
> > >
> > >
> >
> org.apache.samza.system.SystemConsumers$$anonfun$stop$2.apply(SystemConsumers.scala:152)
> > >
> > > at
> > >
> > >
> >
> org.apache.samza.system.SystemConsumers$$anonfun$stop$2.apply(SystemConsumers.scala:152)
> > >
> > > at scala.collection.Iterator$class.foreach(Iterator.scala:727)
> > >
> > > at scala.collection.AbstractIterator.foreach(Iterator.scala:1157)
> > >
> > > at
> > >
> scala.collection.MapLike$DefaultValuesIterable.foreach(MapLike.scala:206)
> > >
> > > at
> > org.apache.samza.system.SystemConsumers.stop(SystemConsumers.scala:152)
> > >
> > > at
> > >
> > >
> >
> org.apache.samza.container.SamzaContainer.shutdownConsumers(SamzaContainer.scala:587)
> > >
> > > at
> > org.apache.samza.container.SamzaContainer.run(SamzaContainer.scala:512)
> > >
> > >  at
> org.apache.samza.job.local.ThreadJob$$anon$1.run(ThreadJob.scala:42)
> > >
> > > - Shekar
> > >
> >
>

Re: Samza as a Caching layer

Posted by Shekar Tippur <ct...@gmail.com>.
Awesome .. This works. Thanks a lot.

Now off to my next step.
I want to point to an incoming stream of events. These events are routed
via fluentd. So, fluentd acts as a routing layer where it pushes the events
to kafka. Since it is a push and not a pull, any pointers on how to push it
to samza? Guessing I need a listener on Samza to collect this?

- Shekar


On Fri, Sep 5, 2014 at 1:03 AM, Yan Fang <ya...@gmail.com> wrote:

> Aha, yes, we are almost there. I think I made a mistake in the previous
> email.
>
> 1. modify the *wikipedia-parser.properties ,  NOT
> *wikipedia-feed.properties
> 2. run deploy/samza/bin/run-job.sh
> --config-factory=org.apache.samza.config.factories.PropertiesConfigFactory
> --config-path=file://$PWD/deploy/samza/config/*wikipedia-parser.properties*
> *(NOT *wikipedia-feed,properties*)*
>
> Then you should see the messages in the kafka topic, *wikipedia-edits*
>
> Thanks. Let me know if you have any luck . :)
>
> Cheers,
>
> Fang, Yan
> yanfang724@gmail.com
> +1 (206) 849-4108
>
>
> On Thu, Sep 4, 2014 at 11:19 PM, Shekar Tippur <ct...@gmail.com> wrote:
>
> > Just tried #3. Changed the property file wikipedia-feed.properties
> >
> > job.factory.class=org.apache.samza.job.local.LocalJobFactory
> > Ran ..
> >
> > deploy/samza/bin/run-job.sh
> >
> --config-factory=org.apache.samza.config.factories.PropertiesConfigFactory
> >
> >
> --config-path=file:///home/ctippur/hello-samza/deploy/samza/config/wikipedia-feed.properties
> >
> > I dont see any debug messages that I added to the feed or the parser
> file..
> > I see messages on the kafka-consumer ..
> >
> > However the feed job died with the below message
> >
> >
> > Exception in thread "ThreadJob" java.lang.RuntimeException: Trying to
> > unlisten to a channel that has no listeners in it.
> >
> > at
> >
> >
> samza.examples.wikipedia.system.WikipediaFeed.unlisten(WikipediaFeed.java:98)
> >
> > at
> >
> >
> samza.examples.wikipedia.system.WikipediaConsumer.stop(WikipediaConsumer.java:72)
> >
> > at
> >
> >
> org.apache.samza.system.SystemConsumers$$anonfun$stop$2.apply(SystemConsumers.scala:152)
> >
> > at
> >
> >
> org.apache.samza.system.SystemConsumers$$anonfun$stop$2.apply(SystemConsumers.scala:152)
> >
> > at scala.collection.Iterator$class.foreach(Iterator.scala:727)
> >
> > at scala.collection.AbstractIterator.foreach(Iterator.scala:1157)
> >
> > at
> > scala.collection.MapLike$DefaultValuesIterable.foreach(MapLike.scala:206)
> >
> > at
> org.apache.samza.system.SystemConsumers.stop(SystemConsumers.scala:152)
> >
> > at
> >
> >
> org.apache.samza.container.SamzaContainer.shutdownConsumers(SamzaContainer.scala:587)
> >
> > at
> org.apache.samza.container.SamzaContainer.run(SamzaContainer.scala:512)
> >
> >  at org.apache.samza.job.local.ThreadJob$$anon$1.run(ThreadJob.scala:42)
> >
> > - Shekar
> >
>

Re: Samza as a Caching layer

Posted by Yan Fang <ya...@gmail.com>.
Aha, yes, we are almost there. I think I made a mistake in the previous
email.

1. modify the *wikipedia-parser.properties ,  NOT *wikipedia-feed.properties
2. run deploy/samza/bin/run-job.sh
--config-factory=org.apache.samza.config.factories.PropertiesConfigFactory
--config-path=file://$PWD/deploy/samza/config/*wikipedia-parser.properties*
*(NOT *wikipedia-feed,properties*)*

Then you should see the messages in the kafka topic, *wikipedia-edits*

Thanks. Let me know if you have any luck . :)

Cheers,

Fang, Yan
yanfang724@gmail.com
+1 (206) 849-4108


On Thu, Sep 4, 2014 at 11:19 PM, Shekar Tippur <ct...@gmail.com> wrote:

> Just tried #3. Changed the property file wikipedia-feed.properties
>
> job.factory.class=org.apache.samza.job.local.LocalJobFactory
> Ran ..
>
> deploy/samza/bin/run-job.sh
> --config-factory=org.apache.samza.config.factories.PropertiesConfigFactory
>
> --config-path=file:///home/ctippur/hello-samza/deploy/samza/config/wikipedia-feed.properties
>
> I dont see any debug messages that I added to the feed or the parser file..
> I see messages on the kafka-consumer ..
>
> However the feed job died with the below message
>
>
> Exception in thread "ThreadJob" java.lang.RuntimeException: Trying to
> unlisten to a channel that has no listeners in it.
>
> at
>
> samza.examples.wikipedia.system.WikipediaFeed.unlisten(WikipediaFeed.java:98)
>
> at
>
> samza.examples.wikipedia.system.WikipediaConsumer.stop(WikipediaConsumer.java:72)
>
> at
>
> org.apache.samza.system.SystemConsumers$$anonfun$stop$2.apply(SystemConsumers.scala:152)
>
> at
>
> org.apache.samza.system.SystemConsumers$$anonfun$stop$2.apply(SystemConsumers.scala:152)
>
> at scala.collection.Iterator$class.foreach(Iterator.scala:727)
>
> at scala.collection.AbstractIterator.foreach(Iterator.scala:1157)
>
> at
> scala.collection.MapLike$DefaultValuesIterable.foreach(MapLike.scala:206)
>
> at org.apache.samza.system.SystemConsumers.stop(SystemConsumers.scala:152)
>
> at
>
> org.apache.samza.container.SamzaContainer.shutdownConsumers(SamzaContainer.scala:587)
>
> at org.apache.samza.container.SamzaContainer.run(SamzaContainer.scala:512)
>
>  at org.apache.samza.job.local.ThreadJob$$anon$1.run(ThreadJob.scala:42)
>
> - Shekar
>

Re: Samza as a Caching layer

Posted by Shekar Tippur <ct...@gmail.com>.
Just tried #3. Changed the property file wikipedia-feed.properties

job.factory.class=org.apache.samza.job.local.LocalJobFactory
Ran ..

deploy/samza/bin/run-job.sh
--config-factory=org.apache.samza.config.factories.PropertiesConfigFactory
--config-path=file:///home/ctippur/hello-samza/deploy/samza/config/wikipedia-feed.properties

I dont see any debug messages that I added to the feed or the parser file..
I see messages on the kafka-consumer ..

However the feed job died with the below message


Exception in thread "ThreadJob" java.lang.RuntimeException: Trying to
unlisten to a channel that has no listeners in it.

at
samza.examples.wikipedia.system.WikipediaFeed.unlisten(WikipediaFeed.java:98)

at
samza.examples.wikipedia.system.WikipediaConsumer.stop(WikipediaConsumer.java:72)

at
org.apache.samza.system.SystemConsumers$$anonfun$stop$2.apply(SystemConsumers.scala:152)

at
org.apache.samza.system.SystemConsumers$$anonfun$stop$2.apply(SystemConsumers.scala:152)

at scala.collection.Iterator$class.foreach(Iterator.scala:727)

at scala.collection.AbstractIterator.foreach(Iterator.scala:1157)

at scala.collection.MapLike$DefaultValuesIterable.foreach(MapLike.scala:206)

at org.apache.samza.system.SystemConsumers.stop(SystemConsumers.scala:152)

at
org.apache.samza.container.SamzaContainer.shutdownConsumers(SamzaContainer.scala:587)

at org.apache.samza.container.SamzaContainer.run(SamzaContainer.scala:512)

 at org.apache.samza.job.local.ThreadJob$$anon$1.run(ThreadJob.scala:42)

- Shekar

Re: Samza as a Caching layer

Posted by Yan Fang <ya...@gmail.com>.
Hi Shekar,

Sorry for so much trouble. I guess we miss some pieces here. Let's make
sure something:

1. which branch of hello-samza are you using? latest or master? (suggest
master) (I ask this is because by default, the hello-samza already contains
the log4j and slf4j dependencies. A little confused since you have to add
the dependencies by yourself)

2. when you run bin/produce-wikipedia-raw-data.sh, can you see the
wikipedia-raw topic gets messages in Kafka?

3. since you are running in local mode, if you are using the master branch
(which is using samza 0.7.0), could you change

job.factory.class=org.apache.samza.job.*yarn.YarnJobFactory*

to

job.factory.class=org.apache.samza.job.*local.LocalJobFactory *

in wikipedia-feed.properties ?

Then restart the samza job.

Let me know how it goes. :) Thank you.

Cheers,

Fang, Yan
yanfang724@gmail.com
+1 (206) 849-4108


On Thu, Sep 4, 2014 at 4:32 PM, Shekar Tippur <ct...@gmail.com> wrote:

> Yan,
>
> 1. Added a debug statement to
>
> samza-wikipedia/src/main/java/samza/examples/wikipedia/task/WikipediaParserStreamTask.java
>
> System.out.println("Incoming stream " + envelope.getMessage().toString());
>
> 2. ran mvn clean install
>
> 3. tar'd the folder as instructed
>
> 4. Ensured that Kafka is running
>
> 5. Ran deploy/samza/bin/run-job.sh
> --config-factory=org.apache.samza.config.factories.PropertiesConfigFactory
>
> --config-path=/home/ctippur/incubator-samza-hello-samza/deploy/samza/config/wikipedia-parser.properties
>
> 6. ran bin/produce-wikipedia-raw-data.sh to generate some feeds..
>
> I am not seeing anything on the console. I am not seeing anything
> registered to yarn as well. Clearly I am doing something wrong ..
>
> - Shekar
>
>
>
>
> On Thu, Sep 4, 2014 at 1:01 PM, Shekar Tippur <ct...@gmail.com> wrote:
>
> > Yan -
> >
> > Thanks a lot for your input. That makes sense. I will give it a shot now.
> >
> > - Shekar
> >
>

Re: Samza as a Caching layer

Posted by Shekar Tippur <ct...@gmail.com>.
Yan,

1. Added a debug statement to
samza-wikipedia/src/main/java/samza/examples/wikipedia/task/WikipediaParserStreamTask.java

System.out.println("Incoming stream " + envelope.getMessage().toString());

2. ran mvn clean install

3. tar'd the folder as instructed

4. Ensured that Kafka is running

5. Ran deploy/samza/bin/run-job.sh
--config-factory=org.apache.samza.config.factories.PropertiesConfigFactory
--config-path=/home/ctippur/incubator-samza-hello-samza/deploy/samza/config/wikipedia-parser.properties

6. ran bin/produce-wikipedia-raw-data.sh to generate some feeds..

I am not seeing anything on the console. I am not seeing anything
registered to yarn as well. Clearly I am doing something wrong ..

- Shekar




On Thu, Sep 4, 2014 at 1:01 PM, Shekar Tippur <ct...@gmail.com> wrote:

> Yan -
>
> Thanks a lot for your input. That makes sense. I will give it a shot now.
>
> - Shekar
>

Re: Samza as a Caching layer

Posted by Shekar Tippur <ct...@gmail.com>.
Yan -

Thanks a lot for your input. That makes sense. I will give it a shot now.

- Shekar

Re: Samza as a Caching layer

Posted by Yan Fang <ya...@gmail.com>.
aha, I see. Hi Shekar,

I think I know the problem. It was the confusion of the document. You may
want to run

1. run deploy/samza/bin/run-job.sh
--config-factory=org.apache.samza.config.factories.PropertiesConfigFactory
--config-path=file://$PWD/deploy/samza/config*/wikipedia-parser.properties*

*(NOT** wikipedia-feed.properties**)*

2. modify the *WikipediaParserStreamTask.java (NOT *
*WikipediaFeedStreamTask.java)*

Because what *WikipediaFeedStreamTask.java* does is to *get wiki Feed from
external and send to the kafka*. When you run
bin/produce-wikipedia-raw-data.sh , actually you already finish this step.
So your next step will be to parse the feed, that is, the (1) above
(following steps are here
<http://samza.incubator.apache.org/startup/hello-samza/latest/#generate-wikipedia-statistics>
).

Sorry for the confusion. In
http://samza.incubator.apache.org/learn/tutorials/latest/run-hello-samza-without-internet.html#use-local-data-to-run-hello-samza
, it says "instead of running"... :(  We will fix this confusion.

Hope this helps.

Cheers,

Fang, Yan
yanfang724@gmail.com
+1 (206) 849-4108


On Wed, Sep 3, 2014 at 3:20 PM, Shekar Tippur <ct...@gmail.com> wrote:

> Yan,
>
> I do see these jar files
>
> [ctippur@pppdc9prd2y2 incubator-samza-hello-samza]$ ls -ltr
> deploy/samza/lib/slf4j-*
>
> -rw------- 1 ctippur domain users 25689 Aug 18 10:45
> deploy/samza/lib/slf4j-api-1.6.2.jar
>
> -rw------- 1 ctippur domain users  9752 Aug 18 10:48
> deploy/samza/lib/slf4j-log4j12-1.6.2.jar
>
>
> - Shekar
>
>
> On Wed, Sep 3, 2014 at 3:13 PM, Yan Fang <ya...@gmail.com> wrote:
>
> > Hi Shekar,
> >
> > Since you are running local mode, you must have the libs in the directory
> > such as *deploy/samza/lib*. Can you see the slf4j-log4j12-1.6.2.jar
> there?
> > (not the tar.gz). I ask this is because sometimes people forget to tar
> -xvf
> > after they add dependencies and recompile, while the local mode reads the
> > *deploy/samz*a, instead of the tar.gz file. Thank you.
> >
> > Cheers,
> >
> > Fang, Yan
> > yanfang724@gmail.com
> > +1 (206) 849-4108
> >
> >
> > On Wed, Sep 3, 2014 at 2:59 PM, Shekar Tippur <ct...@gmail.com> wrote:
> >
> > > I do see this
> > >
> > > lib/slf4j-log4j12-1.6.2.jar
> > >
> > > In the tgz file. Also, this message seem to me more of a noise.
> > >
> > > I am not sure if we are getting sidetracked on this.
> > >
> > > In the file WikipediaFeedStreamTask.java, I changed log.info to
> > > System.out.println - thinking that this should throw the message to the
> > > console.
> > >
> > > Is this the right place to trap the incoming message?
> > >
> > >
> > > $ cat
> > >
> > >
> >
> samza-wikipedia/src/main/java/samza/examples/wikipedia/task/WikipediaFeedStreamTask.java
> > >
> > > /*
> > >
> > >  * Licensed to the Apache Software Foundation (ASF) under one
> > >
> > >  * or more contributor license agreements.  See the NOTICE file
> > >
> > >  * distributed with this work for additional information
> > >
> > >  * regarding copyright ownership.  The ASF licenses this file
> > >
> > >  * to you under the Apache License, Version 2.0 (the
> > >
> > >  * "License"); you may not use this file except in compliance
> > >
> > >  * with the License.  You may obtain a copy of the License at
> > >
> > >  *
> > >
> > >  *   http://www.apache.org/licenses/LICENSE-2.0
> > >
> > >  *
> > >
> > >  * Unless required by applicable law or agreed to in writing,
> > >
> > >  * software distributed under the License is distributed on an
> > >
> > >  * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
> > >
> > >  * KIND, either express or implied.  See the License for the
> > >
> > >  * specific language governing permissions and limitations
> > >
> > >  * under the License.
> > >
> > >  */
> > >
> > >
> > > package samza.examples.wikipedia.task;
> > >
> > >
> > > import java.util.Map;
> > >
> > > import org.apache.samza.system.IncomingMessageEnvelope;
> > >
> > > import org.apache.samza.system.OutgoingMessageEnvelope;
> > >
> > > import org.apache.samza.system.SystemStream;
> > >
> > > import org.apache.samza.task.MessageCollector;
> > >
> > > import org.apache.samza.task.StreamTask;
> > >
> > > import org.apache.samza.task.TaskCoordinator;
> > >
> > > import
> samza.examples.wikipedia.system.WikipediaFeed.WikipediaFeedEvent;
> > >
> > >
> > > /**
> > >
> > >  * This task is very simple. All it does is take messages that it
> > receives,
> > > and
> > >
> > >  * sends them to a Kafka topic called wikipedia-raw.
> > >
> > >  */
> > >
> > > public class WikipediaFeedStreamTask implements StreamTask {
> > >
> > >   private static final SystemStream OUTPUT_STREAM = new
> > > SystemStream("kafka", "wikipedia-raw");
> > >
> > >
> > >   @Override
> > >
> > >   public void process(IncomingMessageEnvelope envelope,
> MessageCollector
> > > collector, TaskCoordinator coordinator) {
> > >
> > >     Map<String, Object> outgoingMap =
> > > WikipediaFeedEvent.toMap((WikipediaFeedEvent) envelope.getMessage());
> > >
> > >     System.out.println(envelope.getMessage().toString());
> > >
> > >     collector.send(new OutgoingMessageEnvelope(OUTPUT_STREAM,
> > > outgoingMap));
> > >
> > >   }
> > >
> > > }
> > >
> > >
> > >
> > >
> > > On Wed, Sep 3, 2014 at 2:48 PM, Chris Riccomini <
> > > criccomini@linkedin.com.invalid> wrote:
> > >
> > > > Hey Shekar,
> > > >
> > > > Sorry that you're having such a tough time with this. I'll keep
> trying
> > to
> > > > help as best I can.
> > > >
> > > > Do you see slf4j-log4j12 in your job's .tgz file?
> > > >
> > > > Cheers,
> > > > Chris
> > > >
> > > > On 9/3/14 2:46 PM, "Shekar Tippur" <ct...@gmail.com> wrote:
> > > >
> > > > >Chris,
> > > > >
> > > > >I have added the below dependencies and I still get the same
> message.
> > > > >
> > > > >
> > > > >      <dependency>
> > > > >
> > > > >        <groupId>org.slf4j</groupId>
> > > > >
> > > > >        <artifactId>slf4j-api</artifactId>
> > > > >
> > > > >        <version>1.7.7</version>
> > > > >
> > > > >      </dependency>
> > > > >
> > > > >      <dependency>
> > > > >
> > > > >          <groupId>org.slf4j</groupId>
> > > > >
> > > > >          <artifactId>slf4j-log4j12</artifactId>
> > > > >
> > > > >          <version>1.5.6</version>
> > > > >
> > > > >      </dependency>
> > > > >
> > > > >
> > > > >On Wed, Sep 3, 2014 at 9:01 AM, Chris Riccomini <
> > > > >criccomini@linkedin.com.invalid> wrote:
> > > > >
> > > > >> Hey Shekar,
> > > > >>
> > > > >> The SLF4J stuff is saying that you don't have an slf4j binding on
> > your
> > > > >> classpath. Try adding slf4j-log4j as a runtime dependency on your
> > > > >>project.
> > > > >>
> > > > >> Cheers,
> > > > >> Chris
> > > > >>
> > > > >> On 9/2/14 3:24 PM, "Shekar Tippur" <ct...@gmail.com> wrote:
> > > > >>
> > > > >> >Chris ,
> > > > >> >
> > > > >> >In the current state, I just want Samza to connect to 127.0.0.1.
> > > > >> >
> > > > >> >I have set YARN_HOME to
> > > > >> >
> > > > >> >$ echo $YARN_HOME
> > > > >> >
> > > > >> >/home/ctippur/hello-samza/deploy/yarn
> > > > >> >
> > > > >> >I still dont see anything on hadoop console.
> > > > >> >
> > > > >> > Also, I see this during startup
> > > > >> >
> > > > >> >
> > > > >> >SLF4J: Failed to load class "org.slf4j.impl.StaticLoggerBinder".
> > > > >> >
> > > > >> >SLF4J: Defaulting to no-operation (NOP) logger implementation
> > > > >> >
> > > > >> >SLF4J: See http://www.slf4j.org/codes.html#StaticLoggerBinder
> for
> > > > >>further
> > > > >> >details.
> > > > >> >
> > > > >> >SLF4J: Failed to load class "org.slf4j.impl.StaticLoggerBinder".
> > > > >> >
> > > > >> >SLF4J: Defaulting to no-operation (NOP) logger implementation
> > > > >> >
> > > > >> >SLF4J: See http://www.slf4j.org/codes.html#StaticLoggerBinder
> for
> > > > >>further
> > > > >> >details.
> > > > >> >
> > > > >> >- Shekar
> > > > >> >
> > > > >> >
> > > > >> >On Tue, Sep 2, 2014 at 2:20 PM, Chris Riccomini <
> > > > >> >criccomini@linkedin.com.invalid> wrote:
> > > > >> >
> > > > >> >> Hey Shekar,
> > > > >> >>
> > > > >> >> Ah ha. In that case, do you expect your SamzaContainer to try
> to
> > > > >>connect
> > > > >> >> to the RM at 127.0.0.1, or do you expect it to try to connect
> to
> > > some
> > > > >> >> remote RM? If you expect it to try and connect to a remote RM,
> > it's
> > > > >>not
> > > > >> >> doing that. Perhaps because YARN_HOME isn't set.
> > > > >> >>
> > > > >> >> If you go to your RM's web interface, how many active nodes do
> > you
> > > > >>see
> > > > >> >> listed?
> > > > >> >>
> > > > >> >> Cheers,
> > > > >> >> Chris
> > > > >> >>
> > > > >> >> On 9/2/14 2:17 PM, "Shekar Tippur" <ct...@gmail.com> wrote:
> > > > >> >>
> > > > >> >> >Chris ..
> > > > >> >> >
> > > > >> >> >I am using a rhel server to host all the components (Yarn,
> > kafka,
> > > > >> >>samza).
> > > > >> >> >I
> > > > >> >> >dont have ACLs open to wikipedia.
> > > > >> >> >I am following ..
> > > > >> >> >
> > > > >> >>
> > > > >> >>
> > > > >>
> > > > >>
> > > >
> > >
> >
> http://samza.incubator.apache.org/learn/tutorials/latest/run-hello-samza-
> > > > >> >>w
> > > > >> >> >ithout-internet.html
> > > > >> >> >
> > > > >> >> >- Shekar
> > > > >> >> >
> > > > >> >> >
> > > > >> >> >
> > > > >> >> >On Tue, Sep 2, 2014 at 2:13 PM, Chris Riccomini <
> > > > >> >> >criccomini@linkedin.com.invalid> wrote:
> > > > >> >> >
> > > > >> >> >> Hey Shekar,
> > > > >> >> >>
> > > > >> >> >> Can you try changing that to:
> > > > >> >> >>
> > > > >> >> >>   http://127.0.0.1:8088/cluster
> > > > >> >> >>
> > > > >> >> >>
> > > > >> >> >> And see if you can connect?
> > > > >> >> >>
> > > > >> >> >> Cheers,
> > > > >> >> >> Chris
> > > > >> >> >>
> > > > >> >> >> On 9/2/14 1:21 PM, "Shekar Tippur" <ct...@gmail.com>
> wrote:
> > > > >> >> >>
> > > > >> >> >> >Other observation is ..
> > > > >> >> >> >
> > > > >> >> >> >http://10.132.62.185:8088/cluster shows that no
> applications
> > > are
> > > > >> >> >>running.
> > > > >> >> >> >
> > > > >> >> >> >- Shekar
> > > > >> >> >> >
> > > > >> >> >> >
> > > > >> >> >> >
> > > > >> >> >> >
> > > > >> >> >> >On Tue, Sep 2, 2014 at 1:15 PM, Shekar Tippur <
> > > ctippur@gmail.com
> > > > >
> > > > >> >> >>wrote:
> > > > >> >> >> >
> > > > >> >> >> >> Yarn seem to be running ..
> > > > >> >> >> >>
> > > > >> >> >> >> yarn      5462  0.0  2.0 1641296 161404 ?      Sl   Jun20
> > > > >>95:26
> > > > >> >> >> >> /usr/java/jdk1.6.0_31/bin/java -Dproc_resourcemanager
> > > -Xmx1000m
> > > > >> >> >> >> -Dhadoop.log.dir=/var/log/hadoop-yarn
> > > > >> >> >> >>-Dyarn.log.dir=/var/log/hadoop-yarn
> > > > >> >> >> >>
> > -Dhadoop.log.file=yarn-yarn-resourcemanager-pppdc9prd2y2.log
> > > > >> >> >> >>
> -Dyarn.log.file=yarn-yarn-resourcemanager-pppdc9prd2y2.log
> > > > >> >> >> >> -Dyarn.home.dir=/usr/lib/hadoop-yarn
> > > > >> >> >> >>-Dhadoop.home.dir=/usr/lib/hadoop-yarn
> > > > >> >> >> >> -Dhadoop.root.logger=INFO,RFA -Dyarn.root.logger=INFO,RFA
> > > > >> >> >> >> -Djava.library.path=/usr/lib/hadoop/lib/native -classpath
> > > > >> >> >> >>
> > > > >> >> >>
> > > > >> >>
> > > > >>
> > > >
> > >
> >
> >>>>>>>>/etc/hadoop/conf:/etc/hadoop/conf:/etc/hadoop/conf:/usr/lib/hadoop/
> > > > >>>>>>>>li
> > > > >> >>>>>>b/
> > > > >> >> >>>>*:
> > > > >> >> >>
> > > > >> >>
> > > > >>
> > > >
> > >
> >
> >>>>>>>>/usr/lib/hadoop/.//*:/usr/lib/hadoop-hdfs/./:/usr/lib/hadoop-hdfs/l
> > > > >>>>>>>>ib
> > > > >> >>>>>>/*
> > > > >> >> >>>>:/
> > > > >> >> >>
> > > > >> >>
> > > > >>
> > > >
> > >
> >
> >>>>>>>>usr/lib/hadoop-hdfs/.//*:/usr/lib/hadoop-yarn/lib/*:/usr/lib/hadoop
> > > > >>>>>>>>-y
> > > > >> >>>>>>ar
> > > > >> >> >>>>n/
> > > > >> >> >>
> > > > >> >>
> > > > >>
> > > >
> > >
> >
> >>>>>>>>.//*:/usr/lib/hadoop-mapreduce/lib/*:/usr/lib/hadoop-mapreduce/.//*
> > > > >>>>>>>>:/
> > > > >> >>>>>>us
> > > > >> >> >>>>r/
> > > > >> >> >>
> > > > >> >>
> > > > >>
> > > >
> > >
> >
> >>>>>>>>lib/hadoop-yarn/.//*:/usr/lib/hadoop-yarn/lib/*:/etc/hadoop/conf/rm
> > > > >>>>>>>>-c
> > > > >> >>>>>>on
> > > > >> >> >>>>fi
> > > > >> >> >> >>g/log4j.properties
> > > > >> >> >> >>
> > org.apache.hadoop.yarn.server.resourcemanager.ResourceManager
> > > > >> >> >> >>
> > > > >> >> >> >> I can tail kafka topic as well ..
> > > > >> >> >> >>
> > > > >> >> >> >> deploy/kafka/bin/kafka-console-consumer.sh  --zookeeper
> > > > >> >> >>localhost:2181
> > > > >> >> >> >>--topic wikipedia-raw
> > > > >> >> >> >>
> > > > >> >> >> >>
> > > > >> >> >> >>
> > > > >> >> >> >>
> > > > >> >> >> >> On Tue, Sep 2, 2014 at 1:04 PM, Chris Riccomini <
> > > > >> >> >> >> criccomini@linkedin.com.invalid> wrote:
> > > > >> >> >> >>
> > > > >> >> >> >>> Hey Shekar,
> > > > >> >> >> >>>
> > > > >> >> >> >>> It looks like your job is hanging trying to connect to
> the
> > > RM
> > > > >>on
> > > > >> >> >>your
> > > > >> >> >> >>> localhost. I thought that you said that your job was
> > running
> > > > >>in
> > > > >> >> >>local
> > > > >> >> >> >>> mode. If so, it should be using the LocalJobFactory. If
> > not,
> > > > >>and
> > > > >> >>you
> > > > >> >> >> >>> intend to run on YARN, is your YARN RM up and running on
> > > > >> >>localhost?
> > > > >> >> >> >>>
> > > > >> >> >> >>> Cheers,
> > > > >> >> >> >>> Chris
> > > > >> >> >> >>>
> > > > >> >> >> >>> On 9/2/14 12:22 PM, "Shekar Tippur" <ct...@gmail.com>
> > > > wrote:
> > > > >> >> >> >>>
> > > > >> >> >> >>> >Chris ..
> > > > >> >> >> >>> >
> > > > >> >> >> >>> >$ cat ./deploy/samza/undefined-samza-container-name.log
> > > > >> >> >> >>> >
> > > > >> >> >> >>> >2014-09-02 11:17:58 JobRunner [INFO] job factory:
> > > > >> >> >> >>> >org.apache.samza.job.yarn.YarnJobFactory
> > > > >> >> >> >>> >
> > > > >> >> >> >>> >2014-09-02 11:17:59 ClientHelper [INFO] trying to
> connect
> > > to
> > > > >>RM
> > > > >> >> >> >>> >127.0.0.1:8032
> > > > >> >> >> >>> >
> > > > >> >> >> >>> >2014-09-02 11:17:59 NativeCodeLoader [WARN] Unable to
> > load
> > > > >> >> >> >>>native-hadoop
> > > > >> >> >> >>> >library for your platform... using builtin-java classes
> > > where
> > > > >> >> >> >>>applicable
> > > > >> >> >> >>> >
> > > > >> >> >> >>> >2014-09-02 11:17:59 RMProxy [INFO] Connecting to
> > > > >>ResourceManager
> > > > >> >> >>at /
> > > > >> >> >> >>> >127.0.0.1:8032
> > > > >> >> >> >>> >
> > > > >> >> >> >>> >
> > > > >> >> >> >>> >and Log4j ..
> > > > >> >> >> >>> >
> > > > >> >> >> >>> ><!DOCTYPE log4j:configuration SYSTEM "log4j.dtd">
> > > > >> >> >> >>> >
> > > > >> >> >> >>> ><log4j:configuration
> > > > >> >> >>xmlns:log4j="http://jakarta.apache.org/log4j/">
> > > > >> >> >> >>> >
> > > > >> >> >> >>> >  <appender name="RollingAppender"
> > > > >> >> >> >>> >class="org.apache.log4j.DailyRollingFileAppender">
> > > > >> >> >> >>> >
> > > > >> >> >> >>> >     <param name="File"
> > > > >> >> >> >>> >value="${samza.log.dir}/${samza.container.name}.log"
> > > > >> >> >> >>> >/>
> > > > >> >> >> >>> >
> > > > >> >> >> >>> >     <param name="DatePattern" value="'.'yyyy-MM-dd" />
> > > > >> >> >> >>> >
> > > > >> >> >> >>> >     <layout class="org.apache.log4j.PatternLayout">
> > > > >> >> >> >>> >
> > > > >> >> >> >>> >      <param name="ConversionPattern"
> > value="%d{yyyy-MM-dd
> > > > >> >> >>HH:mm:ss}
> > > > >> >> >> >>> %c{1}
> > > > >> >> >> >>> >[%p] %m%n" />
> > > > >> >> >> >>> >
> > > > >> >> >> >>> >     </layout>
> > > > >> >> >> >>> >
> > > > >> >> >> >>> >  </appender>
> > > > >> >> >> >>> >
> > > > >> >> >> >>> >  <root>
> > > > >> >> >> >>> >
> > > > >> >> >> >>> >    <priority value="info" />
> > > > >> >> >> >>> >
> > > > >> >> >> >>> >    <appender-ref ref="RollingAppender"/>
> > > > >> >> >> >>> >
> > > > >> >> >> >>> >  </root>
> > > > >> >> >> >>> >
> > > > >> >> >> >>> ></log4j:configuration>
> > > > >> >> >> >>> >
> > > > >> >> >> >>> >
> > > > >> >> >> >>> >On Tue, Sep 2, 2014 at 12:02 PM, Chris Riccomini <
> > > > >> >> >> >>> >criccomini@linkedin.com.invalid> wrote:
> > > > >> >> >> >>> >
> > > > >> >> >> >>> >> Hey Shekar,
> > > > >> >> >> >>> >>
> > > > >> >> >> >>> >> Can you attach your log files? I'm wondering if it's
> a
> > > > >> >> >> >>>mis-configured
> > > > >> >> >> >>> >> log4j.xml (or missing slf4j-log4j jar), which is
> > leading
> > > to
> > > > >> >> >>nearly
> > > > >> >> >> >>> empty
> > > > >> >> >> >>> >> log files. Also, I'm wondering if the job starts
> fully.
> > > > >> >>Anything
> > > > >> >> >>you
> > > > >> >> >> >>> can
> > > > >> >> >> >>> >> attach would be helpful.
> > > > >> >> >> >>> >>
> > > > >> >> >> >>> >> Cheers,
> > > > >> >> >> >>> >> Chris
> > > > >> >> >> >>> >>
> > > > >> >> >> >>> >> On 9/2/14 11:43 AM, "Shekar Tippur" <
> ctippur@gmail.com
> > >
> > > > >> wrote:
> > > > >> >> >> >>> >>
> > > > >> >> >> >>> >> >I am running in local mode.
> > > > >> >> >> >>> >> >
> > > > >> >> >> >>> >> >S
> > > > >> >> >> >>> >> >On Sep 2, 2014 11:42 AM, "Yan Fang" <
> > > yanfang724@gmail.com
> > > > >
> > > > >> >> >>wrote:
> > > > >> >> >> >>> >> >
> > > > >> >> >> >>> >> >> Hi Shekar.
> > > > >> >> >> >>> >> >>
> > > > >> >> >> >>> >> >> Are you running job in local mode or yarn mode? If
> > > yarn
> > > > >> >>mode,
> > > > >> >> >>the
> > > > >> >> >> >>> log
> > > > >> >> >> >>> >> >>is in
> > > > >> >> >> >>> >> >> the yarn's container log.
> > > > >> >> >> >>> >> >>
> > > > >> >> >> >>> >> >> Thanks,
> > > > >> >> >> >>> >> >>
> > > > >> >> >> >>> >> >> Fang, Yan
> > > > >> >> >> >>> >> >> yanfang724@gmail.com
> > > > >> >> >> >>> >> >> +1 (206) 849-4108
> > > > >> >> >> >>> >> >>
> > > > >> >> >> >>> >> >>
> > > > >> >> >> >>> >> >> On Tue, Sep 2, 2014 at 11:36 AM, Shekar Tippur
> > > > >> >> >> >>><ct...@gmail.com>
> > > > >> >> >> >>> >> >>wrote:
> > > > >> >> >> >>> >> >>
> > > > >> >> >> >>> >> >> > Chris,
> > > > >> >> >> >>> >> >> >
> > > > >> >> >> >>> >> >> > Got some time to play around a bit more.
> > > > >> >> >> >>> >> >> > I tried to edit
> > > > >> >> >> >>> >> >> >
> > > > >> >> >> >>> >> >> >
> > > > >> >> >> >>> >> >>
> > > > >> >> >> >>> >>
> > > > >> >> >> >>>
> > > > >> >> >> >>>
> > > > >> >> >>
> > > > >> >>
> > > > >>
> > > >
> > >
> >
> >>>>>>>>>>>>>samza-wikipedia/src/main/java/samza/examples/wikipedia/task/Wi
> > > > >>>>>>>>>>>>>ki
> > > > >> >>>>>>>>>>>pe
> > > > >> >> >>>>>>>>>di
> > > > >> >> >> >>>>>>>aFe
> > > > >> >> >> >>> >>>>ed
> > > > >> >> >> >>> >> >>StreamTask.java
> > > > >> >> >> >>> >> >> > to add a logger info statement to tap the
> incoming
> > > > >> >>message.
> > > > >> >> >>I
> > > > >> >> >> >>>dont
> > > > >> >> >> >>> >>see
> > > > >> >> >> >>> >> >> the
> > > > >> >> >> >>> >> >> > messages being printed to the log file.
> > > > >> >> >> >>> >> >> >
> > > > >> >> >> >>> >> >> > Is this the right place to start?
> > > > >> >> >> >>> >> >> >
> > > > >> >> >> >>> >> >> > public class WikipediaFeedStreamTask implements
> > > > >> >>StreamTask {
> > > > >> >> >> >>> >> >> >
> > > > >> >> >> >>> >> >> >   private static final SystemStream
> OUTPUT_STREAM
> > =
> > > > >>new
> > > > >> >> >> >>> >> >> > SystemStream("kafka",
> > > > >> >> >> >>> >> >> > "wikipedia-raw");
> > > > >> >> >> >>> >> >> >
> > > > >> >> >> >>> >> >> >   private static final Logger log =
> > > > >> >>LoggerFactory.getLogger
> > > > >> >> >> >>> >> >> > (WikipediaFeedStreamTask.class);
> > > > >> >> >> >>> >> >> >
> > > > >> >> >> >>> >> >> >   @Override
> > > > >> >> >> >>> >> >> >
> > > > >> >> >> >>> >> >> >   public void process(IncomingMessageEnvelope
> > > > >>envelope,
> > > > >> >> >> >>> >> >>MessageCollector
> > > > >> >> >> >>> >> >> > collector, TaskCoordinator coordinator) {
> > > > >> >> >> >>> >> >> >
> > > > >> >> >> >>> >> >> >     Map<String, Object> outgoingMap =
> > > > >> >> >> >>> >> >> > WikipediaFeedEvent.toMap((WikipediaFeedEvent)
> > > > >> >> >> >>> >>envelope.getMessage());
> > > > >> >> >> >>> >> >> >
> > > > >> >> >> >>> >> >> >     log.info(envelope.getMessage().toString());
> > > > >> >> >> >>> >> >> >
> > > > >> >> >> >>> >> >> >     collector.send(new
> > > > >> >> >>OutgoingMessageEnvelope(OUTPUT_STREAM,
> > > > >> >> >> >>> >> >> > outgoingMap));
> > > > >> >> >> >>> >> >> >
> > > > >> >> >> >>> >> >> >   }
> > > > >> >> >> >>> >> >> >
> > > > >> >> >> >>> >> >> > }
> > > > >> >> >> >>> >> >> >
> > > > >> >> >> >>> >> >> >
> > > > >> >> >> >>> >> >> > On Mon, Aug 25, 2014 at 9:01 AM, Chris
> Riccomini <
> > > > >> >> >> >>> >> >> > criccomini@linkedin.com.invalid> wrote:
> > > > >> >> >> >>> >> >> >
> > > > >> >> >> >>> >> >> > > Hey Shekar,
> > > > >> >> >> >>> >> >> > >
> > > > >> >> >> >>> >> >> > > Your thought process is on the right track.
> It's
> > > > >> >>probably
> > > > >> >> >> >>>best
> > > > >> >> >> >>> to
> > > > >> >> >> >>> >> >>start
> > > > >> >> >> >>> >> >> > > with hello-samza, and modify it to get what
> you
> > > > >>want.
> > > > >> >>To
> > > > >> >> >> >>>start
> > > > >> >> >> >>> >>with,
> > > > >> >> >> >>> >> >> > > you'll want to:
> > > > >> >> >> >>> >> >> > >
> > > > >> >> >> >>> >> >> > > 1. Write a simple StreamTask that just does
> > > > >>something
> > > > >> >> >>silly
> > > > >> >> >> >>>like
> > > > >> >> >> >>> >> >>just
> > > > >> >> >> >>> >> >> > > print messages that it receives.
> > > > >> >> >> >>> >> >> > > 2. Write a configuration for the job that
> > consumes
> > > > >>from
> > > > >> >> >>just
> > > > >> >> >> >>>the
> > > > >> >> >> >>> >> >>stream
> > > > >> >> >> >>> >> >> > > (alerts from different sources).
> > > > >> >> >> >>> >> >> > > 3. Run this to make sure you've got it
> working.
> > > > >> >> >> >>> >> >> > > 4. Now add your table join. This can be
> either a
> > > > >> >> >>change-data
> > > > >> >> >> >>> >>capture
> > > > >> >> >> >>> >> >> > (CDC)
> > > > >> >> >> >>> >> >> > > stream, or via a remote DB call.
> > > > >> >> >> >>> >> >> > >
> > > > >> >> >> >>> >> >> > > That should get you to a point where you've
> got
> > > your
> > > > >> >>job
> > > > >> >> >>up
> > > > >> >> >> >>>and
> > > > >> >> >> >>> >> >> running.
> > > > >> >> >> >>> >> >> > > From there, you could create your own Maven
> > > project,
> > > > >> >>and
> > > > >> >> >> >>>migrate
> > > > >> >> >> >>> >> >>your
> > > > >> >> >> >>> >> >> > code
> > > > >> >> >> >>> >> >> > > over accordingly.
> > > > >> >> >> >>> >> >> > >
> > > > >> >> >> >>> >> >> > > Cheers,
> > > > >> >> >> >>> >> >> > > Chris
> > > > >> >> >> >>> >> >> > >
> > > > >> >> >> >>> >> >> > > On 8/24/14 1:42 AM, "Shekar Tippur"
> > > > >><ctippur@gmail.com
> > > > >> >
> > > > >> >> >> >>>wrote:
> > > > >> >> >> >>> >> >> > >
> > > > >> >> >> >>> >> >> > > >Chris,
> > > > >> >> >> >>> >> >> > > >
> > > > >> >> >> >>> >> >> > > >I have gone thro the documentation and
> decided
> > > that
> > > > >> >>the
> > > > >> >> >> >>>option
> > > > >> >> >> >>> >> >>that is
> > > > >> >> >> >>> >> >> > > >most
> > > > >> >> >> >>> >> >> > > >suitable for me is stream-table.
> > > > >> >> >> >>> >> >> > > >
> > > > >> >> >> >>> >> >> > > >I see the following things:
> > > > >> >> >> >>> >> >> > > >
> > > > >> >> >> >>> >> >> > > >1. Point samza to a table (database)
> > > > >> >> >> >>> >> >> > > >2. Point Samza to a stream - Alert stream
> from
> > > > >> >>different
> > > > >> >> >> >>> sources
> > > > >> >> >> >>> >> >> > > >3. Join key like a hostname
> > > > >> >> >> >>> >> >> > > >
> > > > >> >> >> >>> >> >> > > >I have Hello Samza working. To extend that to
> > do
> > > > >>what
> > > > >> >>my
> > > > >> >> >> >>>needs
> > > > >> >> >> >>> >> >>are, I
> > > > >> >> >> >>> >> >> am
> > > > >> >> >> >>> >> >> > > >not sure where to start (Needs more code
> change
> > > OR
> > > > >> >> >> >>> configuration
> > > > >> >> >> >>> >> >> changes
> > > > >> >> >> >>> >> >> > > >OR
> > > > >> >> >> >>> >> >> > > >both)?
> > > > >> >> >> >>> >> >> > > >
> > > > >> >> >> >>> >> >> > > >I have gone thro
> > > > >> >> >> >>> >> >> > > >
> > > > >> >> >> >>> >> >> >
> > > > >> >> >> >>> >> >>
> > > > >> >> >> >>> >> >>
> > > > >> >> >> >>> >>
> > > > >> >> >> >>> >>
> > > > >> >> >> >>>
> > > > >> >> >> >>>
> > > > >> >> >>
> > > > >> >>
> > > > >>
> > > >
> > http://samza.incubator.apache.org/learn/documentation/latest/api/overvie
> > > > >> >> >> >>>w
> > > > >> >> >> >>> >> >> > > .
> > > > >> >> >> >>> >> >> > > >html
> > > > >> >> >> >>> >> >> > > >
> > > > >> >> >> >>> >> >> > > >Is my thought process on the right track? Can
> > you
> > > > >> >>please
> > > > >> >> >> >>>point
> > > > >> >> >> >>> >>me
> > > > >> >> >> >>> >> >>to
> > > > >> >> >> >>> >> >> the
> > > > >> >> >> >>> >> >> > > >right direction?
> > > > >> >> >> >>> >> >> > > >
> > > > >> >> >> >>> >> >> > > >- Shekar
> > > > >> >> >> >>> >> >> > > >
> > > > >> >> >> >>> >> >> > > >
> > > > >> >> >> >>> >> >> > > >
> > > > >> >> >> >>> >> >> > > >On Thu, Aug 21, 2014 at 1:08 PM, Shekar
> Tippur
> > > > >> >> >> >>> >><ct...@gmail.com>
> > > > >> >> >> >>> >> >> > wrote:
> > > > >> >> >> >>> >> >> > > >
> > > > >> >> >> >>> >> >> > > >> Chris,
> > > > >> >> >> >>> >> >> > > >>
> > > > >> >> >> >>> >> >> > > >> This is perfectly good answer. I will start
> > > > >>poking
> > > > >> >>more
> > > > >> >> >> >>>into
> > > > >> >> >> >>> >> >>option
> > > > >> >> >> >>> >> >> > #4.
> > > > >> >> >> >>> >> >> > > >>
> > > > >> >> >> >>> >> >> > > >> - Shekar
> > > > >> >> >> >>> >> >> > > >>
> > > > >> >> >> >>> >> >> > > >>
> > > > >> >> >> >>> >> >> > > >> On Thu, Aug 21, 2014 at 1:05 PM, Chris
> > > Riccomini
> > > > >><
> > > > >> >> >> >>> >> >> > > >> criccomini@linkedin.com.invalid> wrote:
> > > > >> >> >> >>> >> >> > > >>
> > > > >> >> >> >>> >> >> > > >>> Hey Shekar,
> > > > >> >> >> >>> >> >> > > >>>
> > > > >> >> >> >>> >> >> > > >>> Your two options are really (3) or (4),
> > then.
> > > > >>You
> > > > >> >>can
> > > > >> >> >> >>>either
> > > > >> >> >> >>> >>run
> > > > >> >> >> >>> >> >> some
> > > > >> >> >> >>> >> >> > > >>> external DB that holds the data set, and
> you
> > > can
> > > > >> >> >>query it
> > > > >> >> >> >>> >>from a
> > > > >> >> >> >>> >> >> > > >>> StreamTask, or you can use Samza's state
> > store
> > > > >> >> >>feature to
> > > > >> >> >> >>> >>push
> > > > >> >> >> >>> >> >>data
> > > > >> >> >> >>> >> >> > > >>>into a
> > > > >> >> >> >>> >> >> > > >>> stream that you can then store in a
> > > partitioned
> > > > >> >> >>key-value
> > > > >> >> >> >>> >>store
> > > > >> >> >> >>> >> >> along
> > > > >> >> >> >>> >> >> > > >>>with
> > > > >> >> >> >>> >> >> > > >>> your StreamTasks. There is some
> > documentation
> > > > >>here
> > > > >> >> >>about
> > > > >> >> >> >>>the
> > > > >> >> >> >>> >> >>state
> > > > >> >> >> >>> >> >> > > >>>store
> > > > >> >> >> >>> >> >> > > >>> approach:
> > > > >> >> >> >>> >> >> > > >>>
> > > > >> >> >> >>> >> >> > > >>>
> > > > >> >> >> >>> >> >> > > >>>
> > > > >> >> >> >>> >> >> > > >>>
> > > > >> >> >> >>> >> >> > >
> > > > >> >> >> >>> >> >>
> > > > >> >> >> >>> >>
> > > > >> >> >> >>>
> > > > >> >> >> >>>
> > > > >> >> >>
> > > > >> >>
> > > > >>
> > > >
> > http://samza.incubator.apache.org/learn/documentation/0.7.0/container/st
> > > > >> >> >> >>> >> >> > > >>>ate
> > > > >> >> >> >>> >> >> > > >>> -management.html
> > > > >> >> >> >>> >> >> > > >>>
> > > > >> >> >> >>> >> >> > > >>><
> > > > >> >> >> >>> >> >> > >
> > > > >> >> >> >>> >> >>
> > > > >> >> >> >>>
> > > > >> >> >> >>>>>
> > > > >> >> >>
> > > > >> >>
> > > >
> http://samza.incubator.apache.org/learn/documentation/0.7.0/container/
> > > > >> >> >> >>>>>s
> > > > >> >> >> >>> >> >> > > >>>tate-management.html>
> > > > >> >> >> >>> >> >> > > >>>
> > > > >> >> >> >>> >> >> > > >>>
> > > > >> >> >> >>> >> >> > > >>> (4) is going to require more up front
> effort
> > > > >>from
> > > > >> >>you,
> > > > >> >> >> >>>since
> > > > >> >> >> >>> >> >>you'll
> > > > >> >> >> >>> >> >> > > >>>have
> > > > >> >> >> >>> >> >> > > >>> to understand how Kafka's partitioning
> model
> > > > >>works,
> > > > >> >> >>and
> > > > >> >> >> >>> setup
> > > > >> >> >> >>> >> >>some
> > > > >> >> >> >>> >> >> > > >>> pipeline to push the updates for your
> state.
> > > In
> > > > >>the
> > > > >> >> >>long
> > > > >> >> >> >>> >>run, I
> > > > >> >> >> >>> >> >> > believe
> > > > >> >> >> >>> >> >> > > >>> it's the better approach, though. Local
> > > lookups
> > > > >>on
> > > > >> >>a
> > > > >> >> >> >>> >>key-value
> > > > >> >> >> >>> >> >> store
> > > > >> >> >> >>> >> >> > > >>> should be faster than doing remote RPC
> calls
> > > to
> > > > >>a
> > > > >> >>DB
> > > > >> >> >>for
> > > > >> >> >> >>> >>every
> > > > >> >> >> >>> >> >> > message.
> > > > >> >> >> >>> >> >> > > >>>
> > > > >> >> >> >>> >> >> > > >>> I'm sorry I can't give you a more
> definitive
> > > > >> >>answer.
> > > > >> >> >>It's
> > > > >> >> >> >>> >>really
> > > > >> >> >> >>> >> >> > about
> > > > >> >> >> >>> >> >> > > >>> trade-offs.
> > > > >> >> >> >>> >> >> > > >>>
> > > > >> >> >> >>> >> >> > > >>> Cheers,
> > > > >> >> >> >>> >> >> > > >>> Chris
> > > > >> >> >> >>> >> >> > > >>>
> > > > >> >> >> >>> >> >> > > >>> On 8/21/14 12:22 PM, "Shekar Tippur"
> > > > >> >> >><ct...@gmail.com>
> > > > >> >> >> >>> >>wrote:
> > > > >> >> >> >>> >> >> > > >>>
> > > > >> >> >> >>> >> >> > > >>> >Chris,
> > > > >> >> >> >>> >> >> > > >>> >
> > > > >> >> >> >>> >> >> > > >>> >A big thanks for a swift response. The
> data
> > > > >>set is
> > > > >> >> >>huge
> > > > >> >> >> >>>and
> > > > >> >> >> >>> >>the
> > > > >> >> >> >>> >> >> > > >>>frequency
> > > > >> >> >> >>> >> >> > > >>> >is in burst.
> > > > >> >> >> >>> >> >> > > >>> >What do you suggest?
> > > > >> >> >> >>> >> >> > > >>> >
> > > > >> >> >> >>> >> >> > > >>> >- Shekar
> > > > >> >> >> >>> >> >> > > >>> >
> > > > >> >> >> >>> >> >> > > >>> >
> > > > >> >> >> >>> >> >> > > >>> >On Thu, Aug 21, 2014 at 11:05 AM, Chris
> > > > >>Riccomini
> > > > >> >><
> > > > >> >> >> >>> >> >> > > >>> >criccomini@linkedin.com.invalid> wrote:
> > > > >> >> >> >>> >> >> > > >>> >
> > > > >> >> >> >>> >> >> > > >>> >> Hey Shekar,
> > > > >> >> >> >>> >> >> > > >>> >>
> > > > >> >> >> >>> >> >> > > >>> >> This is feasible, and you are on the
> > right
> > > > >> >>thought
> > > > >> >> >> >>> >>process.
> > > > >> >> >> >>> >> >> > > >>> >>
> > > > >> >> >> >>> >> >> > > >>> >> For the sake of discussion, I'm going
> to
> > > > >>pretend
> > > > >> >> >>that
> > > > >> >> >> >>>you
> > > > >> >> >> >>> >> >>have a
> > > > >> >> >> >>> >> >> > > >>>Kafka
> > > > >> >> >> >>> >> >> > > >>> >> topic called "PageViewEvent", which has
> > > just
> > > > >> >>the IP
> > > > >> >> >> >>> >>address
> > > > >> >> >> >>> >> >>that
> > > > >> >> >> >>> >> >> > was
> > > > >> >> >> >>> >> >> > > >>> >>used
> > > > >> >> >> >>> >> >> > > >>> >> to view a page. These messages will be
> > > logged
> > > > >> >>every
> > > > >> >> >> >>>time
> > > > >> >> >> >>> a
> > > > >> >> >> >>> >> >>page
> > > > >> >> >> >>> >> >> > view
> > > > >> >> >> >>> >> >> > > >>> >> happens. I'm also going to pretend that
> > you
> > > > >>have
> > > > >> >> >>some
> > > > >> >> >> >>> >>state
> > > > >> >> >> >>> >> >> called
> > > > >> >> >> >>> >> >> > > >>> >>"IPGeo"
> > > > >> >> >> >>> >> >> > > >>> >> (e.g. The maxmind data set). In this
> > > example,
> > > > >> >>we'll
> > > > >> >> >> >>>want
> > > > >> >> >> >>> >>to
> > > > >> >> >> >>> >> >>join
> > > > >> >> >> >>> >> >> > the
> > > > >> >> >> >>> >> >> > > >>> >> long/lat geo information from IPGeo to
> > the
> > > > >> >> >> >>>PageViewEvent,
> > > > >> >> >> >>> >>and
> > > > >> >> >> >>> >> >> send
> > > > >> >> >> >>> >> >> > > >>>it
> > > > >> >> >> >>> >> >> > > >>> >>to a
> > > > >> >> >> >>> >> >> > > >>> >> new topic: PageViewEventsWithGeo.
> > > > >> >> >> >>> >> >> > > >>> >>
> > > > >> >> >> >>> >> >> > > >>> >> You have several options on how to
> > > implement
> > > > >> >>this
> > > > >> >> >> >>> example.
> > > > >> >> >> >>> >> >> > > >>> >>
> > > > >> >> >> >>> >> >> > > >>> >> 1. If your joining data set (IPGeo) is
> > > > >> >>relatively
> > > > >> >> >> >>>small
> > > > >> >> >> >>> >>and
> > > > >> >> >> >>> >> >> > changes
> > > > >> >> >> >>> >> >> > > >>> >> infrequently, you can just pack it up
> in
> > > your
> > > > >> >>jar
> > > > >> >> >>or
> > > > >> >> >> >>>.tgz
> > > > >> >> >> >>> >> >>file,
> > > > >> >> >> >>> >> >> > and
> > > > >> >> >> >>> >> >> > > >>> open
> > > > >> >> >> >>> >> >> > > >>> >> it open in every StreamTask.
> > > > >> >> >> >>> >> >> > > >>> >> 2. If your data set is small, but
> changes
> > > > >> >>somewhat
> > > > >> >> >> >>> >> >>frequently,
> > > > >> >> >> >>> >> >> you
> > > > >> >> >> >>> >> >> > > >>>can
> > > > >> >> >> >>> >> >> > > >>> >> throw the data set on some HTTP/HDFS/S3
> > > > >>server
> > > > >> >> >> >>>somewhere,
> > > > >> >> >> >>> >>and
> > > > >> >> >> >>> >> >> have
> > > > >> >> >> >>> >> >> > > >>>your
> > > > >> >> >> >>> >> >> > > >>> >> StreamTask refresh it periodically by
> > > > >> >> >>re-downloading
> > > > >> >> >> >>>it.
> > > > >> >> >> >>> >> >> > > >>> >> 3. You can do remote RPC calls for the
> > > IPGeo
> > > > >> >>data
> > > > >> >> >>on
> > > > >> >> >> >>> every
> > > > >> >> >> >>> >> >>page
> > > > >> >> >> >>> >> >> > view
> > > > >> >> >> >>> >> >> > > >>> >>event
> > > > >> >> >> >>> >> >> > > >>> >> by query some remote service or DB
> (e.g.
> > > > >> >> >>Cassandra).
> > > > >> >> >> >>> >> >> > > >>> >> 4. You can use Samza's state feature to
> > set
> > > > >>your
> > > > >> >> >>IPGeo
> > > > >> >> >> >>> >>data
> > > > >> >> >> >>> >> >>as a
> > > > >> >> >> >>> >> >> > > >>>series
> > > > >> >> >> >>> >> >> > > >>> >>of
> > > > >> >> >> >>> >> >> > > >>> >> messages to a log-compacted Kafka topic
> > > > >> >> >> >>> >> >> > > >>> >> (
> > > > >> >> >> >>> >> >>
> > > > >> >> >>
> > https://cwiki.apache.org/confluence/display/KAFKA/Log+Compaction
> > > > >> >> >> >>> >> >> > ),
> > > > >> >> >> >>> >> >> > > >>> and
> > > > >> >> >> >>> >> >> > > >>> >> configure your Samza job to read this
> > topic
> > > > >>as a
> > > > >> >> >> >>> bootstrap
> > > > >> >> >> >>> >> >> stream
> > > > >> >> >> >>> >> >> > > >>> >> (
> > > > >> >> >> >>> >> >> > > >>> >>
> > > > >> >> >> >>> >> >> > > >>> >>
> > > > >> >> >> >>> >> >> > > >>>
> > > > >> >> >> >>> >> >> > > >>>
> > > > >> >> >> >>> >> >> > >
> > > > >> >> >> >>> >> >>
> > > > >> >> >> >>> >>
> > > > >> >> >> >>>
> > > > >> >> >> >>>
> > > > >> >> >>
> > > > >> >>
> > > > >>
> > > >
> > http://samza.incubator.apache.org/learn/documentation/0.7.0/container/st
> > > > >> >> >> >>> >> >> > > >>>r
> > > > >> >> >> >>> >> >> > > >>> >>e
> > > > >> >> >> >>> >> >> > > >>> >> ams.html).
> > > > >> >> >> >>> >> >> > > >>> >>
> > > > >> >> >> >>> >> >> > > >>> >> For (4), you'd have to partition the
> > IPGeo
> > > > >>state
> > > > >> >> >>topic
> > > > >> >> >> >>> >> >>according
> > > > >> >> >> >>> >> >> > to
> > > > >> >> >> >>> >> >> > > >>>the
> > > > >> >> >> >>> >> >> > > >>> >> same key as PageViewEvent. If
> > PageViewEvent
> > > > >>were
> > > > >> >> >> >>> >>partitioned
> > > > >> >> >> >>> >> >>by,
> > > > >> >> >> >>> >> >> > > >>>say,
> > > > >> >> >> >>> >> >> > > >>> >> member ID, but you want your IPGeo
> state
> > > > >>topic
> > > > >> >>to
> > > > >> >> >>be
> > > > >> >> >> >>> >> >>partitioned
> > > > >> >> >> >>> >> >> > by
> > > > >> >> >> >>> >> >> > > >>>IP
> > > > >> >> >> >>> >> >> > > >>> >> address, then you'd have to have an
> > > upstream
> > > > >>job
> > > > >> >> >>that
> > > > >> >> >> >>> >> >> > re-partitioned
> > > > >> >> >> >>> >> >> > > >>> >> PageViewEvent into some new topic by IP
> > > > >>address.
> > > > >> >> >>This
> > > > >> >> >> >>>new
> > > > >> >> >> >>> >> >>topic
> > > > >> >> >> >>> >> >> > will
> > > > >> >> >> >>> >> >> > > >>> >>have
> > > > >> >> >> >>> >> >> > > >>> >> to have the same number of partitions
> as
> > > the
> > > > >> >>IPGeo
> > > > >> >> >> >>>state
> > > > >> >> >> >>> >> >>topic
> > > > >> >> >> >>> >> >> (if
> > > > >> >> >> >>> >> >> > > >>> IPGeo
> > > > >> >> >> >>> >> >> > > >>> >> has 8 partitions, then the new
> > > > >> >> >> >>>PageViewEventRepartitioned
> > > > >> >> >> >>> >> >>topic
> > > > >> >> >> >>> >> >> > > >>>needs 8
> > > > >> >> >> >>> >> >> > > >>> >>as
> > > > >> >> >> >>> >> >> > > >>> >> well). This will cause your
> > > > >> >> >>PageViewEventRepartitioned
> > > > >> >> >> >>> >>topic
> > > > >> >> >> >>> >> >>and
> > > > >> >> >> >>> >> >> > > >>>your
> > > > >> >> >> >>> >> >> > > >>> >> IPGeo state topic to be aligned such
> that
> > > the
> > > > >> >> >> >>>StreamTask
> > > > >> >> >> >>> >>that
> > > > >> >> >> >>> >> >> gets
> > > > >> >> >> >>> >> >> > > >>>page
> > > > >> >> >> >>> >> >> > > >>> >> views for IP address X will also have
> the
> > > > >>IPGeo
> > > > >> >> >> >>> >>information
> > > > >> >> >> >>> >> >>for
> > > > >> >> >> >>> >> >> IP
> > > > >> >> >> >>> >> >> > > >>> >>address
> > > > >> >> >> >>> >> >> > > >>> >> X.
> > > > >> >> >> >>> >> >> > > >>> >>
> > > > >> >> >> >>> >> >> > > >>> >> Which strategy you pick is really up to
> > > you.
> > > > >>:)
> > > > >> >> >>(4) is
> > > > >> >> >> >>> the
> > > > >> >> >> >>> >> >>most
> > > > >> >> >> >>> >> >> > > >>> >> complicated, but also the most
> flexible,
> > > and
> > > > >> >>most
> > > > >> >> >> >>> >> >>operationally
> > > > >> >> >> >>> >> >> > > >>>sound.
> > > > >> >> >> >>> >> >> > > >>> >>(1)
> > > > >> >> >> >>> >> >> > > >>> >> is the easiest if it fits your needs.
> > > > >> >> >> >>> >> >> > > >>> >>
> > > > >> >> >> >>> >> >> > > >>> >> Cheers,
> > > > >> >> >> >>> >> >> > > >>> >> Chris
> > > > >> >> >> >>> >> >> > > >>> >>
> > > > >> >> >> >>> >> >> > > >>> >> On 8/21/14 10:15 AM, "Shekar Tippur"
> > > > >> >> >> >>><ct...@gmail.com>
> > > > >> >> >> >>> >> >>wrote:
> > > > >> >> >> >>> >> >> > > >>> >>
> > > > >> >> >> >>> >> >> > > >>> >> >Hello,
> > > > >> >> >> >>> >> >> > > >>> >> >
> > > > >> >> >> >>> >> >> > > >>> >> >I am new to Samza. I have just
> installed
> > > > >>Hello
> > > > >> >> >>Samza
> > > > >> >> >> >>>and
> > > > >> >> >> >>> >> >>got it
> > > > >> >> >> >>> >> >> > > >>> >>working.
> > > > >> >> >> >>> >> >> > > >>> >> >
> > > > >> >> >> >>> >> >> > > >>> >> >Here is the use case for which I am
> > trying
> > > > >>to
> > > > >> >>use
> > > > >> >> >> >>>Samza:
> > > > >> >> >> >>> >> >> > > >>> >> >
> > > > >> >> >> >>> >> >> > > >>> >> >
> > > > >> >> >> >>> >> >> > > >>> >> >1. Cache the contextual information
> > which
> > > > >> >>contains
> > > > >> >> >> >>>more
> > > > >> >> >> >>> >> >> > information
> > > > >> >> >> >>> >> >> > > >>> >>about
> > > > >> >> >> >>> >> >> > > >>> >> >the hostname or IP address using
> > > > >> >>Samza/Yarn/Kafka
> > > > >> >> >> >>> >> >> > > >>> >> >2. Collect alert and metric events
> which
> > > > >> >>contain
> > > > >> >> >> >>>either
> > > > >> >> >> >>> >> >> hostname
> > > > >> >> >> >>> >> >> > > >>>or IP
> > > > >> >> >> >>> >> >> > > >>> >> >address
> > > > >> >> >> >>> >> >> > > >>> >> >3. Append contextual information to
> the
> > > > >>alert
> > > > >> >>and
> > > > >> >> >> >>>metric
> > > > >> >> >> >>> >>and
> > > > >> >> >> >>> >> >> > > >>>insert to
> > > > >> >> >> >>> >> >> > > >>> >>a
> > > > >> >> >> >>> >> >> > > >>> >> >Kafka queue from which other
> subscribers
> > > > >>read
> > > > >> >>off
> > > > >> >> >>of.
> > > > >> >> >> >>> >> >> > > >>> >> >
> > > > >> >> >> >>> >> >> > > >>> >> >Can you please shed some light on
> > > > >> >> >> >>> >> >> > > >>> >> >
> > > > >> >> >> >>> >> >> > > >>> >> >1. Is this feasible?
> > > > >> >> >> >>> >> >> > > >>> >> >2. Am I on the right thought process
> > > > >> >> >> >>> >> >> > > >>> >> >3. How do I start
> > > > >> >> >> >>> >> >> > > >>> >> >
> > > > >> >> >> >>> >> >> > > >>> >> >I now have 1 & 2 of them working
> > > > >>disparately. I
> > > > >> >> >>need
> > > > >> >> >> >>>to
> > > > >> >> >> >>> >> >> integrate
> > > > >> >> >> >>> >> >> > > >>> them.
> > > > >> >> >> >>> >> >> > > >>> >> >
> > > > >> >> >> >>> >> >> > > >>> >> >Appreciate any input.
> > > > >> >> >> >>> >> >> > > >>> >> >
> > > > >> >> >> >>> >> >> > > >>> >> >- Shekar
> > > > >> >> >> >>> >> >> > > >>> >>
> > > > >> >> >> >>> >> >> > > >>> >>
> > > > >> >> >> >>> >> >> > > >>>
> > > > >> >> >> >>> >> >> > > >>>
> > > > >> >> >> >>> >> >> > > >>
> > > > >> >> >> >>> >> >> > >
> > > > >> >> >> >>> >> >> > >
> > > > >> >> >> >>> >> >> >
> > > > >> >> >> >>> >> >>
> > > > >> >> >> >>> >>
> > > > >> >> >> >>> >>
> > > > >> >> >> >>>
> > > > >> >> >> >>>
> > > > >> >> >> >>
> > > > >> >> >>
> > > > >> >> >>
> > > > >> >>
> > > > >> >>
> > > > >>
> > > > >>
> > > >
> > > >
> > >
> >
>

Re: Samza as a Caching layer

Posted by Shekar Tippur <ct...@gmail.com>.
Yan,

I do see these jar files

[ctippur@pppdc9prd2y2 incubator-samza-hello-samza]$ ls -ltr
deploy/samza/lib/slf4j-*

-rw------- 1 ctippur domain users 25689 Aug 18 10:45
deploy/samza/lib/slf4j-api-1.6.2.jar

-rw------- 1 ctippur domain users  9752 Aug 18 10:48
deploy/samza/lib/slf4j-log4j12-1.6.2.jar


- Shekar


On Wed, Sep 3, 2014 at 3:13 PM, Yan Fang <ya...@gmail.com> wrote:

> Hi Shekar,
>
> Since you are running local mode, you must have the libs in the directory
> such as *deploy/samza/lib*. Can you see the slf4j-log4j12-1.6.2.jar there?
> (not the tar.gz). I ask this is because sometimes people forget to tar -xvf
> after they add dependencies and recompile, while the local mode reads the
> *deploy/samz*a, instead of the tar.gz file. Thank you.
>
> Cheers,
>
> Fang, Yan
> yanfang724@gmail.com
> +1 (206) 849-4108
>
>
> On Wed, Sep 3, 2014 at 2:59 PM, Shekar Tippur <ct...@gmail.com> wrote:
>
> > I do see this
> >
> > lib/slf4j-log4j12-1.6.2.jar
> >
> > In the tgz file. Also, this message seem to me more of a noise.
> >
> > I am not sure if we are getting sidetracked on this.
> >
> > In the file WikipediaFeedStreamTask.java, I changed log.info to
> > System.out.println - thinking that this should throw the message to the
> > console.
> >
> > Is this the right place to trap the incoming message?
> >
> >
> > $ cat
> >
> >
> samza-wikipedia/src/main/java/samza/examples/wikipedia/task/WikipediaFeedStreamTask.java
> >
> > /*
> >
> >  * Licensed to the Apache Software Foundation (ASF) under one
> >
> >  * or more contributor license agreements.  See the NOTICE file
> >
> >  * distributed with this work for additional information
> >
> >  * regarding copyright ownership.  The ASF licenses this file
> >
> >  * to you under the Apache License, Version 2.0 (the
> >
> >  * "License"); you may not use this file except in compliance
> >
> >  * with the License.  You may obtain a copy of the License at
> >
> >  *
> >
> >  *   http://www.apache.org/licenses/LICENSE-2.0
> >
> >  *
> >
> >  * Unless required by applicable law or agreed to in writing,
> >
> >  * software distributed under the License is distributed on an
> >
> >  * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
> >
> >  * KIND, either express or implied.  See the License for the
> >
> >  * specific language governing permissions and limitations
> >
> >  * under the License.
> >
> >  */
> >
> >
> > package samza.examples.wikipedia.task;
> >
> >
> > import java.util.Map;
> >
> > import org.apache.samza.system.IncomingMessageEnvelope;
> >
> > import org.apache.samza.system.OutgoingMessageEnvelope;
> >
> > import org.apache.samza.system.SystemStream;
> >
> > import org.apache.samza.task.MessageCollector;
> >
> > import org.apache.samza.task.StreamTask;
> >
> > import org.apache.samza.task.TaskCoordinator;
> >
> > import samza.examples.wikipedia.system.WikipediaFeed.WikipediaFeedEvent;
> >
> >
> > /**
> >
> >  * This task is very simple. All it does is take messages that it
> receives,
> > and
> >
> >  * sends them to a Kafka topic called wikipedia-raw.
> >
> >  */
> >
> > public class WikipediaFeedStreamTask implements StreamTask {
> >
> >   private static final SystemStream OUTPUT_STREAM = new
> > SystemStream("kafka", "wikipedia-raw");
> >
> >
> >   @Override
> >
> >   public void process(IncomingMessageEnvelope envelope, MessageCollector
> > collector, TaskCoordinator coordinator) {
> >
> >     Map<String, Object> outgoingMap =
> > WikipediaFeedEvent.toMap((WikipediaFeedEvent) envelope.getMessage());
> >
> >     System.out.println(envelope.getMessage().toString());
> >
> >     collector.send(new OutgoingMessageEnvelope(OUTPUT_STREAM,
> > outgoingMap));
> >
> >   }
> >
> > }
> >
> >
> >
> >
> > On Wed, Sep 3, 2014 at 2:48 PM, Chris Riccomini <
> > criccomini@linkedin.com.invalid> wrote:
> >
> > > Hey Shekar,
> > >
> > > Sorry that you're having such a tough time with this. I'll keep trying
> to
> > > help as best I can.
> > >
> > > Do you see slf4j-log4j12 in your job's .tgz file?
> > >
> > > Cheers,
> > > Chris
> > >
> > > On 9/3/14 2:46 PM, "Shekar Tippur" <ct...@gmail.com> wrote:
> > >
> > > >Chris,
> > > >
> > > >I have added the below dependencies and I still get the same message.
> > > >
> > > >
> > > >      <dependency>
> > > >
> > > >        <groupId>org.slf4j</groupId>
> > > >
> > > >        <artifactId>slf4j-api</artifactId>
> > > >
> > > >        <version>1.7.7</version>
> > > >
> > > >      </dependency>
> > > >
> > > >      <dependency>
> > > >
> > > >          <groupId>org.slf4j</groupId>
> > > >
> > > >          <artifactId>slf4j-log4j12</artifactId>
> > > >
> > > >          <version>1.5.6</version>
> > > >
> > > >      </dependency>
> > > >
> > > >
> > > >On Wed, Sep 3, 2014 at 9:01 AM, Chris Riccomini <
> > > >criccomini@linkedin.com.invalid> wrote:
> > > >
> > > >> Hey Shekar,
> > > >>
> > > >> The SLF4J stuff is saying that you don't have an slf4j binding on
> your
> > > >> classpath. Try adding slf4j-log4j as a runtime dependency on your
> > > >>project.
> > > >>
> > > >> Cheers,
> > > >> Chris
> > > >>
> > > >> On 9/2/14 3:24 PM, "Shekar Tippur" <ct...@gmail.com> wrote:
> > > >>
> > > >> >Chris ,
> > > >> >
> > > >> >In the current state, I just want Samza to connect to 127.0.0.1.
> > > >> >
> > > >> >I have set YARN_HOME to
> > > >> >
> > > >> >$ echo $YARN_HOME
> > > >> >
> > > >> >/home/ctippur/hello-samza/deploy/yarn
> > > >> >
> > > >> >I still dont see anything on hadoop console.
> > > >> >
> > > >> > Also, I see this during startup
> > > >> >
> > > >> >
> > > >> >SLF4J: Failed to load class "org.slf4j.impl.StaticLoggerBinder".
> > > >> >
> > > >> >SLF4J: Defaulting to no-operation (NOP) logger implementation
> > > >> >
> > > >> >SLF4J: See http://www.slf4j.org/codes.html#StaticLoggerBinder for
> > > >>further
> > > >> >details.
> > > >> >
> > > >> >SLF4J: Failed to load class "org.slf4j.impl.StaticLoggerBinder".
> > > >> >
> > > >> >SLF4J: Defaulting to no-operation (NOP) logger implementation
> > > >> >
> > > >> >SLF4J: See http://www.slf4j.org/codes.html#StaticLoggerBinder for
> > > >>further
> > > >> >details.
> > > >> >
> > > >> >- Shekar
> > > >> >
> > > >> >
> > > >> >On Tue, Sep 2, 2014 at 2:20 PM, Chris Riccomini <
> > > >> >criccomini@linkedin.com.invalid> wrote:
> > > >> >
> > > >> >> Hey Shekar,
> > > >> >>
> > > >> >> Ah ha. In that case, do you expect your SamzaContainer to try to
> > > >>connect
> > > >> >> to the RM at 127.0.0.1, or do you expect it to try to connect to
> > some
> > > >> >> remote RM? If you expect it to try and connect to a remote RM,
> it's
> > > >>not
> > > >> >> doing that. Perhaps because YARN_HOME isn't set.
> > > >> >>
> > > >> >> If you go to your RM's web interface, how many active nodes do
> you
> > > >>see
> > > >> >> listed?
> > > >> >>
> > > >> >> Cheers,
> > > >> >> Chris
> > > >> >>
> > > >> >> On 9/2/14 2:17 PM, "Shekar Tippur" <ct...@gmail.com> wrote:
> > > >> >>
> > > >> >> >Chris ..
> > > >> >> >
> > > >> >> >I am using a rhel server to host all the components (Yarn,
> kafka,
> > > >> >>samza).
> > > >> >> >I
> > > >> >> >dont have ACLs open to wikipedia.
> > > >> >> >I am following ..
> > > >> >> >
> > > >> >>
> > > >> >>
> > > >>
> > > >>
> > >
> >
> http://samza.incubator.apache.org/learn/tutorials/latest/run-hello-samza-
> > > >> >>w
> > > >> >> >ithout-internet.html
> > > >> >> >
> > > >> >> >- Shekar
> > > >> >> >
> > > >> >> >
> > > >> >> >
> > > >> >> >On Tue, Sep 2, 2014 at 2:13 PM, Chris Riccomini <
> > > >> >> >criccomini@linkedin.com.invalid> wrote:
> > > >> >> >
> > > >> >> >> Hey Shekar,
> > > >> >> >>
> > > >> >> >> Can you try changing that to:
> > > >> >> >>
> > > >> >> >>   http://127.0.0.1:8088/cluster
> > > >> >> >>
> > > >> >> >>
> > > >> >> >> And see if you can connect?
> > > >> >> >>
> > > >> >> >> Cheers,
> > > >> >> >> Chris
> > > >> >> >>
> > > >> >> >> On 9/2/14 1:21 PM, "Shekar Tippur" <ct...@gmail.com> wrote:
> > > >> >> >>
> > > >> >> >> >Other observation is ..
> > > >> >> >> >
> > > >> >> >> >http://10.132.62.185:8088/cluster shows that no applications
> > are
> > > >> >> >>running.
> > > >> >> >> >
> > > >> >> >> >- Shekar
> > > >> >> >> >
> > > >> >> >> >
> > > >> >> >> >
> > > >> >> >> >
> > > >> >> >> >On Tue, Sep 2, 2014 at 1:15 PM, Shekar Tippur <
> > ctippur@gmail.com
> > > >
> > > >> >> >>wrote:
> > > >> >> >> >
> > > >> >> >> >> Yarn seem to be running ..
> > > >> >> >> >>
> > > >> >> >> >> yarn      5462  0.0  2.0 1641296 161404 ?      Sl   Jun20
> > > >>95:26
> > > >> >> >> >> /usr/java/jdk1.6.0_31/bin/java -Dproc_resourcemanager
> > -Xmx1000m
> > > >> >> >> >> -Dhadoop.log.dir=/var/log/hadoop-yarn
> > > >> >> >> >>-Dyarn.log.dir=/var/log/hadoop-yarn
> > > >> >> >> >>
> -Dhadoop.log.file=yarn-yarn-resourcemanager-pppdc9prd2y2.log
> > > >> >> >> >> -Dyarn.log.file=yarn-yarn-resourcemanager-pppdc9prd2y2.log
> > > >> >> >> >> -Dyarn.home.dir=/usr/lib/hadoop-yarn
> > > >> >> >> >>-Dhadoop.home.dir=/usr/lib/hadoop-yarn
> > > >> >> >> >> -Dhadoop.root.logger=INFO,RFA -Dyarn.root.logger=INFO,RFA
> > > >> >> >> >> -Djava.library.path=/usr/lib/hadoop/lib/native -classpath
> > > >> >> >> >>
> > > >> >> >>
> > > >> >>
> > > >>
> > >
> >
> >>>>>>>>/etc/hadoop/conf:/etc/hadoop/conf:/etc/hadoop/conf:/usr/lib/hadoop/
> > > >>>>>>>>li
> > > >> >>>>>>b/
> > > >> >> >>>>*:
> > > >> >> >>
> > > >> >>
> > > >>
> > >
> >
> >>>>>>>>/usr/lib/hadoop/.//*:/usr/lib/hadoop-hdfs/./:/usr/lib/hadoop-hdfs/l
> > > >>>>>>>>ib
> > > >> >>>>>>/*
> > > >> >> >>>>:/
> > > >> >> >>
> > > >> >>
> > > >>
> > >
> >
> >>>>>>>>usr/lib/hadoop-hdfs/.//*:/usr/lib/hadoop-yarn/lib/*:/usr/lib/hadoop
> > > >>>>>>>>-y
> > > >> >>>>>>ar
> > > >> >> >>>>n/
> > > >> >> >>
> > > >> >>
> > > >>
> > >
> >
> >>>>>>>>.//*:/usr/lib/hadoop-mapreduce/lib/*:/usr/lib/hadoop-mapreduce/.//*
> > > >>>>>>>>:/
> > > >> >>>>>>us
> > > >> >> >>>>r/
> > > >> >> >>
> > > >> >>
> > > >>
> > >
> >
> >>>>>>>>lib/hadoop-yarn/.//*:/usr/lib/hadoop-yarn/lib/*:/etc/hadoop/conf/rm
> > > >>>>>>>>-c
> > > >> >>>>>>on
> > > >> >> >>>>fi
> > > >> >> >> >>g/log4j.properties
> > > >> >> >> >>
> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager
> > > >> >> >> >>
> > > >> >> >> >> I can tail kafka topic as well ..
> > > >> >> >> >>
> > > >> >> >> >> deploy/kafka/bin/kafka-console-consumer.sh  --zookeeper
> > > >> >> >>localhost:2181
> > > >> >> >> >>--topic wikipedia-raw
> > > >> >> >> >>
> > > >> >> >> >>
> > > >> >> >> >>
> > > >> >> >> >>
> > > >> >> >> >> On Tue, Sep 2, 2014 at 1:04 PM, Chris Riccomini <
> > > >> >> >> >> criccomini@linkedin.com.invalid> wrote:
> > > >> >> >> >>
> > > >> >> >> >>> Hey Shekar,
> > > >> >> >> >>>
> > > >> >> >> >>> It looks like your job is hanging trying to connect to the
> > RM
> > > >>on
> > > >> >> >>your
> > > >> >> >> >>> localhost. I thought that you said that your job was
> running
> > > >>in
> > > >> >> >>local
> > > >> >> >> >>> mode. If so, it should be using the LocalJobFactory. If
> not,
> > > >>and
> > > >> >>you
> > > >> >> >> >>> intend to run on YARN, is your YARN RM up and running on
> > > >> >>localhost?
> > > >> >> >> >>>
> > > >> >> >> >>> Cheers,
> > > >> >> >> >>> Chris
> > > >> >> >> >>>
> > > >> >> >> >>> On 9/2/14 12:22 PM, "Shekar Tippur" <ct...@gmail.com>
> > > wrote:
> > > >> >> >> >>>
> > > >> >> >> >>> >Chris ..
> > > >> >> >> >>> >
> > > >> >> >> >>> >$ cat ./deploy/samza/undefined-samza-container-name.log
> > > >> >> >> >>> >
> > > >> >> >> >>> >2014-09-02 11:17:58 JobRunner [INFO] job factory:
> > > >> >> >> >>> >org.apache.samza.job.yarn.YarnJobFactory
> > > >> >> >> >>> >
> > > >> >> >> >>> >2014-09-02 11:17:59 ClientHelper [INFO] trying to connect
> > to
> > > >>RM
> > > >> >> >> >>> >127.0.0.1:8032
> > > >> >> >> >>> >
> > > >> >> >> >>> >2014-09-02 11:17:59 NativeCodeLoader [WARN] Unable to
> load
> > > >> >> >> >>>native-hadoop
> > > >> >> >> >>> >library for your platform... using builtin-java classes
> > where
> > > >> >> >> >>>applicable
> > > >> >> >> >>> >
> > > >> >> >> >>> >2014-09-02 11:17:59 RMProxy [INFO] Connecting to
> > > >>ResourceManager
> > > >> >> >>at /
> > > >> >> >> >>> >127.0.0.1:8032
> > > >> >> >> >>> >
> > > >> >> >> >>> >
> > > >> >> >> >>> >and Log4j ..
> > > >> >> >> >>> >
> > > >> >> >> >>> ><!DOCTYPE log4j:configuration SYSTEM "log4j.dtd">
> > > >> >> >> >>> >
> > > >> >> >> >>> ><log4j:configuration
> > > >> >> >>xmlns:log4j="http://jakarta.apache.org/log4j/">
> > > >> >> >> >>> >
> > > >> >> >> >>> >  <appender name="RollingAppender"
> > > >> >> >> >>> >class="org.apache.log4j.DailyRollingFileAppender">
> > > >> >> >> >>> >
> > > >> >> >> >>> >     <param name="File"
> > > >> >> >> >>> >value="${samza.log.dir}/${samza.container.name}.log"
> > > >> >> >> >>> >/>
> > > >> >> >> >>> >
> > > >> >> >> >>> >     <param name="DatePattern" value="'.'yyyy-MM-dd" />
> > > >> >> >> >>> >
> > > >> >> >> >>> >     <layout class="org.apache.log4j.PatternLayout">
> > > >> >> >> >>> >
> > > >> >> >> >>> >      <param name="ConversionPattern"
> value="%d{yyyy-MM-dd
> > > >> >> >>HH:mm:ss}
> > > >> >> >> >>> %c{1}
> > > >> >> >> >>> >[%p] %m%n" />
> > > >> >> >> >>> >
> > > >> >> >> >>> >     </layout>
> > > >> >> >> >>> >
> > > >> >> >> >>> >  </appender>
> > > >> >> >> >>> >
> > > >> >> >> >>> >  <root>
> > > >> >> >> >>> >
> > > >> >> >> >>> >    <priority value="info" />
> > > >> >> >> >>> >
> > > >> >> >> >>> >    <appender-ref ref="RollingAppender"/>
> > > >> >> >> >>> >
> > > >> >> >> >>> >  </root>
> > > >> >> >> >>> >
> > > >> >> >> >>> ></log4j:configuration>
> > > >> >> >> >>> >
> > > >> >> >> >>> >
> > > >> >> >> >>> >On Tue, Sep 2, 2014 at 12:02 PM, Chris Riccomini <
> > > >> >> >> >>> >criccomini@linkedin.com.invalid> wrote:
> > > >> >> >> >>> >
> > > >> >> >> >>> >> Hey Shekar,
> > > >> >> >> >>> >>
> > > >> >> >> >>> >> Can you attach your log files? I'm wondering if it's a
> > > >> >> >> >>>mis-configured
> > > >> >> >> >>> >> log4j.xml (or missing slf4j-log4j jar), which is
> leading
> > to
> > > >> >> >>nearly
> > > >> >> >> >>> empty
> > > >> >> >> >>> >> log files. Also, I'm wondering if the job starts fully.
> > > >> >>Anything
> > > >> >> >>you
> > > >> >> >> >>> can
> > > >> >> >> >>> >> attach would be helpful.
> > > >> >> >> >>> >>
> > > >> >> >> >>> >> Cheers,
> > > >> >> >> >>> >> Chris
> > > >> >> >> >>> >>
> > > >> >> >> >>> >> On 9/2/14 11:43 AM, "Shekar Tippur" <ctippur@gmail.com
> >
> > > >> wrote:
> > > >> >> >> >>> >>
> > > >> >> >> >>> >> >I am running in local mode.
> > > >> >> >> >>> >> >
> > > >> >> >> >>> >> >S
> > > >> >> >> >>> >> >On Sep 2, 2014 11:42 AM, "Yan Fang" <
> > yanfang724@gmail.com
> > > >
> > > >> >> >>wrote:
> > > >> >> >> >>> >> >
> > > >> >> >> >>> >> >> Hi Shekar.
> > > >> >> >> >>> >> >>
> > > >> >> >> >>> >> >> Are you running job in local mode or yarn mode? If
> > yarn
> > > >> >>mode,
> > > >> >> >>the
> > > >> >> >> >>> log
> > > >> >> >> >>> >> >>is in
> > > >> >> >> >>> >> >> the yarn's container log.
> > > >> >> >> >>> >> >>
> > > >> >> >> >>> >> >> Thanks,
> > > >> >> >> >>> >> >>
> > > >> >> >> >>> >> >> Fang, Yan
> > > >> >> >> >>> >> >> yanfang724@gmail.com
> > > >> >> >> >>> >> >> +1 (206) 849-4108
> > > >> >> >> >>> >> >>
> > > >> >> >> >>> >> >>
> > > >> >> >> >>> >> >> On Tue, Sep 2, 2014 at 11:36 AM, Shekar Tippur
> > > >> >> >> >>><ct...@gmail.com>
> > > >> >> >> >>> >> >>wrote:
> > > >> >> >> >>> >> >>
> > > >> >> >> >>> >> >> > Chris,
> > > >> >> >> >>> >> >> >
> > > >> >> >> >>> >> >> > Got some time to play around a bit more.
> > > >> >> >> >>> >> >> > I tried to edit
> > > >> >> >> >>> >> >> >
> > > >> >> >> >>> >> >> >
> > > >> >> >> >>> >> >>
> > > >> >> >> >>> >>
> > > >> >> >> >>>
> > > >> >> >> >>>
> > > >> >> >>
> > > >> >>
> > > >>
> > >
> >
> >>>>>>>>>>>>>samza-wikipedia/src/main/java/samza/examples/wikipedia/task/Wi
> > > >>>>>>>>>>>>>ki
> > > >> >>>>>>>>>>>pe
> > > >> >> >>>>>>>>>di
> > > >> >> >> >>>>>>>aFe
> > > >> >> >> >>> >>>>ed
> > > >> >> >> >>> >> >>StreamTask.java
> > > >> >> >> >>> >> >> > to add a logger info statement to tap the incoming
> > > >> >>message.
> > > >> >> >>I
> > > >> >> >> >>>dont
> > > >> >> >> >>> >>see
> > > >> >> >> >>> >> >> the
> > > >> >> >> >>> >> >> > messages being printed to the log file.
> > > >> >> >> >>> >> >> >
> > > >> >> >> >>> >> >> > Is this the right place to start?
> > > >> >> >> >>> >> >> >
> > > >> >> >> >>> >> >> > public class WikipediaFeedStreamTask implements
> > > >> >>StreamTask {
> > > >> >> >> >>> >> >> >
> > > >> >> >> >>> >> >> >   private static final SystemStream OUTPUT_STREAM
> =
> > > >>new
> > > >> >> >> >>> >> >> > SystemStream("kafka",
> > > >> >> >> >>> >> >> > "wikipedia-raw");
> > > >> >> >> >>> >> >> >
> > > >> >> >> >>> >> >> >   private static final Logger log =
> > > >> >>LoggerFactory.getLogger
> > > >> >> >> >>> >> >> > (WikipediaFeedStreamTask.class);
> > > >> >> >> >>> >> >> >
> > > >> >> >> >>> >> >> >   @Override
> > > >> >> >> >>> >> >> >
> > > >> >> >> >>> >> >> >   public void process(IncomingMessageEnvelope
> > > >>envelope,
> > > >> >> >> >>> >> >>MessageCollector
> > > >> >> >> >>> >> >> > collector, TaskCoordinator coordinator) {
> > > >> >> >> >>> >> >> >
> > > >> >> >> >>> >> >> >     Map<String, Object> outgoingMap =
> > > >> >> >> >>> >> >> > WikipediaFeedEvent.toMap((WikipediaFeedEvent)
> > > >> >> >> >>> >>envelope.getMessage());
> > > >> >> >> >>> >> >> >
> > > >> >> >> >>> >> >> >     log.info(envelope.getMessage().toString());
> > > >> >> >> >>> >> >> >
> > > >> >> >> >>> >> >> >     collector.send(new
> > > >> >> >>OutgoingMessageEnvelope(OUTPUT_STREAM,
> > > >> >> >> >>> >> >> > outgoingMap));
> > > >> >> >> >>> >> >> >
> > > >> >> >> >>> >> >> >   }
> > > >> >> >> >>> >> >> >
> > > >> >> >> >>> >> >> > }
> > > >> >> >> >>> >> >> >
> > > >> >> >> >>> >> >> >
> > > >> >> >> >>> >> >> > On Mon, Aug 25, 2014 at 9:01 AM, Chris Riccomini <
> > > >> >> >> >>> >> >> > criccomini@linkedin.com.invalid> wrote:
> > > >> >> >> >>> >> >> >
> > > >> >> >> >>> >> >> > > Hey Shekar,
> > > >> >> >> >>> >> >> > >
> > > >> >> >> >>> >> >> > > Your thought process is on the right track. It's
> > > >> >>probably
> > > >> >> >> >>>best
> > > >> >> >> >>> to
> > > >> >> >> >>> >> >>start
> > > >> >> >> >>> >> >> > > with hello-samza, and modify it to get what you
> > > >>want.
> > > >> >>To
> > > >> >> >> >>>start
> > > >> >> >> >>> >>with,
> > > >> >> >> >>> >> >> > > you'll want to:
> > > >> >> >> >>> >> >> > >
> > > >> >> >> >>> >> >> > > 1. Write a simple StreamTask that just does
> > > >>something
> > > >> >> >>silly
> > > >> >> >> >>>like
> > > >> >> >> >>> >> >>just
> > > >> >> >> >>> >> >> > > print messages that it receives.
> > > >> >> >> >>> >> >> > > 2. Write a configuration for the job that
> consumes
> > > >>from
> > > >> >> >>just
> > > >> >> >> >>>the
> > > >> >> >> >>> >> >>stream
> > > >> >> >> >>> >> >> > > (alerts from different sources).
> > > >> >> >> >>> >> >> > > 3. Run this to make sure you've got it working.
> > > >> >> >> >>> >> >> > > 4. Now add your table join. This can be either a
> > > >> >> >>change-data
> > > >> >> >> >>> >>capture
> > > >> >> >> >>> >> >> > (CDC)
> > > >> >> >> >>> >> >> > > stream, or via a remote DB call.
> > > >> >> >> >>> >> >> > >
> > > >> >> >> >>> >> >> > > That should get you to a point where you've got
> > your
> > > >> >>job
> > > >> >> >>up
> > > >> >> >> >>>and
> > > >> >> >> >>> >> >> running.
> > > >> >> >> >>> >> >> > > From there, you could create your own Maven
> > project,
> > > >> >>and
> > > >> >> >> >>>migrate
> > > >> >> >> >>> >> >>your
> > > >> >> >> >>> >> >> > code
> > > >> >> >> >>> >> >> > > over accordingly.
> > > >> >> >> >>> >> >> > >
> > > >> >> >> >>> >> >> > > Cheers,
> > > >> >> >> >>> >> >> > > Chris
> > > >> >> >> >>> >> >> > >
> > > >> >> >> >>> >> >> > > On 8/24/14 1:42 AM, "Shekar Tippur"
> > > >><ctippur@gmail.com
> > > >> >
> > > >> >> >> >>>wrote:
> > > >> >> >> >>> >> >> > >
> > > >> >> >> >>> >> >> > > >Chris,
> > > >> >> >> >>> >> >> > > >
> > > >> >> >> >>> >> >> > > >I have gone thro the documentation and decided
> > that
> > > >> >>the
> > > >> >> >> >>>option
> > > >> >> >> >>> >> >>that is
> > > >> >> >> >>> >> >> > > >most
> > > >> >> >> >>> >> >> > > >suitable for me is stream-table.
> > > >> >> >> >>> >> >> > > >
> > > >> >> >> >>> >> >> > > >I see the following things:
> > > >> >> >> >>> >> >> > > >
> > > >> >> >> >>> >> >> > > >1. Point samza to a table (database)
> > > >> >> >> >>> >> >> > > >2. Point Samza to a stream - Alert stream from
> > > >> >>different
> > > >> >> >> >>> sources
> > > >> >> >> >>> >> >> > > >3. Join key like a hostname
> > > >> >> >> >>> >> >> > > >
> > > >> >> >> >>> >> >> > > >I have Hello Samza working. To extend that to
> do
> > > >>what
> > > >> >>my
> > > >> >> >> >>>needs
> > > >> >> >> >>> >> >>are, I
> > > >> >> >> >>> >> >> am
> > > >> >> >> >>> >> >> > > >not sure where to start (Needs more code change
> > OR
> > > >> >> >> >>> configuration
> > > >> >> >> >>> >> >> changes
> > > >> >> >> >>> >> >> > > >OR
> > > >> >> >> >>> >> >> > > >both)?
> > > >> >> >> >>> >> >> > > >
> > > >> >> >> >>> >> >> > > >I have gone thro
> > > >> >> >> >>> >> >> > > >
> > > >> >> >> >>> >> >> >
> > > >> >> >> >>> >> >>
> > > >> >> >> >>> >> >>
> > > >> >> >> >>> >>
> > > >> >> >> >>> >>
> > > >> >> >> >>>
> > > >> >> >> >>>
> > > >> >> >>
> > > >> >>
> > > >>
> > >
> http://samza.incubator.apache.org/learn/documentation/latest/api/overvie
> > > >> >> >> >>>w
> > > >> >> >> >>> >> >> > > .
> > > >> >> >> >>> >> >> > > >html
> > > >> >> >> >>> >> >> > > >
> > > >> >> >> >>> >> >> > > >Is my thought process on the right track? Can
> you
> > > >> >>please
> > > >> >> >> >>>point
> > > >> >> >> >>> >>me
> > > >> >> >> >>> >> >>to
> > > >> >> >> >>> >> >> the
> > > >> >> >> >>> >> >> > > >right direction?
> > > >> >> >> >>> >> >> > > >
> > > >> >> >> >>> >> >> > > >- Shekar
> > > >> >> >> >>> >> >> > > >
> > > >> >> >> >>> >> >> > > >
> > > >> >> >> >>> >> >> > > >
> > > >> >> >> >>> >> >> > > >On Thu, Aug 21, 2014 at 1:08 PM, Shekar Tippur
> > > >> >> >> >>> >><ct...@gmail.com>
> > > >> >> >> >>> >> >> > wrote:
> > > >> >> >> >>> >> >> > > >
> > > >> >> >> >>> >> >> > > >> Chris,
> > > >> >> >> >>> >> >> > > >>
> > > >> >> >> >>> >> >> > > >> This is perfectly good answer. I will start
> > > >>poking
> > > >> >>more
> > > >> >> >> >>>into
> > > >> >> >> >>> >> >>option
> > > >> >> >> >>> >> >> > #4.
> > > >> >> >> >>> >> >> > > >>
> > > >> >> >> >>> >> >> > > >> - Shekar
> > > >> >> >> >>> >> >> > > >>
> > > >> >> >> >>> >> >> > > >>
> > > >> >> >> >>> >> >> > > >> On Thu, Aug 21, 2014 at 1:05 PM, Chris
> > Riccomini
> > > >><
> > > >> >> >> >>> >> >> > > >> criccomini@linkedin.com.invalid> wrote:
> > > >> >> >> >>> >> >> > > >>
> > > >> >> >> >>> >> >> > > >>> Hey Shekar,
> > > >> >> >> >>> >> >> > > >>>
> > > >> >> >> >>> >> >> > > >>> Your two options are really (3) or (4),
> then.
> > > >>You
> > > >> >>can
> > > >> >> >> >>>either
> > > >> >> >> >>> >>run
> > > >> >> >> >>> >> >> some
> > > >> >> >> >>> >> >> > > >>> external DB that holds the data set, and you
> > can
> > > >> >> >>query it
> > > >> >> >> >>> >>from a
> > > >> >> >> >>> >> >> > > >>> StreamTask, or you can use Samza's state
> store
> > > >> >> >>feature to
> > > >> >> >> >>> >>push
> > > >> >> >> >>> >> >>data
> > > >> >> >> >>> >> >> > > >>>into a
> > > >> >> >> >>> >> >> > > >>> stream that you can then store in a
> > partitioned
> > > >> >> >>key-value
> > > >> >> >> >>> >>store
> > > >> >> >> >>> >> >> along
> > > >> >> >> >>> >> >> > > >>>with
> > > >> >> >> >>> >> >> > > >>> your StreamTasks. There is some
> documentation
> > > >>here
> > > >> >> >>about
> > > >> >> >> >>>the
> > > >> >> >> >>> >> >>state
> > > >> >> >> >>> >> >> > > >>>store
> > > >> >> >> >>> >> >> > > >>> approach:
> > > >> >> >> >>> >> >> > > >>>
> > > >> >> >> >>> >> >> > > >>>
> > > >> >> >> >>> >> >> > > >>>
> > > >> >> >> >>> >> >> > > >>>
> > > >> >> >> >>> >> >> > >
> > > >> >> >> >>> >> >>
> > > >> >> >> >>> >>
> > > >> >> >> >>>
> > > >> >> >> >>>
> > > >> >> >>
> > > >> >>
> > > >>
> > >
> http://samza.incubator.apache.org/learn/documentation/0.7.0/container/st
> > > >> >> >> >>> >> >> > > >>>ate
> > > >> >> >> >>> >> >> > > >>> -management.html
> > > >> >> >> >>> >> >> > > >>>
> > > >> >> >> >>> >> >> > > >>><
> > > >> >> >> >>> >> >> > >
> > > >> >> >> >>> >> >>
> > > >> >> >> >>>
> > > >> >> >> >>>>>
> > > >> >> >>
> > > >> >>
> > > http://samza.incubator.apache.org/learn/documentation/0.7.0/container/
> > > >> >> >> >>>>>s
> > > >> >> >> >>> >> >> > > >>>tate-management.html>
> > > >> >> >> >>> >> >> > > >>>
> > > >> >> >> >>> >> >> > > >>>
> > > >> >> >> >>> >> >> > > >>> (4) is going to require more up front effort
> > > >>from
> > > >> >>you,
> > > >> >> >> >>>since
> > > >> >> >> >>> >> >>you'll
> > > >> >> >> >>> >> >> > > >>>have
> > > >> >> >> >>> >> >> > > >>> to understand how Kafka's partitioning model
> > > >>works,
> > > >> >> >>and
> > > >> >> >> >>> setup
> > > >> >> >> >>> >> >>some
> > > >> >> >> >>> >> >> > > >>> pipeline to push the updates for your state.
> > In
> > > >>the
> > > >> >> >>long
> > > >> >> >> >>> >>run, I
> > > >> >> >> >>> >> >> > believe
> > > >> >> >> >>> >> >> > > >>> it's the better approach, though. Local
> > lookups
> > > >>on
> > > >> >>a
> > > >> >> >> >>> >>key-value
> > > >> >> >> >>> >> >> store
> > > >> >> >> >>> >> >> > > >>> should be faster than doing remote RPC calls
> > to
> > > >>a
> > > >> >>DB
> > > >> >> >>for
> > > >> >> >> >>> >>every
> > > >> >> >> >>> >> >> > message.
> > > >> >> >> >>> >> >> > > >>>
> > > >> >> >> >>> >> >> > > >>> I'm sorry I can't give you a more definitive
> > > >> >>answer.
> > > >> >> >>It's
> > > >> >> >> >>> >>really
> > > >> >> >> >>> >> >> > about
> > > >> >> >> >>> >> >> > > >>> trade-offs.
> > > >> >> >> >>> >> >> > > >>>
> > > >> >> >> >>> >> >> > > >>> Cheers,
> > > >> >> >> >>> >> >> > > >>> Chris
> > > >> >> >> >>> >> >> > > >>>
> > > >> >> >> >>> >> >> > > >>> On 8/21/14 12:22 PM, "Shekar Tippur"
> > > >> >> >><ct...@gmail.com>
> > > >> >> >> >>> >>wrote:
> > > >> >> >> >>> >> >> > > >>>
> > > >> >> >> >>> >> >> > > >>> >Chris,
> > > >> >> >> >>> >> >> > > >>> >
> > > >> >> >> >>> >> >> > > >>> >A big thanks for a swift response. The data
> > > >>set is
> > > >> >> >>huge
> > > >> >> >> >>>and
> > > >> >> >> >>> >>the
> > > >> >> >> >>> >> >> > > >>>frequency
> > > >> >> >> >>> >> >> > > >>> >is in burst.
> > > >> >> >> >>> >> >> > > >>> >What do you suggest?
> > > >> >> >> >>> >> >> > > >>> >
> > > >> >> >> >>> >> >> > > >>> >- Shekar
> > > >> >> >> >>> >> >> > > >>> >
> > > >> >> >> >>> >> >> > > >>> >
> > > >> >> >> >>> >> >> > > >>> >On Thu, Aug 21, 2014 at 11:05 AM, Chris
> > > >>Riccomini
> > > >> >><
> > > >> >> >> >>> >> >> > > >>> >criccomini@linkedin.com.invalid> wrote:
> > > >> >> >> >>> >> >> > > >>> >
> > > >> >> >> >>> >> >> > > >>> >> Hey Shekar,
> > > >> >> >> >>> >> >> > > >>> >>
> > > >> >> >> >>> >> >> > > >>> >> This is feasible, and you are on the
> right
> > > >> >>thought
> > > >> >> >> >>> >>process.
> > > >> >> >> >>> >> >> > > >>> >>
> > > >> >> >> >>> >> >> > > >>> >> For the sake of discussion, I'm going to
> > > >>pretend
> > > >> >> >>that
> > > >> >> >> >>>you
> > > >> >> >> >>> >> >>have a
> > > >> >> >> >>> >> >> > > >>>Kafka
> > > >> >> >> >>> >> >> > > >>> >> topic called "PageViewEvent", which has
> > just
> > > >> >>the IP
> > > >> >> >> >>> >>address
> > > >> >> >> >>> >> >>that
> > > >> >> >> >>> >> >> > was
> > > >> >> >> >>> >> >> > > >>> >>used
> > > >> >> >> >>> >> >> > > >>> >> to view a page. These messages will be
> > logged
> > > >> >>every
> > > >> >> >> >>>time
> > > >> >> >> >>> a
> > > >> >> >> >>> >> >>page
> > > >> >> >> >>> >> >> > view
> > > >> >> >> >>> >> >> > > >>> >> happens. I'm also going to pretend that
> you
> > > >>have
> > > >> >> >>some
> > > >> >> >> >>> >>state
> > > >> >> >> >>> >> >> called
> > > >> >> >> >>> >> >> > > >>> >>"IPGeo"
> > > >> >> >> >>> >> >> > > >>> >> (e.g. The maxmind data set). In this
> > example,
> > > >> >>we'll
> > > >> >> >> >>>want
> > > >> >> >> >>> >>to
> > > >> >> >> >>> >> >>join
> > > >> >> >> >>> >> >> > the
> > > >> >> >> >>> >> >> > > >>> >> long/lat geo information from IPGeo to
> the
> > > >> >> >> >>>PageViewEvent,
> > > >> >> >> >>> >>and
> > > >> >> >> >>> >> >> send
> > > >> >> >> >>> >> >> > > >>>it
> > > >> >> >> >>> >> >> > > >>> >>to a
> > > >> >> >> >>> >> >> > > >>> >> new topic: PageViewEventsWithGeo.
> > > >> >> >> >>> >> >> > > >>> >>
> > > >> >> >> >>> >> >> > > >>> >> You have several options on how to
> > implement
> > > >> >>this
> > > >> >> >> >>> example.
> > > >> >> >> >>> >> >> > > >>> >>
> > > >> >> >> >>> >> >> > > >>> >> 1. If your joining data set (IPGeo) is
> > > >> >>relatively
> > > >> >> >> >>>small
> > > >> >> >> >>> >>and
> > > >> >> >> >>> >> >> > changes
> > > >> >> >> >>> >> >> > > >>> >> infrequently, you can just pack it up in
> > your
> > > >> >>jar
> > > >> >> >>or
> > > >> >> >> >>>.tgz
> > > >> >> >> >>> >> >>file,
> > > >> >> >> >>> >> >> > and
> > > >> >> >> >>> >> >> > > >>> open
> > > >> >> >> >>> >> >> > > >>> >> it open in every StreamTask.
> > > >> >> >> >>> >> >> > > >>> >> 2. If your data set is small, but changes
> > > >> >>somewhat
> > > >> >> >> >>> >> >>frequently,
> > > >> >> >> >>> >> >> you
> > > >> >> >> >>> >> >> > > >>>can
> > > >> >> >> >>> >> >> > > >>> >> throw the data set on some HTTP/HDFS/S3
> > > >>server
> > > >> >> >> >>>somewhere,
> > > >> >> >> >>> >>and
> > > >> >> >> >>> >> >> have
> > > >> >> >> >>> >> >> > > >>>your
> > > >> >> >> >>> >> >> > > >>> >> StreamTask refresh it periodically by
> > > >> >> >>re-downloading
> > > >> >> >> >>>it.
> > > >> >> >> >>> >> >> > > >>> >> 3. You can do remote RPC calls for the
> > IPGeo
> > > >> >>data
> > > >> >> >>on
> > > >> >> >> >>> every
> > > >> >> >> >>> >> >>page
> > > >> >> >> >>> >> >> > view
> > > >> >> >> >>> >> >> > > >>> >>event
> > > >> >> >> >>> >> >> > > >>> >> by query some remote service or DB (e.g.
> > > >> >> >>Cassandra).
> > > >> >> >> >>> >> >> > > >>> >> 4. You can use Samza's state feature to
> set
> > > >>your
> > > >> >> >>IPGeo
> > > >> >> >> >>> >>data
> > > >> >> >> >>> >> >>as a
> > > >> >> >> >>> >> >> > > >>>series
> > > >> >> >> >>> >> >> > > >>> >>of
> > > >> >> >> >>> >> >> > > >>> >> messages to a log-compacted Kafka topic
> > > >> >> >> >>> >> >> > > >>> >> (
> > > >> >> >> >>> >> >>
> > > >> >> >>
> https://cwiki.apache.org/confluence/display/KAFKA/Log+Compaction
> > > >> >> >> >>> >> >> > ),
> > > >> >> >> >>> >> >> > > >>> and
> > > >> >> >> >>> >> >> > > >>> >> configure your Samza job to read this
> topic
> > > >>as a
> > > >> >> >> >>> bootstrap
> > > >> >> >> >>> >> >> stream
> > > >> >> >> >>> >> >> > > >>> >> (
> > > >> >> >> >>> >> >> > > >>> >>
> > > >> >> >> >>> >> >> > > >>> >>
> > > >> >> >> >>> >> >> > > >>>
> > > >> >> >> >>> >> >> > > >>>
> > > >> >> >> >>> >> >> > >
> > > >> >> >> >>> >> >>
> > > >> >> >> >>> >>
> > > >> >> >> >>>
> > > >> >> >> >>>
> > > >> >> >>
> > > >> >>
> > > >>
> > >
> http://samza.incubator.apache.org/learn/documentation/0.7.0/container/st
> > > >> >> >> >>> >> >> > > >>>r
> > > >> >> >> >>> >> >> > > >>> >>e
> > > >> >> >> >>> >> >> > > >>> >> ams.html).
> > > >> >> >> >>> >> >> > > >>> >>
> > > >> >> >> >>> >> >> > > >>> >> For (4), you'd have to partition the
> IPGeo
> > > >>state
> > > >> >> >>topic
> > > >> >> >> >>> >> >>according
> > > >> >> >> >>> >> >> > to
> > > >> >> >> >>> >> >> > > >>>the
> > > >> >> >> >>> >> >> > > >>> >> same key as PageViewEvent. If
> PageViewEvent
> > > >>were
> > > >> >> >> >>> >>partitioned
> > > >> >> >> >>> >> >>by,
> > > >> >> >> >>> >> >> > > >>>say,
> > > >> >> >> >>> >> >> > > >>> >> member ID, but you want your IPGeo state
> > > >>topic
> > > >> >>to
> > > >> >> >>be
> > > >> >> >> >>> >> >>partitioned
> > > >> >> >> >>> >> >> > by
> > > >> >> >> >>> >> >> > > >>>IP
> > > >> >> >> >>> >> >> > > >>> >> address, then you'd have to have an
> > upstream
> > > >>job
> > > >> >> >>that
> > > >> >> >> >>> >> >> > re-partitioned
> > > >> >> >> >>> >> >> > > >>> >> PageViewEvent into some new topic by IP
> > > >>address.
> > > >> >> >>This
> > > >> >> >> >>>new
> > > >> >> >> >>> >> >>topic
> > > >> >> >> >>> >> >> > will
> > > >> >> >> >>> >> >> > > >>> >>have
> > > >> >> >> >>> >> >> > > >>> >> to have the same number of partitions as
> > the
> > > >> >>IPGeo
> > > >> >> >> >>>state
> > > >> >> >> >>> >> >>topic
> > > >> >> >> >>> >> >> (if
> > > >> >> >> >>> >> >> > > >>> IPGeo
> > > >> >> >> >>> >> >> > > >>> >> has 8 partitions, then the new
> > > >> >> >> >>>PageViewEventRepartitioned
> > > >> >> >> >>> >> >>topic
> > > >> >> >> >>> >> >> > > >>>needs 8
> > > >> >> >> >>> >> >> > > >>> >>as
> > > >> >> >> >>> >> >> > > >>> >> well). This will cause your
> > > >> >> >>PageViewEventRepartitioned
> > > >> >> >> >>> >>topic
> > > >> >> >> >>> >> >>and
> > > >> >> >> >>> >> >> > > >>>your
> > > >> >> >> >>> >> >> > > >>> >> IPGeo state topic to be aligned such that
> > the
> > > >> >> >> >>>StreamTask
> > > >> >> >> >>> >>that
> > > >> >> >> >>> >> >> gets
> > > >> >> >> >>> >> >> > > >>>page
> > > >> >> >> >>> >> >> > > >>> >> views for IP address X will also have the
> > > >>IPGeo
> > > >> >> >> >>> >>information
> > > >> >> >> >>> >> >>for
> > > >> >> >> >>> >> >> IP
> > > >> >> >> >>> >> >> > > >>> >>address
> > > >> >> >> >>> >> >> > > >>> >> X.
> > > >> >> >> >>> >> >> > > >>> >>
> > > >> >> >> >>> >> >> > > >>> >> Which strategy you pick is really up to
> > you.
> > > >>:)
> > > >> >> >>(4) is
> > > >> >> >> >>> the
> > > >> >> >> >>> >> >>most
> > > >> >> >> >>> >> >> > > >>> >> complicated, but also the most flexible,
> > and
> > > >> >>most
> > > >> >> >> >>> >> >>operationally
> > > >> >> >> >>> >> >> > > >>>sound.
> > > >> >> >> >>> >> >> > > >>> >>(1)
> > > >> >> >> >>> >> >> > > >>> >> is the easiest if it fits your needs.
> > > >> >> >> >>> >> >> > > >>> >>
> > > >> >> >> >>> >> >> > > >>> >> Cheers,
> > > >> >> >> >>> >> >> > > >>> >> Chris
> > > >> >> >> >>> >> >> > > >>> >>
> > > >> >> >> >>> >> >> > > >>> >> On 8/21/14 10:15 AM, "Shekar Tippur"
> > > >> >> >> >>><ct...@gmail.com>
> > > >> >> >> >>> >> >>wrote:
> > > >> >> >> >>> >> >> > > >>> >>
> > > >> >> >> >>> >> >> > > >>> >> >Hello,
> > > >> >> >> >>> >> >> > > >>> >> >
> > > >> >> >> >>> >> >> > > >>> >> >I am new to Samza. I have just installed
> > > >>Hello
> > > >> >> >>Samza
> > > >> >> >> >>>and
> > > >> >> >> >>> >> >>got it
> > > >> >> >> >>> >> >> > > >>> >>working.
> > > >> >> >> >>> >> >> > > >>> >> >
> > > >> >> >> >>> >> >> > > >>> >> >Here is the use case for which I am
> trying
> > > >>to
> > > >> >>use
> > > >> >> >> >>>Samza:
> > > >> >> >> >>> >> >> > > >>> >> >
> > > >> >> >> >>> >> >> > > >>> >> >
> > > >> >> >> >>> >> >> > > >>> >> >1. Cache the contextual information
> which
> > > >> >>contains
> > > >> >> >> >>>more
> > > >> >> >> >>> >> >> > information
> > > >> >> >> >>> >> >> > > >>> >>about
> > > >> >> >> >>> >> >> > > >>> >> >the hostname or IP address using
> > > >> >>Samza/Yarn/Kafka
> > > >> >> >> >>> >> >> > > >>> >> >2. Collect alert and metric events which
> > > >> >>contain
> > > >> >> >> >>>either
> > > >> >> >> >>> >> >> hostname
> > > >> >> >> >>> >> >> > > >>>or IP
> > > >> >> >> >>> >> >> > > >>> >> >address
> > > >> >> >> >>> >> >> > > >>> >> >3. Append contextual information to the
> > > >>alert
> > > >> >>and
> > > >> >> >> >>>metric
> > > >> >> >> >>> >>and
> > > >> >> >> >>> >> >> > > >>>insert to
> > > >> >> >> >>> >> >> > > >>> >>a
> > > >> >> >> >>> >> >> > > >>> >> >Kafka queue from which other subscribers
> > > >>read
> > > >> >>off
> > > >> >> >>of.
> > > >> >> >> >>> >> >> > > >>> >> >
> > > >> >> >> >>> >> >> > > >>> >> >Can you please shed some light on
> > > >> >> >> >>> >> >> > > >>> >> >
> > > >> >> >> >>> >> >> > > >>> >> >1. Is this feasible?
> > > >> >> >> >>> >> >> > > >>> >> >2. Am I on the right thought process
> > > >> >> >> >>> >> >> > > >>> >> >3. How do I start
> > > >> >> >> >>> >> >> > > >>> >> >
> > > >> >> >> >>> >> >> > > >>> >> >I now have 1 & 2 of them working
> > > >>disparately. I
> > > >> >> >>need
> > > >> >> >> >>>to
> > > >> >> >> >>> >> >> integrate
> > > >> >> >> >>> >> >> > > >>> them.
> > > >> >> >> >>> >> >> > > >>> >> >
> > > >> >> >> >>> >> >> > > >>> >> >Appreciate any input.
> > > >> >> >> >>> >> >> > > >>> >> >
> > > >> >> >> >>> >> >> > > >>> >> >- Shekar
> > > >> >> >> >>> >> >> > > >>> >>
> > > >> >> >> >>> >> >> > > >>> >>
> > > >> >> >> >>> >> >> > > >>>
> > > >> >> >> >>> >> >> > > >>>
> > > >> >> >> >>> >> >> > > >>
> > > >> >> >> >>> >> >> > >
> > > >> >> >> >>> >> >> > >
> > > >> >> >> >>> >> >> >
> > > >> >> >> >>> >> >>
> > > >> >> >> >>> >>
> > > >> >> >> >>> >>
> > > >> >> >> >>>
> > > >> >> >> >>>
> > > >> >> >> >>
> > > >> >> >>
> > > >> >> >>
> > > >> >>
> > > >> >>
> > > >>
> > > >>
> > >
> > >
> >
>

Re: Samza as a Caching layer

Posted by Yan Fang <ya...@gmail.com>.
Hi Shekar,

Since you are running local mode, you must have the libs in the directory
such as *deploy/samza/lib*. Can you see the slf4j-log4j12-1.6.2.jar there?
(not the tar.gz). I ask this is because sometimes people forget to tar -xvf
after they add dependencies and recompile, while the local mode reads the
*deploy/samz*a, instead of the tar.gz file. Thank you.

Cheers,

Fang, Yan
yanfang724@gmail.com
+1 (206) 849-4108


On Wed, Sep 3, 2014 at 2:59 PM, Shekar Tippur <ct...@gmail.com> wrote:

> I do see this
>
> lib/slf4j-log4j12-1.6.2.jar
>
> In the tgz file. Also, this message seem to me more of a noise.
>
> I am not sure if we are getting sidetracked on this.
>
> In the file WikipediaFeedStreamTask.java, I changed log.info to
> System.out.println - thinking that this should throw the message to the
> console.
>
> Is this the right place to trap the incoming message?
>
>
> $ cat
>
> samza-wikipedia/src/main/java/samza/examples/wikipedia/task/WikipediaFeedStreamTask.java
>
> /*
>
>  * Licensed to the Apache Software Foundation (ASF) under one
>
>  * or more contributor license agreements.  See the NOTICE file
>
>  * distributed with this work for additional information
>
>  * regarding copyright ownership.  The ASF licenses this file
>
>  * to you under the Apache License, Version 2.0 (the
>
>  * "License"); you may not use this file except in compliance
>
>  * with the License.  You may obtain a copy of the License at
>
>  *
>
>  *   http://www.apache.org/licenses/LICENSE-2.0
>
>  *
>
>  * Unless required by applicable law or agreed to in writing,
>
>  * software distributed under the License is distributed on an
>
>  * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
>
>  * KIND, either express or implied.  See the License for the
>
>  * specific language governing permissions and limitations
>
>  * under the License.
>
>  */
>
>
> package samza.examples.wikipedia.task;
>
>
> import java.util.Map;
>
> import org.apache.samza.system.IncomingMessageEnvelope;
>
> import org.apache.samza.system.OutgoingMessageEnvelope;
>
> import org.apache.samza.system.SystemStream;
>
> import org.apache.samza.task.MessageCollector;
>
> import org.apache.samza.task.StreamTask;
>
> import org.apache.samza.task.TaskCoordinator;
>
> import samza.examples.wikipedia.system.WikipediaFeed.WikipediaFeedEvent;
>
>
> /**
>
>  * This task is very simple. All it does is take messages that it receives,
> and
>
>  * sends them to a Kafka topic called wikipedia-raw.
>
>  */
>
> public class WikipediaFeedStreamTask implements StreamTask {
>
>   private static final SystemStream OUTPUT_STREAM = new
> SystemStream("kafka", "wikipedia-raw");
>
>
>   @Override
>
>   public void process(IncomingMessageEnvelope envelope, MessageCollector
> collector, TaskCoordinator coordinator) {
>
>     Map<String, Object> outgoingMap =
> WikipediaFeedEvent.toMap((WikipediaFeedEvent) envelope.getMessage());
>
>     System.out.println(envelope.getMessage().toString());
>
>     collector.send(new OutgoingMessageEnvelope(OUTPUT_STREAM,
> outgoingMap));
>
>   }
>
> }
>
>
>
>
> On Wed, Sep 3, 2014 at 2:48 PM, Chris Riccomini <
> criccomini@linkedin.com.invalid> wrote:
>
> > Hey Shekar,
> >
> > Sorry that you're having such a tough time with this. I'll keep trying to
> > help as best I can.
> >
> > Do you see slf4j-log4j12 in your job's .tgz file?
> >
> > Cheers,
> > Chris
> >
> > On 9/3/14 2:46 PM, "Shekar Tippur" <ct...@gmail.com> wrote:
> >
> > >Chris,
> > >
> > >I have added the below dependencies and I still get the same message.
> > >
> > >
> > >      <dependency>
> > >
> > >        <groupId>org.slf4j</groupId>
> > >
> > >        <artifactId>slf4j-api</artifactId>
> > >
> > >        <version>1.7.7</version>
> > >
> > >      </dependency>
> > >
> > >      <dependency>
> > >
> > >          <groupId>org.slf4j</groupId>
> > >
> > >          <artifactId>slf4j-log4j12</artifactId>
> > >
> > >          <version>1.5.6</version>
> > >
> > >      </dependency>
> > >
> > >
> > >On Wed, Sep 3, 2014 at 9:01 AM, Chris Riccomini <
> > >criccomini@linkedin.com.invalid> wrote:
> > >
> > >> Hey Shekar,
> > >>
> > >> The SLF4J stuff is saying that you don't have an slf4j binding on your
> > >> classpath. Try adding slf4j-log4j as a runtime dependency on your
> > >>project.
> > >>
> > >> Cheers,
> > >> Chris
> > >>
> > >> On 9/2/14 3:24 PM, "Shekar Tippur" <ct...@gmail.com> wrote:
> > >>
> > >> >Chris ,
> > >> >
> > >> >In the current state, I just want Samza to connect to 127.0.0.1.
> > >> >
> > >> >I have set YARN_HOME to
> > >> >
> > >> >$ echo $YARN_HOME
> > >> >
> > >> >/home/ctippur/hello-samza/deploy/yarn
> > >> >
> > >> >I still dont see anything on hadoop console.
> > >> >
> > >> > Also, I see this during startup
> > >> >
> > >> >
> > >> >SLF4J: Failed to load class "org.slf4j.impl.StaticLoggerBinder".
> > >> >
> > >> >SLF4J: Defaulting to no-operation (NOP) logger implementation
> > >> >
> > >> >SLF4J: See http://www.slf4j.org/codes.html#StaticLoggerBinder for
> > >>further
> > >> >details.
> > >> >
> > >> >SLF4J: Failed to load class "org.slf4j.impl.StaticLoggerBinder".
> > >> >
> > >> >SLF4J: Defaulting to no-operation (NOP) logger implementation
> > >> >
> > >> >SLF4J: See http://www.slf4j.org/codes.html#StaticLoggerBinder for
> > >>further
> > >> >details.
> > >> >
> > >> >- Shekar
> > >> >
> > >> >
> > >> >On Tue, Sep 2, 2014 at 2:20 PM, Chris Riccomini <
> > >> >criccomini@linkedin.com.invalid> wrote:
> > >> >
> > >> >> Hey Shekar,
> > >> >>
> > >> >> Ah ha. In that case, do you expect your SamzaContainer to try to
> > >>connect
> > >> >> to the RM at 127.0.0.1, or do you expect it to try to connect to
> some
> > >> >> remote RM? If you expect it to try and connect to a remote RM, it's
> > >>not
> > >> >> doing that. Perhaps because YARN_HOME isn't set.
> > >> >>
> > >> >> If you go to your RM's web interface, how many active nodes do you
> > >>see
> > >> >> listed?
> > >> >>
> > >> >> Cheers,
> > >> >> Chris
> > >> >>
> > >> >> On 9/2/14 2:17 PM, "Shekar Tippur" <ct...@gmail.com> wrote:
> > >> >>
> > >> >> >Chris ..
> > >> >> >
> > >> >> >I am using a rhel server to host all the components (Yarn, kafka,
> > >> >>samza).
> > >> >> >I
> > >> >> >dont have ACLs open to wikipedia.
> > >> >> >I am following ..
> > >> >> >
> > >> >>
> > >> >>
> > >>
> > >>
> >
> http://samza.incubator.apache.org/learn/tutorials/latest/run-hello-samza-
> > >> >>w
> > >> >> >ithout-internet.html
> > >> >> >
> > >> >> >- Shekar
> > >> >> >
> > >> >> >
> > >> >> >
> > >> >> >On Tue, Sep 2, 2014 at 2:13 PM, Chris Riccomini <
> > >> >> >criccomini@linkedin.com.invalid> wrote:
> > >> >> >
> > >> >> >> Hey Shekar,
> > >> >> >>
> > >> >> >> Can you try changing that to:
> > >> >> >>
> > >> >> >>   http://127.0.0.1:8088/cluster
> > >> >> >>
> > >> >> >>
> > >> >> >> And see if you can connect?
> > >> >> >>
> > >> >> >> Cheers,
> > >> >> >> Chris
> > >> >> >>
> > >> >> >> On 9/2/14 1:21 PM, "Shekar Tippur" <ct...@gmail.com> wrote:
> > >> >> >>
> > >> >> >> >Other observation is ..
> > >> >> >> >
> > >> >> >> >http://10.132.62.185:8088/cluster shows that no applications
> are
> > >> >> >>running.
> > >> >> >> >
> > >> >> >> >- Shekar
> > >> >> >> >
> > >> >> >> >
> > >> >> >> >
> > >> >> >> >
> > >> >> >> >On Tue, Sep 2, 2014 at 1:15 PM, Shekar Tippur <
> ctippur@gmail.com
> > >
> > >> >> >>wrote:
> > >> >> >> >
> > >> >> >> >> Yarn seem to be running ..
> > >> >> >> >>
> > >> >> >> >> yarn      5462  0.0  2.0 1641296 161404 ?      Sl   Jun20
> > >>95:26
> > >> >> >> >> /usr/java/jdk1.6.0_31/bin/java -Dproc_resourcemanager
> -Xmx1000m
> > >> >> >> >> -Dhadoop.log.dir=/var/log/hadoop-yarn
> > >> >> >> >>-Dyarn.log.dir=/var/log/hadoop-yarn
> > >> >> >> >> -Dhadoop.log.file=yarn-yarn-resourcemanager-pppdc9prd2y2.log
> > >> >> >> >> -Dyarn.log.file=yarn-yarn-resourcemanager-pppdc9prd2y2.log
> > >> >> >> >> -Dyarn.home.dir=/usr/lib/hadoop-yarn
> > >> >> >> >>-Dhadoop.home.dir=/usr/lib/hadoop-yarn
> > >> >> >> >> -Dhadoop.root.logger=INFO,RFA -Dyarn.root.logger=INFO,RFA
> > >> >> >> >> -Djava.library.path=/usr/lib/hadoop/lib/native -classpath
> > >> >> >> >>
> > >> >> >>
> > >> >>
> > >>
> >
> >>>>>>>>/etc/hadoop/conf:/etc/hadoop/conf:/etc/hadoop/conf:/usr/lib/hadoop/
> > >>>>>>>>li
> > >> >>>>>>b/
> > >> >> >>>>*:
> > >> >> >>
> > >> >>
> > >>
> >
> >>>>>>>>/usr/lib/hadoop/.//*:/usr/lib/hadoop-hdfs/./:/usr/lib/hadoop-hdfs/l
> > >>>>>>>>ib
> > >> >>>>>>/*
> > >> >> >>>>:/
> > >> >> >>
> > >> >>
> > >>
> >
> >>>>>>>>usr/lib/hadoop-hdfs/.//*:/usr/lib/hadoop-yarn/lib/*:/usr/lib/hadoop
> > >>>>>>>>-y
> > >> >>>>>>ar
> > >> >> >>>>n/
> > >> >> >>
> > >> >>
> > >>
> >
> >>>>>>>>.//*:/usr/lib/hadoop-mapreduce/lib/*:/usr/lib/hadoop-mapreduce/.//*
> > >>>>>>>>:/
> > >> >>>>>>us
> > >> >> >>>>r/
> > >> >> >>
> > >> >>
> > >>
> >
> >>>>>>>>lib/hadoop-yarn/.//*:/usr/lib/hadoop-yarn/lib/*:/etc/hadoop/conf/rm
> > >>>>>>>>-c
> > >> >>>>>>on
> > >> >> >>>>fi
> > >> >> >> >>g/log4j.properties
> > >> >> >> >> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager
> > >> >> >> >>
> > >> >> >> >> I can tail kafka topic as well ..
> > >> >> >> >>
> > >> >> >> >> deploy/kafka/bin/kafka-console-consumer.sh  --zookeeper
> > >> >> >>localhost:2181
> > >> >> >> >>--topic wikipedia-raw
> > >> >> >> >>
> > >> >> >> >>
> > >> >> >> >>
> > >> >> >> >>
> > >> >> >> >> On Tue, Sep 2, 2014 at 1:04 PM, Chris Riccomini <
> > >> >> >> >> criccomini@linkedin.com.invalid> wrote:
> > >> >> >> >>
> > >> >> >> >>> Hey Shekar,
> > >> >> >> >>>
> > >> >> >> >>> It looks like your job is hanging trying to connect to the
> RM
> > >>on
> > >> >> >>your
> > >> >> >> >>> localhost. I thought that you said that your job was running
> > >>in
> > >> >> >>local
> > >> >> >> >>> mode. If so, it should be using the LocalJobFactory. If not,
> > >>and
> > >> >>you
> > >> >> >> >>> intend to run on YARN, is your YARN RM up and running on
> > >> >>localhost?
> > >> >> >> >>>
> > >> >> >> >>> Cheers,
> > >> >> >> >>> Chris
> > >> >> >> >>>
> > >> >> >> >>> On 9/2/14 12:22 PM, "Shekar Tippur" <ct...@gmail.com>
> > wrote:
> > >> >> >> >>>
> > >> >> >> >>> >Chris ..
> > >> >> >> >>> >
> > >> >> >> >>> >$ cat ./deploy/samza/undefined-samza-container-name.log
> > >> >> >> >>> >
> > >> >> >> >>> >2014-09-02 11:17:58 JobRunner [INFO] job factory:
> > >> >> >> >>> >org.apache.samza.job.yarn.YarnJobFactory
> > >> >> >> >>> >
> > >> >> >> >>> >2014-09-02 11:17:59 ClientHelper [INFO] trying to connect
> to
> > >>RM
> > >> >> >> >>> >127.0.0.1:8032
> > >> >> >> >>> >
> > >> >> >> >>> >2014-09-02 11:17:59 NativeCodeLoader [WARN] Unable to load
> > >> >> >> >>>native-hadoop
> > >> >> >> >>> >library for your platform... using builtin-java classes
> where
> > >> >> >> >>>applicable
> > >> >> >> >>> >
> > >> >> >> >>> >2014-09-02 11:17:59 RMProxy [INFO] Connecting to
> > >>ResourceManager
> > >> >> >>at /
> > >> >> >> >>> >127.0.0.1:8032
> > >> >> >> >>> >
> > >> >> >> >>> >
> > >> >> >> >>> >and Log4j ..
> > >> >> >> >>> >
> > >> >> >> >>> ><!DOCTYPE log4j:configuration SYSTEM "log4j.dtd">
> > >> >> >> >>> >
> > >> >> >> >>> ><log4j:configuration
> > >> >> >>xmlns:log4j="http://jakarta.apache.org/log4j/">
> > >> >> >> >>> >
> > >> >> >> >>> >  <appender name="RollingAppender"
> > >> >> >> >>> >class="org.apache.log4j.DailyRollingFileAppender">
> > >> >> >> >>> >
> > >> >> >> >>> >     <param name="File"
> > >> >> >> >>> >value="${samza.log.dir}/${samza.container.name}.log"
> > >> >> >> >>> >/>
> > >> >> >> >>> >
> > >> >> >> >>> >     <param name="DatePattern" value="'.'yyyy-MM-dd" />
> > >> >> >> >>> >
> > >> >> >> >>> >     <layout class="org.apache.log4j.PatternLayout">
> > >> >> >> >>> >
> > >> >> >> >>> >      <param name="ConversionPattern" value="%d{yyyy-MM-dd
> > >> >> >>HH:mm:ss}
> > >> >> >> >>> %c{1}
> > >> >> >> >>> >[%p] %m%n" />
> > >> >> >> >>> >
> > >> >> >> >>> >     </layout>
> > >> >> >> >>> >
> > >> >> >> >>> >  </appender>
> > >> >> >> >>> >
> > >> >> >> >>> >  <root>
> > >> >> >> >>> >
> > >> >> >> >>> >    <priority value="info" />
> > >> >> >> >>> >
> > >> >> >> >>> >    <appender-ref ref="RollingAppender"/>
> > >> >> >> >>> >
> > >> >> >> >>> >  </root>
> > >> >> >> >>> >
> > >> >> >> >>> ></log4j:configuration>
> > >> >> >> >>> >
> > >> >> >> >>> >
> > >> >> >> >>> >On Tue, Sep 2, 2014 at 12:02 PM, Chris Riccomini <
> > >> >> >> >>> >criccomini@linkedin.com.invalid> wrote:
> > >> >> >> >>> >
> > >> >> >> >>> >> Hey Shekar,
> > >> >> >> >>> >>
> > >> >> >> >>> >> Can you attach your log files? I'm wondering if it's a
> > >> >> >> >>>mis-configured
> > >> >> >> >>> >> log4j.xml (or missing slf4j-log4j jar), which is leading
> to
> > >> >> >>nearly
> > >> >> >> >>> empty
> > >> >> >> >>> >> log files. Also, I'm wondering if the job starts fully.
> > >> >>Anything
> > >> >> >>you
> > >> >> >> >>> can
> > >> >> >> >>> >> attach would be helpful.
> > >> >> >> >>> >>
> > >> >> >> >>> >> Cheers,
> > >> >> >> >>> >> Chris
> > >> >> >> >>> >>
> > >> >> >> >>> >> On 9/2/14 11:43 AM, "Shekar Tippur" <ct...@gmail.com>
> > >> wrote:
> > >> >> >> >>> >>
> > >> >> >> >>> >> >I am running in local mode.
> > >> >> >> >>> >> >
> > >> >> >> >>> >> >S
> > >> >> >> >>> >> >On Sep 2, 2014 11:42 AM, "Yan Fang" <
> yanfang724@gmail.com
> > >
> > >> >> >>wrote:
> > >> >> >> >>> >> >
> > >> >> >> >>> >> >> Hi Shekar.
> > >> >> >> >>> >> >>
> > >> >> >> >>> >> >> Are you running job in local mode or yarn mode? If
> yarn
> > >> >>mode,
> > >> >> >>the
> > >> >> >> >>> log
> > >> >> >> >>> >> >>is in
> > >> >> >> >>> >> >> the yarn's container log.
> > >> >> >> >>> >> >>
> > >> >> >> >>> >> >> Thanks,
> > >> >> >> >>> >> >>
> > >> >> >> >>> >> >> Fang, Yan
> > >> >> >> >>> >> >> yanfang724@gmail.com
> > >> >> >> >>> >> >> +1 (206) 849-4108
> > >> >> >> >>> >> >>
> > >> >> >> >>> >> >>
> > >> >> >> >>> >> >> On Tue, Sep 2, 2014 at 11:36 AM, Shekar Tippur
> > >> >> >> >>><ct...@gmail.com>
> > >> >> >> >>> >> >>wrote:
> > >> >> >> >>> >> >>
> > >> >> >> >>> >> >> > Chris,
> > >> >> >> >>> >> >> >
> > >> >> >> >>> >> >> > Got some time to play around a bit more.
> > >> >> >> >>> >> >> > I tried to edit
> > >> >> >> >>> >> >> >
> > >> >> >> >>> >> >> >
> > >> >> >> >>> >> >>
> > >> >> >> >>> >>
> > >> >> >> >>>
> > >> >> >> >>>
> > >> >> >>
> > >> >>
> > >>
> >
> >>>>>>>>>>>>>samza-wikipedia/src/main/java/samza/examples/wikipedia/task/Wi
> > >>>>>>>>>>>>>ki
> > >> >>>>>>>>>>>pe
> > >> >> >>>>>>>>>di
> > >> >> >> >>>>>>>aFe
> > >> >> >> >>> >>>>ed
> > >> >> >> >>> >> >>StreamTask.java
> > >> >> >> >>> >> >> > to add a logger info statement to tap the incoming
> > >> >>message.
> > >> >> >>I
> > >> >> >> >>>dont
> > >> >> >> >>> >>see
> > >> >> >> >>> >> >> the
> > >> >> >> >>> >> >> > messages being printed to the log file.
> > >> >> >> >>> >> >> >
> > >> >> >> >>> >> >> > Is this the right place to start?
> > >> >> >> >>> >> >> >
> > >> >> >> >>> >> >> > public class WikipediaFeedStreamTask implements
> > >> >>StreamTask {
> > >> >> >> >>> >> >> >
> > >> >> >> >>> >> >> >   private static final SystemStream OUTPUT_STREAM =
> > >>new
> > >> >> >> >>> >> >> > SystemStream("kafka",
> > >> >> >> >>> >> >> > "wikipedia-raw");
> > >> >> >> >>> >> >> >
> > >> >> >> >>> >> >> >   private static final Logger log =
> > >> >>LoggerFactory.getLogger
> > >> >> >> >>> >> >> > (WikipediaFeedStreamTask.class);
> > >> >> >> >>> >> >> >
> > >> >> >> >>> >> >> >   @Override
> > >> >> >> >>> >> >> >
> > >> >> >> >>> >> >> >   public void process(IncomingMessageEnvelope
> > >>envelope,
> > >> >> >> >>> >> >>MessageCollector
> > >> >> >> >>> >> >> > collector, TaskCoordinator coordinator) {
> > >> >> >> >>> >> >> >
> > >> >> >> >>> >> >> >     Map<String, Object> outgoingMap =
> > >> >> >> >>> >> >> > WikipediaFeedEvent.toMap((WikipediaFeedEvent)
> > >> >> >> >>> >>envelope.getMessage());
> > >> >> >> >>> >> >> >
> > >> >> >> >>> >> >> >     log.info(envelope.getMessage().toString());
> > >> >> >> >>> >> >> >
> > >> >> >> >>> >> >> >     collector.send(new
> > >> >> >>OutgoingMessageEnvelope(OUTPUT_STREAM,
> > >> >> >> >>> >> >> > outgoingMap));
> > >> >> >> >>> >> >> >
> > >> >> >> >>> >> >> >   }
> > >> >> >> >>> >> >> >
> > >> >> >> >>> >> >> > }
> > >> >> >> >>> >> >> >
> > >> >> >> >>> >> >> >
> > >> >> >> >>> >> >> > On Mon, Aug 25, 2014 at 9:01 AM, Chris Riccomini <
> > >> >> >> >>> >> >> > criccomini@linkedin.com.invalid> wrote:
> > >> >> >> >>> >> >> >
> > >> >> >> >>> >> >> > > Hey Shekar,
> > >> >> >> >>> >> >> > >
> > >> >> >> >>> >> >> > > Your thought process is on the right track. It's
> > >> >>probably
> > >> >> >> >>>best
> > >> >> >> >>> to
> > >> >> >> >>> >> >>start
> > >> >> >> >>> >> >> > > with hello-samza, and modify it to get what you
> > >>want.
> > >> >>To
> > >> >> >> >>>start
> > >> >> >> >>> >>with,
> > >> >> >> >>> >> >> > > you'll want to:
> > >> >> >> >>> >> >> > >
> > >> >> >> >>> >> >> > > 1. Write a simple StreamTask that just does
> > >>something
> > >> >> >>silly
> > >> >> >> >>>like
> > >> >> >> >>> >> >>just
> > >> >> >> >>> >> >> > > print messages that it receives.
> > >> >> >> >>> >> >> > > 2. Write a configuration for the job that consumes
> > >>from
> > >> >> >>just
> > >> >> >> >>>the
> > >> >> >> >>> >> >>stream
> > >> >> >> >>> >> >> > > (alerts from different sources).
> > >> >> >> >>> >> >> > > 3. Run this to make sure you've got it working.
> > >> >> >> >>> >> >> > > 4. Now add your table join. This can be either a
> > >> >> >>change-data
> > >> >> >> >>> >>capture
> > >> >> >> >>> >> >> > (CDC)
> > >> >> >> >>> >> >> > > stream, or via a remote DB call.
> > >> >> >> >>> >> >> > >
> > >> >> >> >>> >> >> > > That should get you to a point where you've got
> your
> > >> >>job
> > >> >> >>up
> > >> >> >> >>>and
> > >> >> >> >>> >> >> running.
> > >> >> >> >>> >> >> > > From there, you could create your own Maven
> project,
> > >> >>and
> > >> >> >> >>>migrate
> > >> >> >> >>> >> >>your
> > >> >> >> >>> >> >> > code
> > >> >> >> >>> >> >> > > over accordingly.
> > >> >> >> >>> >> >> > >
> > >> >> >> >>> >> >> > > Cheers,
> > >> >> >> >>> >> >> > > Chris
> > >> >> >> >>> >> >> > >
> > >> >> >> >>> >> >> > > On 8/24/14 1:42 AM, "Shekar Tippur"
> > >><ctippur@gmail.com
> > >> >
> > >> >> >> >>>wrote:
> > >> >> >> >>> >> >> > >
> > >> >> >> >>> >> >> > > >Chris,
> > >> >> >> >>> >> >> > > >
> > >> >> >> >>> >> >> > > >I have gone thro the documentation and decided
> that
> > >> >>the
> > >> >> >> >>>option
> > >> >> >> >>> >> >>that is
> > >> >> >> >>> >> >> > > >most
> > >> >> >> >>> >> >> > > >suitable for me is stream-table.
> > >> >> >> >>> >> >> > > >
> > >> >> >> >>> >> >> > > >I see the following things:
> > >> >> >> >>> >> >> > > >
> > >> >> >> >>> >> >> > > >1. Point samza to a table (database)
> > >> >> >> >>> >> >> > > >2. Point Samza to a stream - Alert stream from
> > >> >>different
> > >> >> >> >>> sources
> > >> >> >> >>> >> >> > > >3. Join key like a hostname
> > >> >> >> >>> >> >> > > >
> > >> >> >> >>> >> >> > > >I have Hello Samza working. To extend that to do
> > >>what
> > >> >>my
> > >> >> >> >>>needs
> > >> >> >> >>> >> >>are, I
> > >> >> >> >>> >> >> am
> > >> >> >> >>> >> >> > > >not sure where to start (Needs more code change
> OR
> > >> >> >> >>> configuration
> > >> >> >> >>> >> >> changes
> > >> >> >> >>> >> >> > > >OR
> > >> >> >> >>> >> >> > > >both)?
> > >> >> >> >>> >> >> > > >
> > >> >> >> >>> >> >> > > >I have gone thro
> > >> >> >> >>> >> >> > > >
> > >> >> >> >>> >> >> >
> > >> >> >> >>> >> >>
> > >> >> >> >>> >> >>
> > >> >> >> >>> >>
> > >> >> >> >>> >>
> > >> >> >> >>>
> > >> >> >> >>>
> > >> >> >>
> > >> >>
> > >>
> > http://samza.incubator.apache.org/learn/documentation/latest/api/overvie
> > >> >> >> >>>w
> > >> >> >> >>> >> >> > > .
> > >> >> >> >>> >> >> > > >html
> > >> >> >> >>> >> >> > > >
> > >> >> >> >>> >> >> > > >Is my thought process on the right track? Can you
> > >> >>please
> > >> >> >> >>>point
> > >> >> >> >>> >>me
> > >> >> >> >>> >> >>to
> > >> >> >> >>> >> >> the
> > >> >> >> >>> >> >> > > >right direction?
> > >> >> >> >>> >> >> > > >
> > >> >> >> >>> >> >> > > >- Shekar
> > >> >> >> >>> >> >> > > >
> > >> >> >> >>> >> >> > > >
> > >> >> >> >>> >> >> > > >
> > >> >> >> >>> >> >> > > >On Thu, Aug 21, 2014 at 1:08 PM, Shekar Tippur
> > >> >> >> >>> >><ct...@gmail.com>
> > >> >> >> >>> >> >> > wrote:
> > >> >> >> >>> >> >> > > >
> > >> >> >> >>> >> >> > > >> Chris,
> > >> >> >> >>> >> >> > > >>
> > >> >> >> >>> >> >> > > >> This is perfectly good answer. I will start
> > >>poking
> > >> >>more
> > >> >> >> >>>into
> > >> >> >> >>> >> >>option
> > >> >> >> >>> >> >> > #4.
> > >> >> >> >>> >> >> > > >>
> > >> >> >> >>> >> >> > > >> - Shekar
> > >> >> >> >>> >> >> > > >>
> > >> >> >> >>> >> >> > > >>
> > >> >> >> >>> >> >> > > >> On Thu, Aug 21, 2014 at 1:05 PM, Chris
> Riccomini
> > >><
> > >> >> >> >>> >> >> > > >> criccomini@linkedin.com.invalid> wrote:
> > >> >> >> >>> >> >> > > >>
> > >> >> >> >>> >> >> > > >>> Hey Shekar,
> > >> >> >> >>> >> >> > > >>>
> > >> >> >> >>> >> >> > > >>> Your two options are really (3) or (4), then.
> > >>You
> > >> >>can
> > >> >> >> >>>either
> > >> >> >> >>> >>run
> > >> >> >> >>> >> >> some
> > >> >> >> >>> >> >> > > >>> external DB that holds the data set, and you
> can
> > >> >> >>query it
> > >> >> >> >>> >>from a
> > >> >> >> >>> >> >> > > >>> StreamTask, or you can use Samza's state store
> > >> >> >>feature to
> > >> >> >> >>> >>push
> > >> >> >> >>> >> >>data
> > >> >> >> >>> >> >> > > >>>into a
> > >> >> >> >>> >> >> > > >>> stream that you can then store in a
> partitioned
> > >> >> >>key-value
> > >> >> >> >>> >>store
> > >> >> >> >>> >> >> along
> > >> >> >> >>> >> >> > > >>>with
> > >> >> >> >>> >> >> > > >>> your StreamTasks. There is some documentation
> > >>here
> > >> >> >>about
> > >> >> >> >>>the
> > >> >> >> >>> >> >>state
> > >> >> >> >>> >> >> > > >>>store
> > >> >> >> >>> >> >> > > >>> approach:
> > >> >> >> >>> >> >> > > >>>
> > >> >> >> >>> >> >> > > >>>
> > >> >> >> >>> >> >> > > >>>
> > >> >> >> >>> >> >> > > >>>
> > >> >> >> >>> >> >> > >
> > >> >> >> >>> >> >>
> > >> >> >> >>> >>
> > >> >> >> >>>
> > >> >> >> >>>
> > >> >> >>
> > >> >>
> > >>
> > http://samza.incubator.apache.org/learn/documentation/0.7.0/container/st
> > >> >> >> >>> >> >> > > >>>ate
> > >> >> >> >>> >> >> > > >>> -management.html
> > >> >> >> >>> >> >> > > >>>
> > >> >> >> >>> >> >> > > >>><
> > >> >> >> >>> >> >> > >
> > >> >> >> >>> >> >>
> > >> >> >> >>>
> > >> >> >> >>>>>
> > >> >> >>
> > >> >>
> > http://samza.incubator.apache.org/learn/documentation/0.7.0/container/
> > >> >> >> >>>>>s
> > >> >> >> >>> >> >> > > >>>tate-management.html>
> > >> >> >> >>> >> >> > > >>>
> > >> >> >> >>> >> >> > > >>>
> > >> >> >> >>> >> >> > > >>> (4) is going to require more up front effort
> > >>from
> > >> >>you,
> > >> >> >> >>>since
> > >> >> >> >>> >> >>you'll
> > >> >> >> >>> >> >> > > >>>have
> > >> >> >> >>> >> >> > > >>> to understand how Kafka's partitioning model
> > >>works,
> > >> >> >>and
> > >> >> >> >>> setup
> > >> >> >> >>> >> >>some
> > >> >> >> >>> >> >> > > >>> pipeline to push the updates for your state.
> In
> > >>the
> > >> >> >>long
> > >> >> >> >>> >>run, I
> > >> >> >> >>> >> >> > believe
> > >> >> >> >>> >> >> > > >>> it's the better approach, though. Local
> lookups
> > >>on
> > >> >>a
> > >> >> >> >>> >>key-value
> > >> >> >> >>> >> >> store
> > >> >> >> >>> >> >> > > >>> should be faster than doing remote RPC calls
> to
> > >>a
> > >> >>DB
> > >> >> >>for
> > >> >> >> >>> >>every
> > >> >> >> >>> >> >> > message.
> > >> >> >> >>> >> >> > > >>>
> > >> >> >> >>> >> >> > > >>> I'm sorry I can't give you a more definitive
> > >> >>answer.
> > >> >> >>It's
> > >> >> >> >>> >>really
> > >> >> >> >>> >> >> > about
> > >> >> >> >>> >> >> > > >>> trade-offs.
> > >> >> >> >>> >> >> > > >>>
> > >> >> >> >>> >> >> > > >>> Cheers,
> > >> >> >> >>> >> >> > > >>> Chris
> > >> >> >> >>> >> >> > > >>>
> > >> >> >> >>> >> >> > > >>> On 8/21/14 12:22 PM, "Shekar Tippur"
> > >> >> >><ct...@gmail.com>
> > >> >> >> >>> >>wrote:
> > >> >> >> >>> >> >> > > >>>
> > >> >> >> >>> >> >> > > >>> >Chris,
> > >> >> >> >>> >> >> > > >>> >
> > >> >> >> >>> >> >> > > >>> >A big thanks for a swift response. The data
> > >>set is
> > >> >> >>huge
> > >> >> >> >>>and
> > >> >> >> >>> >>the
> > >> >> >> >>> >> >> > > >>>frequency
> > >> >> >> >>> >> >> > > >>> >is in burst.
> > >> >> >> >>> >> >> > > >>> >What do you suggest?
> > >> >> >> >>> >> >> > > >>> >
> > >> >> >> >>> >> >> > > >>> >- Shekar
> > >> >> >> >>> >> >> > > >>> >
> > >> >> >> >>> >> >> > > >>> >
> > >> >> >> >>> >> >> > > >>> >On Thu, Aug 21, 2014 at 11:05 AM, Chris
> > >>Riccomini
> > >> >><
> > >> >> >> >>> >> >> > > >>> >criccomini@linkedin.com.invalid> wrote:
> > >> >> >> >>> >> >> > > >>> >
> > >> >> >> >>> >> >> > > >>> >> Hey Shekar,
> > >> >> >> >>> >> >> > > >>> >>
> > >> >> >> >>> >> >> > > >>> >> This is feasible, and you are on the right
> > >> >>thought
> > >> >> >> >>> >>process.
> > >> >> >> >>> >> >> > > >>> >>
> > >> >> >> >>> >> >> > > >>> >> For the sake of discussion, I'm going to
> > >>pretend
> > >> >> >>that
> > >> >> >> >>>you
> > >> >> >> >>> >> >>have a
> > >> >> >> >>> >> >> > > >>>Kafka
> > >> >> >> >>> >> >> > > >>> >> topic called "PageViewEvent", which has
> just
> > >> >>the IP
> > >> >> >> >>> >>address
> > >> >> >> >>> >> >>that
> > >> >> >> >>> >> >> > was
> > >> >> >> >>> >> >> > > >>> >>used
> > >> >> >> >>> >> >> > > >>> >> to view a page. These messages will be
> logged
> > >> >>every
> > >> >> >> >>>time
> > >> >> >> >>> a
> > >> >> >> >>> >> >>page
> > >> >> >> >>> >> >> > view
> > >> >> >> >>> >> >> > > >>> >> happens. I'm also going to pretend that you
> > >>have
> > >> >> >>some
> > >> >> >> >>> >>state
> > >> >> >> >>> >> >> called
> > >> >> >> >>> >> >> > > >>> >>"IPGeo"
> > >> >> >> >>> >> >> > > >>> >> (e.g. The maxmind data set). In this
> example,
> > >> >>we'll
> > >> >> >> >>>want
> > >> >> >> >>> >>to
> > >> >> >> >>> >> >>join
> > >> >> >> >>> >> >> > the
> > >> >> >> >>> >> >> > > >>> >> long/lat geo information from IPGeo to the
> > >> >> >> >>>PageViewEvent,
> > >> >> >> >>> >>and
> > >> >> >> >>> >> >> send
> > >> >> >> >>> >> >> > > >>>it
> > >> >> >> >>> >> >> > > >>> >>to a
> > >> >> >> >>> >> >> > > >>> >> new topic: PageViewEventsWithGeo.
> > >> >> >> >>> >> >> > > >>> >>
> > >> >> >> >>> >> >> > > >>> >> You have several options on how to
> implement
> > >> >>this
> > >> >> >> >>> example.
> > >> >> >> >>> >> >> > > >>> >>
> > >> >> >> >>> >> >> > > >>> >> 1. If your joining data set (IPGeo) is
> > >> >>relatively
> > >> >> >> >>>small
> > >> >> >> >>> >>and
> > >> >> >> >>> >> >> > changes
> > >> >> >> >>> >> >> > > >>> >> infrequently, you can just pack it up in
> your
> > >> >>jar
> > >> >> >>or
> > >> >> >> >>>.tgz
> > >> >> >> >>> >> >>file,
> > >> >> >> >>> >> >> > and
> > >> >> >> >>> >> >> > > >>> open
> > >> >> >> >>> >> >> > > >>> >> it open in every StreamTask.
> > >> >> >> >>> >> >> > > >>> >> 2. If your data set is small, but changes
> > >> >>somewhat
> > >> >> >> >>> >> >>frequently,
> > >> >> >> >>> >> >> you
> > >> >> >> >>> >> >> > > >>>can
> > >> >> >> >>> >> >> > > >>> >> throw the data set on some HTTP/HDFS/S3
> > >>server
> > >> >> >> >>>somewhere,
> > >> >> >> >>> >>and
> > >> >> >> >>> >> >> have
> > >> >> >> >>> >> >> > > >>>your
> > >> >> >> >>> >> >> > > >>> >> StreamTask refresh it periodically by
> > >> >> >>re-downloading
> > >> >> >> >>>it.
> > >> >> >> >>> >> >> > > >>> >> 3. You can do remote RPC calls for the
> IPGeo
> > >> >>data
> > >> >> >>on
> > >> >> >> >>> every
> > >> >> >> >>> >> >>page
> > >> >> >> >>> >> >> > view
> > >> >> >> >>> >> >> > > >>> >>event
> > >> >> >> >>> >> >> > > >>> >> by query some remote service or DB (e.g.
> > >> >> >>Cassandra).
> > >> >> >> >>> >> >> > > >>> >> 4. You can use Samza's state feature to set
> > >>your
> > >> >> >>IPGeo
> > >> >> >> >>> >>data
> > >> >> >> >>> >> >>as a
> > >> >> >> >>> >> >> > > >>>series
> > >> >> >> >>> >> >> > > >>> >>of
> > >> >> >> >>> >> >> > > >>> >> messages to a log-compacted Kafka topic
> > >> >> >> >>> >> >> > > >>> >> (
> > >> >> >> >>> >> >>
> > >> >> >>https://cwiki.apache.org/confluence/display/KAFKA/Log+Compaction
> > >> >> >> >>> >> >> > ),
> > >> >> >> >>> >> >> > > >>> and
> > >> >> >> >>> >> >> > > >>> >> configure your Samza job to read this topic
> > >>as a
> > >> >> >> >>> bootstrap
> > >> >> >> >>> >> >> stream
> > >> >> >> >>> >> >> > > >>> >> (
> > >> >> >> >>> >> >> > > >>> >>
> > >> >> >> >>> >> >> > > >>> >>
> > >> >> >> >>> >> >> > > >>>
> > >> >> >> >>> >> >> > > >>>
> > >> >> >> >>> >> >> > >
> > >> >> >> >>> >> >>
> > >> >> >> >>> >>
> > >> >> >> >>>
> > >> >> >> >>>
> > >> >> >>
> > >> >>
> > >>
> > http://samza.incubator.apache.org/learn/documentation/0.7.0/container/st
> > >> >> >> >>> >> >> > > >>>r
> > >> >> >> >>> >> >> > > >>> >>e
> > >> >> >> >>> >> >> > > >>> >> ams.html).
> > >> >> >> >>> >> >> > > >>> >>
> > >> >> >> >>> >> >> > > >>> >> For (4), you'd have to partition the IPGeo
> > >>state
> > >> >> >>topic
> > >> >> >> >>> >> >>according
> > >> >> >> >>> >> >> > to
> > >> >> >> >>> >> >> > > >>>the
> > >> >> >> >>> >> >> > > >>> >> same key as PageViewEvent. If PageViewEvent
> > >>were
> > >> >> >> >>> >>partitioned
> > >> >> >> >>> >> >>by,
> > >> >> >> >>> >> >> > > >>>say,
> > >> >> >> >>> >> >> > > >>> >> member ID, but you want your IPGeo state
> > >>topic
> > >> >>to
> > >> >> >>be
> > >> >> >> >>> >> >>partitioned
> > >> >> >> >>> >> >> > by
> > >> >> >> >>> >> >> > > >>>IP
> > >> >> >> >>> >> >> > > >>> >> address, then you'd have to have an
> upstream
> > >>job
> > >> >> >>that
> > >> >> >> >>> >> >> > re-partitioned
> > >> >> >> >>> >> >> > > >>> >> PageViewEvent into some new topic by IP
> > >>address.
> > >> >> >>This
> > >> >> >> >>>new
> > >> >> >> >>> >> >>topic
> > >> >> >> >>> >> >> > will
> > >> >> >> >>> >> >> > > >>> >>have
> > >> >> >> >>> >> >> > > >>> >> to have the same number of partitions as
> the
> > >> >>IPGeo
> > >> >> >> >>>state
> > >> >> >> >>> >> >>topic
> > >> >> >> >>> >> >> (if
> > >> >> >> >>> >> >> > > >>> IPGeo
> > >> >> >> >>> >> >> > > >>> >> has 8 partitions, then the new
> > >> >> >> >>>PageViewEventRepartitioned
> > >> >> >> >>> >> >>topic
> > >> >> >> >>> >> >> > > >>>needs 8
> > >> >> >> >>> >> >> > > >>> >>as
> > >> >> >> >>> >> >> > > >>> >> well). This will cause your
> > >> >> >>PageViewEventRepartitioned
> > >> >> >> >>> >>topic
> > >> >> >> >>> >> >>and
> > >> >> >> >>> >> >> > > >>>your
> > >> >> >> >>> >> >> > > >>> >> IPGeo state topic to be aligned such that
> the
> > >> >> >> >>>StreamTask
> > >> >> >> >>> >>that
> > >> >> >> >>> >> >> gets
> > >> >> >> >>> >> >> > > >>>page
> > >> >> >> >>> >> >> > > >>> >> views for IP address X will also have the
> > >>IPGeo
> > >> >> >> >>> >>information
> > >> >> >> >>> >> >>for
> > >> >> >> >>> >> >> IP
> > >> >> >> >>> >> >> > > >>> >>address
> > >> >> >> >>> >> >> > > >>> >> X.
> > >> >> >> >>> >> >> > > >>> >>
> > >> >> >> >>> >> >> > > >>> >> Which strategy you pick is really up to
> you.
> > >>:)
> > >> >> >>(4) is
> > >> >> >> >>> the
> > >> >> >> >>> >> >>most
> > >> >> >> >>> >> >> > > >>> >> complicated, but also the most flexible,
> and
> > >> >>most
> > >> >> >> >>> >> >>operationally
> > >> >> >> >>> >> >> > > >>>sound.
> > >> >> >> >>> >> >> > > >>> >>(1)
> > >> >> >> >>> >> >> > > >>> >> is the easiest if it fits your needs.
> > >> >> >> >>> >> >> > > >>> >>
> > >> >> >> >>> >> >> > > >>> >> Cheers,
> > >> >> >> >>> >> >> > > >>> >> Chris
> > >> >> >> >>> >> >> > > >>> >>
> > >> >> >> >>> >> >> > > >>> >> On 8/21/14 10:15 AM, "Shekar Tippur"
> > >> >> >> >>><ct...@gmail.com>
> > >> >> >> >>> >> >>wrote:
> > >> >> >> >>> >> >> > > >>> >>
> > >> >> >> >>> >> >> > > >>> >> >Hello,
> > >> >> >> >>> >> >> > > >>> >> >
> > >> >> >> >>> >> >> > > >>> >> >I am new to Samza. I have just installed
> > >>Hello
> > >> >> >>Samza
> > >> >> >> >>>and
> > >> >> >> >>> >> >>got it
> > >> >> >> >>> >> >> > > >>> >>working.
> > >> >> >> >>> >> >> > > >>> >> >
> > >> >> >> >>> >> >> > > >>> >> >Here is the use case for which I am trying
> > >>to
> > >> >>use
> > >> >> >> >>>Samza:
> > >> >> >> >>> >> >> > > >>> >> >
> > >> >> >> >>> >> >> > > >>> >> >
> > >> >> >> >>> >> >> > > >>> >> >1. Cache the contextual information which
> > >> >>contains
> > >> >> >> >>>more
> > >> >> >> >>> >> >> > information
> > >> >> >> >>> >> >> > > >>> >>about
> > >> >> >> >>> >> >> > > >>> >> >the hostname or IP address using
> > >> >>Samza/Yarn/Kafka
> > >> >> >> >>> >> >> > > >>> >> >2. Collect alert and metric events which
> > >> >>contain
> > >> >> >> >>>either
> > >> >> >> >>> >> >> hostname
> > >> >> >> >>> >> >> > > >>>or IP
> > >> >> >> >>> >> >> > > >>> >> >address
> > >> >> >> >>> >> >> > > >>> >> >3. Append contextual information to the
> > >>alert
> > >> >>and
> > >> >> >> >>>metric
> > >> >> >> >>> >>and
> > >> >> >> >>> >> >> > > >>>insert to
> > >> >> >> >>> >> >> > > >>> >>a
> > >> >> >> >>> >> >> > > >>> >> >Kafka queue from which other subscribers
> > >>read
> > >> >>off
> > >> >> >>of.
> > >> >> >> >>> >> >> > > >>> >> >
> > >> >> >> >>> >> >> > > >>> >> >Can you please shed some light on
> > >> >> >> >>> >> >> > > >>> >> >
> > >> >> >> >>> >> >> > > >>> >> >1. Is this feasible?
> > >> >> >> >>> >> >> > > >>> >> >2. Am I on the right thought process
> > >> >> >> >>> >> >> > > >>> >> >3. How do I start
> > >> >> >> >>> >> >> > > >>> >> >
> > >> >> >> >>> >> >> > > >>> >> >I now have 1 & 2 of them working
> > >>disparately. I
> > >> >> >>need
> > >> >> >> >>>to
> > >> >> >> >>> >> >> integrate
> > >> >> >> >>> >> >> > > >>> them.
> > >> >> >> >>> >> >> > > >>> >> >
> > >> >> >> >>> >> >> > > >>> >> >Appreciate any input.
> > >> >> >> >>> >> >> > > >>> >> >
> > >> >> >> >>> >> >> > > >>> >> >- Shekar
> > >> >> >> >>> >> >> > > >>> >>
> > >> >> >> >>> >> >> > > >>> >>
> > >> >> >> >>> >> >> > > >>>
> > >> >> >> >>> >> >> > > >>>
> > >> >> >> >>> >> >> > > >>
> > >> >> >> >>> >> >> > >
> > >> >> >> >>> >> >> > >
> > >> >> >> >>> >> >> >
> > >> >> >> >>> >> >>
> > >> >> >> >>> >>
> > >> >> >> >>> >>
> > >> >> >> >>>
> > >> >> >> >>>
> > >> >> >> >>
> > >> >> >>
> > >> >> >>
> > >> >>
> > >> >>
> > >>
> > >>
> >
> >
>

Re: Samza as a Caching layer

Posted by Shekar Tippur <ct...@gmail.com>.
I do see this

lib/slf4j-log4j12-1.6.2.jar

In the tgz file. Also, this message seem to me more of a noise.

I am not sure if we are getting sidetracked on this.

In the file WikipediaFeedStreamTask.java, I changed log.info to
System.out.println - thinking that this should throw the message to the
console.

Is this the right place to trap the incoming message?


$ cat
samza-wikipedia/src/main/java/samza/examples/wikipedia/task/WikipediaFeedStreamTask.java

/*

 * Licensed to the Apache Software Foundation (ASF) under one

 * or more contributor license agreements.  See the NOTICE file

 * distributed with this work for additional information

 * regarding copyright ownership.  The ASF licenses this file

 * to you under the Apache License, Version 2.0 (the

 * "License"); you may not use this file except in compliance

 * with the License.  You may obtain a copy of the License at

 *

 *   http://www.apache.org/licenses/LICENSE-2.0

 *

 * Unless required by applicable law or agreed to in writing,

 * software distributed under the License is distributed on an

 * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY

 * KIND, either express or implied.  See the License for the

 * specific language governing permissions and limitations

 * under the License.

 */


package samza.examples.wikipedia.task;


import java.util.Map;

import org.apache.samza.system.IncomingMessageEnvelope;

import org.apache.samza.system.OutgoingMessageEnvelope;

import org.apache.samza.system.SystemStream;

import org.apache.samza.task.MessageCollector;

import org.apache.samza.task.StreamTask;

import org.apache.samza.task.TaskCoordinator;

import samza.examples.wikipedia.system.WikipediaFeed.WikipediaFeedEvent;


/**

 * This task is very simple. All it does is take messages that it receives,
and

 * sends them to a Kafka topic called wikipedia-raw.

 */

public class WikipediaFeedStreamTask implements StreamTask {

  private static final SystemStream OUTPUT_STREAM = new
SystemStream("kafka", "wikipedia-raw");


  @Override

  public void process(IncomingMessageEnvelope envelope, MessageCollector
collector, TaskCoordinator coordinator) {

    Map<String, Object> outgoingMap =
WikipediaFeedEvent.toMap((WikipediaFeedEvent) envelope.getMessage());

    System.out.println(envelope.getMessage().toString());

    collector.send(new OutgoingMessageEnvelope(OUTPUT_STREAM, outgoingMap));

  }

}




On Wed, Sep 3, 2014 at 2:48 PM, Chris Riccomini <
criccomini@linkedin.com.invalid> wrote:

> Hey Shekar,
>
> Sorry that you're having such a tough time with this. I'll keep trying to
> help as best I can.
>
> Do you see slf4j-log4j12 in your job's .tgz file?
>
> Cheers,
> Chris
>
> On 9/3/14 2:46 PM, "Shekar Tippur" <ct...@gmail.com> wrote:
>
> >Chris,
> >
> >I have added the below dependencies and I still get the same message.
> >
> >
> >      <dependency>
> >
> >        <groupId>org.slf4j</groupId>
> >
> >        <artifactId>slf4j-api</artifactId>
> >
> >        <version>1.7.7</version>
> >
> >      </dependency>
> >
> >      <dependency>
> >
> >          <groupId>org.slf4j</groupId>
> >
> >          <artifactId>slf4j-log4j12</artifactId>
> >
> >          <version>1.5.6</version>
> >
> >      </dependency>
> >
> >
> >On Wed, Sep 3, 2014 at 9:01 AM, Chris Riccomini <
> >criccomini@linkedin.com.invalid> wrote:
> >
> >> Hey Shekar,
> >>
> >> The SLF4J stuff is saying that you don't have an slf4j binding on your
> >> classpath. Try adding slf4j-log4j as a runtime dependency on your
> >>project.
> >>
> >> Cheers,
> >> Chris
> >>
> >> On 9/2/14 3:24 PM, "Shekar Tippur" <ct...@gmail.com> wrote:
> >>
> >> >Chris ,
> >> >
> >> >In the current state, I just want Samza to connect to 127.0.0.1.
> >> >
> >> >I have set YARN_HOME to
> >> >
> >> >$ echo $YARN_HOME
> >> >
> >> >/home/ctippur/hello-samza/deploy/yarn
> >> >
> >> >I still dont see anything on hadoop console.
> >> >
> >> > Also, I see this during startup
> >> >
> >> >
> >> >SLF4J: Failed to load class "org.slf4j.impl.StaticLoggerBinder".
> >> >
> >> >SLF4J: Defaulting to no-operation (NOP) logger implementation
> >> >
> >> >SLF4J: See http://www.slf4j.org/codes.html#StaticLoggerBinder for
> >>further
> >> >details.
> >> >
> >> >SLF4J: Failed to load class "org.slf4j.impl.StaticLoggerBinder".
> >> >
> >> >SLF4J: Defaulting to no-operation (NOP) logger implementation
> >> >
> >> >SLF4J: See http://www.slf4j.org/codes.html#StaticLoggerBinder for
> >>further
> >> >details.
> >> >
> >> >- Shekar
> >> >
> >> >
> >> >On Tue, Sep 2, 2014 at 2:20 PM, Chris Riccomini <
> >> >criccomini@linkedin.com.invalid> wrote:
> >> >
> >> >> Hey Shekar,
> >> >>
> >> >> Ah ha. In that case, do you expect your SamzaContainer to try to
> >>connect
> >> >> to the RM at 127.0.0.1, or do you expect it to try to connect to some
> >> >> remote RM? If you expect it to try and connect to a remote RM, it's
> >>not
> >> >> doing that. Perhaps because YARN_HOME isn't set.
> >> >>
> >> >> If you go to your RM's web interface, how many active nodes do you
> >>see
> >> >> listed?
> >> >>
> >> >> Cheers,
> >> >> Chris
> >> >>
> >> >> On 9/2/14 2:17 PM, "Shekar Tippur" <ct...@gmail.com> wrote:
> >> >>
> >> >> >Chris ..
> >> >> >
> >> >> >I am using a rhel server to host all the components (Yarn, kafka,
> >> >>samza).
> >> >> >I
> >> >> >dont have ACLs open to wikipedia.
> >> >> >I am following ..
> >> >> >
> >> >>
> >> >>
> >>
> >>
> http://samza.incubator.apache.org/learn/tutorials/latest/run-hello-samza-
> >> >>w
> >> >> >ithout-internet.html
> >> >> >
> >> >> >- Shekar
> >> >> >
> >> >> >
> >> >> >
> >> >> >On Tue, Sep 2, 2014 at 2:13 PM, Chris Riccomini <
> >> >> >criccomini@linkedin.com.invalid> wrote:
> >> >> >
> >> >> >> Hey Shekar,
> >> >> >>
> >> >> >> Can you try changing that to:
> >> >> >>
> >> >> >>   http://127.0.0.1:8088/cluster
> >> >> >>
> >> >> >>
> >> >> >> And see if you can connect?
> >> >> >>
> >> >> >> Cheers,
> >> >> >> Chris
> >> >> >>
> >> >> >> On 9/2/14 1:21 PM, "Shekar Tippur" <ct...@gmail.com> wrote:
> >> >> >>
> >> >> >> >Other observation is ..
> >> >> >> >
> >> >> >> >http://10.132.62.185:8088/cluster shows that no applications are
> >> >> >>running.
> >> >> >> >
> >> >> >> >- Shekar
> >> >> >> >
> >> >> >> >
> >> >> >> >
> >> >> >> >
> >> >> >> >On Tue, Sep 2, 2014 at 1:15 PM, Shekar Tippur <ctippur@gmail.com
> >
> >> >> >>wrote:
> >> >> >> >
> >> >> >> >> Yarn seem to be running ..
> >> >> >> >>
> >> >> >> >> yarn      5462  0.0  2.0 1641296 161404 ?      Sl   Jun20
> >>95:26
> >> >> >> >> /usr/java/jdk1.6.0_31/bin/java -Dproc_resourcemanager -Xmx1000m
> >> >> >> >> -Dhadoop.log.dir=/var/log/hadoop-yarn
> >> >> >> >>-Dyarn.log.dir=/var/log/hadoop-yarn
> >> >> >> >> -Dhadoop.log.file=yarn-yarn-resourcemanager-pppdc9prd2y2.log
> >> >> >> >> -Dyarn.log.file=yarn-yarn-resourcemanager-pppdc9prd2y2.log
> >> >> >> >> -Dyarn.home.dir=/usr/lib/hadoop-yarn
> >> >> >> >>-Dhadoop.home.dir=/usr/lib/hadoop-yarn
> >> >> >> >> -Dhadoop.root.logger=INFO,RFA -Dyarn.root.logger=INFO,RFA
> >> >> >> >> -Djava.library.path=/usr/lib/hadoop/lib/native -classpath
> >> >> >> >>
> >> >> >>
> >> >>
> >>
> >>>>>>>>/etc/hadoop/conf:/etc/hadoop/conf:/etc/hadoop/conf:/usr/lib/hadoop/
> >>>>>>>>li
> >> >>>>>>b/
> >> >> >>>>*:
> >> >> >>
> >> >>
> >>
> >>>>>>>>/usr/lib/hadoop/.//*:/usr/lib/hadoop-hdfs/./:/usr/lib/hadoop-hdfs/l
> >>>>>>>>ib
> >> >>>>>>/*
> >> >> >>>>:/
> >> >> >>
> >> >>
> >>
> >>>>>>>>usr/lib/hadoop-hdfs/.//*:/usr/lib/hadoop-yarn/lib/*:/usr/lib/hadoop
> >>>>>>>>-y
> >> >>>>>>ar
> >> >> >>>>n/
> >> >> >>
> >> >>
> >>
> >>>>>>>>.//*:/usr/lib/hadoop-mapreduce/lib/*:/usr/lib/hadoop-mapreduce/.//*
> >>>>>>>>:/
> >> >>>>>>us
> >> >> >>>>r/
> >> >> >>
> >> >>
> >>
> >>>>>>>>lib/hadoop-yarn/.//*:/usr/lib/hadoop-yarn/lib/*:/etc/hadoop/conf/rm
> >>>>>>>>-c
> >> >>>>>>on
> >> >> >>>>fi
> >> >> >> >>g/log4j.properties
> >> >> >> >> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager
> >> >> >> >>
> >> >> >> >> I can tail kafka topic as well ..
> >> >> >> >>
> >> >> >> >> deploy/kafka/bin/kafka-console-consumer.sh  --zookeeper
> >> >> >>localhost:2181
> >> >> >> >>--topic wikipedia-raw
> >> >> >> >>
> >> >> >> >>
> >> >> >> >>
> >> >> >> >>
> >> >> >> >> On Tue, Sep 2, 2014 at 1:04 PM, Chris Riccomini <
> >> >> >> >> criccomini@linkedin.com.invalid> wrote:
> >> >> >> >>
> >> >> >> >>> Hey Shekar,
> >> >> >> >>>
> >> >> >> >>> It looks like your job is hanging trying to connect to the RM
> >>on
> >> >> >>your
> >> >> >> >>> localhost. I thought that you said that your job was running
> >>in
> >> >> >>local
> >> >> >> >>> mode. If so, it should be using the LocalJobFactory. If not,
> >>and
> >> >>you
> >> >> >> >>> intend to run on YARN, is your YARN RM up and running on
> >> >>localhost?
> >> >> >> >>>
> >> >> >> >>> Cheers,
> >> >> >> >>> Chris
> >> >> >> >>>
> >> >> >> >>> On 9/2/14 12:22 PM, "Shekar Tippur" <ct...@gmail.com>
> wrote:
> >> >> >> >>>
> >> >> >> >>> >Chris ..
> >> >> >> >>> >
> >> >> >> >>> >$ cat ./deploy/samza/undefined-samza-container-name.log
> >> >> >> >>> >
> >> >> >> >>> >2014-09-02 11:17:58 JobRunner [INFO] job factory:
> >> >> >> >>> >org.apache.samza.job.yarn.YarnJobFactory
> >> >> >> >>> >
> >> >> >> >>> >2014-09-02 11:17:59 ClientHelper [INFO] trying to connect to
> >>RM
> >> >> >> >>> >127.0.0.1:8032
> >> >> >> >>> >
> >> >> >> >>> >2014-09-02 11:17:59 NativeCodeLoader [WARN] Unable to load
> >> >> >> >>>native-hadoop
> >> >> >> >>> >library for your platform... using builtin-java classes where
> >> >> >> >>>applicable
> >> >> >> >>> >
> >> >> >> >>> >2014-09-02 11:17:59 RMProxy [INFO] Connecting to
> >>ResourceManager
> >> >> >>at /
> >> >> >> >>> >127.0.0.1:8032
> >> >> >> >>> >
> >> >> >> >>> >
> >> >> >> >>> >and Log4j ..
> >> >> >> >>> >
> >> >> >> >>> ><!DOCTYPE log4j:configuration SYSTEM "log4j.dtd">
> >> >> >> >>> >
> >> >> >> >>> ><log4j:configuration
> >> >> >>xmlns:log4j="http://jakarta.apache.org/log4j/">
> >> >> >> >>> >
> >> >> >> >>> >  <appender name="RollingAppender"
> >> >> >> >>> >class="org.apache.log4j.DailyRollingFileAppender">
> >> >> >> >>> >
> >> >> >> >>> >     <param name="File"
> >> >> >> >>> >value="${samza.log.dir}/${samza.container.name}.log"
> >> >> >> >>> >/>
> >> >> >> >>> >
> >> >> >> >>> >     <param name="DatePattern" value="'.'yyyy-MM-dd" />
> >> >> >> >>> >
> >> >> >> >>> >     <layout class="org.apache.log4j.PatternLayout">
> >> >> >> >>> >
> >> >> >> >>> >      <param name="ConversionPattern" value="%d{yyyy-MM-dd
> >> >> >>HH:mm:ss}
> >> >> >> >>> %c{1}
> >> >> >> >>> >[%p] %m%n" />
> >> >> >> >>> >
> >> >> >> >>> >     </layout>
> >> >> >> >>> >
> >> >> >> >>> >  </appender>
> >> >> >> >>> >
> >> >> >> >>> >  <root>
> >> >> >> >>> >
> >> >> >> >>> >    <priority value="info" />
> >> >> >> >>> >
> >> >> >> >>> >    <appender-ref ref="RollingAppender"/>
> >> >> >> >>> >
> >> >> >> >>> >  </root>
> >> >> >> >>> >
> >> >> >> >>> ></log4j:configuration>
> >> >> >> >>> >
> >> >> >> >>> >
> >> >> >> >>> >On Tue, Sep 2, 2014 at 12:02 PM, Chris Riccomini <
> >> >> >> >>> >criccomini@linkedin.com.invalid> wrote:
> >> >> >> >>> >
> >> >> >> >>> >> Hey Shekar,
> >> >> >> >>> >>
> >> >> >> >>> >> Can you attach your log files? I'm wondering if it's a
> >> >> >> >>>mis-configured
> >> >> >> >>> >> log4j.xml (or missing slf4j-log4j jar), which is leading to
> >> >> >>nearly
> >> >> >> >>> empty
> >> >> >> >>> >> log files. Also, I'm wondering if the job starts fully.
> >> >>Anything
> >> >> >>you
> >> >> >> >>> can
> >> >> >> >>> >> attach would be helpful.
> >> >> >> >>> >>
> >> >> >> >>> >> Cheers,
> >> >> >> >>> >> Chris
> >> >> >> >>> >>
> >> >> >> >>> >> On 9/2/14 11:43 AM, "Shekar Tippur" <ct...@gmail.com>
> >> wrote:
> >> >> >> >>> >>
> >> >> >> >>> >> >I am running in local mode.
> >> >> >> >>> >> >
> >> >> >> >>> >> >S
> >> >> >> >>> >> >On Sep 2, 2014 11:42 AM, "Yan Fang" <yanfang724@gmail.com
> >
> >> >> >>wrote:
> >> >> >> >>> >> >
> >> >> >> >>> >> >> Hi Shekar.
> >> >> >> >>> >> >>
> >> >> >> >>> >> >> Are you running job in local mode or yarn mode? If yarn
> >> >>mode,
> >> >> >>the
> >> >> >> >>> log
> >> >> >> >>> >> >>is in
> >> >> >> >>> >> >> the yarn's container log.
> >> >> >> >>> >> >>
> >> >> >> >>> >> >> Thanks,
> >> >> >> >>> >> >>
> >> >> >> >>> >> >> Fang, Yan
> >> >> >> >>> >> >> yanfang724@gmail.com
> >> >> >> >>> >> >> +1 (206) 849-4108
> >> >> >> >>> >> >>
> >> >> >> >>> >> >>
> >> >> >> >>> >> >> On Tue, Sep 2, 2014 at 11:36 AM, Shekar Tippur
> >> >> >> >>><ct...@gmail.com>
> >> >> >> >>> >> >>wrote:
> >> >> >> >>> >> >>
> >> >> >> >>> >> >> > Chris,
> >> >> >> >>> >> >> >
> >> >> >> >>> >> >> > Got some time to play around a bit more.
> >> >> >> >>> >> >> > I tried to edit
> >> >> >> >>> >> >> >
> >> >> >> >>> >> >> >
> >> >> >> >>> >> >>
> >> >> >> >>> >>
> >> >> >> >>>
> >> >> >> >>>
> >> >> >>
> >> >>
> >>
> >>>>>>>>>>>>>samza-wikipedia/src/main/java/samza/examples/wikipedia/task/Wi
> >>>>>>>>>>>>>ki
> >> >>>>>>>>>>>pe
> >> >> >>>>>>>>>di
> >> >> >> >>>>>>>aFe
> >> >> >> >>> >>>>ed
> >> >> >> >>> >> >>StreamTask.java
> >> >> >> >>> >> >> > to add a logger info statement to tap the incoming
> >> >>message.
> >> >> >>I
> >> >> >> >>>dont
> >> >> >> >>> >>see
> >> >> >> >>> >> >> the
> >> >> >> >>> >> >> > messages being printed to the log file.
> >> >> >> >>> >> >> >
> >> >> >> >>> >> >> > Is this the right place to start?
> >> >> >> >>> >> >> >
> >> >> >> >>> >> >> > public class WikipediaFeedStreamTask implements
> >> >>StreamTask {
> >> >> >> >>> >> >> >
> >> >> >> >>> >> >> >   private static final SystemStream OUTPUT_STREAM =
> >>new
> >> >> >> >>> >> >> > SystemStream("kafka",
> >> >> >> >>> >> >> > "wikipedia-raw");
> >> >> >> >>> >> >> >
> >> >> >> >>> >> >> >   private static final Logger log =
> >> >>LoggerFactory.getLogger
> >> >> >> >>> >> >> > (WikipediaFeedStreamTask.class);
> >> >> >> >>> >> >> >
> >> >> >> >>> >> >> >   @Override
> >> >> >> >>> >> >> >
> >> >> >> >>> >> >> >   public void process(IncomingMessageEnvelope
> >>envelope,
> >> >> >> >>> >> >>MessageCollector
> >> >> >> >>> >> >> > collector, TaskCoordinator coordinator) {
> >> >> >> >>> >> >> >
> >> >> >> >>> >> >> >     Map<String, Object> outgoingMap =
> >> >> >> >>> >> >> > WikipediaFeedEvent.toMap((WikipediaFeedEvent)
> >> >> >> >>> >>envelope.getMessage());
> >> >> >> >>> >> >> >
> >> >> >> >>> >> >> >     log.info(envelope.getMessage().toString());
> >> >> >> >>> >> >> >
> >> >> >> >>> >> >> >     collector.send(new
> >> >> >>OutgoingMessageEnvelope(OUTPUT_STREAM,
> >> >> >> >>> >> >> > outgoingMap));
> >> >> >> >>> >> >> >
> >> >> >> >>> >> >> >   }
> >> >> >> >>> >> >> >
> >> >> >> >>> >> >> > }
> >> >> >> >>> >> >> >
> >> >> >> >>> >> >> >
> >> >> >> >>> >> >> > On Mon, Aug 25, 2014 at 9:01 AM, Chris Riccomini <
> >> >> >> >>> >> >> > criccomini@linkedin.com.invalid> wrote:
> >> >> >> >>> >> >> >
> >> >> >> >>> >> >> > > Hey Shekar,
> >> >> >> >>> >> >> > >
> >> >> >> >>> >> >> > > Your thought process is on the right track. It's
> >> >>probably
> >> >> >> >>>best
> >> >> >> >>> to
> >> >> >> >>> >> >>start
> >> >> >> >>> >> >> > > with hello-samza, and modify it to get what you
> >>want.
> >> >>To
> >> >> >> >>>start
> >> >> >> >>> >>with,
> >> >> >> >>> >> >> > > you'll want to:
> >> >> >> >>> >> >> > >
> >> >> >> >>> >> >> > > 1. Write a simple StreamTask that just does
> >>something
> >> >> >>silly
> >> >> >> >>>like
> >> >> >> >>> >> >>just
> >> >> >> >>> >> >> > > print messages that it receives.
> >> >> >> >>> >> >> > > 2. Write a configuration for the job that consumes
> >>from
> >> >> >>just
> >> >> >> >>>the
> >> >> >> >>> >> >>stream
> >> >> >> >>> >> >> > > (alerts from different sources).
> >> >> >> >>> >> >> > > 3. Run this to make sure you've got it working.
> >> >> >> >>> >> >> > > 4. Now add your table join. This can be either a
> >> >> >>change-data
> >> >> >> >>> >>capture
> >> >> >> >>> >> >> > (CDC)
> >> >> >> >>> >> >> > > stream, or via a remote DB call.
> >> >> >> >>> >> >> > >
> >> >> >> >>> >> >> > > That should get you to a point where you've got your
> >> >>job
> >> >> >>up
> >> >> >> >>>and
> >> >> >> >>> >> >> running.
> >> >> >> >>> >> >> > > From there, you could create your own Maven project,
> >> >>and
> >> >> >> >>>migrate
> >> >> >> >>> >> >>your
> >> >> >> >>> >> >> > code
> >> >> >> >>> >> >> > > over accordingly.
> >> >> >> >>> >> >> > >
> >> >> >> >>> >> >> > > Cheers,
> >> >> >> >>> >> >> > > Chris
> >> >> >> >>> >> >> > >
> >> >> >> >>> >> >> > > On 8/24/14 1:42 AM, "Shekar Tippur"
> >><ctippur@gmail.com
> >> >
> >> >> >> >>>wrote:
> >> >> >> >>> >> >> > >
> >> >> >> >>> >> >> > > >Chris,
> >> >> >> >>> >> >> > > >
> >> >> >> >>> >> >> > > >I have gone thro the documentation and decided that
> >> >>the
> >> >> >> >>>option
> >> >> >> >>> >> >>that is
> >> >> >> >>> >> >> > > >most
> >> >> >> >>> >> >> > > >suitable for me is stream-table.
> >> >> >> >>> >> >> > > >
> >> >> >> >>> >> >> > > >I see the following things:
> >> >> >> >>> >> >> > > >
> >> >> >> >>> >> >> > > >1. Point samza to a table (database)
> >> >> >> >>> >> >> > > >2. Point Samza to a stream - Alert stream from
> >> >>different
> >> >> >> >>> sources
> >> >> >> >>> >> >> > > >3. Join key like a hostname
> >> >> >> >>> >> >> > > >
> >> >> >> >>> >> >> > > >I have Hello Samza working. To extend that to do
> >>what
> >> >>my
> >> >> >> >>>needs
> >> >> >> >>> >> >>are, I
> >> >> >> >>> >> >> am
> >> >> >> >>> >> >> > > >not sure where to start (Needs more code change OR
> >> >> >> >>> configuration
> >> >> >> >>> >> >> changes
> >> >> >> >>> >> >> > > >OR
> >> >> >> >>> >> >> > > >both)?
> >> >> >> >>> >> >> > > >
> >> >> >> >>> >> >> > > >I have gone thro
> >> >> >> >>> >> >> > > >
> >> >> >> >>> >> >> >
> >> >> >> >>> >> >>
> >> >> >> >>> >> >>
> >> >> >> >>> >>
> >> >> >> >>> >>
> >> >> >> >>>
> >> >> >> >>>
> >> >> >>
> >> >>
> >>
> http://samza.incubator.apache.org/learn/documentation/latest/api/overvie
> >> >> >> >>>w
> >> >> >> >>> >> >> > > .
> >> >> >> >>> >> >> > > >html
> >> >> >> >>> >> >> > > >
> >> >> >> >>> >> >> > > >Is my thought process on the right track? Can you
> >> >>please
> >> >> >> >>>point
> >> >> >> >>> >>me
> >> >> >> >>> >> >>to
> >> >> >> >>> >> >> the
> >> >> >> >>> >> >> > > >right direction?
> >> >> >> >>> >> >> > > >
> >> >> >> >>> >> >> > > >- Shekar
> >> >> >> >>> >> >> > > >
> >> >> >> >>> >> >> > > >
> >> >> >> >>> >> >> > > >
> >> >> >> >>> >> >> > > >On Thu, Aug 21, 2014 at 1:08 PM, Shekar Tippur
> >> >> >> >>> >><ct...@gmail.com>
> >> >> >> >>> >> >> > wrote:
> >> >> >> >>> >> >> > > >
> >> >> >> >>> >> >> > > >> Chris,
> >> >> >> >>> >> >> > > >>
> >> >> >> >>> >> >> > > >> This is perfectly good answer. I will start
> >>poking
> >> >>more
> >> >> >> >>>into
> >> >> >> >>> >> >>option
> >> >> >> >>> >> >> > #4.
> >> >> >> >>> >> >> > > >>
> >> >> >> >>> >> >> > > >> - Shekar
> >> >> >> >>> >> >> > > >>
> >> >> >> >>> >> >> > > >>
> >> >> >> >>> >> >> > > >> On Thu, Aug 21, 2014 at 1:05 PM, Chris Riccomini
> >><
> >> >> >> >>> >> >> > > >> criccomini@linkedin.com.invalid> wrote:
> >> >> >> >>> >> >> > > >>
> >> >> >> >>> >> >> > > >>> Hey Shekar,
> >> >> >> >>> >> >> > > >>>
> >> >> >> >>> >> >> > > >>> Your two options are really (3) or (4), then.
> >>You
> >> >>can
> >> >> >> >>>either
> >> >> >> >>> >>run
> >> >> >> >>> >> >> some
> >> >> >> >>> >> >> > > >>> external DB that holds the data set, and you can
> >> >> >>query it
> >> >> >> >>> >>from a
> >> >> >> >>> >> >> > > >>> StreamTask, or you can use Samza's state store
> >> >> >>feature to
> >> >> >> >>> >>push
> >> >> >> >>> >> >>data
> >> >> >> >>> >> >> > > >>>into a
> >> >> >> >>> >> >> > > >>> stream that you can then store in a partitioned
> >> >> >>key-value
> >> >> >> >>> >>store
> >> >> >> >>> >> >> along
> >> >> >> >>> >> >> > > >>>with
> >> >> >> >>> >> >> > > >>> your StreamTasks. There is some documentation
> >>here
> >> >> >>about
> >> >> >> >>>the
> >> >> >> >>> >> >>state
> >> >> >> >>> >> >> > > >>>store
> >> >> >> >>> >> >> > > >>> approach:
> >> >> >> >>> >> >> > > >>>
> >> >> >> >>> >> >> > > >>>
> >> >> >> >>> >> >> > > >>>
> >> >> >> >>> >> >> > > >>>
> >> >> >> >>> >> >> > >
> >> >> >> >>> >> >>
> >> >> >> >>> >>
> >> >> >> >>>
> >> >> >> >>>
> >> >> >>
> >> >>
> >>
> http://samza.incubator.apache.org/learn/documentation/0.7.0/container/st
> >> >> >> >>> >> >> > > >>>ate
> >> >> >> >>> >> >> > > >>> -management.html
> >> >> >> >>> >> >> > > >>>
> >> >> >> >>> >> >> > > >>><
> >> >> >> >>> >> >> > >
> >> >> >> >>> >> >>
> >> >> >> >>>
> >> >> >> >>>>>
> >> >> >>
> >> >>
> http://samza.incubator.apache.org/learn/documentation/0.7.0/container/
> >> >> >> >>>>>s
> >> >> >> >>> >> >> > > >>>tate-management.html>
> >> >> >> >>> >> >> > > >>>
> >> >> >> >>> >> >> > > >>>
> >> >> >> >>> >> >> > > >>> (4) is going to require more up front effort
> >>from
> >> >>you,
> >> >> >> >>>since
> >> >> >> >>> >> >>you'll
> >> >> >> >>> >> >> > > >>>have
> >> >> >> >>> >> >> > > >>> to understand how Kafka's partitioning model
> >>works,
> >> >> >>and
> >> >> >> >>> setup
> >> >> >> >>> >> >>some
> >> >> >> >>> >> >> > > >>> pipeline to push the updates for your state. In
> >>the
> >> >> >>long
> >> >> >> >>> >>run, I
> >> >> >> >>> >> >> > believe
> >> >> >> >>> >> >> > > >>> it's the better approach, though. Local lookups
> >>on
> >> >>a
> >> >> >> >>> >>key-value
> >> >> >> >>> >> >> store
> >> >> >> >>> >> >> > > >>> should be faster than doing remote RPC calls to
> >>a
> >> >>DB
> >> >> >>for
> >> >> >> >>> >>every
> >> >> >> >>> >> >> > message.
> >> >> >> >>> >> >> > > >>>
> >> >> >> >>> >> >> > > >>> I'm sorry I can't give you a more definitive
> >> >>answer.
> >> >> >>It's
> >> >> >> >>> >>really
> >> >> >> >>> >> >> > about
> >> >> >> >>> >> >> > > >>> trade-offs.
> >> >> >> >>> >> >> > > >>>
> >> >> >> >>> >> >> > > >>> Cheers,
> >> >> >> >>> >> >> > > >>> Chris
> >> >> >> >>> >> >> > > >>>
> >> >> >> >>> >> >> > > >>> On 8/21/14 12:22 PM, "Shekar Tippur"
> >> >> >><ct...@gmail.com>
> >> >> >> >>> >>wrote:
> >> >> >> >>> >> >> > > >>>
> >> >> >> >>> >> >> > > >>> >Chris,
> >> >> >> >>> >> >> > > >>> >
> >> >> >> >>> >> >> > > >>> >A big thanks for a swift response. The data
> >>set is
> >> >> >>huge
> >> >> >> >>>and
> >> >> >> >>> >>the
> >> >> >> >>> >> >> > > >>>frequency
> >> >> >> >>> >> >> > > >>> >is in burst.
> >> >> >> >>> >> >> > > >>> >What do you suggest?
> >> >> >> >>> >> >> > > >>> >
> >> >> >> >>> >> >> > > >>> >- Shekar
> >> >> >> >>> >> >> > > >>> >
> >> >> >> >>> >> >> > > >>> >
> >> >> >> >>> >> >> > > >>> >On Thu, Aug 21, 2014 at 11:05 AM, Chris
> >>Riccomini
> >> >><
> >> >> >> >>> >> >> > > >>> >criccomini@linkedin.com.invalid> wrote:
> >> >> >> >>> >> >> > > >>> >
> >> >> >> >>> >> >> > > >>> >> Hey Shekar,
> >> >> >> >>> >> >> > > >>> >>
> >> >> >> >>> >> >> > > >>> >> This is feasible, and you are on the right
> >> >>thought
> >> >> >> >>> >>process.
> >> >> >> >>> >> >> > > >>> >>
> >> >> >> >>> >> >> > > >>> >> For the sake of discussion, I'm going to
> >>pretend
> >> >> >>that
> >> >> >> >>>you
> >> >> >> >>> >> >>have a
> >> >> >> >>> >> >> > > >>>Kafka
> >> >> >> >>> >> >> > > >>> >> topic called "PageViewEvent", which has just
> >> >>the IP
> >> >> >> >>> >>address
> >> >> >> >>> >> >>that
> >> >> >> >>> >> >> > was
> >> >> >> >>> >> >> > > >>> >>used
> >> >> >> >>> >> >> > > >>> >> to view a page. These messages will be logged
> >> >>every
> >> >> >> >>>time
> >> >> >> >>> a
> >> >> >> >>> >> >>page
> >> >> >> >>> >> >> > view
> >> >> >> >>> >> >> > > >>> >> happens. I'm also going to pretend that you
> >>have
> >> >> >>some
> >> >> >> >>> >>state
> >> >> >> >>> >> >> called
> >> >> >> >>> >> >> > > >>> >>"IPGeo"
> >> >> >> >>> >> >> > > >>> >> (e.g. The maxmind data set). In this example,
> >> >>we'll
> >> >> >> >>>want
> >> >> >> >>> >>to
> >> >> >> >>> >> >>join
> >> >> >> >>> >> >> > the
> >> >> >> >>> >> >> > > >>> >> long/lat geo information from IPGeo to the
> >> >> >> >>>PageViewEvent,
> >> >> >> >>> >>and
> >> >> >> >>> >> >> send
> >> >> >> >>> >> >> > > >>>it
> >> >> >> >>> >> >> > > >>> >>to a
> >> >> >> >>> >> >> > > >>> >> new topic: PageViewEventsWithGeo.
> >> >> >> >>> >> >> > > >>> >>
> >> >> >> >>> >> >> > > >>> >> You have several options on how to implement
> >> >>this
> >> >> >> >>> example.
> >> >> >> >>> >> >> > > >>> >>
> >> >> >> >>> >> >> > > >>> >> 1. If your joining data set (IPGeo) is
> >> >>relatively
> >> >> >> >>>small
> >> >> >> >>> >>and
> >> >> >> >>> >> >> > changes
> >> >> >> >>> >> >> > > >>> >> infrequently, you can just pack it up in your
> >> >>jar
> >> >> >>or
> >> >> >> >>>.tgz
> >> >> >> >>> >> >>file,
> >> >> >> >>> >> >> > and
> >> >> >> >>> >> >> > > >>> open
> >> >> >> >>> >> >> > > >>> >> it open in every StreamTask.
> >> >> >> >>> >> >> > > >>> >> 2. If your data set is small, but changes
> >> >>somewhat
> >> >> >> >>> >> >>frequently,
> >> >> >> >>> >> >> you
> >> >> >> >>> >> >> > > >>>can
> >> >> >> >>> >> >> > > >>> >> throw the data set on some HTTP/HDFS/S3
> >>server
> >> >> >> >>>somewhere,
> >> >> >> >>> >>and
> >> >> >> >>> >> >> have
> >> >> >> >>> >> >> > > >>>your
> >> >> >> >>> >> >> > > >>> >> StreamTask refresh it periodically by
> >> >> >>re-downloading
> >> >> >> >>>it.
> >> >> >> >>> >> >> > > >>> >> 3. You can do remote RPC calls for the IPGeo
> >> >>data
> >> >> >>on
> >> >> >> >>> every
> >> >> >> >>> >> >>page
> >> >> >> >>> >> >> > view
> >> >> >> >>> >> >> > > >>> >>event
> >> >> >> >>> >> >> > > >>> >> by query some remote service or DB (e.g.
> >> >> >>Cassandra).
> >> >> >> >>> >> >> > > >>> >> 4. You can use Samza's state feature to set
> >>your
> >> >> >>IPGeo
> >> >> >> >>> >>data
> >> >> >> >>> >> >>as a
> >> >> >> >>> >> >> > > >>>series
> >> >> >> >>> >> >> > > >>> >>of
> >> >> >> >>> >> >> > > >>> >> messages to a log-compacted Kafka topic
> >> >> >> >>> >> >> > > >>> >> (
> >> >> >> >>> >> >>
> >> >> >>https://cwiki.apache.org/confluence/display/KAFKA/Log+Compaction
> >> >> >> >>> >> >> > ),
> >> >> >> >>> >> >> > > >>> and
> >> >> >> >>> >> >> > > >>> >> configure your Samza job to read this topic
> >>as a
> >> >> >> >>> bootstrap
> >> >> >> >>> >> >> stream
> >> >> >> >>> >> >> > > >>> >> (
> >> >> >> >>> >> >> > > >>> >>
> >> >> >> >>> >> >> > > >>> >>
> >> >> >> >>> >> >> > > >>>
> >> >> >> >>> >> >> > > >>>
> >> >> >> >>> >> >> > >
> >> >> >> >>> >> >>
> >> >> >> >>> >>
> >> >> >> >>>
> >> >> >> >>>
> >> >> >>
> >> >>
> >>
> http://samza.incubator.apache.org/learn/documentation/0.7.0/container/st
> >> >> >> >>> >> >> > > >>>r
> >> >> >> >>> >> >> > > >>> >>e
> >> >> >> >>> >> >> > > >>> >> ams.html).
> >> >> >> >>> >> >> > > >>> >>
> >> >> >> >>> >> >> > > >>> >> For (4), you'd have to partition the IPGeo
> >>state
> >> >> >>topic
> >> >> >> >>> >> >>according
> >> >> >> >>> >> >> > to
> >> >> >> >>> >> >> > > >>>the
> >> >> >> >>> >> >> > > >>> >> same key as PageViewEvent. If PageViewEvent
> >>were
> >> >> >> >>> >>partitioned
> >> >> >> >>> >> >>by,
> >> >> >> >>> >> >> > > >>>say,
> >> >> >> >>> >> >> > > >>> >> member ID, but you want your IPGeo state
> >>topic
> >> >>to
> >> >> >>be
> >> >> >> >>> >> >>partitioned
> >> >> >> >>> >> >> > by
> >> >> >> >>> >> >> > > >>>IP
> >> >> >> >>> >> >> > > >>> >> address, then you'd have to have an upstream
> >>job
> >> >> >>that
> >> >> >> >>> >> >> > re-partitioned
> >> >> >> >>> >> >> > > >>> >> PageViewEvent into some new topic by IP
> >>address.
> >> >> >>This
> >> >> >> >>>new
> >> >> >> >>> >> >>topic
> >> >> >> >>> >> >> > will
> >> >> >> >>> >> >> > > >>> >>have
> >> >> >> >>> >> >> > > >>> >> to have the same number of partitions as the
> >> >>IPGeo
> >> >> >> >>>state
> >> >> >> >>> >> >>topic
> >> >> >> >>> >> >> (if
> >> >> >> >>> >> >> > > >>> IPGeo
> >> >> >> >>> >> >> > > >>> >> has 8 partitions, then the new
> >> >> >> >>>PageViewEventRepartitioned
> >> >> >> >>> >> >>topic
> >> >> >> >>> >> >> > > >>>needs 8
> >> >> >> >>> >> >> > > >>> >>as
> >> >> >> >>> >> >> > > >>> >> well). This will cause your
> >> >> >>PageViewEventRepartitioned
> >> >> >> >>> >>topic
> >> >> >> >>> >> >>and
> >> >> >> >>> >> >> > > >>>your
> >> >> >> >>> >> >> > > >>> >> IPGeo state topic to be aligned such that the
> >> >> >> >>>StreamTask
> >> >> >> >>> >>that
> >> >> >> >>> >> >> gets
> >> >> >> >>> >> >> > > >>>page
> >> >> >> >>> >> >> > > >>> >> views for IP address X will also have the
> >>IPGeo
> >> >> >> >>> >>information
> >> >> >> >>> >> >>for
> >> >> >> >>> >> >> IP
> >> >> >> >>> >> >> > > >>> >>address
> >> >> >> >>> >> >> > > >>> >> X.
> >> >> >> >>> >> >> > > >>> >>
> >> >> >> >>> >> >> > > >>> >> Which strategy you pick is really up to you.
> >>:)
> >> >> >>(4) is
> >> >> >> >>> the
> >> >> >> >>> >> >>most
> >> >> >> >>> >> >> > > >>> >> complicated, but also the most flexible, and
> >> >>most
> >> >> >> >>> >> >>operationally
> >> >> >> >>> >> >> > > >>>sound.
> >> >> >> >>> >> >> > > >>> >>(1)
> >> >> >> >>> >> >> > > >>> >> is the easiest if it fits your needs.
> >> >> >> >>> >> >> > > >>> >>
> >> >> >> >>> >> >> > > >>> >> Cheers,
> >> >> >> >>> >> >> > > >>> >> Chris
> >> >> >> >>> >> >> > > >>> >>
> >> >> >> >>> >> >> > > >>> >> On 8/21/14 10:15 AM, "Shekar Tippur"
> >> >> >> >>><ct...@gmail.com>
> >> >> >> >>> >> >>wrote:
> >> >> >> >>> >> >> > > >>> >>
> >> >> >> >>> >> >> > > >>> >> >Hello,
> >> >> >> >>> >> >> > > >>> >> >
> >> >> >> >>> >> >> > > >>> >> >I am new to Samza. I have just installed
> >>Hello
> >> >> >>Samza
> >> >> >> >>>and
> >> >> >> >>> >> >>got it
> >> >> >> >>> >> >> > > >>> >>working.
> >> >> >> >>> >> >> > > >>> >> >
> >> >> >> >>> >> >> > > >>> >> >Here is the use case for which I am trying
> >>to
> >> >>use
> >> >> >> >>>Samza:
> >> >> >> >>> >> >> > > >>> >> >
> >> >> >> >>> >> >> > > >>> >> >
> >> >> >> >>> >> >> > > >>> >> >1. Cache the contextual information which
> >> >>contains
> >> >> >> >>>more
> >> >> >> >>> >> >> > information
> >> >> >> >>> >> >> > > >>> >>about
> >> >> >> >>> >> >> > > >>> >> >the hostname or IP address using
> >> >>Samza/Yarn/Kafka
> >> >> >> >>> >> >> > > >>> >> >2. Collect alert and metric events which
> >> >>contain
> >> >> >> >>>either
> >> >> >> >>> >> >> hostname
> >> >> >> >>> >> >> > > >>>or IP
> >> >> >> >>> >> >> > > >>> >> >address
> >> >> >> >>> >> >> > > >>> >> >3. Append contextual information to the
> >>alert
> >> >>and
> >> >> >> >>>metric
> >> >> >> >>> >>and
> >> >> >> >>> >> >> > > >>>insert to
> >> >> >> >>> >> >> > > >>> >>a
> >> >> >> >>> >> >> > > >>> >> >Kafka queue from which other subscribers
> >>read
> >> >>off
> >> >> >>of.
> >> >> >> >>> >> >> > > >>> >> >
> >> >> >> >>> >> >> > > >>> >> >Can you please shed some light on
> >> >> >> >>> >> >> > > >>> >> >
> >> >> >> >>> >> >> > > >>> >> >1. Is this feasible?
> >> >> >> >>> >> >> > > >>> >> >2. Am I on the right thought process
> >> >> >> >>> >> >> > > >>> >> >3. How do I start
> >> >> >> >>> >> >> > > >>> >> >
> >> >> >> >>> >> >> > > >>> >> >I now have 1 & 2 of them working
> >>disparately. I
> >> >> >>need
> >> >> >> >>>to
> >> >> >> >>> >> >> integrate
> >> >> >> >>> >> >> > > >>> them.
> >> >> >> >>> >> >> > > >>> >> >
> >> >> >> >>> >> >> > > >>> >> >Appreciate any input.
> >> >> >> >>> >> >> > > >>> >> >
> >> >> >> >>> >> >> > > >>> >> >- Shekar
> >> >> >> >>> >> >> > > >>> >>
> >> >> >> >>> >> >> > > >>> >>
> >> >> >> >>> >> >> > > >>>
> >> >> >> >>> >> >> > > >>>
> >> >> >> >>> >> >> > > >>
> >> >> >> >>> >> >> > >
> >> >> >> >>> >> >> > >
> >> >> >> >>> >> >> >
> >> >> >> >>> >> >>
> >> >> >> >>> >>
> >> >> >> >>> >>
> >> >> >> >>>
> >> >> >> >>>
> >> >> >> >>
> >> >> >>
> >> >> >>
> >> >>
> >> >>
> >>
> >>
>
>

Re: Samza as a Caching layer

Posted by Chris Riccomini <cr...@linkedin.com.INVALID>.
Hey Shekar,

Sorry that you're having such a tough time with this. I'll keep trying to
help as best I can.

Do you see slf4j-log4j12 in your job's .tgz file?

Cheers,
Chris

On 9/3/14 2:46 PM, "Shekar Tippur" <ct...@gmail.com> wrote:

>Chris,
>
>I have added the below dependencies and I still get the same message.
>
>
>      <dependency>
>
>        <groupId>org.slf4j</groupId>
>
>        <artifactId>slf4j-api</artifactId>
>
>        <version>1.7.7</version>
>
>      </dependency>
>
>      <dependency>
>
>          <groupId>org.slf4j</groupId>
>
>          <artifactId>slf4j-log4j12</artifactId>
>
>          <version>1.5.6</version>
>
>      </dependency>
>
>
>On Wed, Sep 3, 2014 at 9:01 AM, Chris Riccomini <
>criccomini@linkedin.com.invalid> wrote:
>
>> Hey Shekar,
>>
>> The SLF4J stuff is saying that you don't have an slf4j binding on your
>> classpath. Try adding slf4j-log4j as a runtime dependency on your
>>project.
>>
>> Cheers,
>> Chris
>>
>> On 9/2/14 3:24 PM, "Shekar Tippur" <ct...@gmail.com> wrote:
>>
>> >Chris ,
>> >
>> >In the current state, I just want Samza to connect to 127.0.0.1.
>> >
>> >I have set YARN_HOME to
>> >
>> >$ echo $YARN_HOME
>> >
>> >/home/ctippur/hello-samza/deploy/yarn
>> >
>> >I still dont see anything on hadoop console.
>> >
>> > Also, I see this during startup
>> >
>> >
>> >SLF4J: Failed to load class "org.slf4j.impl.StaticLoggerBinder".
>> >
>> >SLF4J: Defaulting to no-operation (NOP) logger implementation
>> >
>> >SLF4J: See http://www.slf4j.org/codes.html#StaticLoggerBinder for
>>further
>> >details.
>> >
>> >SLF4J: Failed to load class "org.slf4j.impl.StaticLoggerBinder".
>> >
>> >SLF4J: Defaulting to no-operation (NOP) logger implementation
>> >
>> >SLF4J: See http://www.slf4j.org/codes.html#StaticLoggerBinder for
>>further
>> >details.
>> >
>> >- Shekar
>> >
>> >
>> >On Tue, Sep 2, 2014 at 2:20 PM, Chris Riccomini <
>> >criccomini@linkedin.com.invalid> wrote:
>> >
>> >> Hey Shekar,
>> >>
>> >> Ah ha. In that case, do you expect your SamzaContainer to try to
>>connect
>> >> to the RM at 127.0.0.1, or do you expect it to try to connect to some
>> >> remote RM? If you expect it to try and connect to a remote RM, it's
>>not
>> >> doing that. Perhaps because YARN_HOME isn't set.
>> >>
>> >> If you go to your RM's web interface, how many active nodes do you
>>see
>> >> listed?
>> >>
>> >> Cheers,
>> >> Chris
>> >>
>> >> On 9/2/14 2:17 PM, "Shekar Tippur" <ct...@gmail.com> wrote:
>> >>
>> >> >Chris ..
>> >> >
>> >> >I am using a rhel server to host all the components (Yarn, kafka,
>> >>samza).
>> >> >I
>> >> >dont have ACLs open to wikipedia.
>> >> >I am following ..
>> >> >
>> >>
>> >>
>> 
>>http://samza.incubator.apache.org/learn/tutorials/latest/run-hello-samza-
>> >>w
>> >> >ithout-internet.html
>> >> >
>> >> >- Shekar
>> >> >
>> >> >
>> >> >
>> >> >On Tue, Sep 2, 2014 at 2:13 PM, Chris Riccomini <
>> >> >criccomini@linkedin.com.invalid> wrote:
>> >> >
>> >> >> Hey Shekar,
>> >> >>
>> >> >> Can you try changing that to:
>> >> >>
>> >> >>   http://127.0.0.1:8088/cluster
>> >> >>
>> >> >>
>> >> >> And see if you can connect?
>> >> >>
>> >> >> Cheers,
>> >> >> Chris
>> >> >>
>> >> >> On 9/2/14 1:21 PM, "Shekar Tippur" <ct...@gmail.com> wrote:
>> >> >>
>> >> >> >Other observation is ..
>> >> >> >
>> >> >> >http://10.132.62.185:8088/cluster shows that no applications are
>> >> >>running.
>> >> >> >
>> >> >> >- Shekar
>> >> >> >
>> >> >> >
>> >> >> >
>> >> >> >
>> >> >> >On Tue, Sep 2, 2014 at 1:15 PM, Shekar Tippur <ct...@gmail.com>
>> >> >>wrote:
>> >> >> >
>> >> >> >> Yarn seem to be running ..
>> >> >> >>
>> >> >> >> yarn      5462  0.0  2.0 1641296 161404 ?      Sl   Jun20
>>95:26
>> >> >> >> /usr/java/jdk1.6.0_31/bin/java -Dproc_resourcemanager -Xmx1000m
>> >> >> >> -Dhadoop.log.dir=/var/log/hadoop-yarn
>> >> >> >>-Dyarn.log.dir=/var/log/hadoop-yarn
>> >> >> >> -Dhadoop.log.file=yarn-yarn-resourcemanager-pppdc9prd2y2.log
>> >> >> >> -Dyarn.log.file=yarn-yarn-resourcemanager-pppdc9prd2y2.log
>> >> >> >> -Dyarn.home.dir=/usr/lib/hadoop-yarn
>> >> >> >>-Dhadoop.home.dir=/usr/lib/hadoop-yarn
>> >> >> >> -Dhadoop.root.logger=INFO,RFA -Dyarn.root.logger=INFO,RFA
>> >> >> >> -Djava.library.path=/usr/lib/hadoop/lib/native -classpath
>> >> >> >>
>> >> >>
>> >>
>> 
>>>>>>>>/etc/hadoop/conf:/etc/hadoop/conf:/etc/hadoop/conf:/usr/lib/hadoop/
>>>>>>>>li
>> >>>>>>b/
>> >> >>>>*:
>> >> >>
>> >>
>> 
>>>>>>>>/usr/lib/hadoop/.//*:/usr/lib/hadoop-hdfs/./:/usr/lib/hadoop-hdfs/l
>>>>>>>>ib
>> >>>>>>/*
>> >> >>>>:/
>> >> >>
>> >>
>> 
>>>>>>>>usr/lib/hadoop-hdfs/.//*:/usr/lib/hadoop-yarn/lib/*:/usr/lib/hadoop
>>>>>>>>-y
>> >>>>>>ar
>> >> >>>>n/
>> >> >>
>> >>
>> 
>>>>>>>>.//*:/usr/lib/hadoop-mapreduce/lib/*:/usr/lib/hadoop-mapreduce/.//*
>>>>>>>>:/
>> >>>>>>us
>> >> >>>>r/
>> >> >>
>> >>
>> 
>>>>>>>>lib/hadoop-yarn/.//*:/usr/lib/hadoop-yarn/lib/*:/etc/hadoop/conf/rm
>>>>>>>>-c
>> >>>>>>on
>> >> >>>>fi
>> >> >> >>g/log4j.properties
>> >> >> >> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager
>> >> >> >>
>> >> >> >> I can tail kafka topic as well ..
>> >> >> >>
>> >> >> >> deploy/kafka/bin/kafka-console-consumer.sh  --zookeeper
>> >> >>localhost:2181
>> >> >> >>--topic wikipedia-raw
>> >> >> >>
>> >> >> >>
>> >> >> >>
>> >> >> >>
>> >> >> >> On Tue, Sep 2, 2014 at 1:04 PM, Chris Riccomini <
>> >> >> >> criccomini@linkedin.com.invalid> wrote:
>> >> >> >>
>> >> >> >>> Hey Shekar,
>> >> >> >>>
>> >> >> >>> It looks like your job is hanging trying to connect to the RM
>>on
>> >> >>your
>> >> >> >>> localhost. I thought that you said that your job was running
>>in
>> >> >>local
>> >> >> >>> mode. If so, it should be using the LocalJobFactory. If not,
>>and
>> >>you
>> >> >> >>> intend to run on YARN, is your YARN RM up and running on
>> >>localhost?
>> >> >> >>>
>> >> >> >>> Cheers,
>> >> >> >>> Chris
>> >> >> >>>
>> >> >> >>> On 9/2/14 12:22 PM, "Shekar Tippur" <ct...@gmail.com> wrote:
>> >> >> >>>
>> >> >> >>> >Chris ..
>> >> >> >>> >
>> >> >> >>> >$ cat ./deploy/samza/undefined-samza-container-name.log
>> >> >> >>> >
>> >> >> >>> >2014-09-02 11:17:58 JobRunner [INFO] job factory:
>> >> >> >>> >org.apache.samza.job.yarn.YarnJobFactory
>> >> >> >>> >
>> >> >> >>> >2014-09-02 11:17:59 ClientHelper [INFO] trying to connect to
>>RM
>> >> >> >>> >127.0.0.1:8032
>> >> >> >>> >
>> >> >> >>> >2014-09-02 11:17:59 NativeCodeLoader [WARN] Unable to load
>> >> >> >>>native-hadoop
>> >> >> >>> >library for your platform... using builtin-java classes where
>> >> >> >>>applicable
>> >> >> >>> >
>> >> >> >>> >2014-09-02 11:17:59 RMProxy [INFO] Connecting to
>>ResourceManager
>> >> >>at /
>> >> >> >>> >127.0.0.1:8032
>> >> >> >>> >
>> >> >> >>> >
>> >> >> >>> >and Log4j ..
>> >> >> >>> >
>> >> >> >>> ><!DOCTYPE log4j:configuration SYSTEM "log4j.dtd">
>> >> >> >>> >
>> >> >> >>> ><log4j:configuration
>> >> >>xmlns:log4j="http://jakarta.apache.org/log4j/">
>> >> >> >>> >
>> >> >> >>> >  <appender name="RollingAppender"
>> >> >> >>> >class="org.apache.log4j.DailyRollingFileAppender">
>> >> >> >>> >
>> >> >> >>> >     <param name="File"
>> >> >> >>> >value="${samza.log.dir}/${samza.container.name}.log"
>> >> >> >>> >/>
>> >> >> >>> >
>> >> >> >>> >     <param name="DatePattern" value="'.'yyyy-MM-dd" />
>> >> >> >>> >
>> >> >> >>> >     <layout class="org.apache.log4j.PatternLayout">
>> >> >> >>> >
>> >> >> >>> >      <param name="ConversionPattern" value="%d{yyyy-MM-dd
>> >> >>HH:mm:ss}
>> >> >> >>> %c{1}
>> >> >> >>> >[%p] %m%n" />
>> >> >> >>> >
>> >> >> >>> >     </layout>
>> >> >> >>> >
>> >> >> >>> >  </appender>
>> >> >> >>> >
>> >> >> >>> >  <root>
>> >> >> >>> >
>> >> >> >>> >    <priority value="info" />
>> >> >> >>> >
>> >> >> >>> >    <appender-ref ref="RollingAppender"/>
>> >> >> >>> >
>> >> >> >>> >  </root>
>> >> >> >>> >
>> >> >> >>> ></log4j:configuration>
>> >> >> >>> >
>> >> >> >>> >
>> >> >> >>> >On Tue, Sep 2, 2014 at 12:02 PM, Chris Riccomini <
>> >> >> >>> >criccomini@linkedin.com.invalid> wrote:
>> >> >> >>> >
>> >> >> >>> >> Hey Shekar,
>> >> >> >>> >>
>> >> >> >>> >> Can you attach your log files? I'm wondering if it's a
>> >> >> >>>mis-configured
>> >> >> >>> >> log4j.xml (or missing slf4j-log4j jar), which is leading to
>> >> >>nearly
>> >> >> >>> empty
>> >> >> >>> >> log files. Also, I'm wondering if the job starts fully.
>> >>Anything
>> >> >>you
>> >> >> >>> can
>> >> >> >>> >> attach would be helpful.
>> >> >> >>> >>
>> >> >> >>> >> Cheers,
>> >> >> >>> >> Chris
>> >> >> >>> >>
>> >> >> >>> >> On 9/2/14 11:43 AM, "Shekar Tippur" <ct...@gmail.com>
>> wrote:
>> >> >> >>> >>
>> >> >> >>> >> >I am running in local mode.
>> >> >> >>> >> >
>> >> >> >>> >> >S
>> >> >> >>> >> >On Sep 2, 2014 11:42 AM, "Yan Fang" <ya...@gmail.com>
>> >> >>wrote:
>> >> >> >>> >> >
>> >> >> >>> >> >> Hi Shekar.
>> >> >> >>> >> >>
>> >> >> >>> >> >> Are you running job in local mode or yarn mode? If yarn
>> >>mode,
>> >> >>the
>> >> >> >>> log
>> >> >> >>> >> >>is in
>> >> >> >>> >> >> the yarn's container log.
>> >> >> >>> >> >>
>> >> >> >>> >> >> Thanks,
>> >> >> >>> >> >>
>> >> >> >>> >> >> Fang, Yan
>> >> >> >>> >> >> yanfang724@gmail.com
>> >> >> >>> >> >> +1 (206) 849-4108
>> >> >> >>> >> >>
>> >> >> >>> >> >>
>> >> >> >>> >> >> On Tue, Sep 2, 2014 at 11:36 AM, Shekar Tippur
>> >> >> >>><ct...@gmail.com>
>> >> >> >>> >> >>wrote:
>> >> >> >>> >> >>
>> >> >> >>> >> >> > Chris,
>> >> >> >>> >> >> >
>> >> >> >>> >> >> > Got some time to play around a bit more.
>> >> >> >>> >> >> > I tried to edit
>> >> >> >>> >> >> >
>> >> >> >>> >> >> >
>> >> >> >>> >> >>
>> >> >> >>> >>
>> >> >> >>>
>> >> >> >>>
>> >> >>
>> >>
>> 
>>>>>>>>>>>>>samza-wikipedia/src/main/java/samza/examples/wikipedia/task/Wi
>>>>>>>>>>>>>ki
>> >>>>>>>>>>>pe
>> >> >>>>>>>>>di
>> >> >> >>>>>>>aFe
>> >> >> >>> >>>>ed
>> >> >> >>> >> >>StreamTask.java
>> >> >> >>> >> >> > to add a logger info statement to tap the incoming
>> >>message.
>> >> >>I
>> >> >> >>>dont
>> >> >> >>> >>see
>> >> >> >>> >> >> the
>> >> >> >>> >> >> > messages being printed to the log file.
>> >> >> >>> >> >> >
>> >> >> >>> >> >> > Is this the right place to start?
>> >> >> >>> >> >> >
>> >> >> >>> >> >> > public class WikipediaFeedStreamTask implements
>> >>StreamTask {
>> >> >> >>> >> >> >
>> >> >> >>> >> >> >   private static final SystemStream OUTPUT_STREAM =
>>new
>> >> >> >>> >> >> > SystemStream("kafka",
>> >> >> >>> >> >> > "wikipedia-raw");
>> >> >> >>> >> >> >
>> >> >> >>> >> >> >   private static final Logger log =
>> >>LoggerFactory.getLogger
>> >> >> >>> >> >> > (WikipediaFeedStreamTask.class);
>> >> >> >>> >> >> >
>> >> >> >>> >> >> >   @Override
>> >> >> >>> >> >> >
>> >> >> >>> >> >> >   public void process(IncomingMessageEnvelope
>>envelope,
>> >> >> >>> >> >>MessageCollector
>> >> >> >>> >> >> > collector, TaskCoordinator coordinator) {
>> >> >> >>> >> >> >
>> >> >> >>> >> >> >     Map<String, Object> outgoingMap =
>> >> >> >>> >> >> > WikipediaFeedEvent.toMap((WikipediaFeedEvent)
>> >> >> >>> >>envelope.getMessage());
>> >> >> >>> >> >> >
>> >> >> >>> >> >> >     log.info(envelope.getMessage().toString());
>> >> >> >>> >> >> >
>> >> >> >>> >> >> >     collector.send(new
>> >> >>OutgoingMessageEnvelope(OUTPUT_STREAM,
>> >> >> >>> >> >> > outgoingMap));
>> >> >> >>> >> >> >
>> >> >> >>> >> >> >   }
>> >> >> >>> >> >> >
>> >> >> >>> >> >> > }
>> >> >> >>> >> >> >
>> >> >> >>> >> >> >
>> >> >> >>> >> >> > On Mon, Aug 25, 2014 at 9:01 AM, Chris Riccomini <
>> >> >> >>> >> >> > criccomini@linkedin.com.invalid> wrote:
>> >> >> >>> >> >> >
>> >> >> >>> >> >> > > Hey Shekar,
>> >> >> >>> >> >> > >
>> >> >> >>> >> >> > > Your thought process is on the right track. It's
>> >>probably
>> >> >> >>>best
>> >> >> >>> to
>> >> >> >>> >> >>start
>> >> >> >>> >> >> > > with hello-samza, and modify it to get what you
>>want.
>> >>To
>> >> >> >>>start
>> >> >> >>> >>with,
>> >> >> >>> >> >> > > you'll want to:
>> >> >> >>> >> >> > >
>> >> >> >>> >> >> > > 1. Write a simple StreamTask that just does
>>something
>> >> >>silly
>> >> >> >>>like
>> >> >> >>> >> >>just
>> >> >> >>> >> >> > > print messages that it receives.
>> >> >> >>> >> >> > > 2. Write a configuration for the job that consumes
>>from
>> >> >>just
>> >> >> >>>the
>> >> >> >>> >> >>stream
>> >> >> >>> >> >> > > (alerts from different sources).
>> >> >> >>> >> >> > > 3. Run this to make sure you've got it working.
>> >> >> >>> >> >> > > 4. Now add your table join. This can be either a
>> >> >>change-data
>> >> >> >>> >>capture
>> >> >> >>> >> >> > (CDC)
>> >> >> >>> >> >> > > stream, or via a remote DB call.
>> >> >> >>> >> >> > >
>> >> >> >>> >> >> > > That should get you to a point where you've got your
>> >>job
>> >> >>up
>> >> >> >>>and
>> >> >> >>> >> >> running.
>> >> >> >>> >> >> > > From there, you could create your own Maven project,
>> >>and
>> >> >> >>>migrate
>> >> >> >>> >> >>your
>> >> >> >>> >> >> > code
>> >> >> >>> >> >> > > over accordingly.
>> >> >> >>> >> >> > >
>> >> >> >>> >> >> > > Cheers,
>> >> >> >>> >> >> > > Chris
>> >> >> >>> >> >> > >
>> >> >> >>> >> >> > > On 8/24/14 1:42 AM, "Shekar Tippur"
>><ctippur@gmail.com
>> >
>> >> >> >>>wrote:
>> >> >> >>> >> >> > >
>> >> >> >>> >> >> > > >Chris,
>> >> >> >>> >> >> > > >
>> >> >> >>> >> >> > > >I have gone thro the documentation and decided that
>> >>the
>> >> >> >>>option
>> >> >> >>> >> >>that is
>> >> >> >>> >> >> > > >most
>> >> >> >>> >> >> > > >suitable for me is stream-table.
>> >> >> >>> >> >> > > >
>> >> >> >>> >> >> > > >I see the following things:
>> >> >> >>> >> >> > > >
>> >> >> >>> >> >> > > >1. Point samza to a table (database)
>> >> >> >>> >> >> > > >2. Point Samza to a stream - Alert stream from
>> >>different
>> >> >> >>> sources
>> >> >> >>> >> >> > > >3. Join key like a hostname
>> >> >> >>> >> >> > > >
>> >> >> >>> >> >> > > >I have Hello Samza working. To extend that to do
>>what
>> >>my
>> >> >> >>>needs
>> >> >> >>> >> >>are, I
>> >> >> >>> >> >> am
>> >> >> >>> >> >> > > >not sure where to start (Needs more code change OR
>> >> >> >>> configuration
>> >> >> >>> >> >> changes
>> >> >> >>> >> >> > > >OR
>> >> >> >>> >> >> > > >both)?
>> >> >> >>> >> >> > > >
>> >> >> >>> >> >> > > >I have gone thro
>> >> >> >>> >> >> > > >
>> >> >> >>> >> >> >
>> >> >> >>> >> >>
>> >> >> >>> >> >>
>> >> >> >>> >>
>> >> >> >>> >>
>> >> >> >>>
>> >> >> >>>
>> >> >>
>> >>
>> http://samza.incubator.apache.org/learn/documentation/latest/api/overvie
>> >> >> >>>w
>> >> >> >>> >> >> > > .
>> >> >> >>> >> >> > > >html
>> >> >> >>> >> >> > > >
>> >> >> >>> >> >> > > >Is my thought process on the right track? Can you
>> >>please
>> >> >> >>>point
>> >> >> >>> >>me
>> >> >> >>> >> >>to
>> >> >> >>> >> >> the
>> >> >> >>> >> >> > > >right direction?
>> >> >> >>> >> >> > > >
>> >> >> >>> >> >> > > >- Shekar
>> >> >> >>> >> >> > > >
>> >> >> >>> >> >> > > >
>> >> >> >>> >> >> > > >
>> >> >> >>> >> >> > > >On Thu, Aug 21, 2014 at 1:08 PM, Shekar Tippur
>> >> >> >>> >><ct...@gmail.com>
>> >> >> >>> >> >> > wrote:
>> >> >> >>> >> >> > > >
>> >> >> >>> >> >> > > >> Chris,
>> >> >> >>> >> >> > > >>
>> >> >> >>> >> >> > > >> This is perfectly good answer. I will start
>>poking
>> >>more
>> >> >> >>>into
>> >> >> >>> >> >>option
>> >> >> >>> >> >> > #4.
>> >> >> >>> >> >> > > >>
>> >> >> >>> >> >> > > >> - Shekar
>> >> >> >>> >> >> > > >>
>> >> >> >>> >> >> > > >>
>> >> >> >>> >> >> > > >> On Thu, Aug 21, 2014 at 1:05 PM, Chris Riccomini
>><
>> >> >> >>> >> >> > > >> criccomini@linkedin.com.invalid> wrote:
>> >> >> >>> >> >> > > >>
>> >> >> >>> >> >> > > >>> Hey Shekar,
>> >> >> >>> >> >> > > >>>
>> >> >> >>> >> >> > > >>> Your two options are really (3) or (4), then.
>>You
>> >>can
>> >> >> >>>either
>> >> >> >>> >>run
>> >> >> >>> >> >> some
>> >> >> >>> >> >> > > >>> external DB that holds the data set, and you can
>> >> >>query it
>> >> >> >>> >>from a
>> >> >> >>> >> >> > > >>> StreamTask, or you can use Samza's state store
>> >> >>feature to
>> >> >> >>> >>push
>> >> >> >>> >> >>data
>> >> >> >>> >> >> > > >>>into a
>> >> >> >>> >> >> > > >>> stream that you can then store in a partitioned
>> >> >>key-value
>> >> >> >>> >>store
>> >> >> >>> >> >> along
>> >> >> >>> >> >> > > >>>with
>> >> >> >>> >> >> > > >>> your StreamTasks. There is some documentation
>>here
>> >> >>about
>> >> >> >>>the
>> >> >> >>> >> >>state
>> >> >> >>> >> >> > > >>>store
>> >> >> >>> >> >> > > >>> approach:
>> >> >> >>> >> >> > > >>>
>> >> >> >>> >> >> > > >>>
>> >> >> >>> >> >> > > >>>
>> >> >> >>> >> >> > > >>>
>> >> >> >>> >> >> > >
>> >> >> >>> >> >>
>> >> >> >>> >>
>> >> >> >>>
>> >> >> >>>
>> >> >>
>> >>
>> http://samza.incubator.apache.org/learn/documentation/0.7.0/container/st
>> >> >> >>> >> >> > > >>>ate
>> >> >> >>> >> >> > > >>> -management.html
>> >> >> >>> >> >> > > >>>
>> >> >> >>> >> >> > > >>><
>> >> >> >>> >> >> > >
>> >> >> >>> >> >>
>> >> >> >>>
>> >> >> >>>>>
>> >> >>
>> >>http://samza.incubator.apache.org/learn/documentation/0.7.0/container/
>> >> >> >>>>>s
>> >> >> >>> >> >> > > >>>tate-management.html>
>> >> >> >>> >> >> > > >>>
>> >> >> >>> >> >> > > >>>
>> >> >> >>> >> >> > > >>> (4) is going to require more up front effort
>>from
>> >>you,
>> >> >> >>>since
>> >> >> >>> >> >>you'll
>> >> >> >>> >> >> > > >>>have
>> >> >> >>> >> >> > > >>> to understand how Kafka's partitioning model
>>works,
>> >> >>and
>> >> >> >>> setup
>> >> >> >>> >> >>some
>> >> >> >>> >> >> > > >>> pipeline to push the updates for your state. In
>>the
>> >> >>long
>> >> >> >>> >>run, I
>> >> >> >>> >> >> > believe
>> >> >> >>> >> >> > > >>> it's the better approach, though. Local lookups
>>on
>> >>a
>> >> >> >>> >>key-value
>> >> >> >>> >> >> store
>> >> >> >>> >> >> > > >>> should be faster than doing remote RPC calls to
>>a
>> >>DB
>> >> >>for
>> >> >> >>> >>every
>> >> >> >>> >> >> > message.
>> >> >> >>> >> >> > > >>>
>> >> >> >>> >> >> > > >>> I'm sorry I can't give you a more definitive
>> >>answer.
>> >> >>It's
>> >> >> >>> >>really
>> >> >> >>> >> >> > about
>> >> >> >>> >> >> > > >>> trade-offs.
>> >> >> >>> >> >> > > >>>
>> >> >> >>> >> >> > > >>> Cheers,
>> >> >> >>> >> >> > > >>> Chris
>> >> >> >>> >> >> > > >>>
>> >> >> >>> >> >> > > >>> On 8/21/14 12:22 PM, "Shekar Tippur"
>> >> >><ct...@gmail.com>
>> >> >> >>> >>wrote:
>> >> >> >>> >> >> > > >>>
>> >> >> >>> >> >> > > >>> >Chris,
>> >> >> >>> >> >> > > >>> >
>> >> >> >>> >> >> > > >>> >A big thanks for a swift response. The data
>>set is
>> >> >>huge
>> >> >> >>>and
>> >> >> >>> >>the
>> >> >> >>> >> >> > > >>>frequency
>> >> >> >>> >> >> > > >>> >is in burst.
>> >> >> >>> >> >> > > >>> >What do you suggest?
>> >> >> >>> >> >> > > >>> >
>> >> >> >>> >> >> > > >>> >- Shekar
>> >> >> >>> >> >> > > >>> >
>> >> >> >>> >> >> > > >>> >
>> >> >> >>> >> >> > > >>> >On Thu, Aug 21, 2014 at 11:05 AM, Chris
>>Riccomini
>> >><
>> >> >> >>> >> >> > > >>> >criccomini@linkedin.com.invalid> wrote:
>> >> >> >>> >> >> > > >>> >
>> >> >> >>> >> >> > > >>> >> Hey Shekar,
>> >> >> >>> >> >> > > >>> >>
>> >> >> >>> >> >> > > >>> >> This is feasible, and you are on the right
>> >>thought
>> >> >> >>> >>process.
>> >> >> >>> >> >> > > >>> >>
>> >> >> >>> >> >> > > >>> >> For the sake of discussion, I'm going to
>>pretend
>> >> >>that
>> >> >> >>>you
>> >> >> >>> >> >>have a
>> >> >> >>> >> >> > > >>>Kafka
>> >> >> >>> >> >> > > >>> >> topic called "PageViewEvent", which has just
>> >>the IP
>> >> >> >>> >>address
>> >> >> >>> >> >>that
>> >> >> >>> >> >> > was
>> >> >> >>> >> >> > > >>> >>used
>> >> >> >>> >> >> > > >>> >> to view a page. These messages will be logged
>> >>every
>> >> >> >>>time
>> >> >> >>> a
>> >> >> >>> >> >>page
>> >> >> >>> >> >> > view
>> >> >> >>> >> >> > > >>> >> happens. I'm also going to pretend that you
>>have
>> >> >>some
>> >> >> >>> >>state
>> >> >> >>> >> >> called
>> >> >> >>> >> >> > > >>> >>"IPGeo"
>> >> >> >>> >> >> > > >>> >> (e.g. The maxmind data set). In this example,
>> >>we'll
>> >> >> >>>want
>> >> >> >>> >>to
>> >> >> >>> >> >>join
>> >> >> >>> >> >> > the
>> >> >> >>> >> >> > > >>> >> long/lat geo information from IPGeo to the
>> >> >> >>>PageViewEvent,
>> >> >> >>> >>and
>> >> >> >>> >> >> send
>> >> >> >>> >> >> > > >>>it
>> >> >> >>> >> >> > > >>> >>to a
>> >> >> >>> >> >> > > >>> >> new topic: PageViewEventsWithGeo.
>> >> >> >>> >> >> > > >>> >>
>> >> >> >>> >> >> > > >>> >> You have several options on how to implement
>> >>this
>> >> >> >>> example.
>> >> >> >>> >> >> > > >>> >>
>> >> >> >>> >> >> > > >>> >> 1. If your joining data set (IPGeo) is
>> >>relatively
>> >> >> >>>small
>> >> >> >>> >>and
>> >> >> >>> >> >> > changes
>> >> >> >>> >> >> > > >>> >> infrequently, you can just pack it up in your
>> >>jar
>> >> >>or
>> >> >> >>>.tgz
>> >> >> >>> >> >>file,
>> >> >> >>> >> >> > and
>> >> >> >>> >> >> > > >>> open
>> >> >> >>> >> >> > > >>> >> it open in every StreamTask.
>> >> >> >>> >> >> > > >>> >> 2. If your data set is small, but changes
>> >>somewhat
>> >> >> >>> >> >>frequently,
>> >> >> >>> >> >> you
>> >> >> >>> >> >> > > >>>can
>> >> >> >>> >> >> > > >>> >> throw the data set on some HTTP/HDFS/S3
>>server
>> >> >> >>>somewhere,
>> >> >> >>> >>and
>> >> >> >>> >> >> have
>> >> >> >>> >> >> > > >>>your
>> >> >> >>> >> >> > > >>> >> StreamTask refresh it periodically by
>> >> >>re-downloading
>> >> >> >>>it.
>> >> >> >>> >> >> > > >>> >> 3. You can do remote RPC calls for the IPGeo
>> >>data
>> >> >>on
>> >> >> >>> every
>> >> >> >>> >> >>page
>> >> >> >>> >> >> > view
>> >> >> >>> >> >> > > >>> >>event
>> >> >> >>> >> >> > > >>> >> by query some remote service or DB (e.g.
>> >> >>Cassandra).
>> >> >> >>> >> >> > > >>> >> 4. You can use Samza's state feature to set
>>your
>> >> >>IPGeo
>> >> >> >>> >>data
>> >> >> >>> >> >>as a
>> >> >> >>> >> >> > > >>>series
>> >> >> >>> >> >> > > >>> >>of
>> >> >> >>> >> >> > > >>> >> messages to a log-compacted Kafka topic
>> >> >> >>> >> >> > > >>> >> (
>> >> >> >>> >> >>
>> >> >>https://cwiki.apache.org/confluence/display/KAFKA/Log+Compaction
>> >> >> >>> >> >> > ),
>> >> >> >>> >> >> > > >>> and
>> >> >> >>> >> >> > > >>> >> configure your Samza job to read this topic
>>as a
>> >> >> >>> bootstrap
>> >> >> >>> >> >> stream
>> >> >> >>> >> >> > > >>> >> (
>> >> >> >>> >> >> > > >>> >>
>> >> >> >>> >> >> > > >>> >>
>> >> >> >>> >> >> > > >>>
>> >> >> >>> >> >> > > >>>
>> >> >> >>> >> >> > >
>> >> >> >>> >> >>
>> >> >> >>> >>
>> >> >> >>>
>> >> >> >>>
>> >> >>
>> >>
>> http://samza.incubator.apache.org/learn/documentation/0.7.0/container/st
>> >> >> >>> >> >> > > >>>r
>> >> >> >>> >> >> > > >>> >>e
>> >> >> >>> >> >> > > >>> >> ams.html).
>> >> >> >>> >> >> > > >>> >>
>> >> >> >>> >> >> > > >>> >> For (4), you'd have to partition the IPGeo
>>state
>> >> >>topic
>> >> >> >>> >> >>according
>> >> >> >>> >> >> > to
>> >> >> >>> >> >> > > >>>the
>> >> >> >>> >> >> > > >>> >> same key as PageViewEvent. If PageViewEvent
>>were
>> >> >> >>> >>partitioned
>> >> >> >>> >> >>by,
>> >> >> >>> >> >> > > >>>say,
>> >> >> >>> >> >> > > >>> >> member ID, but you want your IPGeo state
>>topic
>> >>to
>> >> >>be
>> >> >> >>> >> >>partitioned
>> >> >> >>> >> >> > by
>> >> >> >>> >> >> > > >>>IP
>> >> >> >>> >> >> > > >>> >> address, then you'd have to have an upstream
>>job
>> >> >>that
>> >> >> >>> >> >> > re-partitioned
>> >> >> >>> >> >> > > >>> >> PageViewEvent into some new topic by IP
>>address.
>> >> >>This
>> >> >> >>>new
>> >> >> >>> >> >>topic
>> >> >> >>> >> >> > will
>> >> >> >>> >> >> > > >>> >>have
>> >> >> >>> >> >> > > >>> >> to have the same number of partitions as the
>> >>IPGeo
>> >> >> >>>state
>> >> >> >>> >> >>topic
>> >> >> >>> >> >> (if
>> >> >> >>> >> >> > > >>> IPGeo
>> >> >> >>> >> >> > > >>> >> has 8 partitions, then the new
>> >> >> >>>PageViewEventRepartitioned
>> >> >> >>> >> >>topic
>> >> >> >>> >> >> > > >>>needs 8
>> >> >> >>> >> >> > > >>> >>as
>> >> >> >>> >> >> > > >>> >> well). This will cause your
>> >> >>PageViewEventRepartitioned
>> >> >> >>> >>topic
>> >> >> >>> >> >>and
>> >> >> >>> >> >> > > >>>your
>> >> >> >>> >> >> > > >>> >> IPGeo state topic to be aligned such that the
>> >> >> >>>StreamTask
>> >> >> >>> >>that
>> >> >> >>> >> >> gets
>> >> >> >>> >> >> > > >>>page
>> >> >> >>> >> >> > > >>> >> views for IP address X will also have the
>>IPGeo
>> >> >> >>> >>information
>> >> >> >>> >> >>for
>> >> >> >>> >> >> IP
>> >> >> >>> >> >> > > >>> >>address
>> >> >> >>> >> >> > > >>> >> X.
>> >> >> >>> >> >> > > >>> >>
>> >> >> >>> >> >> > > >>> >> Which strategy you pick is really up to you.
>>:)
>> >> >>(4) is
>> >> >> >>> the
>> >> >> >>> >> >>most
>> >> >> >>> >> >> > > >>> >> complicated, but also the most flexible, and
>> >>most
>> >> >> >>> >> >>operationally
>> >> >> >>> >> >> > > >>>sound.
>> >> >> >>> >> >> > > >>> >>(1)
>> >> >> >>> >> >> > > >>> >> is the easiest if it fits your needs.
>> >> >> >>> >> >> > > >>> >>
>> >> >> >>> >> >> > > >>> >> Cheers,
>> >> >> >>> >> >> > > >>> >> Chris
>> >> >> >>> >> >> > > >>> >>
>> >> >> >>> >> >> > > >>> >> On 8/21/14 10:15 AM, "Shekar Tippur"
>> >> >> >>><ct...@gmail.com>
>> >> >> >>> >> >>wrote:
>> >> >> >>> >> >> > > >>> >>
>> >> >> >>> >> >> > > >>> >> >Hello,
>> >> >> >>> >> >> > > >>> >> >
>> >> >> >>> >> >> > > >>> >> >I am new to Samza. I have just installed
>>Hello
>> >> >>Samza
>> >> >> >>>and
>> >> >> >>> >> >>got it
>> >> >> >>> >> >> > > >>> >>working.
>> >> >> >>> >> >> > > >>> >> >
>> >> >> >>> >> >> > > >>> >> >Here is the use case for which I am trying
>>to
>> >>use
>> >> >> >>>Samza:
>> >> >> >>> >> >> > > >>> >> >
>> >> >> >>> >> >> > > >>> >> >
>> >> >> >>> >> >> > > >>> >> >1. Cache the contextual information which
>> >>contains
>> >> >> >>>more
>> >> >> >>> >> >> > information
>> >> >> >>> >> >> > > >>> >>about
>> >> >> >>> >> >> > > >>> >> >the hostname or IP address using
>> >>Samza/Yarn/Kafka
>> >> >> >>> >> >> > > >>> >> >2. Collect alert and metric events which
>> >>contain
>> >> >> >>>either
>> >> >> >>> >> >> hostname
>> >> >> >>> >> >> > > >>>or IP
>> >> >> >>> >> >> > > >>> >> >address
>> >> >> >>> >> >> > > >>> >> >3. Append contextual information to the
>>alert
>> >>and
>> >> >> >>>metric
>> >> >> >>> >>and
>> >> >> >>> >> >> > > >>>insert to
>> >> >> >>> >> >> > > >>> >>a
>> >> >> >>> >> >> > > >>> >> >Kafka queue from which other subscribers
>>read
>> >>off
>> >> >>of.
>> >> >> >>> >> >> > > >>> >> >
>> >> >> >>> >> >> > > >>> >> >Can you please shed some light on
>> >> >> >>> >> >> > > >>> >> >
>> >> >> >>> >> >> > > >>> >> >1. Is this feasible?
>> >> >> >>> >> >> > > >>> >> >2. Am I on the right thought process
>> >> >> >>> >> >> > > >>> >> >3. How do I start
>> >> >> >>> >> >> > > >>> >> >
>> >> >> >>> >> >> > > >>> >> >I now have 1 & 2 of them working
>>disparately. I
>> >> >>need
>> >> >> >>>to
>> >> >> >>> >> >> integrate
>> >> >> >>> >> >> > > >>> them.
>> >> >> >>> >> >> > > >>> >> >
>> >> >> >>> >> >> > > >>> >> >Appreciate any input.
>> >> >> >>> >> >> > > >>> >> >
>> >> >> >>> >> >> > > >>> >> >- Shekar
>> >> >> >>> >> >> > > >>> >>
>> >> >> >>> >> >> > > >>> >>
>> >> >> >>> >> >> > > >>>
>> >> >> >>> >> >> > > >>>
>> >> >> >>> >> >> > > >>
>> >> >> >>> >> >> > >
>> >> >> >>> >> >> > >
>> >> >> >>> >> >> >
>> >> >> >>> >> >>
>> >> >> >>> >>
>> >> >> >>> >>
>> >> >> >>>
>> >> >> >>>
>> >> >> >>
>> >> >>
>> >> >>
>> >>
>> >>
>>
>>


Re: Samza as a Caching layer

Posted by Shekar Tippur <ct...@gmail.com>.
Chris,

I have added the below dependencies and I still get the same message.


      <dependency>

        <groupId>org.slf4j</groupId>

        <artifactId>slf4j-api</artifactId>

        <version>1.7.7</version>

      </dependency>

      <dependency>

          <groupId>org.slf4j</groupId>

          <artifactId>slf4j-log4j12</artifactId>

          <version>1.5.6</version>

      </dependency>


On Wed, Sep 3, 2014 at 9:01 AM, Chris Riccomini <
criccomini@linkedin.com.invalid> wrote:

> Hey Shekar,
>
> The SLF4J stuff is saying that you don't have an slf4j binding on your
> classpath. Try adding slf4j-log4j as a runtime dependency on your project.
>
> Cheers,
> Chris
>
> On 9/2/14 3:24 PM, "Shekar Tippur" <ct...@gmail.com> wrote:
>
> >Chris ,
> >
> >In the current state, I just want Samza to connect to 127.0.0.1.
> >
> >I have set YARN_HOME to
> >
> >$ echo $YARN_HOME
> >
> >/home/ctippur/hello-samza/deploy/yarn
> >
> >I still dont see anything on hadoop console.
> >
> > Also, I see this during startup
> >
> >
> >SLF4J: Failed to load class "org.slf4j.impl.StaticLoggerBinder".
> >
> >SLF4J: Defaulting to no-operation (NOP) logger implementation
> >
> >SLF4J: See http://www.slf4j.org/codes.html#StaticLoggerBinder for further
> >details.
> >
> >SLF4J: Failed to load class "org.slf4j.impl.StaticLoggerBinder".
> >
> >SLF4J: Defaulting to no-operation (NOP) logger implementation
> >
> >SLF4J: See http://www.slf4j.org/codes.html#StaticLoggerBinder for further
> >details.
> >
> >- Shekar
> >
> >
> >On Tue, Sep 2, 2014 at 2:20 PM, Chris Riccomini <
> >criccomini@linkedin.com.invalid> wrote:
> >
> >> Hey Shekar,
> >>
> >> Ah ha. In that case, do you expect your SamzaContainer to try to connect
> >> to the RM at 127.0.0.1, or do you expect it to try to connect to some
> >> remote RM? If you expect it to try and connect to a remote RM, it's not
> >> doing that. Perhaps because YARN_HOME isn't set.
> >>
> >> If you go to your RM's web interface, how many active nodes do you see
> >> listed?
> >>
> >> Cheers,
> >> Chris
> >>
> >> On 9/2/14 2:17 PM, "Shekar Tippur" <ct...@gmail.com> wrote:
> >>
> >> >Chris ..
> >> >
> >> >I am using a rhel server to host all the components (Yarn, kafka,
> >>samza).
> >> >I
> >> >dont have ACLs open to wikipedia.
> >> >I am following ..
> >> >
> >>
> >>
> http://samza.incubator.apache.org/learn/tutorials/latest/run-hello-samza-
> >>w
> >> >ithout-internet.html
> >> >
> >> >- Shekar
> >> >
> >> >
> >> >
> >> >On Tue, Sep 2, 2014 at 2:13 PM, Chris Riccomini <
> >> >criccomini@linkedin.com.invalid> wrote:
> >> >
> >> >> Hey Shekar,
> >> >>
> >> >> Can you try changing that to:
> >> >>
> >> >>   http://127.0.0.1:8088/cluster
> >> >>
> >> >>
> >> >> And see if you can connect?
> >> >>
> >> >> Cheers,
> >> >> Chris
> >> >>
> >> >> On 9/2/14 1:21 PM, "Shekar Tippur" <ct...@gmail.com> wrote:
> >> >>
> >> >> >Other observation is ..
> >> >> >
> >> >> >http://10.132.62.185:8088/cluster shows that no applications are
> >> >>running.
> >> >> >
> >> >> >- Shekar
> >> >> >
> >> >> >
> >> >> >
> >> >> >
> >> >> >On Tue, Sep 2, 2014 at 1:15 PM, Shekar Tippur <ct...@gmail.com>
> >> >>wrote:
> >> >> >
> >> >> >> Yarn seem to be running ..
> >> >> >>
> >> >> >> yarn      5462  0.0  2.0 1641296 161404 ?      Sl   Jun20  95:26
> >> >> >> /usr/java/jdk1.6.0_31/bin/java -Dproc_resourcemanager -Xmx1000m
> >> >> >> -Dhadoop.log.dir=/var/log/hadoop-yarn
> >> >> >>-Dyarn.log.dir=/var/log/hadoop-yarn
> >> >> >> -Dhadoop.log.file=yarn-yarn-resourcemanager-pppdc9prd2y2.log
> >> >> >> -Dyarn.log.file=yarn-yarn-resourcemanager-pppdc9prd2y2.log
> >> >> >> -Dyarn.home.dir=/usr/lib/hadoop-yarn
> >> >> >>-Dhadoop.home.dir=/usr/lib/hadoop-yarn
> >> >> >> -Dhadoop.root.logger=INFO,RFA -Dyarn.root.logger=INFO,RFA
> >> >> >> -Djava.library.path=/usr/lib/hadoop/lib/native -classpath
> >> >> >>
> >> >>
> >>
> >>>>>>/etc/hadoop/conf:/etc/hadoop/conf:/etc/hadoop/conf:/usr/lib/hadoop/li
> >>>>>>b/
> >> >>>>*:
> >> >>
> >>
> >>>>>>/usr/lib/hadoop/.//*:/usr/lib/hadoop-hdfs/./:/usr/lib/hadoop-hdfs/lib
> >>>>>>/*
> >> >>>>:/
> >> >>
> >>
> >>>>>>usr/lib/hadoop-hdfs/.//*:/usr/lib/hadoop-yarn/lib/*:/usr/lib/hadoop-y
> >>>>>>ar
> >> >>>>n/
> >> >>
> >>
> >>>>>>.//*:/usr/lib/hadoop-mapreduce/lib/*:/usr/lib/hadoop-mapreduce/.//*:/
> >>>>>>us
> >> >>>>r/
> >> >>
> >>
> >>>>>>lib/hadoop-yarn/.//*:/usr/lib/hadoop-yarn/lib/*:/etc/hadoop/conf/rm-c
> >>>>>>on
> >> >>>>fi
> >> >> >>g/log4j.properties
> >> >> >> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager
> >> >> >>
> >> >> >> I can tail kafka topic as well ..
> >> >> >>
> >> >> >> deploy/kafka/bin/kafka-console-consumer.sh  --zookeeper
> >> >>localhost:2181
> >> >> >>--topic wikipedia-raw
> >> >> >>
> >> >> >>
> >> >> >>
> >> >> >>
> >> >> >> On Tue, Sep 2, 2014 at 1:04 PM, Chris Riccomini <
> >> >> >> criccomini@linkedin.com.invalid> wrote:
> >> >> >>
> >> >> >>> Hey Shekar,
> >> >> >>>
> >> >> >>> It looks like your job is hanging trying to connect to the RM on
> >> >>your
> >> >> >>> localhost. I thought that you said that your job was running in
> >> >>local
> >> >> >>> mode. If so, it should be using the LocalJobFactory. If not, and
> >>you
> >> >> >>> intend to run on YARN, is your YARN RM up and running on
> >>localhost?
> >> >> >>>
> >> >> >>> Cheers,
> >> >> >>> Chris
> >> >> >>>
> >> >> >>> On 9/2/14 12:22 PM, "Shekar Tippur" <ct...@gmail.com> wrote:
> >> >> >>>
> >> >> >>> >Chris ..
> >> >> >>> >
> >> >> >>> >$ cat ./deploy/samza/undefined-samza-container-name.log
> >> >> >>> >
> >> >> >>> >2014-09-02 11:17:58 JobRunner [INFO] job factory:
> >> >> >>> >org.apache.samza.job.yarn.YarnJobFactory
> >> >> >>> >
> >> >> >>> >2014-09-02 11:17:59 ClientHelper [INFO] trying to connect to RM
> >> >> >>> >127.0.0.1:8032
> >> >> >>> >
> >> >> >>> >2014-09-02 11:17:59 NativeCodeLoader [WARN] Unable to load
> >> >> >>>native-hadoop
> >> >> >>> >library for your platform... using builtin-java classes where
> >> >> >>>applicable
> >> >> >>> >
> >> >> >>> >2014-09-02 11:17:59 RMProxy [INFO] Connecting to ResourceManager
> >> >>at /
> >> >> >>> >127.0.0.1:8032
> >> >> >>> >
> >> >> >>> >
> >> >> >>> >and Log4j ..
> >> >> >>> >
> >> >> >>> ><!DOCTYPE log4j:configuration SYSTEM "log4j.dtd">
> >> >> >>> >
> >> >> >>> ><log4j:configuration
> >> >>xmlns:log4j="http://jakarta.apache.org/log4j/">
> >> >> >>> >
> >> >> >>> >  <appender name="RollingAppender"
> >> >> >>> >class="org.apache.log4j.DailyRollingFileAppender">
> >> >> >>> >
> >> >> >>> >     <param name="File"
> >> >> >>> >value="${samza.log.dir}/${samza.container.name}.log"
> >> >> >>> >/>
> >> >> >>> >
> >> >> >>> >     <param name="DatePattern" value="'.'yyyy-MM-dd" />
> >> >> >>> >
> >> >> >>> >     <layout class="org.apache.log4j.PatternLayout">
> >> >> >>> >
> >> >> >>> >      <param name="ConversionPattern" value="%d{yyyy-MM-dd
> >> >>HH:mm:ss}
> >> >> >>> %c{1}
> >> >> >>> >[%p] %m%n" />
> >> >> >>> >
> >> >> >>> >     </layout>
> >> >> >>> >
> >> >> >>> >  </appender>
> >> >> >>> >
> >> >> >>> >  <root>
> >> >> >>> >
> >> >> >>> >    <priority value="info" />
> >> >> >>> >
> >> >> >>> >    <appender-ref ref="RollingAppender"/>
> >> >> >>> >
> >> >> >>> >  </root>
> >> >> >>> >
> >> >> >>> ></log4j:configuration>
> >> >> >>> >
> >> >> >>> >
> >> >> >>> >On Tue, Sep 2, 2014 at 12:02 PM, Chris Riccomini <
> >> >> >>> >criccomini@linkedin.com.invalid> wrote:
> >> >> >>> >
> >> >> >>> >> Hey Shekar,
> >> >> >>> >>
> >> >> >>> >> Can you attach your log files? I'm wondering if it's a
> >> >> >>>mis-configured
> >> >> >>> >> log4j.xml (or missing slf4j-log4j jar), which is leading to
> >> >>nearly
> >> >> >>> empty
> >> >> >>> >> log files. Also, I'm wondering if the job starts fully.
> >>Anything
> >> >>you
> >> >> >>> can
> >> >> >>> >> attach would be helpful.
> >> >> >>> >>
> >> >> >>> >> Cheers,
> >> >> >>> >> Chris
> >> >> >>> >>
> >> >> >>> >> On 9/2/14 11:43 AM, "Shekar Tippur" <ct...@gmail.com>
> wrote:
> >> >> >>> >>
> >> >> >>> >> >I am running in local mode.
> >> >> >>> >> >
> >> >> >>> >> >S
> >> >> >>> >> >On Sep 2, 2014 11:42 AM, "Yan Fang" <ya...@gmail.com>
> >> >>wrote:
> >> >> >>> >> >
> >> >> >>> >> >> Hi Shekar.
> >> >> >>> >> >>
> >> >> >>> >> >> Are you running job in local mode or yarn mode? If yarn
> >>mode,
> >> >>the
> >> >> >>> log
> >> >> >>> >> >>is in
> >> >> >>> >> >> the yarn's container log.
> >> >> >>> >> >>
> >> >> >>> >> >> Thanks,
> >> >> >>> >> >>
> >> >> >>> >> >> Fang, Yan
> >> >> >>> >> >> yanfang724@gmail.com
> >> >> >>> >> >> +1 (206) 849-4108
> >> >> >>> >> >>
> >> >> >>> >> >>
> >> >> >>> >> >> On Tue, Sep 2, 2014 at 11:36 AM, Shekar Tippur
> >> >> >>><ct...@gmail.com>
> >> >> >>> >> >>wrote:
> >> >> >>> >> >>
> >> >> >>> >> >> > Chris,
> >> >> >>> >> >> >
> >> >> >>> >> >> > Got some time to play around a bit more.
> >> >> >>> >> >> > I tried to edit
> >> >> >>> >> >> >
> >> >> >>> >> >> >
> >> >> >>> >> >>
> >> >> >>> >>
> >> >> >>>
> >> >> >>>
> >> >>
> >>
> >>>>>>>>>>>samza-wikipedia/src/main/java/samza/examples/wikipedia/task/Wiki
> >>>>>>>>>>>pe
> >> >>>>>>>>>di
> >> >> >>>>>>>aFe
> >> >> >>> >>>>ed
> >> >> >>> >> >>StreamTask.java
> >> >> >>> >> >> > to add a logger info statement to tap the incoming
> >>message.
> >> >>I
> >> >> >>>dont
> >> >> >>> >>see
> >> >> >>> >> >> the
> >> >> >>> >> >> > messages being printed to the log file.
> >> >> >>> >> >> >
> >> >> >>> >> >> > Is this the right place to start?
> >> >> >>> >> >> >
> >> >> >>> >> >> > public class WikipediaFeedStreamTask implements
> >>StreamTask {
> >> >> >>> >> >> >
> >> >> >>> >> >> >   private static final SystemStream OUTPUT_STREAM = new
> >> >> >>> >> >> > SystemStream("kafka",
> >> >> >>> >> >> > "wikipedia-raw");
> >> >> >>> >> >> >
> >> >> >>> >> >> >   private static final Logger log =
> >>LoggerFactory.getLogger
> >> >> >>> >> >> > (WikipediaFeedStreamTask.class);
> >> >> >>> >> >> >
> >> >> >>> >> >> >   @Override
> >> >> >>> >> >> >
> >> >> >>> >> >> >   public void process(IncomingMessageEnvelope envelope,
> >> >> >>> >> >>MessageCollector
> >> >> >>> >> >> > collector, TaskCoordinator coordinator) {
> >> >> >>> >> >> >
> >> >> >>> >> >> >     Map<String, Object> outgoingMap =
> >> >> >>> >> >> > WikipediaFeedEvent.toMap((WikipediaFeedEvent)
> >> >> >>> >>envelope.getMessage());
> >> >> >>> >> >> >
> >> >> >>> >> >> >     log.info(envelope.getMessage().toString());
> >> >> >>> >> >> >
> >> >> >>> >> >> >     collector.send(new
> >> >>OutgoingMessageEnvelope(OUTPUT_STREAM,
> >> >> >>> >> >> > outgoingMap));
> >> >> >>> >> >> >
> >> >> >>> >> >> >   }
> >> >> >>> >> >> >
> >> >> >>> >> >> > }
> >> >> >>> >> >> >
> >> >> >>> >> >> >
> >> >> >>> >> >> > On Mon, Aug 25, 2014 at 9:01 AM, Chris Riccomini <
> >> >> >>> >> >> > criccomini@linkedin.com.invalid> wrote:
> >> >> >>> >> >> >
> >> >> >>> >> >> > > Hey Shekar,
> >> >> >>> >> >> > >
> >> >> >>> >> >> > > Your thought process is on the right track. It's
> >>probably
> >> >> >>>best
> >> >> >>> to
> >> >> >>> >> >>start
> >> >> >>> >> >> > > with hello-samza, and modify it to get what you want.
> >>To
> >> >> >>>start
> >> >> >>> >>with,
> >> >> >>> >> >> > > you'll want to:
> >> >> >>> >> >> > >
> >> >> >>> >> >> > > 1. Write a simple StreamTask that just does something
> >> >>silly
> >> >> >>>like
> >> >> >>> >> >>just
> >> >> >>> >> >> > > print messages that it receives.
> >> >> >>> >> >> > > 2. Write a configuration for the job that consumes from
> >> >>just
> >> >> >>>the
> >> >> >>> >> >>stream
> >> >> >>> >> >> > > (alerts from different sources).
> >> >> >>> >> >> > > 3. Run this to make sure you've got it working.
> >> >> >>> >> >> > > 4. Now add your table join. This can be either a
> >> >>change-data
> >> >> >>> >>capture
> >> >> >>> >> >> > (CDC)
> >> >> >>> >> >> > > stream, or via a remote DB call.
> >> >> >>> >> >> > >
> >> >> >>> >> >> > > That should get you to a point where you've got your
> >>job
> >> >>up
> >> >> >>>and
> >> >> >>> >> >> running.
> >> >> >>> >> >> > > From there, you could create your own Maven project,
> >>and
> >> >> >>>migrate
> >> >> >>> >> >>your
> >> >> >>> >> >> > code
> >> >> >>> >> >> > > over accordingly.
> >> >> >>> >> >> > >
> >> >> >>> >> >> > > Cheers,
> >> >> >>> >> >> > > Chris
> >> >> >>> >> >> > >
> >> >> >>> >> >> > > On 8/24/14 1:42 AM, "Shekar Tippur" <ctippur@gmail.com
> >
> >> >> >>>wrote:
> >> >> >>> >> >> > >
> >> >> >>> >> >> > > >Chris,
> >> >> >>> >> >> > > >
> >> >> >>> >> >> > > >I have gone thro the documentation and decided that
> >>the
> >> >> >>>option
> >> >> >>> >> >>that is
> >> >> >>> >> >> > > >most
> >> >> >>> >> >> > > >suitable for me is stream-table.
> >> >> >>> >> >> > > >
> >> >> >>> >> >> > > >I see the following things:
> >> >> >>> >> >> > > >
> >> >> >>> >> >> > > >1. Point samza to a table (database)
> >> >> >>> >> >> > > >2. Point Samza to a stream - Alert stream from
> >>different
> >> >> >>> sources
> >> >> >>> >> >> > > >3. Join key like a hostname
> >> >> >>> >> >> > > >
> >> >> >>> >> >> > > >I have Hello Samza working. To extend that to do what
> >>my
> >> >> >>>needs
> >> >> >>> >> >>are, I
> >> >> >>> >> >> am
> >> >> >>> >> >> > > >not sure where to start (Needs more code change OR
> >> >> >>> configuration
> >> >> >>> >> >> changes
> >> >> >>> >> >> > > >OR
> >> >> >>> >> >> > > >both)?
> >> >> >>> >> >> > > >
> >> >> >>> >> >> > > >I have gone thro
> >> >> >>> >> >> > > >
> >> >> >>> >> >> >
> >> >> >>> >> >>
> >> >> >>> >> >>
> >> >> >>> >>
> >> >> >>> >>
> >> >> >>>
> >> >> >>>
> >> >>
> >>
> http://samza.incubator.apache.org/learn/documentation/latest/api/overvie
> >> >> >>>w
> >> >> >>> >> >> > > .
> >> >> >>> >> >> > > >html
> >> >> >>> >> >> > > >
> >> >> >>> >> >> > > >Is my thought process on the right track? Can you
> >>please
> >> >> >>>point
> >> >> >>> >>me
> >> >> >>> >> >>to
> >> >> >>> >> >> the
> >> >> >>> >> >> > > >right direction?
> >> >> >>> >> >> > > >
> >> >> >>> >> >> > > >- Shekar
> >> >> >>> >> >> > > >
> >> >> >>> >> >> > > >
> >> >> >>> >> >> > > >
> >> >> >>> >> >> > > >On Thu, Aug 21, 2014 at 1:08 PM, Shekar Tippur
> >> >> >>> >><ct...@gmail.com>
> >> >> >>> >> >> > wrote:
> >> >> >>> >> >> > > >
> >> >> >>> >> >> > > >> Chris,
> >> >> >>> >> >> > > >>
> >> >> >>> >> >> > > >> This is perfectly good answer. I will start poking
> >>more
> >> >> >>>into
> >> >> >>> >> >>option
> >> >> >>> >> >> > #4.
> >> >> >>> >> >> > > >>
> >> >> >>> >> >> > > >> - Shekar
> >> >> >>> >> >> > > >>
> >> >> >>> >> >> > > >>
> >> >> >>> >> >> > > >> On Thu, Aug 21, 2014 at 1:05 PM, Chris Riccomini <
> >> >> >>> >> >> > > >> criccomini@linkedin.com.invalid> wrote:
> >> >> >>> >> >> > > >>
> >> >> >>> >> >> > > >>> Hey Shekar,
> >> >> >>> >> >> > > >>>
> >> >> >>> >> >> > > >>> Your two options are really (3) or (4), then. You
> >>can
> >> >> >>>either
> >> >> >>> >>run
> >> >> >>> >> >> some
> >> >> >>> >> >> > > >>> external DB that holds the data set, and you can
> >> >>query it
> >> >> >>> >>from a
> >> >> >>> >> >> > > >>> StreamTask, or you can use Samza's state store
> >> >>feature to
> >> >> >>> >>push
> >> >> >>> >> >>data
> >> >> >>> >> >> > > >>>into a
> >> >> >>> >> >> > > >>> stream that you can then store in a partitioned
> >> >>key-value
> >> >> >>> >>store
> >> >> >>> >> >> along
> >> >> >>> >> >> > > >>>with
> >> >> >>> >> >> > > >>> your StreamTasks. There is some documentation here
> >> >>about
> >> >> >>>the
> >> >> >>> >> >>state
> >> >> >>> >> >> > > >>>store
> >> >> >>> >> >> > > >>> approach:
> >> >> >>> >> >> > > >>>
> >> >> >>> >> >> > > >>>
> >> >> >>> >> >> > > >>>
> >> >> >>> >> >> > > >>>
> >> >> >>> >> >> > >
> >> >> >>> >> >>
> >> >> >>> >>
> >> >> >>>
> >> >> >>>
> >> >>
> >>
> http://samza.incubator.apache.org/learn/documentation/0.7.0/container/st
> >> >> >>> >> >> > > >>>ate
> >> >> >>> >> >> > > >>> -management.html
> >> >> >>> >> >> > > >>>
> >> >> >>> >> >> > > >>><
> >> >> >>> >> >> > >
> >> >> >>> >> >>
> >> >> >>>
> >> >> >>>>>
> >> >>
> >>http://samza.incubator.apache.org/learn/documentation/0.7.0/container/
> >> >> >>>>>s
> >> >> >>> >> >> > > >>>tate-management.html>
> >> >> >>> >> >> > > >>>
> >> >> >>> >> >> > > >>>
> >> >> >>> >> >> > > >>> (4) is going to require more up front effort from
> >>you,
> >> >> >>>since
> >> >> >>> >> >>you'll
> >> >> >>> >> >> > > >>>have
> >> >> >>> >> >> > > >>> to understand how Kafka's partitioning model works,
> >> >>and
> >> >> >>> setup
> >> >> >>> >> >>some
> >> >> >>> >> >> > > >>> pipeline to push the updates for your state. In the
> >> >>long
> >> >> >>> >>run, I
> >> >> >>> >> >> > believe
> >> >> >>> >> >> > > >>> it's the better approach, though. Local lookups on
> >>a
> >> >> >>> >>key-value
> >> >> >>> >> >> store
> >> >> >>> >> >> > > >>> should be faster than doing remote RPC calls to a
> >>DB
> >> >>for
> >> >> >>> >>every
> >> >> >>> >> >> > message.
> >> >> >>> >> >> > > >>>
> >> >> >>> >> >> > > >>> I'm sorry I can't give you a more definitive
> >>answer.
> >> >>It's
> >> >> >>> >>really
> >> >> >>> >> >> > about
> >> >> >>> >> >> > > >>> trade-offs.
> >> >> >>> >> >> > > >>>
> >> >> >>> >> >> > > >>> Cheers,
> >> >> >>> >> >> > > >>> Chris
> >> >> >>> >> >> > > >>>
> >> >> >>> >> >> > > >>> On 8/21/14 12:22 PM, "Shekar Tippur"
> >> >><ct...@gmail.com>
> >> >> >>> >>wrote:
> >> >> >>> >> >> > > >>>
> >> >> >>> >> >> > > >>> >Chris,
> >> >> >>> >> >> > > >>> >
> >> >> >>> >> >> > > >>> >A big thanks for a swift response. The data set is
> >> >>huge
> >> >> >>>and
> >> >> >>> >>the
> >> >> >>> >> >> > > >>>frequency
> >> >> >>> >> >> > > >>> >is in burst.
> >> >> >>> >> >> > > >>> >What do you suggest?
> >> >> >>> >> >> > > >>> >
> >> >> >>> >> >> > > >>> >- Shekar
> >> >> >>> >> >> > > >>> >
> >> >> >>> >> >> > > >>> >
> >> >> >>> >> >> > > >>> >On Thu, Aug 21, 2014 at 11:05 AM, Chris Riccomini
> >><
> >> >> >>> >> >> > > >>> >criccomini@linkedin.com.invalid> wrote:
> >> >> >>> >> >> > > >>> >
> >> >> >>> >> >> > > >>> >> Hey Shekar,
> >> >> >>> >> >> > > >>> >>
> >> >> >>> >> >> > > >>> >> This is feasible, and you are on the right
> >>thought
> >> >> >>> >>process.
> >> >> >>> >> >> > > >>> >>
> >> >> >>> >> >> > > >>> >> For the sake of discussion, I'm going to pretend
> >> >>that
> >> >> >>>you
> >> >> >>> >> >>have a
> >> >> >>> >> >> > > >>>Kafka
> >> >> >>> >> >> > > >>> >> topic called "PageViewEvent", which has just
> >>the IP
> >> >> >>> >>address
> >> >> >>> >> >>that
> >> >> >>> >> >> > was
> >> >> >>> >> >> > > >>> >>used
> >> >> >>> >> >> > > >>> >> to view a page. These messages will be logged
> >>every
> >> >> >>>time
> >> >> >>> a
> >> >> >>> >> >>page
> >> >> >>> >> >> > view
> >> >> >>> >> >> > > >>> >> happens. I'm also going to pretend that you have
> >> >>some
> >> >> >>> >>state
> >> >> >>> >> >> called
> >> >> >>> >> >> > > >>> >>"IPGeo"
> >> >> >>> >> >> > > >>> >> (e.g. The maxmind data set). In this example,
> >>we'll
> >> >> >>>want
> >> >> >>> >>to
> >> >> >>> >> >>join
> >> >> >>> >> >> > the
> >> >> >>> >> >> > > >>> >> long/lat geo information from IPGeo to the
> >> >> >>>PageViewEvent,
> >> >> >>> >>and
> >> >> >>> >> >> send
> >> >> >>> >> >> > > >>>it
> >> >> >>> >> >> > > >>> >>to a
> >> >> >>> >> >> > > >>> >> new topic: PageViewEventsWithGeo.
> >> >> >>> >> >> > > >>> >>
> >> >> >>> >> >> > > >>> >> You have several options on how to implement
> >>this
> >> >> >>> example.
> >> >> >>> >> >> > > >>> >>
> >> >> >>> >> >> > > >>> >> 1. If your joining data set (IPGeo) is
> >>relatively
> >> >> >>>small
> >> >> >>> >>and
> >> >> >>> >> >> > changes
> >> >> >>> >> >> > > >>> >> infrequently, you can just pack it up in your
> >>jar
> >> >>or
> >> >> >>>.tgz
> >> >> >>> >> >>file,
> >> >> >>> >> >> > and
> >> >> >>> >> >> > > >>> open
> >> >> >>> >> >> > > >>> >> it open in every StreamTask.
> >> >> >>> >> >> > > >>> >> 2. If your data set is small, but changes
> >>somewhat
> >> >> >>> >> >>frequently,
> >> >> >>> >> >> you
> >> >> >>> >> >> > > >>>can
> >> >> >>> >> >> > > >>> >> throw the data set on some HTTP/HDFS/S3 server
> >> >> >>>somewhere,
> >> >> >>> >>and
> >> >> >>> >> >> have
> >> >> >>> >> >> > > >>>your
> >> >> >>> >> >> > > >>> >> StreamTask refresh it periodically by
> >> >>re-downloading
> >> >> >>>it.
> >> >> >>> >> >> > > >>> >> 3. You can do remote RPC calls for the IPGeo
> >>data
> >> >>on
> >> >> >>> every
> >> >> >>> >> >>page
> >> >> >>> >> >> > view
> >> >> >>> >> >> > > >>> >>event
> >> >> >>> >> >> > > >>> >> by query some remote service or DB (e.g.
> >> >>Cassandra).
> >> >> >>> >> >> > > >>> >> 4. You can use Samza's state feature to set your
> >> >>IPGeo
> >> >> >>> >>data
> >> >> >>> >> >>as a
> >> >> >>> >> >> > > >>>series
> >> >> >>> >> >> > > >>> >>of
> >> >> >>> >> >> > > >>> >> messages to a log-compacted Kafka topic
> >> >> >>> >> >> > > >>> >> (
> >> >> >>> >> >>
> >> >>https://cwiki.apache.org/confluence/display/KAFKA/Log+Compaction
> >> >> >>> >> >> > ),
> >> >> >>> >> >> > > >>> and
> >> >> >>> >> >> > > >>> >> configure your Samza job to read this topic as a
> >> >> >>> bootstrap
> >> >> >>> >> >> stream
> >> >> >>> >> >> > > >>> >> (
> >> >> >>> >> >> > > >>> >>
> >> >> >>> >> >> > > >>> >>
> >> >> >>> >> >> > > >>>
> >> >> >>> >> >> > > >>>
> >> >> >>> >> >> > >
> >> >> >>> >> >>
> >> >> >>> >>
> >> >> >>>
> >> >> >>>
> >> >>
> >>
> http://samza.incubator.apache.org/learn/documentation/0.7.0/container/st
> >> >> >>> >> >> > > >>>r
> >> >> >>> >> >> > > >>> >>e
> >> >> >>> >> >> > > >>> >> ams.html).
> >> >> >>> >> >> > > >>> >>
> >> >> >>> >> >> > > >>> >> For (4), you'd have to partition the IPGeo state
> >> >>topic
> >> >> >>> >> >>according
> >> >> >>> >> >> > to
> >> >> >>> >> >> > > >>>the
> >> >> >>> >> >> > > >>> >> same key as PageViewEvent. If PageViewEvent were
> >> >> >>> >>partitioned
> >> >> >>> >> >>by,
> >> >> >>> >> >> > > >>>say,
> >> >> >>> >> >> > > >>> >> member ID, but you want your IPGeo state topic
> >>to
> >> >>be
> >> >> >>> >> >>partitioned
> >> >> >>> >> >> > by
> >> >> >>> >> >> > > >>>IP
> >> >> >>> >> >> > > >>> >> address, then you'd have to have an upstream job
> >> >>that
> >> >> >>> >> >> > re-partitioned
> >> >> >>> >> >> > > >>> >> PageViewEvent into some new topic by IP address.
> >> >>This
> >> >> >>>new
> >> >> >>> >> >>topic
> >> >> >>> >> >> > will
> >> >> >>> >> >> > > >>> >>have
> >> >> >>> >> >> > > >>> >> to have the same number of partitions as the
> >>IPGeo
> >> >> >>>state
> >> >> >>> >> >>topic
> >> >> >>> >> >> (if
> >> >> >>> >> >> > > >>> IPGeo
> >> >> >>> >> >> > > >>> >> has 8 partitions, then the new
> >> >> >>>PageViewEventRepartitioned
> >> >> >>> >> >>topic
> >> >> >>> >> >> > > >>>needs 8
> >> >> >>> >> >> > > >>> >>as
> >> >> >>> >> >> > > >>> >> well). This will cause your
> >> >>PageViewEventRepartitioned
> >> >> >>> >>topic
> >> >> >>> >> >>and
> >> >> >>> >> >> > > >>>your
> >> >> >>> >> >> > > >>> >> IPGeo state topic to be aligned such that the
> >> >> >>>StreamTask
> >> >> >>> >>that
> >> >> >>> >> >> gets
> >> >> >>> >> >> > > >>>page
> >> >> >>> >> >> > > >>> >> views for IP address X will also have the IPGeo
> >> >> >>> >>information
> >> >> >>> >> >>for
> >> >> >>> >> >> IP
> >> >> >>> >> >> > > >>> >>address
> >> >> >>> >> >> > > >>> >> X.
> >> >> >>> >> >> > > >>> >>
> >> >> >>> >> >> > > >>> >> Which strategy you pick is really up to you. :)
> >> >>(4) is
> >> >> >>> the
> >> >> >>> >> >>most
> >> >> >>> >> >> > > >>> >> complicated, but also the most flexible, and
> >>most
> >> >> >>> >> >>operationally
> >> >> >>> >> >> > > >>>sound.
> >> >> >>> >> >> > > >>> >>(1)
> >> >> >>> >> >> > > >>> >> is the easiest if it fits your needs.
> >> >> >>> >> >> > > >>> >>
> >> >> >>> >> >> > > >>> >> Cheers,
> >> >> >>> >> >> > > >>> >> Chris
> >> >> >>> >> >> > > >>> >>
> >> >> >>> >> >> > > >>> >> On 8/21/14 10:15 AM, "Shekar Tippur"
> >> >> >>><ct...@gmail.com>
> >> >> >>> >> >>wrote:
> >> >> >>> >> >> > > >>> >>
> >> >> >>> >> >> > > >>> >> >Hello,
> >> >> >>> >> >> > > >>> >> >
> >> >> >>> >> >> > > >>> >> >I am new to Samza. I have just installed Hello
> >> >>Samza
> >> >> >>>and
> >> >> >>> >> >>got it
> >> >> >>> >> >> > > >>> >>working.
> >> >> >>> >> >> > > >>> >> >
> >> >> >>> >> >> > > >>> >> >Here is the use case for which I am trying to
> >>use
> >> >> >>>Samza:
> >> >> >>> >> >> > > >>> >> >
> >> >> >>> >> >> > > >>> >> >
> >> >> >>> >> >> > > >>> >> >1. Cache the contextual information which
> >>contains
> >> >> >>>more
> >> >> >>> >> >> > information
> >> >> >>> >> >> > > >>> >>about
> >> >> >>> >> >> > > >>> >> >the hostname or IP address using
> >>Samza/Yarn/Kafka
> >> >> >>> >> >> > > >>> >> >2. Collect alert and metric events which
> >>contain
> >> >> >>>either
> >> >> >>> >> >> hostname
> >> >> >>> >> >> > > >>>or IP
> >> >> >>> >> >> > > >>> >> >address
> >> >> >>> >> >> > > >>> >> >3. Append contextual information to the alert
> >>and
> >> >> >>>metric
> >> >> >>> >>and
> >> >> >>> >> >> > > >>>insert to
> >> >> >>> >> >> > > >>> >>a
> >> >> >>> >> >> > > >>> >> >Kafka queue from which other subscribers read
> >>off
> >> >>of.
> >> >> >>> >> >> > > >>> >> >
> >> >> >>> >> >> > > >>> >> >Can you please shed some light on
> >> >> >>> >> >> > > >>> >> >
> >> >> >>> >> >> > > >>> >> >1. Is this feasible?
> >> >> >>> >> >> > > >>> >> >2. Am I on the right thought process
> >> >> >>> >> >> > > >>> >> >3. How do I start
> >> >> >>> >> >> > > >>> >> >
> >> >> >>> >> >> > > >>> >> >I now have 1 & 2 of them working disparately. I
> >> >>need
> >> >> >>>to
> >> >> >>> >> >> integrate
> >> >> >>> >> >> > > >>> them.
> >> >> >>> >> >> > > >>> >> >
> >> >> >>> >> >> > > >>> >> >Appreciate any input.
> >> >> >>> >> >> > > >>> >> >
> >> >> >>> >> >> > > >>> >> >- Shekar
> >> >> >>> >> >> > > >>> >>
> >> >> >>> >> >> > > >>> >>
> >> >> >>> >> >> > > >>>
> >> >> >>> >> >> > > >>>
> >> >> >>> >> >> > > >>
> >> >> >>> >> >> > >
> >> >> >>> >> >> > >
> >> >> >>> >> >> >
> >> >> >>> >> >>
> >> >> >>> >>
> >> >> >>> >>
> >> >> >>>
> >> >> >>>
> >> >> >>
> >> >>
> >> >>
> >>
> >>
>
>

Re: Samza as a Caching layer

Posted by Chris Riccomini <cr...@linkedin.com.INVALID>.
Hey Shekar,

The SLF4J stuff is saying that you don't have an slf4j binding on your
classpath. Try adding slf4j-log4j as a runtime dependency on your project.

Cheers,
Chris

On 9/2/14 3:24 PM, "Shekar Tippur" <ct...@gmail.com> wrote:

>Chris ,
>
>In the current state, I just want Samza to connect to 127.0.0.1.
>
>I have set YARN_HOME to
>
>$ echo $YARN_HOME
>
>/home/ctippur/hello-samza/deploy/yarn
>
>I still dont see anything on hadoop console.
>
> Also, I see this during startup
>
>
>SLF4J: Failed to load class "org.slf4j.impl.StaticLoggerBinder".
>
>SLF4J: Defaulting to no-operation (NOP) logger implementation
>
>SLF4J: See http://www.slf4j.org/codes.html#StaticLoggerBinder for further
>details.
>
>SLF4J: Failed to load class "org.slf4j.impl.StaticLoggerBinder".
>
>SLF4J: Defaulting to no-operation (NOP) logger implementation
>
>SLF4J: See http://www.slf4j.org/codes.html#StaticLoggerBinder for further
>details.
>
>- Shekar
>
>
>On Tue, Sep 2, 2014 at 2:20 PM, Chris Riccomini <
>criccomini@linkedin.com.invalid> wrote:
>
>> Hey Shekar,
>>
>> Ah ha. In that case, do you expect your SamzaContainer to try to connect
>> to the RM at 127.0.0.1, or do you expect it to try to connect to some
>> remote RM? If you expect it to try and connect to a remote RM, it's not
>> doing that. Perhaps because YARN_HOME isn't set.
>>
>> If you go to your RM's web interface, how many active nodes do you see
>> listed?
>>
>> Cheers,
>> Chris
>>
>> On 9/2/14 2:17 PM, "Shekar Tippur" <ct...@gmail.com> wrote:
>>
>> >Chris ..
>> >
>> >I am using a rhel server to host all the components (Yarn, kafka,
>>samza).
>> >I
>> >dont have ACLs open to wikipedia.
>> >I am following ..
>> >
>> 
>>http://samza.incubator.apache.org/learn/tutorials/latest/run-hello-samza-
>>w
>> >ithout-internet.html
>> >
>> >- Shekar
>> >
>> >
>> >
>> >On Tue, Sep 2, 2014 at 2:13 PM, Chris Riccomini <
>> >criccomini@linkedin.com.invalid> wrote:
>> >
>> >> Hey Shekar,
>> >>
>> >> Can you try changing that to:
>> >>
>> >>   http://127.0.0.1:8088/cluster
>> >>
>> >>
>> >> And see if you can connect?
>> >>
>> >> Cheers,
>> >> Chris
>> >>
>> >> On 9/2/14 1:21 PM, "Shekar Tippur" <ct...@gmail.com> wrote:
>> >>
>> >> >Other observation is ..
>> >> >
>> >> >http://10.132.62.185:8088/cluster shows that no applications are
>> >>running.
>> >> >
>> >> >- Shekar
>> >> >
>> >> >
>> >> >
>> >> >
>> >> >On Tue, Sep 2, 2014 at 1:15 PM, Shekar Tippur <ct...@gmail.com>
>> >>wrote:
>> >> >
>> >> >> Yarn seem to be running ..
>> >> >>
>> >> >> yarn      5462  0.0  2.0 1641296 161404 ?      Sl   Jun20  95:26
>> >> >> /usr/java/jdk1.6.0_31/bin/java -Dproc_resourcemanager -Xmx1000m
>> >> >> -Dhadoop.log.dir=/var/log/hadoop-yarn
>> >> >>-Dyarn.log.dir=/var/log/hadoop-yarn
>> >> >> -Dhadoop.log.file=yarn-yarn-resourcemanager-pppdc9prd2y2.log
>> >> >> -Dyarn.log.file=yarn-yarn-resourcemanager-pppdc9prd2y2.log
>> >> >> -Dyarn.home.dir=/usr/lib/hadoop-yarn
>> >> >>-Dhadoop.home.dir=/usr/lib/hadoop-yarn
>> >> >> -Dhadoop.root.logger=INFO,RFA -Dyarn.root.logger=INFO,RFA
>> >> >> -Djava.library.path=/usr/lib/hadoop/lib/native -classpath
>> >> >>
>> >>
>> 
>>>>>>/etc/hadoop/conf:/etc/hadoop/conf:/etc/hadoop/conf:/usr/lib/hadoop/li
>>>>>>b/
>> >>>>*:
>> >>
>> 
>>>>>>/usr/lib/hadoop/.//*:/usr/lib/hadoop-hdfs/./:/usr/lib/hadoop-hdfs/lib
>>>>>>/*
>> >>>>:/
>> >>
>> 
>>>>>>usr/lib/hadoop-hdfs/.//*:/usr/lib/hadoop-yarn/lib/*:/usr/lib/hadoop-y
>>>>>>ar
>> >>>>n/
>> >>
>> 
>>>>>>.//*:/usr/lib/hadoop-mapreduce/lib/*:/usr/lib/hadoop-mapreduce/.//*:/
>>>>>>us
>> >>>>r/
>> >>
>> 
>>>>>>lib/hadoop-yarn/.//*:/usr/lib/hadoop-yarn/lib/*:/etc/hadoop/conf/rm-c
>>>>>>on
>> >>>>fi
>> >> >>g/log4j.properties
>> >> >> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager
>> >> >>
>> >> >> I can tail kafka topic as well ..
>> >> >>
>> >> >> deploy/kafka/bin/kafka-console-consumer.sh  --zookeeper
>> >>localhost:2181
>> >> >>--topic wikipedia-raw
>> >> >>
>> >> >>
>> >> >>
>> >> >>
>> >> >> On Tue, Sep 2, 2014 at 1:04 PM, Chris Riccomini <
>> >> >> criccomini@linkedin.com.invalid> wrote:
>> >> >>
>> >> >>> Hey Shekar,
>> >> >>>
>> >> >>> It looks like your job is hanging trying to connect to the RM on
>> >>your
>> >> >>> localhost. I thought that you said that your job was running in
>> >>local
>> >> >>> mode. If so, it should be using the LocalJobFactory. If not, and
>>you
>> >> >>> intend to run on YARN, is your YARN RM up and running on
>>localhost?
>> >> >>>
>> >> >>> Cheers,
>> >> >>> Chris
>> >> >>>
>> >> >>> On 9/2/14 12:22 PM, "Shekar Tippur" <ct...@gmail.com> wrote:
>> >> >>>
>> >> >>> >Chris ..
>> >> >>> >
>> >> >>> >$ cat ./deploy/samza/undefined-samza-container-name.log
>> >> >>> >
>> >> >>> >2014-09-02 11:17:58 JobRunner [INFO] job factory:
>> >> >>> >org.apache.samza.job.yarn.YarnJobFactory
>> >> >>> >
>> >> >>> >2014-09-02 11:17:59 ClientHelper [INFO] trying to connect to RM
>> >> >>> >127.0.0.1:8032
>> >> >>> >
>> >> >>> >2014-09-02 11:17:59 NativeCodeLoader [WARN] Unable to load
>> >> >>>native-hadoop
>> >> >>> >library for your platform... using builtin-java classes where
>> >> >>>applicable
>> >> >>> >
>> >> >>> >2014-09-02 11:17:59 RMProxy [INFO] Connecting to ResourceManager
>> >>at /
>> >> >>> >127.0.0.1:8032
>> >> >>> >
>> >> >>> >
>> >> >>> >and Log4j ..
>> >> >>> >
>> >> >>> ><!DOCTYPE log4j:configuration SYSTEM "log4j.dtd">
>> >> >>> >
>> >> >>> ><log4j:configuration
>> >>xmlns:log4j="http://jakarta.apache.org/log4j/">
>> >> >>> >
>> >> >>> >  <appender name="RollingAppender"
>> >> >>> >class="org.apache.log4j.DailyRollingFileAppender">
>> >> >>> >
>> >> >>> >     <param name="File"
>> >> >>> >value="${samza.log.dir}/${samza.container.name}.log"
>> >> >>> >/>
>> >> >>> >
>> >> >>> >     <param name="DatePattern" value="'.'yyyy-MM-dd" />
>> >> >>> >
>> >> >>> >     <layout class="org.apache.log4j.PatternLayout">
>> >> >>> >
>> >> >>> >      <param name="ConversionPattern" value="%d{yyyy-MM-dd
>> >>HH:mm:ss}
>> >> >>> %c{1}
>> >> >>> >[%p] %m%n" />
>> >> >>> >
>> >> >>> >     </layout>
>> >> >>> >
>> >> >>> >  </appender>
>> >> >>> >
>> >> >>> >  <root>
>> >> >>> >
>> >> >>> >    <priority value="info" />
>> >> >>> >
>> >> >>> >    <appender-ref ref="RollingAppender"/>
>> >> >>> >
>> >> >>> >  </root>
>> >> >>> >
>> >> >>> ></log4j:configuration>
>> >> >>> >
>> >> >>> >
>> >> >>> >On Tue, Sep 2, 2014 at 12:02 PM, Chris Riccomini <
>> >> >>> >criccomini@linkedin.com.invalid> wrote:
>> >> >>> >
>> >> >>> >> Hey Shekar,
>> >> >>> >>
>> >> >>> >> Can you attach your log files? I'm wondering if it's a
>> >> >>>mis-configured
>> >> >>> >> log4j.xml (or missing slf4j-log4j jar), which is leading to
>> >>nearly
>> >> >>> empty
>> >> >>> >> log files. Also, I'm wondering if the job starts fully.
>>Anything
>> >>you
>> >> >>> can
>> >> >>> >> attach would be helpful.
>> >> >>> >>
>> >> >>> >> Cheers,
>> >> >>> >> Chris
>> >> >>> >>
>> >> >>> >> On 9/2/14 11:43 AM, "Shekar Tippur" <ct...@gmail.com> wrote:
>> >> >>> >>
>> >> >>> >> >I am running in local mode.
>> >> >>> >> >
>> >> >>> >> >S
>> >> >>> >> >On Sep 2, 2014 11:42 AM, "Yan Fang" <ya...@gmail.com>
>> >>wrote:
>> >> >>> >> >
>> >> >>> >> >> Hi Shekar.
>> >> >>> >> >>
>> >> >>> >> >> Are you running job in local mode or yarn mode? If yarn
>>mode,
>> >>the
>> >> >>> log
>> >> >>> >> >>is in
>> >> >>> >> >> the yarn's container log.
>> >> >>> >> >>
>> >> >>> >> >> Thanks,
>> >> >>> >> >>
>> >> >>> >> >> Fang, Yan
>> >> >>> >> >> yanfang724@gmail.com
>> >> >>> >> >> +1 (206) 849-4108
>> >> >>> >> >>
>> >> >>> >> >>
>> >> >>> >> >> On Tue, Sep 2, 2014 at 11:36 AM, Shekar Tippur
>> >> >>><ct...@gmail.com>
>> >> >>> >> >>wrote:
>> >> >>> >> >>
>> >> >>> >> >> > Chris,
>> >> >>> >> >> >
>> >> >>> >> >> > Got some time to play around a bit more.
>> >> >>> >> >> > I tried to edit
>> >> >>> >> >> >
>> >> >>> >> >> >
>> >> >>> >> >>
>> >> >>> >>
>> >> >>>
>> >> >>>
>> >>
>> 
>>>>>>>>>>>samza-wikipedia/src/main/java/samza/examples/wikipedia/task/Wiki
>>>>>>>>>>>pe
>> >>>>>>>>>di
>> >> >>>>>>>aFe
>> >> >>> >>>>ed
>> >> >>> >> >>StreamTask.java
>> >> >>> >> >> > to add a logger info statement to tap the incoming
>>message.
>> >>I
>> >> >>>dont
>> >> >>> >>see
>> >> >>> >> >> the
>> >> >>> >> >> > messages being printed to the log file.
>> >> >>> >> >> >
>> >> >>> >> >> > Is this the right place to start?
>> >> >>> >> >> >
>> >> >>> >> >> > public class WikipediaFeedStreamTask implements
>>StreamTask {
>> >> >>> >> >> >
>> >> >>> >> >> >   private static final SystemStream OUTPUT_STREAM = new
>> >> >>> >> >> > SystemStream("kafka",
>> >> >>> >> >> > "wikipedia-raw");
>> >> >>> >> >> >
>> >> >>> >> >> >   private static final Logger log =
>>LoggerFactory.getLogger
>> >> >>> >> >> > (WikipediaFeedStreamTask.class);
>> >> >>> >> >> >
>> >> >>> >> >> >   @Override
>> >> >>> >> >> >
>> >> >>> >> >> >   public void process(IncomingMessageEnvelope envelope,
>> >> >>> >> >>MessageCollector
>> >> >>> >> >> > collector, TaskCoordinator coordinator) {
>> >> >>> >> >> >
>> >> >>> >> >> >     Map<String, Object> outgoingMap =
>> >> >>> >> >> > WikipediaFeedEvent.toMap((WikipediaFeedEvent)
>> >> >>> >>envelope.getMessage());
>> >> >>> >> >> >
>> >> >>> >> >> >     log.info(envelope.getMessage().toString());
>> >> >>> >> >> >
>> >> >>> >> >> >     collector.send(new
>> >>OutgoingMessageEnvelope(OUTPUT_STREAM,
>> >> >>> >> >> > outgoingMap));
>> >> >>> >> >> >
>> >> >>> >> >> >   }
>> >> >>> >> >> >
>> >> >>> >> >> > }
>> >> >>> >> >> >
>> >> >>> >> >> >
>> >> >>> >> >> > On Mon, Aug 25, 2014 at 9:01 AM, Chris Riccomini <
>> >> >>> >> >> > criccomini@linkedin.com.invalid> wrote:
>> >> >>> >> >> >
>> >> >>> >> >> > > Hey Shekar,
>> >> >>> >> >> > >
>> >> >>> >> >> > > Your thought process is on the right track. It's
>>probably
>> >> >>>best
>> >> >>> to
>> >> >>> >> >>start
>> >> >>> >> >> > > with hello-samza, and modify it to get what you want.
>>To
>> >> >>>start
>> >> >>> >>with,
>> >> >>> >> >> > > you'll want to:
>> >> >>> >> >> > >
>> >> >>> >> >> > > 1. Write a simple StreamTask that just does something
>> >>silly
>> >> >>>like
>> >> >>> >> >>just
>> >> >>> >> >> > > print messages that it receives.
>> >> >>> >> >> > > 2. Write a configuration for the job that consumes from
>> >>just
>> >> >>>the
>> >> >>> >> >>stream
>> >> >>> >> >> > > (alerts from different sources).
>> >> >>> >> >> > > 3. Run this to make sure you've got it working.
>> >> >>> >> >> > > 4. Now add your table join. This can be either a
>> >>change-data
>> >> >>> >>capture
>> >> >>> >> >> > (CDC)
>> >> >>> >> >> > > stream, or via a remote DB call.
>> >> >>> >> >> > >
>> >> >>> >> >> > > That should get you to a point where you've got your
>>job
>> >>up
>> >> >>>and
>> >> >>> >> >> running.
>> >> >>> >> >> > > From there, you could create your own Maven project,
>>and
>> >> >>>migrate
>> >> >>> >> >>your
>> >> >>> >> >> > code
>> >> >>> >> >> > > over accordingly.
>> >> >>> >> >> > >
>> >> >>> >> >> > > Cheers,
>> >> >>> >> >> > > Chris
>> >> >>> >> >> > >
>> >> >>> >> >> > > On 8/24/14 1:42 AM, "Shekar Tippur" <ct...@gmail.com>
>> >> >>>wrote:
>> >> >>> >> >> > >
>> >> >>> >> >> > > >Chris,
>> >> >>> >> >> > > >
>> >> >>> >> >> > > >I have gone thro the documentation and decided that
>>the
>> >> >>>option
>> >> >>> >> >>that is
>> >> >>> >> >> > > >most
>> >> >>> >> >> > > >suitable for me is stream-table.
>> >> >>> >> >> > > >
>> >> >>> >> >> > > >I see the following things:
>> >> >>> >> >> > > >
>> >> >>> >> >> > > >1. Point samza to a table (database)
>> >> >>> >> >> > > >2. Point Samza to a stream - Alert stream from
>>different
>> >> >>> sources
>> >> >>> >> >> > > >3. Join key like a hostname
>> >> >>> >> >> > > >
>> >> >>> >> >> > > >I have Hello Samza working. To extend that to do what
>>my
>> >> >>>needs
>> >> >>> >> >>are, I
>> >> >>> >> >> am
>> >> >>> >> >> > > >not sure where to start (Needs more code change OR
>> >> >>> configuration
>> >> >>> >> >> changes
>> >> >>> >> >> > > >OR
>> >> >>> >> >> > > >both)?
>> >> >>> >> >> > > >
>> >> >>> >> >> > > >I have gone thro
>> >> >>> >> >> > > >
>> >> >>> >> >> >
>> >> >>> >> >>
>> >> >>> >> >>
>> >> >>> >>
>> >> >>> >>
>> >> >>>
>> >> >>>
>> >>
>> http://samza.incubator.apache.org/learn/documentation/latest/api/overvie
>> >> >>>w
>> >> >>> >> >> > > .
>> >> >>> >> >> > > >html
>> >> >>> >> >> > > >
>> >> >>> >> >> > > >Is my thought process on the right track? Can you
>>please
>> >> >>>point
>> >> >>> >>me
>> >> >>> >> >>to
>> >> >>> >> >> the
>> >> >>> >> >> > > >right direction?
>> >> >>> >> >> > > >
>> >> >>> >> >> > > >- Shekar
>> >> >>> >> >> > > >
>> >> >>> >> >> > > >
>> >> >>> >> >> > > >
>> >> >>> >> >> > > >On Thu, Aug 21, 2014 at 1:08 PM, Shekar Tippur
>> >> >>> >><ct...@gmail.com>
>> >> >>> >> >> > wrote:
>> >> >>> >> >> > > >
>> >> >>> >> >> > > >> Chris,
>> >> >>> >> >> > > >>
>> >> >>> >> >> > > >> This is perfectly good answer. I will start poking
>>more
>> >> >>>into
>> >> >>> >> >>option
>> >> >>> >> >> > #4.
>> >> >>> >> >> > > >>
>> >> >>> >> >> > > >> - Shekar
>> >> >>> >> >> > > >>
>> >> >>> >> >> > > >>
>> >> >>> >> >> > > >> On Thu, Aug 21, 2014 at 1:05 PM, Chris Riccomini <
>> >> >>> >> >> > > >> criccomini@linkedin.com.invalid> wrote:
>> >> >>> >> >> > > >>
>> >> >>> >> >> > > >>> Hey Shekar,
>> >> >>> >> >> > > >>>
>> >> >>> >> >> > > >>> Your two options are really (3) or (4), then. You
>>can
>> >> >>>either
>> >> >>> >>run
>> >> >>> >> >> some
>> >> >>> >> >> > > >>> external DB that holds the data set, and you can
>> >>query it
>> >> >>> >>from a
>> >> >>> >> >> > > >>> StreamTask, or you can use Samza's state store
>> >>feature to
>> >> >>> >>push
>> >> >>> >> >>data
>> >> >>> >> >> > > >>>into a
>> >> >>> >> >> > > >>> stream that you can then store in a partitioned
>> >>key-value
>> >> >>> >>store
>> >> >>> >> >> along
>> >> >>> >> >> > > >>>with
>> >> >>> >> >> > > >>> your StreamTasks. There is some documentation here
>> >>about
>> >> >>>the
>> >> >>> >> >>state
>> >> >>> >> >> > > >>>store
>> >> >>> >> >> > > >>> approach:
>> >> >>> >> >> > > >>>
>> >> >>> >> >> > > >>>
>> >> >>> >> >> > > >>>
>> >> >>> >> >> > > >>>
>> >> >>> >> >> > >
>> >> >>> >> >>
>> >> >>> >>
>> >> >>>
>> >> >>>
>> >>
>> http://samza.incubator.apache.org/learn/documentation/0.7.0/container/st
>> >> >>> >> >> > > >>>ate
>> >> >>> >> >> > > >>> -management.html
>> >> >>> >> >> > > >>>
>> >> >>> >> >> > > >>><
>> >> >>> >> >> > >
>> >> >>> >> >>
>> >> >>>
>> >> >>>>>
>> >> 
>>http://samza.incubator.apache.org/learn/documentation/0.7.0/container/
>> >> >>>>>s
>> >> >>> >> >> > > >>>tate-management.html>
>> >> >>> >> >> > > >>>
>> >> >>> >> >> > > >>>
>> >> >>> >> >> > > >>> (4) is going to require more up front effort from
>>you,
>> >> >>>since
>> >> >>> >> >>you'll
>> >> >>> >> >> > > >>>have
>> >> >>> >> >> > > >>> to understand how Kafka's partitioning model works,
>> >>and
>> >> >>> setup
>> >> >>> >> >>some
>> >> >>> >> >> > > >>> pipeline to push the updates for your state. In the
>> >>long
>> >> >>> >>run, I
>> >> >>> >> >> > believe
>> >> >>> >> >> > > >>> it's the better approach, though. Local lookups on
>>a
>> >> >>> >>key-value
>> >> >>> >> >> store
>> >> >>> >> >> > > >>> should be faster than doing remote RPC calls to a
>>DB
>> >>for
>> >> >>> >>every
>> >> >>> >> >> > message.
>> >> >>> >> >> > > >>>
>> >> >>> >> >> > > >>> I'm sorry I can't give you a more definitive
>>answer.
>> >>It's
>> >> >>> >>really
>> >> >>> >> >> > about
>> >> >>> >> >> > > >>> trade-offs.
>> >> >>> >> >> > > >>>
>> >> >>> >> >> > > >>> Cheers,
>> >> >>> >> >> > > >>> Chris
>> >> >>> >> >> > > >>>
>> >> >>> >> >> > > >>> On 8/21/14 12:22 PM, "Shekar Tippur"
>> >><ct...@gmail.com>
>> >> >>> >>wrote:
>> >> >>> >> >> > > >>>
>> >> >>> >> >> > > >>> >Chris,
>> >> >>> >> >> > > >>> >
>> >> >>> >> >> > > >>> >A big thanks for a swift response. The data set is
>> >>huge
>> >> >>>and
>> >> >>> >>the
>> >> >>> >> >> > > >>>frequency
>> >> >>> >> >> > > >>> >is in burst.
>> >> >>> >> >> > > >>> >What do you suggest?
>> >> >>> >> >> > > >>> >
>> >> >>> >> >> > > >>> >- Shekar
>> >> >>> >> >> > > >>> >
>> >> >>> >> >> > > >>> >
>> >> >>> >> >> > > >>> >On Thu, Aug 21, 2014 at 11:05 AM, Chris Riccomini
>><
>> >> >>> >> >> > > >>> >criccomini@linkedin.com.invalid> wrote:
>> >> >>> >> >> > > >>> >
>> >> >>> >> >> > > >>> >> Hey Shekar,
>> >> >>> >> >> > > >>> >>
>> >> >>> >> >> > > >>> >> This is feasible, and you are on the right
>>thought
>> >> >>> >>process.
>> >> >>> >> >> > > >>> >>
>> >> >>> >> >> > > >>> >> For the sake of discussion, I'm going to pretend
>> >>that
>> >> >>>you
>> >> >>> >> >>have a
>> >> >>> >> >> > > >>>Kafka
>> >> >>> >> >> > > >>> >> topic called "PageViewEvent", which has just
>>the IP
>> >> >>> >>address
>> >> >>> >> >>that
>> >> >>> >> >> > was
>> >> >>> >> >> > > >>> >>used
>> >> >>> >> >> > > >>> >> to view a page. These messages will be logged
>>every
>> >> >>>time
>> >> >>> a
>> >> >>> >> >>page
>> >> >>> >> >> > view
>> >> >>> >> >> > > >>> >> happens. I'm also going to pretend that you have
>> >>some
>> >> >>> >>state
>> >> >>> >> >> called
>> >> >>> >> >> > > >>> >>"IPGeo"
>> >> >>> >> >> > > >>> >> (e.g. The maxmind data set). In this example,
>>we'll
>> >> >>>want
>> >> >>> >>to
>> >> >>> >> >>join
>> >> >>> >> >> > the
>> >> >>> >> >> > > >>> >> long/lat geo information from IPGeo to the
>> >> >>>PageViewEvent,
>> >> >>> >>and
>> >> >>> >> >> send
>> >> >>> >> >> > > >>>it
>> >> >>> >> >> > > >>> >>to a
>> >> >>> >> >> > > >>> >> new topic: PageViewEventsWithGeo.
>> >> >>> >> >> > > >>> >>
>> >> >>> >> >> > > >>> >> You have several options on how to implement
>>this
>> >> >>> example.
>> >> >>> >> >> > > >>> >>
>> >> >>> >> >> > > >>> >> 1. If your joining data set (IPGeo) is
>>relatively
>> >> >>>small
>> >> >>> >>and
>> >> >>> >> >> > changes
>> >> >>> >> >> > > >>> >> infrequently, you can just pack it up in your
>>jar
>> >>or
>> >> >>>.tgz
>> >> >>> >> >>file,
>> >> >>> >> >> > and
>> >> >>> >> >> > > >>> open
>> >> >>> >> >> > > >>> >> it open in every StreamTask.
>> >> >>> >> >> > > >>> >> 2. If your data set is small, but changes
>>somewhat
>> >> >>> >> >>frequently,
>> >> >>> >> >> you
>> >> >>> >> >> > > >>>can
>> >> >>> >> >> > > >>> >> throw the data set on some HTTP/HDFS/S3 server
>> >> >>>somewhere,
>> >> >>> >>and
>> >> >>> >> >> have
>> >> >>> >> >> > > >>>your
>> >> >>> >> >> > > >>> >> StreamTask refresh it periodically by
>> >>re-downloading
>> >> >>>it.
>> >> >>> >> >> > > >>> >> 3. You can do remote RPC calls for the IPGeo
>>data
>> >>on
>> >> >>> every
>> >> >>> >> >>page
>> >> >>> >> >> > view
>> >> >>> >> >> > > >>> >>event
>> >> >>> >> >> > > >>> >> by query some remote service or DB (e.g.
>> >>Cassandra).
>> >> >>> >> >> > > >>> >> 4. You can use Samza's state feature to set your
>> >>IPGeo
>> >> >>> >>data
>> >> >>> >> >>as a
>> >> >>> >> >> > > >>>series
>> >> >>> >> >> > > >>> >>of
>> >> >>> >> >> > > >>> >> messages to a log-compacted Kafka topic
>> >> >>> >> >> > > >>> >> (
>> >> >>> >> >>
>> >>https://cwiki.apache.org/confluence/display/KAFKA/Log+Compaction
>> >> >>> >> >> > ),
>> >> >>> >> >> > > >>> and
>> >> >>> >> >> > > >>> >> configure your Samza job to read this topic as a
>> >> >>> bootstrap
>> >> >>> >> >> stream
>> >> >>> >> >> > > >>> >> (
>> >> >>> >> >> > > >>> >>
>> >> >>> >> >> > > >>> >>
>> >> >>> >> >> > > >>>
>> >> >>> >> >> > > >>>
>> >> >>> >> >> > >
>> >> >>> >> >>
>> >> >>> >>
>> >> >>>
>> >> >>>
>> >>
>> http://samza.incubator.apache.org/learn/documentation/0.7.0/container/st
>> >> >>> >> >> > > >>>r
>> >> >>> >> >> > > >>> >>e
>> >> >>> >> >> > > >>> >> ams.html).
>> >> >>> >> >> > > >>> >>
>> >> >>> >> >> > > >>> >> For (4), you'd have to partition the IPGeo state
>> >>topic
>> >> >>> >> >>according
>> >> >>> >> >> > to
>> >> >>> >> >> > > >>>the
>> >> >>> >> >> > > >>> >> same key as PageViewEvent. If PageViewEvent were
>> >> >>> >>partitioned
>> >> >>> >> >>by,
>> >> >>> >> >> > > >>>say,
>> >> >>> >> >> > > >>> >> member ID, but you want your IPGeo state topic
>>to
>> >>be
>> >> >>> >> >>partitioned
>> >> >>> >> >> > by
>> >> >>> >> >> > > >>>IP
>> >> >>> >> >> > > >>> >> address, then you'd have to have an upstream job
>> >>that
>> >> >>> >> >> > re-partitioned
>> >> >>> >> >> > > >>> >> PageViewEvent into some new topic by IP address.
>> >>This
>> >> >>>new
>> >> >>> >> >>topic
>> >> >>> >> >> > will
>> >> >>> >> >> > > >>> >>have
>> >> >>> >> >> > > >>> >> to have the same number of partitions as the
>>IPGeo
>> >> >>>state
>> >> >>> >> >>topic
>> >> >>> >> >> (if
>> >> >>> >> >> > > >>> IPGeo
>> >> >>> >> >> > > >>> >> has 8 partitions, then the new
>> >> >>>PageViewEventRepartitioned
>> >> >>> >> >>topic
>> >> >>> >> >> > > >>>needs 8
>> >> >>> >> >> > > >>> >>as
>> >> >>> >> >> > > >>> >> well). This will cause your
>> >>PageViewEventRepartitioned
>> >> >>> >>topic
>> >> >>> >> >>and
>> >> >>> >> >> > > >>>your
>> >> >>> >> >> > > >>> >> IPGeo state topic to be aligned such that the
>> >> >>>StreamTask
>> >> >>> >>that
>> >> >>> >> >> gets
>> >> >>> >> >> > > >>>page
>> >> >>> >> >> > > >>> >> views for IP address X will also have the IPGeo
>> >> >>> >>information
>> >> >>> >> >>for
>> >> >>> >> >> IP
>> >> >>> >> >> > > >>> >>address
>> >> >>> >> >> > > >>> >> X.
>> >> >>> >> >> > > >>> >>
>> >> >>> >> >> > > >>> >> Which strategy you pick is really up to you. :)
>> >>(4) is
>> >> >>> the
>> >> >>> >> >>most
>> >> >>> >> >> > > >>> >> complicated, but also the most flexible, and
>>most
>> >> >>> >> >>operationally
>> >> >>> >> >> > > >>>sound.
>> >> >>> >> >> > > >>> >>(1)
>> >> >>> >> >> > > >>> >> is the easiest if it fits your needs.
>> >> >>> >> >> > > >>> >>
>> >> >>> >> >> > > >>> >> Cheers,
>> >> >>> >> >> > > >>> >> Chris
>> >> >>> >> >> > > >>> >>
>> >> >>> >> >> > > >>> >> On 8/21/14 10:15 AM, "Shekar Tippur"
>> >> >>><ct...@gmail.com>
>> >> >>> >> >>wrote:
>> >> >>> >> >> > > >>> >>
>> >> >>> >> >> > > >>> >> >Hello,
>> >> >>> >> >> > > >>> >> >
>> >> >>> >> >> > > >>> >> >I am new to Samza. I have just installed Hello
>> >>Samza
>> >> >>>and
>> >> >>> >> >>got it
>> >> >>> >> >> > > >>> >>working.
>> >> >>> >> >> > > >>> >> >
>> >> >>> >> >> > > >>> >> >Here is the use case for which I am trying to
>>use
>> >> >>>Samza:
>> >> >>> >> >> > > >>> >> >
>> >> >>> >> >> > > >>> >> >
>> >> >>> >> >> > > >>> >> >1. Cache the contextual information which
>>contains
>> >> >>>more
>> >> >>> >> >> > information
>> >> >>> >> >> > > >>> >>about
>> >> >>> >> >> > > >>> >> >the hostname or IP address using
>>Samza/Yarn/Kafka
>> >> >>> >> >> > > >>> >> >2. Collect alert and metric events which
>>contain
>> >> >>>either
>> >> >>> >> >> hostname
>> >> >>> >> >> > > >>>or IP
>> >> >>> >> >> > > >>> >> >address
>> >> >>> >> >> > > >>> >> >3. Append contextual information to the alert
>>and
>> >> >>>metric
>> >> >>> >>and
>> >> >>> >> >> > > >>>insert to
>> >> >>> >> >> > > >>> >>a
>> >> >>> >> >> > > >>> >> >Kafka queue from which other subscribers read
>>off
>> >>of.
>> >> >>> >> >> > > >>> >> >
>> >> >>> >> >> > > >>> >> >Can you please shed some light on
>> >> >>> >> >> > > >>> >> >
>> >> >>> >> >> > > >>> >> >1. Is this feasible?
>> >> >>> >> >> > > >>> >> >2. Am I on the right thought process
>> >> >>> >> >> > > >>> >> >3. How do I start
>> >> >>> >> >> > > >>> >> >
>> >> >>> >> >> > > >>> >> >I now have 1 & 2 of them working disparately. I
>> >>need
>> >> >>>to
>> >> >>> >> >> integrate
>> >> >>> >> >> > > >>> them.
>> >> >>> >> >> > > >>> >> >
>> >> >>> >> >> > > >>> >> >Appreciate any input.
>> >> >>> >> >> > > >>> >> >
>> >> >>> >> >> > > >>> >> >- Shekar
>> >> >>> >> >> > > >>> >>
>> >> >>> >> >> > > >>> >>
>> >> >>> >> >> > > >>>
>> >> >>> >> >> > > >>>
>> >> >>> >> >> > > >>
>> >> >>> >> >> > >
>> >> >>> >> >> > >
>> >> >>> >> >> >
>> >> >>> >> >>
>> >> >>> >>
>> >> >>> >>
>> >> >>>
>> >> >>>
>> >> >>
>> >>
>> >>
>>
>>


Re: Samza as a Caching layer

Posted by Shekar Tippur <ct...@gmail.com>.
If I want to get the stream of messages flowing into Samza, what is the
right file to tap in?

- Shekar

Re: Samza as a Caching layer

Posted by Shekar Tippur <ct...@gmail.com>.
Chris ,

In the current state, I just want Samza to connect to 127.0.0.1.

I have set YARN_HOME to

$ echo $YARN_HOME

/home/ctippur/hello-samza/deploy/yarn

I still dont see anything on hadoop console.

 Also, I see this during startup


SLF4J: Failed to load class "org.slf4j.impl.StaticLoggerBinder".

SLF4J: Defaulting to no-operation (NOP) logger implementation

SLF4J: See http://www.slf4j.org/codes.html#StaticLoggerBinder for further
details.

SLF4J: Failed to load class "org.slf4j.impl.StaticLoggerBinder".

SLF4J: Defaulting to no-operation (NOP) logger implementation

SLF4J: See http://www.slf4j.org/codes.html#StaticLoggerBinder for further
details.

- Shekar


On Tue, Sep 2, 2014 at 2:20 PM, Chris Riccomini <
criccomini@linkedin.com.invalid> wrote:

> Hey Shekar,
>
> Ah ha. In that case, do you expect your SamzaContainer to try to connect
> to the RM at 127.0.0.1, or do you expect it to try to connect to some
> remote RM? If you expect it to try and connect to a remote RM, it's not
> doing that. Perhaps because YARN_HOME isn't set.
>
> If you go to your RM's web interface, how many active nodes do you see
> listed?
>
> Cheers,
> Chris
>
> On 9/2/14 2:17 PM, "Shekar Tippur" <ct...@gmail.com> wrote:
>
> >Chris ..
> >
> >I am using a rhel server to host all the components (Yarn, kafka, samza).
> >I
> >dont have ACLs open to wikipedia.
> >I am following ..
> >
> http://samza.incubator.apache.org/learn/tutorials/latest/run-hello-samza-w
> >ithout-internet.html
> >
> >- Shekar
> >
> >
> >
> >On Tue, Sep 2, 2014 at 2:13 PM, Chris Riccomini <
> >criccomini@linkedin.com.invalid> wrote:
> >
> >> Hey Shekar,
> >>
> >> Can you try changing that to:
> >>
> >>   http://127.0.0.1:8088/cluster
> >>
> >>
> >> And see if you can connect?
> >>
> >> Cheers,
> >> Chris
> >>
> >> On 9/2/14 1:21 PM, "Shekar Tippur" <ct...@gmail.com> wrote:
> >>
> >> >Other observation is ..
> >> >
> >> >http://10.132.62.185:8088/cluster shows that no applications are
> >>running.
> >> >
> >> >- Shekar
> >> >
> >> >
> >> >
> >> >
> >> >On Tue, Sep 2, 2014 at 1:15 PM, Shekar Tippur <ct...@gmail.com>
> >>wrote:
> >> >
> >> >> Yarn seem to be running ..
> >> >>
> >> >> yarn      5462  0.0  2.0 1641296 161404 ?      Sl   Jun20  95:26
> >> >> /usr/java/jdk1.6.0_31/bin/java -Dproc_resourcemanager -Xmx1000m
> >> >> -Dhadoop.log.dir=/var/log/hadoop-yarn
> >> >>-Dyarn.log.dir=/var/log/hadoop-yarn
> >> >> -Dhadoop.log.file=yarn-yarn-resourcemanager-pppdc9prd2y2.log
> >> >> -Dyarn.log.file=yarn-yarn-resourcemanager-pppdc9prd2y2.log
> >> >> -Dyarn.home.dir=/usr/lib/hadoop-yarn
> >> >>-Dhadoop.home.dir=/usr/lib/hadoop-yarn
> >> >> -Dhadoop.root.logger=INFO,RFA -Dyarn.root.logger=INFO,RFA
> >> >> -Djava.library.path=/usr/lib/hadoop/lib/native -classpath
> >> >>
> >>
> >>>>/etc/hadoop/conf:/etc/hadoop/conf:/etc/hadoop/conf:/usr/lib/hadoop/lib/
> >>>>*:
> >>
> >>>>/usr/lib/hadoop/.//*:/usr/lib/hadoop-hdfs/./:/usr/lib/hadoop-hdfs/lib/*
> >>>>:/
> >>
> >>>>usr/lib/hadoop-hdfs/.//*:/usr/lib/hadoop-yarn/lib/*:/usr/lib/hadoop-yar
> >>>>n/
> >>
> >>>>.//*:/usr/lib/hadoop-mapreduce/lib/*:/usr/lib/hadoop-mapreduce/.//*:/us
> >>>>r/
> >>
> >>>>lib/hadoop-yarn/.//*:/usr/lib/hadoop-yarn/lib/*:/etc/hadoop/conf/rm-con
> >>>>fi
> >> >>g/log4j.properties
> >> >> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager
> >> >>
> >> >> I can tail kafka topic as well ..
> >> >>
> >> >> deploy/kafka/bin/kafka-console-consumer.sh  --zookeeper
> >>localhost:2181
> >> >>--topic wikipedia-raw
> >> >>
> >> >>
> >> >>
> >> >>
> >> >> On Tue, Sep 2, 2014 at 1:04 PM, Chris Riccomini <
> >> >> criccomini@linkedin.com.invalid> wrote:
> >> >>
> >> >>> Hey Shekar,
> >> >>>
> >> >>> It looks like your job is hanging trying to connect to the RM on
> >>your
> >> >>> localhost. I thought that you said that your job was running in
> >>local
> >> >>> mode. If so, it should be using the LocalJobFactory. If not, and you
> >> >>> intend to run on YARN, is your YARN RM up and running on localhost?
> >> >>>
> >> >>> Cheers,
> >> >>> Chris
> >> >>>
> >> >>> On 9/2/14 12:22 PM, "Shekar Tippur" <ct...@gmail.com> wrote:
> >> >>>
> >> >>> >Chris ..
> >> >>> >
> >> >>> >$ cat ./deploy/samza/undefined-samza-container-name.log
> >> >>> >
> >> >>> >2014-09-02 11:17:58 JobRunner [INFO] job factory:
> >> >>> >org.apache.samza.job.yarn.YarnJobFactory
> >> >>> >
> >> >>> >2014-09-02 11:17:59 ClientHelper [INFO] trying to connect to RM
> >> >>> >127.0.0.1:8032
> >> >>> >
> >> >>> >2014-09-02 11:17:59 NativeCodeLoader [WARN] Unable to load
> >> >>>native-hadoop
> >> >>> >library for your platform... using builtin-java classes where
> >> >>>applicable
> >> >>> >
> >> >>> >2014-09-02 11:17:59 RMProxy [INFO] Connecting to ResourceManager
> >>at /
> >> >>> >127.0.0.1:8032
> >> >>> >
> >> >>> >
> >> >>> >and Log4j ..
> >> >>> >
> >> >>> ><!DOCTYPE log4j:configuration SYSTEM "log4j.dtd">
> >> >>> >
> >> >>> ><log4j:configuration
> >>xmlns:log4j="http://jakarta.apache.org/log4j/">
> >> >>> >
> >> >>> >  <appender name="RollingAppender"
> >> >>> >class="org.apache.log4j.DailyRollingFileAppender">
> >> >>> >
> >> >>> >     <param name="File"
> >> >>> >value="${samza.log.dir}/${samza.container.name}.log"
> >> >>> >/>
> >> >>> >
> >> >>> >     <param name="DatePattern" value="'.'yyyy-MM-dd" />
> >> >>> >
> >> >>> >     <layout class="org.apache.log4j.PatternLayout">
> >> >>> >
> >> >>> >      <param name="ConversionPattern" value="%d{yyyy-MM-dd
> >>HH:mm:ss}
> >> >>> %c{1}
> >> >>> >[%p] %m%n" />
> >> >>> >
> >> >>> >     </layout>
> >> >>> >
> >> >>> >  </appender>
> >> >>> >
> >> >>> >  <root>
> >> >>> >
> >> >>> >    <priority value="info" />
> >> >>> >
> >> >>> >    <appender-ref ref="RollingAppender"/>
> >> >>> >
> >> >>> >  </root>
> >> >>> >
> >> >>> ></log4j:configuration>
> >> >>> >
> >> >>> >
> >> >>> >On Tue, Sep 2, 2014 at 12:02 PM, Chris Riccomini <
> >> >>> >criccomini@linkedin.com.invalid> wrote:
> >> >>> >
> >> >>> >> Hey Shekar,
> >> >>> >>
> >> >>> >> Can you attach your log files? I'm wondering if it's a
> >> >>>mis-configured
> >> >>> >> log4j.xml (or missing slf4j-log4j jar), which is leading to
> >>nearly
> >> >>> empty
> >> >>> >> log files. Also, I'm wondering if the job starts fully. Anything
> >>you
> >> >>> can
> >> >>> >> attach would be helpful.
> >> >>> >>
> >> >>> >> Cheers,
> >> >>> >> Chris
> >> >>> >>
> >> >>> >> On 9/2/14 11:43 AM, "Shekar Tippur" <ct...@gmail.com> wrote:
> >> >>> >>
> >> >>> >> >I am running in local mode.
> >> >>> >> >
> >> >>> >> >S
> >> >>> >> >On Sep 2, 2014 11:42 AM, "Yan Fang" <ya...@gmail.com>
> >>wrote:
> >> >>> >> >
> >> >>> >> >> Hi Shekar.
> >> >>> >> >>
> >> >>> >> >> Are you running job in local mode or yarn mode? If yarn mode,
> >>the
> >> >>> log
> >> >>> >> >>is in
> >> >>> >> >> the yarn's container log.
> >> >>> >> >>
> >> >>> >> >> Thanks,
> >> >>> >> >>
> >> >>> >> >> Fang, Yan
> >> >>> >> >> yanfang724@gmail.com
> >> >>> >> >> +1 (206) 849-4108
> >> >>> >> >>
> >> >>> >> >>
> >> >>> >> >> On Tue, Sep 2, 2014 at 11:36 AM, Shekar Tippur
> >> >>><ct...@gmail.com>
> >> >>> >> >>wrote:
> >> >>> >> >>
> >> >>> >> >> > Chris,
> >> >>> >> >> >
> >> >>> >> >> > Got some time to play around a bit more.
> >> >>> >> >> > I tried to edit
> >> >>> >> >> >
> >> >>> >> >> >
> >> >>> >> >>
> >> >>> >>
> >> >>>
> >> >>>
> >>
> >>>>>>>>>samza-wikipedia/src/main/java/samza/examples/wikipedia/task/Wikipe
> >>>>>>>>>di
> >> >>>>>>>aFe
> >> >>> >>>>ed
> >> >>> >> >>StreamTask.java
> >> >>> >> >> > to add a logger info statement to tap the incoming message.
> >>I
> >> >>>dont
> >> >>> >>see
> >> >>> >> >> the
> >> >>> >> >> > messages being printed to the log file.
> >> >>> >> >> >
> >> >>> >> >> > Is this the right place to start?
> >> >>> >> >> >
> >> >>> >> >> > public class WikipediaFeedStreamTask implements StreamTask {
> >> >>> >> >> >
> >> >>> >> >> >   private static final SystemStream OUTPUT_STREAM = new
> >> >>> >> >> > SystemStream("kafka",
> >> >>> >> >> > "wikipedia-raw");
> >> >>> >> >> >
> >> >>> >> >> >   private static final Logger log = LoggerFactory.getLogger
> >> >>> >> >> > (WikipediaFeedStreamTask.class);
> >> >>> >> >> >
> >> >>> >> >> >   @Override
> >> >>> >> >> >
> >> >>> >> >> >   public void process(IncomingMessageEnvelope envelope,
> >> >>> >> >>MessageCollector
> >> >>> >> >> > collector, TaskCoordinator coordinator) {
> >> >>> >> >> >
> >> >>> >> >> >     Map<String, Object> outgoingMap =
> >> >>> >> >> > WikipediaFeedEvent.toMap((WikipediaFeedEvent)
> >> >>> >>envelope.getMessage());
> >> >>> >> >> >
> >> >>> >> >> >     log.info(envelope.getMessage().toString());
> >> >>> >> >> >
> >> >>> >> >> >     collector.send(new
> >>OutgoingMessageEnvelope(OUTPUT_STREAM,
> >> >>> >> >> > outgoingMap));
> >> >>> >> >> >
> >> >>> >> >> >   }
> >> >>> >> >> >
> >> >>> >> >> > }
> >> >>> >> >> >
> >> >>> >> >> >
> >> >>> >> >> > On Mon, Aug 25, 2014 at 9:01 AM, Chris Riccomini <
> >> >>> >> >> > criccomini@linkedin.com.invalid> wrote:
> >> >>> >> >> >
> >> >>> >> >> > > Hey Shekar,
> >> >>> >> >> > >
> >> >>> >> >> > > Your thought process is on the right track. It's probably
> >> >>>best
> >> >>> to
> >> >>> >> >>start
> >> >>> >> >> > > with hello-samza, and modify it to get what you want. To
> >> >>>start
> >> >>> >>with,
> >> >>> >> >> > > you'll want to:
> >> >>> >> >> > >
> >> >>> >> >> > > 1. Write a simple StreamTask that just does something
> >>silly
> >> >>>like
> >> >>> >> >>just
> >> >>> >> >> > > print messages that it receives.
> >> >>> >> >> > > 2. Write a configuration for the job that consumes from
> >>just
> >> >>>the
> >> >>> >> >>stream
> >> >>> >> >> > > (alerts from different sources).
> >> >>> >> >> > > 3. Run this to make sure you've got it working.
> >> >>> >> >> > > 4. Now add your table join. This can be either a
> >>change-data
> >> >>> >>capture
> >> >>> >> >> > (CDC)
> >> >>> >> >> > > stream, or via a remote DB call.
> >> >>> >> >> > >
> >> >>> >> >> > > That should get you to a point where you've got your job
> >>up
> >> >>>and
> >> >>> >> >> running.
> >> >>> >> >> > > From there, you could create your own Maven project, and
> >> >>>migrate
> >> >>> >> >>your
> >> >>> >> >> > code
> >> >>> >> >> > > over accordingly.
> >> >>> >> >> > >
> >> >>> >> >> > > Cheers,
> >> >>> >> >> > > Chris
> >> >>> >> >> > >
> >> >>> >> >> > > On 8/24/14 1:42 AM, "Shekar Tippur" <ct...@gmail.com>
> >> >>>wrote:
> >> >>> >> >> > >
> >> >>> >> >> > > >Chris,
> >> >>> >> >> > > >
> >> >>> >> >> > > >I have gone thro the documentation and decided that the
> >> >>>option
> >> >>> >> >>that is
> >> >>> >> >> > > >most
> >> >>> >> >> > > >suitable for me is stream-table.
> >> >>> >> >> > > >
> >> >>> >> >> > > >I see the following things:
> >> >>> >> >> > > >
> >> >>> >> >> > > >1. Point samza to a table (database)
> >> >>> >> >> > > >2. Point Samza to a stream - Alert stream from different
> >> >>> sources
> >> >>> >> >> > > >3. Join key like a hostname
> >> >>> >> >> > > >
> >> >>> >> >> > > >I have Hello Samza working. To extend that to do what my
> >> >>>needs
> >> >>> >> >>are, I
> >> >>> >> >> am
> >> >>> >> >> > > >not sure where to start (Needs more code change OR
> >> >>> configuration
> >> >>> >> >> changes
> >> >>> >> >> > > >OR
> >> >>> >> >> > > >both)?
> >> >>> >> >> > > >
> >> >>> >> >> > > >I have gone thro
> >> >>> >> >> > > >
> >> >>> >> >> >
> >> >>> >> >>
> >> >>> >> >>
> >> >>> >>
> >> >>> >>
> >> >>>
> >> >>>
> >>
> http://samza.incubator.apache.org/learn/documentation/latest/api/overvie
> >> >>>w
> >> >>> >> >> > > .
> >> >>> >> >> > > >html
> >> >>> >> >> > > >
> >> >>> >> >> > > >Is my thought process on the right track? Can you please
> >> >>>point
> >> >>> >>me
> >> >>> >> >>to
> >> >>> >> >> the
> >> >>> >> >> > > >right direction?
> >> >>> >> >> > > >
> >> >>> >> >> > > >- Shekar
> >> >>> >> >> > > >
> >> >>> >> >> > > >
> >> >>> >> >> > > >
> >> >>> >> >> > > >On Thu, Aug 21, 2014 at 1:08 PM, Shekar Tippur
> >> >>> >><ct...@gmail.com>
> >> >>> >> >> > wrote:
> >> >>> >> >> > > >
> >> >>> >> >> > > >> Chris,
> >> >>> >> >> > > >>
> >> >>> >> >> > > >> This is perfectly good answer. I will start poking more
> >> >>>into
> >> >>> >> >>option
> >> >>> >> >> > #4.
> >> >>> >> >> > > >>
> >> >>> >> >> > > >> - Shekar
> >> >>> >> >> > > >>
> >> >>> >> >> > > >>
> >> >>> >> >> > > >> On Thu, Aug 21, 2014 at 1:05 PM, Chris Riccomini <
> >> >>> >> >> > > >> criccomini@linkedin.com.invalid> wrote:
> >> >>> >> >> > > >>
> >> >>> >> >> > > >>> Hey Shekar,
> >> >>> >> >> > > >>>
> >> >>> >> >> > > >>> Your two options are really (3) or (4), then. You can
> >> >>>either
> >> >>> >>run
> >> >>> >> >> some
> >> >>> >> >> > > >>> external DB that holds the data set, and you can
> >>query it
> >> >>> >>from a
> >> >>> >> >> > > >>> StreamTask, or you can use Samza's state store
> >>feature to
> >> >>> >>push
> >> >>> >> >>data
> >> >>> >> >> > > >>>into a
> >> >>> >> >> > > >>> stream that you can then store in a partitioned
> >>key-value
> >> >>> >>store
> >> >>> >> >> along
> >> >>> >> >> > > >>>with
> >> >>> >> >> > > >>> your StreamTasks. There is some documentation here
> >>about
> >> >>>the
> >> >>> >> >>state
> >> >>> >> >> > > >>>store
> >> >>> >> >> > > >>> approach:
> >> >>> >> >> > > >>>
> >> >>> >> >> > > >>>
> >> >>> >> >> > > >>>
> >> >>> >> >> > > >>>
> >> >>> >> >> > >
> >> >>> >> >>
> >> >>> >>
> >> >>>
> >> >>>
> >>
> http://samza.incubator.apache.org/learn/documentation/0.7.0/container/st
> >> >>> >> >> > > >>>ate
> >> >>> >> >> > > >>> -management.html
> >> >>> >> >> > > >>>
> >> >>> >> >> > > >>><
> >> >>> >> >> > >
> >> >>> >> >>
> >> >>>
> >> >>>>>
> >> http://samza.incubator.apache.org/learn/documentation/0.7.0/container/
> >> >>>>>s
> >> >>> >> >> > > >>>tate-management.html>
> >> >>> >> >> > > >>>
> >> >>> >> >> > > >>>
> >> >>> >> >> > > >>> (4) is going to require more up front effort from you,
> >> >>>since
> >> >>> >> >>you'll
> >> >>> >> >> > > >>>have
> >> >>> >> >> > > >>> to understand how Kafka's partitioning model works,
> >>and
> >> >>> setup
> >> >>> >> >>some
> >> >>> >> >> > > >>> pipeline to push the updates for your state. In the
> >>long
> >> >>> >>run, I
> >> >>> >> >> > believe
> >> >>> >> >> > > >>> it's the better approach, though. Local lookups on a
> >> >>> >>key-value
> >> >>> >> >> store
> >> >>> >> >> > > >>> should be faster than doing remote RPC calls to a DB
> >>for
> >> >>> >>every
> >> >>> >> >> > message.
> >> >>> >> >> > > >>>
> >> >>> >> >> > > >>> I'm sorry I can't give you a more definitive answer.
> >>It's
> >> >>> >>really
> >> >>> >> >> > about
> >> >>> >> >> > > >>> trade-offs.
> >> >>> >> >> > > >>>
> >> >>> >> >> > > >>> Cheers,
> >> >>> >> >> > > >>> Chris
> >> >>> >> >> > > >>>
> >> >>> >> >> > > >>> On 8/21/14 12:22 PM, "Shekar Tippur"
> >><ct...@gmail.com>
> >> >>> >>wrote:
> >> >>> >> >> > > >>>
> >> >>> >> >> > > >>> >Chris,
> >> >>> >> >> > > >>> >
> >> >>> >> >> > > >>> >A big thanks for a swift response. The data set is
> >>huge
> >> >>>and
> >> >>> >>the
> >> >>> >> >> > > >>>frequency
> >> >>> >> >> > > >>> >is in burst.
> >> >>> >> >> > > >>> >What do you suggest?
> >> >>> >> >> > > >>> >
> >> >>> >> >> > > >>> >- Shekar
> >> >>> >> >> > > >>> >
> >> >>> >> >> > > >>> >
> >> >>> >> >> > > >>> >On Thu, Aug 21, 2014 at 11:05 AM, Chris Riccomini <
> >> >>> >> >> > > >>> >criccomini@linkedin.com.invalid> wrote:
> >> >>> >> >> > > >>> >
> >> >>> >> >> > > >>> >> Hey Shekar,
> >> >>> >> >> > > >>> >>
> >> >>> >> >> > > >>> >> This is feasible, and you are on the right thought
> >> >>> >>process.
> >> >>> >> >> > > >>> >>
> >> >>> >> >> > > >>> >> For the sake of discussion, I'm going to pretend
> >>that
> >> >>>you
> >> >>> >> >>have a
> >> >>> >> >> > > >>>Kafka
> >> >>> >> >> > > >>> >> topic called "PageViewEvent", which has just the IP
> >> >>> >>address
> >> >>> >> >>that
> >> >>> >> >> > was
> >> >>> >> >> > > >>> >>used
> >> >>> >> >> > > >>> >> to view a page. These messages will be logged every
> >> >>>time
> >> >>> a
> >> >>> >> >>page
> >> >>> >> >> > view
> >> >>> >> >> > > >>> >> happens. I'm also going to pretend that you have
> >>some
> >> >>> >>state
> >> >>> >> >> called
> >> >>> >> >> > > >>> >>"IPGeo"
> >> >>> >> >> > > >>> >> (e.g. The maxmind data set). In this example, we'll
> >> >>>want
> >> >>> >>to
> >> >>> >> >>join
> >> >>> >> >> > the
> >> >>> >> >> > > >>> >> long/lat geo information from IPGeo to the
> >> >>>PageViewEvent,
> >> >>> >>and
> >> >>> >> >> send
> >> >>> >> >> > > >>>it
> >> >>> >> >> > > >>> >>to a
> >> >>> >> >> > > >>> >> new topic: PageViewEventsWithGeo.
> >> >>> >> >> > > >>> >>
> >> >>> >> >> > > >>> >> You have several options on how to implement this
> >> >>> example.
> >> >>> >> >> > > >>> >>
> >> >>> >> >> > > >>> >> 1. If your joining data set (IPGeo) is relatively
> >> >>>small
> >> >>> >>and
> >> >>> >> >> > changes
> >> >>> >> >> > > >>> >> infrequently, you can just pack it up in your jar
> >>or
> >> >>>.tgz
> >> >>> >> >>file,
> >> >>> >> >> > and
> >> >>> >> >> > > >>> open
> >> >>> >> >> > > >>> >> it open in every StreamTask.
> >> >>> >> >> > > >>> >> 2. If your data set is small, but changes somewhat
> >> >>> >> >>frequently,
> >> >>> >> >> you
> >> >>> >> >> > > >>>can
> >> >>> >> >> > > >>> >> throw the data set on some HTTP/HDFS/S3 server
> >> >>>somewhere,
> >> >>> >>and
> >> >>> >> >> have
> >> >>> >> >> > > >>>your
> >> >>> >> >> > > >>> >> StreamTask refresh it periodically by
> >>re-downloading
> >> >>>it.
> >> >>> >> >> > > >>> >> 3. You can do remote RPC calls for the IPGeo data
> >>on
> >> >>> every
> >> >>> >> >>page
> >> >>> >> >> > view
> >> >>> >> >> > > >>> >>event
> >> >>> >> >> > > >>> >> by query some remote service or DB (e.g.
> >>Cassandra).
> >> >>> >> >> > > >>> >> 4. You can use Samza's state feature to set your
> >>IPGeo
> >> >>> >>data
> >> >>> >> >>as a
> >> >>> >> >> > > >>>series
> >> >>> >> >> > > >>> >>of
> >> >>> >> >> > > >>> >> messages to a log-compacted Kafka topic
> >> >>> >> >> > > >>> >> (
> >> >>> >> >>
> >>https://cwiki.apache.org/confluence/display/KAFKA/Log+Compaction
> >> >>> >> >> > ),
> >> >>> >> >> > > >>> and
> >> >>> >> >> > > >>> >> configure your Samza job to read this topic as a
> >> >>> bootstrap
> >> >>> >> >> stream
> >> >>> >> >> > > >>> >> (
> >> >>> >> >> > > >>> >>
> >> >>> >> >> > > >>> >>
> >> >>> >> >> > > >>>
> >> >>> >> >> > > >>>
> >> >>> >> >> > >
> >> >>> >> >>
> >> >>> >>
> >> >>>
> >> >>>
> >>
> http://samza.incubator.apache.org/learn/documentation/0.7.0/container/st
> >> >>> >> >> > > >>>r
> >> >>> >> >> > > >>> >>e
> >> >>> >> >> > > >>> >> ams.html).
> >> >>> >> >> > > >>> >>
> >> >>> >> >> > > >>> >> For (4), you'd have to partition the IPGeo state
> >>topic
> >> >>> >> >>according
> >> >>> >> >> > to
> >> >>> >> >> > > >>>the
> >> >>> >> >> > > >>> >> same key as PageViewEvent. If PageViewEvent were
> >> >>> >>partitioned
> >> >>> >> >>by,
> >> >>> >> >> > > >>>say,
> >> >>> >> >> > > >>> >> member ID, but you want your IPGeo state topic to
> >>be
> >> >>> >> >>partitioned
> >> >>> >> >> > by
> >> >>> >> >> > > >>>IP
> >> >>> >> >> > > >>> >> address, then you'd have to have an upstream job
> >>that
> >> >>> >> >> > re-partitioned
> >> >>> >> >> > > >>> >> PageViewEvent into some new topic by IP address.
> >>This
> >> >>>new
> >> >>> >> >>topic
> >> >>> >> >> > will
> >> >>> >> >> > > >>> >>have
> >> >>> >> >> > > >>> >> to have the same number of partitions as the IPGeo
> >> >>>state
> >> >>> >> >>topic
> >> >>> >> >> (if
> >> >>> >> >> > > >>> IPGeo
> >> >>> >> >> > > >>> >> has 8 partitions, then the new
> >> >>>PageViewEventRepartitioned
> >> >>> >> >>topic
> >> >>> >> >> > > >>>needs 8
> >> >>> >> >> > > >>> >>as
> >> >>> >> >> > > >>> >> well). This will cause your
> >>PageViewEventRepartitioned
> >> >>> >>topic
> >> >>> >> >>and
> >> >>> >> >> > > >>>your
> >> >>> >> >> > > >>> >> IPGeo state topic to be aligned such that the
> >> >>>StreamTask
> >> >>> >>that
> >> >>> >> >> gets
> >> >>> >> >> > > >>>page
> >> >>> >> >> > > >>> >> views for IP address X will also have the IPGeo
> >> >>> >>information
> >> >>> >> >>for
> >> >>> >> >> IP
> >> >>> >> >> > > >>> >>address
> >> >>> >> >> > > >>> >> X.
> >> >>> >> >> > > >>> >>
> >> >>> >> >> > > >>> >> Which strategy you pick is really up to you. :)
> >>(4) is
> >> >>> the
> >> >>> >> >>most
> >> >>> >> >> > > >>> >> complicated, but also the most flexible, and most
> >> >>> >> >>operationally
> >> >>> >> >> > > >>>sound.
> >> >>> >> >> > > >>> >>(1)
> >> >>> >> >> > > >>> >> is the easiest if it fits your needs.
> >> >>> >> >> > > >>> >>
> >> >>> >> >> > > >>> >> Cheers,
> >> >>> >> >> > > >>> >> Chris
> >> >>> >> >> > > >>> >>
> >> >>> >> >> > > >>> >> On 8/21/14 10:15 AM, "Shekar Tippur"
> >> >>><ct...@gmail.com>
> >> >>> >> >>wrote:
> >> >>> >> >> > > >>> >>
> >> >>> >> >> > > >>> >> >Hello,
> >> >>> >> >> > > >>> >> >
> >> >>> >> >> > > >>> >> >I am new to Samza. I have just installed Hello
> >>Samza
> >> >>>and
> >> >>> >> >>got it
> >> >>> >> >> > > >>> >>working.
> >> >>> >> >> > > >>> >> >
> >> >>> >> >> > > >>> >> >Here is the use case for which I am trying to use
> >> >>>Samza:
> >> >>> >> >> > > >>> >> >
> >> >>> >> >> > > >>> >> >
> >> >>> >> >> > > >>> >> >1. Cache the contextual information which contains
> >> >>>more
> >> >>> >> >> > information
> >> >>> >> >> > > >>> >>about
> >> >>> >> >> > > >>> >> >the hostname or IP address using Samza/Yarn/Kafka
> >> >>> >> >> > > >>> >> >2. Collect alert and metric events which contain
> >> >>>either
> >> >>> >> >> hostname
> >> >>> >> >> > > >>>or IP
> >> >>> >> >> > > >>> >> >address
> >> >>> >> >> > > >>> >> >3. Append contextual information to the alert and
> >> >>>metric
> >> >>> >>and
> >> >>> >> >> > > >>>insert to
> >> >>> >> >> > > >>> >>a
> >> >>> >> >> > > >>> >> >Kafka queue from which other subscribers read off
> >>of.
> >> >>> >> >> > > >>> >> >
> >> >>> >> >> > > >>> >> >Can you please shed some light on
> >> >>> >> >> > > >>> >> >
> >> >>> >> >> > > >>> >> >1. Is this feasible?
> >> >>> >> >> > > >>> >> >2. Am I on the right thought process
> >> >>> >> >> > > >>> >> >3. How do I start
> >> >>> >> >> > > >>> >> >
> >> >>> >> >> > > >>> >> >I now have 1 & 2 of them working disparately. I
> >>need
> >> >>>to
> >> >>> >> >> integrate
> >> >>> >> >> > > >>> them.
> >> >>> >> >> > > >>> >> >
> >> >>> >> >> > > >>> >> >Appreciate any input.
> >> >>> >> >> > > >>> >> >
> >> >>> >> >> > > >>> >> >- Shekar
> >> >>> >> >> > > >>> >>
> >> >>> >> >> > > >>> >>
> >> >>> >> >> > > >>>
> >> >>> >> >> > > >>>
> >> >>> >> >> > > >>
> >> >>> >> >> > >
> >> >>> >> >> > >
> >> >>> >> >> >
> >> >>> >> >>
> >> >>> >>
> >> >>> >>
> >> >>>
> >> >>>
> >> >>
> >>
> >>
>
>

Re: Samza as a Caching layer

Posted by Chris Riccomini <cr...@linkedin.com.INVALID>.
Hey Shekar,

Ah ha. In that case, do you expect your SamzaContainer to try to connect
to the RM at 127.0.0.1, or do you expect it to try to connect to some
remote RM? If you expect it to try and connect to a remote RM, it's not
doing that. Perhaps because YARN_HOME isn't set.

If you go to your RM's web interface, how many active nodes do you see
listed?

Cheers,
Chris

On 9/2/14 2:17 PM, "Shekar Tippur" <ct...@gmail.com> wrote:

>Chris ..
>
>I am using a rhel server to host all the components (Yarn, kafka, samza).
>I
>dont have ACLs open to wikipedia.
>I am following ..
>http://samza.incubator.apache.org/learn/tutorials/latest/run-hello-samza-w
>ithout-internet.html
>
>- Shekar
>
>
>
>On Tue, Sep 2, 2014 at 2:13 PM, Chris Riccomini <
>criccomini@linkedin.com.invalid> wrote:
>
>> Hey Shekar,
>>
>> Can you try changing that to:
>>
>>   http://127.0.0.1:8088/cluster
>>
>>
>> And see if you can connect?
>>
>> Cheers,
>> Chris
>>
>> On 9/2/14 1:21 PM, "Shekar Tippur" <ct...@gmail.com> wrote:
>>
>> >Other observation is ..
>> >
>> >http://10.132.62.185:8088/cluster shows that no applications are
>>running.
>> >
>> >- Shekar
>> >
>> >
>> >
>> >
>> >On Tue, Sep 2, 2014 at 1:15 PM, Shekar Tippur <ct...@gmail.com>
>>wrote:
>> >
>> >> Yarn seem to be running ..
>> >>
>> >> yarn      5462  0.0  2.0 1641296 161404 ?      Sl   Jun20  95:26
>> >> /usr/java/jdk1.6.0_31/bin/java -Dproc_resourcemanager -Xmx1000m
>> >> -Dhadoop.log.dir=/var/log/hadoop-yarn
>> >>-Dyarn.log.dir=/var/log/hadoop-yarn
>> >> -Dhadoop.log.file=yarn-yarn-resourcemanager-pppdc9prd2y2.log
>> >> -Dyarn.log.file=yarn-yarn-resourcemanager-pppdc9prd2y2.log
>> >> -Dyarn.home.dir=/usr/lib/hadoop-yarn
>> >>-Dhadoop.home.dir=/usr/lib/hadoop-yarn
>> >> -Dhadoop.root.logger=INFO,RFA -Dyarn.root.logger=INFO,RFA
>> >> -Djava.library.path=/usr/lib/hadoop/lib/native -classpath
>> >>
>> 
>>>>/etc/hadoop/conf:/etc/hadoop/conf:/etc/hadoop/conf:/usr/lib/hadoop/lib/
>>>>*:
>> 
>>>>/usr/lib/hadoop/.//*:/usr/lib/hadoop-hdfs/./:/usr/lib/hadoop-hdfs/lib/*
>>>>:/
>> 
>>>>usr/lib/hadoop-hdfs/.//*:/usr/lib/hadoop-yarn/lib/*:/usr/lib/hadoop-yar
>>>>n/
>> 
>>>>.//*:/usr/lib/hadoop-mapreduce/lib/*:/usr/lib/hadoop-mapreduce/.//*:/us
>>>>r/
>> 
>>>>lib/hadoop-yarn/.//*:/usr/lib/hadoop-yarn/lib/*:/etc/hadoop/conf/rm-con
>>>>fi
>> >>g/log4j.properties
>> >> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager
>> >>
>> >> I can tail kafka topic as well ..
>> >>
>> >> deploy/kafka/bin/kafka-console-consumer.sh  --zookeeper
>>localhost:2181
>> >>--topic wikipedia-raw
>> >>
>> >>
>> >>
>> >>
>> >> On Tue, Sep 2, 2014 at 1:04 PM, Chris Riccomini <
>> >> criccomini@linkedin.com.invalid> wrote:
>> >>
>> >>> Hey Shekar,
>> >>>
>> >>> It looks like your job is hanging trying to connect to the RM on
>>your
>> >>> localhost. I thought that you said that your job was running in
>>local
>> >>> mode. If so, it should be using the LocalJobFactory. If not, and you
>> >>> intend to run on YARN, is your YARN RM up and running on localhost?
>> >>>
>> >>> Cheers,
>> >>> Chris
>> >>>
>> >>> On 9/2/14 12:22 PM, "Shekar Tippur" <ct...@gmail.com> wrote:
>> >>>
>> >>> >Chris ..
>> >>> >
>> >>> >$ cat ./deploy/samza/undefined-samza-container-name.log
>> >>> >
>> >>> >2014-09-02 11:17:58 JobRunner [INFO] job factory:
>> >>> >org.apache.samza.job.yarn.YarnJobFactory
>> >>> >
>> >>> >2014-09-02 11:17:59 ClientHelper [INFO] trying to connect to RM
>> >>> >127.0.0.1:8032
>> >>> >
>> >>> >2014-09-02 11:17:59 NativeCodeLoader [WARN] Unable to load
>> >>>native-hadoop
>> >>> >library for your platform... using builtin-java classes where
>> >>>applicable
>> >>> >
>> >>> >2014-09-02 11:17:59 RMProxy [INFO] Connecting to ResourceManager
>>at /
>> >>> >127.0.0.1:8032
>> >>> >
>> >>> >
>> >>> >and Log4j ..
>> >>> >
>> >>> ><!DOCTYPE log4j:configuration SYSTEM "log4j.dtd">
>> >>> >
>> >>> ><log4j:configuration
>>xmlns:log4j="http://jakarta.apache.org/log4j/">
>> >>> >
>> >>> >  <appender name="RollingAppender"
>> >>> >class="org.apache.log4j.DailyRollingFileAppender">
>> >>> >
>> >>> >     <param name="File"
>> >>> >value="${samza.log.dir}/${samza.container.name}.log"
>> >>> >/>
>> >>> >
>> >>> >     <param name="DatePattern" value="'.'yyyy-MM-dd" />
>> >>> >
>> >>> >     <layout class="org.apache.log4j.PatternLayout">
>> >>> >
>> >>> >      <param name="ConversionPattern" value="%d{yyyy-MM-dd
>>HH:mm:ss}
>> >>> %c{1}
>> >>> >[%p] %m%n" />
>> >>> >
>> >>> >     </layout>
>> >>> >
>> >>> >  </appender>
>> >>> >
>> >>> >  <root>
>> >>> >
>> >>> >    <priority value="info" />
>> >>> >
>> >>> >    <appender-ref ref="RollingAppender"/>
>> >>> >
>> >>> >  </root>
>> >>> >
>> >>> ></log4j:configuration>
>> >>> >
>> >>> >
>> >>> >On Tue, Sep 2, 2014 at 12:02 PM, Chris Riccomini <
>> >>> >criccomini@linkedin.com.invalid> wrote:
>> >>> >
>> >>> >> Hey Shekar,
>> >>> >>
>> >>> >> Can you attach your log files? I'm wondering if it's a
>> >>>mis-configured
>> >>> >> log4j.xml (or missing slf4j-log4j jar), which is leading to
>>nearly
>> >>> empty
>> >>> >> log files. Also, I'm wondering if the job starts fully. Anything
>>you
>> >>> can
>> >>> >> attach would be helpful.
>> >>> >>
>> >>> >> Cheers,
>> >>> >> Chris
>> >>> >>
>> >>> >> On 9/2/14 11:43 AM, "Shekar Tippur" <ct...@gmail.com> wrote:
>> >>> >>
>> >>> >> >I am running in local mode.
>> >>> >> >
>> >>> >> >S
>> >>> >> >On Sep 2, 2014 11:42 AM, "Yan Fang" <ya...@gmail.com>
>>wrote:
>> >>> >> >
>> >>> >> >> Hi Shekar.
>> >>> >> >>
>> >>> >> >> Are you running job in local mode or yarn mode? If yarn mode,
>>the
>> >>> log
>> >>> >> >>is in
>> >>> >> >> the yarn's container log.
>> >>> >> >>
>> >>> >> >> Thanks,
>> >>> >> >>
>> >>> >> >> Fang, Yan
>> >>> >> >> yanfang724@gmail.com
>> >>> >> >> +1 (206) 849-4108
>> >>> >> >>
>> >>> >> >>
>> >>> >> >> On Tue, Sep 2, 2014 at 11:36 AM, Shekar Tippur
>> >>><ct...@gmail.com>
>> >>> >> >>wrote:
>> >>> >> >>
>> >>> >> >> > Chris,
>> >>> >> >> >
>> >>> >> >> > Got some time to play around a bit more.
>> >>> >> >> > I tried to edit
>> >>> >> >> >
>> >>> >> >> >
>> >>> >> >>
>> >>> >>
>> >>>
>> >>>
>> 
>>>>>>>>>samza-wikipedia/src/main/java/samza/examples/wikipedia/task/Wikipe
>>>>>>>>>di
>> >>>>>>>aFe
>> >>> >>>>ed
>> >>> >> >>StreamTask.java
>> >>> >> >> > to add a logger info statement to tap the incoming message.
>>I
>> >>>dont
>> >>> >>see
>> >>> >> >> the
>> >>> >> >> > messages being printed to the log file.
>> >>> >> >> >
>> >>> >> >> > Is this the right place to start?
>> >>> >> >> >
>> >>> >> >> > public class WikipediaFeedStreamTask implements StreamTask {
>> >>> >> >> >
>> >>> >> >> >   private static final SystemStream OUTPUT_STREAM = new
>> >>> >> >> > SystemStream("kafka",
>> >>> >> >> > "wikipedia-raw");
>> >>> >> >> >
>> >>> >> >> >   private static final Logger log = LoggerFactory.getLogger
>> >>> >> >> > (WikipediaFeedStreamTask.class);
>> >>> >> >> >
>> >>> >> >> >   @Override
>> >>> >> >> >
>> >>> >> >> >   public void process(IncomingMessageEnvelope envelope,
>> >>> >> >>MessageCollector
>> >>> >> >> > collector, TaskCoordinator coordinator) {
>> >>> >> >> >
>> >>> >> >> >     Map<String, Object> outgoingMap =
>> >>> >> >> > WikipediaFeedEvent.toMap((WikipediaFeedEvent)
>> >>> >>envelope.getMessage());
>> >>> >> >> >
>> >>> >> >> >     log.info(envelope.getMessage().toString());
>> >>> >> >> >
>> >>> >> >> >     collector.send(new
>>OutgoingMessageEnvelope(OUTPUT_STREAM,
>> >>> >> >> > outgoingMap));
>> >>> >> >> >
>> >>> >> >> >   }
>> >>> >> >> >
>> >>> >> >> > }
>> >>> >> >> >
>> >>> >> >> >
>> >>> >> >> > On Mon, Aug 25, 2014 at 9:01 AM, Chris Riccomini <
>> >>> >> >> > criccomini@linkedin.com.invalid> wrote:
>> >>> >> >> >
>> >>> >> >> > > Hey Shekar,
>> >>> >> >> > >
>> >>> >> >> > > Your thought process is on the right track. It's probably
>> >>>best
>> >>> to
>> >>> >> >>start
>> >>> >> >> > > with hello-samza, and modify it to get what you want. To
>> >>>start
>> >>> >>with,
>> >>> >> >> > > you'll want to:
>> >>> >> >> > >
>> >>> >> >> > > 1. Write a simple StreamTask that just does something
>>silly
>> >>>like
>> >>> >> >>just
>> >>> >> >> > > print messages that it receives.
>> >>> >> >> > > 2. Write a configuration for the job that consumes from
>>just
>> >>>the
>> >>> >> >>stream
>> >>> >> >> > > (alerts from different sources).
>> >>> >> >> > > 3. Run this to make sure you've got it working.
>> >>> >> >> > > 4. Now add your table join. This can be either a
>>change-data
>> >>> >>capture
>> >>> >> >> > (CDC)
>> >>> >> >> > > stream, or via a remote DB call.
>> >>> >> >> > >
>> >>> >> >> > > That should get you to a point where you've got your job
>>up
>> >>>and
>> >>> >> >> running.
>> >>> >> >> > > From there, you could create your own Maven project, and
>> >>>migrate
>> >>> >> >>your
>> >>> >> >> > code
>> >>> >> >> > > over accordingly.
>> >>> >> >> > >
>> >>> >> >> > > Cheers,
>> >>> >> >> > > Chris
>> >>> >> >> > >
>> >>> >> >> > > On 8/24/14 1:42 AM, "Shekar Tippur" <ct...@gmail.com>
>> >>>wrote:
>> >>> >> >> > >
>> >>> >> >> > > >Chris,
>> >>> >> >> > > >
>> >>> >> >> > > >I have gone thro the documentation and decided that the
>> >>>option
>> >>> >> >>that is
>> >>> >> >> > > >most
>> >>> >> >> > > >suitable for me is stream-table.
>> >>> >> >> > > >
>> >>> >> >> > > >I see the following things:
>> >>> >> >> > > >
>> >>> >> >> > > >1. Point samza to a table (database)
>> >>> >> >> > > >2. Point Samza to a stream - Alert stream from different
>> >>> sources
>> >>> >> >> > > >3. Join key like a hostname
>> >>> >> >> > > >
>> >>> >> >> > > >I have Hello Samza working. To extend that to do what my
>> >>>needs
>> >>> >> >>are, I
>> >>> >> >> am
>> >>> >> >> > > >not sure where to start (Needs more code change OR
>> >>> configuration
>> >>> >> >> changes
>> >>> >> >> > > >OR
>> >>> >> >> > > >both)?
>> >>> >> >> > > >
>> >>> >> >> > > >I have gone thro
>> >>> >> >> > > >
>> >>> >> >> >
>> >>> >> >>
>> >>> >> >>
>> >>> >>
>> >>> >>
>> >>>
>> >>>
>> http://samza.incubator.apache.org/learn/documentation/latest/api/overvie
>> >>>w
>> >>> >> >> > > .
>> >>> >> >> > > >html
>> >>> >> >> > > >
>> >>> >> >> > > >Is my thought process on the right track? Can you please
>> >>>point
>> >>> >>me
>> >>> >> >>to
>> >>> >> >> the
>> >>> >> >> > > >right direction?
>> >>> >> >> > > >
>> >>> >> >> > > >- Shekar
>> >>> >> >> > > >
>> >>> >> >> > > >
>> >>> >> >> > > >
>> >>> >> >> > > >On Thu, Aug 21, 2014 at 1:08 PM, Shekar Tippur
>> >>> >><ct...@gmail.com>
>> >>> >> >> > wrote:
>> >>> >> >> > > >
>> >>> >> >> > > >> Chris,
>> >>> >> >> > > >>
>> >>> >> >> > > >> This is perfectly good answer. I will start poking more
>> >>>into
>> >>> >> >>option
>> >>> >> >> > #4.
>> >>> >> >> > > >>
>> >>> >> >> > > >> - Shekar
>> >>> >> >> > > >>
>> >>> >> >> > > >>
>> >>> >> >> > > >> On Thu, Aug 21, 2014 at 1:05 PM, Chris Riccomini <
>> >>> >> >> > > >> criccomini@linkedin.com.invalid> wrote:
>> >>> >> >> > > >>
>> >>> >> >> > > >>> Hey Shekar,
>> >>> >> >> > > >>>
>> >>> >> >> > > >>> Your two options are really (3) or (4), then. You can
>> >>>either
>> >>> >>run
>> >>> >> >> some
>> >>> >> >> > > >>> external DB that holds the data set, and you can
>>query it
>> >>> >>from a
>> >>> >> >> > > >>> StreamTask, or you can use Samza's state store
>>feature to
>> >>> >>push
>> >>> >> >>data
>> >>> >> >> > > >>>into a
>> >>> >> >> > > >>> stream that you can then store in a partitioned
>>key-value
>> >>> >>store
>> >>> >> >> along
>> >>> >> >> > > >>>with
>> >>> >> >> > > >>> your StreamTasks. There is some documentation here
>>about
>> >>>the
>> >>> >> >>state
>> >>> >> >> > > >>>store
>> >>> >> >> > > >>> approach:
>> >>> >> >> > > >>>
>> >>> >> >> > > >>>
>> >>> >> >> > > >>>
>> >>> >> >> > > >>>
>> >>> >> >> > >
>> >>> >> >>
>> >>> >>
>> >>>
>> >>>
>> http://samza.incubator.apache.org/learn/documentation/0.7.0/container/st
>> >>> >> >> > > >>>ate
>> >>> >> >> > > >>> -management.html
>> >>> >> >> > > >>>
>> >>> >> >> > > >>><
>> >>> >> >> > >
>> >>> >> >>
>> >>>
>> >>>>>
>> http://samza.incubator.apache.org/learn/documentation/0.7.0/container/
>> >>>>>s
>> >>> >> >> > > >>>tate-management.html>
>> >>> >> >> > > >>>
>> >>> >> >> > > >>>
>> >>> >> >> > > >>> (4) is going to require more up front effort from you,
>> >>>since
>> >>> >> >>you'll
>> >>> >> >> > > >>>have
>> >>> >> >> > > >>> to understand how Kafka's partitioning model works,
>>and
>> >>> setup
>> >>> >> >>some
>> >>> >> >> > > >>> pipeline to push the updates for your state. In the
>>long
>> >>> >>run, I
>> >>> >> >> > believe
>> >>> >> >> > > >>> it's the better approach, though. Local lookups on a
>> >>> >>key-value
>> >>> >> >> store
>> >>> >> >> > > >>> should be faster than doing remote RPC calls to a DB
>>for
>> >>> >>every
>> >>> >> >> > message.
>> >>> >> >> > > >>>
>> >>> >> >> > > >>> I'm sorry I can't give you a more definitive answer.
>>It's
>> >>> >>really
>> >>> >> >> > about
>> >>> >> >> > > >>> trade-offs.
>> >>> >> >> > > >>>
>> >>> >> >> > > >>> Cheers,
>> >>> >> >> > > >>> Chris
>> >>> >> >> > > >>>
>> >>> >> >> > > >>> On 8/21/14 12:22 PM, "Shekar Tippur"
>><ct...@gmail.com>
>> >>> >>wrote:
>> >>> >> >> > > >>>
>> >>> >> >> > > >>> >Chris,
>> >>> >> >> > > >>> >
>> >>> >> >> > > >>> >A big thanks for a swift response. The data set is
>>huge
>> >>>and
>> >>> >>the
>> >>> >> >> > > >>>frequency
>> >>> >> >> > > >>> >is in burst.
>> >>> >> >> > > >>> >What do you suggest?
>> >>> >> >> > > >>> >
>> >>> >> >> > > >>> >- Shekar
>> >>> >> >> > > >>> >
>> >>> >> >> > > >>> >
>> >>> >> >> > > >>> >On Thu, Aug 21, 2014 at 11:05 AM, Chris Riccomini <
>> >>> >> >> > > >>> >criccomini@linkedin.com.invalid> wrote:
>> >>> >> >> > > >>> >
>> >>> >> >> > > >>> >> Hey Shekar,
>> >>> >> >> > > >>> >>
>> >>> >> >> > > >>> >> This is feasible, and you are on the right thought
>> >>> >>process.
>> >>> >> >> > > >>> >>
>> >>> >> >> > > >>> >> For the sake of discussion, I'm going to pretend
>>that
>> >>>you
>> >>> >> >>have a
>> >>> >> >> > > >>>Kafka
>> >>> >> >> > > >>> >> topic called "PageViewEvent", which has just the IP
>> >>> >>address
>> >>> >> >>that
>> >>> >> >> > was
>> >>> >> >> > > >>> >>used
>> >>> >> >> > > >>> >> to view a page. These messages will be logged every
>> >>>time
>> >>> a
>> >>> >> >>page
>> >>> >> >> > view
>> >>> >> >> > > >>> >> happens. I'm also going to pretend that you have
>>some
>> >>> >>state
>> >>> >> >> called
>> >>> >> >> > > >>> >>"IPGeo"
>> >>> >> >> > > >>> >> (e.g. The maxmind data set). In this example, we'll
>> >>>want
>> >>> >>to
>> >>> >> >>join
>> >>> >> >> > the
>> >>> >> >> > > >>> >> long/lat geo information from IPGeo to the
>> >>>PageViewEvent,
>> >>> >>and
>> >>> >> >> send
>> >>> >> >> > > >>>it
>> >>> >> >> > > >>> >>to a
>> >>> >> >> > > >>> >> new topic: PageViewEventsWithGeo.
>> >>> >> >> > > >>> >>
>> >>> >> >> > > >>> >> You have several options on how to implement this
>> >>> example.
>> >>> >> >> > > >>> >>
>> >>> >> >> > > >>> >> 1. If your joining data set (IPGeo) is relatively
>> >>>small
>> >>> >>and
>> >>> >> >> > changes
>> >>> >> >> > > >>> >> infrequently, you can just pack it up in your jar
>>or
>> >>>.tgz
>> >>> >> >>file,
>> >>> >> >> > and
>> >>> >> >> > > >>> open
>> >>> >> >> > > >>> >> it open in every StreamTask.
>> >>> >> >> > > >>> >> 2. If your data set is small, but changes somewhat
>> >>> >> >>frequently,
>> >>> >> >> you
>> >>> >> >> > > >>>can
>> >>> >> >> > > >>> >> throw the data set on some HTTP/HDFS/S3 server
>> >>>somewhere,
>> >>> >>and
>> >>> >> >> have
>> >>> >> >> > > >>>your
>> >>> >> >> > > >>> >> StreamTask refresh it periodically by
>>re-downloading
>> >>>it.
>> >>> >> >> > > >>> >> 3. You can do remote RPC calls for the IPGeo data
>>on
>> >>> every
>> >>> >> >>page
>> >>> >> >> > view
>> >>> >> >> > > >>> >>event
>> >>> >> >> > > >>> >> by query some remote service or DB (e.g.
>>Cassandra).
>> >>> >> >> > > >>> >> 4. You can use Samza's state feature to set your
>>IPGeo
>> >>> >>data
>> >>> >> >>as a
>> >>> >> >> > > >>>series
>> >>> >> >> > > >>> >>of
>> >>> >> >> > > >>> >> messages to a log-compacted Kafka topic
>> >>> >> >> > > >>> >> (
>> >>> >> >> 
>>https://cwiki.apache.org/confluence/display/KAFKA/Log+Compaction
>> >>> >> >> > ),
>> >>> >> >> > > >>> and
>> >>> >> >> > > >>> >> configure your Samza job to read this topic as a
>> >>> bootstrap
>> >>> >> >> stream
>> >>> >> >> > > >>> >> (
>> >>> >> >> > > >>> >>
>> >>> >> >> > > >>> >>
>> >>> >> >> > > >>>
>> >>> >> >> > > >>>
>> >>> >> >> > >
>> >>> >> >>
>> >>> >>
>> >>>
>> >>>
>> http://samza.incubator.apache.org/learn/documentation/0.7.0/container/st
>> >>> >> >> > > >>>r
>> >>> >> >> > > >>> >>e
>> >>> >> >> > > >>> >> ams.html).
>> >>> >> >> > > >>> >>
>> >>> >> >> > > >>> >> For (4), you'd have to partition the IPGeo state
>>topic
>> >>> >> >>according
>> >>> >> >> > to
>> >>> >> >> > > >>>the
>> >>> >> >> > > >>> >> same key as PageViewEvent. If PageViewEvent were
>> >>> >>partitioned
>> >>> >> >>by,
>> >>> >> >> > > >>>say,
>> >>> >> >> > > >>> >> member ID, but you want your IPGeo state topic to
>>be
>> >>> >> >>partitioned
>> >>> >> >> > by
>> >>> >> >> > > >>>IP
>> >>> >> >> > > >>> >> address, then you'd have to have an upstream job
>>that
>> >>> >> >> > re-partitioned
>> >>> >> >> > > >>> >> PageViewEvent into some new topic by IP address.
>>This
>> >>>new
>> >>> >> >>topic
>> >>> >> >> > will
>> >>> >> >> > > >>> >>have
>> >>> >> >> > > >>> >> to have the same number of partitions as the IPGeo
>> >>>state
>> >>> >> >>topic
>> >>> >> >> (if
>> >>> >> >> > > >>> IPGeo
>> >>> >> >> > > >>> >> has 8 partitions, then the new
>> >>>PageViewEventRepartitioned
>> >>> >> >>topic
>> >>> >> >> > > >>>needs 8
>> >>> >> >> > > >>> >>as
>> >>> >> >> > > >>> >> well). This will cause your
>>PageViewEventRepartitioned
>> >>> >>topic
>> >>> >> >>and
>> >>> >> >> > > >>>your
>> >>> >> >> > > >>> >> IPGeo state topic to be aligned such that the
>> >>>StreamTask
>> >>> >>that
>> >>> >> >> gets
>> >>> >> >> > > >>>page
>> >>> >> >> > > >>> >> views for IP address X will also have the IPGeo
>> >>> >>information
>> >>> >> >>for
>> >>> >> >> IP
>> >>> >> >> > > >>> >>address
>> >>> >> >> > > >>> >> X.
>> >>> >> >> > > >>> >>
>> >>> >> >> > > >>> >> Which strategy you pick is really up to you. :)
>>(4) is
>> >>> the
>> >>> >> >>most
>> >>> >> >> > > >>> >> complicated, but also the most flexible, and most
>> >>> >> >>operationally
>> >>> >> >> > > >>>sound.
>> >>> >> >> > > >>> >>(1)
>> >>> >> >> > > >>> >> is the easiest if it fits your needs.
>> >>> >> >> > > >>> >>
>> >>> >> >> > > >>> >> Cheers,
>> >>> >> >> > > >>> >> Chris
>> >>> >> >> > > >>> >>
>> >>> >> >> > > >>> >> On 8/21/14 10:15 AM, "Shekar Tippur"
>> >>><ct...@gmail.com>
>> >>> >> >>wrote:
>> >>> >> >> > > >>> >>
>> >>> >> >> > > >>> >> >Hello,
>> >>> >> >> > > >>> >> >
>> >>> >> >> > > >>> >> >I am new to Samza. I have just installed Hello
>>Samza
>> >>>and
>> >>> >> >>got it
>> >>> >> >> > > >>> >>working.
>> >>> >> >> > > >>> >> >
>> >>> >> >> > > >>> >> >Here is the use case for which I am trying to use
>> >>>Samza:
>> >>> >> >> > > >>> >> >
>> >>> >> >> > > >>> >> >
>> >>> >> >> > > >>> >> >1. Cache the contextual information which contains
>> >>>more
>> >>> >> >> > information
>> >>> >> >> > > >>> >>about
>> >>> >> >> > > >>> >> >the hostname or IP address using Samza/Yarn/Kafka
>> >>> >> >> > > >>> >> >2. Collect alert and metric events which contain
>> >>>either
>> >>> >> >> hostname
>> >>> >> >> > > >>>or IP
>> >>> >> >> > > >>> >> >address
>> >>> >> >> > > >>> >> >3. Append contextual information to the alert and
>> >>>metric
>> >>> >>and
>> >>> >> >> > > >>>insert to
>> >>> >> >> > > >>> >>a
>> >>> >> >> > > >>> >> >Kafka queue from which other subscribers read off
>>of.
>> >>> >> >> > > >>> >> >
>> >>> >> >> > > >>> >> >Can you please shed some light on
>> >>> >> >> > > >>> >> >
>> >>> >> >> > > >>> >> >1. Is this feasible?
>> >>> >> >> > > >>> >> >2. Am I on the right thought process
>> >>> >> >> > > >>> >> >3. How do I start
>> >>> >> >> > > >>> >> >
>> >>> >> >> > > >>> >> >I now have 1 & 2 of them working disparately. I
>>need
>> >>>to
>> >>> >> >> integrate
>> >>> >> >> > > >>> them.
>> >>> >> >> > > >>> >> >
>> >>> >> >> > > >>> >> >Appreciate any input.
>> >>> >> >> > > >>> >> >
>> >>> >> >> > > >>> >> >- Shekar
>> >>> >> >> > > >>> >>
>> >>> >> >> > > >>> >>
>> >>> >> >> > > >>>
>> >>> >> >> > > >>>
>> >>> >> >> > > >>
>> >>> >> >> > >
>> >>> >> >> > >
>> >>> >> >> >
>> >>> >> >>
>> >>> >>
>> >>> >>
>> >>>
>> >>>
>> >>
>>
>>


Re: Samza as a Caching layer

Posted by Shekar Tippur <ct...@gmail.com>.
Chris ..

I am using a rhel server to host all the components (Yarn, kafka, samza). I
dont have ACLs open to wikipedia.
I am following ..
http://samza.incubator.apache.org/learn/tutorials/latest/run-hello-samza-without-internet.html

- Shekar



On Tue, Sep 2, 2014 at 2:13 PM, Chris Riccomini <
criccomini@linkedin.com.invalid> wrote:

> Hey Shekar,
>
> Can you try changing that to:
>
>   http://127.0.0.1:8088/cluster
>
>
> And see if you can connect?
>
> Cheers,
> Chris
>
> On 9/2/14 1:21 PM, "Shekar Tippur" <ct...@gmail.com> wrote:
>
> >Other observation is ..
> >
> >http://10.132.62.185:8088/cluster shows that no applications are running.
> >
> >- Shekar
> >
> >
> >
> >
> >On Tue, Sep 2, 2014 at 1:15 PM, Shekar Tippur <ct...@gmail.com> wrote:
> >
> >> Yarn seem to be running ..
> >>
> >> yarn      5462  0.0  2.0 1641296 161404 ?      Sl   Jun20  95:26
> >> /usr/java/jdk1.6.0_31/bin/java -Dproc_resourcemanager -Xmx1000m
> >> -Dhadoop.log.dir=/var/log/hadoop-yarn
> >>-Dyarn.log.dir=/var/log/hadoop-yarn
> >> -Dhadoop.log.file=yarn-yarn-resourcemanager-pppdc9prd2y2.log
> >> -Dyarn.log.file=yarn-yarn-resourcemanager-pppdc9prd2y2.log
> >> -Dyarn.home.dir=/usr/lib/hadoop-yarn
> >>-Dhadoop.home.dir=/usr/lib/hadoop-yarn
> >> -Dhadoop.root.logger=INFO,RFA -Dyarn.root.logger=INFO,RFA
> >> -Djava.library.path=/usr/lib/hadoop/lib/native -classpath
> >>
> >>/etc/hadoop/conf:/etc/hadoop/conf:/etc/hadoop/conf:/usr/lib/hadoop/lib/*:
> >>/usr/lib/hadoop/.//*:/usr/lib/hadoop-hdfs/./:/usr/lib/hadoop-hdfs/lib/*:/
> >>usr/lib/hadoop-hdfs/.//*:/usr/lib/hadoop-yarn/lib/*:/usr/lib/hadoop-yarn/
> >>.//*:/usr/lib/hadoop-mapreduce/lib/*:/usr/lib/hadoop-mapreduce/.//*:/usr/
> >>lib/hadoop-yarn/.//*:/usr/lib/hadoop-yarn/lib/*:/etc/hadoop/conf/rm-confi
> >>g/log4j.properties
> >> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager
> >>
> >> I can tail kafka topic as well ..
> >>
> >> deploy/kafka/bin/kafka-console-consumer.sh  --zookeeper localhost:2181
> >>--topic wikipedia-raw
> >>
> >>
> >>
> >>
> >> On Tue, Sep 2, 2014 at 1:04 PM, Chris Riccomini <
> >> criccomini@linkedin.com.invalid> wrote:
> >>
> >>> Hey Shekar,
> >>>
> >>> It looks like your job is hanging trying to connect to the RM on your
> >>> localhost. I thought that you said that your job was running in local
> >>> mode. If so, it should be using the LocalJobFactory. If not, and you
> >>> intend to run on YARN, is your YARN RM up and running on localhost?
> >>>
> >>> Cheers,
> >>> Chris
> >>>
> >>> On 9/2/14 12:22 PM, "Shekar Tippur" <ct...@gmail.com> wrote:
> >>>
> >>> >Chris ..
> >>> >
> >>> >$ cat ./deploy/samza/undefined-samza-container-name.log
> >>> >
> >>> >2014-09-02 11:17:58 JobRunner [INFO] job factory:
> >>> >org.apache.samza.job.yarn.YarnJobFactory
> >>> >
> >>> >2014-09-02 11:17:59 ClientHelper [INFO] trying to connect to RM
> >>> >127.0.0.1:8032
> >>> >
> >>> >2014-09-02 11:17:59 NativeCodeLoader [WARN] Unable to load
> >>>native-hadoop
> >>> >library for your platform... using builtin-java classes where
> >>>applicable
> >>> >
> >>> >2014-09-02 11:17:59 RMProxy [INFO] Connecting to ResourceManager at /
> >>> >127.0.0.1:8032
> >>> >
> >>> >
> >>> >and Log4j ..
> >>> >
> >>> ><!DOCTYPE log4j:configuration SYSTEM "log4j.dtd">
> >>> >
> >>> ><log4j:configuration xmlns:log4j="http://jakarta.apache.org/log4j/">
> >>> >
> >>> >  <appender name="RollingAppender"
> >>> >class="org.apache.log4j.DailyRollingFileAppender">
> >>> >
> >>> >     <param name="File"
> >>> >value="${samza.log.dir}/${samza.container.name}.log"
> >>> >/>
> >>> >
> >>> >     <param name="DatePattern" value="'.'yyyy-MM-dd" />
> >>> >
> >>> >     <layout class="org.apache.log4j.PatternLayout">
> >>> >
> >>> >      <param name="ConversionPattern" value="%d{yyyy-MM-dd HH:mm:ss}
> >>> %c{1}
> >>> >[%p] %m%n" />
> >>> >
> >>> >     </layout>
> >>> >
> >>> >  </appender>
> >>> >
> >>> >  <root>
> >>> >
> >>> >    <priority value="info" />
> >>> >
> >>> >    <appender-ref ref="RollingAppender"/>
> >>> >
> >>> >  </root>
> >>> >
> >>> ></log4j:configuration>
> >>> >
> >>> >
> >>> >On Tue, Sep 2, 2014 at 12:02 PM, Chris Riccomini <
> >>> >criccomini@linkedin.com.invalid> wrote:
> >>> >
> >>> >> Hey Shekar,
> >>> >>
> >>> >> Can you attach your log files? I'm wondering if it's a
> >>>mis-configured
> >>> >> log4j.xml (or missing slf4j-log4j jar), which is leading to nearly
> >>> empty
> >>> >> log files. Also, I'm wondering if the job starts fully. Anything you
> >>> can
> >>> >> attach would be helpful.
> >>> >>
> >>> >> Cheers,
> >>> >> Chris
> >>> >>
> >>> >> On 9/2/14 11:43 AM, "Shekar Tippur" <ct...@gmail.com> wrote:
> >>> >>
> >>> >> >I am running in local mode.
> >>> >> >
> >>> >> >S
> >>> >> >On Sep 2, 2014 11:42 AM, "Yan Fang" <ya...@gmail.com> wrote:
> >>> >> >
> >>> >> >> Hi Shekar.
> >>> >> >>
> >>> >> >> Are you running job in local mode or yarn mode? If yarn mode, the
> >>> log
> >>> >> >>is in
> >>> >> >> the yarn's container log.
> >>> >> >>
> >>> >> >> Thanks,
> >>> >> >>
> >>> >> >> Fang, Yan
> >>> >> >> yanfang724@gmail.com
> >>> >> >> +1 (206) 849-4108
> >>> >> >>
> >>> >> >>
> >>> >> >> On Tue, Sep 2, 2014 at 11:36 AM, Shekar Tippur
> >>><ct...@gmail.com>
> >>> >> >>wrote:
> >>> >> >>
> >>> >> >> > Chris,
> >>> >> >> >
> >>> >> >> > Got some time to play around a bit more.
> >>> >> >> > I tried to edit
> >>> >> >> >
> >>> >> >> >
> >>> >> >>
> >>> >>
> >>>
> >>>
> >>>>>>>samza-wikipedia/src/main/java/samza/examples/wikipedia/task/Wikipedi
> >>>>>>>aFe
> >>> >>>>ed
> >>> >> >>StreamTask.java
> >>> >> >> > to add a logger info statement to tap the incoming message. I
> >>>dont
> >>> >>see
> >>> >> >> the
> >>> >> >> > messages being printed to the log file.
> >>> >> >> >
> >>> >> >> > Is this the right place to start?
> >>> >> >> >
> >>> >> >> > public class WikipediaFeedStreamTask implements StreamTask {
> >>> >> >> >
> >>> >> >> >   private static final SystemStream OUTPUT_STREAM = new
> >>> >> >> > SystemStream("kafka",
> >>> >> >> > "wikipedia-raw");
> >>> >> >> >
> >>> >> >> >   private static final Logger log = LoggerFactory.getLogger
> >>> >> >> > (WikipediaFeedStreamTask.class);
> >>> >> >> >
> >>> >> >> >   @Override
> >>> >> >> >
> >>> >> >> >   public void process(IncomingMessageEnvelope envelope,
> >>> >> >>MessageCollector
> >>> >> >> > collector, TaskCoordinator coordinator) {
> >>> >> >> >
> >>> >> >> >     Map<String, Object> outgoingMap =
> >>> >> >> > WikipediaFeedEvent.toMap((WikipediaFeedEvent)
> >>> >>envelope.getMessage());
> >>> >> >> >
> >>> >> >> >     log.info(envelope.getMessage().toString());
> >>> >> >> >
> >>> >> >> >     collector.send(new OutgoingMessageEnvelope(OUTPUT_STREAM,
> >>> >> >> > outgoingMap));
> >>> >> >> >
> >>> >> >> >   }
> >>> >> >> >
> >>> >> >> > }
> >>> >> >> >
> >>> >> >> >
> >>> >> >> > On Mon, Aug 25, 2014 at 9:01 AM, Chris Riccomini <
> >>> >> >> > criccomini@linkedin.com.invalid> wrote:
> >>> >> >> >
> >>> >> >> > > Hey Shekar,
> >>> >> >> > >
> >>> >> >> > > Your thought process is on the right track. It's probably
> >>>best
> >>> to
> >>> >> >>start
> >>> >> >> > > with hello-samza, and modify it to get what you want. To
> >>>start
> >>> >>with,
> >>> >> >> > > you'll want to:
> >>> >> >> > >
> >>> >> >> > > 1. Write a simple StreamTask that just does something silly
> >>>like
> >>> >> >>just
> >>> >> >> > > print messages that it receives.
> >>> >> >> > > 2. Write a configuration for the job that consumes from just
> >>>the
> >>> >> >>stream
> >>> >> >> > > (alerts from different sources).
> >>> >> >> > > 3. Run this to make sure you've got it working.
> >>> >> >> > > 4. Now add your table join. This can be either a change-data
> >>> >>capture
> >>> >> >> > (CDC)
> >>> >> >> > > stream, or via a remote DB call.
> >>> >> >> > >
> >>> >> >> > > That should get you to a point where you've got your job up
> >>>and
> >>> >> >> running.
> >>> >> >> > > From there, you could create your own Maven project, and
> >>>migrate
> >>> >> >>your
> >>> >> >> > code
> >>> >> >> > > over accordingly.
> >>> >> >> > >
> >>> >> >> > > Cheers,
> >>> >> >> > > Chris
> >>> >> >> > >
> >>> >> >> > > On 8/24/14 1:42 AM, "Shekar Tippur" <ct...@gmail.com>
> >>>wrote:
> >>> >> >> > >
> >>> >> >> > > >Chris,
> >>> >> >> > > >
> >>> >> >> > > >I have gone thro the documentation and decided that the
> >>>option
> >>> >> >>that is
> >>> >> >> > > >most
> >>> >> >> > > >suitable for me is stream-table.
> >>> >> >> > > >
> >>> >> >> > > >I see the following things:
> >>> >> >> > > >
> >>> >> >> > > >1. Point samza to a table (database)
> >>> >> >> > > >2. Point Samza to a stream - Alert stream from different
> >>> sources
> >>> >> >> > > >3. Join key like a hostname
> >>> >> >> > > >
> >>> >> >> > > >I have Hello Samza working. To extend that to do what my
> >>>needs
> >>> >> >>are, I
> >>> >> >> am
> >>> >> >> > > >not sure where to start (Needs more code change OR
> >>> configuration
> >>> >> >> changes
> >>> >> >> > > >OR
> >>> >> >> > > >both)?
> >>> >> >> > > >
> >>> >> >> > > >I have gone thro
> >>> >> >> > > >
> >>> >> >> >
> >>> >> >>
> >>> >> >>
> >>> >>
> >>> >>
> >>>
> >>>
> http://samza.incubator.apache.org/learn/documentation/latest/api/overvie
> >>>w
> >>> >> >> > > .
> >>> >> >> > > >html
> >>> >> >> > > >
> >>> >> >> > > >Is my thought process on the right track? Can you please
> >>>point
> >>> >>me
> >>> >> >>to
> >>> >> >> the
> >>> >> >> > > >right direction?
> >>> >> >> > > >
> >>> >> >> > > >- Shekar
> >>> >> >> > > >
> >>> >> >> > > >
> >>> >> >> > > >
> >>> >> >> > > >On Thu, Aug 21, 2014 at 1:08 PM, Shekar Tippur
> >>> >><ct...@gmail.com>
> >>> >> >> > wrote:
> >>> >> >> > > >
> >>> >> >> > > >> Chris,
> >>> >> >> > > >>
> >>> >> >> > > >> This is perfectly good answer. I will start poking more
> >>>into
> >>> >> >>option
> >>> >> >> > #4.
> >>> >> >> > > >>
> >>> >> >> > > >> - Shekar
> >>> >> >> > > >>
> >>> >> >> > > >>
> >>> >> >> > > >> On Thu, Aug 21, 2014 at 1:05 PM, Chris Riccomini <
> >>> >> >> > > >> criccomini@linkedin.com.invalid> wrote:
> >>> >> >> > > >>
> >>> >> >> > > >>> Hey Shekar,
> >>> >> >> > > >>>
> >>> >> >> > > >>> Your two options are really (3) or (4), then. You can
> >>>either
> >>> >>run
> >>> >> >> some
> >>> >> >> > > >>> external DB that holds the data set, and you can query it
> >>> >>from a
> >>> >> >> > > >>> StreamTask, or you can use Samza's state store feature to
> >>> >>push
> >>> >> >>data
> >>> >> >> > > >>>into a
> >>> >> >> > > >>> stream that you can then store in a partitioned key-value
> >>> >>store
> >>> >> >> along
> >>> >> >> > > >>>with
> >>> >> >> > > >>> your StreamTasks. There is some documentation here about
> >>>the
> >>> >> >>state
> >>> >> >> > > >>>store
> >>> >> >> > > >>> approach:
> >>> >> >> > > >>>
> >>> >> >> > > >>>
> >>> >> >> > > >>>
> >>> >> >> > > >>>
> >>> >> >> > >
> >>> >> >>
> >>> >>
> >>>
> >>>
> http://samza.incubator.apache.org/learn/documentation/0.7.0/container/st
> >>> >> >> > > >>>ate
> >>> >> >> > > >>> -management.html
> >>> >> >> > > >>>
> >>> >> >> > > >>><
> >>> >> >> > >
> >>> >> >>
> >>>
> >>>>>
> http://samza.incubator.apache.org/learn/documentation/0.7.0/container/
> >>>>>s
> >>> >> >> > > >>>tate-management.html>
> >>> >> >> > > >>>
> >>> >> >> > > >>>
> >>> >> >> > > >>> (4) is going to require more up front effort from you,
> >>>since
> >>> >> >>you'll
> >>> >> >> > > >>>have
> >>> >> >> > > >>> to understand how Kafka's partitioning model works, and
> >>> setup
> >>> >> >>some
> >>> >> >> > > >>> pipeline to push the updates for your state. In the long
> >>> >>run, I
> >>> >> >> > believe
> >>> >> >> > > >>> it's the better approach, though. Local lookups on a
> >>> >>key-value
> >>> >> >> store
> >>> >> >> > > >>> should be faster than doing remote RPC calls to a DB for
> >>> >>every
> >>> >> >> > message.
> >>> >> >> > > >>>
> >>> >> >> > > >>> I'm sorry I can't give you a more definitive answer. It's
> >>> >>really
> >>> >> >> > about
> >>> >> >> > > >>> trade-offs.
> >>> >> >> > > >>>
> >>> >> >> > > >>> Cheers,
> >>> >> >> > > >>> Chris
> >>> >> >> > > >>>
> >>> >> >> > > >>> On 8/21/14 12:22 PM, "Shekar Tippur" <ct...@gmail.com>
> >>> >>wrote:
> >>> >> >> > > >>>
> >>> >> >> > > >>> >Chris,
> >>> >> >> > > >>> >
> >>> >> >> > > >>> >A big thanks for a swift response. The data set is huge
> >>>and
> >>> >>the
> >>> >> >> > > >>>frequency
> >>> >> >> > > >>> >is in burst.
> >>> >> >> > > >>> >What do you suggest?
> >>> >> >> > > >>> >
> >>> >> >> > > >>> >- Shekar
> >>> >> >> > > >>> >
> >>> >> >> > > >>> >
> >>> >> >> > > >>> >On Thu, Aug 21, 2014 at 11:05 AM, Chris Riccomini <
> >>> >> >> > > >>> >criccomini@linkedin.com.invalid> wrote:
> >>> >> >> > > >>> >
> >>> >> >> > > >>> >> Hey Shekar,
> >>> >> >> > > >>> >>
> >>> >> >> > > >>> >> This is feasible, and you are on the right thought
> >>> >>process.
> >>> >> >> > > >>> >>
> >>> >> >> > > >>> >> For the sake of discussion, I'm going to pretend that
> >>>you
> >>> >> >>have a
> >>> >> >> > > >>>Kafka
> >>> >> >> > > >>> >> topic called "PageViewEvent", which has just the IP
> >>> >>address
> >>> >> >>that
> >>> >> >> > was
> >>> >> >> > > >>> >>used
> >>> >> >> > > >>> >> to view a page. These messages will be logged every
> >>>time
> >>> a
> >>> >> >>page
> >>> >> >> > view
> >>> >> >> > > >>> >> happens. I'm also going to pretend that you have some
> >>> >>state
> >>> >> >> called
> >>> >> >> > > >>> >>"IPGeo"
> >>> >> >> > > >>> >> (e.g. The maxmind data set). In this example, we'll
> >>>want
> >>> >>to
> >>> >> >>join
> >>> >> >> > the
> >>> >> >> > > >>> >> long/lat geo information from IPGeo to the
> >>>PageViewEvent,
> >>> >>and
> >>> >> >> send
> >>> >> >> > > >>>it
> >>> >> >> > > >>> >>to a
> >>> >> >> > > >>> >> new topic: PageViewEventsWithGeo.
> >>> >> >> > > >>> >>
> >>> >> >> > > >>> >> You have several options on how to implement this
> >>> example.
> >>> >> >> > > >>> >>
> >>> >> >> > > >>> >> 1. If your joining data set (IPGeo) is relatively
> >>>small
> >>> >>and
> >>> >> >> > changes
> >>> >> >> > > >>> >> infrequently, you can just pack it up in your jar or
> >>>.tgz
> >>> >> >>file,
> >>> >> >> > and
> >>> >> >> > > >>> open
> >>> >> >> > > >>> >> it open in every StreamTask.
> >>> >> >> > > >>> >> 2. If your data set is small, but changes somewhat
> >>> >> >>frequently,
> >>> >> >> you
> >>> >> >> > > >>>can
> >>> >> >> > > >>> >> throw the data set on some HTTP/HDFS/S3 server
> >>>somewhere,
> >>> >>and
> >>> >> >> have
> >>> >> >> > > >>>your
> >>> >> >> > > >>> >> StreamTask refresh it periodically by re-downloading
> >>>it.
> >>> >> >> > > >>> >> 3. You can do remote RPC calls for the IPGeo data on
> >>> every
> >>> >> >>page
> >>> >> >> > view
> >>> >> >> > > >>> >>event
> >>> >> >> > > >>> >> by query some remote service or DB (e.g. Cassandra).
> >>> >> >> > > >>> >> 4. You can use Samza's state feature to set your IPGeo
> >>> >>data
> >>> >> >>as a
> >>> >> >> > > >>>series
> >>> >> >> > > >>> >>of
> >>> >> >> > > >>> >> messages to a log-compacted Kafka topic
> >>> >> >> > > >>> >> (
> >>> >> >> https://cwiki.apache.org/confluence/display/KAFKA/Log+Compaction
> >>> >> >> > ),
> >>> >> >> > > >>> and
> >>> >> >> > > >>> >> configure your Samza job to read this topic as a
> >>> bootstrap
> >>> >> >> stream
> >>> >> >> > > >>> >> (
> >>> >> >> > > >>> >>
> >>> >> >> > > >>> >>
> >>> >> >> > > >>>
> >>> >> >> > > >>>
> >>> >> >> > >
> >>> >> >>
> >>> >>
> >>>
> >>>
> http://samza.incubator.apache.org/learn/documentation/0.7.0/container/st
> >>> >> >> > > >>>r
> >>> >> >> > > >>> >>e
> >>> >> >> > > >>> >> ams.html).
> >>> >> >> > > >>> >>
> >>> >> >> > > >>> >> For (4), you'd have to partition the IPGeo state topic
> >>> >> >>according
> >>> >> >> > to
> >>> >> >> > > >>>the
> >>> >> >> > > >>> >> same key as PageViewEvent. If PageViewEvent were
> >>> >>partitioned
> >>> >> >>by,
> >>> >> >> > > >>>say,
> >>> >> >> > > >>> >> member ID, but you want your IPGeo state topic to be
> >>> >> >>partitioned
> >>> >> >> > by
> >>> >> >> > > >>>IP
> >>> >> >> > > >>> >> address, then you'd have to have an upstream job that
> >>> >> >> > re-partitioned
> >>> >> >> > > >>> >> PageViewEvent into some new topic by IP address. This
> >>>new
> >>> >> >>topic
> >>> >> >> > will
> >>> >> >> > > >>> >>have
> >>> >> >> > > >>> >> to have the same number of partitions as the IPGeo
> >>>state
> >>> >> >>topic
> >>> >> >> (if
> >>> >> >> > > >>> IPGeo
> >>> >> >> > > >>> >> has 8 partitions, then the new
> >>>PageViewEventRepartitioned
> >>> >> >>topic
> >>> >> >> > > >>>needs 8
> >>> >> >> > > >>> >>as
> >>> >> >> > > >>> >> well). This will cause your PageViewEventRepartitioned
> >>> >>topic
> >>> >> >>and
> >>> >> >> > > >>>your
> >>> >> >> > > >>> >> IPGeo state topic to be aligned such that the
> >>>StreamTask
> >>> >>that
> >>> >> >> gets
> >>> >> >> > > >>>page
> >>> >> >> > > >>> >> views for IP address X will also have the IPGeo
> >>> >>information
> >>> >> >>for
> >>> >> >> IP
> >>> >> >> > > >>> >>address
> >>> >> >> > > >>> >> X.
> >>> >> >> > > >>> >>
> >>> >> >> > > >>> >> Which strategy you pick is really up to you. :) (4) is
> >>> the
> >>> >> >>most
> >>> >> >> > > >>> >> complicated, but also the most flexible, and most
> >>> >> >>operationally
> >>> >> >> > > >>>sound.
> >>> >> >> > > >>> >>(1)
> >>> >> >> > > >>> >> is the easiest if it fits your needs.
> >>> >> >> > > >>> >>
> >>> >> >> > > >>> >> Cheers,
> >>> >> >> > > >>> >> Chris
> >>> >> >> > > >>> >>
> >>> >> >> > > >>> >> On 8/21/14 10:15 AM, "Shekar Tippur"
> >>><ct...@gmail.com>
> >>> >> >>wrote:
> >>> >> >> > > >>> >>
> >>> >> >> > > >>> >> >Hello,
> >>> >> >> > > >>> >> >
> >>> >> >> > > >>> >> >I am new to Samza. I have just installed Hello Samza
> >>>and
> >>> >> >>got it
> >>> >> >> > > >>> >>working.
> >>> >> >> > > >>> >> >
> >>> >> >> > > >>> >> >Here is the use case for which I am trying to use
> >>>Samza:
> >>> >> >> > > >>> >> >
> >>> >> >> > > >>> >> >
> >>> >> >> > > >>> >> >1. Cache the contextual information which contains
> >>>more
> >>> >> >> > information
> >>> >> >> > > >>> >>about
> >>> >> >> > > >>> >> >the hostname or IP address using Samza/Yarn/Kafka
> >>> >> >> > > >>> >> >2. Collect alert and metric events which contain
> >>>either
> >>> >> >> hostname
> >>> >> >> > > >>>or IP
> >>> >> >> > > >>> >> >address
> >>> >> >> > > >>> >> >3. Append contextual information to the alert and
> >>>metric
> >>> >>and
> >>> >> >> > > >>>insert to
> >>> >> >> > > >>> >>a
> >>> >> >> > > >>> >> >Kafka queue from which other subscribers read off of.
> >>> >> >> > > >>> >> >
> >>> >> >> > > >>> >> >Can you please shed some light on
> >>> >> >> > > >>> >> >
> >>> >> >> > > >>> >> >1. Is this feasible?
> >>> >> >> > > >>> >> >2. Am I on the right thought process
> >>> >> >> > > >>> >> >3. How do I start
> >>> >> >> > > >>> >> >
> >>> >> >> > > >>> >> >I now have 1 & 2 of them working disparately. I need
> >>>to
> >>> >> >> integrate
> >>> >> >> > > >>> them.
> >>> >> >> > > >>> >> >
> >>> >> >> > > >>> >> >Appreciate any input.
> >>> >> >> > > >>> >> >
> >>> >> >> > > >>> >> >- Shekar
> >>> >> >> > > >>> >>
> >>> >> >> > > >>> >>
> >>> >> >> > > >>>
> >>> >> >> > > >>>
> >>> >> >> > > >>
> >>> >> >> > >
> >>> >> >> > >
> >>> >> >> >
> >>> >> >>
> >>> >>
> >>> >>
> >>>
> >>>
> >>
>
>

Re: Samza as a Caching layer

Posted by Chris Riccomini <cr...@linkedin.com.INVALID>.
Hey Shekar,

Can you try changing that to:

  http://127.0.0.1:8088/cluster


And see if you can connect?

Cheers,
Chris

On 9/2/14 1:21 PM, "Shekar Tippur" <ct...@gmail.com> wrote:

>Other observation is ..
>
>http://10.132.62.185:8088/cluster shows that no applications are running.
>
>- Shekar
>
>
>
>
>On Tue, Sep 2, 2014 at 1:15 PM, Shekar Tippur <ct...@gmail.com> wrote:
>
>> Yarn seem to be running ..
>>
>> yarn      5462  0.0  2.0 1641296 161404 ?      Sl   Jun20  95:26
>> /usr/java/jdk1.6.0_31/bin/java -Dproc_resourcemanager -Xmx1000m
>> -Dhadoop.log.dir=/var/log/hadoop-yarn
>>-Dyarn.log.dir=/var/log/hadoop-yarn
>> -Dhadoop.log.file=yarn-yarn-resourcemanager-pppdc9prd2y2.log
>> -Dyarn.log.file=yarn-yarn-resourcemanager-pppdc9prd2y2.log
>> -Dyarn.home.dir=/usr/lib/hadoop-yarn
>>-Dhadoop.home.dir=/usr/lib/hadoop-yarn
>> -Dhadoop.root.logger=INFO,RFA -Dyarn.root.logger=INFO,RFA
>> -Djava.library.path=/usr/lib/hadoop/lib/native -classpath
>> 
>>/etc/hadoop/conf:/etc/hadoop/conf:/etc/hadoop/conf:/usr/lib/hadoop/lib/*:
>>/usr/lib/hadoop/.//*:/usr/lib/hadoop-hdfs/./:/usr/lib/hadoop-hdfs/lib/*:/
>>usr/lib/hadoop-hdfs/.//*:/usr/lib/hadoop-yarn/lib/*:/usr/lib/hadoop-yarn/
>>.//*:/usr/lib/hadoop-mapreduce/lib/*:/usr/lib/hadoop-mapreduce/.//*:/usr/
>>lib/hadoop-yarn/.//*:/usr/lib/hadoop-yarn/lib/*:/etc/hadoop/conf/rm-confi
>>g/log4j.properties
>> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager
>>
>> I can tail kafka topic as well ..
>>
>> deploy/kafka/bin/kafka-console-consumer.sh  --zookeeper localhost:2181
>>--topic wikipedia-raw
>>
>>
>>
>>
>> On Tue, Sep 2, 2014 at 1:04 PM, Chris Riccomini <
>> criccomini@linkedin.com.invalid> wrote:
>>
>>> Hey Shekar,
>>>
>>> It looks like your job is hanging trying to connect to the RM on your
>>> localhost. I thought that you said that your job was running in local
>>> mode. If so, it should be using the LocalJobFactory. If not, and you
>>> intend to run on YARN, is your YARN RM up and running on localhost?
>>>
>>> Cheers,
>>> Chris
>>>
>>> On 9/2/14 12:22 PM, "Shekar Tippur" <ct...@gmail.com> wrote:
>>>
>>> >Chris ..
>>> >
>>> >$ cat ./deploy/samza/undefined-samza-container-name.log
>>> >
>>> >2014-09-02 11:17:58 JobRunner [INFO] job factory:
>>> >org.apache.samza.job.yarn.YarnJobFactory
>>> >
>>> >2014-09-02 11:17:59 ClientHelper [INFO] trying to connect to RM
>>> >127.0.0.1:8032
>>> >
>>> >2014-09-02 11:17:59 NativeCodeLoader [WARN] Unable to load
>>>native-hadoop
>>> >library for your platform... using builtin-java classes where
>>>applicable
>>> >
>>> >2014-09-02 11:17:59 RMProxy [INFO] Connecting to ResourceManager at /
>>> >127.0.0.1:8032
>>> >
>>> >
>>> >and Log4j ..
>>> >
>>> ><!DOCTYPE log4j:configuration SYSTEM "log4j.dtd">
>>> >
>>> ><log4j:configuration xmlns:log4j="http://jakarta.apache.org/log4j/">
>>> >
>>> >  <appender name="RollingAppender"
>>> >class="org.apache.log4j.DailyRollingFileAppender">
>>> >
>>> >     <param name="File"
>>> >value="${samza.log.dir}/${samza.container.name}.log"
>>> >/>
>>> >
>>> >     <param name="DatePattern" value="'.'yyyy-MM-dd" />
>>> >
>>> >     <layout class="org.apache.log4j.PatternLayout">
>>> >
>>> >      <param name="ConversionPattern" value="%d{yyyy-MM-dd HH:mm:ss}
>>> %c{1}
>>> >[%p] %m%n" />
>>> >
>>> >     </layout>
>>> >
>>> >  </appender>
>>> >
>>> >  <root>
>>> >
>>> >    <priority value="info" />
>>> >
>>> >    <appender-ref ref="RollingAppender"/>
>>> >
>>> >  </root>
>>> >
>>> ></log4j:configuration>
>>> >
>>> >
>>> >On Tue, Sep 2, 2014 at 12:02 PM, Chris Riccomini <
>>> >criccomini@linkedin.com.invalid> wrote:
>>> >
>>> >> Hey Shekar,
>>> >>
>>> >> Can you attach your log files? I'm wondering if it's a
>>>mis-configured
>>> >> log4j.xml (or missing slf4j-log4j jar), which is leading to nearly
>>> empty
>>> >> log files. Also, I'm wondering if the job starts fully. Anything you
>>> can
>>> >> attach would be helpful.
>>> >>
>>> >> Cheers,
>>> >> Chris
>>> >>
>>> >> On 9/2/14 11:43 AM, "Shekar Tippur" <ct...@gmail.com> wrote:
>>> >>
>>> >> >I am running in local mode.
>>> >> >
>>> >> >S
>>> >> >On Sep 2, 2014 11:42 AM, "Yan Fang" <ya...@gmail.com> wrote:
>>> >> >
>>> >> >> Hi Shekar.
>>> >> >>
>>> >> >> Are you running job in local mode or yarn mode? If yarn mode, the
>>> log
>>> >> >>is in
>>> >> >> the yarn's container log.
>>> >> >>
>>> >> >> Thanks,
>>> >> >>
>>> >> >> Fang, Yan
>>> >> >> yanfang724@gmail.com
>>> >> >> +1 (206) 849-4108
>>> >> >>
>>> >> >>
>>> >> >> On Tue, Sep 2, 2014 at 11:36 AM, Shekar Tippur
>>><ct...@gmail.com>
>>> >> >>wrote:
>>> >> >>
>>> >> >> > Chris,
>>> >> >> >
>>> >> >> > Got some time to play around a bit more.
>>> >> >> > I tried to edit
>>> >> >> >
>>> >> >> >
>>> >> >>
>>> >>
>>>
>>> 
>>>>>>>samza-wikipedia/src/main/java/samza/examples/wikipedia/task/Wikipedi
>>>>>>>aFe
>>> >>>>ed
>>> >> >>StreamTask.java
>>> >> >> > to add a logger info statement to tap the incoming message. I
>>>dont
>>> >>see
>>> >> >> the
>>> >> >> > messages being printed to the log file.
>>> >> >> >
>>> >> >> > Is this the right place to start?
>>> >> >> >
>>> >> >> > public class WikipediaFeedStreamTask implements StreamTask {
>>> >> >> >
>>> >> >> >   private static final SystemStream OUTPUT_STREAM = new
>>> >> >> > SystemStream("kafka",
>>> >> >> > "wikipedia-raw");
>>> >> >> >
>>> >> >> >   private static final Logger log = LoggerFactory.getLogger
>>> >> >> > (WikipediaFeedStreamTask.class);
>>> >> >> >
>>> >> >> >   @Override
>>> >> >> >
>>> >> >> >   public void process(IncomingMessageEnvelope envelope,
>>> >> >>MessageCollector
>>> >> >> > collector, TaskCoordinator coordinator) {
>>> >> >> >
>>> >> >> >     Map<String, Object> outgoingMap =
>>> >> >> > WikipediaFeedEvent.toMap((WikipediaFeedEvent)
>>> >>envelope.getMessage());
>>> >> >> >
>>> >> >> >     log.info(envelope.getMessage().toString());
>>> >> >> >
>>> >> >> >     collector.send(new OutgoingMessageEnvelope(OUTPUT_STREAM,
>>> >> >> > outgoingMap));
>>> >> >> >
>>> >> >> >   }
>>> >> >> >
>>> >> >> > }
>>> >> >> >
>>> >> >> >
>>> >> >> > On Mon, Aug 25, 2014 at 9:01 AM, Chris Riccomini <
>>> >> >> > criccomini@linkedin.com.invalid> wrote:
>>> >> >> >
>>> >> >> > > Hey Shekar,
>>> >> >> > >
>>> >> >> > > Your thought process is on the right track. It's probably
>>>best
>>> to
>>> >> >>start
>>> >> >> > > with hello-samza, and modify it to get what you want. To
>>>start
>>> >>with,
>>> >> >> > > you'll want to:
>>> >> >> > >
>>> >> >> > > 1. Write a simple StreamTask that just does something silly
>>>like
>>> >> >>just
>>> >> >> > > print messages that it receives.
>>> >> >> > > 2. Write a configuration for the job that consumes from just
>>>the
>>> >> >>stream
>>> >> >> > > (alerts from different sources).
>>> >> >> > > 3. Run this to make sure you've got it working.
>>> >> >> > > 4. Now add your table join. This can be either a change-data
>>> >>capture
>>> >> >> > (CDC)
>>> >> >> > > stream, or via a remote DB call.
>>> >> >> > >
>>> >> >> > > That should get you to a point where you've got your job up
>>>and
>>> >> >> running.
>>> >> >> > > From there, you could create your own Maven project, and
>>>migrate
>>> >> >>your
>>> >> >> > code
>>> >> >> > > over accordingly.
>>> >> >> > >
>>> >> >> > > Cheers,
>>> >> >> > > Chris
>>> >> >> > >
>>> >> >> > > On 8/24/14 1:42 AM, "Shekar Tippur" <ct...@gmail.com>
>>>wrote:
>>> >> >> > >
>>> >> >> > > >Chris,
>>> >> >> > > >
>>> >> >> > > >I have gone thro the documentation and decided that the
>>>option
>>> >> >>that is
>>> >> >> > > >most
>>> >> >> > > >suitable for me is stream-table.
>>> >> >> > > >
>>> >> >> > > >I see the following things:
>>> >> >> > > >
>>> >> >> > > >1. Point samza to a table (database)
>>> >> >> > > >2. Point Samza to a stream - Alert stream from different
>>> sources
>>> >> >> > > >3. Join key like a hostname
>>> >> >> > > >
>>> >> >> > > >I have Hello Samza working. To extend that to do what my
>>>needs
>>> >> >>are, I
>>> >> >> am
>>> >> >> > > >not sure where to start (Needs more code change OR
>>> configuration
>>> >> >> changes
>>> >> >> > > >OR
>>> >> >> > > >both)?
>>> >> >> > > >
>>> >> >> > > >I have gone thro
>>> >> >> > > >
>>> >> >> >
>>> >> >>
>>> >> >>
>>> >>
>>> >>
>>> 
>>>http://samza.incubator.apache.org/learn/documentation/latest/api/overvie
>>>w
>>> >> >> > > .
>>> >> >> > > >html
>>> >> >> > > >
>>> >> >> > > >Is my thought process on the right track? Can you please
>>>point
>>> >>me
>>> >> >>to
>>> >> >> the
>>> >> >> > > >right direction?
>>> >> >> > > >
>>> >> >> > > >- Shekar
>>> >> >> > > >
>>> >> >> > > >
>>> >> >> > > >
>>> >> >> > > >On Thu, Aug 21, 2014 at 1:08 PM, Shekar Tippur
>>> >><ct...@gmail.com>
>>> >> >> > wrote:
>>> >> >> > > >
>>> >> >> > > >> Chris,
>>> >> >> > > >>
>>> >> >> > > >> This is perfectly good answer. I will start poking more
>>>into
>>> >> >>option
>>> >> >> > #4.
>>> >> >> > > >>
>>> >> >> > > >> - Shekar
>>> >> >> > > >>
>>> >> >> > > >>
>>> >> >> > > >> On Thu, Aug 21, 2014 at 1:05 PM, Chris Riccomini <
>>> >> >> > > >> criccomini@linkedin.com.invalid> wrote:
>>> >> >> > > >>
>>> >> >> > > >>> Hey Shekar,
>>> >> >> > > >>>
>>> >> >> > > >>> Your two options are really (3) or (4), then. You can
>>>either
>>> >>run
>>> >> >> some
>>> >> >> > > >>> external DB that holds the data set, and you can query it
>>> >>from a
>>> >> >> > > >>> StreamTask, or you can use Samza's state store feature to
>>> >>push
>>> >> >>data
>>> >> >> > > >>>into a
>>> >> >> > > >>> stream that you can then store in a partitioned key-value
>>> >>store
>>> >> >> along
>>> >> >> > > >>>with
>>> >> >> > > >>> your StreamTasks. There is some documentation here about
>>>the
>>> >> >>state
>>> >> >> > > >>>store
>>> >> >> > > >>> approach:
>>> >> >> > > >>>
>>> >> >> > > >>>
>>> >> >> > > >>>
>>> >> >> > > >>>
>>> >> >> > >
>>> >> >>
>>> >>
>>> 
>>>http://samza.incubator.apache.org/learn/documentation/0.7.0/container/st
>>> >> >> > > >>>ate
>>> >> >> > > >>> -management.html
>>> >> >> > > >>>
>>> >> >> > > >>><
>>> >> >> > >
>>> >> >>
>>> 
>>>>>http://samza.incubator.apache.org/learn/documentation/0.7.0/container/
>>>>>s
>>> >> >> > > >>>tate-management.html>
>>> >> >> > > >>>
>>> >> >> > > >>>
>>> >> >> > > >>> (4) is going to require more up front effort from you,
>>>since
>>> >> >>you'll
>>> >> >> > > >>>have
>>> >> >> > > >>> to understand how Kafka's partitioning model works, and
>>> setup
>>> >> >>some
>>> >> >> > > >>> pipeline to push the updates for your state. In the long
>>> >>run, I
>>> >> >> > believe
>>> >> >> > > >>> it's the better approach, though. Local lookups on a
>>> >>key-value
>>> >> >> store
>>> >> >> > > >>> should be faster than doing remote RPC calls to a DB for
>>> >>every
>>> >> >> > message.
>>> >> >> > > >>>
>>> >> >> > > >>> I'm sorry I can't give you a more definitive answer. It's
>>> >>really
>>> >> >> > about
>>> >> >> > > >>> trade-offs.
>>> >> >> > > >>>
>>> >> >> > > >>> Cheers,
>>> >> >> > > >>> Chris
>>> >> >> > > >>>
>>> >> >> > > >>> On 8/21/14 12:22 PM, "Shekar Tippur" <ct...@gmail.com>
>>> >>wrote:
>>> >> >> > > >>>
>>> >> >> > > >>> >Chris,
>>> >> >> > > >>> >
>>> >> >> > > >>> >A big thanks for a swift response. The data set is huge
>>>and
>>> >>the
>>> >> >> > > >>>frequency
>>> >> >> > > >>> >is in burst.
>>> >> >> > > >>> >What do you suggest?
>>> >> >> > > >>> >
>>> >> >> > > >>> >- Shekar
>>> >> >> > > >>> >
>>> >> >> > > >>> >
>>> >> >> > > >>> >On Thu, Aug 21, 2014 at 11:05 AM, Chris Riccomini <
>>> >> >> > > >>> >criccomini@linkedin.com.invalid> wrote:
>>> >> >> > > >>> >
>>> >> >> > > >>> >> Hey Shekar,
>>> >> >> > > >>> >>
>>> >> >> > > >>> >> This is feasible, and you are on the right thought
>>> >>process.
>>> >> >> > > >>> >>
>>> >> >> > > >>> >> For the sake of discussion, I'm going to pretend that
>>>you
>>> >> >>have a
>>> >> >> > > >>>Kafka
>>> >> >> > > >>> >> topic called "PageViewEvent", which has just the IP
>>> >>address
>>> >> >>that
>>> >> >> > was
>>> >> >> > > >>> >>used
>>> >> >> > > >>> >> to view a page. These messages will be logged every
>>>time
>>> a
>>> >> >>page
>>> >> >> > view
>>> >> >> > > >>> >> happens. I'm also going to pretend that you have some
>>> >>state
>>> >> >> called
>>> >> >> > > >>> >>"IPGeo"
>>> >> >> > > >>> >> (e.g. The maxmind data set). In this example, we'll
>>>want
>>> >>to
>>> >> >>join
>>> >> >> > the
>>> >> >> > > >>> >> long/lat geo information from IPGeo to the
>>>PageViewEvent,
>>> >>and
>>> >> >> send
>>> >> >> > > >>>it
>>> >> >> > > >>> >>to a
>>> >> >> > > >>> >> new topic: PageViewEventsWithGeo.
>>> >> >> > > >>> >>
>>> >> >> > > >>> >> You have several options on how to implement this
>>> example.
>>> >> >> > > >>> >>
>>> >> >> > > >>> >> 1. If your joining data set (IPGeo) is relatively
>>>small
>>> >>and
>>> >> >> > changes
>>> >> >> > > >>> >> infrequently, you can just pack it up in your jar or
>>>.tgz
>>> >> >>file,
>>> >> >> > and
>>> >> >> > > >>> open
>>> >> >> > > >>> >> it open in every StreamTask.
>>> >> >> > > >>> >> 2. If your data set is small, but changes somewhat
>>> >> >>frequently,
>>> >> >> you
>>> >> >> > > >>>can
>>> >> >> > > >>> >> throw the data set on some HTTP/HDFS/S3 server
>>>somewhere,
>>> >>and
>>> >> >> have
>>> >> >> > > >>>your
>>> >> >> > > >>> >> StreamTask refresh it periodically by re-downloading
>>>it.
>>> >> >> > > >>> >> 3. You can do remote RPC calls for the IPGeo data on
>>> every
>>> >> >>page
>>> >> >> > view
>>> >> >> > > >>> >>event
>>> >> >> > > >>> >> by query some remote service or DB (e.g. Cassandra).
>>> >> >> > > >>> >> 4. You can use Samza's state feature to set your IPGeo
>>> >>data
>>> >> >>as a
>>> >> >> > > >>>series
>>> >> >> > > >>> >>of
>>> >> >> > > >>> >> messages to a log-compacted Kafka topic
>>> >> >> > > >>> >> (
>>> >> >> https://cwiki.apache.org/confluence/display/KAFKA/Log+Compaction
>>> >> >> > ),
>>> >> >> > > >>> and
>>> >> >> > > >>> >> configure your Samza job to read this topic as a
>>> bootstrap
>>> >> >> stream
>>> >> >> > > >>> >> (
>>> >> >> > > >>> >>
>>> >> >> > > >>> >>
>>> >> >> > > >>>
>>> >> >> > > >>>
>>> >> >> > >
>>> >> >>
>>> >>
>>> 
>>>http://samza.incubator.apache.org/learn/documentation/0.7.0/container/st
>>> >> >> > > >>>r
>>> >> >> > > >>> >>e
>>> >> >> > > >>> >> ams.html).
>>> >> >> > > >>> >>
>>> >> >> > > >>> >> For (4), you'd have to partition the IPGeo state topic
>>> >> >>according
>>> >> >> > to
>>> >> >> > > >>>the
>>> >> >> > > >>> >> same key as PageViewEvent. If PageViewEvent were
>>> >>partitioned
>>> >> >>by,
>>> >> >> > > >>>say,
>>> >> >> > > >>> >> member ID, but you want your IPGeo state topic to be
>>> >> >>partitioned
>>> >> >> > by
>>> >> >> > > >>>IP
>>> >> >> > > >>> >> address, then you'd have to have an upstream job that
>>> >> >> > re-partitioned
>>> >> >> > > >>> >> PageViewEvent into some new topic by IP address. This
>>>new
>>> >> >>topic
>>> >> >> > will
>>> >> >> > > >>> >>have
>>> >> >> > > >>> >> to have the same number of partitions as the IPGeo
>>>state
>>> >> >>topic
>>> >> >> (if
>>> >> >> > > >>> IPGeo
>>> >> >> > > >>> >> has 8 partitions, then the new
>>>PageViewEventRepartitioned
>>> >> >>topic
>>> >> >> > > >>>needs 8
>>> >> >> > > >>> >>as
>>> >> >> > > >>> >> well). This will cause your PageViewEventRepartitioned
>>> >>topic
>>> >> >>and
>>> >> >> > > >>>your
>>> >> >> > > >>> >> IPGeo state topic to be aligned such that the
>>>StreamTask
>>> >>that
>>> >> >> gets
>>> >> >> > > >>>page
>>> >> >> > > >>> >> views for IP address X will also have the IPGeo
>>> >>information
>>> >> >>for
>>> >> >> IP
>>> >> >> > > >>> >>address
>>> >> >> > > >>> >> X.
>>> >> >> > > >>> >>
>>> >> >> > > >>> >> Which strategy you pick is really up to you. :) (4) is
>>> the
>>> >> >>most
>>> >> >> > > >>> >> complicated, but also the most flexible, and most
>>> >> >>operationally
>>> >> >> > > >>>sound.
>>> >> >> > > >>> >>(1)
>>> >> >> > > >>> >> is the easiest if it fits your needs.
>>> >> >> > > >>> >>
>>> >> >> > > >>> >> Cheers,
>>> >> >> > > >>> >> Chris
>>> >> >> > > >>> >>
>>> >> >> > > >>> >> On 8/21/14 10:15 AM, "Shekar Tippur"
>>><ct...@gmail.com>
>>> >> >>wrote:
>>> >> >> > > >>> >>
>>> >> >> > > >>> >> >Hello,
>>> >> >> > > >>> >> >
>>> >> >> > > >>> >> >I am new to Samza. I have just installed Hello Samza
>>>and
>>> >> >>got it
>>> >> >> > > >>> >>working.
>>> >> >> > > >>> >> >
>>> >> >> > > >>> >> >Here is the use case for which I am trying to use
>>>Samza:
>>> >> >> > > >>> >> >
>>> >> >> > > >>> >> >
>>> >> >> > > >>> >> >1. Cache the contextual information which contains
>>>more
>>> >> >> > information
>>> >> >> > > >>> >>about
>>> >> >> > > >>> >> >the hostname or IP address using Samza/Yarn/Kafka
>>> >> >> > > >>> >> >2. Collect alert and metric events which contain
>>>either
>>> >> >> hostname
>>> >> >> > > >>>or IP
>>> >> >> > > >>> >> >address
>>> >> >> > > >>> >> >3. Append contextual information to the alert and
>>>metric
>>> >>and
>>> >> >> > > >>>insert to
>>> >> >> > > >>> >>a
>>> >> >> > > >>> >> >Kafka queue from which other subscribers read off of.
>>> >> >> > > >>> >> >
>>> >> >> > > >>> >> >Can you please shed some light on
>>> >> >> > > >>> >> >
>>> >> >> > > >>> >> >1. Is this feasible?
>>> >> >> > > >>> >> >2. Am I on the right thought process
>>> >> >> > > >>> >> >3. How do I start
>>> >> >> > > >>> >> >
>>> >> >> > > >>> >> >I now have 1 & 2 of them working disparately. I need
>>>to
>>> >> >> integrate
>>> >> >> > > >>> them.
>>> >> >> > > >>> >> >
>>> >> >> > > >>> >> >Appreciate any input.
>>> >> >> > > >>> >> >
>>> >> >> > > >>> >> >- Shekar
>>> >> >> > > >>> >>
>>> >> >> > > >>> >>
>>> >> >> > > >>>
>>> >> >> > > >>>
>>> >> >> > > >>
>>> >> >> > >
>>> >> >> > >
>>> >> >> >
>>> >> >>
>>> >>
>>> >>
>>>
>>>
>>


Re: Samza as a Caching layer

Posted by Shekar Tippur <ct...@gmail.com>.
Other observation is ..

http://10.132.62.185:8088/cluster shows that no applications are running.

- Shekar




On Tue, Sep 2, 2014 at 1:15 PM, Shekar Tippur <ct...@gmail.com> wrote:

> Yarn seem to be running ..
>
> yarn      5462  0.0  2.0 1641296 161404 ?      Sl   Jun20  95:26
> /usr/java/jdk1.6.0_31/bin/java -Dproc_resourcemanager -Xmx1000m
> -Dhadoop.log.dir=/var/log/hadoop-yarn -Dyarn.log.dir=/var/log/hadoop-yarn
> -Dhadoop.log.file=yarn-yarn-resourcemanager-pppdc9prd2y2.log
> -Dyarn.log.file=yarn-yarn-resourcemanager-pppdc9prd2y2.log
> -Dyarn.home.dir=/usr/lib/hadoop-yarn -Dhadoop.home.dir=/usr/lib/hadoop-yarn
> -Dhadoop.root.logger=INFO,RFA -Dyarn.root.logger=INFO,RFA
> -Djava.library.path=/usr/lib/hadoop/lib/native -classpath
> /etc/hadoop/conf:/etc/hadoop/conf:/etc/hadoop/conf:/usr/lib/hadoop/lib/*:/usr/lib/hadoop/.//*:/usr/lib/hadoop-hdfs/./:/usr/lib/hadoop-hdfs/lib/*:/usr/lib/hadoop-hdfs/.//*:/usr/lib/hadoop-yarn/lib/*:/usr/lib/hadoop-yarn/.//*:/usr/lib/hadoop-mapreduce/lib/*:/usr/lib/hadoop-mapreduce/.//*:/usr/lib/hadoop-yarn/.//*:/usr/lib/hadoop-yarn/lib/*:/etc/hadoop/conf/rm-config/log4j.properties
> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager
>
> I can tail kafka topic as well ..
>
> deploy/kafka/bin/kafka-console-consumer.sh  --zookeeper localhost:2181 --topic wikipedia-raw
>
>
>
>
> On Tue, Sep 2, 2014 at 1:04 PM, Chris Riccomini <
> criccomini@linkedin.com.invalid> wrote:
>
>> Hey Shekar,
>>
>> It looks like your job is hanging trying to connect to the RM on your
>> localhost. I thought that you said that your job was running in local
>> mode. If so, it should be using the LocalJobFactory. If not, and you
>> intend to run on YARN, is your YARN RM up and running on localhost?
>>
>> Cheers,
>> Chris
>>
>> On 9/2/14 12:22 PM, "Shekar Tippur" <ct...@gmail.com> wrote:
>>
>> >Chris ..
>> >
>> >$ cat ./deploy/samza/undefined-samza-container-name.log
>> >
>> >2014-09-02 11:17:58 JobRunner [INFO] job factory:
>> >org.apache.samza.job.yarn.YarnJobFactory
>> >
>> >2014-09-02 11:17:59 ClientHelper [INFO] trying to connect to RM
>> >127.0.0.1:8032
>> >
>> >2014-09-02 11:17:59 NativeCodeLoader [WARN] Unable to load native-hadoop
>> >library for your platform... using builtin-java classes where applicable
>> >
>> >2014-09-02 11:17:59 RMProxy [INFO] Connecting to ResourceManager at /
>> >127.0.0.1:8032
>> >
>> >
>> >and Log4j ..
>> >
>> ><!DOCTYPE log4j:configuration SYSTEM "log4j.dtd">
>> >
>> ><log4j:configuration xmlns:log4j="http://jakarta.apache.org/log4j/">
>> >
>> >  <appender name="RollingAppender"
>> >class="org.apache.log4j.DailyRollingFileAppender">
>> >
>> >     <param name="File"
>> >value="${samza.log.dir}/${samza.container.name}.log"
>> >/>
>> >
>> >     <param name="DatePattern" value="'.'yyyy-MM-dd" />
>> >
>> >     <layout class="org.apache.log4j.PatternLayout">
>> >
>> >      <param name="ConversionPattern" value="%d{yyyy-MM-dd HH:mm:ss}
>> %c{1}
>> >[%p] %m%n" />
>> >
>> >     </layout>
>> >
>> >  </appender>
>> >
>> >  <root>
>> >
>> >    <priority value="info" />
>> >
>> >    <appender-ref ref="RollingAppender"/>
>> >
>> >  </root>
>> >
>> ></log4j:configuration>
>> >
>> >
>> >On Tue, Sep 2, 2014 at 12:02 PM, Chris Riccomini <
>> >criccomini@linkedin.com.invalid> wrote:
>> >
>> >> Hey Shekar,
>> >>
>> >> Can you attach your log files? I'm wondering if it's a mis-configured
>> >> log4j.xml (or missing slf4j-log4j jar), which is leading to nearly
>> empty
>> >> log files. Also, I'm wondering if the job starts fully. Anything you
>> can
>> >> attach would be helpful.
>> >>
>> >> Cheers,
>> >> Chris
>> >>
>> >> On 9/2/14 11:43 AM, "Shekar Tippur" <ct...@gmail.com> wrote:
>> >>
>> >> >I am running in local mode.
>> >> >
>> >> >S
>> >> >On Sep 2, 2014 11:42 AM, "Yan Fang" <ya...@gmail.com> wrote:
>> >> >
>> >> >> Hi Shekar.
>> >> >>
>> >> >> Are you running job in local mode or yarn mode? If yarn mode, the
>> log
>> >> >>is in
>> >> >> the yarn's container log.
>> >> >>
>> >> >> Thanks,
>> >> >>
>> >> >> Fang, Yan
>> >> >> yanfang724@gmail.com
>> >> >> +1 (206) 849-4108
>> >> >>
>> >> >>
>> >> >> On Tue, Sep 2, 2014 at 11:36 AM, Shekar Tippur <ct...@gmail.com>
>> >> >>wrote:
>> >> >>
>> >> >> > Chris,
>> >> >> >
>> >> >> > Got some time to play around a bit more.
>> >> >> > I tried to edit
>> >> >> >
>> >> >> >
>> >> >>
>> >>
>>
>> >>>>samza-wikipedia/src/main/java/samza/examples/wikipedia/task/WikipediaFe
>> >>>>ed
>> >> >>StreamTask.java
>> >> >> > to add a logger info statement to tap the incoming message. I dont
>> >>see
>> >> >> the
>> >> >> > messages being printed to the log file.
>> >> >> >
>> >> >> > Is this the right place to start?
>> >> >> >
>> >> >> > public class WikipediaFeedStreamTask implements StreamTask {
>> >> >> >
>> >> >> >   private static final SystemStream OUTPUT_STREAM = new
>> >> >> > SystemStream("kafka",
>> >> >> > "wikipedia-raw");
>> >> >> >
>> >> >> >   private static final Logger log = LoggerFactory.getLogger
>> >> >> > (WikipediaFeedStreamTask.class);
>> >> >> >
>> >> >> >   @Override
>> >> >> >
>> >> >> >   public void process(IncomingMessageEnvelope envelope,
>> >> >>MessageCollector
>> >> >> > collector, TaskCoordinator coordinator) {
>> >> >> >
>> >> >> >     Map<String, Object> outgoingMap =
>> >> >> > WikipediaFeedEvent.toMap((WikipediaFeedEvent)
>> >>envelope.getMessage());
>> >> >> >
>> >> >> >     log.info(envelope.getMessage().toString());
>> >> >> >
>> >> >> >     collector.send(new OutgoingMessageEnvelope(OUTPUT_STREAM,
>> >> >> > outgoingMap));
>> >> >> >
>> >> >> >   }
>> >> >> >
>> >> >> > }
>> >> >> >
>> >> >> >
>> >> >> > On Mon, Aug 25, 2014 at 9:01 AM, Chris Riccomini <
>> >> >> > criccomini@linkedin.com.invalid> wrote:
>> >> >> >
>> >> >> > > Hey Shekar,
>> >> >> > >
>> >> >> > > Your thought process is on the right track. It's probably best
>> to
>> >> >>start
>> >> >> > > with hello-samza, and modify it to get what you want. To start
>> >>with,
>> >> >> > > you'll want to:
>> >> >> > >
>> >> >> > > 1. Write a simple StreamTask that just does something silly like
>> >> >>just
>> >> >> > > print messages that it receives.
>> >> >> > > 2. Write a configuration for the job that consumes from just the
>> >> >>stream
>> >> >> > > (alerts from different sources).
>> >> >> > > 3. Run this to make sure you've got it working.
>> >> >> > > 4. Now add your table join. This can be either a change-data
>> >>capture
>> >> >> > (CDC)
>> >> >> > > stream, or via a remote DB call.
>> >> >> > >
>> >> >> > > That should get you to a point where you've got your job up and
>> >> >> running.
>> >> >> > > From there, you could create your own Maven project, and migrate
>> >> >>your
>> >> >> > code
>> >> >> > > over accordingly.
>> >> >> > >
>> >> >> > > Cheers,
>> >> >> > > Chris
>> >> >> > >
>> >> >> > > On 8/24/14 1:42 AM, "Shekar Tippur" <ct...@gmail.com> wrote:
>> >> >> > >
>> >> >> > > >Chris,
>> >> >> > > >
>> >> >> > > >I have gone thro the documentation and decided that the option
>> >> >>that is
>> >> >> > > >most
>> >> >> > > >suitable for me is stream-table.
>> >> >> > > >
>> >> >> > > >I see the following things:
>> >> >> > > >
>> >> >> > > >1. Point samza to a table (database)
>> >> >> > > >2. Point Samza to a stream - Alert stream from different
>> sources
>> >> >> > > >3. Join key like a hostname
>> >> >> > > >
>> >> >> > > >I have Hello Samza working. To extend that to do what my needs
>> >> >>are, I
>> >> >> am
>> >> >> > > >not sure where to start (Needs more code change OR
>> configuration
>> >> >> changes
>> >> >> > > >OR
>> >> >> > > >both)?
>> >> >> > > >
>> >> >> > > >I have gone thro
>> >> >> > > >
>> >> >> >
>> >> >>
>> >> >>
>> >>
>> >>
>> http://samza.incubator.apache.org/learn/documentation/latest/api/overview
>> >> >> > > .
>> >> >> > > >html
>> >> >> > > >
>> >> >> > > >Is my thought process on the right track? Can you please point
>> >>me
>> >> >>to
>> >> >> the
>> >> >> > > >right direction?
>> >> >> > > >
>> >> >> > > >- Shekar
>> >> >> > > >
>> >> >> > > >
>> >> >> > > >
>> >> >> > > >On Thu, Aug 21, 2014 at 1:08 PM, Shekar Tippur
>> >><ct...@gmail.com>
>> >> >> > wrote:
>> >> >> > > >
>> >> >> > > >> Chris,
>> >> >> > > >>
>> >> >> > > >> This is perfectly good answer. I will start poking more into
>> >> >>option
>> >> >> > #4.
>> >> >> > > >>
>> >> >> > > >> - Shekar
>> >> >> > > >>
>> >> >> > > >>
>> >> >> > > >> On Thu, Aug 21, 2014 at 1:05 PM, Chris Riccomini <
>> >> >> > > >> criccomini@linkedin.com.invalid> wrote:
>> >> >> > > >>
>> >> >> > > >>> Hey Shekar,
>> >> >> > > >>>
>> >> >> > > >>> Your two options are really (3) or (4), then. You can either
>> >>run
>> >> >> some
>> >> >> > > >>> external DB that holds the data set, and you can query it
>> >>from a
>> >> >> > > >>> StreamTask, or you can use Samza's state store feature to
>> >>push
>> >> >>data
>> >> >> > > >>>into a
>> >> >> > > >>> stream that you can then store in a partitioned key-value
>> >>store
>> >> >> along
>> >> >> > > >>>with
>> >> >> > > >>> your StreamTasks. There is some documentation here about the
>> >> >>state
>> >> >> > > >>>store
>> >> >> > > >>> approach:
>> >> >> > > >>>
>> >> >> > > >>>
>> >> >> > > >>>
>> >> >> > > >>>
>> >> >> > >
>> >> >>
>> >>
>> http://samza.incubator.apache.org/learn/documentation/0.7.0/container/st
>> >> >> > > >>>ate
>> >> >> > > >>> -management.html
>> >> >> > > >>>
>> >> >> > > >>><
>> >> >> > >
>> >> >>
>> >>http://samza.incubator.apache.org/learn/documentation/0.7.0/container/s
>> >> >> > > >>>tate-management.html>
>> >> >> > > >>>
>> >> >> > > >>>
>> >> >> > > >>> (4) is going to require more up front effort from you, since
>> >> >>you'll
>> >> >> > > >>>have
>> >> >> > > >>> to understand how Kafka's partitioning model works, and
>> setup
>> >> >>some
>> >> >> > > >>> pipeline to push the updates for your state. In the long
>> >>run, I
>> >> >> > believe
>> >> >> > > >>> it's the better approach, though. Local lookups on a
>> >>key-value
>> >> >> store
>> >> >> > > >>> should be faster than doing remote RPC calls to a DB for
>> >>every
>> >> >> > message.
>> >> >> > > >>>
>> >> >> > > >>> I'm sorry I can't give you a more definitive answer. It's
>> >>really
>> >> >> > about
>> >> >> > > >>> trade-offs.
>> >> >> > > >>>
>> >> >> > > >>> Cheers,
>> >> >> > > >>> Chris
>> >> >> > > >>>
>> >> >> > > >>> On 8/21/14 12:22 PM, "Shekar Tippur" <ct...@gmail.com>
>> >>wrote:
>> >> >> > > >>>
>> >> >> > > >>> >Chris,
>> >> >> > > >>> >
>> >> >> > > >>> >A big thanks for a swift response. The data set is huge and
>> >>the
>> >> >> > > >>>frequency
>> >> >> > > >>> >is in burst.
>> >> >> > > >>> >What do you suggest?
>> >> >> > > >>> >
>> >> >> > > >>> >- Shekar
>> >> >> > > >>> >
>> >> >> > > >>> >
>> >> >> > > >>> >On Thu, Aug 21, 2014 at 11:05 AM, Chris Riccomini <
>> >> >> > > >>> >criccomini@linkedin.com.invalid> wrote:
>> >> >> > > >>> >
>> >> >> > > >>> >> Hey Shekar,
>> >> >> > > >>> >>
>> >> >> > > >>> >> This is feasible, and you are on the right thought
>> >>process.
>> >> >> > > >>> >>
>> >> >> > > >>> >> For the sake of discussion, I'm going to pretend that you
>> >> >>have a
>> >> >> > > >>>Kafka
>> >> >> > > >>> >> topic called "PageViewEvent", which has just the IP
>> >>address
>> >> >>that
>> >> >> > was
>> >> >> > > >>> >>used
>> >> >> > > >>> >> to view a page. These messages will be logged every time
>> a
>> >> >>page
>> >> >> > view
>> >> >> > > >>> >> happens. I'm also going to pretend that you have some
>> >>state
>> >> >> called
>> >> >> > > >>> >>"IPGeo"
>> >> >> > > >>> >> (e.g. The maxmind data set). In this example, we'll want
>> >>to
>> >> >>join
>> >> >> > the
>> >> >> > > >>> >> long/lat geo information from IPGeo to the PageViewEvent,
>> >>and
>> >> >> send
>> >> >> > > >>>it
>> >> >> > > >>> >>to a
>> >> >> > > >>> >> new topic: PageViewEventsWithGeo.
>> >> >> > > >>> >>
>> >> >> > > >>> >> You have several options on how to implement this
>> example.
>> >> >> > > >>> >>
>> >> >> > > >>> >> 1. If your joining data set (IPGeo) is relatively small
>> >>and
>> >> >> > changes
>> >> >> > > >>> >> infrequently, you can just pack it up in your jar or .tgz
>> >> >>file,
>> >> >> > and
>> >> >> > > >>> open
>> >> >> > > >>> >> it open in every StreamTask.
>> >> >> > > >>> >> 2. If your data set is small, but changes somewhat
>> >> >>frequently,
>> >> >> you
>> >> >> > > >>>can
>> >> >> > > >>> >> throw the data set on some HTTP/HDFS/S3 server somewhere,
>> >>and
>> >> >> have
>> >> >> > > >>>your
>> >> >> > > >>> >> StreamTask refresh it periodically by re-downloading it.
>> >> >> > > >>> >> 3. You can do remote RPC calls for the IPGeo data on
>> every
>> >> >>page
>> >> >> > view
>> >> >> > > >>> >>event
>> >> >> > > >>> >> by query some remote service or DB (e.g. Cassandra).
>> >> >> > > >>> >> 4. You can use Samza's state feature to set your IPGeo
>> >>data
>> >> >>as a
>> >> >> > > >>>series
>> >> >> > > >>> >>of
>> >> >> > > >>> >> messages to a log-compacted Kafka topic
>> >> >> > > >>> >> (
>> >> >> https://cwiki.apache.org/confluence/display/KAFKA/Log+Compaction
>> >> >> > ),
>> >> >> > > >>> and
>> >> >> > > >>> >> configure your Samza job to read this topic as a
>> bootstrap
>> >> >> stream
>> >> >> > > >>> >> (
>> >> >> > > >>> >>
>> >> >> > > >>> >>
>> >> >> > > >>>
>> >> >> > > >>>
>> >> >> > >
>> >> >>
>> >>
>> http://samza.incubator.apache.org/learn/documentation/0.7.0/container/st
>> >> >> > > >>>r
>> >> >> > > >>> >>e
>> >> >> > > >>> >> ams.html).
>> >> >> > > >>> >>
>> >> >> > > >>> >> For (4), you'd have to partition the IPGeo state topic
>> >> >>according
>> >> >> > to
>> >> >> > > >>>the
>> >> >> > > >>> >> same key as PageViewEvent. If PageViewEvent were
>> >>partitioned
>> >> >>by,
>> >> >> > > >>>say,
>> >> >> > > >>> >> member ID, but you want your IPGeo state topic to be
>> >> >>partitioned
>> >> >> > by
>> >> >> > > >>>IP
>> >> >> > > >>> >> address, then you'd have to have an upstream job that
>> >> >> > re-partitioned
>> >> >> > > >>> >> PageViewEvent into some new topic by IP address. This new
>> >> >>topic
>> >> >> > will
>> >> >> > > >>> >>have
>> >> >> > > >>> >> to have the same number of partitions as the IPGeo state
>> >> >>topic
>> >> >> (if
>> >> >> > > >>> IPGeo
>> >> >> > > >>> >> has 8 partitions, then the new PageViewEventRepartitioned
>> >> >>topic
>> >> >> > > >>>needs 8
>> >> >> > > >>> >>as
>> >> >> > > >>> >> well). This will cause your PageViewEventRepartitioned
>> >>topic
>> >> >>and
>> >> >> > > >>>your
>> >> >> > > >>> >> IPGeo state topic to be aligned such that the StreamTask
>> >>that
>> >> >> gets
>> >> >> > > >>>page
>> >> >> > > >>> >> views for IP address X will also have the IPGeo
>> >>information
>> >> >>for
>> >> >> IP
>> >> >> > > >>> >>address
>> >> >> > > >>> >> X.
>> >> >> > > >>> >>
>> >> >> > > >>> >> Which strategy you pick is really up to you. :) (4) is
>> the
>> >> >>most
>> >> >> > > >>> >> complicated, but also the most flexible, and most
>> >> >>operationally
>> >> >> > > >>>sound.
>> >> >> > > >>> >>(1)
>> >> >> > > >>> >> is the easiest if it fits your needs.
>> >> >> > > >>> >>
>> >> >> > > >>> >> Cheers,
>> >> >> > > >>> >> Chris
>> >> >> > > >>> >>
>> >> >> > > >>> >> On 8/21/14 10:15 AM, "Shekar Tippur" <ct...@gmail.com>
>> >> >>wrote:
>> >> >> > > >>> >>
>> >> >> > > >>> >> >Hello,
>> >> >> > > >>> >> >
>> >> >> > > >>> >> >I am new to Samza. I have just installed Hello Samza and
>> >> >>got it
>> >> >> > > >>> >>working.
>> >> >> > > >>> >> >
>> >> >> > > >>> >> >Here is the use case for which I am trying to use Samza:
>> >> >> > > >>> >> >
>> >> >> > > >>> >> >
>> >> >> > > >>> >> >1. Cache the contextual information which contains more
>> >> >> > information
>> >> >> > > >>> >>about
>> >> >> > > >>> >> >the hostname or IP address using Samza/Yarn/Kafka
>> >> >> > > >>> >> >2. Collect alert and metric events which contain either
>> >> >> hostname
>> >> >> > > >>>or IP
>> >> >> > > >>> >> >address
>> >> >> > > >>> >> >3. Append contextual information to the alert and metric
>> >>and
>> >> >> > > >>>insert to
>> >> >> > > >>> >>a
>> >> >> > > >>> >> >Kafka queue from which other subscribers read off of.
>> >> >> > > >>> >> >
>> >> >> > > >>> >> >Can you please shed some light on
>> >> >> > > >>> >> >
>> >> >> > > >>> >> >1. Is this feasible?
>> >> >> > > >>> >> >2. Am I on the right thought process
>> >> >> > > >>> >> >3. How do I start
>> >> >> > > >>> >> >
>> >> >> > > >>> >> >I now have 1 & 2 of them working disparately. I need to
>> >> >> integrate
>> >> >> > > >>> them.
>> >> >> > > >>> >> >
>> >> >> > > >>> >> >Appreciate any input.
>> >> >> > > >>> >> >
>> >> >> > > >>> >> >- Shekar
>> >> >> > > >>> >>
>> >> >> > > >>> >>
>> >> >> > > >>>
>> >> >> > > >>>
>> >> >> > > >>
>> >> >> > >
>> >> >> > >
>> >> >> >
>> >> >>
>> >>
>> >>
>>
>>
>

Re: Samza as a Caching layer

Posted by Shekar Tippur <ct...@gmail.com>.
Yarn seem to be running ..

yarn      5462  0.0  2.0 1641296 161404 ?      Sl   Jun20  95:26
/usr/java/jdk1.6.0_31/bin/java -Dproc_resourcemanager -Xmx1000m
-Dhadoop.log.dir=/var/log/hadoop-yarn -Dyarn.log.dir=/var/log/hadoop-yarn
-Dhadoop.log.file=yarn-yarn-resourcemanager-pppdc9prd2y2.log
-Dyarn.log.file=yarn-yarn-resourcemanager-pppdc9prd2y2.log
-Dyarn.home.dir=/usr/lib/hadoop-yarn -Dhadoop.home.dir=/usr/lib/hadoop-yarn
-Dhadoop.root.logger=INFO,RFA -Dyarn.root.logger=INFO,RFA
-Djava.library.path=/usr/lib/hadoop/lib/native -classpath
/etc/hadoop/conf:/etc/hadoop/conf:/etc/hadoop/conf:/usr/lib/hadoop/lib/*:/usr/lib/hadoop/.//*:/usr/lib/hadoop-hdfs/./:/usr/lib/hadoop-hdfs/lib/*:/usr/lib/hadoop-hdfs/.//*:/usr/lib/hadoop-yarn/lib/*:/usr/lib/hadoop-yarn/.//*:/usr/lib/hadoop-mapreduce/lib/*:/usr/lib/hadoop-mapreduce/.//*:/usr/lib/hadoop-yarn/.//*:/usr/lib/hadoop-yarn/lib/*:/etc/hadoop/conf/rm-config/log4j.properties
org.apache.hadoop.yarn.server.resourcemanager.ResourceManager

I can tail kafka topic as well ..

deploy/kafka/bin/kafka-console-consumer.sh  --zookeeper localhost:2181
--topic wikipedia-raw




On Tue, Sep 2, 2014 at 1:04 PM, Chris Riccomini <
criccomini@linkedin.com.invalid> wrote:

> Hey Shekar,
>
> It looks like your job is hanging trying to connect to the RM on your
> localhost. I thought that you said that your job was running in local
> mode. If so, it should be using the LocalJobFactory. If not, and you
> intend to run on YARN, is your YARN RM up and running on localhost?
>
> Cheers,
> Chris
>
> On 9/2/14 12:22 PM, "Shekar Tippur" <ct...@gmail.com> wrote:
>
> >Chris ..
> >
> >$ cat ./deploy/samza/undefined-samza-container-name.log
> >
> >2014-09-02 11:17:58 JobRunner [INFO] job factory:
> >org.apache.samza.job.yarn.YarnJobFactory
> >
> >2014-09-02 11:17:59 ClientHelper [INFO] trying to connect to RM
> >127.0.0.1:8032
> >
> >2014-09-02 11:17:59 NativeCodeLoader [WARN] Unable to load native-hadoop
> >library for your platform... using builtin-java classes where applicable
> >
> >2014-09-02 11:17:59 RMProxy [INFO] Connecting to ResourceManager at /
> >127.0.0.1:8032
> >
> >
> >and Log4j ..
> >
> ><!DOCTYPE log4j:configuration SYSTEM "log4j.dtd">
> >
> ><log4j:configuration xmlns:log4j="http://jakarta.apache.org/log4j/">
> >
> >  <appender name="RollingAppender"
> >class="org.apache.log4j.DailyRollingFileAppender">
> >
> >     <param name="File"
> >value="${samza.log.dir}/${samza.container.name}.log"
> >/>
> >
> >     <param name="DatePattern" value="'.'yyyy-MM-dd" />
> >
> >     <layout class="org.apache.log4j.PatternLayout">
> >
> >      <param name="ConversionPattern" value="%d{yyyy-MM-dd HH:mm:ss} %c{1}
> >[%p] %m%n" />
> >
> >     </layout>
> >
> >  </appender>
> >
> >  <root>
> >
> >    <priority value="info" />
> >
> >    <appender-ref ref="RollingAppender"/>
> >
> >  </root>
> >
> ></log4j:configuration>
> >
> >
> >On Tue, Sep 2, 2014 at 12:02 PM, Chris Riccomini <
> >criccomini@linkedin.com.invalid> wrote:
> >
> >> Hey Shekar,
> >>
> >> Can you attach your log files? I'm wondering if it's a mis-configured
> >> log4j.xml (or missing slf4j-log4j jar), which is leading to nearly empty
> >> log files. Also, I'm wondering if the job starts fully. Anything you can
> >> attach would be helpful.
> >>
> >> Cheers,
> >> Chris
> >>
> >> On 9/2/14 11:43 AM, "Shekar Tippur" <ct...@gmail.com> wrote:
> >>
> >> >I am running in local mode.
> >> >
> >> >S
> >> >On Sep 2, 2014 11:42 AM, "Yan Fang" <ya...@gmail.com> wrote:
> >> >
> >> >> Hi Shekar.
> >> >>
> >> >> Are you running job in local mode or yarn mode? If yarn mode, the log
> >> >>is in
> >> >> the yarn's container log.
> >> >>
> >> >> Thanks,
> >> >>
> >> >> Fang, Yan
> >> >> yanfang724@gmail.com
> >> >> +1 (206) 849-4108
> >> >>
> >> >>
> >> >> On Tue, Sep 2, 2014 at 11:36 AM, Shekar Tippur <ct...@gmail.com>
> >> >>wrote:
> >> >>
> >> >> > Chris,
> >> >> >
> >> >> > Got some time to play around a bit more.
> >> >> > I tried to edit
> >> >> >
> >> >> >
> >> >>
> >>
> >>>>samza-wikipedia/src/main/java/samza/examples/wikipedia/task/WikipediaFe
> >>>>ed
> >> >>StreamTask.java
> >> >> > to add a logger info statement to tap the incoming message. I dont
> >>see
> >> >> the
> >> >> > messages being printed to the log file.
> >> >> >
> >> >> > Is this the right place to start?
> >> >> >
> >> >> > public class WikipediaFeedStreamTask implements StreamTask {
> >> >> >
> >> >> >   private static final SystemStream OUTPUT_STREAM = new
> >> >> > SystemStream("kafka",
> >> >> > "wikipedia-raw");
> >> >> >
> >> >> >   private static final Logger log = LoggerFactory.getLogger
> >> >> > (WikipediaFeedStreamTask.class);
> >> >> >
> >> >> >   @Override
> >> >> >
> >> >> >   public void process(IncomingMessageEnvelope envelope,
> >> >>MessageCollector
> >> >> > collector, TaskCoordinator coordinator) {
> >> >> >
> >> >> >     Map<String, Object> outgoingMap =
> >> >> > WikipediaFeedEvent.toMap((WikipediaFeedEvent)
> >>envelope.getMessage());
> >> >> >
> >> >> >     log.info(envelope.getMessage().toString());
> >> >> >
> >> >> >     collector.send(new OutgoingMessageEnvelope(OUTPUT_STREAM,
> >> >> > outgoingMap));
> >> >> >
> >> >> >   }
> >> >> >
> >> >> > }
> >> >> >
> >> >> >
> >> >> > On Mon, Aug 25, 2014 at 9:01 AM, Chris Riccomini <
> >> >> > criccomini@linkedin.com.invalid> wrote:
> >> >> >
> >> >> > > Hey Shekar,
> >> >> > >
> >> >> > > Your thought process is on the right track. It's probably best to
> >> >>start
> >> >> > > with hello-samza, and modify it to get what you want. To start
> >>with,
> >> >> > > you'll want to:
> >> >> > >
> >> >> > > 1. Write a simple StreamTask that just does something silly like
> >> >>just
> >> >> > > print messages that it receives.
> >> >> > > 2. Write a configuration for the job that consumes from just the
> >> >>stream
> >> >> > > (alerts from different sources).
> >> >> > > 3. Run this to make sure you've got it working.
> >> >> > > 4. Now add your table join. This can be either a change-data
> >>capture
> >> >> > (CDC)
> >> >> > > stream, or via a remote DB call.
> >> >> > >
> >> >> > > That should get you to a point where you've got your job up and
> >> >> running.
> >> >> > > From there, you could create your own Maven project, and migrate
> >> >>your
> >> >> > code
> >> >> > > over accordingly.
> >> >> > >
> >> >> > > Cheers,
> >> >> > > Chris
> >> >> > >
> >> >> > > On 8/24/14 1:42 AM, "Shekar Tippur" <ct...@gmail.com> wrote:
> >> >> > >
> >> >> > > >Chris,
> >> >> > > >
> >> >> > > >I have gone thro the documentation and decided that the option
> >> >>that is
> >> >> > > >most
> >> >> > > >suitable for me is stream-table.
> >> >> > > >
> >> >> > > >I see the following things:
> >> >> > > >
> >> >> > > >1. Point samza to a table (database)
> >> >> > > >2. Point Samza to a stream - Alert stream from different sources
> >> >> > > >3. Join key like a hostname
> >> >> > > >
> >> >> > > >I have Hello Samza working. To extend that to do what my needs
> >> >>are, I
> >> >> am
> >> >> > > >not sure where to start (Needs more code change OR configuration
> >> >> changes
> >> >> > > >OR
> >> >> > > >both)?
> >> >> > > >
> >> >> > > >I have gone thro
> >> >> > > >
> >> >> >
> >> >>
> >> >>
> >>
> >>
> http://samza.incubator.apache.org/learn/documentation/latest/api/overview
> >> >> > > .
> >> >> > > >html
> >> >> > > >
> >> >> > > >Is my thought process on the right track? Can you please point
> >>me
> >> >>to
> >> >> the
> >> >> > > >right direction?
> >> >> > > >
> >> >> > > >- Shekar
> >> >> > > >
> >> >> > > >
> >> >> > > >
> >> >> > > >On Thu, Aug 21, 2014 at 1:08 PM, Shekar Tippur
> >><ct...@gmail.com>
> >> >> > wrote:
> >> >> > > >
> >> >> > > >> Chris,
> >> >> > > >>
> >> >> > > >> This is perfectly good answer. I will start poking more into
> >> >>option
> >> >> > #4.
> >> >> > > >>
> >> >> > > >> - Shekar
> >> >> > > >>
> >> >> > > >>
> >> >> > > >> On Thu, Aug 21, 2014 at 1:05 PM, Chris Riccomini <
> >> >> > > >> criccomini@linkedin.com.invalid> wrote:
> >> >> > > >>
> >> >> > > >>> Hey Shekar,
> >> >> > > >>>
> >> >> > > >>> Your two options are really (3) or (4), then. You can either
> >>run
> >> >> some
> >> >> > > >>> external DB that holds the data set, and you can query it
> >>from a
> >> >> > > >>> StreamTask, or you can use Samza's state store feature to
> >>push
> >> >>data
> >> >> > > >>>into a
> >> >> > > >>> stream that you can then store in a partitioned key-value
> >>store
> >> >> along
> >> >> > > >>>with
> >> >> > > >>> your StreamTasks. There is some documentation here about the
> >> >>state
> >> >> > > >>>store
> >> >> > > >>> approach:
> >> >> > > >>>
> >> >> > > >>>
> >> >> > > >>>
> >> >> > > >>>
> >> >> > >
> >> >>
> >>
> http://samza.incubator.apache.org/learn/documentation/0.7.0/container/st
> >> >> > > >>>ate
> >> >> > > >>> -management.html
> >> >> > > >>>
> >> >> > > >>><
> >> >> > >
> >> >>
> >>http://samza.incubator.apache.org/learn/documentation/0.7.0/container/s
> >> >> > > >>>tate-management.html>
> >> >> > > >>>
> >> >> > > >>>
> >> >> > > >>> (4) is going to require more up front effort from you, since
> >> >>you'll
> >> >> > > >>>have
> >> >> > > >>> to understand how Kafka's partitioning model works, and setup
> >> >>some
> >> >> > > >>> pipeline to push the updates for your state. In the long
> >>run, I
> >> >> > believe
> >> >> > > >>> it's the better approach, though. Local lookups on a
> >>key-value
> >> >> store
> >> >> > > >>> should be faster than doing remote RPC calls to a DB for
> >>every
> >> >> > message.
> >> >> > > >>>
> >> >> > > >>> I'm sorry I can't give you a more definitive answer. It's
> >>really
> >> >> > about
> >> >> > > >>> trade-offs.
> >> >> > > >>>
> >> >> > > >>> Cheers,
> >> >> > > >>> Chris
> >> >> > > >>>
> >> >> > > >>> On 8/21/14 12:22 PM, "Shekar Tippur" <ct...@gmail.com>
> >>wrote:
> >> >> > > >>>
> >> >> > > >>> >Chris,
> >> >> > > >>> >
> >> >> > > >>> >A big thanks for a swift response. The data set is huge and
> >>the
> >> >> > > >>>frequency
> >> >> > > >>> >is in burst.
> >> >> > > >>> >What do you suggest?
> >> >> > > >>> >
> >> >> > > >>> >- Shekar
> >> >> > > >>> >
> >> >> > > >>> >
> >> >> > > >>> >On Thu, Aug 21, 2014 at 11:05 AM, Chris Riccomini <
> >> >> > > >>> >criccomini@linkedin.com.invalid> wrote:
> >> >> > > >>> >
> >> >> > > >>> >> Hey Shekar,
> >> >> > > >>> >>
> >> >> > > >>> >> This is feasible, and you are on the right thought
> >>process.
> >> >> > > >>> >>
> >> >> > > >>> >> For the sake of discussion, I'm going to pretend that you
> >> >>have a
> >> >> > > >>>Kafka
> >> >> > > >>> >> topic called "PageViewEvent", which has just the IP
> >>address
> >> >>that
> >> >> > was
> >> >> > > >>> >>used
> >> >> > > >>> >> to view a page. These messages will be logged every time a
> >> >>page
> >> >> > view
> >> >> > > >>> >> happens. I'm also going to pretend that you have some
> >>state
> >> >> called
> >> >> > > >>> >>"IPGeo"
> >> >> > > >>> >> (e.g. The maxmind data set). In this example, we'll want
> >>to
> >> >>join
> >> >> > the
> >> >> > > >>> >> long/lat geo information from IPGeo to the PageViewEvent,
> >>and
> >> >> send
> >> >> > > >>>it
> >> >> > > >>> >>to a
> >> >> > > >>> >> new topic: PageViewEventsWithGeo.
> >> >> > > >>> >>
> >> >> > > >>> >> You have several options on how to implement this example.
> >> >> > > >>> >>
> >> >> > > >>> >> 1. If your joining data set (IPGeo) is relatively small
> >>and
> >> >> > changes
> >> >> > > >>> >> infrequently, you can just pack it up in your jar or .tgz
> >> >>file,
> >> >> > and
> >> >> > > >>> open
> >> >> > > >>> >> it open in every StreamTask.
> >> >> > > >>> >> 2. If your data set is small, but changes somewhat
> >> >>frequently,
> >> >> you
> >> >> > > >>>can
> >> >> > > >>> >> throw the data set on some HTTP/HDFS/S3 server somewhere,
> >>and
> >> >> have
> >> >> > > >>>your
> >> >> > > >>> >> StreamTask refresh it periodically by re-downloading it.
> >> >> > > >>> >> 3. You can do remote RPC calls for the IPGeo data on every
> >> >>page
> >> >> > view
> >> >> > > >>> >>event
> >> >> > > >>> >> by query some remote service or DB (e.g. Cassandra).
> >> >> > > >>> >> 4. You can use Samza's state feature to set your IPGeo
> >>data
> >> >>as a
> >> >> > > >>>series
> >> >> > > >>> >>of
> >> >> > > >>> >> messages to a log-compacted Kafka topic
> >> >> > > >>> >> (
> >> >> https://cwiki.apache.org/confluence/display/KAFKA/Log+Compaction
> >> >> > ),
> >> >> > > >>> and
> >> >> > > >>> >> configure your Samza job to read this topic as a bootstrap
> >> >> stream
> >> >> > > >>> >> (
> >> >> > > >>> >>
> >> >> > > >>> >>
> >> >> > > >>>
> >> >> > > >>>
> >> >> > >
> >> >>
> >>
> http://samza.incubator.apache.org/learn/documentation/0.7.0/container/st
> >> >> > > >>>r
> >> >> > > >>> >>e
> >> >> > > >>> >> ams.html).
> >> >> > > >>> >>
> >> >> > > >>> >> For (4), you'd have to partition the IPGeo state topic
> >> >>according
> >> >> > to
> >> >> > > >>>the
> >> >> > > >>> >> same key as PageViewEvent. If PageViewEvent were
> >>partitioned
> >> >>by,
> >> >> > > >>>say,
> >> >> > > >>> >> member ID, but you want your IPGeo state topic to be
> >> >>partitioned
> >> >> > by
> >> >> > > >>>IP
> >> >> > > >>> >> address, then you'd have to have an upstream job that
> >> >> > re-partitioned
> >> >> > > >>> >> PageViewEvent into some new topic by IP address. This new
> >> >>topic
> >> >> > will
> >> >> > > >>> >>have
> >> >> > > >>> >> to have the same number of partitions as the IPGeo state
> >> >>topic
> >> >> (if
> >> >> > > >>> IPGeo
> >> >> > > >>> >> has 8 partitions, then the new PageViewEventRepartitioned
> >> >>topic
> >> >> > > >>>needs 8
> >> >> > > >>> >>as
> >> >> > > >>> >> well). This will cause your PageViewEventRepartitioned
> >>topic
> >> >>and
> >> >> > > >>>your
> >> >> > > >>> >> IPGeo state topic to be aligned such that the StreamTask
> >>that
> >> >> gets
> >> >> > > >>>page
> >> >> > > >>> >> views for IP address X will also have the IPGeo
> >>information
> >> >>for
> >> >> IP
> >> >> > > >>> >>address
> >> >> > > >>> >> X.
> >> >> > > >>> >>
> >> >> > > >>> >> Which strategy you pick is really up to you. :) (4) is the
> >> >>most
> >> >> > > >>> >> complicated, but also the most flexible, and most
> >> >>operationally
> >> >> > > >>>sound.
> >> >> > > >>> >>(1)
> >> >> > > >>> >> is the easiest if it fits your needs.
> >> >> > > >>> >>
> >> >> > > >>> >> Cheers,
> >> >> > > >>> >> Chris
> >> >> > > >>> >>
> >> >> > > >>> >> On 8/21/14 10:15 AM, "Shekar Tippur" <ct...@gmail.com>
> >> >>wrote:
> >> >> > > >>> >>
> >> >> > > >>> >> >Hello,
> >> >> > > >>> >> >
> >> >> > > >>> >> >I am new to Samza. I have just installed Hello Samza and
> >> >>got it
> >> >> > > >>> >>working.
> >> >> > > >>> >> >
> >> >> > > >>> >> >Here is the use case for which I am trying to use Samza:
> >> >> > > >>> >> >
> >> >> > > >>> >> >
> >> >> > > >>> >> >1. Cache the contextual information which contains more
> >> >> > information
> >> >> > > >>> >>about
> >> >> > > >>> >> >the hostname or IP address using Samza/Yarn/Kafka
> >> >> > > >>> >> >2. Collect alert and metric events which contain either
> >> >> hostname
> >> >> > > >>>or IP
> >> >> > > >>> >> >address
> >> >> > > >>> >> >3. Append contextual information to the alert and metric
> >>and
> >> >> > > >>>insert to
> >> >> > > >>> >>a
> >> >> > > >>> >> >Kafka queue from which other subscribers read off of.
> >> >> > > >>> >> >
> >> >> > > >>> >> >Can you please shed some light on
> >> >> > > >>> >> >
> >> >> > > >>> >> >1. Is this feasible?
> >> >> > > >>> >> >2. Am I on the right thought process
> >> >> > > >>> >> >3. How do I start
> >> >> > > >>> >> >
> >> >> > > >>> >> >I now have 1 & 2 of them working disparately. I need to
> >> >> integrate
> >> >> > > >>> them.
> >> >> > > >>> >> >
> >> >> > > >>> >> >Appreciate any input.
> >> >> > > >>> >> >
> >> >> > > >>> >> >- Shekar
> >> >> > > >>> >>
> >> >> > > >>> >>
> >> >> > > >>>
> >> >> > > >>>
> >> >> > > >>
> >> >> > >
> >> >> > >
> >> >> >
> >> >>
> >>
> >>
>
>

Re: Samza as a Caching layer

Posted by Chris Riccomini <cr...@linkedin.com.INVALID>.
Hey Shekar,

It looks like your job is hanging trying to connect to the RM on your
localhost. I thought that you said that your job was running in local
mode. If so, it should be using the LocalJobFactory. If not, and you
intend to run on YARN, is your YARN RM up and running on localhost?

Cheers,
Chris

On 9/2/14 12:22 PM, "Shekar Tippur" <ct...@gmail.com> wrote:

>Chris ..
>
>$ cat ./deploy/samza/undefined-samza-container-name.log
>
>2014-09-02 11:17:58 JobRunner [INFO] job factory:
>org.apache.samza.job.yarn.YarnJobFactory
>
>2014-09-02 11:17:59 ClientHelper [INFO] trying to connect to RM
>127.0.0.1:8032
>
>2014-09-02 11:17:59 NativeCodeLoader [WARN] Unable to load native-hadoop
>library for your platform... using builtin-java classes where applicable
>
>2014-09-02 11:17:59 RMProxy [INFO] Connecting to ResourceManager at /
>127.0.0.1:8032
>
>
>and Log4j ..
>
><!DOCTYPE log4j:configuration SYSTEM "log4j.dtd">
>
><log4j:configuration xmlns:log4j="http://jakarta.apache.org/log4j/">
>
>  <appender name="RollingAppender"
>class="org.apache.log4j.DailyRollingFileAppender">
>
>     <param name="File"
>value="${samza.log.dir}/${samza.container.name}.log"
>/>
>
>     <param name="DatePattern" value="'.'yyyy-MM-dd" />
>
>     <layout class="org.apache.log4j.PatternLayout">
>
>      <param name="ConversionPattern" value="%d{yyyy-MM-dd HH:mm:ss} %c{1}
>[%p] %m%n" />
>
>     </layout>
>
>  </appender>
>
>  <root>
>
>    <priority value="info" />
>
>    <appender-ref ref="RollingAppender"/>
>
>  </root>
>
></log4j:configuration>
>
>
>On Tue, Sep 2, 2014 at 12:02 PM, Chris Riccomini <
>criccomini@linkedin.com.invalid> wrote:
>
>> Hey Shekar,
>>
>> Can you attach your log files? I'm wondering if it's a mis-configured
>> log4j.xml (or missing slf4j-log4j jar), which is leading to nearly empty
>> log files. Also, I'm wondering if the job starts fully. Anything you can
>> attach would be helpful.
>>
>> Cheers,
>> Chris
>>
>> On 9/2/14 11:43 AM, "Shekar Tippur" <ct...@gmail.com> wrote:
>>
>> >I am running in local mode.
>> >
>> >S
>> >On Sep 2, 2014 11:42 AM, "Yan Fang" <ya...@gmail.com> wrote:
>> >
>> >> Hi Shekar.
>> >>
>> >> Are you running job in local mode or yarn mode? If yarn mode, the log
>> >>is in
>> >> the yarn's container log.
>> >>
>> >> Thanks,
>> >>
>> >> Fang, Yan
>> >> yanfang724@gmail.com
>> >> +1 (206) 849-4108
>> >>
>> >>
>> >> On Tue, Sep 2, 2014 at 11:36 AM, Shekar Tippur <ct...@gmail.com>
>> >>wrote:
>> >>
>> >> > Chris,
>> >> >
>> >> > Got some time to play around a bit more.
>> >> > I tried to edit
>> >> >
>> >> >
>> >>
>> 
>>>>samza-wikipedia/src/main/java/samza/examples/wikipedia/task/WikipediaFe
>>>>ed
>> >>StreamTask.java
>> >> > to add a logger info statement to tap the incoming message. I dont
>>see
>> >> the
>> >> > messages being printed to the log file.
>> >> >
>> >> > Is this the right place to start?
>> >> >
>> >> > public class WikipediaFeedStreamTask implements StreamTask {
>> >> >
>> >> >   private static final SystemStream OUTPUT_STREAM = new
>> >> > SystemStream("kafka",
>> >> > "wikipedia-raw");
>> >> >
>> >> >   private static final Logger log = LoggerFactory.getLogger
>> >> > (WikipediaFeedStreamTask.class);
>> >> >
>> >> >   @Override
>> >> >
>> >> >   public void process(IncomingMessageEnvelope envelope,
>> >>MessageCollector
>> >> > collector, TaskCoordinator coordinator) {
>> >> >
>> >> >     Map<String, Object> outgoingMap =
>> >> > WikipediaFeedEvent.toMap((WikipediaFeedEvent)
>>envelope.getMessage());
>> >> >
>> >> >     log.info(envelope.getMessage().toString());
>> >> >
>> >> >     collector.send(new OutgoingMessageEnvelope(OUTPUT_STREAM,
>> >> > outgoingMap));
>> >> >
>> >> >   }
>> >> >
>> >> > }
>> >> >
>> >> >
>> >> > On Mon, Aug 25, 2014 at 9:01 AM, Chris Riccomini <
>> >> > criccomini@linkedin.com.invalid> wrote:
>> >> >
>> >> > > Hey Shekar,
>> >> > >
>> >> > > Your thought process is on the right track. It's probably best to
>> >>start
>> >> > > with hello-samza, and modify it to get what you want. To start
>>with,
>> >> > > you'll want to:
>> >> > >
>> >> > > 1. Write a simple StreamTask that just does something silly like
>> >>just
>> >> > > print messages that it receives.
>> >> > > 2. Write a configuration for the job that consumes from just the
>> >>stream
>> >> > > (alerts from different sources).
>> >> > > 3. Run this to make sure you've got it working.
>> >> > > 4. Now add your table join. This can be either a change-data
>>capture
>> >> > (CDC)
>> >> > > stream, or via a remote DB call.
>> >> > >
>> >> > > That should get you to a point where you've got your job up and
>> >> running.
>> >> > > From there, you could create your own Maven project, and migrate
>> >>your
>> >> > code
>> >> > > over accordingly.
>> >> > >
>> >> > > Cheers,
>> >> > > Chris
>> >> > >
>> >> > > On 8/24/14 1:42 AM, "Shekar Tippur" <ct...@gmail.com> wrote:
>> >> > >
>> >> > > >Chris,
>> >> > > >
>> >> > > >I have gone thro the documentation and decided that the option
>> >>that is
>> >> > > >most
>> >> > > >suitable for me is stream-table.
>> >> > > >
>> >> > > >I see the following things:
>> >> > > >
>> >> > > >1. Point samza to a table (database)
>> >> > > >2. Point Samza to a stream - Alert stream from different sources
>> >> > > >3. Join key like a hostname
>> >> > > >
>> >> > > >I have Hello Samza working. To extend that to do what my needs
>> >>are, I
>> >> am
>> >> > > >not sure where to start (Needs more code change OR configuration
>> >> changes
>> >> > > >OR
>> >> > > >both)?
>> >> > > >
>> >> > > >I have gone thro
>> >> > > >
>> >> >
>> >>
>> >>
>> 
>>http://samza.incubator.apache.org/learn/documentation/latest/api/overview
>> >> > > .
>> >> > > >html
>> >> > > >
>> >> > > >Is my thought process on the right track? Can you please point
>>me
>> >>to
>> >> the
>> >> > > >right direction?
>> >> > > >
>> >> > > >- Shekar
>> >> > > >
>> >> > > >
>> >> > > >
>> >> > > >On Thu, Aug 21, 2014 at 1:08 PM, Shekar Tippur
>><ct...@gmail.com>
>> >> > wrote:
>> >> > > >
>> >> > > >> Chris,
>> >> > > >>
>> >> > > >> This is perfectly good answer. I will start poking more into
>> >>option
>> >> > #4.
>> >> > > >>
>> >> > > >> - Shekar
>> >> > > >>
>> >> > > >>
>> >> > > >> On Thu, Aug 21, 2014 at 1:05 PM, Chris Riccomini <
>> >> > > >> criccomini@linkedin.com.invalid> wrote:
>> >> > > >>
>> >> > > >>> Hey Shekar,
>> >> > > >>>
>> >> > > >>> Your two options are really (3) or (4), then. You can either
>>run
>> >> some
>> >> > > >>> external DB that holds the data set, and you can query it
>>from a
>> >> > > >>> StreamTask, or you can use Samza's state store feature to
>>push
>> >>data
>> >> > > >>>into a
>> >> > > >>> stream that you can then store in a partitioned key-value
>>store
>> >> along
>> >> > > >>>with
>> >> > > >>> your StreamTasks. There is some documentation here about the
>> >>state
>> >> > > >>>store
>> >> > > >>> approach:
>> >> > > >>>
>> >> > > >>>
>> >> > > >>>
>> >> > > >>>
>> >> > >
>> >>
>> http://samza.incubator.apache.org/learn/documentation/0.7.0/container/st
>> >> > > >>>ate
>> >> > > >>> -management.html
>> >> > > >>>
>> >> > > >>><
>> >> > >
>> >> 
>>http://samza.incubator.apache.org/learn/documentation/0.7.0/container/s
>> >> > > >>>tate-management.html>
>> >> > > >>>
>> >> > > >>>
>> >> > > >>> (4) is going to require more up front effort from you, since
>> >>you'll
>> >> > > >>>have
>> >> > > >>> to understand how Kafka's partitioning model works, and setup
>> >>some
>> >> > > >>> pipeline to push the updates for your state. In the long
>>run, I
>> >> > believe
>> >> > > >>> it's the better approach, though. Local lookups on a
>>key-value
>> >> store
>> >> > > >>> should be faster than doing remote RPC calls to a DB for
>>every
>> >> > message.
>> >> > > >>>
>> >> > > >>> I'm sorry I can't give you a more definitive answer. It's
>>really
>> >> > about
>> >> > > >>> trade-offs.
>> >> > > >>>
>> >> > > >>> Cheers,
>> >> > > >>> Chris
>> >> > > >>>
>> >> > > >>> On 8/21/14 12:22 PM, "Shekar Tippur" <ct...@gmail.com>
>>wrote:
>> >> > > >>>
>> >> > > >>> >Chris,
>> >> > > >>> >
>> >> > > >>> >A big thanks for a swift response. The data set is huge and
>>the
>> >> > > >>>frequency
>> >> > > >>> >is in burst.
>> >> > > >>> >What do you suggest?
>> >> > > >>> >
>> >> > > >>> >- Shekar
>> >> > > >>> >
>> >> > > >>> >
>> >> > > >>> >On Thu, Aug 21, 2014 at 11:05 AM, Chris Riccomini <
>> >> > > >>> >criccomini@linkedin.com.invalid> wrote:
>> >> > > >>> >
>> >> > > >>> >> Hey Shekar,
>> >> > > >>> >>
>> >> > > >>> >> This is feasible, and you are on the right thought
>>process.
>> >> > > >>> >>
>> >> > > >>> >> For the sake of discussion, I'm going to pretend that you
>> >>have a
>> >> > > >>>Kafka
>> >> > > >>> >> topic called "PageViewEvent", which has just the IP
>>address
>> >>that
>> >> > was
>> >> > > >>> >>used
>> >> > > >>> >> to view a page. These messages will be logged every time a
>> >>page
>> >> > view
>> >> > > >>> >> happens. I'm also going to pretend that you have some
>>state
>> >> called
>> >> > > >>> >>"IPGeo"
>> >> > > >>> >> (e.g. The maxmind data set). In this example, we'll want
>>to
>> >>join
>> >> > the
>> >> > > >>> >> long/lat geo information from IPGeo to the PageViewEvent,
>>and
>> >> send
>> >> > > >>>it
>> >> > > >>> >>to a
>> >> > > >>> >> new topic: PageViewEventsWithGeo.
>> >> > > >>> >>
>> >> > > >>> >> You have several options on how to implement this example.
>> >> > > >>> >>
>> >> > > >>> >> 1. If your joining data set (IPGeo) is relatively small
>>and
>> >> > changes
>> >> > > >>> >> infrequently, you can just pack it up in your jar or .tgz
>> >>file,
>> >> > and
>> >> > > >>> open
>> >> > > >>> >> it open in every StreamTask.
>> >> > > >>> >> 2. If your data set is small, but changes somewhat
>> >>frequently,
>> >> you
>> >> > > >>>can
>> >> > > >>> >> throw the data set on some HTTP/HDFS/S3 server somewhere,
>>and
>> >> have
>> >> > > >>>your
>> >> > > >>> >> StreamTask refresh it periodically by re-downloading it.
>> >> > > >>> >> 3. You can do remote RPC calls for the IPGeo data on every
>> >>page
>> >> > view
>> >> > > >>> >>event
>> >> > > >>> >> by query some remote service or DB (e.g. Cassandra).
>> >> > > >>> >> 4. You can use Samza's state feature to set your IPGeo
>>data
>> >>as a
>> >> > > >>>series
>> >> > > >>> >>of
>> >> > > >>> >> messages to a log-compacted Kafka topic
>> >> > > >>> >> (
>> >> https://cwiki.apache.org/confluence/display/KAFKA/Log+Compaction
>> >> > ),
>> >> > > >>> and
>> >> > > >>> >> configure your Samza job to read this topic as a bootstrap
>> >> stream
>> >> > > >>> >> (
>> >> > > >>> >>
>> >> > > >>> >>
>> >> > > >>>
>> >> > > >>>
>> >> > >
>> >>
>> http://samza.incubator.apache.org/learn/documentation/0.7.0/container/st
>> >> > > >>>r
>> >> > > >>> >>e
>> >> > > >>> >> ams.html).
>> >> > > >>> >>
>> >> > > >>> >> For (4), you'd have to partition the IPGeo state topic
>> >>according
>> >> > to
>> >> > > >>>the
>> >> > > >>> >> same key as PageViewEvent. If PageViewEvent were
>>partitioned
>> >>by,
>> >> > > >>>say,
>> >> > > >>> >> member ID, but you want your IPGeo state topic to be
>> >>partitioned
>> >> > by
>> >> > > >>>IP
>> >> > > >>> >> address, then you'd have to have an upstream job that
>> >> > re-partitioned
>> >> > > >>> >> PageViewEvent into some new topic by IP address. This new
>> >>topic
>> >> > will
>> >> > > >>> >>have
>> >> > > >>> >> to have the same number of partitions as the IPGeo state
>> >>topic
>> >> (if
>> >> > > >>> IPGeo
>> >> > > >>> >> has 8 partitions, then the new PageViewEventRepartitioned
>> >>topic
>> >> > > >>>needs 8
>> >> > > >>> >>as
>> >> > > >>> >> well). This will cause your PageViewEventRepartitioned
>>topic
>> >>and
>> >> > > >>>your
>> >> > > >>> >> IPGeo state topic to be aligned such that the StreamTask
>>that
>> >> gets
>> >> > > >>>page
>> >> > > >>> >> views for IP address X will also have the IPGeo
>>information
>> >>for
>> >> IP
>> >> > > >>> >>address
>> >> > > >>> >> X.
>> >> > > >>> >>
>> >> > > >>> >> Which strategy you pick is really up to you. :) (4) is the
>> >>most
>> >> > > >>> >> complicated, but also the most flexible, and most
>> >>operationally
>> >> > > >>>sound.
>> >> > > >>> >>(1)
>> >> > > >>> >> is the easiest if it fits your needs.
>> >> > > >>> >>
>> >> > > >>> >> Cheers,
>> >> > > >>> >> Chris
>> >> > > >>> >>
>> >> > > >>> >> On 8/21/14 10:15 AM, "Shekar Tippur" <ct...@gmail.com>
>> >>wrote:
>> >> > > >>> >>
>> >> > > >>> >> >Hello,
>> >> > > >>> >> >
>> >> > > >>> >> >I am new to Samza. I have just installed Hello Samza and
>> >>got it
>> >> > > >>> >>working.
>> >> > > >>> >> >
>> >> > > >>> >> >Here is the use case for which I am trying to use Samza:
>> >> > > >>> >> >
>> >> > > >>> >> >
>> >> > > >>> >> >1. Cache the contextual information which contains more
>> >> > information
>> >> > > >>> >>about
>> >> > > >>> >> >the hostname or IP address using Samza/Yarn/Kafka
>> >> > > >>> >> >2. Collect alert and metric events which contain either
>> >> hostname
>> >> > > >>>or IP
>> >> > > >>> >> >address
>> >> > > >>> >> >3. Append contextual information to the alert and metric
>>and
>> >> > > >>>insert to
>> >> > > >>> >>a
>> >> > > >>> >> >Kafka queue from which other subscribers read off of.
>> >> > > >>> >> >
>> >> > > >>> >> >Can you please shed some light on
>> >> > > >>> >> >
>> >> > > >>> >> >1. Is this feasible?
>> >> > > >>> >> >2. Am I on the right thought process
>> >> > > >>> >> >3. How do I start
>> >> > > >>> >> >
>> >> > > >>> >> >I now have 1 & 2 of them working disparately. I need to
>> >> integrate
>> >> > > >>> them.
>> >> > > >>> >> >
>> >> > > >>> >> >Appreciate any input.
>> >> > > >>> >> >
>> >> > > >>> >> >- Shekar
>> >> > > >>> >>
>> >> > > >>> >>
>> >> > > >>>
>> >> > > >>>
>> >> > > >>
>> >> > >
>> >> > >
>> >> >
>> >>
>>
>>


Re: Samza as a Caching layer

Posted by Shekar Tippur <ct...@gmail.com>.
Chris ..

$ cat ./deploy/samza/undefined-samza-container-name.log

2014-09-02 11:17:58 JobRunner [INFO] job factory:
org.apache.samza.job.yarn.YarnJobFactory

2014-09-02 11:17:59 ClientHelper [INFO] trying to connect to RM
127.0.0.1:8032

2014-09-02 11:17:59 NativeCodeLoader [WARN] Unable to load native-hadoop
library for your platform... using builtin-java classes where applicable

2014-09-02 11:17:59 RMProxy [INFO] Connecting to ResourceManager at /
127.0.0.1:8032


and Log4j ..

<!DOCTYPE log4j:configuration SYSTEM "log4j.dtd">

<log4j:configuration xmlns:log4j="http://jakarta.apache.org/log4j/">

  <appender name="RollingAppender"
class="org.apache.log4j.DailyRollingFileAppender">

     <param name="File" value="${samza.log.dir}/${samza.container.name}.log"
/>

     <param name="DatePattern" value="'.'yyyy-MM-dd" />

     <layout class="org.apache.log4j.PatternLayout">

      <param name="ConversionPattern" value="%d{yyyy-MM-dd HH:mm:ss} %c{1}
[%p] %m%n" />

     </layout>

  </appender>

  <root>

    <priority value="info" />

    <appender-ref ref="RollingAppender"/>

  </root>

</log4j:configuration>


On Tue, Sep 2, 2014 at 12:02 PM, Chris Riccomini <
criccomini@linkedin.com.invalid> wrote:

> Hey Shekar,
>
> Can you attach your log files? I'm wondering if it's a mis-configured
> log4j.xml (or missing slf4j-log4j jar), which is leading to nearly empty
> log files. Also, I'm wondering if the job starts fully. Anything you can
> attach would be helpful.
>
> Cheers,
> Chris
>
> On 9/2/14 11:43 AM, "Shekar Tippur" <ct...@gmail.com> wrote:
>
> >I am running in local mode.
> >
> >S
> >On Sep 2, 2014 11:42 AM, "Yan Fang" <ya...@gmail.com> wrote:
> >
> >> Hi Shekar.
> >>
> >> Are you running job in local mode or yarn mode? If yarn mode, the log
> >>is in
> >> the yarn's container log.
> >>
> >> Thanks,
> >>
> >> Fang, Yan
> >> yanfang724@gmail.com
> >> +1 (206) 849-4108
> >>
> >>
> >> On Tue, Sep 2, 2014 at 11:36 AM, Shekar Tippur <ct...@gmail.com>
> >>wrote:
> >>
> >> > Chris,
> >> >
> >> > Got some time to play around a bit more.
> >> > I tried to edit
> >> >
> >> >
> >>
> >>samza-wikipedia/src/main/java/samza/examples/wikipedia/task/WikipediaFeed
> >>StreamTask.java
> >> > to add a logger info statement to tap the incoming message. I dont see
> >> the
> >> > messages being printed to the log file.
> >> >
> >> > Is this the right place to start?
> >> >
> >> > public class WikipediaFeedStreamTask implements StreamTask {
> >> >
> >> >   private static final SystemStream OUTPUT_STREAM = new
> >> > SystemStream("kafka",
> >> > "wikipedia-raw");
> >> >
> >> >   private static final Logger log = LoggerFactory.getLogger
> >> > (WikipediaFeedStreamTask.class);
> >> >
> >> >   @Override
> >> >
> >> >   public void process(IncomingMessageEnvelope envelope,
> >>MessageCollector
> >> > collector, TaskCoordinator coordinator) {
> >> >
> >> >     Map<String, Object> outgoingMap =
> >> > WikipediaFeedEvent.toMap((WikipediaFeedEvent) envelope.getMessage());
> >> >
> >> >     log.info(envelope.getMessage().toString());
> >> >
> >> >     collector.send(new OutgoingMessageEnvelope(OUTPUT_STREAM,
> >> > outgoingMap));
> >> >
> >> >   }
> >> >
> >> > }
> >> >
> >> >
> >> > On Mon, Aug 25, 2014 at 9:01 AM, Chris Riccomini <
> >> > criccomini@linkedin.com.invalid> wrote:
> >> >
> >> > > Hey Shekar,
> >> > >
> >> > > Your thought process is on the right track. It's probably best to
> >>start
> >> > > with hello-samza, and modify it to get what you want. To start with,
> >> > > you'll want to:
> >> > >
> >> > > 1. Write a simple StreamTask that just does something silly like
> >>just
> >> > > print messages that it receives.
> >> > > 2. Write a configuration for the job that consumes from just the
> >>stream
> >> > > (alerts from different sources).
> >> > > 3. Run this to make sure you've got it working.
> >> > > 4. Now add your table join. This can be either a change-data capture
> >> > (CDC)
> >> > > stream, or via a remote DB call.
> >> > >
> >> > > That should get you to a point where you've got your job up and
> >> running.
> >> > > From there, you could create your own Maven project, and migrate
> >>your
> >> > code
> >> > > over accordingly.
> >> > >
> >> > > Cheers,
> >> > > Chris
> >> > >
> >> > > On 8/24/14 1:42 AM, "Shekar Tippur" <ct...@gmail.com> wrote:
> >> > >
> >> > > >Chris,
> >> > > >
> >> > > >I have gone thro the documentation and decided that the option
> >>that is
> >> > > >most
> >> > > >suitable for me is stream-table.
> >> > > >
> >> > > >I see the following things:
> >> > > >
> >> > > >1. Point samza to a table (database)
> >> > > >2. Point Samza to a stream - Alert stream from different sources
> >> > > >3. Join key like a hostname
> >> > > >
> >> > > >I have Hello Samza working. To extend that to do what my needs
> >>are, I
> >> am
> >> > > >not sure where to start (Needs more code change OR configuration
> >> changes
> >> > > >OR
> >> > > >both)?
> >> > > >
> >> > > >I have gone thro
> >> > > >
> >> >
> >>
> >>
> http://samza.incubator.apache.org/learn/documentation/latest/api/overview
> >> > > .
> >> > > >html
> >> > > >
> >> > > >Is my thought process on the right track? Can you please point me
> >>to
> >> the
> >> > > >right direction?
> >> > > >
> >> > > >- Shekar
> >> > > >
> >> > > >
> >> > > >
> >> > > >On Thu, Aug 21, 2014 at 1:08 PM, Shekar Tippur <ct...@gmail.com>
> >> > wrote:
> >> > > >
> >> > > >> Chris,
> >> > > >>
> >> > > >> This is perfectly good answer. I will start poking more into
> >>option
> >> > #4.
> >> > > >>
> >> > > >> - Shekar
> >> > > >>
> >> > > >>
> >> > > >> On Thu, Aug 21, 2014 at 1:05 PM, Chris Riccomini <
> >> > > >> criccomini@linkedin.com.invalid> wrote:
> >> > > >>
> >> > > >>> Hey Shekar,
> >> > > >>>
> >> > > >>> Your two options are really (3) or (4), then. You can either run
> >> some
> >> > > >>> external DB that holds the data set, and you can query it from a
> >> > > >>> StreamTask, or you can use Samza's state store feature to push
> >>data
> >> > > >>>into a
> >> > > >>> stream that you can then store in a partitioned key-value store
> >> along
> >> > > >>>with
> >> > > >>> your StreamTasks. There is some documentation here about the
> >>state
> >> > > >>>store
> >> > > >>> approach:
> >> > > >>>
> >> > > >>>
> >> > > >>>
> >> > > >>>
> >> > >
> >>
> http://samza.incubator.apache.org/learn/documentation/0.7.0/container/st
> >> > > >>>ate
> >> > > >>> -management.html
> >> > > >>>
> >> > > >>><
> >> > >
> >> http://samza.incubator.apache.org/learn/documentation/0.7.0/container/s
> >> > > >>>tate-management.html>
> >> > > >>>
> >> > > >>>
> >> > > >>> (4) is going to require more up front effort from you, since
> >>you'll
> >> > > >>>have
> >> > > >>> to understand how Kafka's partitioning model works, and setup
> >>some
> >> > > >>> pipeline to push the updates for your state. In the long run, I
> >> > believe
> >> > > >>> it's the better approach, though. Local lookups on a key-value
> >> store
> >> > > >>> should be faster than doing remote RPC calls to a DB for every
> >> > message.
> >> > > >>>
> >> > > >>> I'm sorry I can't give you a more definitive answer. It's really
> >> > about
> >> > > >>> trade-offs.
> >> > > >>>
> >> > > >>> Cheers,
> >> > > >>> Chris
> >> > > >>>
> >> > > >>> On 8/21/14 12:22 PM, "Shekar Tippur" <ct...@gmail.com> wrote:
> >> > > >>>
> >> > > >>> >Chris,
> >> > > >>> >
> >> > > >>> >A big thanks for a swift response. The data set is huge and the
> >> > > >>>frequency
> >> > > >>> >is in burst.
> >> > > >>> >What do you suggest?
> >> > > >>> >
> >> > > >>> >- Shekar
> >> > > >>> >
> >> > > >>> >
> >> > > >>> >On Thu, Aug 21, 2014 at 11:05 AM, Chris Riccomini <
> >> > > >>> >criccomini@linkedin.com.invalid> wrote:
> >> > > >>> >
> >> > > >>> >> Hey Shekar,
> >> > > >>> >>
> >> > > >>> >> This is feasible, and you are on the right thought process.
> >> > > >>> >>
> >> > > >>> >> For the sake of discussion, I'm going to pretend that you
> >>have a
> >> > > >>>Kafka
> >> > > >>> >> topic called "PageViewEvent", which has just the IP address
> >>that
> >> > was
> >> > > >>> >>used
> >> > > >>> >> to view a page. These messages will be logged every time a
> >>page
> >> > view
> >> > > >>> >> happens. I'm also going to pretend that you have some state
> >> called
> >> > > >>> >>"IPGeo"
> >> > > >>> >> (e.g. The maxmind data set). In this example, we'll want to
> >>join
> >> > the
> >> > > >>> >> long/lat geo information from IPGeo to the PageViewEvent, and
> >> send
> >> > > >>>it
> >> > > >>> >>to a
> >> > > >>> >> new topic: PageViewEventsWithGeo.
> >> > > >>> >>
> >> > > >>> >> You have several options on how to implement this example.
> >> > > >>> >>
> >> > > >>> >> 1. If your joining data set (IPGeo) is relatively small and
> >> > changes
> >> > > >>> >> infrequently, you can just pack it up in your jar or .tgz
> >>file,
> >> > and
> >> > > >>> open
> >> > > >>> >> it open in every StreamTask.
> >> > > >>> >> 2. If your data set is small, but changes somewhat
> >>frequently,
> >> you
> >> > > >>>can
> >> > > >>> >> throw the data set on some HTTP/HDFS/S3 server somewhere, and
> >> have
> >> > > >>>your
> >> > > >>> >> StreamTask refresh it periodically by re-downloading it.
> >> > > >>> >> 3. You can do remote RPC calls for the IPGeo data on every
> >>page
> >> > view
> >> > > >>> >>event
> >> > > >>> >> by query some remote service or DB (e.g. Cassandra).
> >> > > >>> >> 4. You can use Samza's state feature to set your IPGeo data
> >>as a
> >> > > >>>series
> >> > > >>> >>of
> >> > > >>> >> messages to a log-compacted Kafka topic
> >> > > >>> >> (
> >> https://cwiki.apache.org/confluence/display/KAFKA/Log+Compaction
> >> > ),
> >> > > >>> and
> >> > > >>> >> configure your Samza job to read this topic as a bootstrap
> >> stream
> >> > > >>> >> (
> >> > > >>> >>
> >> > > >>> >>
> >> > > >>>
> >> > > >>>
> >> > >
> >>
> http://samza.incubator.apache.org/learn/documentation/0.7.0/container/st
> >> > > >>>r
> >> > > >>> >>e
> >> > > >>> >> ams.html).
> >> > > >>> >>
> >> > > >>> >> For (4), you'd have to partition the IPGeo state topic
> >>according
> >> > to
> >> > > >>>the
> >> > > >>> >> same key as PageViewEvent. If PageViewEvent were partitioned
> >>by,
> >> > > >>>say,
> >> > > >>> >> member ID, but you want your IPGeo state topic to be
> >>partitioned
> >> > by
> >> > > >>>IP
> >> > > >>> >> address, then you'd have to have an upstream job that
> >> > re-partitioned
> >> > > >>> >> PageViewEvent into some new topic by IP address. This new
> >>topic
> >> > will
> >> > > >>> >>have
> >> > > >>> >> to have the same number of partitions as the IPGeo state
> >>topic
> >> (if
> >> > > >>> IPGeo
> >> > > >>> >> has 8 partitions, then the new PageViewEventRepartitioned
> >>topic
> >> > > >>>needs 8
> >> > > >>> >>as
> >> > > >>> >> well). This will cause your PageViewEventRepartitioned topic
> >>and
> >> > > >>>your
> >> > > >>> >> IPGeo state topic to be aligned such that the StreamTask that
> >> gets
> >> > > >>>page
> >> > > >>> >> views for IP address X will also have the IPGeo information
> >>for
> >> IP
> >> > > >>> >>address
> >> > > >>> >> X.
> >> > > >>> >>
> >> > > >>> >> Which strategy you pick is really up to you. :) (4) is the
> >>most
> >> > > >>> >> complicated, but also the most flexible, and most
> >>operationally
> >> > > >>>sound.
> >> > > >>> >>(1)
> >> > > >>> >> is the easiest if it fits your needs.
> >> > > >>> >>
> >> > > >>> >> Cheers,
> >> > > >>> >> Chris
> >> > > >>> >>
> >> > > >>> >> On 8/21/14 10:15 AM, "Shekar Tippur" <ct...@gmail.com>
> >>wrote:
> >> > > >>> >>
> >> > > >>> >> >Hello,
> >> > > >>> >> >
> >> > > >>> >> >I am new to Samza. I have just installed Hello Samza and
> >>got it
> >> > > >>> >>working.
> >> > > >>> >> >
> >> > > >>> >> >Here is the use case for which I am trying to use Samza:
> >> > > >>> >> >
> >> > > >>> >> >
> >> > > >>> >> >1. Cache the contextual information which contains more
> >> > information
> >> > > >>> >>about
> >> > > >>> >> >the hostname or IP address using Samza/Yarn/Kafka
> >> > > >>> >> >2. Collect alert and metric events which contain either
> >> hostname
> >> > > >>>or IP
> >> > > >>> >> >address
> >> > > >>> >> >3. Append contextual information to the alert and metric and
> >> > > >>>insert to
> >> > > >>> >>a
> >> > > >>> >> >Kafka queue from which other subscribers read off of.
> >> > > >>> >> >
> >> > > >>> >> >Can you please shed some light on
> >> > > >>> >> >
> >> > > >>> >> >1. Is this feasible?
> >> > > >>> >> >2. Am I on the right thought process
> >> > > >>> >> >3. How do I start
> >> > > >>> >> >
> >> > > >>> >> >I now have 1 & 2 of them working disparately. I need to
> >> integrate
> >> > > >>> them.
> >> > > >>> >> >
> >> > > >>> >> >Appreciate any input.
> >> > > >>> >> >
> >> > > >>> >> >- Shekar
> >> > > >>> >>
> >> > > >>> >>
> >> > > >>>
> >> > > >>>
> >> > > >>
> >> > >
> >> > >
> >> >
> >>
>
>

Re: Samza as a Caching layer

Posted by Chris Riccomini <cr...@linkedin.com.INVALID>.
Hey Shekar,

Can you attach your log files? I'm wondering if it's a mis-configured
log4j.xml (or missing slf4j-log4j jar), which is leading to nearly empty
log files. Also, I'm wondering if the job starts fully. Anything you can
attach would be helpful.

Cheers,
Chris

On 9/2/14 11:43 AM, "Shekar Tippur" <ct...@gmail.com> wrote:

>I am running in local mode.
>
>S
>On Sep 2, 2014 11:42 AM, "Yan Fang" <ya...@gmail.com> wrote:
>
>> Hi Shekar.
>>
>> Are you running job in local mode or yarn mode? If yarn mode, the log
>>is in
>> the yarn's container log.
>>
>> Thanks,
>>
>> Fang, Yan
>> yanfang724@gmail.com
>> +1 (206) 849-4108
>>
>>
>> On Tue, Sep 2, 2014 at 11:36 AM, Shekar Tippur <ct...@gmail.com>
>>wrote:
>>
>> > Chris,
>> >
>> > Got some time to play around a bit more.
>> > I tried to edit
>> >
>> >
>> 
>>samza-wikipedia/src/main/java/samza/examples/wikipedia/task/WikipediaFeed
>>StreamTask.java
>> > to add a logger info statement to tap the incoming message. I dont see
>> the
>> > messages being printed to the log file.
>> >
>> > Is this the right place to start?
>> >
>> > public class WikipediaFeedStreamTask implements StreamTask {
>> >
>> >   private static final SystemStream OUTPUT_STREAM = new
>> > SystemStream("kafka",
>> > "wikipedia-raw");
>> >
>> >   private static final Logger log = LoggerFactory.getLogger
>> > (WikipediaFeedStreamTask.class);
>> >
>> >   @Override
>> >
>> >   public void process(IncomingMessageEnvelope envelope,
>>MessageCollector
>> > collector, TaskCoordinator coordinator) {
>> >
>> >     Map<String, Object> outgoingMap =
>> > WikipediaFeedEvent.toMap((WikipediaFeedEvent) envelope.getMessage());
>> >
>> >     log.info(envelope.getMessage().toString());
>> >
>> >     collector.send(new OutgoingMessageEnvelope(OUTPUT_STREAM,
>> > outgoingMap));
>> >
>> >   }
>> >
>> > }
>> >
>> >
>> > On Mon, Aug 25, 2014 at 9:01 AM, Chris Riccomini <
>> > criccomini@linkedin.com.invalid> wrote:
>> >
>> > > Hey Shekar,
>> > >
>> > > Your thought process is on the right track. It's probably best to
>>start
>> > > with hello-samza, and modify it to get what you want. To start with,
>> > > you'll want to:
>> > >
>> > > 1. Write a simple StreamTask that just does something silly like
>>just
>> > > print messages that it receives.
>> > > 2. Write a configuration for the job that consumes from just the
>>stream
>> > > (alerts from different sources).
>> > > 3. Run this to make sure you've got it working.
>> > > 4. Now add your table join. This can be either a change-data capture
>> > (CDC)
>> > > stream, or via a remote DB call.
>> > >
>> > > That should get you to a point where you've got your job up and
>> running.
>> > > From there, you could create your own Maven project, and migrate
>>your
>> > code
>> > > over accordingly.
>> > >
>> > > Cheers,
>> > > Chris
>> > >
>> > > On 8/24/14 1:42 AM, "Shekar Tippur" <ct...@gmail.com> wrote:
>> > >
>> > > >Chris,
>> > > >
>> > > >I have gone thro the documentation and decided that the option
>>that is
>> > > >most
>> > > >suitable for me is stream-table.
>> > > >
>> > > >I see the following things:
>> > > >
>> > > >1. Point samza to a table (database)
>> > > >2. Point Samza to a stream - Alert stream from different sources
>> > > >3. Join key like a hostname
>> > > >
>> > > >I have Hello Samza working. To extend that to do what my needs
>>are, I
>> am
>> > > >not sure where to start (Needs more code change OR configuration
>> changes
>> > > >OR
>> > > >both)?
>> > > >
>> > > >I have gone thro
>> > > >
>> >
>> 
>>http://samza.incubator.apache.org/learn/documentation/latest/api/overview
>> > > .
>> > > >html
>> > > >
>> > > >Is my thought process on the right track? Can you please point me
>>to
>> the
>> > > >right direction?
>> > > >
>> > > >- Shekar
>> > > >
>> > > >
>> > > >
>> > > >On Thu, Aug 21, 2014 at 1:08 PM, Shekar Tippur <ct...@gmail.com>
>> > wrote:
>> > > >
>> > > >> Chris,
>> > > >>
>> > > >> This is perfectly good answer. I will start poking more into
>>option
>> > #4.
>> > > >>
>> > > >> - Shekar
>> > > >>
>> > > >>
>> > > >> On Thu, Aug 21, 2014 at 1:05 PM, Chris Riccomini <
>> > > >> criccomini@linkedin.com.invalid> wrote:
>> > > >>
>> > > >>> Hey Shekar,
>> > > >>>
>> > > >>> Your two options are really (3) or (4), then. You can either run
>> some
>> > > >>> external DB that holds the data set, and you can query it from a
>> > > >>> StreamTask, or you can use Samza's state store feature to push
>>data
>> > > >>>into a
>> > > >>> stream that you can then store in a partitioned key-value store
>> along
>> > > >>>with
>> > > >>> your StreamTasks. There is some documentation here about the
>>state
>> > > >>>store
>> > > >>> approach:
>> > > >>>
>> > > >>>
>> > > >>>
>> > > >>>
>> > >
>> http://samza.incubator.apache.org/learn/documentation/0.7.0/container/st
>> > > >>>ate
>> > > >>> -management.html
>> > > >>>
>> > > >>><
>> > >
>> http://samza.incubator.apache.org/learn/documentation/0.7.0/container/s
>> > > >>>tate-management.html>
>> > > >>>
>> > > >>>
>> > > >>> (4) is going to require more up front effort from you, since
>>you'll
>> > > >>>have
>> > > >>> to understand how Kafka's partitioning model works, and setup
>>some
>> > > >>> pipeline to push the updates for your state. In the long run, I
>> > believe
>> > > >>> it's the better approach, though. Local lookups on a key-value
>> store
>> > > >>> should be faster than doing remote RPC calls to a DB for every
>> > message.
>> > > >>>
>> > > >>> I'm sorry I can't give you a more definitive answer. It's really
>> > about
>> > > >>> trade-offs.
>> > > >>>
>> > > >>> Cheers,
>> > > >>> Chris
>> > > >>>
>> > > >>> On 8/21/14 12:22 PM, "Shekar Tippur" <ct...@gmail.com> wrote:
>> > > >>>
>> > > >>> >Chris,
>> > > >>> >
>> > > >>> >A big thanks for a swift response. The data set is huge and the
>> > > >>>frequency
>> > > >>> >is in burst.
>> > > >>> >What do you suggest?
>> > > >>> >
>> > > >>> >- Shekar
>> > > >>> >
>> > > >>> >
>> > > >>> >On Thu, Aug 21, 2014 at 11:05 AM, Chris Riccomini <
>> > > >>> >criccomini@linkedin.com.invalid> wrote:
>> > > >>> >
>> > > >>> >> Hey Shekar,
>> > > >>> >>
>> > > >>> >> This is feasible, and you are on the right thought process.
>> > > >>> >>
>> > > >>> >> For the sake of discussion, I'm going to pretend that you
>>have a
>> > > >>>Kafka
>> > > >>> >> topic called "PageViewEvent", which has just the IP address
>>that
>> > was
>> > > >>> >>used
>> > > >>> >> to view a page. These messages will be logged every time a
>>page
>> > view
>> > > >>> >> happens. I'm also going to pretend that you have some state
>> called
>> > > >>> >>"IPGeo"
>> > > >>> >> (e.g. The maxmind data set). In this example, we'll want to
>>join
>> > the
>> > > >>> >> long/lat geo information from IPGeo to the PageViewEvent, and
>> send
>> > > >>>it
>> > > >>> >>to a
>> > > >>> >> new topic: PageViewEventsWithGeo.
>> > > >>> >>
>> > > >>> >> You have several options on how to implement this example.
>> > > >>> >>
>> > > >>> >> 1. If your joining data set (IPGeo) is relatively small and
>> > changes
>> > > >>> >> infrequently, you can just pack it up in your jar or .tgz
>>file,
>> > and
>> > > >>> open
>> > > >>> >> it open in every StreamTask.
>> > > >>> >> 2. If your data set is small, but changes somewhat
>>frequently,
>> you
>> > > >>>can
>> > > >>> >> throw the data set on some HTTP/HDFS/S3 server somewhere, and
>> have
>> > > >>>your
>> > > >>> >> StreamTask refresh it periodically by re-downloading it.
>> > > >>> >> 3. You can do remote RPC calls for the IPGeo data on every
>>page
>> > view
>> > > >>> >>event
>> > > >>> >> by query some remote service or DB (e.g. Cassandra).
>> > > >>> >> 4. You can use Samza's state feature to set your IPGeo data
>>as a
>> > > >>>series
>> > > >>> >>of
>> > > >>> >> messages to a log-compacted Kafka topic
>> > > >>> >> (
>> https://cwiki.apache.org/confluence/display/KAFKA/Log+Compaction
>> > ),
>> > > >>> and
>> > > >>> >> configure your Samza job to read this topic as a bootstrap
>> stream
>> > > >>> >> (
>> > > >>> >>
>> > > >>> >>
>> > > >>>
>> > > >>>
>> > >
>> http://samza.incubator.apache.org/learn/documentation/0.7.0/container/st
>> > > >>>r
>> > > >>> >>e
>> > > >>> >> ams.html).
>> > > >>> >>
>> > > >>> >> For (4), you'd have to partition the IPGeo state topic
>>according
>> > to
>> > > >>>the
>> > > >>> >> same key as PageViewEvent. If PageViewEvent were partitioned
>>by,
>> > > >>>say,
>> > > >>> >> member ID, but you want your IPGeo state topic to be
>>partitioned
>> > by
>> > > >>>IP
>> > > >>> >> address, then you'd have to have an upstream job that
>> > re-partitioned
>> > > >>> >> PageViewEvent into some new topic by IP address. This new
>>topic
>> > will
>> > > >>> >>have
>> > > >>> >> to have the same number of partitions as the IPGeo state
>>topic
>> (if
>> > > >>> IPGeo
>> > > >>> >> has 8 partitions, then the new PageViewEventRepartitioned
>>topic
>> > > >>>needs 8
>> > > >>> >>as
>> > > >>> >> well). This will cause your PageViewEventRepartitioned topic
>>and
>> > > >>>your
>> > > >>> >> IPGeo state topic to be aligned such that the StreamTask that
>> gets
>> > > >>>page
>> > > >>> >> views for IP address X will also have the IPGeo information
>>for
>> IP
>> > > >>> >>address
>> > > >>> >> X.
>> > > >>> >>
>> > > >>> >> Which strategy you pick is really up to you. :) (4) is the
>>most
>> > > >>> >> complicated, but also the most flexible, and most
>>operationally
>> > > >>>sound.
>> > > >>> >>(1)
>> > > >>> >> is the easiest if it fits your needs.
>> > > >>> >>
>> > > >>> >> Cheers,
>> > > >>> >> Chris
>> > > >>> >>
>> > > >>> >> On 8/21/14 10:15 AM, "Shekar Tippur" <ct...@gmail.com>
>>wrote:
>> > > >>> >>
>> > > >>> >> >Hello,
>> > > >>> >> >
>> > > >>> >> >I am new to Samza. I have just installed Hello Samza and
>>got it
>> > > >>> >>working.
>> > > >>> >> >
>> > > >>> >> >Here is the use case for which I am trying to use Samza:
>> > > >>> >> >
>> > > >>> >> >
>> > > >>> >> >1. Cache the contextual information which contains more
>> > information
>> > > >>> >>about
>> > > >>> >> >the hostname or IP address using Samza/Yarn/Kafka
>> > > >>> >> >2. Collect alert and metric events which contain either
>> hostname
>> > > >>>or IP
>> > > >>> >> >address
>> > > >>> >> >3. Append contextual information to the alert and metric and
>> > > >>>insert to
>> > > >>> >>a
>> > > >>> >> >Kafka queue from which other subscribers read off of.
>> > > >>> >> >
>> > > >>> >> >Can you please shed some light on
>> > > >>> >> >
>> > > >>> >> >1. Is this feasible?
>> > > >>> >> >2. Am I on the right thought process
>> > > >>> >> >3. How do I start
>> > > >>> >> >
>> > > >>> >> >I now have 1 & 2 of them working disparately. I need to
>> integrate
>> > > >>> them.
>> > > >>> >> >
>> > > >>> >> >Appreciate any input.
>> > > >>> >> >
>> > > >>> >> >- Shekar
>> > > >>> >>
>> > > >>> >>
>> > > >>>
>> > > >>>
>> > > >>
>> > >
>> > >
>> >
>>


Re: Samza as a Caching layer

Posted by Shekar Tippur <ct...@gmail.com>.
I am running in local mode.

S
On Sep 2, 2014 11:42 AM, "Yan Fang" <ya...@gmail.com> wrote:

> Hi Shekar.
>
> Are you running job in local mode or yarn mode? If yarn mode, the log is in
> the yarn's container log.
>
> Thanks,
>
> Fang, Yan
> yanfang724@gmail.com
> +1 (206) 849-4108
>
>
> On Tue, Sep 2, 2014 at 11:36 AM, Shekar Tippur <ct...@gmail.com> wrote:
>
> > Chris,
> >
> > Got some time to play around a bit more.
> > I tried to edit
> >
> >
> samza-wikipedia/src/main/java/samza/examples/wikipedia/task/WikipediaFeedStreamTask.java
> > to add a logger info statement to tap the incoming message. I dont see
> the
> > messages being printed to the log file.
> >
> > Is this the right place to start?
> >
> > public class WikipediaFeedStreamTask implements StreamTask {
> >
> >   private static final SystemStream OUTPUT_STREAM = new
> > SystemStream("kafka",
> > "wikipedia-raw");
> >
> >   private static final Logger log = LoggerFactory.getLogger
> > (WikipediaFeedStreamTask.class);
> >
> >   @Override
> >
> >   public void process(IncomingMessageEnvelope envelope, MessageCollector
> > collector, TaskCoordinator coordinator) {
> >
> >     Map<String, Object> outgoingMap =
> > WikipediaFeedEvent.toMap((WikipediaFeedEvent) envelope.getMessage());
> >
> >     log.info(envelope.getMessage().toString());
> >
> >     collector.send(new OutgoingMessageEnvelope(OUTPUT_STREAM,
> > outgoingMap));
> >
> >   }
> >
> > }
> >
> >
> > On Mon, Aug 25, 2014 at 9:01 AM, Chris Riccomini <
> > criccomini@linkedin.com.invalid> wrote:
> >
> > > Hey Shekar,
> > >
> > > Your thought process is on the right track. It's probably best to start
> > > with hello-samza, and modify it to get what you want. To start with,
> > > you'll want to:
> > >
> > > 1. Write a simple StreamTask that just does something silly like just
> > > print messages that it receives.
> > > 2. Write a configuration for the job that consumes from just the stream
> > > (alerts from different sources).
> > > 3. Run this to make sure you've got it working.
> > > 4. Now add your table join. This can be either a change-data capture
> > (CDC)
> > > stream, or via a remote DB call.
> > >
> > > That should get you to a point where you've got your job up and
> running.
> > > From there, you could create your own Maven project, and migrate your
> > code
> > > over accordingly.
> > >
> > > Cheers,
> > > Chris
> > >
> > > On 8/24/14 1:42 AM, "Shekar Tippur" <ct...@gmail.com> wrote:
> > >
> > > >Chris,
> > > >
> > > >I have gone thro the documentation and decided that the option that is
> > > >most
> > > >suitable for me is stream-table.
> > > >
> > > >I see the following things:
> > > >
> > > >1. Point samza to a table (database)
> > > >2. Point Samza to a stream - Alert stream from different sources
> > > >3. Join key like a hostname
> > > >
> > > >I have Hello Samza working. To extend that to do what my needs are, I
> am
> > > >not sure where to start (Needs more code change OR configuration
> changes
> > > >OR
> > > >both)?
> > > >
> > > >I have gone thro
> > > >
> >
> http://samza.incubator.apache.org/learn/documentation/latest/api/overview
> > > .
> > > >html
> > > >
> > > >Is my thought process on the right track? Can you please point me to
> the
> > > >right direction?
> > > >
> > > >- Shekar
> > > >
> > > >
> > > >
> > > >On Thu, Aug 21, 2014 at 1:08 PM, Shekar Tippur <ct...@gmail.com>
> > wrote:
> > > >
> > > >> Chris,
> > > >>
> > > >> This is perfectly good answer. I will start poking more into option
> > #4.
> > > >>
> > > >> - Shekar
> > > >>
> > > >>
> > > >> On Thu, Aug 21, 2014 at 1:05 PM, Chris Riccomini <
> > > >> criccomini@linkedin.com.invalid> wrote:
> > > >>
> > > >>> Hey Shekar,
> > > >>>
> > > >>> Your two options are really (3) or (4), then. You can either run
> some
> > > >>> external DB that holds the data set, and you can query it from a
> > > >>> StreamTask, or you can use Samza's state store feature to push data
> > > >>>into a
> > > >>> stream that you can then store in a partitioned key-value store
> along
> > > >>>with
> > > >>> your StreamTasks. There is some documentation here about the state
> > > >>>store
> > > >>> approach:
> > > >>>
> > > >>>
> > > >>>
> > > >>>
> > >
> http://samza.incubator.apache.org/learn/documentation/0.7.0/container/st
> > > >>>ate
> > > >>> -management.html
> > > >>>
> > > >>><
> > >
> http://samza.incubator.apache.org/learn/documentation/0.7.0/container/s
> > > >>>tate-management.html>
> > > >>>
> > > >>>
> > > >>> (4) is going to require more up front effort from you, since you'll
> > > >>>have
> > > >>> to understand how Kafka's partitioning model works, and setup some
> > > >>> pipeline to push the updates for your state. In the long run, I
> > believe
> > > >>> it's the better approach, though. Local lookups on a key-value
> store
> > > >>> should be faster than doing remote RPC calls to a DB for every
> > message.
> > > >>>
> > > >>> I'm sorry I can't give you a more definitive answer. It's really
> > about
> > > >>> trade-offs.
> > > >>>
> > > >>> Cheers,
> > > >>> Chris
> > > >>>
> > > >>> On 8/21/14 12:22 PM, "Shekar Tippur" <ct...@gmail.com> wrote:
> > > >>>
> > > >>> >Chris,
> > > >>> >
> > > >>> >A big thanks for a swift response. The data set is huge and the
> > > >>>frequency
> > > >>> >is in burst.
> > > >>> >What do you suggest?
> > > >>> >
> > > >>> >- Shekar
> > > >>> >
> > > >>> >
> > > >>> >On Thu, Aug 21, 2014 at 11:05 AM, Chris Riccomini <
> > > >>> >criccomini@linkedin.com.invalid> wrote:
> > > >>> >
> > > >>> >> Hey Shekar,
> > > >>> >>
> > > >>> >> This is feasible, and you are on the right thought process.
> > > >>> >>
> > > >>> >> For the sake of discussion, I'm going to pretend that you have a
> > > >>>Kafka
> > > >>> >> topic called "PageViewEvent", which has just the IP address that
> > was
> > > >>> >>used
> > > >>> >> to view a page. These messages will be logged every time a page
> > view
> > > >>> >> happens. I'm also going to pretend that you have some state
> called
> > > >>> >>"IPGeo"
> > > >>> >> (e.g. The maxmind data set). In this example, we'll want to join
> > the
> > > >>> >> long/lat geo information from IPGeo to the PageViewEvent, and
> send
> > > >>>it
> > > >>> >>to a
> > > >>> >> new topic: PageViewEventsWithGeo.
> > > >>> >>
> > > >>> >> You have several options on how to implement this example.
> > > >>> >>
> > > >>> >> 1. If your joining data set (IPGeo) is relatively small and
> > changes
> > > >>> >> infrequently, you can just pack it up in your jar or .tgz file,
> > and
> > > >>> open
> > > >>> >> it open in every StreamTask.
> > > >>> >> 2. If your data set is small, but changes somewhat frequently,
> you
> > > >>>can
> > > >>> >> throw the data set on some HTTP/HDFS/S3 server somewhere, and
> have
> > > >>>your
> > > >>> >> StreamTask refresh it periodically by re-downloading it.
> > > >>> >> 3. You can do remote RPC calls for the IPGeo data on every page
> > view
> > > >>> >>event
> > > >>> >> by query some remote service or DB (e.g. Cassandra).
> > > >>> >> 4. You can use Samza's state feature to set your IPGeo data as a
> > > >>>series
> > > >>> >>of
> > > >>> >> messages to a log-compacted Kafka topic
> > > >>> >> (
> https://cwiki.apache.org/confluence/display/KAFKA/Log+Compaction
> > ),
> > > >>> and
> > > >>> >> configure your Samza job to read this topic as a bootstrap
> stream
> > > >>> >> (
> > > >>> >>
> > > >>> >>
> > > >>>
> > > >>>
> > >
> http://samza.incubator.apache.org/learn/documentation/0.7.0/container/st
> > > >>>r
> > > >>> >>e
> > > >>> >> ams.html).
> > > >>> >>
> > > >>> >> For (4), you'd have to partition the IPGeo state topic according
> > to
> > > >>>the
> > > >>> >> same key as PageViewEvent. If PageViewEvent were partitioned by,
> > > >>>say,
> > > >>> >> member ID, but you want your IPGeo state topic to be partitioned
> > by
> > > >>>IP
> > > >>> >> address, then you'd have to have an upstream job that
> > re-partitioned
> > > >>> >> PageViewEvent into some new topic by IP address. This new topic
> > will
> > > >>> >>have
> > > >>> >> to have the same number of partitions as the IPGeo state topic
> (if
> > > >>> IPGeo
> > > >>> >> has 8 partitions, then the new PageViewEventRepartitioned topic
> > > >>>needs 8
> > > >>> >>as
> > > >>> >> well). This will cause your PageViewEventRepartitioned topic and
> > > >>>your
> > > >>> >> IPGeo state topic to be aligned such that the StreamTask that
> gets
> > > >>>page
> > > >>> >> views for IP address X will also have the IPGeo information for
> IP
> > > >>> >>address
> > > >>> >> X.
> > > >>> >>
> > > >>> >> Which strategy you pick is really up to you. :) (4) is the most
> > > >>> >> complicated, but also the most flexible, and most operationally
> > > >>>sound.
> > > >>> >>(1)
> > > >>> >> is the easiest if it fits your needs.
> > > >>> >>
> > > >>> >> Cheers,
> > > >>> >> Chris
> > > >>> >>
> > > >>> >> On 8/21/14 10:15 AM, "Shekar Tippur" <ct...@gmail.com> wrote:
> > > >>> >>
> > > >>> >> >Hello,
> > > >>> >> >
> > > >>> >> >I am new to Samza. I have just installed Hello Samza and got it
> > > >>> >>working.
> > > >>> >> >
> > > >>> >> >Here is the use case for which I am trying to use Samza:
> > > >>> >> >
> > > >>> >> >
> > > >>> >> >1. Cache the contextual information which contains more
> > information
> > > >>> >>about
> > > >>> >> >the hostname or IP address using Samza/Yarn/Kafka
> > > >>> >> >2. Collect alert and metric events which contain either
> hostname
> > > >>>or IP
> > > >>> >> >address
> > > >>> >> >3. Append contextual information to the alert and metric and
> > > >>>insert to
> > > >>> >>a
> > > >>> >> >Kafka queue from which other subscribers read off of.
> > > >>> >> >
> > > >>> >> >Can you please shed some light on
> > > >>> >> >
> > > >>> >> >1. Is this feasible?
> > > >>> >> >2. Am I on the right thought process
> > > >>> >> >3. How do I start
> > > >>> >> >
> > > >>> >> >I now have 1 & 2 of them working disparately. I need to
> integrate
> > > >>> them.
> > > >>> >> >
> > > >>> >> >Appreciate any input.
> > > >>> >> >
> > > >>> >> >- Shekar
> > > >>> >>
> > > >>> >>
> > > >>>
> > > >>>
> > > >>
> > >
> > >
> >
>

Re: Samza as a Caching layer

Posted by Yan Fang <ya...@gmail.com>.
Hi Shekar.

Are you running job in local mode or yarn mode? If yarn mode, the log is in
the yarn's container log.

Thanks,

Fang, Yan
yanfang724@gmail.com
+1 (206) 849-4108


On Tue, Sep 2, 2014 at 11:36 AM, Shekar Tippur <ct...@gmail.com> wrote:

> Chris,
>
> Got some time to play around a bit more.
> I tried to edit
>
> samza-wikipedia/src/main/java/samza/examples/wikipedia/task/WikipediaFeedStreamTask.java
> to add a logger info statement to tap the incoming message. I dont see the
> messages being printed to the log file.
>
> Is this the right place to start?
>
> public class WikipediaFeedStreamTask implements StreamTask {
>
>   private static final SystemStream OUTPUT_STREAM = new
> SystemStream("kafka",
> "wikipedia-raw");
>
>   private static final Logger log = LoggerFactory.getLogger
> (WikipediaFeedStreamTask.class);
>
>   @Override
>
>   public void process(IncomingMessageEnvelope envelope, MessageCollector
> collector, TaskCoordinator coordinator) {
>
>     Map<String, Object> outgoingMap =
> WikipediaFeedEvent.toMap((WikipediaFeedEvent) envelope.getMessage());
>
>     log.info(envelope.getMessage().toString());
>
>     collector.send(new OutgoingMessageEnvelope(OUTPUT_STREAM,
> outgoingMap));
>
>   }
>
> }
>
>
> On Mon, Aug 25, 2014 at 9:01 AM, Chris Riccomini <
> criccomini@linkedin.com.invalid> wrote:
>
> > Hey Shekar,
> >
> > Your thought process is on the right track. It's probably best to start
> > with hello-samza, and modify it to get what you want. To start with,
> > you'll want to:
> >
> > 1. Write a simple StreamTask that just does something silly like just
> > print messages that it receives.
> > 2. Write a configuration for the job that consumes from just the stream
> > (alerts from different sources).
> > 3. Run this to make sure you've got it working.
> > 4. Now add your table join. This can be either a change-data capture
> (CDC)
> > stream, or via a remote DB call.
> >
> > That should get you to a point where you've got your job up and running.
> > From there, you could create your own Maven project, and migrate your
> code
> > over accordingly.
> >
> > Cheers,
> > Chris
> >
> > On 8/24/14 1:42 AM, "Shekar Tippur" <ct...@gmail.com> wrote:
> >
> > >Chris,
> > >
> > >I have gone thro the documentation and decided that the option that is
> > >most
> > >suitable for me is stream-table.
> > >
> > >I see the following things:
> > >
> > >1. Point samza to a table (database)
> > >2. Point Samza to a stream - Alert stream from different sources
> > >3. Join key like a hostname
> > >
> > >I have Hello Samza working. To extend that to do what my needs are, I am
> > >not sure where to start (Needs more code change OR configuration changes
> > >OR
> > >both)?
> > >
> > >I have gone thro
> > >
> http://samza.incubator.apache.org/learn/documentation/latest/api/overview
> > .
> > >html
> > >
> > >Is my thought process on the right track? Can you please point me to the
> > >right direction?
> > >
> > >- Shekar
> > >
> > >
> > >
> > >On Thu, Aug 21, 2014 at 1:08 PM, Shekar Tippur <ct...@gmail.com>
> wrote:
> > >
> > >> Chris,
> > >>
> > >> This is perfectly good answer. I will start poking more into option
> #4.
> > >>
> > >> - Shekar
> > >>
> > >>
> > >> On Thu, Aug 21, 2014 at 1:05 PM, Chris Riccomini <
> > >> criccomini@linkedin.com.invalid> wrote:
> > >>
> > >>> Hey Shekar,
> > >>>
> > >>> Your two options are really (3) or (4), then. You can either run some
> > >>> external DB that holds the data set, and you can query it from a
> > >>> StreamTask, or you can use Samza's state store feature to push data
> > >>>into a
> > >>> stream that you can then store in a partitioned key-value store along
> > >>>with
> > >>> your StreamTasks. There is some documentation here about the state
> > >>>store
> > >>> approach:
> > >>>
> > >>>
> > >>>
> > >>>
> > http://samza.incubator.apache.org/learn/documentation/0.7.0/container/st
> > >>>ate
> > >>> -management.html
> > >>>
> > >>><
> > http://samza.incubator.apache.org/learn/documentation/0.7.0/container/s
> > >>>tate-management.html>
> > >>>
> > >>>
> > >>> (4) is going to require more up front effort from you, since you'll
> > >>>have
> > >>> to understand how Kafka's partitioning model works, and setup some
> > >>> pipeline to push the updates for your state. In the long run, I
> believe
> > >>> it's the better approach, though. Local lookups on a key-value store
> > >>> should be faster than doing remote RPC calls to a DB for every
> message.
> > >>>
> > >>> I'm sorry I can't give you a more definitive answer. It's really
> about
> > >>> trade-offs.
> > >>>
> > >>> Cheers,
> > >>> Chris
> > >>>
> > >>> On 8/21/14 12:22 PM, "Shekar Tippur" <ct...@gmail.com> wrote:
> > >>>
> > >>> >Chris,
> > >>> >
> > >>> >A big thanks for a swift response. The data set is huge and the
> > >>>frequency
> > >>> >is in burst.
> > >>> >What do you suggest?
> > >>> >
> > >>> >- Shekar
> > >>> >
> > >>> >
> > >>> >On Thu, Aug 21, 2014 at 11:05 AM, Chris Riccomini <
> > >>> >criccomini@linkedin.com.invalid> wrote:
> > >>> >
> > >>> >> Hey Shekar,
> > >>> >>
> > >>> >> This is feasible, and you are on the right thought process.
> > >>> >>
> > >>> >> For the sake of discussion, I'm going to pretend that you have a
> > >>>Kafka
> > >>> >> topic called "PageViewEvent", which has just the IP address that
> was
> > >>> >>used
> > >>> >> to view a page. These messages will be logged every time a page
> view
> > >>> >> happens. I'm also going to pretend that you have some state called
> > >>> >>"IPGeo"
> > >>> >> (e.g. The maxmind data set). In this example, we'll want to join
> the
> > >>> >> long/lat geo information from IPGeo to the PageViewEvent, and send
> > >>>it
> > >>> >>to a
> > >>> >> new topic: PageViewEventsWithGeo.
> > >>> >>
> > >>> >> You have several options on how to implement this example.
> > >>> >>
> > >>> >> 1. If your joining data set (IPGeo) is relatively small and
> changes
> > >>> >> infrequently, you can just pack it up in your jar or .tgz file,
> and
> > >>> open
> > >>> >> it open in every StreamTask.
> > >>> >> 2. If your data set is small, but changes somewhat frequently, you
> > >>>can
> > >>> >> throw the data set on some HTTP/HDFS/S3 server somewhere, and have
> > >>>your
> > >>> >> StreamTask refresh it periodically by re-downloading it.
> > >>> >> 3. You can do remote RPC calls for the IPGeo data on every page
> view
> > >>> >>event
> > >>> >> by query some remote service or DB (e.g. Cassandra).
> > >>> >> 4. You can use Samza's state feature to set your IPGeo data as a
> > >>>series
> > >>> >>of
> > >>> >> messages to a log-compacted Kafka topic
> > >>> >> (https://cwiki.apache.org/confluence/display/KAFKA/Log+Compaction
> ),
> > >>> and
> > >>> >> configure your Samza job to read this topic as a bootstrap stream
> > >>> >> (
> > >>> >>
> > >>> >>
> > >>>
> > >>>
> > http://samza.incubator.apache.org/learn/documentation/0.7.0/container/st
> > >>>r
> > >>> >>e
> > >>> >> ams.html).
> > >>> >>
> > >>> >> For (4), you'd have to partition the IPGeo state topic according
> to
> > >>>the
> > >>> >> same key as PageViewEvent. If PageViewEvent were partitioned by,
> > >>>say,
> > >>> >> member ID, but you want your IPGeo state topic to be partitioned
> by
> > >>>IP
> > >>> >> address, then you'd have to have an upstream job that
> re-partitioned
> > >>> >> PageViewEvent into some new topic by IP address. This new topic
> will
> > >>> >>have
> > >>> >> to have the same number of partitions as the IPGeo state topic (if
> > >>> IPGeo
> > >>> >> has 8 partitions, then the new PageViewEventRepartitioned topic
> > >>>needs 8
> > >>> >>as
> > >>> >> well). This will cause your PageViewEventRepartitioned topic and
> > >>>your
> > >>> >> IPGeo state topic to be aligned such that the StreamTask that gets
> > >>>page
> > >>> >> views for IP address X will also have the IPGeo information for IP
> > >>> >>address
> > >>> >> X.
> > >>> >>
> > >>> >> Which strategy you pick is really up to you. :) (4) is the most
> > >>> >> complicated, but also the most flexible, and most operationally
> > >>>sound.
> > >>> >>(1)
> > >>> >> is the easiest if it fits your needs.
> > >>> >>
> > >>> >> Cheers,
> > >>> >> Chris
> > >>> >>
> > >>> >> On 8/21/14 10:15 AM, "Shekar Tippur" <ct...@gmail.com> wrote:
> > >>> >>
> > >>> >> >Hello,
> > >>> >> >
> > >>> >> >I am new to Samza. I have just installed Hello Samza and got it
> > >>> >>working.
> > >>> >> >
> > >>> >> >Here is the use case for which I am trying to use Samza:
> > >>> >> >
> > >>> >> >
> > >>> >> >1. Cache the contextual information which contains more
> information
> > >>> >>about
> > >>> >> >the hostname or IP address using Samza/Yarn/Kafka
> > >>> >> >2. Collect alert and metric events which contain either hostname
> > >>>or IP
> > >>> >> >address
> > >>> >> >3. Append contextual information to the alert and metric and
> > >>>insert to
> > >>> >>a
> > >>> >> >Kafka queue from which other subscribers read off of.
> > >>> >> >
> > >>> >> >Can you please shed some light on
> > >>> >> >
> > >>> >> >1. Is this feasible?
> > >>> >> >2. Am I on the right thought process
> > >>> >> >3. How do I start
> > >>> >> >
> > >>> >> >I now have 1 & 2 of them working disparately. I need to integrate
> > >>> them.
> > >>> >> >
> > >>> >> >Appreciate any input.
> > >>> >> >
> > >>> >> >- Shekar
> > >>> >>
> > >>> >>
> > >>>
> > >>>
> > >>
> >
> >
>

Re: Samza as a Caching layer

Posted by Shekar Tippur <ct...@gmail.com>.
Chris,

Got some time to play around a bit more.
I tried to edit
samza-wikipedia/src/main/java/samza/examples/wikipedia/task/WikipediaFeedStreamTask.java
to add a logger info statement to tap the incoming message. I dont see the
messages being printed to the log file.

Is this the right place to start?

public class WikipediaFeedStreamTask implements StreamTask {

  private static final SystemStream OUTPUT_STREAM = new SystemStream("kafka",
"wikipedia-raw");

  private static final Logger log = LoggerFactory.getLogger
(WikipediaFeedStreamTask.class);

  @Override

  public void process(IncomingMessageEnvelope envelope, MessageCollector
collector, TaskCoordinator coordinator) {

    Map<String, Object> outgoingMap =
WikipediaFeedEvent.toMap((WikipediaFeedEvent) envelope.getMessage());

    log.info(envelope.getMessage().toString());

    collector.send(new OutgoingMessageEnvelope(OUTPUT_STREAM, outgoingMap));

  }

}


On Mon, Aug 25, 2014 at 9:01 AM, Chris Riccomini <
criccomini@linkedin.com.invalid> wrote:

> Hey Shekar,
>
> Your thought process is on the right track. It's probably best to start
> with hello-samza, and modify it to get what you want. To start with,
> you'll want to:
>
> 1. Write a simple StreamTask that just does something silly like just
> print messages that it receives.
> 2. Write a configuration for the job that consumes from just the stream
> (alerts from different sources).
> 3. Run this to make sure you've got it working.
> 4. Now add your table join. This can be either a change-data capture (CDC)
> stream, or via a remote DB call.
>
> That should get you to a point where you've got your job up and running.
> From there, you could create your own Maven project, and migrate your code
> over accordingly.
>
> Cheers,
> Chris
>
> On 8/24/14 1:42 AM, "Shekar Tippur" <ct...@gmail.com> wrote:
>
> >Chris,
> >
> >I have gone thro the documentation and decided that the option that is
> >most
> >suitable for me is stream-table.
> >
> >I see the following things:
> >
> >1. Point samza to a table (database)
> >2. Point Samza to a stream - Alert stream from different sources
> >3. Join key like a hostname
> >
> >I have Hello Samza working. To extend that to do what my needs are, I am
> >not sure where to start (Needs more code change OR configuration changes
> >OR
> >both)?
> >
> >I have gone thro
> >http://samza.incubator.apache.org/learn/documentation/latest/api/overview
> .
> >html
> >
> >Is my thought process on the right track? Can you please point me to the
> >right direction?
> >
> >- Shekar
> >
> >
> >
> >On Thu, Aug 21, 2014 at 1:08 PM, Shekar Tippur <ct...@gmail.com> wrote:
> >
> >> Chris,
> >>
> >> This is perfectly good answer. I will start poking more into option #4.
> >>
> >> - Shekar
> >>
> >>
> >> On Thu, Aug 21, 2014 at 1:05 PM, Chris Riccomini <
> >> criccomini@linkedin.com.invalid> wrote:
> >>
> >>> Hey Shekar,
> >>>
> >>> Your two options are really (3) or (4), then. You can either run some
> >>> external DB that holds the data set, and you can query it from a
> >>> StreamTask, or you can use Samza's state store feature to push data
> >>>into a
> >>> stream that you can then store in a partitioned key-value store along
> >>>with
> >>> your StreamTasks. There is some documentation here about the state
> >>>store
> >>> approach:
> >>>
> >>>
> >>>
> >>>
> http://samza.incubator.apache.org/learn/documentation/0.7.0/container/st
> >>>ate
> >>> -management.html
> >>>
> >>><
> http://samza.incubator.apache.org/learn/documentation/0.7.0/container/s
> >>>tate-management.html>
> >>>
> >>>
> >>> (4) is going to require more up front effort from you, since you'll
> >>>have
> >>> to understand how Kafka's partitioning model works, and setup some
> >>> pipeline to push the updates for your state. In the long run, I believe
> >>> it's the better approach, though. Local lookups on a key-value store
> >>> should be faster than doing remote RPC calls to a DB for every message.
> >>>
> >>> I'm sorry I can't give you a more definitive answer. It's really about
> >>> trade-offs.
> >>>
> >>> Cheers,
> >>> Chris
> >>>
> >>> On 8/21/14 12:22 PM, "Shekar Tippur" <ct...@gmail.com> wrote:
> >>>
> >>> >Chris,
> >>> >
> >>> >A big thanks for a swift response. The data set is huge and the
> >>>frequency
> >>> >is in burst.
> >>> >What do you suggest?
> >>> >
> >>> >- Shekar
> >>> >
> >>> >
> >>> >On Thu, Aug 21, 2014 at 11:05 AM, Chris Riccomini <
> >>> >criccomini@linkedin.com.invalid> wrote:
> >>> >
> >>> >> Hey Shekar,
> >>> >>
> >>> >> This is feasible, and you are on the right thought process.
> >>> >>
> >>> >> For the sake of discussion, I'm going to pretend that you have a
> >>>Kafka
> >>> >> topic called "PageViewEvent", which has just the IP address that was
> >>> >>used
> >>> >> to view a page. These messages will be logged every time a page view
> >>> >> happens. I'm also going to pretend that you have some state called
> >>> >>"IPGeo"
> >>> >> (e.g. The maxmind data set). In this example, we'll want to join the
> >>> >> long/lat geo information from IPGeo to the PageViewEvent, and send
> >>>it
> >>> >>to a
> >>> >> new topic: PageViewEventsWithGeo.
> >>> >>
> >>> >> You have several options on how to implement this example.
> >>> >>
> >>> >> 1. If your joining data set (IPGeo) is relatively small and changes
> >>> >> infrequently, you can just pack it up in your jar or .tgz file, and
> >>> open
> >>> >> it open in every StreamTask.
> >>> >> 2. If your data set is small, but changes somewhat frequently, you
> >>>can
> >>> >> throw the data set on some HTTP/HDFS/S3 server somewhere, and have
> >>>your
> >>> >> StreamTask refresh it periodically by re-downloading it.
> >>> >> 3. You can do remote RPC calls for the IPGeo data on every page view
> >>> >>event
> >>> >> by query some remote service or DB (e.g. Cassandra).
> >>> >> 4. You can use Samza's state feature to set your IPGeo data as a
> >>>series
> >>> >>of
> >>> >> messages to a log-compacted Kafka topic
> >>> >> (https://cwiki.apache.org/confluence/display/KAFKA/Log+Compaction),
> >>> and
> >>> >> configure your Samza job to read this topic as a bootstrap stream
> >>> >> (
> >>> >>
> >>> >>
> >>>
> >>>
> http://samza.incubator.apache.org/learn/documentation/0.7.0/container/st
> >>>r
> >>> >>e
> >>> >> ams.html).
> >>> >>
> >>> >> For (4), you'd have to partition the IPGeo state topic according to
> >>>the
> >>> >> same key as PageViewEvent. If PageViewEvent were partitioned by,
> >>>say,
> >>> >> member ID, but you want your IPGeo state topic to be partitioned by
> >>>IP
> >>> >> address, then you'd have to have an upstream job that re-partitioned
> >>> >> PageViewEvent into some new topic by IP address. This new topic will
> >>> >>have
> >>> >> to have the same number of partitions as the IPGeo state topic (if
> >>> IPGeo
> >>> >> has 8 partitions, then the new PageViewEventRepartitioned topic
> >>>needs 8
> >>> >>as
> >>> >> well). This will cause your PageViewEventRepartitioned topic and
> >>>your
> >>> >> IPGeo state topic to be aligned such that the StreamTask that gets
> >>>page
> >>> >> views for IP address X will also have the IPGeo information for IP
> >>> >>address
> >>> >> X.
> >>> >>
> >>> >> Which strategy you pick is really up to you. :) (4) is the most
> >>> >> complicated, but also the most flexible, and most operationally
> >>>sound.
> >>> >>(1)
> >>> >> is the easiest if it fits your needs.
> >>> >>
> >>> >> Cheers,
> >>> >> Chris
> >>> >>
> >>> >> On 8/21/14 10:15 AM, "Shekar Tippur" <ct...@gmail.com> wrote:
> >>> >>
> >>> >> >Hello,
> >>> >> >
> >>> >> >I am new to Samza. I have just installed Hello Samza and got it
> >>> >>working.
> >>> >> >
> >>> >> >Here is the use case for which I am trying to use Samza:
> >>> >> >
> >>> >> >
> >>> >> >1. Cache the contextual information which contains more information
> >>> >>about
> >>> >> >the hostname or IP address using Samza/Yarn/Kafka
> >>> >> >2. Collect alert and metric events which contain either hostname
> >>>or IP
> >>> >> >address
> >>> >> >3. Append contextual information to the alert and metric and
> >>>insert to
> >>> >>a
> >>> >> >Kafka queue from which other subscribers read off of.
> >>> >> >
> >>> >> >Can you please shed some light on
> >>> >> >
> >>> >> >1. Is this feasible?
> >>> >> >2. Am I on the right thought process
> >>> >> >3. How do I start
> >>> >> >
> >>> >> >I now have 1 & 2 of them working disparately. I need to integrate
> >>> them.
> >>> >> >
> >>> >> >Appreciate any input.
> >>> >> >
> >>> >> >- Shekar
> >>> >>
> >>> >>
> >>>
> >>>
> >>
>
>

Re: Samza as a Caching layer

Posted by Chris Riccomini <cr...@linkedin.com.INVALID>.
Hey Shekar,

Your thought process is on the right track. It's probably best to start
with hello-samza, and modify it to get what you want. To start with,
you'll want to:

1. Write a simple StreamTask that just does something silly like just
print messages that it receives.
2. Write a configuration for the job that consumes from just the stream
(alerts from different sources).
3. Run this to make sure you've got it working.
4. Now add your table join. This can be either a change-data capture (CDC)
stream, or via a remote DB call.

That should get you to a point where you've got your job up and running.
>From there, you could create your own Maven project, and migrate your code
over accordingly.

Cheers,
Chris

On 8/24/14 1:42 AM, "Shekar Tippur" <ct...@gmail.com> wrote:

>Chris,
>
>I have gone thro the documentation and decided that the option that is
>most
>suitable for me is stream-table.
>
>I see the following things:
>
>1. Point samza to a table (database)
>2. Point Samza to a stream - Alert stream from different sources
>3. Join key like a hostname
>
>I have Hello Samza working. To extend that to do what my needs are, I am
>not sure where to start (Needs more code change OR configuration changes
>OR
>both)?
>
>I have gone thro
>http://samza.incubator.apache.org/learn/documentation/latest/api/overview.
>html
>
>Is my thought process on the right track? Can you please point me to the
>right direction?
>
>- Shekar
>
>
>
>On Thu, Aug 21, 2014 at 1:08 PM, Shekar Tippur <ct...@gmail.com> wrote:
>
>> Chris,
>>
>> This is perfectly good answer. I will start poking more into option #4.
>>
>> - Shekar
>>
>>
>> On Thu, Aug 21, 2014 at 1:05 PM, Chris Riccomini <
>> criccomini@linkedin.com.invalid> wrote:
>>
>>> Hey Shekar,
>>>
>>> Your two options are really (3) or (4), then. You can either run some
>>> external DB that holds the data set, and you can query it from a
>>> StreamTask, or you can use Samza's state store feature to push data
>>>into a
>>> stream that you can then store in a partitioned key-value store along
>>>with
>>> your StreamTasks. There is some documentation here about the state
>>>store
>>> approach:
>>>
>>>
>>> 
>>>http://samza.incubator.apache.org/learn/documentation/0.7.0/container/st
>>>ate
>>> -management.html
>>> 
>>><http://samza.incubator.apache.org/learn/documentation/0.7.0/container/s
>>>tate-management.html>
>>>
>>>
>>> (4) is going to require more up front effort from you, since you'll
>>>have
>>> to understand how Kafka's partitioning model works, and setup some
>>> pipeline to push the updates for your state. In the long run, I believe
>>> it's the better approach, though. Local lookups on a key-value store
>>> should be faster than doing remote RPC calls to a DB for every message.
>>>
>>> I'm sorry I can't give you a more definitive answer. It's really about
>>> trade-offs.
>>>
>>> Cheers,
>>> Chris
>>>
>>> On 8/21/14 12:22 PM, "Shekar Tippur" <ct...@gmail.com> wrote:
>>>
>>> >Chris,
>>> >
>>> >A big thanks for a swift response. The data set is huge and the
>>>frequency
>>> >is in burst.
>>> >What do you suggest?
>>> >
>>> >- Shekar
>>> >
>>> >
>>> >On Thu, Aug 21, 2014 at 11:05 AM, Chris Riccomini <
>>> >criccomini@linkedin.com.invalid> wrote:
>>> >
>>> >> Hey Shekar,
>>> >>
>>> >> This is feasible, and you are on the right thought process.
>>> >>
>>> >> For the sake of discussion, I'm going to pretend that you have a
>>>Kafka
>>> >> topic called "PageViewEvent", which has just the IP address that was
>>> >>used
>>> >> to view a page. These messages will be logged every time a page view
>>> >> happens. I'm also going to pretend that you have some state called
>>> >>"IPGeo"
>>> >> (e.g. The maxmind data set). In this example, we'll want to join the
>>> >> long/lat geo information from IPGeo to the PageViewEvent, and send
>>>it
>>> >>to a
>>> >> new topic: PageViewEventsWithGeo.
>>> >>
>>> >> You have several options on how to implement this example.
>>> >>
>>> >> 1. If your joining data set (IPGeo) is relatively small and changes
>>> >> infrequently, you can just pack it up in your jar or .tgz file, and
>>> open
>>> >> it open in every StreamTask.
>>> >> 2. If your data set is small, but changes somewhat frequently, you
>>>can
>>> >> throw the data set on some HTTP/HDFS/S3 server somewhere, and have
>>>your
>>> >> StreamTask refresh it periodically by re-downloading it.
>>> >> 3. You can do remote RPC calls for the IPGeo data on every page view
>>> >>event
>>> >> by query some remote service or DB (e.g. Cassandra).
>>> >> 4. You can use Samza's state feature to set your IPGeo data as a
>>>series
>>> >>of
>>> >> messages to a log-compacted Kafka topic
>>> >> (https://cwiki.apache.org/confluence/display/KAFKA/Log+Compaction),
>>> and
>>> >> configure your Samza job to read this topic as a bootstrap stream
>>> >> (
>>> >>
>>> >>
>>> 
>>>http://samza.incubator.apache.org/learn/documentation/0.7.0/container/st
>>>r
>>> >>e
>>> >> ams.html).
>>> >>
>>> >> For (4), you'd have to partition the IPGeo state topic according to
>>>the
>>> >> same key as PageViewEvent. If PageViewEvent were partitioned by,
>>>say,
>>> >> member ID, but you want your IPGeo state topic to be partitioned by
>>>IP
>>> >> address, then you'd have to have an upstream job that re-partitioned
>>> >> PageViewEvent into some new topic by IP address. This new topic will
>>> >>have
>>> >> to have the same number of partitions as the IPGeo state topic (if
>>> IPGeo
>>> >> has 8 partitions, then the new PageViewEventRepartitioned topic
>>>needs 8
>>> >>as
>>> >> well). This will cause your PageViewEventRepartitioned topic and
>>>your
>>> >> IPGeo state topic to be aligned such that the StreamTask that gets
>>>page
>>> >> views for IP address X will also have the IPGeo information for IP
>>> >>address
>>> >> X.
>>> >>
>>> >> Which strategy you pick is really up to you. :) (4) is the most
>>> >> complicated, but also the most flexible, and most operationally
>>>sound.
>>> >>(1)
>>> >> is the easiest if it fits your needs.
>>> >>
>>> >> Cheers,
>>> >> Chris
>>> >>
>>> >> On 8/21/14 10:15 AM, "Shekar Tippur" <ct...@gmail.com> wrote:
>>> >>
>>> >> >Hello,
>>> >> >
>>> >> >I am new to Samza. I have just installed Hello Samza and got it
>>> >>working.
>>> >> >
>>> >> >Here is the use case for which I am trying to use Samza:
>>> >> >
>>> >> >
>>> >> >1. Cache the contextual information which contains more information
>>> >>about
>>> >> >the hostname or IP address using Samza/Yarn/Kafka
>>> >> >2. Collect alert and metric events which contain either hostname
>>>or IP
>>> >> >address
>>> >> >3. Append contextual information to the alert and metric and
>>>insert to
>>> >>a
>>> >> >Kafka queue from which other subscribers read off of.
>>> >> >
>>> >> >Can you please shed some light on
>>> >> >
>>> >> >1. Is this feasible?
>>> >> >2. Am I on the right thought process
>>> >> >3. How do I start
>>> >> >
>>> >> >I now have 1 & 2 of them working disparately. I need to integrate
>>> them.
>>> >> >
>>> >> >Appreciate any input.
>>> >> >
>>> >> >- Shekar
>>> >>
>>> >>
>>>
>>>
>>


Re: Samza as a Caching layer

Posted by Shekar Tippur <ct...@gmail.com>.
Chris,

I have gone thro the documentation and decided that the option that is most
suitable for me is stream-table.

I see the following things:

1. Point samza to a table (database)
2. Point Samza to a stream - Alert stream from different sources
3. Join key like a hostname

I have Hello Samza working. To extend that to do what my needs are, I am
not sure where to start (Needs more code change OR configuration changes OR
both)?

I have gone thro
http://samza.incubator.apache.org/learn/documentation/latest/api/overview.html

Is my thought process on the right track? Can you please point me to the
right direction?

- Shekar



On Thu, Aug 21, 2014 at 1:08 PM, Shekar Tippur <ct...@gmail.com> wrote:

> Chris,
>
> This is perfectly good answer. I will start poking more into option #4.
>
> - Shekar
>
>
> On Thu, Aug 21, 2014 at 1:05 PM, Chris Riccomini <
> criccomini@linkedin.com.invalid> wrote:
>
>> Hey Shekar,
>>
>> Your two options are really (3) or (4), then. You can either run some
>> external DB that holds the data set, and you can query it from a
>> StreamTask, or you can use Samza's state store feature to push data into a
>> stream that you can then store in a partitioned key-value store along with
>> your StreamTasks. There is some documentation here about the state store
>> approach:
>>
>>
>> http://samza.incubator.apache.org/learn/documentation/0.7.0/container/state
>> -management.html
>> <http://samza.incubator.apache.org/learn/documentation/0.7.0/container/state-management.html>
>>
>>
>> (4) is going to require more up front effort from you, since you'll have
>> to understand how Kafka's partitioning model works, and setup some
>> pipeline to push the updates for your state. In the long run, I believe
>> it's the better approach, though. Local lookups on a key-value store
>> should be faster than doing remote RPC calls to a DB for every message.
>>
>> I'm sorry I can't give you a more definitive answer. It's really about
>> trade-offs.
>>
>> Cheers,
>> Chris
>>
>> On 8/21/14 12:22 PM, "Shekar Tippur" <ct...@gmail.com> wrote:
>>
>> >Chris,
>> >
>> >A big thanks for a swift response. The data set is huge and the frequency
>> >is in burst.
>> >What do you suggest?
>> >
>> >- Shekar
>> >
>> >
>> >On Thu, Aug 21, 2014 at 11:05 AM, Chris Riccomini <
>> >criccomini@linkedin.com.invalid> wrote:
>> >
>> >> Hey Shekar,
>> >>
>> >> This is feasible, and you are on the right thought process.
>> >>
>> >> For the sake of discussion, I'm going to pretend that you have a Kafka
>> >> topic called "PageViewEvent", which has just the IP address that was
>> >>used
>> >> to view a page. These messages will be logged every time a page view
>> >> happens. I'm also going to pretend that you have some state called
>> >>"IPGeo"
>> >> (e.g. The maxmind data set). In this example, we'll want to join the
>> >> long/lat geo information from IPGeo to the PageViewEvent, and send it
>> >>to a
>> >> new topic: PageViewEventsWithGeo.
>> >>
>> >> You have several options on how to implement this example.
>> >>
>> >> 1. If your joining data set (IPGeo) is relatively small and changes
>> >> infrequently, you can just pack it up in your jar or .tgz file, and
>> open
>> >> it open in every StreamTask.
>> >> 2. If your data set is small, but changes somewhat frequently, you can
>> >> throw the data set on some HTTP/HDFS/S3 server somewhere, and have your
>> >> StreamTask refresh it periodically by re-downloading it.
>> >> 3. You can do remote RPC calls for the IPGeo data on every page view
>> >>event
>> >> by query some remote service or DB (e.g. Cassandra).
>> >> 4. You can use Samza's state feature to set your IPGeo data as a series
>> >>of
>> >> messages to a log-compacted Kafka topic
>> >> (https://cwiki.apache.org/confluence/display/KAFKA/Log+Compaction),
>> and
>> >> configure your Samza job to read this topic as a bootstrap stream
>> >> (
>> >>
>> >>
>> http://samza.incubator.apache.org/learn/documentation/0.7.0/container/str
>> >>e
>> >> ams.html).
>> >>
>> >> For (4), you'd have to partition the IPGeo state topic according to the
>> >> same key as PageViewEvent. If PageViewEvent were partitioned by, say,
>> >> member ID, but you want your IPGeo state topic to be partitioned by IP
>> >> address, then you'd have to have an upstream job that re-partitioned
>> >> PageViewEvent into some new topic by IP address. This new topic will
>> >>have
>> >> to have the same number of partitions as the IPGeo state topic (if
>> IPGeo
>> >> has 8 partitions, then the new PageViewEventRepartitioned topic needs 8
>> >>as
>> >> well). This will cause your PageViewEventRepartitioned topic and your
>> >> IPGeo state topic to be aligned such that the StreamTask that gets page
>> >> views for IP address X will also have the IPGeo information for IP
>> >>address
>> >> X.
>> >>
>> >> Which strategy you pick is really up to you. :) (4) is the most
>> >> complicated, but also the most flexible, and most operationally sound.
>> >>(1)
>> >> is the easiest if it fits your needs.
>> >>
>> >> Cheers,
>> >> Chris
>> >>
>> >> On 8/21/14 10:15 AM, "Shekar Tippur" <ct...@gmail.com> wrote:
>> >>
>> >> >Hello,
>> >> >
>> >> >I am new to Samza. I have just installed Hello Samza and got it
>> >>working.
>> >> >
>> >> >Here is the use case for which I am trying to use Samza:
>> >> >
>> >> >
>> >> >1. Cache the contextual information which contains more information
>> >>about
>> >> >the hostname or IP address using Samza/Yarn/Kafka
>> >> >2. Collect alert and metric events which contain either hostname or IP
>> >> >address
>> >> >3. Append contextual information to the alert and metric and insert to
>> >>a
>> >> >Kafka queue from which other subscribers read off of.
>> >> >
>> >> >Can you please shed some light on
>> >> >
>> >> >1. Is this feasible?
>> >> >2. Am I on the right thought process
>> >> >3. How do I start
>> >> >
>> >> >I now have 1 & 2 of them working disparately. I need to integrate
>> them.
>> >> >
>> >> >Appreciate any input.
>> >> >
>> >> >- Shekar
>> >>
>> >>
>>
>>
>

Re: Samza as a Caching layer

Posted by Shekar Tippur <ct...@gmail.com>.
Chris,

This is perfectly good answer. I will start poking more into option #4.

- Shekar


On Thu, Aug 21, 2014 at 1:05 PM, Chris Riccomini <
criccomini@linkedin.com.invalid> wrote:

> Hey Shekar,
>
> Your two options are really (3) or (4), then. You can either run some
> external DB that holds the data set, and you can query it from a
> StreamTask, or you can use Samza's state store feature to push data into a
> stream that you can then store in a partitioned key-value store along with
> your StreamTasks. There is some documentation here about the state store
> approach:
>
> http://samza.incubator.apache.org/learn/documentation/0.7.0/container/state
> -management.html
>
>
> (4) is going to require more up front effort from you, since you'll have
> to understand how Kafka's partitioning model works, and setup some
> pipeline to push the updates for your state. In the long run, I believe
> it's the better approach, though. Local lookups on a key-value store
> should be faster than doing remote RPC calls to a DB for every message.
>
> I'm sorry I can't give you a more definitive answer. It's really about
> trade-offs.
>
> Cheers,
> Chris
>
> On 8/21/14 12:22 PM, "Shekar Tippur" <ct...@gmail.com> wrote:
>
> >Chris,
> >
> >A big thanks for a swift response. The data set is huge and the frequency
> >is in burst.
> >What do you suggest?
> >
> >- Shekar
> >
> >
> >On Thu, Aug 21, 2014 at 11:05 AM, Chris Riccomini <
> >criccomini@linkedin.com.invalid> wrote:
> >
> >> Hey Shekar,
> >>
> >> This is feasible, and you are on the right thought process.
> >>
> >> For the sake of discussion, I'm going to pretend that you have a Kafka
> >> topic called "PageViewEvent", which has just the IP address that was
> >>used
> >> to view a page. These messages will be logged every time a page view
> >> happens. I'm also going to pretend that you have some state called
> >>"IPGeo"
> >> (e.g. The maxmind data set). In this example, we'll want to join the
> >> long/lat geo information from IPGeo to the PageViewEvent, and send it
> >>to a
> >> new topic: PageViewEventsWithGeo.
> >>
> >> You have several options on how to implement this example.
> >>
> >> 1. If your joining data set (IPGeo) is relatively small and changes
> >> infrequently, you can just pack it up in your jar or .tgz file, and open
> >> it open in every StreamTask.
> >> 2. If your data set is small, but changes somewhat frequently, you can
> >> throw the data set on some HTTP/HDFS/S3 server somewhere, and have your
> >> StreamTask refresh it periodically by re-downloading it.
> >> 3. You can do remote RPC calls for the IPGeo data on every page view
> >>event
> >> by query some remote service or DB (e.g. Cassandra).
> >> 4. You can use Samza's state feature to set your IPGeo data as a series
> >>of
> >> messages to a log-compacted Kafka topic
> >> (https://cwiki.apache.org/confluence/display/KAFKA/Log+Compaction), and
> >> configure your Samza job to read this topic as a bootstrap stream
> >> (
> >>
> >>
> http://samza.incubator.apache.org/learn/documentation/0.7.0/container/str
> >>e
> >> ams.html).
> >>
> >> For (4), you'd have to partition the IPGeo state topic according to the
> >> same key as PageViewEvent. If PageViewEvent were partitioned by, say,
> >> member ID, but you want your IPGeo state topic to be partitioned by IP
> >> address, then you'd have to have an upstream job that re-partitioned
> >> PageViewEvent into some new topic by IP address. This new topic will
> >>have
> >> to have the same number of partitions as the IPGeo state topic (if IPGeo
> >> has 8 partitions, then the new PageViewEventRepartitioned topic needs 8
> >>as
> >> well). This will cause your PageViewEventRepartitioned topic and your
> >> IPGeo state topic to be aligned such that the StreamTask that gets page
> >> views for IP address X will also have the IPGeo information for IP
> >>address
> >> X.
> >>
> >> Which strategy you pick is really up to you. :) (4) is the most
> >> complicated, but also the most flexible, and most operationally sound.
> >>(1)
> >> is the easiest if it fits your needs.
> >>
> >> Cheers,
> >> Chris
> >>
> >> On 8/21/14 10:15 AM, "Shekar Tippur" <ct...@gmail.com> wrote:
> >>
> >> >Hello,
> >> >
> >> >I am new to Samza. I have just installed Hello Samza and got it
> >>working.
> >> >
> >> >Here is the use case for which I am trying to use Samza:
> >> >
> >> >
> >> >1. Cache the contextual information which contains more information
> >>about
> >> >the hostname or IP address using Samza/Yarn/Kafka
> >> >2. Collect alert and metric events which contain either hostname or IP
> >> >address
> >> >3. Append contextual information to the alert and metric and insert to
> >>a
> >> >Kafka queue from which other subscribers read off of.
> >> >
> >> >Can you please shed some light on
> >> >
> >> >1. Is this feasible?
> >> >2. Am I on the right thought process
> >> >3. How do I start
> >> >
> >> >I now have 1 & 2 of them working disparately. I need to integrate them.
> >> >
> >> >Appreciate any input.
> >> >
> >> >- Shekar
> >>
> >>
>
>

Re: Samza as a Caching layer

Posted by Chris Riccomini <cr...@linkedin.com.INVALID>.
Hey Shekar,

Your two options are really (3) or (4), then. You can either run some
external DB that holds the data set, and you can query it from a
StreamTask, or you can use Samza's state store feature to push data into a
stream that you can then store in a partitioned key-value store along with
your StreamTasks. There is some documentation here about the state store
approach:

http://samza.incubator.apache.org/learn/documentation/0.7.0/container/state
-management.html


(4) is going to require more up front effort from you, since you'll have
to understand how Kafka's partitioning model works, and setup some
pipeline to push the updates for your state. In the long run, I believe
it's the better approach, though. Local lookups on a key-value store
should be faster than doing remote RPC calls to a DB for every message.

I'm sorry I can't give you a more definitive answer. It's really about
trade-offs.

Cheers,
Chris

On 8/21/14 12:22 PM, "Shekar Tippur" <ct...@gmail.com> wrote:

>Chris,
>
>A big thanks for a swift response. The data set is huge and the frequency
>is in burst.
>What do you suggest?
>
>- Shekar
>
>
>On Thu, Aug 21, 2014 at 11:05 AM, Chris Riccomini <
>criccomini@linkedin.com.invalid> wrote:
>
>> Hey Shekar,
>>
>> This is feasible, and you are on the right thought process.
>>
>> For the sake of discussion, I'm going to pretend that you have a Kafka
>> topic called "PageViewEvent", which has just the IP address that was
>>used
>> to view a page. These messages will be logged every time a page view
>> happens. I'm also going to pretend that you have some state called
>>"IPGeo"
>> (e.g. The maxmind data set). In this example, we'll want to join the
>> long/lat geo information from IPGeo to the PageViewEvent, and send it
>>to a
>> new topic: PageViewEventsWithGeo.
>>
>> You have several options on how to implement this example.
>>
>> 1. If your joining data set (IPGeo) is relatively small and changes
>> infrequently, you can just pack it up in your jar or .tgz file, and open
>> it open in every StreamTask.
>> 2. If your data set is small, but changes somewhat frequently, you can
>> throw the data set on some HTTP/HDFS/S3 server somewhere, and have your
>> StreamTask refresh it periodically by re-downloading it.
>> 3. You can do remote RPC calls for the IPGeo data on every page view
>>event
>> by query some remote service or DB (e.g. Cassandra).
>> 4. You can use Samza's state feature to set your IPGeo data as a series
>>of
>> messages to a log-compacted Kafka topic
>> (https://cwiki.apache.org/confluence/display/KAFKA/Log+Compaction), and
>> configure your Samza job to read this topic as a bootstrap stream
>> (
>> 
>>http://samza.incubator.apache.org/learn/documentation/0.7.0/container/str
>>e
>> ams.html).
>>
>> For (4), you'd have to partition the IPGeo state topic according to the
>> same key as PageViewEvent. If PageViewEvent were partitioned by, say,
>> member ID, but you want your IPGeo state topic to be partitioned by IP
>> address, then you'd have to have an upstream job that re-partitioned
>> PageViewEvent into some new topic by IP address. This new topic will
>>have
>> to have the same number of partitions as the IPGeo state topic (if IPGeo
>> has 8 partitions, then the new PageViewEventRepartitioned topic needs 8
>>as
>> well). This will cause your PageViewEventRepartitioned topic and your
>> IPGeo state topic to be aligned such that the StreamTask that gets page
>> views for IP address X will also have the IPGeo information for IP
>>address
>> X.
>>
>> Which strategy you pick is really up to you. :) (4) is the most
>> complicated, but also the most flexible, and most operationally sound.
>>(1)
>> is the easiest if it fits your needs.
>>
>> Cheers,
>> Chris
>>
>> On 8/21/14 10:15 AM, "Shekar Tippur" <ct...@gmail.com> wrote:
>>
>> >Hello,
>> >
>> >I am new to Samza. I have just installed Hello Samza and got it
>>working.
>> >
>> >Here is the use case for which I am trying to use Samza:
>> >
>> >
>> >1. Cache the contextual information which contains more information
>>about
>> >the hostname or IP address using Samza/Yarn/Kafka
>> >2. Collect alert and metric events which contain either hostname or IP
>> >address
>> >3. Append contextual information to the alert and metric and insert to
>>a
>> >Kafka queue from which other subscribers read off of.
>> >
>> >Can you please shed some light on
>> >
>> >1. Is this feasible?
>> >2. Am I on the right thought process
>> >3. How do I start
>> >
>> >I now have 1 & 2 of them working disparately. I need to integrate them.
>> >
>> >Appreciate any input.
>> >
>> >- Shekar
>>
>>


Re: Samza as a Caching layer

Posted by Shekar Tippur <ct...@gmail.com>.
Chris,

A big thanks for a swift response. The data set is huge and the frequency
is in burst.
What do you suggest?

- Shekar


On Thu, Aug 21, 2014 at 11:05 AM, Chris Riccomini <
criccomini@linkedin.com.invalid> wrote:

> Hey Shekar,
>
> This is feasible, and you are on the right thought process.
>
> For the sake of discussion, I'm going to pretend that you have a Kafka
> topic called "PageViewEvent", which has just the IP address that was used
> to view a page. These messages will be logged every time a page view
> happens. I'm also going to pretend that you have some state called "IPGeo"
> (e.g. The maxmind data set). In this example, we'll want to join the
> long/lat geo information from IPGeo to the PageViewEvent, and send it to a
> new topic: PageViewEventsWithGeo.
>
> You have several options on how to implement this example.
>
> 1. If your joining data set (IPGeo) is relatively small and changes
> infrequently, you can just pack it up in your jar or .tgz file, and open
> it open in every StreamTask.
> 2. If your data set is small, but changes somewhat frequently, you can
> throw the data set on some HTTP/HDFS/S3 server somewhere, and have your
> StreamTask refresh it periodically by re-downloading it.
> 3. You can do remote RPC calls for the IPGeo data on every page view event
> by query some remote service or DB (e.g. Cassandra).
> 4. You can use Samza's state feature to set your IPGeo data as a series of
> messages to a log-compacted Kafka topic
> (https://cwiki.apache.org/confluence/display/KAFKA/Log+Compaction), and
> configure your Samza job to read this topic as a bootstrap stream
> (
> http://samza.incubator.apache.org/learn/documentation/0.7.0/container/stre
> ams.html).
>
> For (4), you'd have to partition the IPGeo state topic according to the
> same key as PageViewEvent. If PageViewEvent were partitioned by, say,
> member ID, but you want your IPGeo state topic to be partitioned by IP
> address, then you'd have to have an upstream job that re-partitioned
> PageViewEvent into some new topic by IP address. This new topic will have
> to have the same number of partitions as the IPGeo state topic (if IPGeo
> has 8 partitions, then the new PageViewEventRepartitioned topic needs 8 as
> well). This will cause your PageViewEventRepartitioned topic and your
> IPGeo state topic to be aligned such that the StreamTask that gets page
> views for IP address X will also have the IPGeo information for IP address
> X.
>
> Which strategy you pick is really up to you. :) (4) is the most
> complicated, but also the most flexible, and most operationally sound. (1)
> is the easiest if it fits your needs.
>
> Cheers,
> Chris
>
> On 8/21/14 10:15 AM, "Shekar Tippur" <ct...@gmail.com> wrote:
>
> >Hello,
> >
> >I am new to Samza. I have just installed Hello Samza and got it working.
> >
> >Here is the use case for which I am trying to use Samza:
> >
> >
> >1. Cache the contextual information which contains more information about
> >the hostname or IP address using Samza/Yarn/Kafka
> >2. Collect alert and metric events which contain either hostname or IP
> >address
> >3. Append contextual information to the alert and metric and insert to a
> >Kafka queue from which other subscribers read off of.
> >
> >Can you please shed some light on
> >
> >1. Is this feasible?
> >2. Am I on the right thought process
> >3. How do I start
> >
> >I now have 1 & 2 of them working disparately. I need to integrate them.
> >
> >Appreciate any input.
> >
> >- Shekar
>
>

Re: Samza as a Caching layer

Posted by Chris Riccomini <cr...@linkedin.com.INVALID>.
Hey Shekar,

This is feasible, and you are on the right thought process.

For the sake of discussion, I'm going to pretend that you have a Kafka
topic called "PageViewEvent", which has just the IP address that was used
to view a page. These messages will be logged every time a page view
happens. I'm also going to pretend that you have some state called "IPGeo"
(e.g. The maxmind data set). In this example, we'll want to join the
long/lat geo information from IPGeo to the PageViewEvent, and send it to a
new topic: PageViewEventsWithGeo.

You have several options on how to implement this example.

1. If your joining data set (IPGeo) is relatively small and changes
infrequently, you can just pack it up in your jar or .tgz file, and open
it open in every StreamTask.
2. If your data set is small, but changes somewhat frequently, you can
throw the data set on some HTTP/HDFS/S3 server somewhere, and have your
StreamTask refresh it periodically by re-downloading it.
3. You can do remote RPC calls for the IPGeo data on every page view event
by query some remote service or DB (e.g. Cassandra).
4. You can use Samza's state feature to set your IPGeo data as a series of
messages to a log-compacted Kafka topic
(https://cwiki.apache.org/confluence/display/KAFKA/Log+Compaction), and
configure your Samza job to read this topic as a bootstrap stream
(http://samza.incubator.apache.org/learn/documentation/0.7.0/container/stre
ams.html). 

For (4), you'd have to partition the IPGeo state topic according to the
same key as PageViewEvent. If PageViewEvent were partitioned by, say,
member ID, but you want your IPGeo state topic to be partitioned by IP
address, then you'd have to have an upstream job that re-partitioned
PageViewEvent into some new topic by IP address. This new topic will have
to have the same number of partitions as the IPGeo state topic (if IPGeo
has 8 partitions, then the new PageViewEventRepartitioned topic needs 8 as
well). This will cause your PageViewEventRepartitioned topic and your
IPGeo state topic to be aligned such that the StreamTask that gets page
views for IP address X will also have the IPGeo information for IP address
X.

Which strategy you pick is really up to you. :) (4) is the most
complicated, but also the most flexible, and most operationally sound. (1)
is the easiest if it fits your needs.

Cheers,
Chris

On 8/21/14 10:15 AM, "Shekar Tippur" <ct...@gmail.com> wrote:

>Hello,
>
>I am new to Samza. I have just installed Hello Samza and got it working.
>
>Here is the use case for which I am trying to use Samza:
>
>
>1. Cache the contextual information which contains more information about
>the hostname or IP address using Samza/Yarn/Kafka
>2. Collect alert and metric events which contain either hostname or IP
>address
>3. Append contextual information to the alert and metric and insert to a
>Kafka queue from which other subscribers read off of.
>
>Can you please shed some light on
>
>1. Is this feasible?
>2. Am I on the right thought process
>3. How do I start
>
>I now have 1 & 2 of them working disparately. I need to integrate them.
>
>Appreciate any input.
>
>- Shekar