Re: [DISCUSS] Samza 1.1.0 release

2019-03-07 Thread Jacob Maes
I think the schedule sounds good. A release would be great. On Thu, Mar 7, 2019 at 10:38 AM Prateek Maheshwari wrote: > Daniel, let's try to include the following change in the release as well. > SAMZA-2116: Make sendTo and sink operators non-terminal > > Other than that, +1 (binding). > > -

Re: [VOTE] SEP-12: Integration Test Framework

2018-05-17 Thread Jacob Maes
+1 On Thu, May 17, 2018 at 9:56 AM, Jagadish Venkatraman < jagadish1...@gmail.com> wrote: > Thanks Sanil for the proposal. This will go a long way in simplifying > testing of Samza applications. > > +1 (binding) > > > > On Thu, May 17, 2018 at 9:45 AM, Daniel Nishimura >

Re: Question about auto created Kafka topics

2017-12-14 Thread Jacob Maes
Samza has configurations for the replication factors used to create various topics. Here is how to set each one: *Checkpoint* task.checkpoint.replication.factor -> systems..default.stream.replication.factor -> 3 Which means, if "task.checkpoint.replication.factor" is configured, it is used. Else

Re: [DISCUSS] Samza 0.14.0 release

2017-11-28 Thread Jacob Maes
+1 On Mon, Nov 27, 2017 at 8:15 PM, Fred Haifeng Ji wrote: > +1! Thanks Bharath! > > Fred > > On Mon, Nov 27, 2017 at 11:10 AM, Yi Pan wrote: > > > Thanks for driving this! +1 > > > > A few minor things that are pending that I think we should pull in:

Re: Per task/topic checkpoint?

2017-11-01 Thread Jacob Maes
compression - Prometheus kind of compression techniques), > whereas configuration messages are much more expensive to process and so I > want to commit any changes due to these more frequently - say every 2 mins > and checkpointing the associated topic. > > On Wed, 1 Nov 2017 at 9:11

Re: Per task/topic checkpoint?

2017-11-01 Thread Jacob Maes
Hey Gaurav, Samza automatically keeps track of the offsets your job has successfully processed for each SSP. When your task requests a checkpoint, Samza will write the offset of the latest successfully-processed message for each SSP that task consumes. So if task0 consumes partition 0 of two

Re: Issue with TopicExistsException in 0.13.0

2017-08-15 Thread Jacob Maes
nHgQFe8bv7-emnODgdhciwPkVKB_BE-ZnZmhwA18Q7rimVruRFx5g0vsvC9cG > t2jrAYfAucx0goYepLp8ZyfPAPxCv0Xh9CQVXTrqVMnByrbWTNcczkXashg2 > zljIWFPYiRKbG_5H2BvM~ > > So, you'll want to be using kafka 0.10.1.1. > > On Mon, Aug 14, 2017 at 2:00 PM, Jacob Maes <jacob.m...@gmail.com> wro

Re: Issue with TopicExistsException in 0.13.0

2017-08-14 Thread Jacob Maes
-2beed65293202568c54b61109564ccd1R29 So, you'll want to be using kafka 0.10.1.1. On Mon, Aug 14, 2017 at 2:00 PM, Jacob Maes <jacob.m...@gmail.com> wrote: > Hey Jeremiah, > > It looks like the TopicExistsException should be handled by the system > admin and not rethrown: > https://github.

Re: [Discuss] Samza 0.13.1 release

2017-08-11 Thread Jacob Maes
Looks good! +1 On Thu, Aug 10, 2017 at 6:53 PM, Jagadish Venkatraman < jagadish1...@gmail.com> wrote: > +1 for the release. thanks for the summary and for driving this Fred! > > On Thu, Aug 10, 2017 at 5:15 PM Fred Haifeng Ji > wrote: > > > The format was messed up when

Re: [Discuss] Samza 0.13.1 release

2017-08-10 Thread Jacob Maes
Hey Fred, The email seems to have some whitespace issues. Can you try sending it again? Also, I'd like to include the patch for https://issues.apache.org/ jira/browse/SAMZA-1387 in the release. The PR is up for review now. Please take a look and let me know what you think. -Jake On Thu, Aug

Re: Re[2]: IncomingMessageEnvelope - memory leak

2017-06-30 Thread Jacob Maes
On Fri, Jun 30, 2017 at 7:25 AM, Evgeny wrote: > Sorry didn't understand this question: "Is the container able to keep up > with its inputs?" What do you mean keep up with its inputs? > Apologies for the vague phrasing. I meant to ask if the input topics have a high volume of

