[
https://issues.apache.org/jira/browse/SAMZA-310?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14081653#comment-14081653
]
Chris Riccomini commented on SAMZA-310:
---------------------------------------
Your approach seems like it should work.
bq. add partition information when sending messages to broker in
KafkaLog4jAppender.scala
What we really need is a way to specify a key for the message. For us, the key
should be the ApplicationMaster, or Container ID (taskGroup). This would
guarantee that, for a given container, all log messages will end up fully
ordered in a single partition (mixed in with log messages from other
containers). If a user just wants a single container's logs, they can just grep.
bq. tag container information to the log
You might want to have a look at Log4J's
[MDC|http://veerasundar.com/blog/2009/11/log4j-mdc-mapped-diagnostic-context-example-code/].
The MDC allows us to statically set/update properties. These properties can
then be accessed via the ConversionPattern in the log4j.xml file.
This would allow us to set/update the MDC to include the things like
containerName, taskName, etc. You could then include them in the logs at your
convenience via %X{containerName}, %X{taskName}, etc.
bq. One thing bugs me is that, the log4j topic name is defined in the
log4j.xml, Samza seems not have the information before hand and so it can not
create the topic in, say, AM, if it does not read log4j.xml.
This should be OK for now. Sending the log message will trigger a topic
creation with the cluster defaults (replication, partition count, retention,
etc) for the topic. We can always patch KafkaLog4jAppender.scala (or extend it)
to add this feature if we really needed it, but I don't think that we do.
This JIRA suffers from the same troubles that I'm having with SAMZA-350.
Ideally, I'd like to have this work without depending on log4j in samza-core.
The only injection point that I can think of right now to manage things like
setting the MDC to update the taskName when we process a message is via the
TaskLifecycleListener. This would mean that, to use this feature, you'd have to
(1) include the samza-log4j package on your classpath and (2) define some
Log4JTaskLifecycleListener in your configs. Does that seem tolerable to people?
> Publish container logs to a SystemStream
> ----------------------------------------
>
> Key: SAMZA-310
> URL: https://issues.apache.org/jira/browse/SAMZA-310
> Project: Samza
> Issue Type: New Feature
> Components: container
> Affects Versions: 0.7.0
> Reporter: Martin Kleppmann
>
> At the moment, it's a bit awkward to get to a Samza job's logs: assuming
> you're running on YARN, you have to navigate around the YARN web interface,
> and you can only see one container's logs at a time.
> Given that Samza is all about streams, it would make sense for the logs
> generated by Samza jobs to also be sent to a stream. There, they could be
> indexed with [Kibana|http://www.elasticsearch.org/overview/kibana/], consumed
> by an exception-tracking system, etc.
> Notes:
> - The serde for encoding logs into a suitable wire format should be
> pluggable. There can be a default implementation that uses JSON, analogous to
> MetricsSnapshotSerdeFactory for metrics, but organisations that already have
> a standardised in-house encoding for logs should be able to use it.
> - Should this be at the level of Slf4j or Log4j? Currently the log
> configuration for YARN jobs uses Log4j, which has the advantage that any
> frameworks/libraries that use Log4j but not Slf4j appear in the logs.
> However, Samza itself currently only depends on Slf4j. If we tie this feature
> to Log4j, it would somewhat defeat the purpose of using Slf4j.
> - Do we need to consider partitioning? Perhaps we can use the container name
> as partitioning key, so that the ordering of logs from each container is
> preserved.
--
This message was sent by Atlassian JIRA
(v6.2#6252)