You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flume.apache.org by bu...@apache.org on 2017/10/04 16:24:03 UTC

svn commit: r1019094 [2/3] - in /websites/staging/flume/trunk/content: ./ .doctrees/ .doctrees/releases/ _sources/ _sources/releases/ releases/

Modified: websites/staging/flume/trunk/content/_sources/FlumeUserGuide.txt
==============================================================================
--- websites/staging/flume/trunk/content/_sources/FlumeUserGuide.txt (original)
+++ websites/staging/flume/trunk/content/_sources/FlumeUserGuide.txt Wed Oct  4 16:24:02 2017
@@ -15,7 +15,7 @@
 
 
 ======================================
-Flume 1.7.0 User Guide
+Flume 1.8.0 User Guide
 ======================================
 
 Introduction
@@ -50,7 +50,7 @@ in the latest architecture.
 System Requirements
 -------------------
 
-#. Java Runtime Environment - Java 1.7 or later
+#. Java Runtime Environment - Java 1.8 or later
 #. Memory - Sufficient memory for configurations used by sources, channels or sinks
 #. Disk Space - Sufficient disk space for configurations used by channels or sinks
 #. Directory Permissions - Read/Write permissions for directories used by agent
@@ -234,6 +234,25 @@ The original Flume terminal will output
 
 Congratulations - you've successfully configured and deployed a Flume agent! Subsequent sections cover agent configuration in much more detail.
 
+Using environment variables in configuration files
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+Flume has the ability to substitute environment variables in the configuration. For example::
+
+  a1.sources = r1
+  a1.sources.r1.type = netcat
+  a1.sources.r1.bind = 0.0.0.0
+  a1.sources.r1.port = ${NC_PORT}
+  a1.sources.r1.channels = c1
+
+NB: it currently works for values only, not for keys. (Ie. only on the "right side" of the `=` mark of the config lines.)
+
+This can be enabled via Java system properties on agent invocation by setting `propertiesImplementation = org.apache.flume.node.EnvVarResolverProperties`.
+
+For example::
+  $ NC_PORT=44444 bin/flume-ng agent --conf conf --conf-file example.conf --name a1 -Dflume.root.logger=INFO,console -DpropertiesImplementation=org.apache.flume.node.EnvVarResolverProperties
+
+Note the above is just an example, environment variables can be configured in other ways, including being set in `conf/flume-env.sh`.
+
 Logging raw data
 ~~~~~~~~~~~~~~~~
 
@@ -357,8 +376,6 @@ There's an exec source that executes a g
 single 'line' of output ie. text followed by carriage return ('\\r') or line
 feed ('\\n') or both together.
 
-.. note:: Flume does not support tail as a source. One can wrap the tail command in an exec source to stream the file.
-
 Network streams
 ~~~~~~~~~~~~~~~
 
@@ -578,9 +595,9 @@ Weblog agent config:
   agent_foo.sinks.avro-forward-sink.channel = file-channel
 
   # avro sink properties
-  agent_foo.sources.avro-forward-sink.type = avro
-  agent_foo.sources.avro-forward-sink.hostname = 10.1.1.100
-  agent_foo.sources.avro-forward-sink.port = 10000
+  agent_foo.sinks.avro-forward-sink.type = avro
+  agent_foo.sinks.avro-forward-sink.hostname = 10.1.1.100
+  agent_foo.sinks.avro-forward-sink.port = 10000
 
   # configure other pieces
   #...
@@ -599,7 +616,7 @@ HDFS agent config:
   agent_foo.sources.avro-collection-source.channels = mem-channel
   agent_foo.sinks.hdfs-sink.channel = mem-channel
 
-  # avro sink properties
+  # avro source properties
   agent_foo.sources.avro-collection-source.type = avro
   agent_foo.sources.avro-collection-source.bind = 10.1.1.100
   agent_foo.sources.avro-collection-source.port = 10000
@@ -882,13 +899,9 @@ interceptors.*
              asynchronous interface such as ExecSource! As an extension of this
              warning - and to be completely clear - there is absolutely zero guarantee
              of event delivery when using this source. For stronger reliability
-             guarantees, consider the Spooling Directory Source or direct integration
+             guarantees, consider the Spooling Directory Source, Taildir Source or direct integration
              with Flume via the SDK.
 
-.. note:: You can use ExecSource to emulate TailSource from Flume 0.9x (flume og).
-          Just use unix command ``tail -F /full/path/to/your/file``. Parameter
-          -F is better in this case than -f as it will also follow file rotation.
-
 Example for agent named a1:
 
 .. code-block:: properties
@@ -941,6 +954,12 @@ batchSize                   100
 converter.type              DEFAULT      Class to use to convert messages to flume events. See below.
 converter.*                 --           Converter properties.
 converter.charset           UTF-8        Default converter only. Charset to use when converting JMS TextMessages to byte arrays.
+createDurableSubscription   false        Whether to create durable subscription. Durable subscription can only be used with
+                                         destinationType topic. If true, "clientId" and "durableSubscriptionName"
+                                         have to be specified.
+clientId                    --           JMS client identifier set on Connection right after it is created.
+                                         Required for durable subscriptions.
+durableSubscriptionName     --           Name used to identify the durable subscription. Required for durable subscriptions.
 =========================   ===========  ==============================================================
 
 
@@ -1265,6 +1284,12 @@ useFlumeEventFormat                 fals
                                                  true to read events as the Flume Avro binary format. Used in conjunction with the same property
                                                  on the KafkaSink or with the parseAsFlumeEvent property on the Kafka Channel this will preserve
                                                  any Flume headers sent on the producing side.
+setTopicHeader                      true         When set to true, stores the topic of the retrieved message into a header, defined by the
+                                                 ``topicHeader`` property.
+topicHeader                         topic        Defines the name of the header in which to store the name of the topic the message was received
+                                                 from, if the ``setTopicHeader`` property is set to ``true``. Care should be taken if combining
+                                                 with the Kafka Sink ``topicHeader`` property so as to avoid sending the message back to the same
+                                                 topic in a loop.
 migrateZookeeperOffsets             true         When no Kafka stored offset is found, look up the offsets in Zookeeper and commit them to Kafka.
                                                  This should be true to support seamless Kafka client migration from older versions of Flume.
                                                  Once migrated this can be set to false, though that should generally not be required.
@@ -1351,13 +1376,13 @@ Example configuration with server side a
 
 .. code-block:: properties
 
-    a1.channels.channel1.type = org.apache.flume.channel.kafka.KafkaChannel
-    a1.channels.channel1.kafka.bootstrap.servers = kafka-1:9093,kafka-2:9093,kafka-3:9093
-    a1.channels.channel1.kafka.topic = channel1
-    a1.channels.channel1.kafka.consumer.group.id = flume-consumer
-    a1.channels.channel1.kafka.consumer.security.protocol = SSL
-    a1.channels.channel1.kafka.consumer.ssl.truststore.location=/path/to/truststore.jks
-    a1.channels.channel1.kafka.consumer.ssl.truststore.password=<password to access the truststore>
+    a1.sources.source1.type = org.apache.flume.source.kafka.KafkaSource
+    a1.sources.source1.kafka.bootstrap.servers = kafka-1:9093,kafka-2:9093,kafka-3:9093
+    a1.sources.source1.kafka.topics = mytopic
+    a1.sources.source1.kafka.consumer.group.id = flume-consumer
+    a1.sources.source1.kafka.consumer.security.protocol = SSL
+    a1.sources.source1.kafka.consumer.ssl.truststore.location=/path/to/truststore.jks
+    a1.sources.source1.kafka.consumer.ssl.truststore.password=<password to access the truststore>
 
 
 Note: By default the property ``ssl.endpoint.identification.algorithm``
@@ -1366,7 +1391,7 @@ In order to enable hostname verification
 
 .. code-block:: properties
 
-    a1.channels.channel1.kafka.consumer.ssl.endpoint.identification.algorithm=HTTPS
+    a1.sources.source1.kafka.consumer.ssl.endpoint.identification.algorithm=HTTPS
 
 Once enabled, clients will verify the server's fully qualified domain name (FQDN)
 against one of the following two fields:
@@ -1381,15 +1406,15 @@ which in turn is trusted by Kafka broker
 
 .. code-block:: properties
 
-    a1.channels.channel1.kafka.consumer.ssl.keystore.location=/path/to/client.keystore.jks
-    a1.channels.channel1.kafka.consumer.ssl.keystore.password=<password to access the keystore>
+    a1.sources.source1.kafka.consumer.ssl.keystore.location=/path/to/client.keystore.jks
+    a1.sources.source1.kafka.consumer.ssl.keystore.password=<password to access the keystore>
 
 If keystore and key use different password protection then ``ssl.key.password`` property will
 provide the required additional secret for both consumer keystores:
 
 .. code-block:: properties
 
-    a1.channels.channel1.kafka.consumer.ssl.key.password=<password to access the key>
+    a1.sources.source1.kafka.consumer.ssl.key.password=<password to access the key>
 
 
 **Kerberos and Kafka Source:**
@@ -1408,27 +1433,27 @@ Example secure configuration using SASL_
 
 .. code-block:: properties
 
-    a1.channels.channel1.type = org.apache.flume.channel.kafka.KafkaChannel
-    a1.channels.channel1.kafka.bootstrap.servers = kafka-1:9093,kafka-2:9093,kafka-3:9093
-    a1.channels.channel1.kafka.topic = channel1
-    a1.channels.channel1.kafka.consumer.group.id = flume-consumer
-    a1.channels.channel1.kafka.consumer.security.protocol = SASL_PLAINTEXT
-    a1.channels.channel1.kafka.consumer.sasl.mechanism = GSSAPI
-    a1.channels.channel1.kafka.consumer.sasl.kerberos.service.name = kafka
+    a1.sources.source1.type = org.apache.flume.source.kafka.KafkaSource
+    a1.sources.source1.kafka.bootstrap.servers = kafka-1:9093,kafka-2:9093,kafka-3:9093
+    a1.sources.source1.kafka.topics = mytopic
+    a1.sources.source1.kafka.consumer.group.id = flume-consumer
+    a1.sources.source1.kafka.consumer.security.protocol = SASL_PLAINTEXT
+    a1.sources.source1.kafka.consumer.sasl.mechanism = GSSAPI
+    a1.sources.source1.kafka.consumer.sasl.kerberos.service.name = kafka
 
 Example secure configuration using SASL_SSL:
 
 .. code-block:: properties
 
-    a1.channels.channel1.type = org.apache.flume.channel.kafka.KafkaChannel
-    a1.channels.channel1.kafka.bootstrap.servers = kafka-1:9093,kafka-2:9093,kafka-3:9093
-    a1.channels.channel1.kafka.topic = channel1
-    a1.channels.channel1.kafka.consumer.group.id = flume-consumer
-    a1.channels.channel1.kafka.consumer.security.protocol = SASL_SSL
-    a1.channels.channel1.kafka.consumer.sasl.mechanism = GSSAPI
-    a1.channels.channel1.kafka.consumer.sasl.kerberos.service.name = kafka
-    a1.channels.channel1.kafka.consumer.ssl.truststore.location=/path/to/truststore.jks
-    a1.channels.channel1.kafka.consumer.ssl.truststore.password=<password to access the truststore>
+    a1.sources.source1.type = org.apache.flume.source.kafka.KafkaSource
+    a1.sources.source1.kafka.bootstrap.servers = kafka-1:9093,kafka-2:9093,kafka-3:9093
+    a1.sources.source1.kafka.topics = mytopic
+    a1.sources.source1.kafka.consumer.group.id = flume-consumer
+    a1.sources.source1.kafka.consumer.security.protocol = SASL_SSL
+    a1.sources.source1.kafka.consumer.sasl.mechanism = GSSAPI
+    a1.sources.source1.kafka.consumer.sasl.kerberos.service.name = kafka
+    a1.sources.source1.kafka.consumer.ssl.truststore.location=/path/to/truststore.jks
+    a1.sources.source1.kafka.consumer.ssl.truststore.password=<password to access the truststore>
 
 
 Sample JAAS file. For reference of its content please see client config sections of the desired authentication mechanism (GSSAPI/PLAIN)