Re: IncomingMessageEnvelope - memory leak

2017-06-29 Thread Jacob Maes
Hey Evgeny, We haven't seen this with our jobs, but here are some thoughts and questions for you to consider. Is the container able to keep up with its inputs? The further behind, the more it will fill buffers. Samza has one buffer, which is controlled by configs

Re: [DISCUSS] SEP-5: Enable partition expansion of input streams

2017-06-15 Thread Jacob Maes
ng extra latency/cost and without requiring users to change their > code. And we can recommend user to use the extra partitioning stage if the > coordination among different organization is indeed a concern. > > Can other developers also provide feedback regarding your preference >

Re: [DISCUSS] SEP-5: Enable partition expansion of input streams

2017-06-13 Thread Jacob Maes
point was that the partition->task mapping is not sufficient by itself. So adding it by itself is potentially misleading. On Mon, Jun 12, 2017 at 8:34 PM, Dong Lin <lindon...@gmail.com> wrote: > Thanks for the reply Jacob. Please see my comment inline. > > On Mon, Jun 12, 201

Re: [DISCUSS] SEP-5: Enable partition expansion of input streams

2017-06-12 Thread Jacob Maes
e some definition of the key-to-partition > assignments if Samza decides to get and use this information in the > future. Can > you be more specific why Partition-to-task mapping is not meaningful > without > some definition of the key-to-partition assignments and why it

Re: [DISCUSS] SEP-5: Enable partition expansion of input streams

2017-06-12 Thread Jacob Maes
to-task mapping. And this information may be useful for other > use-case such as debugging. > > - By having the new interface take the previous task-to-partition > assignment instead of a topic-to-partition-count mapping as new parameter, > we can potentially have grouper implementation

Re: [VOTE] Apache Samza 0.13.0 RC6

2017-06-06 Thread Jacob Maes
+1 (non-binding) Built and tested on both OSX and RHEL with gradle 2.0 and 2.2 respectively. Also verified the high level API + YARN host affinity on a test job with 32 containers. On Tue, Jun 6, 2017 at 9:14 AM, xinyu liu wrote: > +1 (non-binding). > > Downloaded the

Re: [DISCUSS] SEP-5: Enable partition expansion of input streams

2017-06-05 Thread Jacob Maes
Hey Dong, Thanks for the SEP. Supporting partition changes is critically important for stateful Samza jobs, so it's great to see some ideas on that front! Sorry for the late feedback, but I have a few thoughts to contribute. Big +1 on Navina's comment: > My biggest gripe with this SEP is that

Re: [VOTE] Apache Samza 0.13.0 RC2

2017-06-01 Thread Jacob Maes
Verified PGP key The build and check-all scripts were successful for me. I had trouble with the integration tests. Looks like `./samza-test/src/main/python/configs/tests.json` needs to be updated s.t. '"samza_executable": "samza-test_2.11-0.13.0.tgz"' On Thu, Jun 1, 2017 at 11:55 AM, Boris S

Re: About examples of Samza applications

2017-05-25 Thread Jacob Maes
Hello Li, The hello-samza example is a great way to start with Samza. You can access the tutorial here: http://samza.apache.org/startup/hello-samza/latest/ It describes how to run and verify 3 jobs on samza which process events from Wikipedia. The source code for the jobs are available on

Re: [DISCUSS] Samza 0.13.0 release

2017-05-08 Thread Jacob Maes
+1 on the release. The plan looks good. It goes without saying that documentation is also required. SAMZA-1234 is the umbrella ticket for that. On Fri, May 5, 2017 at 4:42 PM, Prateek Maheshwari < pmaheshw...@linkedin.com.invalid> wrote: > Hi all, > > There have been quite a lot of new

Re: [VOTE] SEP 3 : Heart-beat mechanism between JobCoordinator and all running containers

2017-05-03 Thread Jacob Maes
+1 non-binding I think this will go a long way toward improving samza's reliability on YARN. On Wed, May 3, 2017 at 11:46 AM, Abhishek Shivanna wrote: > Hey everyone, > > This is the voting thread for SEP 3: Heart-beat mechanism between > JobCoordinator and all running

Re: [DISCUSS] SEP-2: ApplicationRunner Design

2017-04-20 Thread Jacob Maes
Thanks for the SEP! +1 on introducing these new components -1 on the current definition of their roles (see Design feedback below) *Design* - If LocalJobRunner and RemoteJobRunner handle the different methods of launching a Job, what additional value do the different types of

