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

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

bq. 1) when you guys talk about "default", does it mean the default 
configuration provided by the Samza or by the user when he starts the job first 
time ?

We're talking about the "default" value that's used when a developer has *not* 
specified a value for a given config. This usually happens in the code when we 
do a .getOrElse() on a config object.

bq. 2) there is another use case where the user wants to check the latest 
configuration used in a finished job and probably reuses the same 
configuration. Then querying the AM seems not working because the job is 
already finished. The only way of getting the latest configuration of a 
finished job is reading the whole ConfigStream. Or we can dump the latest 
configuration into a file when the job finishes? But the problem is that, at 
most time, the job is killed rather than finished gracefully by itself.

Yea, I think you'd have to read the whole config stream.

bq. If yes, why can't this be a default setting when the URI starts with 
"kafka://"?

I agree, we should just default it to the proper system factory for known 
system types (kafka, file).

bq. Maybe I misunderstood something. When we use the control-job.sh to control 
the job, do we still have the properties file ? 

No.

bq. If no, is it possible for the control-job.sh to accept a properties file to 
get the system information? 

It is possible, but it seems kind of hacky. Following this train of thought, 
the control-job.sh could receive a properties file with a system defined for 
the ConfigStream. The problem is that the coordinator (AM) will also need this 
file, and it'll be running elsewhere. This leads to the same problem we have 
today: how do you get the file to the AM? JSON encoded environment variable, 
HDFS, HTTP, local file system?

I feel like this idea inevitably leads to one of two outcomes:

# A system like we have today, where we pass the static properties file between 
machines (in our case, via environment variable).
# The properties file is placed on every host, and an environment variable is 
defined so that control-job.sh and the job coordinator (AM) can both find it.

bq. Because the main goal of ConfigStream is to solve the problems such as 
dynamic changing config, bigger configuration size, not to get rid of the 
properties file.

True. I just want to make this as easy to use as possible. Having the kafka:// 
URI, with default system factories seems more usable than another properties 
file.

bq. Also, adding a --file for the control-job.sh maybe helpful

We will certainly need a way to pipe config files into the ConfigStream. 
Something like {--file} makes a lot of sense.

> 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