@@ -1456,8 +1481,8 @@ Also please make sure that the operating
     };
 
 
-NetCat Source
-~~~~~~~~~~~~~
+NetCat TCP Source
+~~~~~~~~~~~~~~~~~
 
 A netcat-like source that listens on a given port and turns each line of text
 into an event. Acts like ``nc -k -l [host] [port]``. In other words,
@@ -1493,12 +1518,48 @@ Example for agent named a1:
   a1.sources.r1.port = 6666
   a1.sources.r1.channels = c1
 
+NetCat UDP Source
+~~~~~~~~~~~~~~~~~
+
+As per the original Netcat (TCP) source, this source that listens on a given
+port and turns each line of text into an event and sent via the connected channel.
+Acts like ``nc -u -k -l [host] [port]``.
+
+Required properties are in **bold**.
+
+===================  ===========  ===========================================
+Property Name        Default      Description
+===================  ===========  ===========================================
+**channels**         --
+**type**             --           The component type name, needs to be ``netcatudp``
+**bind**             --           Host name or IP address to bind to
+**port**             --           Port # to bind to
+remoteAddressHeader  --
+selector.type        replicating  replicating or multiplexing
+selector.*                        Depends on the selector.type value
+interceptors         --           Space-separated list of interceptors
+interceptors.*
+===================  ===========  ===========================================
+
+Example for agent named a1:
+
+.. code-block:: properties
+
+  a1.sources = r1
+  a1.channels = c1
+  a1.sources.r1.type = netcatudp
+  a1.sources.r1.bind = 0.0.0.0
+  a1.sources.r1.port = 6666
+  a1.sources.r1.channels = c1
+
 Sequence Generator Source
 ~~~~~~~~~~~~~~~~~~~~~~~~~
 
 A simple sequence generator that continuously generates events with a counter that starts from 0,
 increments by 1 and stops at totalEvents. Retries when it can't send events to the channel. Useful
-mainly for testing. Required properties are in **bold**.
+mainly for testing. During retries it keeps the body of the retried messages the same as before so
+that the number of unique events - after de-duplication at destination - is expected to be
+equal to the specified ``totalEvents``. Required properties are in **bold**.
 
 ==============  ===============  ========================================
 Property Name   Default          Description
@@ -1509,7 +1570,7 @@ selector.type                    replica
 selector.*      replicating      Depends on the selector.type value
 interceptors    --               Space-separated list of interceptors
 interceptors.*
-batchSize       1
+batchSize       1                Number of events to attempt to process per request loop.
 totalEvents     Long.MAX_VALUE   Number of unique events sent by the source.
 ==============  ===============  ========================================
 
@@ -1997,7 +2058,7 @@ hdfs.fileType           SequenceFile  Fi
                                       (2)CompressedStream requires set hdfs.codeC with an available codeC
 hdfs.maxOpenFiles       5000          Allow only this number of open files. If this number is exceeded, the oldest file is closed.
 hdfs.minBlockReplicas   --            Specify minimum number of replicas per HDFS block. If not specified, it comes from the default Hadoop config in the classpath.
-hdfs.writeFormat        --            Format for sequence file records. One of "Text" or "Writable" (the default).
+hdfs.writeFormat        Writable      Format for sequence file records. One of ``Text`` or ``Writable``. Set to ``Text`` before creating data files with Flume, otherwise those files cannot be read by either Apache Impala (incubating) or Apache Hive.
 hdfs.callTimeout        10000         Number of milliseconds allowed for HDFS operations, such as open, write, flush, close.
                                       This number should be increased if many HDFS timeout operations are occurring.
 hdfs.threadsPoolSize    10            Number of threads per HDFS sink for HDFS IO ops (open, write, etc.)
@@ -2712,6 +2773,8 @@ kafka.topic                         defa
                                                          messages will be published to this topic.
                                                          If the event header contains a "topic" field, the event will be published to that topic
                                                          overriding the topic configured here.
+                                                         Arbitrary header substitution is supported, eg. %{header} is replaced with value of event header named "header".
+                                                         (If using the substitution, it is recommended to set "auto.create.topics.enable" property of Kafka broker to true.)
 flumeBatchSize                      100                  How many messages to process in one batch. Larger batches improve throughput while adding latency.
 kafka.producer.acks                 1                    How many replicas must acknowledge a message before its considered successfully written.
                                                          Accepted values are 0 (Never wait for acknowledgement), 1 (wait for leader only), -1 (wait for all replicas)
@@ -2728,6 +2791,9 @@ partitionIdHeader                   --
                                                          from the event header and send the message to the specified partition of the topic. If the
                                                          value represents an invalid partition, an EventDeliveryException will be thrown. If the header value
                                                          is present then this setting overrides ``defaultPartitionId``.
+allowTopicOverride                  true                 When set, the sink will allow a message to be produced into a topic specified by the ``topicHeader`` property (if provided).
+topicHeader                         topic                When set in conjunction with ``allowTopicOverride`` will produce a message into the value of the header named using the value of this property.
+                                                         Care should be taken when using in conjunction with the Kafka Source ``topicHeader`` property to avoid creating a loopback.
 kafka.producer.security.protocol    PLAINTEXT            Set to SASL_PLAINTEXT, SASL_SSL or SSL if writing to Kafka using some level of security. See below for additional info on secure setup.
 *more producer security props*                           If using SASL_PLAINTEXT, SASL_SSL or SSL refer to `Kafka security <http://kafka.apache.org/documentation.html#security>`_ for additional
                                                          properties that need to be set on producer.
@@ -2773,7 +2839,7 @@ argument.
     a1.sinks.k1.kafka.flumeBatchSize = 20
     a1.sinks.k1.kafka.producer.acks = 1
     a1.sinks.k1.kafka.producer.linger.ms = 1
-    a1.sinks.ki.kafka.producer.compression.type = snappy
+    a1.sinks.k1.kafka.producer.compression.type = snappy
 
 
 **Security and Kafka Sink:**
@@ -2807,12 +2873,12 @@ Example configuration with server side a
 
 .. code-block:: properties
 
-    a1.channels.channel1.type = org.apache.flume.channel.kafka.KafkaChannel
-    a1.channels.channel1.kafka.bootstrap.servers = kafka-1:9093,kafka-2:9093,kafka-3:9093
-    a1.channels.channel1.kafka.topic = channel1
-    a1.channels.channel1.kafka.producer.security.protocol = SSL
-    a1.channels.channel1.kafka.producer.ssl.truststore.location = /path/to/truststore.jks
-    a1.channels.channel1.kafka.producer.ssl.truststore.password = <password to access the truststore>
+    a1.sinks.sink1.type = org.apache.flume.sink.kafka.KafkaSink
+    a1.sinks.sink1.kafka.bootstrap.servers = kafka-1:9093,kafka-2:9093,kafka-3:9093
+    a1.sinks.sink1.kafka.topic = mytopic
+    a1.sinks.sink1.kafka.producer.security.protocol = SSL
+    a1.sinks.sink1.kafka.producer.ssl.truststore.location = /path/to/truststore.jks
+    a1.sinks.sink1.kafka.producer.ssl.truststore.password = <password to access the truststore>
 
 
 Note: By default the property ``ssl.endpoint.identification.algorithm``
@@ -2821,7 +2887,7 @@ In order to enable hostname verification
 
 .. code-block:: properties
 
-    a1.channels.channel1.kafka.producer.ssl.endpoint.identification.algorithm = HTTPS
+    a1.sinks.sink1.kafka.producer.ssl.endpoint.identification.algorithm = HTTPS
 
 Once enabled, clients will verify the server's fully qualified domain name (FQDN)
 against one of the following two fields:
@@ -2836,15 +2902,15 @@ which in turn is trusted by Kafka broker
 
 .. code-block:: properties
 
-    a1.channels.channel1.kafka.producer.ssl.keystore.location = /path/to/client.keystore.jks
-    a1.channels.channel1.kafka.producer.ssl.keystore.password = <password to access the keystore>
+    a1.sinks.sink1.kafka.producer.ssl.keystore.location = /path/to/client.keystore.jks
+    a1.sinks.sink1.kafka.producer.ssl.keystore.password = <password to access the keystore>
 
 If keystore and key use different password protection then ``ssl.key.password`` property will
 provide the required additional secret for producer keystore:
 
 .. code-block:: properties
 
-    a1.channels.channel1.kafka.producer.ssl.key.password = <password to access the key>
+    a1.sinks.sink1.kafka.producer.ssl.key.password = <password to access the key>
 
 
 **Kerberos and Kafka Sink:**
@@ -2863,26 +2929,26 @@ Example secure configuration using SASL_
 
 .. code-block:: properties
 
-    a1.channels.channel1.type = org.apache.flume.channel.kafka.KafkaChannel
-    a1.channels.channel1.kafka.bootstrap.servers = kafka-1:9093,kafka-2:9093,kafka-3:9093
-    a1.channels.channel1.kafka.topic = channel1
-    a1.channels.channel1.kafka.producer.security.protocol = SASL_PLAINTEXT
-    a1.channels.channel1.kafka.producer.sasl.mechanism = GSSAPI
-    a1.channels.channel1.kafka.producer.sasl.kerberos.service.name = kafka
+    a1.sinks.sink1.type = org.apache.flume.sink.kafka.KafkaSink
+    a1.sinks.sink1.kafka.bootstrap.servers = kafka-1:9093,kafka-2:9093,kafka-3:9093
+    a1.sinks.sink1.kafka.topic = mytopic
+    a1.sinks.sink1.kafka.producer.security.protocol = SASL_PLAINTEXT
+    a1.sinks.sink1.kafka.producer.sasl.mechanism = GSSAPI
+    a1.sinks.sink1.kafka.producer.sasl.kerberos.service.name = kafka
 
 
 Example secure configuration using SASL_SSL:
 
 .. code-block:: properties
 
-    a1.channels.channel1.type = org.apache.flume.channel.kafka.KafkaChannel
-    a1.channels.channel1.kafka.bootstrap.servers = kafka-1:9093,kafka-2:9093,kafka-3:9093
-    a1.channels.channel1.kafka.topic = channel1
-    a1.channels.channel1.kafka.producer.security.protocol = SASL_SSL
-    a1.channels.channel1.kafka.producer.sasl.mechanism = GSSAPI
-    a1.channels.channel1.kafka.producer.sasl.kerberos.service.name = kafka
-    a1.channels.channel1.kafka.producer.ssl.truststore.location = /path/to/truststore.jks
-    a1.channels.channel1.kafka.producer.ssl.truststore.password = <password to access the truststore>
+    a1.sinks.sink1.type = org.apache.flume.sink.kafka.KafkaSink
+    a1.sinks.sink1.kafka.bootstrap.servers = kafka-1:9093,kafka-2:9093,kafka-3:9093
+    a1.sinks.sink1.kafka.topic = mytopic
+    a1.sinks.sink1.kafka.producer.security.protocol = SASL_SSL
+    a1.sinks.sink1.kafka.producer.sasl.mechanism = GSSAPI
+    a1.sinks.sink1.kafka.producer.sasl.kerberos.service.name = kafka
+    a1.sinks.sink1.kafka.producer.ssl.truststore.location = /path/to/truststore.jks
+    a1.sinks.sink1.kafka.producer.ssl.truststore.password = <password to access the truststore>
 
 
 Sample JAAS file. For reference of its content please see client config sections of the desired authentication mechanism (GSSAPI/PLAIN)
