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

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

bq. support a message payload format that is nested (K: \{K:V, ...})

Do you have an estimate of how big these payloads are likely to get? Thinking 
about [~jonbringhurst]'s remark about the Kinesis message size limit.

Apart from that, I agree with your observations about the advantages and 
disadvantages of each message format. I mildly prefer the K:V format as it's a 
bit cleaner, and transactionality is much needed anyway. But I'm ok either way.

bq. Adding a getConfig() API seems mildly hacky. The main problem with this 
approach is how to determine which SystemFactory to use based on the URI.

We could define that the scheme is the name of a system defined in the job 
configuration. 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.

Sensible defaults (like defining file:// and kafka:// by default) would then be 
a nice usability improvement without sacrificing generality.

bq. How will this work in a dev environment?

I think it would be very good if the Kafka runtime dependency is optional, so 
that Samza remains friendly to jobs which choose to use a different message 
broker. So I would be keen for Samza to be able to use files for checkpoints 
and config in dev.

bq. {{kafka://<broker-list>:<broker ports>?zk=<zk-list>:<zk-port>}}

Nit: Kafka's configuration repeats the port for each ZK IP address, rather than 
specifying one port for all IP addresses. Might be better to stick with the 
same convention.

bq. I propose ripping (2) and (3) out into separate tickets, and just focusing 
on implementing the ConfigStream (1) in this ticket.

+1

> 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