Re: Steps to Upgrading Samza (0.9 to 0.12)

2017-03-27 Thread Jacob Maes
As I recall, samza 0.10 introduced the coordinator stream and there was code to do an automatic migration to use that feature. @navina, @yi, do you know if that migration code is still in samza 12? If not, then it's probably better to update from 0.9.1 to 0.10.0 and then to 0.12.0. I don't think

Re: [VOTE] Apache Samza 0.12.0 RC2

2017-02-08 Thread Jacob Maes
Build and integration tests were successful for me. +1 non-binding On Wed, Feb 8, 2017 at 4:48 PM, xinyu liu wrote: > Ran build, checkAll and integration tests. All passed. > > +1 non-binding. > > Thanks, > Xinyu > > On Wed, Feb 8, 2017 at 4:18 PM, Boris S

Re: [VOTE] Apache Samza 0.12.0 RC0

2017-02-02 Thread Jacob Maes
The build and check-all scripts were successful for me. I had trouble with the integration tests. Looks like `./samza-test/src/main/python/configs/tests.json` needs to be updated s.t. '"samza_executable": "samza-test_2.11-0.12.0.tgz"' After that, the integration tests were successful. Also,

Re: [DISCUSS] [VOTE] Apache Samza 0.11.0 RC0

2016-10-04 Thread Jacob Maes
+1 (non-binding) Downloaded and bin/check-all.sh on OSX Downloaded, built and ran unit tests on RHEL I got a checkstyle error when I tried to run bin/check-all.sh on RHEL, but I think it's something environmental. Looks good. On Mon, Oct 3, 2016 at 4:19 PM, Jagadish Venkatraman

Re: Review for SAMZA-1012

2016-10-04 Thread Jacob Maes
I'm on it! On Tue, Oct 4, 2016 at 6:22 AM, Tommy Becker wrote: > Could I bother a committer for a review of this patch? > > > https://reviews.apache.org/r/51689/ > > -- > Tommy Becker > Senior Software Engineer > > Digitalsmiths > A TiVo Company > >

Re: Job coordinator stream and job redeployment

2016-08-25 Thread Jacob Maes
Hey David, Yes, the config is rewritten by the JobRunner each time the job is restarted as dictated by the "resetJobConfig" property (which defaults to true) here:

Re: Debug Samza consumer lag issue

2016-08-24 Thread Jacob Maes
tion most of the time, maybe I can decrease the # of containers and > save some resources? > > Thanks, > David > > On Wed, Aug 24, 2016 at 1:27 PM Jacob Maes <jacob.m...@gmail.com> wrote: > > > > > > > Based on what you have described, the following shoul

Re: Debug Samza consumer lag issue

2016-08-24 Thread Jacob Maes
So we should see pretty > consistent process-ns (which we do). > > Based on what you have described, the following should be true in 0.10.1: > event-loop-ns = choose-ns + process-ns + window-ns (if necessary) + > commit-ns (if necessary) > > Is this correct? > Thanks, > David &g

Re: Debug Samza consumer lag issue

2016-08-24 Thread Jacob Maes
om/s/06n3awdwn8ntfxd/Screenshot%202016 >> -08-24%2000.26.07.png?dl=0>. >> > Not sure what would be the cause of this. >> > 2. We are seeing very different process-envelopes values across >> containers >> > <https://www.dropbox.com/s/n1wxtngquv607nb/Screens

Re: Debug Samza consumer lag issue

2016-08-24 Thread Jacob Maes
.21.05.png?dl=0> > > . > > > > > > > > On Tue, Aug 23, 2016 at 5:56 PM David Yu <david...@optimizely.com> > wrote: > > > >> Hi, Jake, > >> > >> Thanks for your suggestions. Some of my answers inline: > >> >

Re: Latest hello-samza demo not compiling?

2016-08-12 Thread Jacob Maes
/samza/samza-api/0.11.0-SNAPSHOT/samza-api-0.11.0-SNAPSHOT.pom Still, I will file a ticket. Thanks for reporting the issue! On Fri, Aug 12, 2016 at 11:12 AM, Jacob Maes <jacob.m...@gmail.com> wrote: > It does seem that the instructions need some clarification. > > They ce

Re: Latest hello-samza demo not compiling?

