[
https://issues.apache.org/jira/browse/SAMZA-348?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14172090#comment-14172090
]
Yan Fang commented on SAMZA-348:
--------------------------------
{quote}
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.
{quote}
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 ?
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.
{quote}
So in order to use a kafka:// URI, you must include a
systems.kafka.samza.factory=org.apache.samza.system.kafka.KafkaSystemFactory in
the job configuration. That configuration parameter would need to be provided
as a command-line argument.
{quote}
I do not quite get this. Do you mean something like
{code}
kafka://<broker-list>:<broker ports>?zk=<zk-list>:<zk-port>
systems.kafka.samza.factory=org.apache.samza.system.kafka.KafkaSystemFactory
{code}
If yes, why can't this be a default setting when the URI starts with "kafka://"?
{quote}
This seems to be the cleanest/most general way to do things. It seems not too
great from a usability perspective, but if we provide defaults for the
file/kafka systems, then it should be OK.
{quote}
+1. When we have other systems, we could add more. This should not scare people
away.
{quote}
How does the control-job.sh script use the SystemConsumer/SystemProducer?
{quote}
Maybe I misunderstood something. When we use the control-job.sh to control the
job, do we still have the properties file ? If no, is it possible for the
control-job.sh to accept a properties file to get the system information?
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. If yes, then why cannot we put the system information in the properties
file?
Also, adding a --file for the control-job.sh maybe helpful, like
{code}
control-job.sh --location kafka://localhost:10251 --file /path/to/config/file
{code}
because when you start the job at the first time, you may input a lot of
configurations. Having a file containing the starting properties is useful.
> 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)