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

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

bq. Maybe we can still standardize this. The configure-job.sh script can take a 
job name for which a config stream is to be written. We can simply wait for the 
Kafka topic deletion to be available - to solve the problem of resetting the 
config.

Yea, I think I'm leaning this way now as well. Something like:

{noformat}
$ configure-job.sh --location kafka://localhost:10251 --job.name <job name> 
--job.id <job id> --property task.inputs=kafka.foo
$ run-job.sh --location kafka://localhost:10251 --job.name <job name> --job.id 
<job id>
{noformat}

bq. I think we can make the auto-config by the Samza AM - as a tunable 
property. This should be used when the user does not want to keep tuning the 
config. 

I was just thinking that we could ignore this problem. As long as we keep the 
job coordinator as the *only* thing that programmatically mutates 
configuration, then the race condition is between the coordinator and the human 
dev. I think this should be fine.

The offset checkpoint messages pose a bigger problem, though. Since they're 
checkpointed once per minute, the chance of an over-write is relatively high in 
the case where a developer is trying to set all offsets back to 0, for example. 
If we suppose there are 64 StreamTasks, and each one is checkpointing once perm 
minute, then you're getting 1 checkpoint per second, on average. If the 
developer writes all 64 offset messages back to offset 0, it seems likely that 
a SamzaContainer might over-write the developer's offset message with its own 
before the container is killed off and restarted. The order of events would be:

{noformat}
SamzaContainer: offset 3976
SamzaContainer: offset 4320
Developer: offset 0
SamzaContainer: offset 5320
Job coordinator: restarts SamzaContainer
SamzaContainer: starts with offset 5320
{noformat}

I don't have a good idea on how to solve this. One way would be to add some 
generation number that invalidates all future offsets from SamzaContainer after 
the developer's message is written. I haven't thought about this in great 
detail.

bq. In addition, maybe it is better for the user to make any config related 
changes from a web based endpoint (maybe hosted in the AM). This way, the 
config hosted by the AM becomes the source of truth and not cfg2 (something 
similar to what Azkaban also does).

I was thinking the source of truth would be the underlying stream, since this 
is what the job coordinator will use to run the job. Whether the config is 
mutated from the AM web UI, or from a CLI, I haven't considered very much.

Maybe you're trying to get at the idea that we could try and funnel all 
mutations to the ConfigStrem through a single writer?

> 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