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

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

bq. Maybe "metadata" rather than "config", but "config" is shorter.

Hmm. MetadataStream. That might be technically more accurate and descriptive. I 
agree "config" might be construed as technically correct, but I'm a little 
worried that it's not really what people think of when they hear the word 
"config".

bq. 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?

Currently, when a StreamTask's offsets are committed, they're sent as a single 
message with Kafka. In the new proposal, they would be sent as a series of 
messages: one for each input SSP. I suppose "non-transactional" might be a bet 
more correct. The difference is that a container can fail half-way through a 
single StreamTask's offset commit in the new proposal, but not in our current 
implementation.

bq. 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.

This needs to be thought through a bit, I think. 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.

Perhaps a simple query of the AM's HTTP JSON web service would be a better 
solution, though this would require the job being up and running to fetch 
config defaults.

bq. 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)?

Correct.

bq. 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)

I had a note about this in one of my drafts, but I deleted it. The current 
proposal is one ConfigStream per-job. I couldn't come up with a good reason to 
have multiple jobs share a ConfigStream.

bq. 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.

I went back and forth on this. The trade-offs between the two approaches (at 
least, as I see it) are listed in the proposal. I opted for the simpler (from a 
dev's perspective) approach in the latest proposal.

bq. What's the difference between control-job.sh and configure-job.sh?

They are the same thing. I started calling it configure-job.sh, but I've 
switched to calling it control-job.sh, since it might do things like restart 
the job, etc. Looks like I missed a few renames in the latest design doc. 
Everything should read control-job.sh.

bq. 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.)

Oops, maybe I wasn't explicit enough. 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.

bq. 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.

Yea, it's kind of annoying, but I can live with it, I think. This estimate is 
pretty pessimistic, though, in terms of job size. One other short-circuit that 
we could employ would be to have the control-job.sh script start by trying to 
query the AM's HTTP JSON server (though this would require the control-job.sh 
script to somehow know where the HTTP:PORT is).

bq. Will ConfigStream be a general-purpose mechanism for parts of the framework 
which need to remember some information across job restarts?

Hmm. I hadn't thought about this in great detail. It seems like it'd be useful 
to provide a "write config" facility to pluggable parts of the framework, like 
SystemConsumers, as you've said. Given a good implementation of the 
ConfigStream, it seems possible to expose it.

bq. 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?

The current porposal is to stay off of Kafka's offset management, and just use 
our own. Technically, we'll still have to interact with Kafka's, since that's 
where transactions are managed, but we won't store any offsets in it--we'll 
just tell it to commit transactions. 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. It could also cause 
another bifurcation between the way offsets are stored, and the way other 
config is stored.

> 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