A FirehoseFactory for RabbitMQ.
It will receive it's configuration through the realtime.spec file and expects to find a
consumerProps element in the firehose definition with values for a number of configuration options.
Below is a complete example for a RabbitMQ firehose configuration with some explanation. Options
that have defaults can be skipped but options with no defaults must be specified with the exception
of the URI property. If the URI property is set, it will override any other property that was also
set.
File:
realtime.spec
"firehose" : {
"type" : "rabbitmq",
"connection" : {
"host": "localhost", # The hostname of the RabbitMQ broker to connect to. Default: 'localhost'
"port": "5672", # The port number to connect to on the RabbitMQ broker. Default: '5672'
"username": "test-dude", # The username to use to connect to RabbitMQ. Default: 'guest'
"password": "test-word", # The password to use to connect to RabbitMQ. Default: 'guest'
"virtualHost": "test-vhost", # The virtual host to connect to. Default: '/'
"uri": "amqp://mqserver:1234/vhost", # The URI string to use to connect to RabbitMQ. No default and not needed
},
"config" : {
"exchange": "test-exchange", # The exchange to connect to. No default
"queue" : "druidtest", # The queue to connect to or create. No default
"routingKey": "#", # The routing key to use to bind the queue to the exchange. No default
"durable": "true", # Whether the queue should be durable. Default: 'false'
"exclusive": "false", # Whether the queue should be exclusive. Default: 'false'
"autoDelete": "false", # Whether the queue should auto-delete on disconnect. Default: 'false'
"maxRetries": "10", # The max number of reconnection retry attempts
"retryIntervalSeconds": "1", # The reconnection interval
"maxDurationSeconds": "300" # The max duration of trying to reconnect
},
"parser" : {
"timestampSpec" : { "column" : "utcdt", "format" : "iso" },
"data" : { "format" : "json" },
"dimensionExclusions" : ["wp"]
}
},
Limitations: This implementation will not attempt to reconnect to the MQ broker if the
connection to it is lost. Furthermore it does not support any automatic failover on high availability
RabbitMQ clusters. This is not supported by the underlying AMQP client library and while the behavior
could be "faked" to some extent we haven't implemented that yet. However, if a policy is defined in
the RabbitMQ cluster that sets the "ha-mode" and "ha-sync-mode" properly on the queue that this
Firehose connects to, messages should survive an MQ broker node failure and be delivered once a
connection to another node is set up.
For more information on RabbitMQ high availability please see:
http://www.rabbitmq.com/ha.html.