@@ -2901,6 +2967,74 @@ that the operating system user of the Fl
     };
 
 
+HTTP Sink
+~~~~~~~~~
+
+Behaviour of this sink is that it will take events from the channel, and
+send those events to a remote service using an HTTP POST request. The event
+content is sent as the POST body.
+
+Error handling behaviour of this sink depends on the HTTP response returned
+by the target server. The sink backoff/ready status is configurable, as is the
+transaction commit/rollback result and whether the event contributes to the
+successful event drain count.
+
+Any malformed HTTP response returned by the server where the status code is
+not readable will result in a backoff signal and the event is not consumed
+from the channel.
+
+Required properties are in **bold**.
+
+========================== ================= ===========================================================================================
+Property Name              Default           Description
+========================== ================= ===========================================================================================
+**channel**                --
+**type**                   --                The component type name, needs to be ``http``.
+**endpoint**               --                The fully qualified URL endpoint to POST to
+connectTimeout             5000              The socket connection timeout in milliseconds
+requestTimeout             5000              The maximum request processing time in milliseconds
+contentTypeHeader          text/plain        The HTTP Content-Type header
+acceptHeader               text/plain        The HTTP Accept header value
+defaultBackoff             true              Whether to backoff by default on receiving all HTTP status codes
+defaultRollback            true              Whether to rollback by default on receiving all HTTP status codes
+defaultIncrementMetrics    false             Whether to increment metrics by default on receiving all HTTP status codes
+backoff.CODE               --                Configures a specific backoff for an individual (i.e. 200) code or a group (i.e. 2XX) code
+rollback.CODE              --                Configures a specific rollback for an individual (i.e. 200) code or a group (i.e. 2XX) code
+incrementMetrics.CODE      --                Configures a specific metrics increment for an individual (i.e. 200) code or a group (i.e. 2XX) code
+========================== ================= ===========================================================================================
+
+Note that the most specific HTTP status code match is used for the backoff,
+rollback and incrementMetrics configuration options. If there are configuration
+values for both 2XX and 200 status codes, then 200 HTTP codes will use the 200
+value, and all other HTTP codes in the 201-299 range will use the 2XX value.
+
+Any empty or null events are consumed without any request being made to the
+HTTP endpoint.
+
+Example for agent named a1:
+
+.. code-block:: properties
+
+  a1.channels = c1
+  a1.sinks = k1
+  a1.sinks.k1.type = http
+  a1.sinks.k1.channel = c1
+  a1.sinks.k1.endpoint = http://localhost:8080/someuri
+  a1.sinks.k1.connectTimeout = 2000
+  a1.sinks.k1.requestTimeout = 2000
+  a1.sinks.k1.acceptHeader = application/json
+  a1.sinks.k1.contentTypeHeader = application/json
+  a1.sinks.k1.defaultBackoff = true
+  a1.sinks.k1.defaultRollback = true
+  a1.sinks.k1.defaultIncrementMetrics = false
+  a1.sinks.k1.backoff.4XX = false
+  a1.sinks.k1.rollback.4XX = false
+  a1.sinks.k1.incrementMetrics.4XX = true
+  a1.sinks.k1.backoff.200 = false
+  a1.sinks.k1.rollback.200 = false
+  a1.sinks.k1.incrementMetrics.200 = true
+
+
 Custom Sink
 ~~~~~~~~~~~
 
@@ -3062,7 +3196,7 @@ pollTimeout
                                                                      https://kafka.apache.org/090/javadoc/org/apache/kafka/clients/consumer/KafkaConsumer.html#poll(long)
 defaultPartitionId                       --                          Specifies a Kafka partition ID (integer) for all events in this channel to be sent to, unless
                                                                      overriden by ``partitionIdHeader``. By default, if this property is not set, events will be
-                                                                     distributed by the Kafka Producer's partitioner - including by ``key`` if specified (or by a
+                                                                     distributed by the Kafka Producer's partitioner - including by ``key`` if specified (or by a 
                                                                      partitioner specified by ``kafka.partitioner.class``).
 partitionIdHeader                        --                          When set, the producer will take the value of the field named using the value of this property
                                                                      from the event header and send the message to the specified partition of the topic. If the
@@ -3831,15 +3965,16 @@ Timestamp Interceptor
 ~~~~~~~~~~~~~~~~~~~~~
 
 This interceptor inserts into the event headers, the time in millis at which it processes the event. This interceptor
-inserts a header with key ``timestamp`` whose value is the relevant timestamp. This interceptor
-can preserve an existing timestamp if it is already present in the configuration.
+inserts a header with key ``timestamp`` (or as specified by the ``header`` property) whose value is the relevant timestamp.
+This interceptor can preserve an existing timestamp if it is already present in the configuration.
 
-================  =======  ========================================================================
-Property Name     Default  Description
-================  =======  ========================================================================
-**type**          --       The component type name, has to be ``timestamp`` or the FQCN
-preserveExisting  false    If the timestamp already exists, should it be preserved - true or false
-================  =======  ========================================================================
+================  =========  ========================================================================
+Property Name     Default    Description
+================  =========  ========================================================================
+**type**          --         The component type name, has to be ``timestamp`` or the FQCN
+header            timestamp  The name of the header in which to place the generated timestamp.
+preserveExisting  false      If the timestamp already exists, should it be preserved - true or false
+================  =========  ========================================================================
 
 Example for agent named a1:
 
@@ -3875,7 +4010,6 @@ Example for agent named a1:
   a1.channels = c1
   a1.sources.r1.interceptors = i1
   a1.sources.r1.interceptors.i1.type = host
-  a1.sources.r1.interceptors.i1.hostHeader = hostname
 
 Static Interceptor
 ~~~~~~~~~~~~~~~~~~
@@ -3907,6 +4041,25 @@ Example for agent named a1:
   a1.sources.r1.interceptors.i1.key = datacenter
   a1.sources.r1.interceptors.i1.value = NEW_YORK
 
+
+Remove Header Interceptor
+~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+This interceptor manipulates Flume event headers, by removing one or many headers. It can remove a statically defined header, headers based on a regular expression or headers in a list. If none of these is defined, or if no header matches the criteria, the Flume events are not modified.
+
+Note that if only one header needs to be removed, specifying it by name provides performance benefits over the other 2 methods.
+
+=====================  ===========  ===============================================================
+Property Name          Default      Description
+=====================  ===========  ===============================================================
+**type**               --           The component type name has to be ``remove_header``
+withName               --           Name of the header to remove
+fromList               --           List of headers to remove, separated with the separator specified by ``fromListSeparator``
+fromListSeparator      \\s*,\\s*    Regular expression used to separate multiple header names in the list specified by ``fromList``. Default is a comma surrounded by any number of whitespace characters
+matching               --           All the headers which names match this regular expression are removed
+=====================  ===========  ===============================================================
+
+
 UUID Interceptor
 ~~~~~~~~~~~~~~~~~~~~~~~~~~~
 
@@ -4101,7 +4254,7 @@ Log4J Appender
 
 Appends Log4j events to a flume agent's avro source. A client using this
 appender must have the flume-ng-sdk in the classpath (eg,
-flume-ng-sdk-1.8.0-SNAPSHOT.jar).
+flume-ng-sdk-1.8.0.jar).
 Required properties are in **bold**.
 
 =====================  =======  ==================================================================================
@@ -4165,7 +4318,7 @@ Load Balancing Log4J Appender
 
 Appends Log4j events to a list of flume agent's avro source. A client using this
 appender must have the flume-ng-sdk in the classpath (eg,
-flume-ng-sdk-1.8.0-SNAPSHOT.jar). This appender supports a round-robin and random
+flume-ng-sdk-1.8.0.jar). This appender supports a round-robin and random
 scheme for performing the load balancing. It also supports a configurable backoff
 timeout so that down agents are removed temporarily from the set of hosts
 Required properties are in **bold**.

Modified: websites/staging/flume/trunk/content/_sources/download.txt
==============================================================================
--- websites/staging/flume/trunk/content/_sources/download.txt (original)
+++ websites/staging/flume/trunk/content/_sources/download.txt Wed Oct  4 16:24:02 2017
@@ -10,8 +10,8 @@ originals on the main distribution serve
 
 .. csv-table::
 
-   "Apache Flume binary (tar.gz)",  `apache-flume-1.7.0-bin.tar.gz <http://www.apache.org/dyn/closer.lua/flume/1.7.0/apache-flume-1.7.0-bin.tar.gz>`_, `apache-flume-1.7.0-bin.tar.gz.md5 <http://www.apache.org/dist/flume/1.7.0/apache-flume-1.7.0-bin.tar.gz.md5>`_, `apache-flume-1.7.0-bin.tar.gz.sha1 <http://www.apache.org/dist/flume/1.7.0/apache-flume-1.7.0-bin.tar.gz.sha1>`_, `apache-flume-1.7.0-bin.tar.gz.asc <http://www.apache.org/dist/flume/1.7.0/apache-flume-1.7.0-bin.tar.gz.asc>`_
-  "Apache Flume source (tar.gz)",  `apache-flume-1.7.0-src.tar.gz <http://www.apache.org/dyn/closer.lua/flume/1.7.0/apache-flume-1.7.0-src.tar.gz>`_, `apache-flume-1.7.0-src.tar.gz.md5 <http://www.apache.org/dist/flume/1.7.0/apache-flume-1.7.0-src.tar.gz.md5>`_, `apache-flume-1.7.0-src.tar.gz.sha1 <http://www.apache.org/dist/flume/1.7.0/apache-flume-1.7.0-src.tar.gz.sha1>`_, `apache-flume-1.7.0-src.tar.gz.asc <http://www.apache.org/dist/flume/1.7.0/apache-flume-1.7.0-src.tar.gz.asc>`_
+   "Apache Flume binary (tar.gz)",  `apache-flume-1.8.0-bin.tar.gz <http://www.apache.org/dyn/closer.lua/flume/1.8.0/apache-flume-1.8.0-bin.tar.gz>`_, `apache-flume-1.8.0-bin.tar.gz.md5 <http://www.apache.org/dist/flume/1.8.0/apache-flume-1.8.0-bin.tar.gz.md5>`_, `apache-flume-1.8.0-bin.tar.gz.sha1 <http://www.apache.org/dist/flume/1.8.0/apache-flume-1.8.0-bin.tar.gz.sha1>`_, `apache-flume-1.8.0-bin.tar.gz.asc <http://www.apache.org/dist/flume/1.8.0/apache-flume-1.8.0-bin.tar.gz.asc>`_
+  "Apache Flume source (tar.gz)",  `apache-flume-1.8.0-src.tar.gz <http://www.apache.org/dyn/closer.lua/flume/1.8.0/apache-flume-1.8.0-src.tar.gz>`_, `apache-flume-1.8.0-src.tar.gz.md5 <http://www.apache.org/dist/flume/1.8.0/apache-flume-1.8.0-src.tar.gz.md5>`_, `apache-flume-1.8.0-src.tar.gz.sha1 <http://www.apache.org/dist/flume/1.8.0/apache-flume-1.8.0-src.tar.gz.sha1>`_, `apache-flume-1.8.0-src.tar.gz.asc <http://www.apache.org/dist/flume/1.8.0/apache-flume-1.8.0-src.tar.gz.asc>`_
 
 It is essential that you verify the integrity of the downloaded files using the PGP or MD5 signatures. Please read
 `Verifying Apache HTTP Server Releases <http://httpd.apache.org/dev/verification.html>`_ for more information on