2016-08-12 Thread Jacob Maes
oes seem that the instructions need some clarification. Or > maybe just my brain? > > -rb > > On Fri, Aug 12, 2016 at 11:01 AM, Jacob Maes <jacob.m...@gmail.com> wrote: > > > Hey Ryan, > > > > There are a couple things I noticed at first glance: > > 1

Re: Samza yarn job - cannot bind to local host

2016-08-10 Thread Jacob Maes
PM, Shekar Tippur <ctip...@gmail.com> wrote: > > > Thanks a lot guys. It was a system problem. /etc/hosts file had wrong > > permissions. localhost was being resolved to a weird ip for certain users > > via DNS. > > - Shekar > > > > On Fri, Aug 5, 2016 at 11:

Re: State store changelog format

2016-08-08 Thread Jacob Maes
; > Is this a 0.8.x.x Kafka producer limitation? > > On Fri, Aug 5, 2016 at 4:25 PM Jacob Maes <jacob.m...@gmail.com> wrote: > > > Hey David, > > > > If you check your container logs Kafka should print the producer config > > > > >

Re: Samza yarn job - cannot bind to local host

2016-08-05 Thread Jacob Maes
Hey Shekar, There's currently no way to force the port or disable the JMXServer. We've seen port conflicts (fixed in 10.1) and connection issues due to VPN but no connection issue for other reasons. While googling that exception, I found a couple cases without clear resolutions, but they were

Re: Samza yarn job - cannot bind to local host

2016-08-04 Thread Jacob Maes
Hey Shekar, Are your YARN hosts connected to a VPN by chance? The only time I've seen problems with the localhost was when I ran unit tests on my Mac while connected to VPN. Apparently, its related to a bug in Java http://bugs.java.com/view_bug.do?bug_id=7180557 -Jake On Thu, Aug 4, 2016 at

Re: State store changelog format

2016-08-03 Thread Jacob Maes
Hey David, what gets written to the changelog topic The changelog gets the same value as the store, which is the serialized form of the key and value. The serdes for the store are configured with the properties: stores.store-name.key.serde stores.store-name.msg.serde If I want to compress the

Re: [DISCUSS] [VOTE] Apache Samza 0.10.1 RC0

2016-08-01 Thread Jacob Maes
+1 non-binding Lots of good fixes in this release. On Mon, Aug 1, 2016 at 11:41 AM, Jagadish Venkatraman < jagadish1...@gmail.com> wrote: > Thanks for driving this release.+1 approve (non-binding) > > Thanks, > Jagadish > > On Mon, Aug 1, 2016 at 11:39 AM, Navina Ramesh >

Re: [NEED COMMENTS] import-control & checkstyle plugin

2016-07-11 Thread Jacob Maes
I don't particularly mind that it doesn't get cleaned up. Theoretically if it was once reasonable for one class/package to be referenced within another, it will continue to be reasonable, even if the code no longer makes the reference. That said, import control has been a pain every time we add

Re: flushing changelog & checkpointing

2016-07-11 Thread Jacob Maes
Hey Ramanan, Confirmed. It all happens in commit (at the "checkpoint interval") The operations are executed serially for each task. The order is exactly as Yi described. The order was chosen with crashes in mind. That is, the checkpoint is not written until state has been updated and output

Re: java.rmi.server.ExportException: Port already in use

2016-06-13 Thread Jacob Maes
.ms > ​ > window. Increasing yarn.container.retry.count​seems to solve the issue for > the time being. > It still remains unknown why my tasks are failing so often. There is no > error messages in the log to help... > > Thanks, > Jack > > > On Fri, Jun 10, 2016 at 1:2

Re: java.rmi.server.ExportException: Port already in use

2016-06-10 Thread Jacob Maes
Hey Jack, The port collisions are fixed in https://issues.apache.org/jira/browse/SAMZA-932. However, they usually don't occur frequently enough to cause the job to fail. Usually only a couple containers fail with this exception but the job restarts them and eventually runs smoothly. Your GC log

Re: Samza job killed by left orphaned on YARN

2016-05-18 Thread Jacob Maes
NM is running on the node: > > > > $ ps aux | grep java > > ... > > yarn 25623 0.5 0.8 2366536 275488 ? Sl May17 7:04 > > /usr/java/jdk1.8.0_51/bin/java -Dproc_nodemanager > > ... org.apache.hadoop.yarn.server.nodemanager.NodeManager > &g

Re: Howto Use The Elasticsearch Producer

