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

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

bq. In the new proposal, they would be sent as a series of messages: one for 
each input SSP.

Ah yes, I missed that. My hunch is that it's not a big problem, since state 
changelogs are currently also not atomically tied to the checkpoints either, so 
the semantics of container restart are somewhat vague already. And we generally 
don't give any guarantee regarding cross-SSP coordination. Once we get Kafka 
transactions, these things can all be tied together atomically.

Alternatively, would it be an option to continue writing checkpoints in the 
current form (per-StreamTask rather than per-SSP)? I don't see why the change 
to a MetadataStream forces us to use per-SSP checkpoint messages.

bq. The main concern was that the control-job.sh script might be a different 
version than what the job is running. In such a case, you (the developer) might 
wish to know what value is being used for config "X", but the default of the 
Samza version that control-job.sh is running might be different from the 
version that the job is running on.

I think it's quite reasonable to expect that if you want to know the 
'interpreted' value of a config property (after default is filled in, and 
perhaps also after it is parsed or otherwise processed) then you need to ask 
the AM, either via the web interface or via a command-line tool that talks to 
an API. Then control-job.sh is concerned only with the config that is 
explicitly declared, and not what is inferred.

If you want control-job.sh to also include docs ({{--help}}), you have the same 
version mismatch problem: if the job and the command-line tool are running 
different versions of Samza, the meaning of some of the properties may have 
changed.

bq. The current proposal is to use JSON for both keys and values. I think that 
I just used shorthand notation at various points in the docs. Agree we'll need 
to define an ordering for the keys, which is a bit odd/error prone. I'm not 
sure of a good way around this.

Ok. An alternative would be to use a serialization format with a canonical 
representation (most binary serialization formats would qualify: Avro, Protocol 
Buffers, ASN.1 DER, etc), but that makes it less convenient if you want to 
inspect the stream using kafka-console-consumer. I don't have a strong opinion 
either way.

bq. I haven't thought about this in great depth, but my gut reasoning is that 
it's best to stay away from dependency directly on Kafka for things like offset 
checkpoints, especially if there's the possible for non-Kafka offsets needing 
to be checkpointed.

Agree on the gut reasoning, though I haven't thought about it in great depth.

> 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