@@ -23,9 +23,9 @@ as well as the asc signature file for th
 Then verify the signatures using::
 
     % gpg --import KEYS
-    % gpg --verify apache-flume-1.7.0-src.tar.gz.asc
+    % gpg --verify apache-flume-1.8.0-src.tar.gz.asc
 
-Apache Flume 1.7.0 is signed by Bessenyei Balázs Donát 35EAD82D
+Apache Flume 1.8.0 is signed by Denes Arvay 4199ACFF
 
 Alternatively, you can verify the MD5 or SHA1 signatures of the files. A program called md5, md5sum, or shasum is included in many
 Unix distributions for this purpose.

Modified: websites/staging/flume/trunk/content/_sources/index.txt
==============================================================================
--- websites/staging/flume/trunk/content/_sources/index.txt (original)
+++ websites/staging/flume/trunk/content/_sources/index.txt Wed Oct  4 16:24:02 2017
@@ -33,6 +33,72 @@ application.
 
 .. raw:: html
 
+   <h3>October 4, 2017 - Apache Flume 1.8.0 Released</h3>
+
+The Apache Flume team is pleased to announce the release of Flume 1.8.0.
+
+Flume is a distributed, reliable, and available service for efficiently
+collecting, aggregating, and moving large amounts of streaming event data.
+
+Version 1.8.0 is the eleventh Flume release as an Apache top-level project. Flume
+1.8.0 is stable, production-ready software, and is backwards-compatible with
+previous versions of the Flume 1.x codeline.
+
+Several months of active development went into this release: about 80 patches were committed since 1.7.0, representing many features, enhancements, and bug fixes. While the full change log can be found on the 1.8.0 release page (link below), here are a few new feature highlights:
+
+    * Add Interceptor to remove headers from event
+    * Provide netcat UDP source as alternative to TCP
+    * Support environment variables in configuration files
+
+Below is the list of people (from Git logs) who submitted and/or reviewed
+improvements to Flume during the 1.8.0 development cycle:
+
+* Andras Beni
+* Ashish Paliwal
+* Attila Simon
+* Ben Wheeler
+* Bessenyei Balázs Donát
+* Chris Horrocks
+* Denes Arvay
+* Ferenc Szabo
+* Gabriel Commeau
+* Hari Shreedharan
+* Jeff Holoman
+* Lior Zeno
+* Marcell Hegedus
+* Mike Percy
+* Miklos Csanady
+* Peter Ableda
+* Peter Chen
+* Ping Wang
+* Pravin D'silva
+* Robin Wang
+* Roshan Naik
+* Satoshi Iijima
+* Shang Wu
+* Siddharth Ahuja
+* Takafumi Saito
+* TeddyBear1314
+* Theodore michael Malaska
+* Tristan Stevens
+* dengkai02
+* eskrm
+* filippovmn
+* loleek
+
+The full change log and documentation are available on the
+`Flume 1.8.0 release page <releases/1.8.0.html>`__.
+
+This release can be downloaded from the Flume `Download <download.html>`__ page.
+
+Your contributions, feedback, help and support make Flume better!
+For more information on how to report problems or contribute,
+please visit our `Get Involved <getinvolved.html>`__ page.
+
+The Apache Flume Team
+
+.. raw:: html
+
    <h3>October 17, 2016 - Apache Flume 1.7.0 Released</h3>
 
 The Apache Flume team is pleased to announce the release of Flume 1.7.0.

Added: websites/staging/flume/trunk/content/_sources/releases/1.8.0.txt
==============================================================================
--- websites/staging/flume/trunk/content/_sources/releases/1.8.0.txt (added)
+++ websites/staging/flume/trunk/content/_sources/releases/1.8.0.txt Wed Oct  4 16:24:02 2017
@@ -0,0 +1,80 @@
+===============
+Version 1.8.0
+===============
+
+.. rubric:: Status of this release
+
+Apache Flume 1.8.0 is the eleventh release of Flume as an Apache top-level project
+(TLP). Apache Flume 1.8.0 is production-ready software.
+
+.. rubric:: Release Documentation
+
+* `Flume 1.8.0 User Guide <../FlumeUserGuide.html>`__ (also in `pdf <content/1.8.0/FlumeUserGuide.pdf>`__)
+* `Flume 1.8.0 Developer Guide <../FlumeDeveloperGuide.html>`__ (also in `pdf <content/1.8.0/FlumeDeveloperGuide.pdf>`__)
+* `Flume 1.8.0 API Documentation <content/1.8.0/apidocs/index.html>`__
+
+.. rubric:: Changes
+
+Release Notes - Flume - Version v1.8.0
+
+** New Feature
+    * [`FLUME-2171 <https://issues.apache.org/jira/browse/FLUME-2171>`__] - Add Interceptor to remove headers from event
+    * [`FLUME-2917 <https://issues.apache.org/jira/browse/FLUME-2917>`__] - Provide netcat UDP source as alternative to TCP
+    * [`FLUME-2993 <https://issues.apache.org/jira/browse/FLUME-2993>`__] - Support environment variables in configuration files
+
+** Improvement
+    * [`FLUME-1520 <https://issues.apache.org/jira/browse/FLUME-1520>`__] - Timestamp interceptor should support custom headers
+    * [`FLUME-2945 <https://issues.apache.org/jira/browse/FLUME-2945>`__] - Bump java target version to 1.8
+    * [`FLUME-3020 <https://issues.apache.org/jira/browse/FLUME-3020>`__] - Improve HDFSEventSink Escape Ingestion by more then 10x by not getting InetAddress on every record
+    * [`FLUME-3025 <https://issues.apache.org/jira/browse/FLUME-3025>`__] - Expose FileChannel.open on JMX
+    * [`FLUME-3072 <https://issues.apache.org/jira/browse/FLUME-3072>`__] - Add IP address to headers in flume log4j appender
+    * [`FLUME-3092 <https://issues.apache.org/jira/browse/FLUME-3092>`__] - Extend the FileChannel's monitoring metrics
+    * [`FLUME-3100 <https://issues.apache.org/jira/browse/FLUME-3100>`__] - Support arbitrary header substitution for topic of Kafka
+    * [`FLUME-3144 <https://issues.apache.org/jira/browse/FLUME-3144>`__] - Improve Log4jAppender's performance by allowing logging collection of messages
+
+** Bug
+    * [`FLUME-2620 <https://issues.apache.org/jira/browse/FLUME-2620>`__] - File channel throws NullPointerException if a header value is null
+    * [`FLUME-2752 <https://issues.apache.org/jira/browse/FLUME-2752>`__] - Flume AvroSource will leak the memory and the OOM will be happened.
+    * [`FLUME-2812 <https://issues.apache.org/jira/browse/FLUME-2812>`__] - Exception in thread "SinkRunner-PollingRunner-DefaultSinkProcessor" java.lang.Error: Maximum permit count exceeded
+    * [`FLUME-2857 <https://issues.apache.org/jira/browse/FLUME-2857>`__] - Kafka Source/Channel/Sink does not restore default values when live update config
+    * [`FLUME-2905 <https://issues.apache.org/jira/browse/FLUME-2905>`__] - NetcatSource - Socket not closed when an exception is encountered during start() leading to file descriptor leaks
+    * [`FLUME-2991 <https://issues.apache.org/jira/browse/FLUME-2991>`__] - ExecSource command execution starts before starting the sourceCounter
+    * [`FLUME-3027 <https://issues.apache.org/jira/browse/FLUME-3027>`__] - Kafka Channel should clear offsets map after commit
+    * [`FLUME-3031 <https://issues.apache.org/jira/browse/FLUME-3031>`__] - sequence source should reset its counter for event body on channel exception
+    * [`FLUME-3043 <https://issues.apache.org/jira/browse/FLUME-3043>`__] - KafkaSink SinkCallback throws NullPointerException when Log4J level is debug
+    * [`FLUME-3046 <https://issues.apache.org/jira/browse/FLUME-3046>`__] - Kafka Sink and Source Configuration Improvements
+    * [`FLUME-3049 <https://issues.apache.org/jira/browse/FLUME-3049>`__] - Wrapping the exception into SecurityException in UGIExecutor.execute hides the original one
+    * [`FLUME-3057 <https://issues.apache.org/jira/browse/FLUME-3057>`__] - Build fails due to unsupported snappy-java version on ppc64le
+    * [`FLUME-3080 <https://issues.apache.org/jira/browse/FLUME-3080>`__] - Close failure in HDFS Sink might cause data loss
+    * [`FLUME-3083 <https://issues.apache.org/jira/browse/FLUME-3083>`__] - Taildir source can miss events if file updated in same second as file close
+    * [`FLUME-3085 <https://issues.apache.org/jira/browse/FLUME-3085>`__] - HDFS Sink can skip flushing some BucketWriters, might lead to data loss
+    * [`FLUME-3112 <https://issues.apache.org/jira/browse/FLUME-3112>`__] - Upgrade jackson-core library dependency
+    * [`FLUME-3127 <https://issues.apache.org/jira/browse/FLUME-3127>`__] - Upgrade libfb303 library dependency
+    * [`FLUME-3131 <https://issues.apache.org/jira/browse/FLUME-3131>`__] - Upgrade spring framework library dependencies
+    * [`FLUME-3132 <https://issues.apache.org/jira/browse/FLUME-3132>`__] - Upgrade tomcat jasper library dependencies
+    * [`FLUME-3135 <https://issues.apache.org/jira/browse/FLUME-3135>`__] - property logger in org.apache.flume.interceptor.RegexFilteringInterceptor confused
+    * [`FLUME-3141 <https://issues.apache.org/jira/browse/FLUME-3141>`__] - Small typo found in RegexHbaseEventSerializer.java
+    * [`FLUME-3152 <https://issues.apache.org/jira/browse/FLUME-3152>`__] - Add Flume Metric for Backup Checkpoint Errors
+    * [`FLUME-3155 <https://issues.apache.org/jira/browse/FLUME-3155>`__] - Use batch mode in mvn to fix Travis CI error
+    * [`FLUME-3157 <https://issues.apache.org/jira/browse/FLUME-3157>`__] - Refactor TestHDFSEventSinkOnMiniCluster to not use LeaseManager private API
+    * [`FLUME-3173 <https://issues.apache.org/jira/browse/FLUME-3173>`__] - Upgrade joda-time
+    * [`FLUME-3174 <https://issues.apache.org/jira/browse/FLUME-3174>`__] - HdfsSink AWS S3A authentication does not work on JDK 8
+    * [`FLUME-3175 <https://issues.apache.org/jira/browse/FLUME-3175>`__] - Javadoc generation fails due to Java8's strict doclint
+
+** Documentation
+    * [`FLUME-2175 <https://issues.apache.org/jira/browse/FLUME-2175>`__] - Update Developer Guide with notes on how to upgrade Protocol Buffer version
+    * [`FLUME-2817 <https://issues.apache.org/jira/browse/FLUME-2817>`__] - Sink for multi-agent flow example in user guide is set up incorrectly
+
+** Wish
+    * [`FLUME-2579 <https://issues.apache.org/jira/browse/FLUME-2579>`__] - JMS source support durable subscriptions and message listening
+
+** Question
+    * [`FLUME-2427 <https://issues.apache.org/jira/browse/FLUME-2427>`__] - java.lang.NoSuchMethodException and warning on HDFS (S3) sink
+
+** Task
+    * [`FLUME-3093 <https://issues.apache.org/jira/browse/FLUME-3093>`__] - Groundwork for version changes in root pom
+    * [`FLUME-3154 <https://issues.apache.org/jira/browse/FLUME-3154>`__] - Add HBase client version check to AsyncHBaseSink and HBaseSink
+
+** Test
+    * [`FLUME-2997 <https://issues.apache.org/jira/browse/FLUME-2997>`__] - Fix flaky junit test in SpillableMemoryChannel
+    * [`FLUME-3002 <https://issues.apache.org/jira/browse/FLUME-3002>`__] - Some tests in TestBucketWriter are flaky