2016-05-10 Thread Jacob Maes
ed the typo (in bold red), and that solved the problem :) > > Cheers > > Tim > > > On 10 May 2016 at 09:08, Jacob Maes <jacob.m...@gmail.com> wrote: > > > Hey Tim, > > > > Can you include the full log? Since the producer is not showing up,

Re: Howto Use The Elasticsearch Producer

2016-05-10 Thread Jacob Maes
Hey Tim, Can you include the full log? Since the producer is not showing up, I'd expect to see some earlier messages explaining why. In particular, this section of code from SamzaContainer.scala should print errors instantiating the producers: val producers = systemFactories > .map { >

Re: How to implement broadcasted KeyValueStorage with 0.9

2016-05-03 Thread Jacob Maes
This sounds like SAMZA-402 (https://issues.apache.org/jira/browse/SAMZA-402) which, unfortunately, hasn't been implemented yet. On Tue, May 3, 2016 at 7:48 AM, Bae, Jae Hyeon wrote: > Can we make a kafka topic as KV store globally to all tasks, not > partitioned? For

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

2016-05-02 Thread Jacob Maes
their runtime JVM version to JDK8 easily, the > > > latest coming release will still be on JDK7. Question is: how long > should > > > we hold back in waiting for this upgrade? > > > > > > Thanks! > > > > > > -Yi > > > > > &

[Discuss] Moving Samza to Java 1.8 source compatibility.

2016-04-27 Thread Jacob Maes
Hey everyone, I wanted to start a discussion to see what folks think about moving to Java 1.8 source compatibility at some point after the 10.1 release. Java 8 has a number of nice features that can help us build more concise, maintainable, and robust software. A few notable features that would

Re: Running hello-samza for the first time

2016-03-15 Thread Jacob Maes
Hey Maurice, You're right about getOrDefault and it looks like it was my mistake. I'll file a JIRA and patch shortly. Apologies for the trouble. -Jake On Tue, Mar 15, 2016 at 1:08 PM, Maurice McCabe wrote: > Hi Jagadish, > > As I recall compiling samza (with JDK7) during >

Re: checkpoint example?

2016-03-01 Thread Jacob Maes
A couple notes that may be helpful: 1. When you have a stateful processor that dies, the changelog is the default means by which the state is restored. Change logging is enabled with this config: stores.store-name.changelog 2. If, when the job comes back up, it needs to reprocess historical

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

2016-02-18 Thread Jacob Maes
+1 As a relatively new contributor to Samza, I've certainly felt the current process was overly-complicated. On Thu, Feb 18, 2016 at 3:53 PM, Yi Pan wrote: > Hi, all, > > I want to start the discussion on our code review/commit process. > > I felt that our code review and

Re: Zombie writers protection

2016-02-10 Thread Jacob Maes
Hey Rick, If I understand your question, the goal is really to make sure there are no orphaned containers that continue to run "off the books". The newly added SAMZA-871 describes a heart beat mechanism to make sure orphaned containers actually get killed. Also, the YARN Node Manager Restart

[DISCUSS] (SAMZA-865) REST API for starting and stopping Samza jobs

2016-01-27 Thread Jacob Maes
Hey folks, I wanted to solicit feedback on the design for a REST API for starting and stopping jobs. At LinkedIn, this API is going to help us integrate other services/features with Samza and provide a cluster-management-agnostic means of managing Samza jobs. As mentioned in the design doc,

Re: [DISCUSS] (SAMZA-865) REST API for starting and stopping Samza jobs

2016-01-27 Thread Jacob Maes
On Wed, Jan 27, 2016 at 5:34 PM, Jacob Maes <jacob.m...@gmail.com> wrote: > Hey folks, > > I wanted to solicit feedback on the design for a REST API for starting and > stopping jobs. > > At LinkedIn, this API is going to help us integrate other > services/features with

Re: Hello Samza repository + Data Masking

2015-12-23 Thread Jacob Maes
I agree with Jakob. I'll add that for some users it is useful to see the gradle/maven configuration of the samza dependency in an external project, which will most closely reflect how they have to configure their own project. That way they can just copy the snippet without any additional knowledge

Re: [VOTE] Samza 0.10.0 Release Candidate 2

2015-12-09 Thread Jacob Maes
+1 [non binding]. Ran > ./gradlew build, > ./bin/check-all.sh, and > ./bin/integration-tests.sh /tmp/samza-tests from my Mac and all were successful. I saw some checkstyle errors when I used gradle 2.9, but it was fine with gradle 2.2 and 2.3. I created SAMZA-837 for gradle 2.9 compatibility.