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

Martin Kleppmann commented on SAMZA-348:
----------------------------------------

This is a nice proposal, I like it a lot. Various thoughts on the design doc 
(first three are responses to your questions above):

# I think the name "ConfigStream" is actually ok. A checkpoint or 
task-to-changelog-partition assignment can quite reasonably be regarded as 
configuration (even though it arises as a side-effect of execution rather than 
from a config file). Maybe "metadata" rather than "config", but "config" is 
shorter.
# You say the atomicity of per-task checkpointing is broken, but I don't follow 
why. Isn't a checkpoint update still one message written to the ConfigStream?
# I think defaults should not be written to the stream, but rather filled in at 
runtime at the last possible moment when the configuration is read. If they 
were written to the stream, it would become impossible for the framework to 
change defaults in future (as the job cannot distinguish between a default 
written to the stream, which should be replaced by a new default, and an 
explicitly configured parameter, which should be honoured).
# Do I understand correctly that from a container's point of view, the config 
remains immutable once it has been fetched from the AM's HTTP endpoint? The 
container does not consume the ConfigStream, and there is no polling of the 
HTTP endpoint (except for hot standby as discussed in the doc)? I think that's 
good, because a mutable config within a container would require big changes to 
SystemProducers/Consumers etc.
# Were you imagining that every job would have their own ConfigStream, or could 
a single ConfigStream be shared by multiple jobs? (I think each having their 
own would be simpler and nicer)
# Should the config stream location URL include the job name? An early example 
in the design doc (kafka://localhost:10251/my-job) includes the job name, later 
ones do not.
# What's the difference between control-job.sh and configure-job.sh?
# +1 on explicit restart.
# Not wild on the proposed serialisation format for config messages (t=... 
k=... v=...). What about escaping spaces and equals signs within values? Better 
to just use JSON, IMHO. (If using JSON in the key of a message sent to Kafka, 
need to ensure there is a deterministic order of keys, so that compaction 
works.)
# Moving ConfigRewriter into the coordinator: interesting idea, not sure about 
the implications. What are the use cases for ConfigRewriter, besides expanding 
a regex for specifying input streams? e.g. it might be used for fetching config 
from an external configuration management system – in that case, moving 
ConfigRewriter may be intrusive, as the coordinator may not be able to access 
that external system.
# Regarding estimate of time to consume the ConfigStream, I should point out 
that "control-job.sh --list" will take 100 seconds too, which is not so great 
since it's an interactive command. However, most jobs will have vastly smaller 
config, so perhaps that's an edge case we can live with.
# "If 1000 Samza jobs were run in one Samza grid, 50 Kafka brokers would be 
required just to sustain the ConfigStream write rate" — you're talking 1000 
jobs with 1000 containers each here, i.e. 1 million CPU cores. I hope we get to 
see Samza running at that scale one day :)
# The prospect of a work-preserving AM restart is nice.
# Will ConfigStream be a general-purpose mechanism for parts of the framework 
which need to remember some information across job restarts? The assignment of 
tasks to changelog partitions is one example of such a thing that needs to be 
durable, and I could imagine there might be more. For example, a MySQL binlog 
SystemConsumer may want to remember metadata about leader failover events in 
some durable location. This is not a concrete need yet, just an idea to keep in 
mind.
# How does the ConfigStream checkpointing interact with Kafka's own consumer 
offset management? Is the intention that Samza will eventually switch over to 
Kafka's offset management, or will Samza keep doing its own checkpointing 
indefinitely?

> 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, 
> DESIGN-SAMZA-348-1.md, DESIGN-SAMZA-348-1.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