Modified: websites/staging/flume/trunk/content/_sources/releases/index.txt
==============================================================================
--- websites/staging/flume/trunk/content/_sources/releases/index.txt (original)
+++ websites/staging/flume/trunk/content/_sources/releases/index.txt Wed Oct  4 16:24:02 2017
@@ -3,13 +3,13 @@ Releases
 
 .. rubric:: Current Release
 
-The current stable release is `Apache Flume Version 1.7.0 <1.7.0.html>`__.
+The current stable release is `Apache Flume Version 1.8.0 <1.8.0.html>`__.
 
 .. toctree::
    :maxdepth: 1
    :hidden:
 
-   1.7.0
+   1.8.0
 
 .. rubric:: Previous Releases
 
@@ -17,6 +17,7 @@ The current stable release is `Apache Fl
    :maxdepth: 1
    :glob:
 
+   1.7.0
    1.6.0
    1.5.2
    1.5.0.1

Modified: websites/staging/flume/trunk/content/documentation.html
==============================================================================
--- websites/staging/flume/trunk/content/documentation.html (original)
+++ websites/staging/flume/trunk/content/documentation.html Wed Oct  4 16:24:02 2017
@@ -25,7 +25,7 @@
     <script type="text/javascript" src="_static/underscore.js"></script>
     <script type="text/javascript" src="_static/doctools.js"></script>
     <link rel="top" title="Apache Flume" href="index.html" />
-    <link rel="next" title="Flume 1.7.0 User Guide" href="FlumeUserGuide.html" />
+    <link rel="next" title="Flume 1.8.0 User Guide" href="FlumeUserGuide.html" />
     <link rel="prev" title="Download" href="download.html" /> 
   </head>
   <body>
@@ -115,7 +115,7 @@ been released.</p>
       <div class="clearer"></div>
     </div>
 <div class="footer">
-    &copy; Copyright 2009-2012 The Apache Software Foundation. Apache Flume, Flume, Apache, the Apache feather logo, and the Apache Flume project logo are trademarks of The Apache Software Foundation..
+    &copy; Copyright 2009-2017 The Apache Software Foundation. Apache Flume, Flume, Apache, the Apache feather logo, and the Apache Flume project logo are trademarks of The Apache Software Foundation..
 </div>
   </body>
 </html>
\ No newline at end of file

Modified: websites/staging/flume/trunk/content/download.html
==============================================================================
--- websites/staging/flume/trunk/content/download.html (original)
+++ websites/staging/flume/trunk/content/download.html Wed Oct  4 16:24:02 2017
@@ -74,16 +74,16 @@ originals on the main distribution serve
 </colgroup>
 <tbody valign="top">
 <tr class="row-odd"><td>Apache Flume binary (tar.gz)</td>
-<td><a class="reference external" href="http://www.apache.org/dyn/closer.lua/flume/1.7.0/apache-flume-1.7.0-bin.tar.gz">apache-flume-1.7.0-bin.tar.gz</a></td>
-<td><a class="reference external" href="http://www.apache.org/dist/flume/1.7.0/apache-flume-1.7.0-bin.tar.gz.md5">apache-flume-1.7.0-bin.tar.gz.md5</a></td>
-<td><a class="reference external" href="http://www.apache.org/dist/flume/1.7.0/apache-flume-1.7.0-bin.tar.gz.sha1">apache-flume-1.7.0-bin.tar.gz.sha1</a></td>
-<td><a class="reference external" href="http://www.apache.org/dist/flume/1.7.0/apache-flume-1.7.0-bin.tar.gz.asc">apache-flume-1.7.0-bin.tar.gz.asc</a></td>
+<td><a class="reference external" href="http://www.apache.org/dyn/closer.lua/flume/1.8.0/apache-flume-1.8.0-bin.tar.gz">apache-flume-1.8.0-bin.tar.gz</a></td>
+<td><a class="reference external" href="http://www.apache.org/dist/flume/1.8.0/apache-flume-1.8.0-bin.tar.gz.md5">apache-flume-1.8.0-bin.tar.gz.md5</a></td>
+<td><a class="reference external" href="http://www.apache.org/dist/flume/1.8.0/apache-flume-1.8.0-bin.tar.gz.sha1">apache-flume-1.8.0-bin.tar.gz.sha1</a></td>
+<td><a class="reference external" href="http://www.apache.org/dist/flume/1.8.0/apache-flume-1.8.0-bin.tar.gz.asc">apache-flume-1.8.0-bin.tar.gz.asc</a></td>
 </tr>
 <tr class="row-even"><td>Apache Flume source (tar.gz)</td>
-<td><a class="reference external" href="http://www.apache.org/dyn/closer.lua/flume/1.7.0/apache-flume-1.7.0-src.tar.gz">apache-flume-1.7.0-src.tar.gz</a></td>
-<td><a class="reference external" href="http://www.apache.org/dist/flume/1.7.0/apache-flume-1.7.0-src.tar.gz.md5">apache-flume-1.7.0-src.tar.gz.md5</a></td>
-<td><a class="reference external" href="http://www.apache.org/dist/flume/1.7.0/apache-flume-1.7.0-src.tar.gz.sha1">apache-flume-1.7.0-src.tar.gz.sha1</a></td>
-<td><a class="reference external" href="http://www.apache.org/dist/flume/1.7.0/apache-flume-1.7.0-src.tar.gz.asc">apache-flume-1.7.0-src.tar.gz.asc</a></td>
+<td><a class="reference external" href="http://www.apache.org/dyn/closer.lua/flume/1.8.0/apache-flume-1.8.0-src.tar.gz">apache-flume-1.8.0-src.tar.gz</a></td>
+<td><a class="reference external" href="http://www.apache.org/dist/flume/1.8.0/apache-flume-1.8.0-src.tar.gz.md5">apache-flume-1.8.0-src.tar.gz.md5</a></td>
+<td><a class="reference external" href="http://www.apache.org/dist/flume/1.8.0/apache-flume-1.8.0-src.tar.gz.sha1">apache-flume-1.8.0-src.tar.gz.sha1</a></td>
+<td><a class="reference external" href="http://www.apache.org/dist/flume/1.8.0/apache-flume-1.8.0-src.tar.gz.asc">apache-flume-1.8.0-src.tar.gz.asc</a></td>
 </tr>
 </tbody>
 </table>
@@ -95,10 +95,10 @@ as well as the asc signature file for th
 <a class="reference external" href="http://www.apache.org/dist/flume/">main distribution directory</a> rather than from a mirror.
 Then verify the signatures using:</p>
 <div class="highlight-none"><div class="highlight"><pre>% gpg --import KEYS
-% gpg --verify apache-flume-1.7.0-src.tar.gz.asc
+% gpg --verify apache-flume-1.8.0-src.tar.gz.asc
 </pre></div>
 </div>
-<p>Apache Flume 1.7.0 is signed by Bessenyei Balázs Donát 35EAD82D</p>
+<p>Apache Flume 1.8.0 is signed by Denes Arvay 4199ACFF</p>
 <p>Alternatively, you can verify the MD5 or SHA1 signatures of the files. A program called md5, md5sum, or shasum is included in many
 Unix distributions for this purpose.</p>
 <p class="rubric">Previous_Releases</p>
@@ -149,7 +149,7 @@ Unix distributions for this purpose.</p>
       <div class="clearer"></div>
     </div>
 <div class="footer">
-    &copy; Copyright 2009-2012 The Apache Software Foundation. Apache Flume, Flume, Apache, the Apache feather logo, and the Apache Flume project logo are trademarks of The Apache Software Foundation..
+    &copy; Copyright 2009-2017 The Apache Software Foundation. Apache Flume, Flume, Apache, the Apache feather logo, and the Apache Flume project logo are trademarks of The Apache Software Foundation..
 </div>
   </body>
 </html>
\ No newline at end of file

Modified: websites/staging/flume/trunk/content/getinvolved.html
==============================================================================
--- websites/staging/flume/trunk/content/getinvolved.html (original)
+++ websites/staging/flume/trunk/content/getinvolved.html Wed Oct  4 16:24:02 2017
@@ -115,7 +115,7 @@ you find at: <a class="reference externa
       <div class="clearer"></div>
     </div>
 <div class="footer">
-    &copy; Copyright 2009-2012 The Apache Software Foundation. Apache Flume, Flume, Apache, the Apache feather logo, and the Apache Flume project logo are trademarks of The Apache Software Foundation..
+    &copy; Copyright 2009-2017 The Apache Software Foundation. Apache Flume, Flume, Apache, the Apache feather logo, and the Apache Flume project logo are trademarks of The Apache Software Foundation..
 </div>
   </body>
 </html>
\ No newline at end of file

Modified: websites/staging/flume/trunk/content/index.html
==============================================================================
--- websites/staging/flume/trunk/content/index.html (original)
+++ websites/staging/flume/trunk/content/index.html Wed Oct  4 16:24:02 2017
@@ -70,6 +70,63 @@ application.</p>
 <img alt="Agent component diagram" src="_images/DevGuide_image00.png" />
 </div>
 <p class="rubric">News</p>
+<h3>October 4, 2017 - Apache Flume 1.8.0 Released</h3><p>The Apache Flume team is pleased to announce the release of Flume 1.8.0.</p>
+<p>Flume is a distributed, reliable, and available service for efficiently
+collecting, aggregating, and moving large amounts of streaming event data.</p>
+<p>Version 1.8.0 is the eleventh Flume release as an Apache top-level project. Flume
+1.8.0 is stable, production-ready software, and is backwards-compatible with
+previous versions of the Flume 1.x codeline.</p>
+<p>Several months of active development went into this release: about 80 patches were committed since 1.7.0, representing many features, enhancements, and bug fixes. While the full change log can be found on the 1.8.0 release page (link below), here are a few new feature highlights:</p>
+<blockquote>
+<div><ul class="simple">
+<li>Add Interceptor to remove headers from event</li>
+<li>Provide netcat UDP source as alternative to TCP</li>
+<li>Support environment variables in configuration files</li>
+</ul>
+</div></blockquote>
+<p>Below is the list of people (from Git logs) who submitted and/or reviewed
+improvements to Flume during the 1.8.0 development cycle:</p>
+<ul class="simple">
+<li>Andras Beni</li>
+<li>Ashish Paliwal</li>
+<li>Attila Simon</li>
+<li>Ben Wheeler</li>
+<li>Bessenyei Balázs Donát</li>
+<li>Chris Horrocks</li>
+<li>Denes Arvay</li>
+<li>Ferenc Szabo</li>
+<li>Gabriel Commeau</li>
+<li>Hari Shreedharan</li>
+<li>Jeff Holoman</li>
+<li>Lior Zeno</li>
+<li>Marcell Hegedus</li>
+<li>Mike Percy</li>
+<li>Miklos Csanady</li>
+<li>Peter Ableda</li>
+<li>Peter Chen</li>
+<li>Ping Wang</li>
+<li>Pravin D&#8217;silva</li>
+<li>Robin Wang</li>
+<li>Roshan Naik</li>
+<li>Satoshi Iijima</li>
+<li>Shang Wu</li>
+<li>Siddharth Ahuja</li>
+<li>Takafumi Saito</li>
+<li>TeddyBear1314</li>
+<li>Theodore michael Malaska</li>
+<li>Tristan Stevens</li>
+<li>dengkai02</li>
+<li>eskrm</li>
+<li>filippovmn</li>
+<li>loleek</li>
+</ul>
+<p>The full change log and documentation are available on the
+<a class="reference external" href="releases/1.8.0.html">Flume 1.8.0 release page</a>.</p>
+<p>This release can be downloaded from the Flume <a class="reference external" href="download.html">Download</a> page.</p>
+<p>Your contributions, feedback, help and support make Flume better!
+For more information on how to report problems or contribute,
+please visit our <a class="reference external" href="getinvolved.html">Get Involved</a> page.</p>
+<p>The Apache Flume Team</p>
 <h3>October 17, 2016 - Apache Flume 1.7.0 Released</h3><p>The Apache Flume team is pleased to announce the release of Flume 1.7.0.</p>
 <p>Flume is a distributed, reliable, and available service for efficiently
 collecting, aggregating, and moving large amounts of streaming event data.</p>
@@ -406,7 +463,7 @@ Feel free to post to the User&#8217;s ma
       <div class="clearer"></div>
     </div>
 <div class="footer">
-    &copy; Copyright 2009-2012 The Apache Software Foundation. Apache Flume, Flume, Apache, the Apache feather logo, and the Apache Flume project logo are trademarks of The Apache Software Foundation..
+    &copy; Copyright 2009-2017 The Apache Software Foundation. Apache Flume, Flume, Apache, the Apache feather logo, and the Apache Flume project logo are trademarks of The Apache Software Foundation..
 </div>
   </body>
 </html>
\ No newline at end of file

Modified: websites/staging/flume/trunk/content/license.html
==============================================================================
--- websites/staging/flume/trunk/content/license.html (original)
+++ websites/staging/flume/trunk/content/license.html Wed Oct  4 16:24:02 2017
@@ -198,7 +198,7 @@ accepting any such warranty or additiona
       <div class="clearer"></div>
     </div>
 <div class="footer">
-    &copy; Copyright 2009-2012 The Apache Software Foundation. Apache Flume, Flume, Apache, the Apache feather logo, and the Apache Flume project logo are trademarks of The Apache Software Foundation..
+    &copy; Copyright 2009-2017 The Apache Software Foundation. Apache Flume, Flume, Apache, the Apache feather logo, and the Apache Flume project logo are trademarks of The Apache Software Foundation..
 </div>
   </body>
 </html>
\ No newline at end of file

Modified: websites/staging/flume/trunk/content/mailinglists.html
==============================================================================
--- websites/staging/flume/trunk/content/mailinglists.html (original)
+++ websites/staging/flume/trunk/content/mailinglists.html Wed Oct  4 16:24:02 2017
@@ -144,7 +144,7 @@
       <div class="clearer"></div>
     </div>
 <div class="footer">
-    &copy; Copyright 2009-2012 The Apache Software Foundation. Apache Flume, Flume, Apache, the Apache feather logo, and the Apache Flume project logo are trademarks of The Apache Software Foundation..
+    &copy; Copyright 2009-2017 The Apache Software Foundation. Apache Flume, Flume, Apache, the Apache feather logo, and the Apache Flume project logo are trademarks of The Apache Software Foundation..
 </div>
   </body>
 </html>
\ No newline at end of file

Modified: websites/staging/flume/trunk/content/project-reports.html
==============================================================================
--- websites/staging/flume/trunk/content/project-reports.html (original)
+++ websites/staging/flume/trunk/content/project-reports.html Wed Oct  4 16:24:02 2017
@@ -1,5 +1,5 @@
 <!DOCTYPE html PUBLIC "-//W3C//DTD XHTML 1.0 Transitional//EN" "http://www.w3.org/TR/xhtml1/DTD/xhtml1-transitional.dtd">
-<!-- Generated by Apache Maven Doxia at Oct 17, 2016 -->
+<!-- Generated by Apache Maven Doxia at Oct 4, 2017 -->
 <html xmlns="http://www.w3.org/1999/xhtml" xml:lang="en" lang="en">
   <head>
     <meta http-equiv="Content-Type" content="text/html; charset=UTF-8" />
@@ -12,7 +12,7 @@
     <script type="text/javascript" src="./js/apache-maven-fluido.min.js"></script>
 
     
-    <meta name="Date-Revision-yyyymmdd" content="20161017" />
+    <meta name="Date-Revision-yyyymmdd" content="20171004" />
     <meta http-equiv="Content-Language" content="en" />
     
         </head>
@@ -27,7 +27,7 @@
                 
                             
                 
-                  <li id="publishDate" class="pull-right">Last Published: 2016-10-17</li> 
+                  <li id="publishDate" class="pull-right">Last Published: 2017-10-04</li> 
                     </ul>
       </div>
 
@@ -57,7 +57,7 @@
 
     <footer class="footer">
             <div class="container-fluid">
-              <div class="row span16">Copyright &copy;                    2009-2016
+              <div class="row span16">Copyright &copy;                    2009-2017
                         <a href="http://www.apache.org/">The Apache Software Foundation</a>.
             All Rights Reserved.      
                 

Modified: websites/staging/flume/trunk/content/releases/1.0.0.html
==============================================================================
--- websites/staging/flume/trunk/content/releases/1.0.0.html (original)
+++ websites/staging/flume/trunk/content/releases/1.0.0.html Wed Oct  4 16:24:02 2017
@@ -313,7 +313,7 @@ Incubator, and is labelled and should be
       <div class="clearer"></div>
     </div>
 <div class="footer">
-    &copy; Copyright 2009-2012 The Apache Software Foundation. Apache Flume, Flume, Apache, the Apache feather logo, and the Apache Flume project logo are trademarks of The Apache Software Foundation..
+    &copy; Copyright 2009-2017 The Apache Software Foundation. Apache Flume, Flume, Apache, the Apache feather logo, and the Apache Flume project logo are trademarks of The Apache Software Foundation..
 </div>
   </body>
 </html>
\ No newline at end of file

Modified: websites/staging/flume/trunk/content/releases/1.1.0.html
==============================================================================
--- websites/staging/flume/trunk/content/releases/1.1.0.html (original)
+++ websites/staging/flume/trunk/content/releases/1.1.0.html Wed Oct  4 16:24:02 2017
@@ -199,7 +199,7 @@ it is likely to change until a stable re
       <div class="clearer"></div>
     </div>
 <div class="footer">
-    &copy; Copyright 2009-2012 The Apache Software Foundation. Apache Flume, Flume, Apache, the Apache feather logo, and the Apache Flume project logo are trademarks of The Apache Software Foundation..
+    &copy; Copyright 2009-2017 The Apache Software Foundation. Apache Flume, Flume, Apache, the Apache feather logo, and the Apache Flume project logo are trademarks of The Apache Software Foundation..
 </div>
   </body>
 </html>
\ No newline at end of file

Modified: websites/staging/flume/trunk/content/releases/1.2.0.html
==============================================================================
--- websites/staging/flume/trunk/content/releases/1.2.0.html (original)
+++ websites/staging/flume/trunk/content/releases/1.2.0.html Wed Oct  4 16:24:02 2017
@@ -320,7 +320,7 @@ and functionality along with bug fixes a
       <div class="clearer"></div>
     </div>
 <div class="footer">
-    &copy; Copyright 2009-2012 The Apache Software Foundation. Apache Flume, Flume, Apache, the Apache feather logo, and the Apache Flume project logo are trademarks of The Apache Software Foundation..
+    &copy; Copyright 2009-2017 The Apache Software Foundation. Apache Flume, Flume, Apache, the Apache feather logo, and the Apache Flume project logo are trademarks of The Apache Software Foundation..
 </div>
   </body>
 </html>
\ No newline at end of file

Modified: websites/staging/flume/trunk/content/releases/1.3.0.html
==============================================================================
--- websites/staging/flume/trunk/content/releases/1.3.0.html (original)
+++ websites/staging/flume/trunk/content/releases/1.3.0.html Wed Oct  4 16:24:02 2017
@@ -305,7 +305,7 @@ enhancements.</p>
       <div class="clearer"></div>
     </div>
 <div class="footer">
-    &copy; Copyright 2009-2012 The Apache Software Foundation. Apache Flume, Flume, Apache, the Apache feather logo, and the Apache Flume project logo are trademarks of The Apache Software Foundation..
+    &copy; Copyright 2009-2017 The Apache Software Foundation. Apache Flume, Flume, Apache, the Apache feather logo, and the Apache Flume project logo are trademarks of The Apache Software Foundation..
 </div>
   </body>
 </html>
\ No newline at end of file

Modified: websites/staging/flume/trunk/content/releases/1.3.1.html
==============================================================================
--- websites/staging/flume/trunk/content/releases/1.3.1.html (original)
+++ websites/staging/flume/trunk/content/releases/1.3.1.html Wed Oct  4 16:24:02 2017
@@ -142,7 +142,7 @@ several bug fixes and performance enhanc
       <div class="clearer"></div>
     </div>
 <div class="footer">
-    &copy; Copyright 2009-2012 The Apache Software Foundation. Apache Flume, Flume, Apache, the Apache feather logo, and the Apache Flume project logo are trademarks of The Apache Software Foundation..
+    &copy; Copyright 2009-2017 The Apache Software Foundation. Apache Flume, Flume, Apache, the Apache feather logo, and the Apache Flume project logo are trademarks of The Apache Software Foundation..
 </div>
   </body>
 </html>
\ No newline at end of file

Modified: websites/staging/flume/trunk/content/releases/1.4.0.html
==============================================================================
--- websites/staging/flume/trunk/content/releases/1.4.0.html (original)
+++ websites/staging/flume/trunk/content/releases/1.4.0.html Wed Oct  4 16:24:02 2017
@@ -327,7 +327,7 @@
       <div class="clearer"></div>
     </div>
 <div class="footer">
-    &copy; Copyright 2009-2012 The Apache Software Foundation. Apache Flume, Flume, Apache, the Apache feather logo, and the Apache Flume project logo are trademarks of The Apache Software Foundation..
+    &copy; Copyright 2009-2017 The Apache Software Foundation. Apache Flume, Flume, Apache, the Apache feather logo, and the Apache Flume project logo are trademarks of The Apache Software Foundation..
 </div>
   </body>
 </html>
\ No newline at end of file

Modified: websites/staging/flume/trunk/content/releases/1.5.0.1.html
==============================================================================
--- websites/staging/flume/trunk/content/releases/1.5.0.1.html (original)
+++ websites/staging/flume/trunk/content/releases/1.5.0.1.html Wed Oct  4 16:24:02 2017
@@ -127,7 +127,7 @@
       <div class="clearer"></div>
     </div>
 <div class="footer">
-    &copy; Copyright 2009-2012 The Apache Software Foundation. Apache Flume, Flume, Apache, the Apache feather logo, and the Apache Flume project logo are trademarks of The Apache Software Foundation..
+    &copy; Copyright 2009-2017 The Apache Software Foundation. Apache Flume, Flume, Apache, the Apache feather logo, and the Apache Flume project logo are trademarks of The Apache Software Foundation..
 </div>
   </body>
 </html>
\ No newline at end of file

Modified: websites/staging/flume/trunk/content/releases/1.5.0.html
==============================================================================
--- websites/staging/flume/trunk/content/releases/1.5.0.html (original)
+++ websites/staging/flume/trunk/content/releases/1.5.0.html Wed Oct  4 16:24:02 2017
@@ -259,7 +259,7 @@
       <div class="clearer"></div>
     </div>
 <div class="footer">
-    &copy; Copyright 2009-2012 The Apache Software Foundation. Apache Flume, Flume, Apache, the Apache feather logo, and the Apache Flume project logo are trademarks of The Apache Software Foundation..
+    &copy; Copyright 2009-2017 The Apache Software Foundation. Apache Flume, Flume, Apache, the Apache feather logo, and the Apache Flume project logo are trademarks of The Apache Software Foundation..
 </div>
   </body>
 </html>
\ No newline at end of file

Modified: websites/staging/flume/trunk/content/releases/1.5.2.html
==============================================================================
--- websites/staging/flume/trunk/content/releases/1.5.2.html (original)
+++ websites/staging/flume/trunk/content/releases/1.5.2.html Wed Oct  4 16:24:02 2017
@@ -132,7 +132,7 @@ sources and sinks.</p>
       <div class="clearer"></div>
     </div>
 <div class="footer">
-    &copy; Copyright 2009-2012 The Apache Software Foundation. Apache Flume, Flume, Apache, the Apache feather logo, and the Apache Flume project logo are trademarks of The Apache Software Foundation..
+    &copy; Copyright 2009-2017 The Apache Software Foundation. Apache Flume, Flume, Apache, the Apache feather logo, and the Apache Flume project logo are trademarks of The Apache Software Foundation..
 </div>
   </body>
 </html>
\ No newline at end of file

Modified: websites/staging/flume/trunk/content/releases/1.6.0.html
==============================================================================
--- websites/staging/flume/trunk/content/releases/1.6.0.html (original)
+++ websites/staging/flume/trunk/content/releases/1.6.0.html Wed Oct  4 16:24:02 2017
@@ -257,7 +257,7 @@
       <div class="clearer"></div>
     </div>
 <div class="footer">
-    &copy; Copyright 2009-2012 The Apache Software Foundation. Apache Flume, Flume, Apache, the Apache feather logo, and the Apache Flume project logo are trademarks of The Apache Software Foundation..
+    &copy; Copyright 2009-2017 The Apache Software Foundation. Apache Flume, Flume, Apache, the Apache feather logo, and the Apache Flume project logo are trademarks of The Apache Software Foundation..
 </div>
   </body>
 </html>
\ No newline at end of file

Modified: websites/staging/flume/trunk/content/releases/1.7.0.html
==============================================================================
--- websites/staging/flume/trunk/content/releases/1.7.0.html (original)
+++ websites/staging/flume/trunk/content/releases/1.7.0.html Wed Oct  4 16:24:02 2017
@@ -27,7 +27,7 @@
     <link rel="top" title="Apache Flume" href="../index.html" />
     <link rel="up" title="Releases" href="index.html" />
     <link rel="next" title="Version 1.6.0" href="1.6.0.html" />
-    <link rel="prev" title="Releases" href="index.html" /> 
+    <link rel="prev" title="Version 1.8.0" href="1.8.0.html" /> 
   </head>
   <body>
 <div class="header">
@@ -245,7 +245,7 @@
       <div class="clearer"></div>
     </div>
 <div class="footer">
-    &copy; Copyright 2009-2012 The Apache Software Foundation. Apache Flume, Flume, Apache, the Apache feather logo, and the Apache Flume project logo are trademarks of The Apache Software Foundation..
+    &copy; Copyright 2009-2017 The Apache Software Foundation. Apache Flume, Flume, Apache, the Apache feather logo, and the Apache Flume project logo are trademarks of The Apache Software Foundation..
 </div>
   </body>
 </html>
\ No newline at end of file

Added: websites/staging/flume/trunk/content/releases/1.8.0.html
==============================================================================
--- websites/staging/flume/trunk/content/releases/1.8.0.html (added)
+++ websites/staging/flume/trunk/content/releases/1.8.0.html Wed Oct  4 16:24:02 2017
@@ -0,0 +1,204 @@
+
+<!DOCTYPE html PUBLIC "-//W3C//DTD XHTML 1.0 Transitional//EN"
+  "http://www.w3.org/TR/xhtml1/DTD/xhtml1-transitional.dtd">
+
+
+<html xmlns="http://www.w3.org/1999/xhtml">
+  <head>
+    <meta http-equiv="Content-Type" content="text/html; charset=utf-8" />
+    
+    <title>Version 1.8.0 &mdash; Apache Flume</title>
+    
+    <link rel="stylesheet" href="../_static/flume.css" type="text/css" />
+    <link rel="stylesheet" href="../_static/pygments.css" type="text/css" />
+    
+    <script type="text/javascript">
+      var DOCUMENTATION_OPTIONS = {
+        URL_ROOT:    '../',
+        VERSION:     '',
+        COLLAPSE_INDEX: false,
+        FILE_SUFFIX: '.html',
+        HAS_SOURCE:  true
+      };
+    </script>
+    <script type="text/javascript" src="../_static/jquery.js"></script>
+    <script type="text/javascript" src="../_static/underscore.js"></script>
+    <script type="text/javascript" src="../_static/doctools.js"></script>
+    <link rel="top" title="Apache Flume" href="../index.html" />
+    <link rel="up" title="Releases" href="index.html" />
+    <link rel="next" title="Version 1.7.0" href="1.7.0.html" />
+    <link rel="prev" title="Releases" href="index.html" /> 
+  </head>
+  <body>
+<div class="header">
+  <table width="100%" border="0">
+    <tr>
+      <td width="10%">
+        <div class="logo">
+          <a href="../index.html">
+            <img class="logo" src="../_static/flume-logo.png" alt="Logo"/>
+        </div>
+      </td>
+      <td width="2%">
+          <span class="trademark">&trade;</span>
+      </td>
+      <td width="68%" align="center" class="pageTitle">Apache Flume<sup><span class="trademark">&trade;</span></sup>
+      </td>
+      <td width="20%">
+          <a href="http://www.apache.org">
+            <img src="../_static/feather-small.png" alt="Apache Software Foundation" height="70"/>
+          </a>
+      </td>
+    </tr>
+  </table>
+</div>
+  
+
+    <div class="document">
+      <div class="documentwrapper">
+        <div class="bodywrapper">
+          <div class="body">
+            
+  <div class="section" id="version-1-8-0">
+<h1>Version 1.8.0<a class="headerlink" href="#version-1-8-0" title="Permalink to this headline">¶</a></h1>
+<p class="rubric">Status of this release</p>
+<p>Apache Flume 1.8.0 is the eleventh release of Flume as an Apache top-level project
+(TLP). Apache Flume 1.8.0 is production-ready software.</p>
+<p class="rubric">Release Documentation</p>
+<ul class="simple">
+<li><a class="reference external" href="../FlumeUserGuide.html">Flume 1.8.0 User Guide</a> (also in <a class="reference external" href="content/1.8.0/FlumeUserGuide.pdf">pdf</a>)</li>
+<li><a class="reference external" href="../FlumeDeveloperGuide.html">Flume 1.8.0 Developer Guide</a> (also in <a class="reference external" href="content/1.8.0/FlumeDeveloperGuide.pdf">pdf</a>)</li>
+<li><a class="reference external" href="content/1.8.0/apidocs/index.html">Flume 1.8.0 API Documentation</a></li>
+</ul>
+<p class="rubric">Changes</p>
+<p>Release Notes - Flume - Version v1.8.0</p>
+<dl class="docutils">
+<dt>** New Feature</dt>
+<dd><ul class="first last simple">
+<li>[<a class="reference external" href="https://issues.apache.org/jira/browse/FLUME-2171">FLUME-2171</a>] - Add Interceptor to remove headers from event</li>
+<li>[<a class="reference external" href="https://issues.apache.org/jira/browse/FLUME-2917">FLUME-2917</a>] - Provide netcat UDP source as alternative to TCP</li>
+<li>[<a class="reference external" href="https://issues.apache.org/jira/browse/FLUME-2993">FLUME-2993</a>] - Support environment variables in configuration files</li>
+</ul>
+</dd>
+<dt>** Improvement</dt>
+<dd><ul class="first last simple">
+<li>[<a class="reference external" href="https://issues.apache.org/jira/browse/FLUME-1520">FLUME-1520</a>] - Timestamp interceptor should support custom headers</li>
+<li>[<a class="reference external" href="https://issues.apache.org/jira/browse/FLUME-2945">FLUME-2945</a>] - Bump java target version to 1.8</li>
+<li>[<a class="reference external" href="https://issues.apache.org/jira/browse/FLUME-3020">FLUME-3020</a>] - Improve HDFSEventSink Escape Ingestion by more then 10x by not getting InetAddress on every record</li>
+<li>[<a class="reference external" href="https://issues.apache.org/jira/browse/FLUME-3025">FLUME-3025</a>] - Expose FileChannel.open on JMX</li>
+<li>[<a class="reference external" href="https://issues.apache.org/jira/browse/FLUME-3072">FLUME-3072</a>] - Add IP address to headers in flume log4j appender</li>
+<li>[<a class="reference external" href="https://issues.apache.org/jira/browse/FLUME-3092">FLUME-3092</a>] - Extend the FileChannel&#8217;s monitoring metrics</li>
+<li>[<a class="reference external" href="https://issues.apache.org/jira/browse/FLUME-3100">FLUME-3100</a>] - Support arbitrary header substitution for topic of Kafka</li>
+<li>[<a class="reference external" href="https://issues.apache.org/jira/browse/FLUME-3144">FLUME-3144</a>] - Improve Log4jAppender&#8217;s performance by allowing logging collection of messages</li>
+</ul>
+</dd>
+<dt>** Bug</dt>
+<dd><ul class="first last simple">
+<li>[<a class="reference external" href="https://issues.apache.org/jira/browse/FLUME-2620">FLUME-2620</a>] - File channel throws NullPointerException if a header value is null</li>
+<li>[<a class="reference external" href="https://issues.apache.org/jira/browse/FLUME-2752">FLUME-2752</a>] - Flume AvroSource will leak the memory and the OOM will be happened.</li>
+<li>[<a class="reference external" href="https://issues.apache.org/jira/browse/FLUME-2812">FLUME-2812</a>] - Exception in thread &#8220;SinkRunner-PollingRunner-DefaultSinkProcessor&#8221; java.lang.Error: Maximum permit count exceeded</li>
+<li>[<a class="reference external" href="https://issues.apache.org/jira/browse/FLUME-2857">FLUME-2857</a>] - Kafka Source/Channel/Sink does not restore default values when live update config</li>
+<li>[<a class="reference external" href="https://issues.apache.org/jira/browse/FLUME-2905">FLUME-2905</a>] - NetcatSource - Socket not closed when an exception is encountered during start() leading to file descriptor leaks</li>
+<li>[<a class="reference external" href="https://issues.apache.org/jira/browse/FLUME-2991">FLUME-2991</a>] - ExecSource command execution starts before starting the sourceCounter</li>
+<li>[<a class="reference external" href="https://issues.apache.org/jira/browse/FLUME-3027">FLUME-3027</a>] - Kafka Channel should clear offsets map after commit</li>
+<li>[<a class="reference external" href="https://issues.apache.org/jira/browse/FLUME-3031">FLUME-3031</a>] - sequence source should reset its counter for event body on channel exception</li>
+<li>[<a class="reference external" href="https://issues.apache.org/jira/browse/FLUME-3043">FLUME-3043</a>] - KafkaSink SinkCallback throws NullPointerException when Log4J level is debug</li>
+<li>[<a class="reference external" href="https://issues.apache.org/jira/browse/FLUME-3046">FLUME-3046</a>] - Kafka Sink and Source Configuration Improvements</li>
+<li>[<a class="reference external" href="https://issues.apache.org/jira/browse/FLUME-3049">FLUME-3049</a>] - Wrapping the exception into SecurityException in UGIExecutor.execute hides the original one</li>
+<li>[<a class="reference external" href="https://issues.apache.org/jira/browse/FLUME-3057">FLUME-3057</a>] - Build fails due to unsupported snappy-java version on ppc64le</li>
+<li>[<a class="reference external" href="https://issues.apache.org/jira/browse/FLUME-3080">FLUME-3080</a>] - Close failure in HDFS Sink might cause data loss</li>
+<li>[<a class="reference external" href="https://issues.apache.org/jira/browse/FLUME-3083">FLUME-3083</a>] - Taildir source can miss events if file updated in same second as file close</li>
+<li>[<a class="reference external" href="https://issues.apache.org/jira/browse/FLUME-3085">FLUME-3085</a>] - HDFS Sink can skip flushing some BucketWriters, might lead to data loss</li>
+<li>[<a class="reference external" href="https://issues.apache.org/jira/browse/FLUME-3112">FLUME-3112</a>] - Upgrade jackson-core library dependency</li>
+<li>[<a class="reference external" href="https://issues.apache.org/jira/browse/FLUME-3127">FLUME-3127</a>] - Upgrade libfb303 library dependency</li>
+<li>[<a class="reference external" href="https://issues.apache.org/jira/browse/FLUME-3131">FLUME-3131</a>] - Upgrade spring framework library dependencies</li>
+<li>[<a class="reference external" href="https://issues.apache.org/jira/browse/FLUME-3132">FLUME-3132</a>] - Upgrade tomcat jasper library dependencies</li>
+<li>[<a class="reference external" href="https://issues.apache.org/jira/browse/FLUME-3135">FLUME-3135</a>] - property logger in org.apache.flume.interceptor.RegexFilteringInterceptor confused</li>
+<li>[<a class="reference external" href="https://issues.apache.org/jira/browse/FLUME-3141">FLUME-3141</a>] - Small typo found in RegexHbaseEventSerializer.java</li>
+<li>[<a class="reference external" href="https://issues.apache.org/jira/browse/FLUME-3152">FLUME-3152</a>] - Add Flume Metric for Backup Checkpoint Errors</li>
+<li>[<a class="reference external" href="https://issues.apache.org/jira/browse/FLUME-3155">FLUME-3155</a>] - Use batch mode in mvn to fix Travis CI error</li>
+<li>[<a class="reference external" href="https://issues.apache.org/jira/browse/FLUME-3157">FLUME-3157</a>] - Refactor TestHDFSEventSinkOnMiniCluster to not use LeaseManager private API</li>
+<li>[<a class="reference external" href="https://issues.apache.org/jira/browse/FLUME-3173">FLUME-3173</a>] - Upgrade joda-time</li>
+<li>[<a class="reference external" href="https://issues.apache.org/jira/browse/FLUME-3174">FLUME-3174</a>] - HdfsSink AWS S3A authentication does not work on JDK 8</li>
+<li>[<a class="reference external" href="https://issues.apache.org/jira/browse/FLUME-3175">FLUME-3175</a>] - Javadoc generation fails due to Java8&#8217;s strict doclint</li>
+</ul>
+</dd>
+<dt>** Documentation</dt>
+<dd><ul class="first last simple">
+<li>[<a class="reference external" href="https://issues.apache.org/jira/browse/FLUME-2175">FLUME-2175</a>] - Update Developer Guide with notes on how to upgrade Protocol Buffer version</li>
+<li>[<a class="reference external" href="https://issues.apache.org/jira/browse/FLUME-2817">FLUME-2817</a>] - Sink for multi-agent flow example in user guide is set up incorrectly</li>
+</ul>
+</dd>
+<dt>** Wish</dt>
+<dd><ul class="first last simple">
+<li>[<a class="reference external" href="https://issues.apache.org/jira/browse/FLUME-2579">FLUME-2579</a>] - JMS source support durable subscriptions and message listening</li>
+</ul>
+</dd>
+<dt>** Question</dt>
+<dd><ul class="first last simple">
+<li>[<a class="reference external" href="https://issues.apache.org/jira/browse/FLUME-2427">FLUME-2427</a>] - java.lang.NoSuchMethodException and warning on HDFS (S3) sink</li>
+</ul>
+</dd>
+<dt>** Task</dt>
+<dd><ul class="first last simple">
+<li>[<a class="reference external" href="https://issues.apache.org/jira/browse/FLUME-3093">FLUME-3093</a>] - Groundwork for version changes in root pom</li>
+<li>[<a class="reference external" href="https://issues.apache.org/jira/browse/FLUME-3154">FLUME-3154</a>] - Add HBase client version check to AsyncHBaseSink and HBaseSink</li>
+</ul>
+</dd>
+<dt>** Test</dt>
+<dd><ul class="first last simple">
+<li>[<a class="reference external" href="https://issues.apache.org/jira/browse/FLUME-2997">FLUME-2997</a>] - Fix flaky junit test in SpillableMemoryChannel</li>
+<li>[<a class="reference external" href="https://issues.apache.org/jira/browse/FLUME-3002">FLUME-3002</a>] - Some tests in TestBucketWriter are flaky</li>
+</ul>
+</dd>
+</dl>
+</div>
+
+
+          </div>
+        </div>
+      </div>
+      <div class="sphinxsidebar">
+        <div class="sphinxsidebarwrapper"><h3><a href="../index.html">Apache Flume</a></h3>
+<ul>
+<li class="toctree-l1"><a class="reference internal" href="../getinvolved.html">How to Get Involved</a></li>
+<li class="toctree-l1"><a class="reference internal" href="../download.html">Download</a></li>
+<li class="toctree-l1"><a class="reference internal" href="../documentation.html">Documentation</a></li>
+<li class="toctree-l1"><a class="reference internal" href="index.html">Releases</a></li>
+<li class="toctree-l1"><a class="reference internal" href="../mailinglists.html">Mailing lists</a></li>
+<li class="toctree-l1"><a class="reference internal" href="../team.html">Team</a></li>
+<li class="toctree-l1"><a class="reference internal" href="../source.html">Source Repository</a></li>
+<li class="toctree-l1"><a class="reference internal" href="../license.html">Apache License</a></li>
+</ul>
+
+<h3>Resources</h3>
+
+<ul class="this-page-menu">
+    <li><a href="https://issues.apache.org/jira/browse/FLUME">Flume Issue Tracking (Jira)</a></li>
+    <li><a href="http://cwiki.apache.org/confluence/display/FLUME">Flume Wiki</a></li>
+    <li><a href="http://cwiki.apache.org/confluence/display/FLUME/Getting+Started">Getting Started Guide</a></li>
+    <li><a href="https://builds.apache.org/job/flume-trunk/">Jenkins Continuous Integration Server</a></li>
+    <li><a href="https://analysis.apache.org/">Sonar Code Quality Reports</a</li>
+</ul>
+
+<h3>Apache</h3>
+
+<ul class="this-page-menu">
+    <li><a href="http://www.apache.org">Home</a></li>
+    <li><a href="http://www.apache.org/foundation/sponsorship.html">Sponsorship</a></li>
+    <li><a href="http://www.apache.org/licenses">Licenses</a> </li>
+    <li><a href="http://www.apache.org/foundation/thanks.html">Thanks</a></li>
+    <li><a href="http://www.apachecon.com">Conferences</a></li>
+    <li><a href="http://www.apache.org/security/">Security</a></li>
+</ul>
+
+
+        </div>
+      </div>
+      <div class="clearer"></div>
+    </div>
+<div class="footer">
+    &copy; Copyright 2009-2017 The Apache Software Foundation. Apache Flume, Flume, Apache, the Apache feather logo, and the Apache Flume project logo are trademarks of The Apache Software Foundation..
+</div>
+  </body>
+</html>
\ No newline at end of file

Modified: websites/staging/flume/trunk/content/releases/index.html
==============================================================================
--- websites/staging/flume/trunk/content/releases/index.html (original)
+++ websites/staging/flume/trunk/content/releases/index.html Wed Oct  4 16:24:02 2017
@@ -25,8 +25,8 @@
     <script type="text/javascript" src="../_static/underscore.js"></script>
     <script type="text/javascript" src="../_static/doctools.js"></script>
     <link rel="top" title="Apache Flume" href="../index.html" />
-    <link rel="next" title="Version 1.7.0" href="1.7.0.html" />
-    <link rel="prev" title="Flume 1.7.0 Developer Guide" href="../FlumeDeveloperGuide.html" /> 
+    <link rel="next" title="Version 1.8.0" href="1.8.0.html" />
+    <link rel="prev" title="Flume 1.8.0 Developer Guide" href="../FlumeDeveloperGuide.html" /> 
   </head>
   <body>
 <div class="header">
@@ -61,12 +61,13 @@
   <div class="section" id="releases">
 <h1>Releases<a class="headerlink" href="#releases" title="Permalink to this headline">¶</a></h1>
 <p class="rubric">Current Release</p>
-<p>The current stable release is <a class="reference external" href="1.7.0.html">Apache Flume Version 1.7.0</a>.</p>
+<p>The current stable release is <a class="reference external" href="1.8.0.html">Apache Flume Version 1.8.0</a>.</p>
 <div class="toctree-wrapper compound">
 </div>
 <p class="rubric">Previous Releases</p>
 <div class="toctree-wrapper compound">
 <ul>
+<li class="toctree-l1"><a class="reference internal" href="1.7.0.html">Version 1.7.0</a></li>
 <li class="toctree-l1"><a class="reference internal" href="1.6.0.html">Version 1.6.0</a></li>
 <li class="toctree-l1"><a class="reference internal" href="1.5.2.html">Version 1.5.2</a></li>
 <li class="toctree-l1"><a class="reference internal" href="1.5.0.1.html">Version 1.5.0.1</a></li>
@@ -125,7 +126,7 @@
       <div class="clearer"></div>
     </div>
 <div class="footer">
-    &copy; Copyright 2009-2012 The Apache Software Foundation. Apache Flume, Flume, Apache, the Apache feather logo, and the Apache Flume project logo are trademarks of The Apache Software Foundation..
+    &copy; Copyright 2009-2017 The Apache Software Foundation. Apache Flume, Flume, Apache, the Apache feather logo, and the Apache Flume project logo are trademarks of The Apache Software Foundation..
 </div>
   </body>
 </html>
\ No newline at end of file

Modified: websites/staging/flume/trunk/content/search.html
==============================================================================
--- websites/staging/flume/trunk/content/search.html (original)
+++ websites/staging/flume/trunk/content/search.html Wed Oct  4 16:24:02 2017
@@ -129,7 +129,7 @@
       <div class="clearer"></div>
     </div>
 <div class="footer">
-    &copy; Copyright 2009-2012 The Apache Software Foundation. Apache Flume, Flume, Apache, the Apache feather logo, and the Apache Flume project logo are trademarks of The Apache Software Foundation..
+    &copy; Copyright 2009-2017 The Apache Software Foundation. Apache Flume, Flume, Apache, the Apache feather logo, and the Apache Flume project logo are trademarks of The Apache Software Foundation..
 </div>
   </body>
 </html>
\ No newline at end of file