You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@samza.apache.org by Tony Du <th...@gmail.com> on 2017/10/30 21:02:24 UTC

Samza questions (downtime during deployment and num partition per task)

Hi, we're looking into Samza for doing real-time processing. We have couple
questions w.r.t Samza functionality

1. One "must-have" requirement for us is zero/minimal downtime during
deployment of Samza jobs. One approach that we're thinking of is to start a
new instance of the same Samza job and make sure it's running before take
down the running one. Is there any problem to this approach? If so, is
there any suggestion for this problem as I'm unable to find anything
related to this in documentation.

2. Is it possible to configure multiple Kafka topic partitions to a single
task instance within a job? Since each task has its own JVM and some of the
job that we have use a lot of memory, it would be very wasteful to run many
instances of them

Thanks much for your help!

------------------------------------------------------------------------------
Tony T Du
Sr. Software Engineer - Dataminr

I hear I forget. I see I remember. I do I understand.
------------------------------------------------------------------------------

Re: Samza questions (downtime during deployment and num partition per task)

Posted by xinyu liu <xi...@gmail.com>.
Hi, Tony,

For your questions:

1) Having a hot-standby job instance for fail-over may introduce certain
operational complications. For example, if they produce to the same output,
then both will be running in a short period of time, which might lead to
duplicates in output. If the jobs has local state, it will be more
complicated to continue using the previous states from the active job. So
if your job doesn't have state and can handle duplicates, this might work.
Another option I can think of is to use the zk-based Samza deployment. This
is a new feature we added to Samza. The description is
http://samza.apache.org/startup/preview/#flexible-deployment-model, and
there is a hello-samza example for it:
http://samza.apache.org/learn/tutorials/latest/hello-samza-high-level-zk.html.
For zk-based deployment, we support rolling bounce so you can upgrade your
container one at a time.

2) Yes, you can implement your own SystemStreamPartitionGrouper if you want
a static assignment (from config). For example, please take a look at
AllSspToSingleTaskGrouper which groups all system stream partitions into a
single task.

Thanks,
Xinyu

On Mon, Oct 30, 2017 at 2:02 PM, Tony Du <th...@gmail.com> wrote:

> Hi, we're looking into Samza for doing real-time processing. We have couple
> questions w.r.t Samza functionality
>
> 1. One "must-have" requirement for us is zero/minimal downtime during
> deployment of Samza jobs. One approach that we're thinking of is to start a
> new instance of the same Samza job and make sure it's running before take
> down the running one. Is there any problem to this approach? If so, is
> there any suggestion for this problem as I'm unable to find anything
> related to this in documentation.
>
> 2. Is it possible to configure multiple Kafka topic partitions to a single
> task instance within a job? Since each task has its own JVM and some of the
> job that we have use a lot of memory, it would be very wasteful to run many
> instances of them
>
> Thanks much for your help!
>
> ------------------------------------------------------------
> ------------------
> Tony T Du
> Sr. Software Engineer - Dataminr
>
> I hear I forget. I see I remember. I do I understand.
> ------------------------------------------------------------
> ------------------
>