blob: 2bccb5156b111757e359b6cfa5c2356c4ad70db8 [file] [log] [blame]
---+ Operationalizing Falcon
---++ Overview
Apache Falcon provides various tools to operationalize Falcon consisting of Alerts for
unrecoverable errors, Audits of user actions, Metrics, and Notifications. They are detailed below.
++ Lineage
Currently Lineage has no way to access or restore information about entity instances created during the time lineage
was disabled. Information about entities however, is preserved and bootstrapped when lineage is enabled. If you have to
reset the graph db then you can delete the graph db files as specified in the startup.properties and restart the falcon.
Please note: you will loose all the information about the instances if you delete the graph db.
---++ Monitoring
Falcon provides monitoring of various events by capturing metrics of those events.
The metric numbers can then be used to monitor performance and health of the Falcon system and
the entire processing pipelines.
Falcon also exposes [[https://github.com/thinkaurelius/titan/wiki/Titan-Performance-and-Monitoring][metrics for titandb]]
Users can view the logs of these events in the metric.log file, by default this file is created
under ${user.dir}/logs/ directory. Users may also extend the Falcon monitoring framework to send
events to systems like Mondemand/lwes by implementingorg.apache.falcon.plugin.MonitoringPlugin
interface.
The following events are captured by Falcon for logging the metrics:
1. New cluster definitions posted to Falcon (success & failures)
1. New feed definition posted to Falcon (success & failures)
1. New process definition posted to Falcon (success & failures)
1. Process update events (success & failures)
1. Feed update events (success & failures)
1. Cluster update events (success & failures)
1. Process suspend events (success & failures)
1. Feed suspend events (success & failures)
1. Process resume events (success & failures)
1. Feed resume events (success & failures)
1. Process remove events (success & failures)
1. Feed remove events (success & failures)
1. Cluster remove events (success & failures)
1. Process instance kill events (success & failures)
1. Process instance re-run events (success & failures)
1. Process instance generation events
1. Process instance failure events
1. Process instance auto-retry events
1. Process instance retry exhaust events
1. Feed instance deletion event
1. Feed instance deletion failure event (no retries)
1. Feed instance replication event
1. Feed instance replication failure event
1. Feed instance replication auto-retry event
1. Feed instance replication retry exhaust event
1. Feed instance late arrival event
1. Feed instance post cut-off arrival event
1. Process re-run due to late feed event
1. Transaction rollback failed event
The metric logged for an event has the following properties:
1. Action - Name of the event.
2. Dimensions - A list of name/value pairs of various attributes for a given action.
3. Status- Status of an action FAILED/SUCCEEDED.
4. Time-taken - Time taken in nanoseconds for a given action.
An example for an event logged for a submit of a new process definition:
2012-05-04 12:23:34,026 {Action:submit, Dimensions:{entityType=process}, Status: SUCCEEDED, Time-taken:97087000 ns}
Users may parse the metric.log or capture these events from custom monitoring frameworks and can plot various graphs
or send alerts according to their requirements.
---++ Notifications
Falcon has two types of notifications - System and User notifications.
---+++ System notifications
The System notifications are internally generated and used by Falcon to monitor the Falcon orchestrated workflow jobs.
By default, Falcon starts an ActiveMQ embedded JMS server on Falcon machine on port 61616 as a daemon. Alternatively,
users can make Falcon to use an existing JMS server instead of starting an embedded instance by doing the
following 2 steps:
* Setting the property broker.url in the startup.properties as below
<verbatim>
*.broker.url=tcp://jms-server-host:61616
</verbatim>
* Set the system property falcon.embeddedmq to false as below
<verbatim>
<FALCON-INSTALL-DIR>/bin/falcon-start -Dfalcon.embeddedmq=false
</verbatim>
Falcon uses FALCON.ENTITY.TOPIC to publish system notifications. This topic and the Map Message fields are internal
and could change between releases.
---+++ User notifications
Falcon, in addition to the FALCON.ENTITY.TOPIC, also creates a JMS topic for every process/feed that is scheduled in
Falcon as part of User notification. To enable User notifications, the broker url and implementation class of the JMS
engine need to be specified in the cluster definition associated with the feed/process. Users may register consumers
on the required topic to check the availability or status of feed instances. The User notification JMS broker instance
can be same as the System notification or different.
The name of the JMS topic is same as the process/feed name. Falcon sends a map message for every feed instance that is
created/deleted/replicated/imported/exported to the JMS topic. The JMS Map Message sent to a topic has the following
fields:
1. cluster - name of the current cluster the feed/process is dependent on.
1. entityType - type of the entity (feed or process).
1. entityName - name of the entity.
1. nominalTime - instance time (or data date).
1. operation - operation like generate, delete, replicate, import, export.
1. feedNames - name of the feeds which are generated/replicated/deleted/imported/exported.
1. feedInstancePaths - comma separated feed instance paths.
1. workflowId - current workflow-id of the instance.
1. workflowUser - user who owns the feed instance (i.e partition).
1. runId - current run-id of the instance.
1. status - status of the user workflow instance.
1. timeStamp - current timestamp.
1. logDir - log dir where lineage can be recorded.
The JMS messages are automatically purged after a certain period (default 3 days) by the Falcon JMS house-keeping
service. TTL (Time-to-live) for JMS message can be configured in the Falcon's startup.properties file.
The following example shows how to enable and read user notification by connecting to the JMS broker.
First, specify the JMS broker url in the cluster definition XML as shown below.
<verbatim>
<?xml version="1.0"?>
<!-- filename : primaryCluster.xml -->
<cluster colo="USWestOregon" description="oregonHadoopCluster" name="primaryCluster" xmlns="uri:falcon:cluster:0.1">
<interfaces>
...
...
<interface type="messaging" endpoint="tcp://user-jms-broker-host:61616?daemon=true" version="5.1.6" />
...
</interfaces>
</cluster>
</verbatim>
Next, use a JMS consumer (example below in Java) to read the message from the topic with the name
FALCON.<feed-or-process-name>
<verbatim>
import org.apache.activemq.ActiveMQConnectionFactory;
import org.apache.activemq.command.ActiveMQMapMessage;
import javax.jms.ConnectionFactory;
import javax.jms.Connection;
import javax.jms.MessageConsumer;
import javax.jms.Topic;
import javax.jms.Session;
import javax.jms.TopicSession;
public class FalconUserJMSClient {
public static void main(String[] args)throws Exception {
// Note: specify the JMS broker URL
String brokerUrl = "tcp://localhost:61616";
ConnectionFactory connectionFactory = new ActiveMQConnectionFactory(brokerUrl);
Connection connection = connectionFactory.createConnection();
connection.setClientID("Falcon User JMS Consumer");
TopicSession session = (TopicSession) connection.createSession(false, Session.AUTO_ACKNOWLEDGE);
try {
// Note: the topic name for the feed will be FALCON.<feed-name>
Topic falconTopic = session.createTopic("FALCON.feed-sample");
MessageConsumer consumer = session.createConsumer(falconTopic);
connection.start();
while (true) {
ActiveMQMapMessage msg = (ActiveMQMapMessage) consumer.receive();
System.out.println("cluster : " + msg.getString("cluster"));
System.out.println("entityType : " + msg.getString("entityType"));
System.out.println("entityName : " + msg.getString("entityName"));
System.out.println("nominalTime : " + msg.getString("nominalTime"));
System.out.println("operation : " + msg.getString("operation"));
System.out.println("feedNames : " + msg.getString("feedNames"));
System.out.println("feedInstancePaths : " + msg.getString("feedInstancePaths"));
System.out.println("workflowId : " + msg.getString("workflowId"));
System.out.println("workflowUser : " + msg.getString("workflowUser"));
System.out.println("runId : " + msg.getString("runId"));
System.out.println("status : " + msg.getString("status"));
System.out.println("timeStamp : " + msg.getString("timeStamp"));
System.out.println("logDir : " + msg.getString("logDir"));
System.out.println("brokerUrl : " + msg.getString("brokerUrl"));
System.out.println("brokerImplClass : " + msg.getString("brokerImplClass"));
System.out.println("logFile : " + msg.getString("logFile"));
System.out.println("topicName : " + msg.getString("topicName"));
System.out.println("brokerTTL : " + msg.getString("brokerTTL"));
}
} finally {
if (session != null) {
session.close();
}
if (connection != null) {
connection.close();
}
}
}
}
</verbatim>
---++ Alerts
Falcon generates alerts for unrecoverable errors into a log file by default.
Users can view these alerts in the alerts.log file, by default this file is created
under ${user.dir}/logs/ directory.
Users may also extend the Falcon Alerting plugin to send events to systems like Nagios, etc. by
extending org.apache.falcon.plugin.AlertingPlugin interface.
---++ Audits
Falcon audits all user activity and captures them into a log file by default.
Users can view these audits in the audit.log file, by default this file is created
under ${user.dir}/logs/ directory.
Users may also extend the Falcon Audit plugin to send audits to systems like Apache Argus, etc. by
extending org.apache.falcon.plugin.AuditingPlugin interface.
---++ Metrics Collection In Graphite
Falcon has support to send metrics to graphite more details regarding this can be found on [[GraphiteMetricCollection][Graphite Metric Collection]]