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

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

bq. are we talking about embedding a Kafka consumer inside the YARN AM

Yes, though in a round about way. We'd probably make the interface pluggable, 
and then provide a YARN-based implementation. I don't think we've thought about 
this in great detail yet, but the short answer is that there could end up being 
a Kafka consumer inside the YARN AM process.

bq. The way I imagined this is that the AM will read the config from some Kafka 
topic (per Samza topology) which will then initialize (or modify) containers 
based on this ?

If we follow this strategy, then we need to provide some RPC mechanism for each 
SamzaContainer to call back and get the config for itself. This could be done 
via HTTP, and I believe we already expose configs over HTTP in the YARN AM, so 
it should be pretty easy to add as a JSON blob.

An alternative strategy would be to have each SamzaContainer fully read the 
ConfigLog topic, and not coordinate directly with the AM.

One decision that needs to be made is whether the centralized coordinator 
should be the AM, or whether the SamzaContainers themselves should have some 
influence on restart decisions. It seems to me, at first glance, than having 
the containers call back to the AM to get their config, and having the AM be in 
complete control over when containers are restarted is the most desirable way 
to do things since it means the containers can be completely dumb, and we have 
a single place to make all container-related decisions (the AM).

bq. Do we also need the AM -> container communication mechanism, discussed in a 
previous ticket as part of this ? OR - in case of a config change, the AM 
simply destroys and re-creates the required containers ?

The simplest approach is to not have AM-to-container communication, and simply 
have the AM restart the containers when it needs to change them. The trade-off 
is that if your container has state, and you change config/restart the 
container, then you have to wait for the container to restore its state.

> 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
>
> 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.2#6252)

Reply via email to