[ 
https://issues.apache.org/jira/browse/SAMZA-348?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14133423#comment-14133423
 ] 

Chris Riccomini commented on SAMZA-348:
---------------------------------------

bq. It might be a good idea to have this as an interface named "ConfigState" 
(or something like that) instead of "ConfigStream", unless there's a specific 
feature of this that can only exist when it's backed by something similiar to a 
Kafka stream (instead of Zookeeper for example).

We started by referring to this as state, but there was strong feedback from 
[~martinkl] not to use the word "state" again, since it's already overloaded, 
and used for Samza's state management feature. The other idea was "ConfigLog".

bq. Along that line of thought, does this avoid using Zookeeper to avoid a new 
dependency? Or for some other reason?

The two reasons for not using ZK are:

# Avoid adding a new dependency. Thus far, we've gotten by without it.
# If we use Kafka as the system, it lets us transactionally store offset 
checkpoints in the ConfigStream. If we have two systems (ZK, and Kafka), then 
we can no longer transactionally commit everything at once (offset checkpoint, 
output, state changelog, etc).

Regarding (2), it looks more and more like it will be a requirement for 
transactionality to have the ConfigStream and a job's output streams go to the 
same underlying system. It's not necessarily the case that only Kafka can 
support this. I believe HBase would work as well.

bq. Is the intent for this API to allow changing the config without restarting 
the AM?
Yes, this is the intent. In its most naive form, the flow looks something like:

# AM updates config that's exposed via HTTP JSON API.
# AM kills all existing containers.
# AM brings up new containers.
# New containers query AM's HTTP JSON API, which has new configs.

bq. If so, this might be a nice way to manage several Samza jobs with an 
external system (perhaps one with a nice web UI, like a Mesos meta-scheduler).

I hadn't considered this in detail, but it seems like it might work. I think as 
long as the HTTP JSON API is well defined, I don't the an individual 
SamzaContainer should care about whether the API it's querying is just managing 
one job, or many jobs.

> Configure Samza jobs through a stream
> -------------------------------------
>
>                 Key: SAMZA-348
>                 URL: https://issues.apache.org/jira/browse/SAMZA-348
>             Project: Samza
>          Issue Type: Bug
>    Affects Versions: 0.7.0
>            Reporter: Chris Riccomini
>              Labels: project
>         Attachments: DESIGN-SAMZA-348-0.md, DESIGN-SAMZA-348-0.pdf
>
>
> Samza's existing config setup is problematic for a number of reasons:
> # It's completely immutable once a job starts. This prevents any dynamic 
> reconfiguration and auto-scaling. It is debatable whether we want these 
> feature or not, but our existing implementation actively prevents it. See 
> SAMZA-334 for discussion.
> # We pass existing configuration through environment variables. YARN exports 
> environment variables in a shell script, which limits the size to the varargs 
> length on the machine. This is usually ~128KB. See SAMZA-333 and SAMZA-337 
> for details.
> # User-defined configuration (the Config object) and programmatic 
> configuration (checkpoints and TaskName:State mappings (see SAMZA-123)) are 
> handled differently. It's debatable whether this makes sense.
> In SAMZA-123, [~jghoman] and I propose implementing a ConfigLog. This log 
> would replace both the checkpoint topic and the existing config environment 
> variables in SamzaContainer and Samza's YARN AM.
> I'd like to keep this ticket's scope limited to just the implementation of 
> the ConfigLog, and not re-designing how Samza's config is used in the code 
> (SAMZA-40). We should, however, discuss how this feature would affect dynamic 
> reconfiguration/auto-scaling.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to