You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flume.apache.org by rg...@apache.org on 2023/03/20 01:50:23 UTC

[flume-kafka] branch main created (now 5fb9539)

This is an automated email from the ASF dual-hosted git repository.

rgoers pushed a change to branch main
in repository https://gitbox.apache.org/repos/asf/flume-kafka.git


      at 5fb9539  FLUME-3457 - Move Kafka components to their own repo

This branch includes the following new commits:

     new 5fb9539  FLUME-3457 - Move Kafka components to their own repo

The 1 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.



[flume-kafka] 01/01: FLUME-3457 - Move Kafka components to their own repo

Posted by rg...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

rgoers pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/flume-kafka.git

commit 5fb9539ad1890aa169614646497ffbca07a432a1
Author: Ralph Goers <rg...@apache.org>
AuthorDate: Sun Mar 19 18:50:14 2023 -0700

    FLUME-3457 - Move Kafka components to their own repo
---
 .asf.yaml                                          |  40 +
 CHANGELOG                                          |   7 +
 LICENSE.txt                                        | 245 ++++++
 NOTICE.txt                                         |   3 +
 README.md                                          |  58 ++
 RELEASE-NOTES.txt                                  |  27 +
 checkstyle-header.txt                              |  16 +
 findbugs-exclude-filter.xml                        |  31 +
 flume-kafka-channel/pom.xml                        |  93 ++
 .../apache/flume/channel/kafka/KafkaChannel.java   | 718 ++++++++++++++++
 .../channel/kafka/KafkaChannelConfiguration.java   |  59 ++
 .../channel/kafka/TestBasicFunctionality.java      | 220 +++++
 .../flume/channel/kafka/TestKafkaChannelBase.java  | 272 ++++++
 .../channel/kafka/TestOffsetsAndMigration.java     |  77 ++
 .../flume/channel/kafka/TestParseAsFlumeEvent.java | 132 +++
 .../apache/flume/channel/kafka/TestPartitions.java | 179 ++++
 .../apache/flume/channel/kafka/TestRollback.java   | 105 +++
 .../src/test/resources/kafka-server.properties     | 133 +++
 .../src/test/resources/keystorefile.jks            | Bin 0 -> 1294 bytes
 flume-kafka-channel/src/test/resources/log4j2.xml  |  72 ++
 .../src/test/resources/truststorefile.jks          | Bin 0 -> 887 bytes
 .../src/test/resources/zookeeper.properties        |  20 +
 flume-kafka-dist/pom.xml                           | 165 ++++
 flume-kafka-dist/src/assembly/bin.xml              |  53 ++
 flume-kafka-dist/src/assembly/src.xml              |  45 +
 flume-kafka-sink/pom.xml                           | 134 +++
 .../org/apache/flume/sink/kafka/KafkaSink.java     | 490 +++++++++++
 .../flume/sink/kafka/KafkaSinkConstants.java       |  62 ++
 .../org/apache/flume/sink/kafka/TestConstants.java |  31 +
 .../org/apache/flume/sink/kafka/TestKafkaSink.java | 839 +++++++++++++++++++
 .../apache/flume/sink/kafka/util/KafkaLocal.java   |  53 ++
 .../org/apache/flume/sink/kafka/util/TestUtil.java | 294 +++++++
 .../flume/sink/kafka/util/ZooKeeperLocal.java      |  62 ++
 .../src/test/resources/kafka-server.properties     | 131 +++
 .../src/test/resources/keystorefile.jks            | Bin 0 -> 1294 bytes
 flume-kafka-sink/src/test/resources/log4j2.xml     |  72 ++
 .../src/test/resources/testutil.properties         |   3 +
 .../src/test/resources/truststorefile.jks          | Bin 0 -> 887 bytes
 .../src/test/resources/zookeeper.properties        |  20 +
 flume-kafka-source/pom.xml                         | 102 +++
 .../org/apache/flume/source/kafka/KafkaSource.java | 539 ++++++++++++
 .../flume/source/kafka/KafkaSourceConstants.java   |  57 ++
 .../source/kafka/KafkaSourceEmbeddedKafka.java     | 209 +++++
 .../source/kafka/KafkaSourceEmbeddedZookeeper.java |  70 ++
 .../apache/flume/source/kafka/TestKafkaSource.java | 932 +++++++++++++++++++++
 .../src/test/resources/keystorefile.jks            | Bin 0 -> 1294 bytes
 flume-kafka-source/src/test/resources/log4j2.xml   |  31 +
 .../src/test/resources/truststorefile.jks          | Bin 0 -> 887 bytes
 flume-shared/flume-shared-kafka-test/pom.xml       |  86 ++
 .../shared/kafka/test/KafkaPartitionTestUtil.java  | 220 +++++
 .../flume/shared/kafka/test/PartitionOption.java   |  25 +
 .../shared/kafka/test/PartitionTestScenario.java   |  26 +
 flume-shared/flume-shared-kafka/pom.xml            |  54 ++
 .../apache/flume/shared/kafka/KafkaSSLUtil.java    |  75 ++
 .../flume/shared/kafka/KafkaSSLUtilTest.java       | 174 ++++
 flume-shared/pom.xml                               |  40 +
 pom.xml                                            | 384 +++++++++
 57 files changed, 7985 insertions(+)

diff --git a/.asf.yaml b/.asf.yaml
new file mode 100644
index 0000000..4a6ee27
--- /dev/null
+++ b/.asf.yaml
@@ -0,0 +1,40 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#      http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+# `.asf.yaml` is a branch-specific YAML configuration file for Git repositories to control features such as notifications, GitHub settings, etc.
+# See its documentation for details: https://cwiki.apache.org/confluence/display/INFRA/Git+-+.asf.yaml+features
+
+notifications:
+  # GitHub already provides notifications for PRs and issues.
+  # Please don't duplicate that noise here!
+  commits: commits@flume.apache.org
+  jira_options: link label
+github:
+  description: "Apache Flume Kafka contains the Flume components that interact with Kafka."
+  homepage: https://logging.apache.org/flume/
+  features:
+    issues: true
+  del_branch_on_merge: true
+  autolink_jira:
+    - FLUME
+  labels:
+    - apache
+    - api
+    - java
+    - jvm
+    - library
+    - flume
+  protected_branches:
+    main: {}
diff --git a/CHANGELOG b/CHANGELOG
new file mode 100644
index 0000000..da27772
--- /dev/null
+++ b/CHANGELOG
@@ -0,0 +1,7 @@
+Release Notes - Flume Kafka - Version 2.0.0
+
+** Bug
+    *
+
+
+
diff --git a/LICENSE.txt b/LICENSE.txt
new file mode 100644
index 0000000..9fa7156
--- /dev/null
+++ b/LICENSE.txt
@@ -0,0 +1,245 @@
+
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   Licensed under the Apache License, Version 2.0 (the "License");
+   you may not use this file except in compliance with the License.
+   You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+
+====
+
+The following files are included under the 2-Clause BSD License
+
+flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/resources/solr/collection1/conf/lang/stopwords_ar.txt
+flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/resources/solr/collection1/conf/lang/stopwords_bg.txt
+flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/resources/solr/collection1/conf/lang/stopwords_da.txt
+flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/resources/solr/collection1/conf/lang/stopwords_de.txt
+flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/resources/solr/collection1/conf/lang/stopwords_es.txt
+flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/resources/solr/collection1/conf/lang/stopwords_fa.txt
+flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/resources/solr/collection1/conf/lang/stopwords_fi.txt
+flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/resources/solr/collection1/conf/lang/stopwords_fr.txt
+flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/resources/solr/collection1/conf/lang/stopwords_hi.txt
+flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/resources/solr/collection1/conf/lang/stopwords_hu.txt
+flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/resources/solr/collection1/conf/lang/stopwords_it.txt
+flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/resources/solr/collection1/conf/lang/stopwords_nl.txt
+flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/resources/solr/collection1/conf/lang/stopwords_no.txt
+flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/resources/solr/collection1/conf/lang/stopwords_pt.txt
+flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/resources/solr/collection1/conf/lang/stopwords_ro.txt
+flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/resources/solr/collection1/conf/lang/stopwords_ru.txt
+flume-ng-sinks/flume-ng-morphline-solr-sink/src/test/resources/solr/collection1/conf/lang/stopwords_sv.txt
+
+Redistribution and use in source and binary forms, with or without modification,
+are permitted provided that the following conditions are met:
+
+1. Redistributions of source code must retain the above copyright notice, this
+list of conditions and the following disclaimer.
+
+2. Redistributions in binary form must reproduce the above copyright notice,
+this list of conditions and the following disclaimer in the documentation and/or
+other materials provided with the distribution.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR
+ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON
+ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
diff --git a/NOTICE.txt b/NOTICE.txt
new file mode 100644
index 0000000..33c6200
--- /dev/null
+++ b/NOTICE.txt
@@ -0,0 +1,3 @@
+Apache Flume Spring Boot
+Copyright 2022-2023 The Apache Software Foundation
+
diff --git a/README.md b/README.md
new file mode 100644
index 0000000..071f426
--- /dev/null
+++ b/README.md
@@ -0,0 +1,58 @@
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+
+# Welcome to Apache Flume Kafka!
+
+Apache Flume is a distributed, reliable, and available service for efficiently
+collecting, aggregating, and moving large amounts of event data. It has a simple
+and flexible architecture based on streaming data flows. It is robust and fault
+tolerant with tunable reliability mechanisms and many failover and recovery
+mechanisms. The system is centrally managed and allows for intelligent dynamic
+management. It uses a simple extensible data model that allows for online
+analytic application.
+
+The Apache Flume Kafka provides a Kafka Channel, Kafka Sink, and a Kafka Source.
+
+Apache Flume Kafka is open-sourced under the Apache Software Foundation License v2.0.
+
+## Documentation
+
+Documentation is included in the binary distribution under the docs directory.
+In source form, it can be found in the flume-ng-doc directory.
+
+The Flume 1.x guide and FAQ are available here:
+
+* https://cwiki.apache.org/FLUME
+* https://cwiki.apache.org/confluence/display/FLUME/Getting+Started
+
+## Contact us!
+
+* Mailing lists: https://cwiki.apache.org/confluence/display/FLUME/Mailing+Lists
+* Slack channel #flume on https://the-asf.slack.com/
+
+Bug and Issue tracker.
+
+* https://github.com/apache/flume-kafka/issues
+
+## Compiling Flume Kafka
+
+Compiling Flume Spring Boot requires the following tools:
+
+* Oracle Java JDK 8
+* Apache Maven 3.x
diff --git a/RELEASE-NOTES.txt b/RELEASE-NOTES.txt
new file mode 100644
index 0000000..500c878
--- /dev/null
+++ b/RELEASE-NOTES.txt
@@ -0,0 +1,27 @@
+Apache Flume Kafka 2.0.0
+
+CONTENTS
+1. What is Apache Flume Kafka
+2. Major changes in this Release
+3. How to Get Involved
+4. How to Report Issues
+
+1. What is Apache Flume Kafka
+Flume is a distributed, reliable, and available service for
+efficiently collecting, aggregating, and moving large amounts of event
+data. Flume Kafka provides a Channel, Source, and Sink for publishing
+and consuming events to and from Kafka..
+
+2. Major changes in this Release
+For a detailed list of changes, please see the CHANGELOG file included
+in this distribution.
+
+3. How to Get Involved
+The Apache Flume project really needs and appreciates any contributions,
+including documentation help, source code and feedback. If you are interested
+in contributing, please visit:
+https://cwiki.apache.org/confluence/display/FLUME/How+to+Contribute
+
+4. How to Report Issues
+The Apache Flume Spring Boot project uses GitHub issues for issue tracking. Please see
+https://github.com/apache/flume-spring-boot/issues
diff --git a/checkstyle-header.txt b/checkstyle-header.txt
new file mode 100644
index 0000000..4f33236
--- /dev/null
+++ b/checkstyle-header.txt
@@ -0,0 +1,16 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache license, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the license for the specific language governing permissions and
+ * limitations under the license.
+ */
diff --git a/findbugs-exclude-filter.xml b/findbugs-exclude-filter.xml
new file mode 100644
index 0000000..327be31
--- /dev/null
+++ b/findbugs-exclude-filter.xml
@@ -0,0 +1,31 @@
+<?xml version="1.0" encoding="iso-8859-1"?>
+<!--
+   Licensed to the Apache Software Foundation (ASF) under one or more
+   contributor license agreements.  See the NOTICE file distributed with
+   this work for additional information regarding copyright ownership.
+   The ASF licenses this file to You under the Apache License, Version 2.0
+   (the "License"); you may not use this file except in compliance with
+   the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+-->
+
+<!-- ===================================================================== -->
+<!-- $Id: findbugs-exclude-filter.xml 773234 2009-05-09 15:27:59Z rgoers $ -->
+<!-- ===================================================================== -->
+<FindBugsFilter>
+  <!-- Enable only high priority warnings -->
+  <Match>
+    <Priority value="2"/>
+  </Match>
+
+  <Match>
+    <Priority value="3"/>
+  </Match>
+</FindBugsFilter>
diff --git a/flume-kafka-channel/pom.xml b/flume-kafka-channel/pom.xml
new file mode 100644
index 0000000..1f93b5b
--- /dev/null
+++ b/flume-kafka-channel/pom.xml
@@ -0,0 +1,93 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+   http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <parent>
+    <artifactId>flume-kafka-parent</artifactId>
+    <groupId>org.apache.flume</groupId>
+    <version>2.0.0-SNAPSHOT</version>
+  </parent>
+  <modelVersion>4.0.0</modelVersion>
+
+  <groupId>org.apache.flume</groupId>
+  <artifactId>flume-kafka-channel</artifactId>
+
+  <properties>
+    <!-- TODO fix spotbugs/pmd violations -->
+    <spotbugs.maxAllowedViolations>13</spotbugs.maxAllowedViolations>
+    <pmd.maxAllowedViolations>13</pmd.maxAllowedViolations>
+    <module.name>org.apache.flume.channel.kafka</module.name>
+  </properties>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.flume</groupId>
+      <artifactId>flume-ng-core</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.flume</groupId>
+      <artifactId>flume-ng-sdk</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.flume</groupId>
+      <artifactId>flume-shared-kafka</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.flume</groupId>
+      <artifactId>flume-shared-kafka-test</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.kafka</groupId>
+      <artifactId>kafka_${scala.version}</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.kafka</groupId>
+      <artifactId>kafka_${scala.version}</artifactId>
+      <classifier>test</classifier>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.kafka</groupId>
+      <artifactId>kafka-clients</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.kafka</groupId>
+      <artifactId>kafka-clients</artifactId>
+      <version>${kafka.version}</version>
+      <classifier>test</classifier>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.flume</groupId>
+      <artifactId>flume-kafka-sink</artifactId>
+      <classifier>tests</classifier>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-all</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.logging.log4j</groupId>
+      <artifactId>log4j-slf4j-impl</artifactId>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+
+</project>
diff --git a/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannel.java b/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannel.java
new file mode 100644
index 0000000..acfb3b1
--- /dev/null
+++ b/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannel.java
@@ -0,0 +1,718 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.flume.channel.kafka;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Optional;
+import org.apache.avro.io.BinaryDecoder;
+import org.apache.avro.io.BinaryEncoder;
+import org.apache.avro.io.DecoderFactory;
+import org.apache.avro.io.EncoderFactory;
+import org.apache.avro.specific.SpecificDatumReader;
+import org.apache.avro.specific.SpecificDatumWriter;
+import org.apache.flume.ChannelException;
+import org.apache.flume.Context;
+import org.apache.flume.Event;
+import org.apache.flume.FlumeException;
+import org.apache.flume.channel.BasicChannelSemantics;
+import org.apache.flume.channel.BasicTransactionSemantics;
+import org.apache.flume.conf.ConfigurationException;
+import org.apache.flume.conf.LogPrivacyUtil;
+import org.apache.flume.event.EventBuilder;
+import org.apache.flume.instrumentation.kafka.KafkaChannelCounter;
+import org.apache.flume.shared.kafka.KafkaSSLUtil;
+import org.apache.flume.source.avro.AvroFlumeEvent;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.clients.producer.Callback;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.clients.producer.RecordMetadata;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.config.SslConfigs;
+import org.apache.kafka.common.errors.WakeupException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.nio.ByteBuffer;
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.UUID;
+import java.util.concurrent.Future;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+import static org.apache.flume.channel.kafka.KafkaChannelConfiguration.*;
+import static org.apache.flume.shared.kafka.KafkaSSLUtil.SSL_DISABLE_FQDN_CHECK;
+import static org.apache.flume.shared.kafka.KafkaSSLUtil.isSSLEnabled;
+
+public class KafkaChannel extends BasicChannelSemantics {
+
+  private static final Logger logger =
+          LoggerFactory.getLogger(KafkaChannel.class);
+
+  private final Properties consumerProps = new Properties();
+  private final Properties producerProps = new Properties();
+
+  private KafkaProducer<String, byte[]> producer;
+
+  // Used to ensure that we don't have multiple threads attempting to use the KafkaProducer (above) for transactions
+  // at the same time as whilst the KafkaProducer is thread-safe you cannot have more than one in-flight transaction.
+  private Lock kafkaTxLock = new ReentrantLock();
+
+  private final String channelUUID = UUID.randomUUID().toString();
+
+  private AtomicReference<String> topic = new AtomicReference<String>();
+  private boolean parseAsFlumeEvent = DEFAULT_PARSE_AS_FLUME_EVENT;
+  private String topicStr = DEFAULT_TOPIC;
+  private String groupId = DEFAULT_GROUP_ID;
+  private String partitionHeader = null;
+  private Integer staticPartitionId;
+  private boolean useKafkaTransactions;
+
+  // used to indicate if a rebalance has occurred during the current transaction
+  AtomicBoolean rebalanceFlag = new AtomicBoolean();
+  // This isn't a Kafka property per se, but we allow it to be configurable
+  private long pollTimeout = DEFAULT_POLL_TIMEOUT;
+
+  // Track all consumers to close them eventually.
+  private final List<ConsumerAndRecords> consumers =
+          Collections.synchronizedList(new LinkedList<ConsumerAndRecords>());
+
+  private KafkaChannelCounter counter;
+
+  /* Each Consumer commit will commit all partitions owned by it. To
+  * ensure that each partition is only committed when all events are
+  * actually done, we will need to keep a Consumer per thread.
+  */
+
+  private final ThreadLocal<ConsumerAndRecords> consumerAndRecords =
+      new ThreadLocal<ConsumerAndRecords>() {
+        @Override
+        public ConsumerAndRecords initialValue() {
+          return createConsumerAndRecords();
+        }
+      };
+
+  @Override
+  public void start() {
+    logger.info("Starting Kafka Channel: {}", getName());
+    producer = new KafkaProducer<String, byte[]>(producerProps);
+    if (useKafkaTransactions) {
+      logger.debug("Initializing Kafka Transaction");
+      producer.initTransactions();
+    }
+    // We always have just one topic being read by one thread
+    logger.info("Topic = {}", topic.get());
+    counter.start();
+    super.start();
+  }
+
+  @Override
+  public void stop() {
+    for (ConsumerAndRecords c : consumers) {
+      try {
+        decommissionConsumerAndRecords(c);
+      } catch (Exception ex) {
+        logger.warn("Error while shutting down consumer.", ex);
+      }
+    }
+    producer.close();
+    counter.stop();
+    super.stop();
+    logger.info("Kafka channel {} stopped.", getName());
+  }
+
+  @Override
+  protected BasicTransactionSemantics createTransaction() {
+    return new KafkaTransaction();
+  }
+
+  @Override
+  public void configure(Context ctx) {
+    topicStr = ctx.getString(TOPIC_CONFIG);
+    if (topicStr == null || topicStr.isEmpty()) {
+      topicStr = DEFAULT_TOPIC;
+      logger.info("Topic was not specified. Using {} as the topic.", topicStr);
+    }
+    topic.set(topicStr);
+
+    groupId = ctx.getString(KAFKA_CONSUMER_PREFIX + ConsumerConfig.GROUP_ID_CONFIG);
+    if (groupId == null || groupId.isEmpty()) {
+      groupId = DEFAULT_GROUP_ID;
+      logger.info("Group ID was not specified. Using {} as the group id.", groupId);
+    }
+
+    String bootStrapServers = ctx.getString(BOOTSTRAP_SERVERS_CONFIG);
+    if (bootStrapServers == null || bootStrapServers.isEmpty()) {
+      throw new ConfigurationException("Bootstrap Servers must be specified");
+    }
+
+    String transactionalID = ctx.getString(TRANSACTIONAL_ID);
+    if (transactionalID != null) {
+      try {
+        ctx.put(TRANSACTIONAL_ID, InetAddress.getLocalHost().getCanonicalHostName() +
+                Thread.currentThread().getName() + transactionalID);
+        useKafkaTransactions = true;
+      } catch (UnknownHostException e) {
+        throw new ConfigurationException("Unable to configure transactional id, as cannot work out hostname", e);
+      }
+    }
+
+    setProducerProps(ctx, bootStrapServers);
+    setConsumerProps(ctx, bootStrapServers);
+
+    parseAsFlumeEvent = ctx.getBoolean(PARSE_AS_FLUME_EVENT, DEFAULT_PARSE_AS_FLUME_EVENT);
+    pollTimeout = ctx.getLong(POLL_TIMEOUT, DEFAULT_POLL_TIMEOUT);
+
+    staticPartitionId = ctx.getInteger(STATIC_PARTITION_CONF);
+    partitionHeader = ctx.getString(PARTITION_HEADER_NAME);
+
+    if (logger.isDebugEnabled() && LogPrivacyUtil.allowLogPrintConfig()) {
+      logger.debug("Kafka properties: {}", ctx);
+    }
+
+    if (counter == null) {
+      counter = new KafkaChannelCounter(getName());
+    }
+  }
+
+  private void setProducerProps(Context ctx, String bootStrapServers) {
+    producerProps.clear();
+    producerProps.put(ProducerConfig.ACKS_CONFIG, DEFAULT_ACKS);
+    producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, DEFAULT_KEY_SERIALIZER);
+    producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, DEFAULT_VALUE_SERIAIZER);
+    // Defaults overridden based on config
+    producerProps.putAll(ctx.getSubProperties(KAFKA_PRODUCER_PREFIX));
+    producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, bootStrapServers);
+    //  The default value of `ssl.endpoint.identification.algorithm`
+    //  is changed to `https`, since kafka client 2.0+
+    //  And because flume does not accept an empty string as property value,
+    //  so we need to use an alternative custom property
+    //  `ssl.disableTLSHostnameVerification` to check if enable fqdn check.
+    if (isSSLEnabled(producerProps) && "true".equalsIgnoreCase(producerProps.getProperty(SSL_DISABLE_FQDN_CHECK))) {
+      producerProps.put(SslConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG, "");
+    }
+    KafkaSSLUtil.addGlobalSSLParameters(producerProps);
+  }
+
+  protected Properties getProducerProps() {
+    return producerProps;
+  }
+
+  private void setConsumerProps(Context ctx, String bootStrapServers) {
+    consumerProps.clear();
+    consumerProps.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, DEFAULT_KEY_DESERIALIZER);
+    consumerProps.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, DEFAULT_VALUE_DESERIAIZER);
+    consumerProps.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, DEFAULT_AUTO_OFFSET_RESET);
+    // Defaults overridden based on config
+    consumerProps.putAll(ctx.getSubProperties(KAFKA_CONSUMER_PREFIX));
+    // These always take precedence over config
+    consumerProps.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, bootStrapServers);
+    consumerProps.put(ConsumerConfig.GROUP_ID_CONFIG, groupId);
+    consumerProps.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, false);
+    //  The default value of `ssl.endpoint.identification.algorithm`
+    //  is changed to `https`, since kafka client 2.0+
+    //  And because flume does not accept an empty string as property value,
+    //  so we need to use an alternative custom property
+    //  `ssl.disableTLSHostnameVerification` to check if enable fqdn check.
+    if (isSSLEnabled(consumerProps) && "true".equalsIgnoreCase(consumerProps.getProperty(SSL_DISABLE_FQDN_CHECK))) {
+      consumerProps.put(SslConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG, "");
+    }
+    KafkaSSLUtil.addGlobalSSLParameters(consumerProps);
+  }
+
+  protected Properties getConsumerProps() {
+    return consumerProps;
+  }
+
+  private synchronized ConsumerAndRecords createConsumerAndRecords() {
+    try {
+      KafkaConsumer<String, byte[]> consumer = new KafkaConsumer<String, byte[]>(consumerProps);
+      ConsumerAndRecords car = new ConsumerAndRecords(consumer, channelUUID);
+      logger.info("Created new consumer to connect to Kafka");
+      car.consumer.subscribe(Arrays.asList(topic.get()),
+                             new ChannelRebalanceListener(rebalanceFlag));
+      car.offsets = new HashMap<TopicPartition, OffsetAndMetadata>();
+      consumers.add(car);
+      return car;
+    } catch (Exception e) {
+      throw new FlumeException("Unable to connect to Kafka", e);
+    }
+  }
+
+  private void decommissionConsumerAndRecords(ConsumerAndRecords c) {
+    c.consumer.wakeup();
+    c.consumer.close();
+  }
+
+  @VisibleForTesting
+  void registerThread() {
+    try {
+      consumerAndRecords.get();
+    } catch (Exception e) {
+      logger.error(e.getMessage());
+      e.printStackTrace();
+    }
+  }
+
+  private enum TransactionType {
+    PUT,
+    TAKE,
+    NONE
+  }
+
+  private class KafkaTransaction extends BasicTransactionSemantics {
+
+    private TransactionType type = TransactionType.NONE;
+    private Optional<ByteArrayOutputStream> tempOutStream = Optional
+            .absent();
+    // For put transactions, serialize the events and hold them until the commit goes is requested.
+    private Optional<LinkedList<ProducerRecord<String, byte[]>>> producerRecords =
+        Optional.absent();
+    // For take transactions, deserialize and hold them till commit goes through
+    private Optional<LinkedList<Event>> events = Optional.absent();
+    private Optional<SpecificDatumWriter<AvroFlumeEvent>> writer =
+            Optional.absent();
+    private Optional<SpecificDatumReader<AvroFlumeEvent>> reader =
+            Optional.absent();
+    private Optional<LinkedList<Future<RecordMetadata>>> kafkaFutures =
+            Optional.absent();
+    private final String batchUUID = UUID.randomUUID().toString();
+
+    // Fine to use null for initial value, Avro will create new ones if this
+    // is null
+    private BinaryEncoder encoder = null;
+    private BinaryDecoder decoder = null;
+    private boolean eventTaken = false;
+
+    @Override
+    protected void doBegin() throws InterruptedException {
+      rebalanceFlag.set(false);
+    }
+
+    @Override
+    protected void doPut(Event event) throws InterruptedException {
+      type = TransactionType.PUT;
+      if (!producerRecords.isPresent()) {
+        producerRecords = Optional.of(new LinkedList<ProducerRecord<String, byte[]>>());
+      }
+      String key = event.getHeaders().get(KEY_HEADER);
+
+      Integer partitionId = null;
+      try {
+        if (staticPartitionId != null) {
+          partitionId = staticPartitionId;
+        }
+        // Allow a specified header to override a static ID
+        if (partitionHeader != null) {
+          String headerVal = event.getHeaders().get(partitionHeader);
+          if (headerVal != null) {
+            partitionId = Integer.parseInt(headerVal);
+          }
+        }
+        if (partitionId != null) {
+          producerRecords.get().add(
+              new ProducerRecord<String, byte[]>(topic.get(), partitionId, key,
+                                                 serializeValue(event, parseAsFlumeEvent)));
+        } else {
+          producerRecords.get().add(
+              new ProducerRecord<String, byte[]>(topic.get(), key,
+                                                 serializeValue(event, parseAsFlumeEvent)));
+        }
+        counter.incrementEventPutAttemptCount();
+      } catch (NumberFormatException e) {
+        throw new ChannelException("Non integer partition id specified", e);
+      } catch (Exception e) {
+        throw new ChannelException("Error while serializing event", e);
+      }
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    protected Event doTake() throws InterruptedException {
+      logger.trace("Starting event take");
+      type = TransactionType.TAKE;
+      try {
+        if (!(consumerAndRecords.get().uuid.equals(channelUUID))) {
+          logger.info("UUID mismatch, creating new consumer");
+          decommissionConsumerAndRecords(consumerAndRecords.get());
+          consumerAndRecords.remove();
+        }
+      } catch (Exception ex) {
+        logger.warn("Error while shutting down consumer", ex);
+      }
+      if (!events.isPresent()) {
+        events = Optional.of(new LinkedList<Event>());
+      }
+      Event e;
+      // Give the channel a chance to commit if there has been a rebalance
+      if (rebalanceFlag.get()) {
+        logger.debug("Returning null event after Consumer rebalance.");
+        return null;
+      }
+      if (!consumerAndRecords.get().failedEvents.isEmpty()) {
+        e = consumerAndRecords.get().failedEvents.removeFirst();
+      } else {
+        if ( logger.isTraceEnabled() ) {
+          logger.trace("Assignment during take: {}",
+              consumerAndRecords.get().consumer.assignment().toString());
+        }
+        try {
+          long startTime = System.nanoTime();
+          if (!consumerAndRecords.get().recordIterator.hasNext()) {
+            consumerAndRecords.get().poll();
+          }
+          if (consumerAndRecords.get().recordIterator.hasNext()) {
+            ConsumerRecord<String, byte[]> record = consumerAndRecords.get().recordIterator.next();
+            e = deserializeValue(record.value(), parseAsFlumeEvent);
+            TopicPartition tp = new TopicPartition(record.topic(), record.partition());
+            OffsetAndMetadata oam = new OffsetAndMetadata(record.offset() + 1, batchUUID);
+            consumerAndRecords.get().saveOffsets(tp,oam);
+
+            //Add the key to the header
+            if (record.key() != null) {
+              e.getHeaders().put(KEY_HEADER, record.key());
+            }
+
+            long endTime = System.nanoTime();
+            counter.addToKafkaEventGetTimer((endTime - startTime) / (1000 * 1000));
+
+            if (logger.isDebugEnabled()) {
+              logger.debug("{} processed output from partition {} offset {}",
+                  new Object[] {getName(), record.partition(), record.offset()});
+            }
+          } else {
+            return null;
+          }
+          counter.incrementEventTakeAttemptCount();
+        } catch (Exception ex) {
+          logger.warn("Error while getting events from Kafka. This is usually caused by " +
+                      "trying to read a non-flume event. Ensure the setting for " +
+                      "parseAsFlumeEvent is correct", ex);
+          throw new ChannelException("Error while getting events from Kafka", ex);
+        }
+      }
+      eventTaken = true;
+      events.get().add(e);
+      return e;
+    }
+
+    @Override
+    protected void doCommit() throws InterruptedException {
+      logger.trace("Starting commit");
+      if (type.equals(TransactionType.NONE)) {
+        return;
+      }
+      if (type.equals(TransactionType.PUT)) {
+        if (!kafkaFutures.isPresent()) {
+          kafkaFutures = Optional.of(new LinkedList<Future<RecordMetadata>>());
+        }
+        try {
+          if (useKafkaTransactions) {
+            kafkaTxLock.lock();
+            logger.debug("Beginning Kafka Transaction");
+            producer.beginTransaction();
+          }
+          long batchSize = producerRecords.get().size();
+          long startTime = System.nanoTime();
+          int index = 0;
+          for (ProducerRecord<String, byte[]> record : producerRecords.get()) {
+            index++;
+            kafkaFutures.get().add(producer.send(record, new ChannelCallback(index, startTime)));
+          }
+
+          if (useKafkaTransactions) {
+            logger.debug("Committing Kafka Transaction");
+            producer.commitTransaction();
+            kafkaTxLock.unlock();
+          } else {
+            // Ensure that the records are actually flushed by the producer, regardless of linger.ms.
+            // Per the Kafka docs we do not need to linger or wait for the callback if we're using transactions
+            producer.flush();
+
+            for (Future<RecordMetadata> future : kafkaFutures.get()) {
+              future.get();
+            }
+          }
+          long endTime = System.nanoTime();
+          counter.addToKafkaEventSendTimer((endTime - startTime) / (1000 * 1000));
+          counter.addToEventPutSuccessCount(batchSize);
+          producerRecords.get().clear();
+          kafkaFutures.get().clear();
+        } catch (Exception ex) {
+          if (useKafkaTransactions) {
+            logger.debug("Aborting transaction");
+            try {
+              producer.abortTransaction();
+            } finally {
+              kafkaTxLock.unlock();
+            }
+          }
+          logger.warn("Sending events to Kafka failed", ex);
+          throw new ChannelException("Commit failed as send to Kafka failed",
+                  ex);
+        }
+      } else {
+        // event taken ensures that we have collected events in this transaction
+        // before committing
+        if (consumerAndRecords.get().failedEvents.isEmpty() && eventTaken) {
+          logger.trace("About to commit batch");
+          long startTime = System.nanoTime();
+          consumerAndRecords.get().commitOffsets();
+          long endTime = System.nanoTime();
+          counter.addToKafkaCommitTimer((endTime - startTime) / (1000 * 1000));
+          if (logger.isDebugEnabled()) {
+            logger.debug(consumerAndRecords.get().getCommittedOffsetsString());
+          }
+        }
+
+        int takes = events.get().size();
+        if (takes > 0) {
+          counter.addToEventTakeSuccessCount(takes);
+          events.get().clear();
+        }
+      }
+    }
+
+    @Override
+    protected void doRollback() throws InterruptedException {
+      if (type.equals(TransactionType.NONE)) {
+        return;
+      }
+      if (type.equals(TransactionType.PUT)) {
+        producerRecords.get().clear();
+        kafkaFutures.get().clear();
+      } else {
+        counter.addToRollbackCounter(events.get().size());
+        consumerAndRecords.get().failedEvents.addAll(events.get());
+        events.get().clear();
+      }
+    }
+
+    private byte[] serializeValue(Event event, boolean parseAsFlumeEvent) throws IOException {
+      byte[] bytes;
+      if (parseAsFlumeEvent) {
+        if (!tempOutStream.isPresent()) {
+          tempOutStream = Optional.of(new ByteArrayOutputStream());
+        }
+        if (!writer.isPresent()) {
+          writer = Optional.of(new
+                  SpecificDatumWriter<AvroFlumeEvent>(AvroFlumeEvent.class));
+        }
+        tempOutStream.get().reset();
+        AvroFlumeEvent e = new AvroFlumeEvent(
+                toCharSeqMap(event.getHeaders()),
+                ByteBuffer.wrap(event.getBody()));
+        encoder = EncoderFactory.get()
+                .directBinaryEncoder(tempOutStream.get(), encoder);
+        writer.get().write(e, encoder);
+        encoder.flush();
+        bytes = tempOutStream.get().toByteArray();
+      } else {
+        bytes = event.getBody();
+      }
+      return bytes;
+    }
+
+    private Event deserializeValue(byte[] value, boolean parseAsFlumeEvent) throws IOException {
+      Event e;
+      if (parseAsFlumeEvent) {
+        ByteArrayInputStream in =
+                new ByteArrayInputStream(value);
+        decoder = DecoderFactory.get().directBinaryDecoder(in, decoder);
+        if (!reader.isPresent()) {
+          reader = Optional.of(
+                  new SpecificDatumReader<AvroFlumeEvent>(AvroFlumeEvent.class));
+        }
+        AvroFlumeEvent event = reader.get().read(null, decoder);
+        e = EventBuilder.withBody(event.getBody().array(),
+                toStringMap(event.getHeaders()));
+      } else {
+        e = EventBuilder.withBody(value, Collections.EMPTY_MAP);
+      }
+      return e;
+    }
+  }
+
+  /**
+   * Helper function to convert a map of String to a map of CharSequence.
+   */
+  private static Map<CharSequence, CharSequence> toCharSeqMap(Map<String, String> stringMap) {
+    Map<CharSequence, CharSequence> charSeqMap =
+            new HashMap<CharSequence, CharSequence>();
+    for (Map.Entry<String, String> entry : stringMap.entrySet()) {
+      charSeqMap.put(entry.getKey(), entry.getValue());
+    }
+    return charSeqMap;
+  }
+
+  /**
+   * Helper function to convert a map of CharSequence to a map of String.
+   */
+  private static Map<String, String> toStringMap(Map<CharSequence, CharSequence> charSeqMap) {
+    Map<String, String> stringMap = new HashMap<String, String>();
+    for (Map.Entry<CharSequence, CharSequence> entry : charSeqMap.entrySet()) {
+      stringMap.put(entry.getKey().toString(), entry.getValue().toString());
+    }
+    return stringMap;
+  }
+
+  /* Object to store our consumer */
+  private class ConsumerAndRecords {
+    final KafkaConsumer<String, byte[]> consumer;
+    final String uuid;
+    final LinkedList<Event> failedEvents = new LinkedList<Event>();
+
+    ConsumerRecords<String, byte[]> records;
+    Iterator<ConsumerRecord<String, byte[]>> recordIterator;
+    Map<TopicPartition, OffsetAndMetadata> offsets;
+
+    ConsumerAndRecords(KafkaConsumer<String, byte[]> consumer, String uuid) {
+      this.consumer = consumer;
+      this.uuid = uuid;
+      this.records = ConsumerRecords.empty();
+      this.recordIterator = records.iterator();
+    }
+
+    private void poll() {
+      logger.trace("Polling with timeout: {}ms channel-{}", pollTimeout, getName());
+      try {
+        records = consumer.poll(Duration.ofMillis(pollTimeout));
+        recordIterator = records.iterator();
+        logger.debug("{} returned {} records from last poll", getName(), records.count());
+      } catch (WakeupException e) {
+        logger.trace("Consumer woken up for channel {}.", getName());
+      }
+    }
+
+    private void commitOffsets() {
+      try {
+        consumer.commitSync(offsets);
+      } catch (Exception e) {
+        logger.info("Error committing offsets.", e);
+      } finally {
+        logger.trace("About to clear offsets map.");
+        offsets.clear();
+      }
+    }
+
+    private String getOffsetMapString() {
+      StringBuilder sb = new StringBuilder();
+      sb.append(getName()).append(" current offsets map: ");
+      for (Map.Entry<TopicPartition, OffsetAndMetadata> entry : offsets.entrySet()) {
+        sb.append("p").append(entry.getKey().partition()).append('-')
+            .append(entry.getValue().offset()).append(' ');
+      }
+      return sb.toString();
+    }
+
+    // This prints the current committed offsets when debug is enabled
+    private String getCommittedOffsetsString() {
+      StringBuilder sb = new StringBuilder();
+      sb.append(getName()).append(" committed: ");
+      for (TopicPartition tp : consumer.assignment()) {
+        try {
+          sb.append("[").append(tp).append(",")
+              .append(consumer.committed(tp).offset())
+              .append("] ");
+        } catch (NullPointerException npe) {
+          logger.debug("Committed {}", tp);
+        }
+      }
+      return sb.toString();
+    }
+
+    private void saveOffsets(TopicPartition tp, OffsetAndMetadata oam) {
+      offsets.put(tp,oam);
+      if (logger.isTraceEnabled()) {
+        logger.trace(getOffsetMapString());
+      }
+    }
+  }
+}
+
+// Throw exception if there is an error
+class ChannelCallback implements Callback {
+  private static final Logger log = LoggerFactory.getLogger(ChannelCallback.class);
+  private int index;
+  private long startTime;
+
+  public ChannelCallback(int index, long startTime) {
+    this.index = index;
+    this.startTime = startTime;
+  }
+
+  public void onCompletion(RecordMetadata metadata, Exception exception) {
+    if (exception != null) {
+      log.trace("Error sending message to Kafka due to " + exception.getMessage());
+    }
+    if (log.isDebugEnabled()) {
+      long batchElapsedTime = System.currentTimeMillis() - startTime;
+      if (metadata != null) {
+        log.debug("Acked message_no " + index + ": " + metadata.topic() + "-" +
+                metadata.partition() + "-" + metadata.offset() + "-" + batchElapsedTime);
+      }
+    }
+  }
+}
+
+class ChannelRebalanceListener implements ConsumerRebalanceListener {
+  private static final Logger log = LoggerFactory.getLogger(ChannelRebalanceListener.class);
+  private AtomicBoolean rebalanceFlag;
+
+  public ChannelRebalanceListener(AtomicBoolean rebalanceFlag) {
+    this.rebalanceFlag = rebalanceFlag;
+  }
+
+  // Set a flag that a rebalance has occurred. Then we can commit the currently written transactions
+  // on the next doTake() pass.
+  public void onPartitionsRevoked(Collection<TopicPartition> partitions) {
+    for (TopicPartition partition : partitions) {
+      log.info("topic {} - partition {} revoked.", partition.topic(), partition.partition());
+      rebalanceFlag.set(true);
+    }
+  }
+
+  public void onPartitionsAssigned(Collection<TopicPartition> partitions) {
+    for (TopicPartition partition : partitions) {
+      log.info("topic {} - partition {} assigned.", partition.topic(), partition.partition());
+    }
+  }
+}
diff --git a/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannelConfiguration.java b/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannelConfiguration.java
new file mode 100644
index 0000000..cc23d1a
--- /dev/null
+++ b/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannelConfiguration.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.flume.channel.kafka;
+
+import org.apache.kafka.clients.CommonClientConfigs;
+import org.apache.kafka.clients.producer.ProducerConfig;
+
+public class KafkaChannelConfiguration {
+
+  public static final String KAFKA_PREFIX = "kafka.";
+  public static final String KAFKA_CONSUMER_PREFIX = KAFKA_PREFIX + "consumer.";
+  public static final String KAFKA_PRODUCER_PREFIX = KAFKA_PREFIX + "producer.";
+  public static final String DEFAULT_ACKS = "all";
+  public static final String DEFAULT_KEY_SERIALIZER =
+      "org.apache.kafka.common.serialization.StringSerializer";
+  public static final String DEFAULT_VALUE_SERIAIZER =
+      "org.apache.kafka.common.serialization.ByteArraySerializer";
+  public static final String DEFAULT_KEY_DESERIALIZER =
+      "org.apache.kafka.common.serialization.StringDeserializer";
+  public static final String DEFAULT_VALUE_DESERIAIZER =
+      "org.apache.kafka.common.serialization.ByteArrayDeserializer";
+  public static final String TOPIC_CONFIG = KAFKA_PREFIX + "topic";
+  public static final String BOOTSTRAP_SERVERS_CONFIG =
+      KAFKA_PREFIX + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG;
+
+  public static final String TRANSACTIONAL_ID =
+          KAFKA_PRODUCER_PREFIX + ProducerConfig.TRANSACTIONAL_ID_CONFIG;
+  public static final String DEFAULT_TOPIC = "flume-channel";
+  public static final String DEFAULT_GROUP_ID = "flume";
+  public static final String POLL_TIMEOUT = KAFKA_PREFIX + "pollTimeout";
+  public static final long DEFAULT_POLL_TIMEOUT = 500;
+
+  public static final String KEY_HEADER = "key";
+
+  public static final String DEFAULT_AUTO_OFFSET_RESET = "earliest";
+
+  public static final String PARSE_AS_FLUME_EVENT = "parseAsFlumeEvent";
+  public static final boolean DEFAULT_PARSE_AS_FLUME_EVENT = true;
+
+  public static final String PARTITION_HEADER_NAME = "partitionIdHeader";
+  public static final String STATIC_PARTITION_CONF = "defaultPartitionId";
+
+}
diff --git a/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestBasicFunctionality.java b/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestBasicFunctionality.java
new file mode 100644
index 0000000..953133f
--- /dev/null
+++ b/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestBasicFunctionality.java
@@ -0,0 +1,220 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.flume.channel.kafka;
+
+import org.apache.flume.Context;
+import org.apache.flume.Event;
+import org.apache.flume.Transaction;
+import org.apache.flume.conf.Configurables;
+import org.apache.flume.instrumentation.kafka.KafkaChannelCounter;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.junit.Assert;
+import org.junit.Test;
+import org.mockito.internal.util.reflection.Whitebox;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Properties;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+import static org.apache.flume.channel.kafka.KafkaChannelConfiguration.BOOTSTRAP_SERVERS_CONFIG;
+import static org.apache.flume.channel.kafka.KafkaChannelConfiguration.KEY_HEADER;
+import static org.apache.flume.channel.kafka.KafkaChannelConfiguration.TOPIC_CONFIG;
+
+public class TestBasicFunctionality extends TestKafkaChannelBase {
+
+  @Test
+  public void testProps() throws Exception {
+    Context context = new Context();
+    context.put("kafka.producer.some-parameter", "1");
+    context.put("kafka.consumer.another-parameter", "1");
+    context.put(BOOTSTRAP_SERVERS_CONFIG, testUtil.getKafkaServerUrl());
+    context.put(TOPIC_CONFIG, topic);
+
+    final KafkaChannel channel = new KafkaChannel();
+    Configurables.configure(channel, context);
+
+    Properties consumerProps = channel.getConsumerProps();
+    Properties producerProps = channel.getProducerProps();
+
+    Assert.assertEquals(producerProps.getProperty("some-parameter"), "1");
+    Assert.assertEquals(consumerProps.getProperty("another-parameter"), "1");
+  }
+
+  @Test
+  public void testStopAndStart() throws Exception {
+    doTestStopAndStart(false, false);
+  }
+
+  @Test
+  public void testStopAndStartWithRollback() throws Exception {
+    doTestStopAndStart(true, true);
+  }
+
+  @Test
+  public void testStopAndStartWithRollbackAndNoRetry() throws Exception {
+    doTestStopAndStart(true, false);
+  }
+
+  @Test
+  public void testNullKeyNoHeader() throws Exception {
+    doTestNullKeyNoHeader();
+  }
+
+  /**
+   * Tests that sub-properties get set correctly if you run the configure() method twice
+   * (fix for FLUME-2857)
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testDefaultSettingsOnReConfigure() throws Exception {
+    String sampleProducerProp = "compression.type";
+    String sampleProducerVal = "snappy";
+
+    String sampleConsumerProp = "fetch.min.bytes";
+    String sampleConsumerVal = "99";
+
+    Context context = prepareDefaultContext(false);
+    context.put(KafkaChannelConfiguration.KAFKA_PRODUCER_PREFIX + sampleProducerProp,
+        sampleProducerVal);
+    context.put(KafkaChannelConfiguration.KAFKA_CONSUMER_PREFIX + sampleConsumerProp,
+        sampleConsumerVal);
+
+    final KafkaChannel channel = createChannel(context);
+
+    Assert.assertEquals(sampleProducerVal,
+        channel.getProducerProps().getProperty(sampleProducerProp));
+    Assert.assertEquals(sampleConsumerVal,
+        channel.getConsumerProps().getProperty(sampleConsumerProp));
+
+    context = prepareDefaultContext(false);
+    channel.configure(context);
+
+    Assert.assertNull(channel.getProducerProps().getProperty(sampleProducerProp));
+    Assert.assertNull(channel.getConsumerProps().getProperty(sampleConsumerProp));
+
+  }
+
+
+  private void doTestNullKeyNoHeader() throws Exception {
+    final KafkaChannel channel = startChannel(false);
+    Properties props = channel.getProducerProps();
+    KafkaProducer<String, byte[]> producer = new KafkaProducer<>(props);
+
+    for (int i = 0; i < 50; i++) {
+      ProducerRecord<String, byte[]> data =
+          new ProducerRecord<>(topic, null, String.valueOf(i).getBytes());
+      producer.send(data).get();
+    }
+    ExecutorCompletionService<Void> submitterSvc = new
+        ExecutorCompletionService<>(Executors.newCachedThreadPool());
+    List<Event> events = pullEvents(channel, submitterSvc,
+        50, false, false);
+    wait(submitterSvc, 5);
+    List<String> finals = new ArrayList<>(50);
+    for (int i = 0; i < 50; i++) {
+      finals.add(i, events.get(i).getHeaders().get(KEY_HEADER));
+    }
+    for (int i = 0; i < 50; i++) {
+      Assert.assertTrue(finals.get(i) == null);
+    }
+    channel.stop();
+  }
+
+  /**
+   * This method starts a channel, puts events into it. The channel is then
+   * stopped and restarted. Then we check to make sure if all events we put
+   * come out. Optionally, 10 events are rolled back,
+   * and optionally we restart the agent immediately after and we try to pull it
+   * out.
+   *
+   * @param rollback
+   * @param retryAfterRollback
+   * @throws Exception
+   */
+  private void doTestStopAndStart(boolean rollback,
+                                  boolean retryAfterRollback) throws Exception {
+    final KafkaChannel channel = startChannel(true);
+    ExecutorService underlying = Executors
+        .newCachedThreadPool();
+    ExecutorCompletionService<Void> submitterSvc =
+        new ExecutorCompletionService<>(underlying);
+    final List<List<Event>> events = createBaseList();
+    putEvents(channel, events, submitterSvc);
+    wait(submitterSvc, 5);
+    channel.stop();
+    final KafkaChannel channel2 = startChannel(true);
+    int total = 50;
+    if (rollback && !retryAfterRollback) {
+      total = 40;
+    }
+    final List<Event> eventsPulled =
+        pullEvents(channel2, submitterSvc, total, rollback, retryAfterRollback);
+    wait(submitterSvc, 5);
+    channel2.stop();
+    if (!retryAfterRollback && rollback) {
+      final KafkaChannel channel3 = startChannel(true);
+      int expectedRemaining = 50 - eventsPulled.size();
+      final List<Event> eventsPulled2 =
+          pullEvents(channel3, submitterSvc, expectedRemaining, false, false);
+      wait(submitterSvc, 5);
+      Assert.assertEquals(expectedRemaining, eventsPulled2.size());
+      eventsPulled.addAll(eventsPulled2);
+      channel3.stop();
+    }
+    underlying.shutdownNow();
+    verify(eventsPulled);
+  }
+
+  @Test
+  public void testMetricsCount() throws Exception {
+    final KafkaChannel channel = startChannel(true);
+    ExecutorService underlying = Executors.newCachedThreadPool();
+    ExecutorCompletionService<Void> submitterSvc = new ExecutorCompletionService<Void>(underlying);
+    final List<List<Event>> events = createBaseList();
+    putEvents(channel, events, submitterSvc);
+    takeEventsWithCommittingTxn(channel,50);
+
+    KafkaChannelCounter counter =
+            (KafkaChannelCounter) Whitebox.getInternalState(channel, "counter");
+    Assert.assertEquals(50, counter.getEventPutAttemptCount());
+    Assert.assertEquals(50, counter.getEventPutSuccessCount());
+    Assert.assertEquals(50, counter.getEventTakeAttemptCount());
+    Assert.assertEquals(50, counter.getEventTakeSuccessCount());
+    channel.stop();
+  }
+
+  private void takeEventsWithCommittingTxn(KafkaChannel channel, long eventsCount) {
+    List<Event> takeEventsList = new ArrayList<>();
+    Transaction txn = channel.getTransaction();
+    txn.begin();
+    while (takeEventsList.size() < eventsCount) {
+      Event event = channel.take();
+      if (event != null) {
+        takeEventsList.add(event);
+      }
+    }
+    txn.commit();
+    txn.close();
+  }
+}
diff --git a/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestKafkaChannelBase.java b/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestKafkaChannelBase.java
new file mode 100644
index 0000000..b370749
--- /dev/null
+++ b/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestKafkaChannelBase.java
@@ -0,0 +1,272 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.flume.channel.kafka;
+
+import com.google.common.collect.Lists;
+import org.apache.commons.lang.RandomStringUtils;
+import org.apache.flume.Context;
+import org.apache.flume.Event;
+import org.apache.flume.Transaction;
+import org.apache.flume.conf.Configurables;
+import org.apache.flume.event.EventBuilder;
+import org.apache.flume.sink.kafka.util.TestUtil;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.apache.flume.channel.kafka.KafkaChannelConfiguration.*;
+import static org.apache.kafka.clients.producer.ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG;
+
+public class TestKafkaChannelBase {
+
+  static TestUtil testUtil = TestUtil.getInstance();
+  String topic = null;
+  private final Set<String> usedTopics = new HashSet<>();
+
+  static final int DEFAULT_TOPIC_PARTITIONS = 5;
+
+  @BeforeClass
+  public static void setupClass() throws Exception {
+    testUtil.prepare();
+    Thread.sleep(2500);
+  }
+
+  @Before
+  public void setup() throws Exception {
+    topic = findUnusedTopic();
+    createTopic(topic, DEFAULT_TOPIC_PARTITIONS);
+    Thread.sleep(2500);
+  }
+
+  @AfterClass
+  public static void tearDown() {
+    testUtil.tearDown();
+  }
+
+  String findUnusedTopic() {
+    String newTopic = null;
+    boolean topicFound = false;
+    while (!topicFound) {
+      newTopic = RandomStringUtils.randomAlphabetic(8);
+      if (!usedTopics.contains(newTopic)) {
+        usedTopics.add(newTopic);
+        topicFound = true;
+      }
+    }
+    return newTopic;
+  }
+
+  static void createTopic(String topicName, int numPartitions) {
+    testUtil.createTopics(Collections.singletonList(topicName), numPartitions);
+  }
+
+  static void deleteTopic(String topicName) {
+    testUtil.deleteTopic(topicName);
+  }
+
+  KafkaChannel startChannel(boolean parseAsFlume) throws Exception {
+    return startChannel(parseAsFlume, false);
+  }
+  KafkaChannel startChannel(boolean parseAsFlume, boolean useKafksTxns) throws Exception {
+    Context context = prepareDefaultContext(parseAsFlume, useKafksTxns);
+    KafkaChannel channel = createChannel(context);
+    channel.start();
+    return channel;
+  }
+
+  Context prepareDefaultContext(boolean parseAsFlume) {
+    return prepareDefaultContext(parseAsFlume, false);
+  }
+
+  Context prepareDefaultContext(boolean parseAsFlume, boolean useKafkaTxns) {
+    // Prepares a default context with Kafka Server Properties
+    Context context = new Context();
+    context.put(BOOTSTRAP_SERVERS_CONFIG, testUtil.getKafkaServerUrl());
+    context.put(PARSE_AS_FLUME_EVENT, String.valueOf(parseAsFlume));
+    context.put(TOPIC_CONFIG, topic);
+    context.put(KAFKA_CONSUMER_PREFIX + "max.poll.interval.ms", "10000");
+    if (useKafkaTxns) {
+      context.put(TRANSACTIONAL_ID, "3");
+      context.put("kafka.producer." + ENABLE_IDEMPOTENCE_CONFIG, "true");
+      context.put("kafka.producer.acks", "all");
+      context.put("kafka.consumer.isolation.level", "read_committed");
+    }
+
+    return context;
+  }
+
+  KafkaChannel createChannel(Context context) throws Exception {
+    final KafkaChannel channel = new KafkaChannel();
+    Configurables.configure(channel, context);
+    return channel;
+  }
+
+  List<Event> pullEvents(final KafkaChannel channel,
+                         ExecutorCompletionService<Void> submitterSvc, final int total,
+                         final boolean testRollbacks, final boolean retryAfterRollback) {
+    final List<Event> eventsPulled = Collections.synchronizedList(new
+        ArrayList<Event>(50));
+    final CyclicBarrier barrier = new CyclicBarrier(5);
+    final AtomicInteger counter = new AtomicInteger(0);
+    final AtomicInteger rolledBackCount = new AtomicInteger(0);
+    final AtomicBoolean startedGettingEvents = new AtomicBoolean(false);
+    final AtomicBoolean rolledBack = new AtomicBoolean(false);
+    for (int k = 0; k < 5; k++) {
+      final int index = k;
+      submitterSvc.submit(new Callable<Void>() {
+        @Override
+        public Void call() throws Exception {
+          Transaction tx = null;
+          final List<Event> eventsLocal = Lists.newLinkedList();
+          channel.registerThread();
+          Thread.sleep(1000);
+          barrier.await();
+          while (counter.get() < (total - rolledBackCount.get())) {
+            if (tx == null) {
+              tx = channel.getTransaction();
+              tx.begin();
+            }
+            try {
+              Event e = channel.take();
+              if (e != null) {
+                startedGettingEvents.set(true);
+                eventsLocal.add(e);
+              } else {
+                if (testRollbacks &&
+                    index == 4 &&
+                    (!rolledBack.get()) &&
+                    startedGettingEvents.get()) {
+                  tx.rollback();
+                  tx.close();
+                  tx = null;
+                  rolledBack.set(true);
+                  final int eventsLocalSize = eventsLocal.size();
+                  eventsLocal.clear();
+                  if (!retryAfterRollback) {
+                    rolledBackCount.set(eventsLocalSize);
+                    return null;
+                  }
+                } else {
+                  tx.commit();
+                  tx.close();
+                  tx = null;
+                  eventsPulled.addAll(eventsLocal);
+                  counter.getAndAdd(eventsLocal.size());
+                  eventsLocal.clear();
+                }
+              }
+            } catch (Exception ex) {
+              eventsLocal.clear();
+              if (tx != null) {
+                tx.rollback();
+                tx.close();
+              }
+              tx = null;
+              ex.printStackTrace();
+            }
+          }
+          // Close txn.
+          return null;
+        }
+      });
+    }
+    return eventsPulled;
+  }
+
+  void wait(ExecutorCompletionService<Void> submitterSvc, int max)
+      throws Exception {
+    int completed = 0;
+    while (completed < max) {
+      submitterSvc.take();
+      completed++;
+    }
+  }
+
+  List<List<Event>> createBaseList() {
+    final List<List<Event>> events = new ArrayList<>();
+    for (int i = 0; i < 5; i++) {
+      List<Event> eventList = new ArrayList<>(10);
+      events.add(eventList);
+      for (int j = 0; j < 10; j++) {
+        Map<String, String> hdrs = new HashMap<>();
+        String v = (String.valueOf(i) + " - " + String
+            .valueOf(j));
+        hdrs.put("header", v);
+        eventList.add(EventBuilder.withBody(v.getBytes(), hdrs));
+      }
+    }
+    return events;
+  }
+
+  void putEvents(final KafkaChannel channel, final List<List<Event>>
+      events, ExecutorCompletionService<Void> submitterSvc) {
+    for (int i = 0; i < 5; i++) {
+      final int index = i;
+      submitterSvc.submit(new Callable<Void>() {
+        @Override
+        public Void call() {
+          Transaction tx = channel.getTransaction();
+          tx.begin();
+          List<Event> eventsToPut = events.get(index);
+          for (int j = 0; j < 10; j++) {
+            channel.put(eventsToPut.get(j));
+          }
+          try {
+            tx.commit();
+          } finally {
+            tx.close();
+          }
+          return null;
+        }
+      });
+    }
+  }
+
+  void verify(List<Event> eventsPulled) {
+    Assert.assertFalse(eventsPulled.isEmpty());
+    Assert.assertEquals(50, eventsPulled.size());
+    Set<String> eventStrings = new HashSet<>();
+    for (Event e : eventsPulled) {
+      Assert.assertEquals(e.getHeaders().get("header"), new String(e.getBody()));
+      eventStrings.add(e.getHeaders().get("header"));
+    }
+    for (int i = 0; i < 5; i++) {
+      for (int j = 0; j < 10; j++) {
+        String v = String.valueOf(i) + " - " + String.valueOf(j);
+        Assert.assertTrue(eventStrings.contains(v));
+        eventStrings.remove(v);
+      }
+    }
+    Assert.assertTrue(eventStrings.isEmpty());
+  }
+}
diff --git a/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestOffsetsAndMigration.java b/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestOffsetsAndMigration.java
new file mode 100644
index 0000000..e58b5b1
--- /dev/null
+++ b/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestOffsetsAndMigration.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.flume.channel.kafka;
+
+import org.apache.flume.Event;
+import org.apache.flume.Transaction;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.Arrays;
+
+public class TestOffsetsAndMigration extends TestKafkaChannelBase {
+
+  @Test
+  public void testOffsetsNotCommittedOnStop() throws Exception {
+    String message = "testOffsetsNotCommittedOnStop-" + System.nanoTime();
+
+    KafkaChannel channel = startChannel(false);
+
+    KafkaProducer<String, byte[]> producer =
+        new KafkaProducer<>(channel.getProducerProps());
+    ProducerRecord<String, byte[]> data =
+        new ProducerRecord<>(topic, "header-" + message, message.getBytes());
+    producer.send(data).get();
+    producer.flush();
+    producer.close();
+
+    Event event = takeEventWithoutCommittingTxn(channel);
+    Assert.assertNotNull(event);
+    Assert.assertTrue(Arrays.equals(message.getBytes(), event.getBody()));
+
+    // Stop the channel without committing the transaction
+    channel.stop();
+
+    channel = startChannel(false);
+
+    // Message should still be available
+    event = takeEventWithoutCommittingTxn(channel);
+    Assert.assertNotNull(event);
+    Assert.assertTrue(Arrays.equals(message.getBytes(), event.getBody()));
+  }
+
+  private Event takeEventWithoutCommittingTxn(KafkaChannel channel) {
+    for (int i = 0; i < 10; i++) {
+      Transaction txn = channel.getTransaction();
+      txn.begin();
+
+      Event event = channel.take();
+      if (event != null) {
+        return event;
+      } else {
+        txn.commit();
+        txn.close();
+      }
+    }
+    return null;
+  }
+
+}
diff --git a/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestParseAsFlumeEvent.java b/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestParseAsFlumeEvent.java
new file mode 100644
index 0000000..6baad77
--- /dev/null
+++ b/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestParseAsFlumeEvent.java
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.flume.channel.kafka;
+
+import org.apache.flume.Event;
+import org.apache.flume.Transaction;
+import org.apache.flume.event.EventBuilder;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.Executors;
+
+import static org.apache.flume.channel.kafka.KafkaChannelConfiguration.KEY_HEADER;
+
+public class TestParseAsFlumeEvent extends TestKafkaChannelBase {
+
+  @Test
+  public void testParseAsFlumeEventFalse() throws Exception {
+    doParseAsFlumeEventFalse(false);
+  }
+
+  @Test
+  public void testParseAsFlumeEventFalseCheckHeader() throws Exception {
+    doParseAsFlumeEventFalse(true);
+  }
+
+  @Test
+  public void testParseAsFlumeEventFalseAsSource() throws Exception {
+    doParseAsFlumeEventFalseAsSource(false);
+  }
+
+  @Test
+  public void testParseAsFlumeEventFalseAsSourceCheckHeader() throws Exception {
+    doParseAsFlumeEventFalseAsSource(true);
+  }
+
+  private void doParseAsFlumeEventFalse(Boolean checkHeaders) throws Exception {
+    final KafkaChannel channel = startChannel(false);
+    Properties props = channel.getProducerProps();
+    KafkaProducer<String, byte[]> producer = new KafkaProducer<>(props);
+
+    for (int i = 0; i < 50; i++) {
+      ProducerRecord<String, byte[]> data =
+          new ProducerRecord<>(topic, String.valueOf(i) + "-header",
+              String.valueOf(i).getBytes());
+      producer.send(data).get();
+    }
+    ExecutorCompletionService<Void> submitterSvc = new
+        ExecutorCompletionService<>(Executors.newCachedThreadPool());
+    List<Event> events = pullEvents(channel, submitterSvc, 50, false, false);
+    wait(submitterSvc, 5);
+    Map<Integer, String> finals = new HashMap<>();
+    for (int i = 0; i < 50; i++) {
+      finals.put(Integer.parseInt(new String(events.get(i).getBody())),
+          events.get(i).getHeaders().get(KEY_HEADER));
+    }
+    for (int i = 0; i < 50; i++) {
+      Assert.assertTrue(finals.keySet().contains(i));
+      if (checkHeaders) {
+        Assert.assertTrue(finals.containsValue(String.valueOf(i) + "-header"));
+      }
+      finals.remove(i);
+    }
+    Assert.assertTrue(finals.isEmpty());
+    channel.stop();
+  }
+
+  /**
+   * Like the previous test but here we write to the channel like a Flume source would do
+   * to verify that the events are written as text and not as an Avro object
+   *
+   * @throws Exception
+   */
+  private void doParseAsFlumeEventFalseAsSource(Boolean checkHeaders) throws Exception {
+    final KafkaChannel channel = startChannel(false);
+
+    List<String> msgs = new ArrayList<>();
+    Map<String, String> headers = new HashMap<>();
+    for (int i = 0; i < 50; i++) {
+      msgs.add(String.valueOf(i));
+    }
+    Transaction tx = channel.getTransaction();
+    tx.begin();
+    for (int i = 0; i < msgs.size(); i++) {
+      headers.put(KEY_HEADER, String.valueOf(i) + "-header");
+      channel.put(EventBuilder.withBody(msgs.get(i).getBytes(), headers));
+    }
+    tx.commit();
+    ExecutorCompletionService<Void> submitterSvc =
+        new ExecutorCompletionService<>(Executors.newCachedThreadPool());
+    List<Event> events = pullEvents(channel, submitterSvc, 50, false, false);
+    wait(submitterSvc, 5);
+    Map<Integer, String> finals = new HashMap<>();
+    for (int i = 0; i < 50; i++) {
+      finals.put(Integer.parseInt(new String(events.get(i).getBody())),
+          events.get(i).getHeaders().get(KEY_HEADER));
+    }
+    for (int i = 0; i < 50; i++) {
+      Assert.assertTrue(finals.keySet().contains(i));
+      if (checkHeaders) {
+        Assert.assertTrue(finals.containsValue(String.valueOf(i) + "-header"));
+      }
+      finals.remove(i);
+    }
+    Assert.assertTrue(finals.isEmpty());
+    channel.stop();
+  }
+}
diff --git a/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestPartitions.java b/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestPartitions.java
new file mode 100644
index 0000000..9652fb1
--- /dev/null
+++ b/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestPartitions.java
@@ -0,0 +1,179 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.flume.channel.kafka;
+
+import org.apache.flume.Context;
+import org.apache.flume.Event;
+import org.apache.flume.Transaction;
+import org.apache.flume.event.EventBuilder;
+import org.apache.flume.shared.kafka.test.KafkaPartitionTestUtil;
+import org.apache.flume.shared.kafka.test.PartitionOption;
+import org.apache.flume.shared.kafka.test.PartitionTestScenario;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.flume.channel.kafka.KafkaChannelConfiguration.PARTITION_HEADER_NAME;
+import static org.apache.flume.channel.kafka.KafkaChannelConfiguration.STATIC_PARTITION_CONF;
+
+public class TestPartitions extends TestKafkaChannelBase {
+
+  @Test
+  public void testPartitionHeaderSet() throws Exception {
+    doPartitionHeader(PartitionTestScenario.PARTITION_ID_HEADER_ONLY);
+  }
+
+  @Test
+  public void testPartitionHeaderNotSet() throws Exception {
+    doPartitionHeader(PartitionTestScenario.NO_PARTITION_HEADERS);
+  }
+
+  @Test
+  public void testStaticPartitionAndHeaderSet() throws Exception {
+    doPartitionHeader(PartitionTestScenario.STATIC_HEADER_AND_PARTITION_ID);
+  }
+
+  @Test
+  public void testStaticPartitionHeaderNotSet() throws Exception {
+    doPartitionHeader(PartitionTestScenario.STATIC_HEADER_ONLY);
+  }
+
+  @Test
+  public void testPartitionHeaderMissing() throws Exception {
+    doPartitionErrors(PartitionOption.NOTSET);
+  }
+
+  @Test(expected = org.apache.flume.ChannelException.class)
+  public void testPartitionHeaderOutOfRange() throws Exception {
+    doPartitionErrors(PartitionOption.VALIDBUTOUTOFRANGE);
+  }
+
+  @Test(expected = org.apache.flume.ChannelException.class)
+  public void testPartitionHeaderInvalid() throws Exception {
+    doPartitionErrors(PartitionOption.NOTANUMBER);
+  }
+
+  /**
+   * This method tests both the default behavior (usePartitionHeader=false)
+   * and the behaviour when the partitionId setting is used.
+   * Under the default behaviour, one would expect an even distribution of
+   * messages to partitions, however when partitionId is used we manually create
+   * a large skew to some partitions and then verify that this actually happened
+   * by reading messages directly using a Kafka Consumer.
+   *
+   * @param scenario
+   * @throws Exception
+   */
+  private void doPartitionHeader(PartitionTestScenario scenario) throws Exception {
+    final int numPtns = DEFAULT_TOPIC_PARTITIONS;
+    final int numMsgs = numPtns * 10;
+    final Integer staticPtn = DEFAULT_TOPIC_PARTITIONS - 2;
+    Context context = prepareDefaultContext(false);
+    if (scenario == PartitionTestScenario.PARTITION_ID_HEADER_ONLY ||
+        scenario == PartitionTestScenario.STATIC_HEADER_AND_PARTITION_ID) {
+      context.put(PARTITION_HEADER_NAME, "partition-header");
+    }
+    if (scenario == PartitionTestScenario.STATIC_HEADER_AND_PARTITION_ID ||
+        scenario == PartitionTestScenario.STATIC_HEADER_ONLY) {
+      context.put(STATIC_PARTITION_CONF, staticPtn.toString());
+    }
+    final KafkaChannel channel = createChannel(context);
+    channel.start();
+
+    // Create a map of PartitionId:List<Messages> according to the desired distribution
+    // Initialise with empty ArrayLists
+    Map<Integer, List<Event>> partitionMap = new HashMap<>(numPtns);
+    for (int i = 0; i < numPtns; i++) {
+      partitionMap.put(i, new ArrayList<Event>());
+    }
+    Transaction tx = channel.getTransaction();
+    tx.begin();
+
+    List<Event> orderedEvents = KafkaPartitionTestUtil.generateSkewedMessageList(scenario, numMsgs,
+        partitionMap, numPtns, staticPtn);
+
+    for (Event event : orderedEvents) {
+      channel.put(event);
+    }
+
+    tx.commit();
+
+    Map<Integer, List<byte[]>> resultsMap = KafkaPartitionTestUtil.retrieveRecordsFromPartitions(
+        topic, numPtns, channel.getConsumerProps());
+
+    KafkaPartitionTestUtil.checkResultsAgainstSkew(scenario, partitionMap, resultsMap, staticPtn,
+        numMsgs);
+
+    channel.stop();
+  }
+
+  /**
+   * This function tests three scenarios:
+   * 1. PartitionOption.VALIDBUTOUTOFRANGE: An integer partition is provided,
+   * however it exceeds the number of partitions available on the topic.
+   * Expected behaviour: ChannelException thrown.
+   * <p>
+   * 2. PartitionOption.NOTSET: The partition header is not actually set.
+   * Expected behaviour: Exception is not thrown because the code avoids an NPE.
+   * <p>
+   * 3. PartitionOption.NOTANUMBER: The partition header is set, but is not an Integer.
+   * Expected behaviour: ChannelExeption thrown.
+   *
+   * @param option
+   * @throws Exception
+   */
+  private void doPartitionErrors(PartitionOption option) throws Exception {
+    Context context = prepareDefaultContext(false);
+    context.put(PARTITION_HEADER_NAME, KafkaPartitionTestUtil.PARTITION_HEADER);
+    String tempTopic = findUnusedTopic();
+    createTopic(tempTopic, 5);
+    final KafkaChannel channel = createChannel(context);
+    channel.start();
+
+    Transaction tx = channel.getTransaction();
+    tx.begin();
+
+    Map<String, String> headers = new HashMap<>();
+    switch (option) {
+      case VALIDBUTOUTOFRANGE:
+        headers.put(KafkaPartitionTestUtil.PARTITION_HEADER,
+            String.valueOf(DEFAULT_TOPIC_PARTITIONS + 2));
+        break;
+      case NOTSET:
+        headers.put("wrong-header", "2");
+        break;
+      case NOTANUMBER:
+        headers.put(KafkaPartitionTestUtil.PARTITION_HEADER, "not-a-number");
+        break;
+      default:
+        break;
+    }
+
+    Event event = EventBuilder.withBody(String.valueOf(9).getBytes(), headers);
+
+    channel.put(event);
+
+    tx.commit();
+
+    deleteTopic(tempTopic);
+  }
+}
diff --git a/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestRollback.java b/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestRollback.java
new file mode 100644
index 0000000..219b026
--- /dev/null
+++ b/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestRollback.java
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.flume.channel.kafka;
+
+import org.apache.flume.Event;
+import org.junit.Test;
+
+import java.util.List;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.Executors;
+
+public class TestRollback extends TestKafkaChannelBase {
+
+  @Test
+  public void testSuccess() throws Exception {
+    doTestSuccessRollback(false, false, false);
+  }
+
+  @Test
+  public void testSuccessInterleave() throws Exception {
+    doTestSuccessRollback(false, true, false);
+  }
+
+  @Test
+  public void testRollbacks() throws Exception {
+    doTestSuccessRollback(true, false, false);
+  }
+
+  @Test
+  public void testRollbacksInterleave() throws Exception {
+    doTestSuccessRollback(true, true, false);
+  }
+
+  @Test
+  public void testSuccessTxns() throws Exception {
+    doTestSuccessRollback(false, false, true);
+  }
+
+  @Test
+  public void testSuccessInterleaveTxns() throws Exception {
+    doTestSuccessRollback(false, true, true);
+  }
+
+  @Test
+  public void testRollbacksTxns() throws Exception {
+    doTestSuccessRollback(true, false, true);
+  }
+
+  @Test
+  public void testRollbacksInterleaveTxns() throws Exception {
+    doTestSuccessRollback(true, true, true);
+  }
+
+  private void doTestSuccessRollback(final boolean rollback,
+                                     final boolean interleave,
+                                     final boolean useKafkaTxns) throws Exception {
+    final KafkaChannel channel = startChannel(true, useKafkaTxns);
+    writeAndVerify(rollback, channel, interleave);
+    channel.stop();
+  }
+
+  private void writeAndVerify(final boolean testRollbacks,
+                              final KafkaChannel channel, final boolean interleave)
+      throws Exception {
+
+    final List<List<Event>> events = createBaseList();
+
+    ExecutorCompletionService<Void> submitterSvc =
+        new ExecutorCompletionService<Void>(Executors.newCachedThreadPool());
+
+    putEvents(channel, events, submitterSvc);
+
+    if (interleave) {
+      wait(submitterSvc, 5);
+    }
+
+    ExecutorCompletionService<Void> submitterSvc2 =
+        new ExecutorCompletionService<Void>(Executors.newCachedThreadPool());
+
+    final List<Event> eventsPulled = pullEvents(channel, submitterSvc2, 50, testRollbacks, true);
+
+    if (!interleave) {
+      wait(submitterSvc, 5);
+    }
+    wait(submitterSvc2, 5);
+
+    verify(eventsPulled);
+  }
+}
diff --git a/flume-kafka-channel/src/test/resources/kafka-server.properties b/flume-kafka-channel/src/test/resources/kafka-server.properties
new file mode 100644
index 0000000..d8a5b28
--- /dev/null
+++ b/flume-kafka-channel/src/test/resources/kafka-server.properties
@@ -0,0 +1,133 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+# see kafka.server.KafkaConfig for additional details and defaults
+
+############################# Server Basics #############################
+
+# The id of the broker. This must be set to a unique integer for each broker.
+broker.id=0
+
+############################# Socket Server Settings #############################
+
+# The port the socket server listens on
+port=9092
+
+# Hostname the broker will bind to. If not set, the server will bind to all interfaces
+#host.name=localhost
+
+# Hostname the broker will advertise to producers and consumers. If not set, it uses the
+# value for "host.name" if configured.  Otherwise, it will use the value returned from
+# java.net.InetAddress.getCanonicalHostName().
+#advertised.host.name=<hostname routable by clients>
+
+# The port to publish to ZooKeeper for clients to use. If this is not set,
+# it will publish the same port that the broker binds to.
+#advertised.port=<port accessible by clients>
+
+# The number of threads handling network requests
+num.network.threads=4
+
+# The number of threads doing disk I/O
+num.io.threads=8
+
+# The send buffer (SO_SNDBUF) used by the socket server
+socket.send.buffer.bytes=1048576
+
+# The receive buffer (SO_RCVBUF) used by the socket server
+socket.receive.buffer.bytes=1048576
+
+# The maximum size of a request that the socket server will accept (protection against OOM)
+socket.request.max.bytes=104857600
+
+
+############################# Log Basics #############################
+
+# A comma seperated list of directories under which to store log files
+log.dirs=target/kafka-logs
+
+# The default number of log partitions per topic. More partitions allow greater
+# parallelism for consumption, but this will also result in more files across
+# the brokers.
+num.partitions=5
+
+############################# Log Flush Policy #############################
+
+# Messages are immediately written to the filesystem but by default we only fsync() to sync
+# the OS cache lazily. The following configurations control the flush of data to disk.
+# There are a few important trade-offs here:
+#    1. Durability: Unflushed data may be lost if you are not using replication.
+#    2. Latency: Very large flush intervals may lead to latency spikes when the flush does occur as
+#    there will be a lot of data to flush.
+#    3. Throughput: The flush is generally the most expensive operation, and a small flush interval
+#    may lead to exceessive seeks.
+# The settings below allow one to configure the flush policy to flush data after a period of time or
+# every N messages (or both). This can be done globally and overridden on a per-topic basis.
+
+# The number of messages to accept before forcing a flush of data to disk
+#log.flush.interval.messages=10000
+
+# The maximum amount of time a message can sit in a log before we force a flush
+#log.flush.interval.ms=1000
+
+############################# Log Retention Policy #############################
+
+# The following configurations control the disposal of log segments. The policy can
+# be set to delete segments after a period of time, or after a given size has accumulated.
+# A segment will be deleted whenever *either* of these criteria are met. Deletion always happens
+# from the end of the log.
+
+# The minimum age of a log file to be eligible for deletion
+log.retention.hours=168
+
+# A size-based retention policy for logs. Segments are pruned from the log as long as the remaining
+# segments don't drop below log.retention.bytes.
+#log.retention.bytes=1073741824
+
+# The maximum size of a log segment file. When this size is reached a new log segment will be
+# created.
+log.segment.bytes=536870912
+
+# The interval at which log segments are checked to see if they can be deleted according
+# to the retention policies
+log.retention.check.interval.ms=60000
+
+# By default the log cleaner is disabled and the log retention policy will default to just delete
+# segments after their retention expires.
+# If log.cleaner.enable=true is set the cleaner will be enabled and individual logs can then be
+# marked for log compaction.
+log.cleaner.enable=false
+
+############################# Transactions #############################
+
+# Settings only to be used for non-production
+transaction.state.log.replication.factor = 1
+transaction.state.log.min.isr = 1
+transaction.state.log.num.partitions = 1
+
+############################# Zookeeper #############################
+
+# Zookeeper connection string (see zookeeper docs for details).
+# This is a comma separated host:port pairs, each corresponding to a zk
+# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002".
+# You can also append an optional chroot string to the urls to specify the
+# root directory for all kafka znodes.
+zookeeper.connect=localhost:2181
+
+# Timeout in ms for connecting to zookeeper
+zookeeper.connection.timeout.ms=1000000
+
+offsets.topic.replication.factor=1
+
+auto.create.topics.enable=false
diff --git a/flume-kafka-channel/src/test/resources/keystorefile.jks b/flume-kafka-channel/src/test/resources/keystorefile.jks
new file mode 100644
index 0000000..20ac6a8
Binary files /dev/null and b/flume-kafka-channel/src/test/resources/keystorefile.jks differ
diff --git a/flume-kafka-channel/src/test/resources/log4j2.xml b/flume-kafka-channel/src/test/resources/log4j2.xml
new file mode 100644
index 0000000..b4b0067
--- /dev/null
+++ b/flume-kafka-channel/src/test/resources/log4j2.xml
@@ -0,0 +1,72 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements.  See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License.  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+
+-->
+<Configuration status="OFF">
+  <Properties>
+    <Property name="kafka.logs.dir">target/logs</Property>
+  </Properties>
+  <Appenders>
+    <Console name="Console" target="SYSTEM_OUT">
+      <PatternLayout pattern="%d (%t) [%p - %l] %m%n" />
+    </Console>
+    <RollingFile name="kafka" fileName="${kafka.logs.dir}/server.log" filePattern="${kafka.logs.dir}/server.log.%i">
+      <PatternLayout pattern="[%d] %p %m (%c)%n" />
+      <CronTriggeringPolicy schedule="0 0 0 * * ?"/>
+    </RollingFile>
+    <RollingFile name="stateChange" fileName="${kafka.logs.dir}/state-change.log" filePattern="${kafka.logs.dir}/state-change.log.%i">
+      <PatternLayout pattern="[%d] %p %m (%c)%n" />
+      <CronTriggeringPolicy schedule="0 0 0 * * ?"/>
+    </RollingFile>
+    <RollingFile name="request" fileName="${kafka.logs.dir}/kafka-request.log" filePattern="${kafka.logs.dir}/kafka-request.log.%i">
+      <PatternLayout pattern="[%d] %p %m (%c)%n" />
+      <CronTriggeringPolicy schedule="0 0 0 * * ?"/>
+    </RollingFile>
+    <RollingFile name="cleaner" fileName="${kafka.logs.dir}/log-cleaner.log" filePattern="${kafka.logs.dir}/log-cleaner.log.%i">
+      <PatternLayout pattern="[%d] %p %m (%c)%n" />
+      <CronTriggeringPolicy schedule="0 0 0 * * ?"/>
+    </RollingFile>
+    <RollingFile name="controller" fileName="${kafka.logs.dir}/controller.log" filePattern="${kafka.logs.dir}/controller.log.%i">
+      <PatternLayout pattern="[%d] %p %m (%c)%n" />
+      <CronTriggeringPolicy schedule="0 0 0 * * ?"/>
+    </RollingFile>
+  </Appenders>
+
+  <Loggers>
+    <Logger name="kafka" level="info">
+      <AppenderRef ref="kafka"/>
+    </Logger>
+    <Logger name="kafka.network.RequestChannel.*" level="WARN" additivity="false">
+      <AppenderRef ref="request"/>
+    </Logger>
+    <Logger name="kafka.request.logger" level="WARN" additivity="false">
+      <AppenderRef ref="request"/>
+    </Logger>
+    <Logger name="kafka.controller" level="TRACE" additivity="false">
+      <AppenderRef ref="controller"/>
+    </Logger>
+    <Logger name="kafka.log.cleaner" level="INFO" additivity="false">
+      <AppenderRef ref="cleaner"/>
+    </Logger>
+    <Logger name="state.change.logger" level="TRACE" additivity="false">
+      <AppenderRef ref="stateChange"/>
+    </Logger>
+    <Root level="INFO">
+      <AppenderRef ref="Console" />
+    </Root>
+  </Loggers>
+</Configuration>
\ No newline at end of file
diff --git a/flume-kafka-channel/src/test/resources/truststorefile.jks b/flume-kafka-channel/src/test/resources/truststorefile.jks
new file mode 100644
index 0000000..a98c490
Binary files /dev/null and b/flume-kafka-channel/src/test/resources/truststorefile.jks differ
diff --git a/flume-kafka-channel/src/test/resources/zookeeper.properties b/flume-kafka-channel/src/test/resources/zookeeper.properties
new file mode 100644
index 0000000..89e1b5e
--- /dev/null
+++ b/flume-kafka-channel/src/test/resources/zookeeper.properties
@@ -0,0 +1,20 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+# the directory where the snapshot is stored.
+dataDir=target
+# the port at which the clients will connect
+clientPort=2181
+# disable the per-ip limit on the number of connections since this is a non-production config
+maxClientCnxns=0
\ No newline at end of file
diff --git a/flume-kafka-dist/pom.xml b/flume-kafka-dist/pom.xml
new file mode 100644
index 0000000..d409499
--- /dev/null
+++ b/flume-kafka-dist/pom.xml
@@ -0,0 +1,165 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+
+  <parent>
+    <groupId>org.apache.flume</groupId>
+    <artifactId>flume-kafka-parent</artifactId>
+    <version>2.0.0-SNAPSHOT</version>
+  </parent>
+
+  <artifactId>flume-kafka-dist</artifactId>
+  <name>Flume Kafka Distribution</name>
+  <packaging>pom</packaging>
+
+  <properties>
+    <maven.deploy.skip>true</maven.deploy.skip>
+    <maven.install.skip>true</maven.install.skip>
+    <maven.test.skip>true</maven.test.skip>
+    <spotless.check.skip>true</spotless.check.skip>
+  </properties>
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.flume</groupId>
+      <artifactId>flume-shared-kafka</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.flume</groupId>
+      <artifactId>flume-kafka-sink</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.flume</groupId>
+      <artifactId>flume-kafka-source</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.flume</groupId>
+      <artifactId>flume-kafka-channel</artifactId>
+    </dependency>
+  </dependencies>
+
+  <build>
+    <plugins>
+      <!-- calculate checksums of source release for Apache dist area -->
+      <plugin>
+        <groupId>net.nicoulaj.maven.plugins</groupId>
+        <artifactId>checksum-maven-plugin</artifactId>
+        <version>${checksum-maven-plugin.version}</version>
+        <executions>
+          <execution>
+            <id>calculate-checksums</id>
+            <goals>
+              <goal>files</goal>
+            </goals>
+            <!-- execute prior to maven-gpg-plugin:sign due to https://github.com/nicoulaj/checksum-maven-plugin/issues/112 -->
+            <phase>post-integration-test</phase>
+            <configuration>
+              <algorithms>
+                <algorithm>SHA-512</algorithm>
+              </algorithms>
+              <!-- https://maven.apache.org/apache-resource-bundles/#source-release-assembly-descriptor -->
+              <fileSets>
+                <fileSet>
+                  <directory>${project.build.directory}</directory>
+                  <includes>
+                    <include>apache-flume-spring-boot-${project.version}-src.zip</include>
+                    <include>apache-flume-spring-boot-${project.version}-src.tar.gz</include>
+                    <include>apache-flume-spring-boot-${project.version}-bin.zip</include>
+                    <include>apache-flume-spring-boot-${project.version}-bin.tar.gz</include>
+                  </includes>
+                </fileSet>
+              </fileSets>
+              <csvSummary>false</csvSummary>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <artifactId>maven-antrun-plugin</artifactId>
+        <executions>
+          <execution>
+            <phase>post-integration-test</phase>
+            <configuration>
+              <target>
+                <property name="spaces" value=" " />
+                <concat destfile="${project.build.directory}/apache-flume-spring-boot-${project.version}-src.zip.sha512" append="yes">${spaces}apache-flume-spring-boot-${project.version}-src.zip</concat>
+                <concat destfile="${project.build.directory}/apache-flume-spring-boot-${project.version}-src.tar.gz.sha512" append="yes">${spaces}apache-flume-spring-boot-${project.version}-src.tar.gz</concat>
+                <concat destfile="${project.build.directory}/apache-flume-spring-boot-${project.version}-bin.zip.sha512" append="yes">${spaces}apache-flume-spring-boot-${project.version}-bin.zip</concat>
+                <concat destfile="${project.build.directory}/apache-flume-spring-boot-${project.version}-bin.tar.gz.sha512" append="yes">${spaces}apache-flume-spring-boot-${project.version}-bin.tar.gz</concat>
+              </target>
+            </configuration>
+            <goals>
+              <goal>run</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <artifactId>maven-assembly-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>source-release-assembly</id>
+            <phase>package</phase>
+            <goals>
+              <goal>single</goal>
+            </goals>
+            <configuration>
+              <finalName>apache-flume-spring-boot-${project.version}</finalName>
+              <descriptors>
+                <descriptor>src/assembly/src.xml</descriptor>
+              </descriptors>
+              <tarLongFileMode>gnu</tarLongFileMode>
+            </configuration>
+          </execution>
+          <execution>
+            <id>binary</id>
+            <configuration>
+              <finalName>apache-flume-spring-boot-${project.version}</finalName>
+              <descriptors>
+                <descriptor>src/assembly/bin.xml</descriptor>
+              </descriptors>
+              <tarLongFileMode>gnu</tarLongFileMode>
+            </configuration>
+            <goals>
+              <goal>single</goal>
+            </goals>
+            <phase>package</phase>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-gpg-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>sign-release-artifacts</id>
+            <goals>
+              <goal>sign</goal>
+            </goals>
+            <configuration>
+              <keyname>${SigningUserName}</keyname>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+
+</project>
diff --git a/flume-kafka-dist/src/assembly/bin.xml b/flume-kafka-dist/src/assembly/bin.xml
new file mode 100644
index 0000000..40af8b6
--- /dev/null
+++ b/flume-kafka-dist/src/assembly/bin.xml
@@ -0,0 +1,53 @@
+<!--
+    Licensed to the Apache Software Foundation (ASF) under one or more
+    contributor license agreements.  See the NOTICE file distributed with
+    this work for additional information regarding copyright ownership.
+    The ASF licenses this file to You under the Apache License, Version 2.0
+    (the "License"); you may not use this file except in compliance with
+    the License.  You may obtain a copy of the License at
+
+http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+-->
+<assembly>
+    <id>bin</id>
+    <formats>
+        <format>tar.gz</format>
+        <format>zip</format>
+    </formats>
+    <baseDirectory>apache-flume-kafka-${project.version}-bin</baseDirectory>
+    <includeSiteDirectory>false</includeSiteDirectory>
+  <moduleSets>
+    <moduleSet>
+      <useAllReactorProjects>true</useAllReactorProjects>
+    </moduleSet>
+  </moduleSets>
+  <dependencySets>
+    <dependencySet>
+      <includes>
+        <include>org.apache.flume:flume-shared-kafka</include>
+        <include>org.apache.flume:flume-kafka-sink</include>
+        <include>org.apache.flume:flume-kafka-source</include>
+        <include>org.apache.flume:flume-kafka-channel</include>
+      </includes>
+      <outputDirectory></outputDirectory>
+      <unpack>false</unpack>
+    </dependencySet>
+  </dependencySets>
+
+    <fileSets>
+        <fileSet>
+            <directory>../</directory>
+            <includes>
+                <include>LICENSE.txt</include>
+                <include>NOTICE.txt</include>
+                <include>RELEASE-NOTES.txt</include>
+            </includes>
+        </fileSet>
+    </fileSets>
+</assembly>
diff --git a/flume-kafka-dist/src/assembly/src.xml b/flume-kafka-dist/src/assembly/src.xml
new file mode 100644
index 0000000..fb768c5
--- /dev/null
+++ b/flume-kafka-dist/src/assembly/src.xml
@@ -0,0 +1,45 @@
+<?xml version='1.0' encoding='UTF-8'?>
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing,
+  software distributed under the License is distributed on an
+  "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+  KIND, either express or implied.  See the License for the
+  specific language governing permissions and limitations
+  under the License.
+-->
+
+<assembly>
+  <id>src</id>
+  <formats>
+    <format>zip</format>
+    <format>tar.gz</format>
+  </formats>
+  <baseDirectory>apache-flume-spring-boot-${project.version}-src</baseDirectory>
+  <fileSets>
+    <fileSet>
+      <directory>../</directory>
+
+      <excludes>
+        <exclude>**/target/**</exclude>
+        <exclude>**/.classpath</exclude>
+        <exclude>**/.project</exclude>
+        <exclude>**/.idea/**</exclude>
+        <exclude>**/*.iml</exclude>
+        <exclude>**/.settings/**</exclude>
+        <exclude>lib/**</exclude>
+        <exclude>**/.DS_Store</exclude>
+        <exclude>./mvn/wrapper/maven-wrapper.jar</exclude>
+      </excludes>
+    </fileSet>
+  </fileSets>
+</assembly>
diff --git a/flume-kafka-sink/pom.xml b/flume-kafka-sink/pom.xml
new file mode 100644
index 0000000..332dc08
--- /dev/null
+++ b/flume-kafka-sink/pom.xml
@@ -0,0 +1,134 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!-- Licensed to the Apache Software Foundation (ASF) under one or more contributor
+  license agreements. See the NOTICE file distributed with this work for additional
+  information regarding copyright ownership. The ASF licenses this file to
+  You under the Apache License, Version 2.0 (the "License"); you may not use
+  this file except in compliance with the License. You may obtain a copy of
+  the License at http://www.apache.org/licenses/LICENSE-2.0 Unless required
+  by applicable law or agreed to in writing, software distributed under the
+  License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS
+  OF ANY KIND, either express or implied. See the License for the specific
+  language governing permissions and limitations under the License. -->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <artifactId>flume-kafka-parent</artifactId>
+    <groupId>org.apache.flume</groupId>
+    <version>2.0.0-SNAPSHOT</version>
+  </parent>
+  <groupId>org.apache.flume</groupId>
+  <artifactId>flume-kafka-sink</artifactId>
+  <name>Flume Kafka Sink</name>
+
+  <properties>
+    <!-- TODO fix spotbugs/pmd violations -->
+    <spotbugs.maxAllowedViolations>3</spotbugs.maxAllowedViolations>
+    <pmd.maxAllowedViolations>3</pmd.maxAllowedViolations>
+    <module.name>org.apache.flume.sink.kafka</module.name>
+  </properties>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+        <executions>
+          <execution>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.flume</groupId>
+      <artifactId>flume-ng-sdk</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.flume</groupId>
+      <artifactId>flume-ng-core</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.flume</groupId>
+      <artifactId>flume-ng-configuration</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.flume</groupId>
+      <artifactId>flume-shared-kafka</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>io.dropwizard.metrics</groupId>
+      <artifactId>metrics-core</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.flume</groupId>
+      <artifactId>flume-shared-kafka-test</artifactId>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-api</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.kafka</groupId>
+      <artifactId>kafka_${scala.version}</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.kafka</groupId>
+      <artifactId>kafka_${scala.version}</artifactId>
+      <classifier>test</classifier>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.kafka</groupId>
+      <artifactId>kafka-clients</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.kafka</groupId>
+      <artifactId>kafka-clients</artifactId>
+      <scope>test</scope>
+      <classifier>test</classifier>
+      <version>${kafka.version}</version>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.logging.log4j</groupId>
+      <artifactId>log4j-slf4j-impl</artifactId>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.logging.log4j</groupId>
+      <artifactId>log4j-1.2-api</artifactId>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-all</artifactId>
+      <scope>test</scope>
+    </dependency>
+
+  </dependencies>
+
+</project>
diff --git a/flume-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSink.java b/flume-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSink.java
new file mode 100644
index 0000000..c090f37
--- /dev/null
+++ b/flume-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSink.java
@@ -0,0 +1,490 @@
+/**
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements.  See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License.  You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+ limitations under the License.
+ */
+
+package org.apache.flume.sink.kafka;
+
+import com.google.common.base.Optional;
+import com.google.common.base.Throwables;
+import org.apache.avro.io.BinaryEncoder;
+import org.apache.avro.io.EncoderFactory;
+import org.apache.avro.specific.SpecificDatumWriter;
+import org.apache.flume.Channel;
+import org.apache.flume.Context;
+import org.apache.flume.Event;
+import org.apache.flume.EventDeliveryException;
+import org.apache.flume.Transaction;
+import org.apache.flume.conf.BatchSizeSupported;
+import org.apache.flume.conf.Configurable;
+import org.apache.flume.conf.ConfigurationException;
+import org.apache.flume.conf.LogPrivacyUtil;
+import org.apache.flume.formatter.output.BucketPath;
+import org.apache.flume.instrumentation.kafka.KafkaSinkCounter;
+import org.apache.flume.shared.kafka.KafkaSSLUtil;
+import org.apache.flume.sink.AbstractSink;
+import org.apache.flume.source.avro.AvroFlumeEvent;
+import org.apache.kafka.clients.producer.Callback;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.clients.producer.RecordMetadata;
+import org.apache.kafka.common.config.SslConfigs;
+import org.apache.kafka.common.errors.ProducerFencedException;
+import org.apache.kafka.common.header.Header;
+import org.apache.kafka.common.header.internals.RecordHeader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.concurrent.Future;
+
+import static org.apache.flume.shared.kafka.KafkaSSLUtil.SSL_DISABLE_FQDN_CHECK;
+import static org.apache.flume.shared.kafka.KafkaSSLUtil.isSSLEnabled;
+
+/**
+ * A Flume Sink that can publish messages to Kafka.
+ * This is a general implementation that can be used with any Flume agent and
+ * a channel.
+ * The message can be any event and the key is a string that we read from the
+ * header
+ * For use of partitioning, use an interceptor to generate a header with the
+ * partition key
+ * <p/>
+ * Mandatory properties are:
+ * brokerList -- can be a partial list, but at least 2 are recommended for HA
+ * <p/>
+ * <p/>
+ * however, any property starting with "kafka." will be passed along to the
+ * Kafka producer
+ * Read the Kafka producer documentation to see which configurations can be used
+ * <p/>
+ * Optional properties
+ * topic - there's a default, and also - this can be in the event header if
+ * you need to support events with
+ * different topics
+ * batchSize - how many messages to process in one batch. Larger batches
+ * improve throughput while adding latency.
+ * requiredAcks -- 0 (unsafe), 1 (accepted by at least one broker, default),
+ * -1 (accepted by all brokers)
+ * useFlumeEventFormat - preserves event headers when serializing onto Kafka
+ * <p/>
+ * header properties (per event):
+ * topic
+ * key
+ */
+public class KafkaSink extends AbstractSink implements Configurable, BatchSizeSupported {
+
+  private static final Logger logger = LoggerFactory.getLogger(KafkaSink.class);
+
+  private final Properties kafkaProps = new Properties();
+  private KafkaProducer<String, byte[]> producer;
+
+  private String topic;
+  private int batchSize;
+  private List<Future<RecordMetadata>> kafkaFutures;
+  private KafkaSinkCounter counter;
+  private boolean useAvroEventFormat;
+  private String partitionHeader = null;
+  private Integer staticPartitionId = null;
+  private boolean allowTopicOverride;
+  private String topicHeader = null;
+  private String timestampHeader = null;
+  private Map<String, String> headerMap;
+
+  private boolean useKafkaTransactions = false;
+
+  private Optional<SpecificDatumWriter<AvroFlumeEvent>> writer = Optional.absent();
+  private Optional<ByteArrayOutputStream> tempOutStream = Optional.absent();
+
+  //Fine to use null for initial value, Avro will create new ones if this
+  // is null
+  private BinaryEncoder encoder = null;
+
+
+  //For testing
+  public String getTopic() {
+    return topic;
+  }
+
+  public long getBatchSize() {
+    return batchSize;
+  }
+
+  @Override
+  public Status process() throws EventDeliveryException {
+    Status result = Status.READY;
+    Channel channel = getChannel();
+    Transaction transaction = null;
+    Event event = null;
+    String eventTopic = null;
+    String eventKey = null;
+
+    try {
+      long processedEvents = 0;
+
+      transaction = channel.getTransaction();
+      transaction.begin();
+      if (useKafkaTransactions) {
+        producer.beginTransaction();
+      }
+
+      kafkaFutures.clear();
+      long batchStartTime = System.nanoTime();
+      for (; processedEvents < batchSize; processedEvents += 1) {
+        event = channel.take();
+
+        if (event == null) {
+          // no events available in channel
+          if (processedEvents == 0) {
+            result = Status.BACKOFF;
+            counter.incrementBatchEmptyCount();
+          } else {
+            counter.incrementBatchUnderflowCount();
+          }
+          break;
+        }
+        counter.incrementEventDrainAttemptCount();
+
+        byte[] eventBody = event.getBody();
+        Map<String, String> headers = event.getHeaders();
+
+        if (allowTopicOverride) {
+          eventTopic = headers.get(topicHeader);
+          if (eventTopic == null) {
+            eventTopic = BucketPath.escapeString(topic, event.getHeaders());
+            logger.debug("{} was set to true but header {} was null. Producing to {}" +
+                " topic instead.",
+                new Object[]{KafkaSinkConstants.ALLOW_TOPIC_OVERRIDE_HEADER,
+                    topicHeader, eventTopic});
+          }
+        } else {
+          eventTopic = topic;
+        }
+
+        eventKey = headers.get(KafkaSinkConstants.KEY_HEADER);
+        if (logger.isTraceEnabled()) {
+          if (LogPrivacyUtil.allowLogRawData()) {
+            logger.trace("{Event} " + eventTopic + " : " + eventKey + " : "
+                + new String(eventBody, StandardCharsets.UTF_8));
+          } else {
+            logger.trace("{Event} " + eventTopic + " : " + eventKey);
+          }
+        }
+        logger.debug("event #{}", processedEvents);
+
+        // create a message and add to buffer
+        long startTime = System.currentTimeMillis();
+
+        Integer partitionId = null;
+        try {
+          ProducerRecord<String, byte[]> record;
+          if (staticPartitionId != null) {
+            partitionId = staticPartitionId;
+          }
+          //Allow a specified header to override a static ID
+          if (partitionHeader != null) {
+            String headerVal = event.getHeaders().get(partitionHeader);
+            if (headerVal != null) {
+              partitionId = Integer.parseInt(headerVal);
+            }
+          }
+          Long timestamp = null;
+          if (timestampHeader != null) {
+            String value = headers.get(timestampHeader);
+            if (value != null) {
+              try {
+                timestamp = Long.parseLong(value);
+              } catch (Exception ex) {
+                logger.warn("Invalid timestamp in header {} - {}", timestampHeader, value);
+              }
+            }
+          }
+          List<Header> kafkaHeaders = null;
+          if (!headerMap.isEmpty()) {
+            List<Header> tempHeaders = new ArrayList<>();
+            for (Map.Entry<String, String> entry : headerMap.entrySet()) {
+              String value = headers.get(entry.getKey());
+              if (value != null) {
+                tempHeaders.add(new RecordHeader(entry.getValue(),
+                    value.getBytes(StandardCharsets.UTF_8)));
+              }
+            }
+            if (!tempHeaders.isEmpty()) {
+              kafkaHeaders = tempHeaders;
+            }
+          }
+
+          if (partitionId != null) {
+            record = new ProducerRecord<>(eventTopic, partitionId, timestamp, eventKey,
+                serializeEvent(event, useAvroEventFormat), kafkaHeaders);
+          } else {
+            record = new ProducerRecord<>(eventTopic, null, timestamp, eventKey,
+                serializeEvent(event, useAvroEventFormat), kafkaHeaders);
+          }
+          kafkaFutures.add(producer.send(record, new SinkCallback(startTime)));
+        } catch (NumberFormatException ex) {
+          throw new EventDeliveryException("Non integer partition id specified", ex);
+        } catch (Exception ex) {
+          // N.B. The producer.send() method throws all sorts of RuntimeExceptions
+          // Catching Exception here to wrap them neatly in an EventDeliveryException
+          // which is what our consumers will expect
+          throw new EventDeliveryException("Could not send event", ex);
+        }
+      }
+
+      if (useKafkaTransactions) {
+        producer.commitTransaction();
+      } else {
+        //Prevent linger.ms from holding the batch
+        producer.flush();
+        for (Future<RecordMetadata> future : kafkaFutures) {
+          future.get();
+        }
+      }
+      // publish batch and commit.
+      if (processedEvents > 0) {
+        long endTime = System.nanoTime();
+        counter.addToKafkaEventSendTimer((endTime - batchStartTime) / (1000 * 1000));
+        counter.addToEventDrainSuccessCount(processedEvents);
+      }
+
+      transaction.commit();
+
+    } catch (Exception ex) {
+      String errorMsg = "Failed to publish events";
+      logger.error("Failed to publish events", ex);
+      counter.incrementEventWriteOrChannelFail(ex);
+      if (transaction != null) {
+        try {
+          kafkaFutures.clear();
+          try {
+            if (useKafkaTransactions) {
+              producer.abortTransaction();
+            }
+          } catch (ProducerFencedException e) {
+            logger.error("Could not rollback transaction as producer fenced", e);
+          } finally {
+            transaction.rollback();
+            counter.incrementRollbackCount();
+          }
+        } catch (Exception e) {
+          logger.error("Transaction rollback failed", e);
+          throw Throwables.propagate(e);
+        }
+      }
+      throw new EventDeliveryException(errorMsg, ex);
+    } finally {
+      if (transaction != null) {
+        transaction.close();
+      }
+    }
+
+    return result;
+  }
+
+  @Override
+  public synchronized void start() {
+    // instantiate the producer
+    producer = new KafkaProducer<>(kafkaProps);
+    if (useKafkaTransactions) {
+      logger.info("Transactions enabled, initializing transactions");
+      producer.initTransactions();
+    }
+    counter.start();
+    super.start();
+  }
+
+  @Override
+  public synchronized void stop() {
+    producer.close();
+    counter.stop();
+    logger.info("Kafka Sink {} stopped. Metrics: {}", getName(), counter);
+    super.stop();
+  }
+
+
+  /**
+   * We configure the sink and generate properties for the Kafka Producer
+   *
+   * Kafka producer properties is generated as follows:
+   * 1. We generate a properties object with some static defaults that
+   * can be overridden by Sink configuration
+   * 2. We add the configuration users added for Kafka (parameters starting
+   * with .kafka. and must be valid Kafka Producer properties
+   * 3. We add the sink's documented parameters which can override other
+   * properties
+   *
+   * @param context The Context.
+   */
+  @Override
+  public void configure(Context context) {
+
+    String topicStr = context.getString(KafkaSinkConstants.TOPIC_CONFIG);
+    if (topicStr == null || topicStr.isEmpty()) {
+      topicStr = KafkaSinkConstants.DEFAULT_TOPIC;
+      logger.warn("Topic was not specified. Using {} as the topic.", topicStr);
+    } else {
+      logger.info("Using the static topic {}. This may be overridden by event headers", topicStr);
+    }
+
+    topic = topicStr;
+
+    timestampHeader = context.getString(KafkaSinkConstants.TIMESTAMP_HEADER);
+
+    headerMap = context.getSubProperties(KafkaSinkConstants.KAFKA_HEADER);
+
+    batchSize = context.getInteger(KafkaSinkConstants.BATCH_SIZE, KafkaSinkConstants.DEFAULT_BATCH_SIZE);
+
+    if (logger.isDebugEnabled()) {
+      logger.debug("Using batch size: {}", batchSize);
+    }
+
+    useAvroEventFormat = context.getBoolean(KafkaSinkConstants.AVRO_EVENT,
+                                            KafkaSinkConstants.DEFAULT_AVRO_EVENT);
+
+    partitionHeader = context.getString(KafkaSinkConstants.PARTITION_HEADER_NAME);
+    staticPartitionId = context.getInteger(KafkaSinkConstants.STATIC_PARTITION_CONF);
+
+    allowTopicOverride = context.getBoolean(KafkaSinkConstants.ALLOW_TOPIC_OVERRIDE_HEADER,
+                                          KafkaSinkConstants.DEFAULT_ALLOW_TOPIC_OVERRIDE_HEADER);
+
+    topicHeader = context.getString(KafkaSinkConstants.TOPIC_OVERRIDE_HEADER,
+                                    KafkaSinkConstants.DEFAULT_TOPIC_OVERRIDE_HEADER);
+
+    String transactionalID = context.getString(KafkaSinkConstants.TRANSACTIONAL_ID);
+    if (transactionalID != null) {
+      try {
+        context.put(KafkaSinkConstants.TRANSACTIONAL_ID, InetAddress.getLocalHost().getCanonicalHostName() +
+                Thread.currentThread().getName() + transactionalID);
+        useKafkaTransactions = true;
+      } catch (UnknownHostException e) {
+        throw new ConfigurationException("Unable to configure transactional id, as cannot work out hostname", e);
+      }
+    }
+
+    if (logger.isDebugEnabled()) {
+      logger.debug(KafkaSinkConstants.AVRO_EVENT + " set to: {}", useAvroEventFormat);
+    }
+
+    kafkaFutures = new LinkedList<Future<RecordMetadata>>();
+
+    String bootStrapServers = context.getString(KafkaSinkConstants.BOOTSTRAP_SERVERS_CONFIG);
+    if (bootStrapServers == null || bootStrapServers.isEmpty()) {
+      throw new ConfigurationException("Bootstrap Servers must be specified");
+    }
+
+    setProducerProps(context, bootStrapServers);
+
+    if (logger.isDebugEnabled() && LogPrivacyUtil.allowLogPrintConfig()) {
+      logger.debug("Kafka producer properties: {}", kafkaProps);
+    }
+
+    if (counter == null) {
+      counter = new KafkaSinkCounter(getName());
+    }
+  }
+
+  private void setProducerProps(Context context, String bootStrapServers) {
+    kafkaProps.clear();
+    kafkaProps.put(ProducerConfig.ACKS_CONFIG, KafkaSinkConstants.DEFAULT_ACKS);
+    //Defaults overridden based on config
+    kafkaProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, KafkaSinkConstants.DEFAULT_KEY_SERIALIZER);
+    kafkaProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, KafkaSinkConstants.DEFAULT_VALUE_SERIAIZER);
+    kafkaProps.putAll(context.getSubProperties(KafkaSinkConstants.KAFKA_PRODUCER_PREFIX));
+    kafkaProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, bootStrapServers);
+    //  The default value of `ssl.endpoint.identification.algorithm`
+    //  is changed to `https`, since kafka client 2.0+
+    //  And because flume does not accept an empty string as property value,
+    //  so we need to use an alternative custom property
+    //  `ssl.disableTLSHostnameVerification` to check if enable fqdn check.
+    if (isSSLEnabled(kafkaProps) && "true".equalsIgnoreCase(kafkaProps.getProperty(SSL_DISABLE_FQDN_CHECK))) {
+      kafkaProps.put(SslConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG, "");
+    }
+    KafkaSSLUtil.addGlobalSSLParameters(kafkaProps);
+  }
+
+  protected Properties getKafkaProps() {
+    return kafkaProps;
+  }
+
+  private byte[] serializeEvent(Event event, boolean useAvroEventFormat) throws IOException {
+    byte[] bytes;
+    if (useAvroEventFormat) {
+      if (!tempOutStream.isPresent()) {
+        tempOutStream = Optional.of(new ByteArrayOutputStream());
+      }
+      if (!writer.isPresent()) {
+        writer = Optional.of(new SpecificDatumWriter<AvroFlumeEvent>(AvroFlumeEvent.class));
+      }
+      tempOutStream.get().reset();
+      AvroFlumeEvent e = new AvroFlumeEvent(toCharSeqMap(event.getHeaders()),
+                                            ByteBuffer.wrap(event.getBody()));
+      encoder = EncoderFactory.get().directBinaryEncoder(tempOutStream.get(), encoder);
+      writer.get().write(e, encoder);
+      encoder.flush();
+      bytes = tempOutStream.get().toByteArray();
+    } else {
+      bytes = event.getBody();
+    }
+    return bytes;
+  }
+
+  private static Map<CharSequence, CharSequence> toCharSeqMap(Map<String, String> stringMap) {
+    Map<CharSequence, CharSequence> charSeqMap = new HashMap<CharSequence, CharSequence>();
+    for (Map.Entry<String, String> entry : stringMap.entrySet()) {
+      charSeqMap.put(entry.getKey(), entry.getValue());
+    }
+    return charSeqMap;
+  }
+
+}
+
+class SinkCallback implements Callback {
+  private static final Logger logger = LoggerFactory.getLogger(SinkCallback.class);
+  private long startTime;
+
+  public SinkCallback(long startTime) {
+    this.startTime = startTime;
+  }
+
+  public void onCompletion(RecordMetadata metadata, Exception exception) {
+    if (exception != null) {
+      logger.warn("Error sending message to Kafka {} ", exception.getMessage());
+    }
+
+    if (logger.isDebugEnabled()) {
+      long eventElapsedTime = System.currentTimeMillis() - startTime;
+      if (metadata != null) {
+        logger.debug("Acked message partition:{} ofset:{}", metadata.partition(),
+                metadata.offset());
+      }
+      logger.debug("Elapsed time for send: {}", eventElapsedTime);
+    }
+  }
+}
+
diff --git a/flume-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSinkConstants.java b/flume-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSinkConstants.java
new file mode 100644
index 0000000..7e1bd23
--- /dev/null
+++ b/flume-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSinkConstants.java
@@ -0,0 +1,62 @@
+/**
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements.  See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License.  You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+ limitations under the License.
+ */
+
+package org.apache.flume.sink.kafka;
+
+import org.apache.kafka.clients.CommonClientConfigs;
+import org.apache.kafka.clients.producer.ProducerConfig;
+
+public class KafkaSinkConstants {
+
+  public static final String KAFKA_PREFIX = "kafka.";
+  public static final String KAFKA_PRODUCER_PREFIX = KAFKA_PREFIX + "producer.";
+
+  /* Properties */
+
+  public static final String TOPIC_CONFIG = KAFKA_PREFIX + "topic";
+  public static final String BATCH_SIZE = "flumeBatchSize";
+  public static final String BOOTSTRAP_SERVERS_CONFIG =
+      KAFKA_PREFIX + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG;
+
+  public static final String TRANSACTIONAL_ID = KAFKA_PREFIX + "producer." +  ProducerConfig.TRANSACTIONAL_ID_CONFIG;
+
+  public static final String KEY_HEADER = "key";
+  public static final String DEFAULT_TOPIC_OVERRIDE_HEADER = "topic";
+  public static final String TOPIC_OVERRIDE_HEADER = "topicHeader";
+  public static final String TIMESTAMP_HEADER = "timestampHeader";
+  public static final String ALLOW_TOPIC_OVERRIDE_HEADER = "allowTopicOverride";
+  public static final boolean DEFAULT_ALLOW_TOPIC_OVERRIDE_HEADER = true;
+  public static final String KAFKA_HEADER = "header.";
+
+  public static final String AVRO_EVENT = "useFlumeEventFormat";
+  public static final boolean DEFAULT_AVRO_EVENT = false;
+
+  public static final String PARTITION_HEADER_NAME = "partitionIdHeader";
+  public static final String STATIC_PARTITION_CONF = "defaultPartitionId";
+
+  public static final String DEFAULT_KEY_SERIALIZER =
+      "org.apache.kafka.common.serialization.StringSerializer";
+  public static final String DEFAULT_VALUE_SERIAIZER =
+      "org.apache.kafka.common.serialization.ByteArraySerializer";
+
+  public static final int DEFAULT_BATCH_SIZE = 100;
+  public static final String DEFAULT_TOPIC = "default-flume-topic";
+  public static final String DEFAULT_ACKS = "1";
+
+}
+
diff --git a/flume-kafka-sink/src/test/java/org/apache/flume/sink/kafka/TestConstants.java b/flume-kafka-sink/src/test/java/org/apache/flume/sink/kafka/TestConstants.java
new file mode 100644
index 0000000..165601d
--- /dev/null
+++ b/flume-kafka-sink/src/test/java/org/apache/flume/sink/kafka/TestConstants.java
@@ -0,0 +1,31 @@
+/**
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements.  See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License.  You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+ limitations under the License.
+ */
+
+package org.apache.flume.sink.kafka;
+
+public class TestConstants {
+  public static final String STATIC_TOPIC = "static-topic";
+  public static final String HEADER_TOPIC = "%{header1}-topic";
+  public static final String CUSTOM_KEY = "custom-key";
+  public static final String CUSTOM_TOPIC = "custom-topic";
+  public static final String TRANSACTIONS_TOPIC = "transactions-topic";
+  public static final String HEADER_1_VALUE = "test-avro-header";
+  public static final String HEADER_1_KEY = "header1";
+  public static final String KAFKA_HEADER_1 = "FLUME_CORRELATOR";
+  public static final String KAFKA_HEADER_2 = "FLUME_METHOD";
+}
diff --git a/flume-kafka-sink/src/test/java/org/apache/flume/sink/kafka/TestKafkaSink.java b/flume-kafka-sink/src/test/java/org/apache/flume/sink/kafka/TestKafkaSink.java
new file mode 100644
index 0000000..2c0f1b8
--- /dev/null
+++ b/flume-kafka-sink/src/test/java/org/apache/flume/sink/kafka/TestKafkaSink.java
@@ -0,0 +1,839 @@
+/**
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements.  See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License.  You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+ limitations under the License.
+ */
+
+package org.apache.flume.sink.kafka;
+
+import com.google.common.base.Charsets;
+import org.apache.avro.io.BinaryDecoder;
+import org.apache.avro.io.DecoderFactory;
+import org.apache.avro.specific.SpecificDatumReader;
+import org.apache.avro.util.Utf8;
+import org.apache.commons.lang.RandomStringUtils;
+import org.apache.flume.Channel;
+import org.apache.flume.Context;
+import org.apache.flume.Event;
+import org.apache.flume.EventDeliveryException;
+import org.apache.flume.Sink;
+import org.apache.flume.Transaction;
+import org.apache.flume.channel.MemoryChannel;
+import org.apache.flume.conf.Configurables;
+import org.apache.flume.event.EventBuilder;
+import org.apache.flume.instrumentation.SinkCounter;
+import org.apache.flume.shared.kafka.test.KafkaPartitionTestUtil;
+import org.apache.flume.shared.kafka.test.PartitionOption;
+import org.apache.flume.shared.kafka.test.PartitionTestScenario;
+import org.apache.flume.sink.kafka.util.TestUtil;
+import org.apache.flume.source.avro.AvroFlumeEvent;
+import org.apache.kafka.clients.admin.TransactionListing;
+import org.apache.kafka.clients.admin.TransactionState;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.common.header.Headers;
+import org.apache.kafka.common.header.internals.RecordHeader;
+import org.apache.kafka.common.header.internals.RecordHeaders;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.mockito.internal.util.reflection.Whitebox;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+
+import static org.apache.flume.shared.kafka.KafkaSSLUtil.SSL_DISABLE_FQDN_CHECK;
+import static org.apache.flume.sink.kafka.KafkaSinkConstants.*;
+import static org.apache.kafka.clients.CommonClientConfigs.SECURITY_PROTOCOL_CONFIG;
+import static org.apache.kafka.clients.producer.ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG;
+import static org.apache.kafka.common.config.SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG;
+import static org.apache.kafka.common.config.SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * Unit tests for Kafka Sink
+ */
+public class TestKafkaSink {
+
+  private static final TestUtil testUtil = TestUtil.getInstance();
+  private static List<String> topicsList;
+  private final Set<String> usedTopics = new HashSet<String>();
+
+  @BeforeClass
+  public static void setup() {
+    testUtil.prepare();
+    topicsList = new ArrayList<String>(3);
+    topicsList.add(DEFAULT_TOPIC);
+    topicsList.add(TestConstants.STATIC_TOPIC);
+    topicsList.add(TestConstants.CUSTOM_TOPIC);
+    topicsList.add(TestConstants.TRANSACTIONS_TOPIC);
+    topicsList.add(TestConstants.HEADER_1_VALUE + "-topic");
+    testUtil.initTopicList(topicsList);
+  }
+
+  @AfterClass
+  public static void tearDown() {
+    testUtil.deleteTopics(topicsList);
+    testUtil.tearDown();
+  }
+
+  @Test
+  public void testKafkaProperties() {
+
+    KafkaSink kafkaSink = new KafkaSink();
+    Context context = new Context();
+    context.put(KAFKA_PREFIX + TOPIC_CONFIG, "");
+    context.put(KAFKA_PRODUCER_PREFIX + ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG,
+                "override.default.serializer");
+    context.put("kafka.producer.fake.property", "kafka.property.value");
+    context.put("kafka.bootstrap.servers", "localhost:9092,localhost:9092");
+    context.put("brokerList", "real-broker-list");
+    Configurables.configure(kafkaSink, context);
+
+    Properties kafkaProps = kafkaSink.getKafkaProps();
+
+    //check that we have defaults set
+    assertEquals(kafkaProps.getProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG),
+                 DEFAULT_KEY_SERIALIZER);
+    //check that kafka properties override the default and get correct name
+    assertEquals(kafkaProps.getProperty(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG),
+                 "override.default.serializer");
+    //check that any kafka-producer property gets in
+    assertEquals(kafkaProps.getProperty("fake.property"),
+                 "kafka.property.value");
+    //check that documented property overrides defaults
+    assertEquals(kafkaProps.getProperty("bootstrap.servers"),
+                 "localhost:9092,localhost:9092");
+  }
+
+  @Test
+  public void testDefaultTopic() {
+    Sink kafkaSink = new KafkaSink();
+    Context context = prepareDefaultContext();
+    Configurables.configure(kafkaSink, context);
+    Channel memoryChannel = new MemoryChannel();
+    Configurables.configure(memoryChannel, context);
+    kafkaSink.setChannel(memoryChannel);
+    kafkaSink.start();
+
+    String msg = "default-topic-test";
+    Transaction tx = memoryChannel.getTransaction();
+    tx.begin();
+    Event event = EventBuilder.withBody(msg.getBytes());
+    memoryChannel.put(event);
+    tx.commit();
+    tx.close();
+
+    try {
+      Sink.Status status = kafkaSink.process();
+      if (status == Sink.Status.BACKOFF) {
+        fail("Error Occurred");
+      }
+    } catch (EventDeliveryException ex) {
+      // ignore
+    }
+
+    checkMessageArrived(msg, DEFAULT_TOPIC);
+  }
+
+  private void checkMessageArrived(String msg, String topic) {
+    checkMessageArrived(msg, topic, null, null);
+  }
+
+  private void checkMessageArrived(String msg, String topic, Long timestamp, Headers headers) {
+    ConsumerRecords<String, String> recs = pollConsumerRecords(topic);
+    assertNotNull(recs);
+    assertTrue(recs.count() > 0);
+    Iterator<ConsumerRecord<String, String>> iter = recs.records(topic).iterator();
+    boolean match = false;
+    while (iter.hasNext()) {
+      ConsumerRecord<String, String> record = iter.next();
+      if (msg.equals(record.value()) && (timestamp == null || timestamp.equals(record.timestamp()))
+          && (headers == null || validateHeaders(headers, record.headers()))) {
+        match = true;
+        break;
+      }
+    }
+    assertTrue("No message matches " + msg, match);
+  }
+
+  private boolean validateHeaders(Headers expected, Headers actual) {
+    return expected.equals(actual);
+  }
+
+  @Test
+  public void testStaticTopic() {
+    Context context = prepareDefaultContext();
+    // add the static topic
+    context.put(TOPIC_CONFIG, TestConstants.STATIC_TOPIC);
+    String msg = "static-topic-test";
+
+    try {
+      Sink.Status status = prepareAndSend(context, msg);
+      if (status == Sink.Status.BACKOFF) {
+        fail("Error Occurred");
+      }
+    } catch (EventDeliveryException ex) {
+      // ignore
+    }
+
+    checkMessageArrived(msg, TestConstants.STATIC_TOPIC);
+  }
+
+  @Test
+  public void testTopicAndKeyFromHeader() {
+    Sink kafkaSink = new KafkaSink();
+    Context context = prepareDefaultContext();
+    Configurables.configure(kafkaSink, context);
+    Channel memoryChannel = new MemoryChannel();
+    Configurables.configure(memoryChannel, context);
+    kafkaSink.setChannel(memoryChannel);
+    kafkaSink.start();
+
+    String msg = "test-topic-and-key-from-header";
+    Map<String, String> headers = new HashMap<String, String>();
+    headers.put("topic", TestConstants.CUSTOM_TOPIC);
+    headers.put("key", TestConstants.CUSTOM_KEY);
+    Transaction tx = memoryChannel.getTransaction();
+    tx.begin();
+    Event event = EventBuilder.withBody(msg.getBytes(), headers);
+    memoryChannel.put(event);
+    tx.commit();
+    tx.close();
+
+    try {
+      Sink.Status status = kafkaSink.process();
+      if (status == Sink.Status.BACKOFF) {
+        fail("Error Occurred");
+      }
+    } catch (EventDeliveryException ex) {
+      // ignore
+    }
+
+    checkMessageArrived(msg, TestConstants.CUSTOM_TOPIC);
+  }
+
+  @Test
+  public void testTimestampAndHeaders() {
+    Sink kafkaSink = new KafkaSink();
+    Context context = prepareDefaultContext();
+    context.put(KafkaSinkConstants.TIMESTAMP_HEADER, "timestamp");
+    context.put("header.correlator", TestConstants.KAFKA_HEADER_1);
+    context.put("header.method", TestConstants.KAFKA_HEADER_2);
+
+    Configurables.configure(kafkaSink, context);
+    Channel memoryChannel = new MemoryChannel();
+    Configurables.configure(memoryChannel, context);
+    kafkaSink.setChannel(memoryChannel);
+    kafkaSink.start();
+
+    String msg = "test-topic-and-key-from-header";
+    Map<String, String> headers = new HashMap<String, String>();
+    long now = System.currentTimeMillis();
+    headers.put("timestamp", Long.toString(now));
+    headers.put("topic", TestConstants.CUSTOM_TOPIC);
+    headers.put("key", TestConstants.CUSTOM_KEY);
+    headers.put("correlator", "12345");
+    headers.put("method", "testTimestampAndHeaders");
+    Transaction tx = memoryChannel.getTransaction();
+    tx.begin();
+    Event event = EventBuilder.withBody(msg.getBytes(), headers);
+    memoryChannel.put(event);
+    tx.commit();
+    tx.close();
+
+    try {
+      Sink.Status status = kafkaSink.process();
+      if (status == Sink.Status.BACKOFF) {
+        fail("Error Occurred");
+      }
+    } catch (EventDeliveryException ex) {
+      fail(ex.getMessage());
+    }
+    Headers expected = new RecordHeaders();
+    expected.add(new RecordHeader(TestConstants.KAFKA_HEADER_1,
+        "12345".getBytes(StandardCharsets.UTF_8)));
+    expected.add(new RecordHeader(TestConstants.KAFKA_HEADER_2,
+        "testTimestampAndHeaders".getBytes(StandardCharsets.UTF_8)));
+    checkMessageArrived(msg, TestConstants.CUSTOM_TOPIC, now, expected);
+  }
+
+  /**
+   * Tests that a message will be produced to a topic as specified by a
+   * custom topicHeader parameter (FLUME-3046).
+   */
+  @Test
+  public void testTopicFromConfHeader() {
+    String customTopicHeader = "customTopicHeader";
+    Sink kafkaSink = new KafkaSink();
+    Context context = prepareDefaultContext();
+    context.put(KafkaSinkConstants.TOPIC_OVERRIDE_HEADER, customTopicHeader);
+    Configurables.configure(kafkaSink, context);
+    Channel memoryChannel = new MemoryChannel();
+    Configurables.configure(memoryChannel, context);
+    kafkaSink.setChannel(memoryChannel);
+    kafkaSink.start();
+
+    String msg = "test-topic-from-config-header";
+    Map<String, String> headers = new HashMap<String, String>();
+    headers.put(customTopicHeader, TestConstants.CUSTOM_TOPIC);
+    Transaction tx = memoryChannel.getTransaction();
+    tx.begin();
+    Event event = EventBuilder.withBody(msg.getBytes(), headers);
+    memoryChannel.put(event);
+    tx.commit();
+    tx.close();
+
+    try {
+      Sink.Status status = kafkaSink.process();
+      if (status == Sink.Status.BACKOFF) {
+        fail("Error Occurred");
+      }
+    } catch (EventDeliveryException ex) {
+      // ignore
+    }
+
+    checkMessageArrived(msg, TestConstants.CUSTOM_TOPIC);
+  }
+
+  /**
+   * Tests that the topicHeader parameter will be ignored if the allowTopicHeader
+   * parameter is set to false (FLUME-3046).
+   */
+  @Test
+  public void testTopicNotFromConfHeader() {
+    Sink kafkaSink = new KafkaSink();
+    Context context = prepareDefaultContext();
+    context.put(KafkaSinkConstants.ALLOW_TOPIC_OVERRIDE_HEADER, "false");
+    context.put(KafkaSinkConstants.TOPIC_OVERRIDE_HEADER, "foo");
+
+    Configurables.configure(kafkaSink, context);
+    Channel memoryChannel = new MemoryChannel();
+    Configurables.configure(memoryChannel, context);
+    kafkaSink.setChannel(memoryChannel);
+    kafkaSink.start();
+
+    String msg = "test-topic-from-config-header";
+    Map<String, String> headers = new HashMap<String, String>();
+    headers.put(KafkaSinkConstants.DEFAULT_TOPIC_OVERRIDE_HEADER, TestConstants.CUSTOM_TOPIC);
+    headers.put("foo", TestConstants.CUSTOM_TOPIC);
+    Transaction tx = memoryChannel.getTransaction();
+    tx.begin();
+    Event event = EventBuilder.withBody(msg.getBytes(), headers);
+    memoryChannel.put(event);
+    tx.commit();
+    tx.close();
+
+    try {
+      Sink.Status status = kafkaSink.process();
+      if (status == Sink.Status.BACKOFF) {
+        fail("Error Occurred");
+      }
+    } catch (EventDeliveryException ex) {
+      // ignore
+    }
+
+    checkMessageArrived(msg, DEFAULT_TOPIC);
+  }
+
+  @Test
+  public void testReplaceSubStringOfTopicWithHeaders() {
+    String topic = TestConstants.HEADER_1_VALUE + "-topic";
+    Sink kafkaSink = new KafkaSink();
+    Context context = prepareDefaultContext();
+    context.put(TOPIC_CONFIG, TestConstants.HEADER_TOPIC);
+    Configurables.configure(kafkaSink, context);
+    Channel memoryChannel = new MemoryChannel();
+    Configurables.configure(memoryChannel, context);
+    kafkaSink.setChannel(memoryChannel);
+    kafkaSink.start();
+
+    String msg = "test-replace-substring-of-topic-with-headers";
+    Map<String, String> headers = new HashMap<>();
+    headers.put(TestConstants.HEADER_1_KEY, TestConstants.HEADER_1_VALUE);
+    Transaction tx = memoryChannel.getTransaction();
+    tx.begin();
+    Event event = EventBuilder.withBody(msg.getBytes(), headers);
+    memoryChannel.put(event);
+    tx.commit();
+    tx.close();
+
+    try {
+      Sink.Status status = kafkaSink.process();
+      if (status == Sink.Status.BACKOFF) {
+        fail("Error Occurred");
+      }
+    } catch (EventDeliveryException ex) {
+      // ignore
+    }
+
+    checkMessageArrived(msg, topic);
+  }
+
+  @SuppressWarnings("rawtypes")
+  @Test
+  public void testAvroEvent() throws IOException, InterruptedException {
+    Sink kafkaSink = new KafkaSink();
+    Context context = prepareDefaultContext();
+    context.put(AVRO_EVENT, "true");
+    Configurables.configure(kafkaSink, context);
+    Channel memoryChannel = new MemoryChannel();
+    Configurables.configure(memoryChannel, context);
+    kafkaSink.setChannel(memoryChannel);
+    kafkaSink.start();
+
+    String msg = "test-avro-event";
+
+    Map<String, String> headers = new HashMap<String, String>();
+    headers.put("topic", TestConstants.CUSTOM_TOPIC);
+    headers.put("key", TestConstants.CUSTOM_KEY);
+    headers.put(TestConstants.HEADER_1_KEY, TestConstants.HEADER_1_VALUE);
+    Transaction tx = memoryChannel.getTransaction();
+    tx.begin();
+    Event event = EventBuilder.withBody(msg.getBytes(), headers);
+    memoryChannel.put(event);
+    tx.commit();
+    tx.close();
+
+    try {
+      Sink.Status status = kafkaSink.process();
+      if (status == Sink.Status.BACKOFF) {
+        fail("Error Occurred");
+      }
+    } catch (EventDeliveryException ex) {
+      // ignore
+    }
+
+    String topic = TestConstants.CUSTOM_TOPIC;
+
+    ConsumerRecords<String, String> recs = pollConsumerRecords(topic);
+    assertNotNull(recs);
+    assertTrue(recs.count() > 0);
+    ConsumerRecord<String, String> consumerRecord = recs.iterator().next();
+    ByteArrayInputStream in = new ByteArrayInputStream(consumerRecord.value().getBytes());
+    BinaryDecoder decoder = DecoderFactory.get().directBinaryDecoder(in, null);
+    SpecificDatumReader<AvroFlumeEvent> reader = new SpecificDatumReader<>(AvroFlumeEvent.class);
+
+    AvroFlumeEvent avroevent = reader.read(null, decoder);
+
+    String eventBody = new String(avroevent.getBody().array(), Charsets.UTF_8);
+    Map<CharSequence, CharSequence> eventHeaders = avroevent.getHeaders();
+
+    assertEquals(msg, eventBody);
+    assertEquals(TestConstants.CUSTOM_KEY, consumerRecord.key());
+
+    assertEquals(TestConstants.HEADER_1_VALUE,
+                 eventHeaders.get(new Utf8(TestConstants.HEADER_1_KEY)).toString());
+    assertEquals(TestConstants.CUSTOM_KEY, eventHeaders.get(new Utf8("key")).toString());
+  }
+
+  private ConsumerRecords<String, String> pollConsumerRecords(String topic) {
+    return pollConsumerRecords(topic, 20);
+  }
+
+  // Note that the topic parameter is completely ignored. If the consumer is subscribed to
+  // multiple topics records for all of them will be returned.
+  private ConsumerRecords<String, String> pollConsumerRecords(String topic, int maxIter) {
+    ConsumerRecords<String, String> recs = null;
+    for (int i = 0; i < maxIter; i++) {
+      recs = testUtil.getNextMessageFromConsumer(topic);
+      if (recs.count() > 0) break;
+      try {
+        Thread.sleep(1000L);
+      } catch (InterruptedException e) {
+        //
+      }
+    }
+    return recs;
+  }
+
+  @Test
+  public void testEmptyChannel() throws EventDeliveryException {
+    Sink kafkaSink = new KafkaSink();
+    Context context = prepareDefaultContext();
+    Configurables.configure(kafkaSink, context);
+    Channel memoryChannel = new MemoryChannel();
+    Configurables.configure(memoryChannel, context);
+    kafkaSink.setChannel(memoryChannel);
+    kafkaSink.start();
+
+    Sink.Status status = kafkaSink.process();
+    if (status != Sink.Status.BACKOFF) {
+      fail("Error Occurred");
+    }
+    ConsumerRecords recs = pollConsumerRecords(DEFAULT_TOPIC, 2);
+    assertNotNull(recs);
+    assertEquals(0, recs.count());
+  }
+
+  @Test
+  public void testPartitionHeaderSet() throws Exception {
+    doPartitionHeader(PartitionTestScenario.PARTITION_ID_HEADER_ONLY);
+  }
+
+  @Test
+  public void testPartitionHeaderNotSet() throws Exception {
+    doPartitionHeader(PartitionTestScenario.NO_PARTITION_HEADERS);
+  }
+
+  @Test
+  public void testStaticPartitionAndHeaderSet() throws Exception {
+    doPartitionHeader(PartitionTestScenario.STATIC_HEADER_AND_PARTITION_ID);
+  }
+
+  @Test
+  public void testStaticPartitionHeaderNotSet() throws Exception {
+    doPartitionHeader(PartitionTestScenario.STATIC_HEADER_ONLY);
+  }
+
+  @Test
+  public void testPartitionHeaderMissing() throws Exception {
+    doPartitionErrors(PartitionOption.NOTSET);
+  }
+
+  @Test
+  public void testPartitionHeaderOutOfRange() throws Exception {
+    Sink kafkaSink = new KafkaSink();
+    try {
+      doPartitionErrors(PartitionOption.VALIDBUTOUTOFRANGE, kafkaSink);
+      fail();
+    } catch (EventDeliveryException e) {
+      //
+    }
+    SinkCounter sinkCounter = (SinkCounter) Whitebox.getInternalState(kafkaSink, "counter");
+    assertEquals(1, sinkCounter.getEventWriteFail());
+  }
+
+  @Test(expected = org.apache.flume.EventDeliveryException.class)
+  public void testPartitionHeaderInvalid() throws Exception {
+    doPartitionErrors(PartitionOption.NOTANUMBER);
+  }
+
+  /**
+   * Tests that sub-properties (kafka.producer.*) apply correctly across multiple invocations
+   * of configure() (fix for FLUME-2857).
+   */
+  @Test
+  public void testDefaultSettingsOnReConfigure() {
+    String sampleProducerProp = "compression.type";
+    String sampleProducerVal = "snappy";
+
+    Context context = prepareDefaultContext();
+    context.put(KafkaSinkConstants.KAFKA_PRODUCER_PREFIX + sampleProducerProp, sampleProducerVal);
+
+    KafkaSink kafkaSink = new KafkaSink();
+
+    Configurables.configure(kafkaSink, context);
+
+    Assert.assertEquals(sampleProducerVal,
+        kafkaSink.getKafkaProps().getProperty(sampleProducerProp));
+
+    context = prepareDefaultContext();
+    Configurables.configure(kafkaSink, context);
+
+    Assert.assertNull(kafkaSink.getKafkaProps().getProperty(sampleProducerProp));
+
+  }
+
+  /**
+   * This function tests three scenarios:
+   * 1. PartitionOption.VALIDBUTOUTOFRANGE: An integer partition is provided,
+   *    however it exceeds the number of partitions available on the topic.
+   *    Expected behaviour: ChannelException thrown.
+   *
+   * 2. PartitionOption.NOTSET: The partition header is not actually set.
+   *    Expected behaviour: Exception is not thrown because the code avoids an NPE.
+   *
+   * 3. PartitionOption.NOTANUMBER: The partition header is set, but is not an Integer.
+   *    Expected behaviour: ChannelException thrown.
+   *
+   */
+  private void doPartitionErrors(PartitionOption option) throws Exception {
+    doPartitionErrors(option, new KafkaSink());
+  }
+
+  private void doPartitionErrors(PartitionOption option, Sink kafkaSink) throws Exception {
+    Context context = prepareDefaultContext();
+    context.put(KafkaSinkConstants.PARTITION_HEADER_NAME, "partition-header");
+
+    Configurables.configure(kafkaSink, context);
+    Channel memoryChannel = new MemoryChannel();
+    Configurables.configure(memoryChannel, context);
+    kafkaSink.setChannel(memoryChannel);
+    kafkaSink.start();
+
+    String topic = findUnusedTopic();
+    createTopic(topic, 5);
+
+    Transaction tx = memoryChannel.getTransaction();
+    tx.begin();
+
+    Map<String, String> headers = new HashMap<String, String>();
+    headers.put("topic", topic);
+    switch (option) {
+      case VALIDBUTOUTOFRANGE:
+        headers.put("partition-header", "9");
+        break;
+      case NOTSET:
+        headers.put("wrong-header", "2");
+        break;
+      case NOTANUMBER:
+        headers.put("partition-header", "not-a-number");
+        break;
+      default:
+        break;
+    }
+
+    Event event = EventBuilder.withBody(String.valueOf(9).getBytes(), headers);
+
+    memoryChannel.put(event);
+    tx.commit();
+    tx.close();
+
+    Sink.Status status = kafkaSink.process();
+    assertEquals(Sink.Status.READY, status);
+
+    deleteTopic(topic);
+
+  }
+
+  /**
+   * This method tests both the default behavior (usePartitionHeader=false)
+   * and the behaviour when the partitionId setting is used.
+   * Under the default behaviour, one would expect an even distribution of
+   * messages to partitions, however when partitionId is used we manually create
+   * a large skew to some partitions and then verify that this actually happened
+   * by reading messages directly using a Kafka Consumer.
+   *
+   */
+  private void doPartitionHeader(PartitionTestScenario scenario) throws Exception {
+    final int numPtns = 5;
+    final int numMsgs = numPtns * 10;
+    final Integer staticPtn = 3;
+
+    String topic = findUnusedTopic();
+    createTopic(topic, numPtns);
+    Context context = prepareDefaultContext();
+    context.put(BATCH_SIZE, "100");
+
+    if (scenario == PartitionTestScenario.PARTITION_ID_HEADER_ONLY ||
+        scenario == PartitionTestScenario.STATIC_HEADER_AND_PARTITION_ID) {
+      context.put(KafkaSinkConstants.PARTITION_HEADER_NAME,
+                  KafkaPartitionTestUtil.PARTITION_HEADER);
+    }
+    if (scenario == PartitionTestScenario.STATIC_HEADER_AND_PARTITION_ID ||
+        scenario == PartitionTestScenario.STATIC_HEADER_ONLY) {
+      context.put(KafkaSinkConstants.STATIC_PARTITION_CONF, staticPtn.toString());
+    }
+    Sink kafkaSink = new KafkaSink();
+
+    Configurables.configure(kafkaSink, context);
+    Channel memoryChannel = new MemoryChannel();
+    Configurables.configure(memoryChannel, context);
+    kafkaSink.setChannel(memoryChannel);
+    kafkaSink.start();
+
+    //Create a map of PartitionId:List<Messages> according to the desired distribution
+    Map<Integer, List<Event>> partitionMap = new HashMap<Integer, List<Event>>(numPtns);
+    for (int i = 0; i < numPtns; i++) {
+      partitionMap.put(i, new ArrayList<Event>());
+    }
+    Transaction tx = memoryChannel.getTransaction();
+    tx.begin();
+
+    List<Event> orderedEvents = KafkaPartitionTestUtil.generateSkewedMessageList(scenario, numMsgs,
+                                                                 partitionMap, numPtns, staticPtn);
+
+    for (Event event : orderedEvents) {
+      event.getHeaders().put("topic", topic);
+      memoryChannel.put(event);
+    }
+
+    tx.commit();
+    tx.close();
+
+    Sink.Status status = kafkaSink.process();
+    assertEquals(Sink.Status.READY, status);
+
+    Properties props = new Properties();
+    props.put("bootstrap.servers", testUtil.getKafkaServerUrl());
+    props.put("group.id", "group_1");
+    props.put("enable.auto.commit", "true");
+    props.put("auto.commit.interval.ms", "1000");
+    props.put("session.timeout.ms", "30000");
+    props.put("key.deserializer",
+        "org.apache.kafka.common.serialization.StringDeserializer");
+    props.put("value.deserializer",
+        "org.apache.kafka.common.serialization.ByteArrayDeserializer");
+    props.put("auto.offset.reset", "earliest");
+    Map<Integer, List<byte[]>> resultsMap =
+        KafkaPartitionTestUtil.retrieveRecordsFromPartitions(topic, numPtns, props);
+
+    KafkaPartitionTestUtil.checkResultsAgainstSkew(scenario, partitionMap, resultsMap, staticPtn,
+                                                   numMsgs);
+
+    memoryChannel.stop();
+    kafkaSink.stop();
+    deleteTopic(topic);
+
+  }
+
+  private Context prepareDefaultContext() {
+    // Prepares a default context with Kafka Server Properties
+    Context context = new Context();
+    context.put(BOOTSTRAP_SERVERS_CONFIG, testUtil.getKafkaServerUrl());
+    context.put(BATCH_SIZE, "1");
+    return context;
+  }
+
+  private Sink.Status prepareAndSend(Context context, String msg)
+      throws EventDeliveryException {
+    Sink kafkaSink = new KafkaSink();
+    Configurables.configure(kafkaSink, context);
+    Channel memoryChannel = new MemoryChannel();
+    Configurables.configure(memoryChannel, context);
+    kafkaSink.setChannel(memoryChannel);
+    kafkaSink.start();
+
+    Transaction tx = memoryChannel.getTransaction();
+    tx.begin();
+    Event event = EventBuilder.withBody(msg.getBytes());
+    memoryChannel.put(event);
+    tx.commit();
+    tx.close();
+
+    return kafkaSink.process();
+  }
+
+  private void createTopic(String topicName, int numPartitions) {
+    testUtil.createTopics(Collections.singletonList(topicName), numPartitions);
+  }
+
+  private void deleteTopic(String topicName) {
+    testUtil.deleteTopic(topicName);
+  }
+
+  private String findUnusedTopic() {
+    String newTopic = null;
+    boolean topicFound = false;
+    while (!topicFound) {
+      newTopic = RandomStringUtils.randomAlphabetic(8);
+      if (!usedTopics.contains(newTopic)) {
+        usedTopics.add(newTopic);
+        topicFound = true;
+      }
+    }
+    return newTopic;
+  }
+
+  @Test
+  public void testSslTopic() {
+    Sink kafkaSink = new KafkaSink();
+    Context context = prepareDefaultContext();
+    context.put(BOOTSTRAP_SERVERS_CONFIG, testUtil.getKafkaServerSslUrl());
+    context.put(KAFKA_PRODUCER_PREFIX + SECURITY_PROTOCOL_CONFIG, "SSL");
+    context.put(KAFKA_PRODUCER_PREFIX + SSL_TRUSTSTORE_LOCATION_CONFIG, "src/test/resources/truststorefile.jks");
+    context.put(KAFKA_PRODUCER_PREFIX + SSL_TRUSTSTORE_PASSWORD_CONFIG, "password");
+    context.put(KAFKA_PRODUCER_PREFIX + SSL_DISABLE_FQDN_CHECK, "true");
+    Configurables.configure(kafkaSink, context);
+
+    Channel memoryChannel = new MemoryChannel();
+    context = prepareDefaultContext();
+    Configurables.configure(memoryChannel, context);
+    kafkaSink.setChannel(memoryChannel);
+    kafkaSink.start();
+
+    String msg = "default-topic-test";
+    Transaction tx = memoryChannel.getTransaction();
+    tx.begin();
+    Event event = EventBuilder.withBody(msg.getBytes());
+    memoryChannel.put(event);
+    tx.commit();
+    tx.close();
+
+    try {
+      Sink.Status status = kafkaSink.process();
+      if (status == Sink.Status.BACKOFF) {
+        fail("Error Occurred");
+      }
+    } catch (EventDeliveryException ex) {
+      // ignore
+    }
+
+    checkMessageArrived(msg, DEFAULT_TOPIC);
+  }
+
+  @Test
+  public void testKafkaTransactions() {
+    Sink kafkaSink = new KafkaSink();
+    Context context = prepareDefaultContext();
+
+    context.put(TOPIC_CONFIG, TestConstants.TRANSACTIONS_TOPIC);
+
+    context.put(TRANSACTIONAL_ID, "3");
+    context.put("kafka.producer." + ENABLE_IDEMPOTENCE_CONFIG, "true");
+    context.put("kafka.producer.acks", "all");
+    Configurables.configure(kafkaSink, context);
+    Channel memoryChannel = new MemoryChannel();
+    Configurables.configure(memoryChannel, context);
+    kafkaSink.setChannel(memoryChannel);
+    kafkaSink.start();
+
+    for (int i = 0; i < 5; i++) {
+      String msg = "test tx message " + i;
+      Transaction tx = memoryChannel.getTransaction();
+      tx.begin();
+      Event event = EventBuilder.withBody(msg.getBytes());
+      memoryChannel.put(event);
+      tx.commit();
+      tx.close();
+
+      try {
+        Sink.Status status = kafkaSink.process();
+        if (status == Sink.Status.BACKOFF) {
+          fail("Error Occurred");
+        }
+      } catch (EventDeliveryException ex) {
+        // ignore
+      }
+      checkMessageArrived(msg, TestConstants.TRANSACTIONS_TOPIC);
+    }
+
+    Collection<TransactionListing> transactions = testUtil.getTransactionState();
+    Assert.assertEquals(1, transactions.size(), 2);
+    for (TransactionListing transaction : transactions) {
+      Assert.assertEquals(transaction.state(), TransactionState.COMPLETE_COMMIT);
+    }
+
+  }
+
+
+}
\ No newline at end of file
diff --git a/flume-kafka-sink/src/test/java/org/apache/flume/sink/kafka/util/KafkaLocal.java b/flume-kafka-sink/src/test/java/org/apache/flume/sink/kafka/util/KafkaLocal.java
new file mode 100644
index 0000000..700c943
--- /dev/null
+++ b/flume-kafka-sink/src/test/java/org/apache/flume/sink/kafka/util/KafkaLocal.java
@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ * limitations under the License.
+ */
+
+package org.apache.flume.sink.kafka.util;
+
+import kafka.server.KafkaConfig;
+import kafka.server.KafkaServer;
+import kafka.utils.TestUtils;
+import org.apache.kafka.common.utils.Time;
+
+import java.io.IOException;
+import java.util.Properties;
+
+/**
+ * A local Kafka server for running unit tests.
+ * Reference: https://gist.github.com/fjavieralba/7930018/
+ */
+public class KafkaLocal {
+
+  public KafkaServer kafka;
+  public ZooKeeperLocal zookeeper;
+
+  public KafkaLocal(Properties kafkaProperties) throws IOException, InterruptedException {
+    KafkaConfig kafkaConfig = KafkaConfig.fromProps(kafkaProperties);
+
+    // start local kafka broker
+    kafka = TestUtils.createServer(kafkaConfig, Time.SYSTEM);
+  }
+
+  public void start() throws Exception {
+    kafka.startup();
+  }
+
+  public void stop() {
+    kafka.shutdown();
+  }
+
+}
\ No newline at end of file
diff --git a/flume-kafka-sink/src/test/java/org/apache/flume/sink/kafka/util/TestUtil.java b/flume-kafka-sink/src/test/java/org/apache/flume/sink/kafka/util/TestUtil.java
new file mode 100644
index 0000000..2073b63
--- /dev/null
+++ b/flume-kafka-sink/src/test/java/org/apache/flume/sink/kafka/util/TestUtil.java
@@ -0,0 +1,294 @@
+/**
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements.  See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License.  You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+ limitations under the License.
+ */
+
+package org.apache.flume.sink.kafka.util;
+
+import org.apache.kafka.clients.admin.AdminClient;
+import org.apache.kafka.clients.admin.CreateTopicsResult;
+import org.apache.kafka.clients.admin.ListTransactionsResult;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.admin.TransactionListing;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.InetAddress;
+import java.net.ServerSocket;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Properties;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.kafka.common.config.SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG;
+import static org.apache.kafka.common.config.SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG;
+import static org.apache.kafka.common.config.SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG;
+import static org.apache.kafka.common.config.SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG;
+
+/**
+ * A utility class for starting/stopping Kafka Server.
+ */
+public class TestUtil {
+
+  private static final Logger logger = LoggerFactory.getLogger(TestUtil.class);
+  private static final TestUtil instance = new TestUtil();
+
+  private KafkaLocal kafkaServer;
+  private boolean externalServers = true;
+  private String kafkaServerUrl;
+  private String kafkaServerSslUrl;
+  private String zkServerUrl;
+  private int kafkaLocalPort;
+  private int kafkaLocalSslPort;
+  private Properties clientProps;
+  private int zkLocalPort;
+  private KafkaConsumer<String, String> consumer;
+  private AdminClient adminClient;
+
+  private TestUtil() {
+    init();
+  }
+
+  public static TestUtil getInstance() {
+    return instance;
+  }
+
+  private void init() {
+    try {
+      Properties settings = new Properties();
+      InputStream in = Class.class.getResourceAsStream("/testutil.properties");
+      if (in != null) {
+        settings.load(in);
+      }
+      externalServers = "true".equalsIgnoreCase(settings.getProperty("external-servers"));
+      if (externalServers) {
+        kafkaServerUrl = settings.getProperty("kafka-server-url");
+        zkServerUrl = settings.getProperty("zk-server-url");
+      } else {
+        String hostname = InetAddress.getLocalHost().getHostName();
+        zkLocalPort = getNextPort();
+        kafkaLocalPort = getNextPort();
+        kafkaLocalSslPort = getNextPort();
+        kafkaServerUrl = hostname + ":" + kafkaLocalPort;
+        kafkaServerSslUrl = hostname + ":" + kafkaLocalSslPort;
+        zkServerUrl = hostname + ":" + zkLocalPort;
+      }
+      clientProps = createClientProperties();
+    } catch (Exception e) {
+      logger.error("Unexpected error", e);
+      throw new RuntimeException("Unexpected error", e);
+    }
+  }
+
+  private boolean startEmbeddedKafkaServer() {
+    Properties kafkaProperties = new Properties();
+    Properties zkProperties = new Properties();
+
+    logger.info("Starting kafka server.");
+    try {
+      //load properties
+      zkProperties.load(Class.class.getResourceAsStream(
+          "/zookeeper.properties"));
+
+      //start local Zookeeper
+      // override the Zookeeper client port with the generated one.
+      zkProperties.setProperty("clientPort", Integer.toString(zkLocalPort));
+      new ZooKeeperLocal(zkProperties);
+
+      logger.info("ZooKeeper instance is successfully started on port " +
+          zkLocalPort);
+
+      kafkaProperties.load(Class.class.getResourceAsStream(
+          "/kafka-server.properties"));
+      // override the Zookeeper url.
+      kafkaProperties.setProperty("zookeeper.connect", getZkUrl());
+      //  to enable ssl feature,
+      //  we need to use listeners instead of using port property
+      //  kafkaProperties.setProperty("port", Integer.toString(kafkaLocalPort));
+      kafkaProperties.put("listeners",
+              String.format("PLAINTEXT://%s,SSL://%s",
+                      getKafkaServerUrl(),
+                      getKafkaServerSslUrl()
+              )
+      );
+      //  ssl configuration
+      kafkaProperties.put(SSL_TRUSTSTORE_LOCATION_CONFIG, "src/test/resources/truststorefile.jks");
+      kafkaProperties.put(SSL_TRUSTSTORE_PASSWORD_CONFIG, "password");
+      kafkaProperties.put(SSL_KEYSTORE_LOCATION_CONFIG, "src/test/resources/keystorefile.jks");
+      kafkaProperties.put(SSL_KEYSTORE_PASSWORD_CONFIG, "password");
+      kafkaServer = new KafkaLocal(kafkaProperties);
+      kafkaServer.start();
+      logger.info("Kafka Server is successfully started on port " + kafkaLocalPort);
+      return true;
+
+    } catch (Exception e) {
+      logger.error("Error starting the Kafka Server.", e);
+      return false;
+    }
+  }
+
+  private AdminClient getAdminClient() {
+    if (adminClient == null) {
+      Properties adminClientProps = createAdminClientProperties();
+      adminClient = AdminClient.create(adminClientProps);
+    }
+    return adminClient;
+  }
+
+  private Properties createClientProperties() {
+    final Properties props = createAdminClientProperties();
+    props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName());
+    props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName());
+    props.put("auto.commit.interval.ms", "1000");
+    props.put("auto.offset.reset", "earliest");
+    props.put("consumer.timeout.ms","10000");
+    props.put("max.poll.interval.ms","10000");
+
+    // Create the consumer using props.
+    return props;
+  }
+
+  private Properties createAdminClientProperties() {
+    final Properties props = new Properties();
+    props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, getKafkaServerUrl());
+    props.put(ConsumerConfig.GROUP_ID_CONFIG, "group_1");
+    return props;
+  }
+
+  public void initTopicList(List<String> topics) {
+    consumer = new KafkaConsumer<>(clientProps);
+    consumer.subscribe(topics);
+  }
+
+  public void createTopics(List<String> topicNames, int numPartitions) {
+    List<NewTopic> newTopics = new ArrayList<>();
+    for (String topicName: topicNames) {
+      NewTopic newTopic = new NewTopic(topicName, numPartitions, (short) 1);
+      newTopics.add(newTopic);
+    }
+    CreateTopicsResult result = getAdminClient().createTopics(newTopics);
+    Throwable throwable = null;
+    for (int i = 0; i < 10; ++i) {
+      try {
+        result.all().get(1, TimeUnit.SECONDS);
+        throwable = null;
+        break;
+      } catch (Exception e) {
+        throwable = e;
+      }
+    }
+    if (throwable != null) {
+      throw new RuntimeException("Error getting topic info", throwable);
+    }
+  }
+  public void deleteTopic(String topicName) {
+    getAdminClient().deleteTopics(Collections.singletonList(topicName));
+  }
+
+  public ConsumerRecords<String, String> getNextMessageFromConsumer(String topic) {
+    ConsumerRecords<String, String> records = consumer.poll(Duration.ofMillis(1000L));
+    consumer.commitSync();
+    return records;
+
+  }
+
+  public void prepare() {
+
+    if (externalServers) {
+      return;
+    }
+    boolean startStatus = startEmbeddedKafkaServer();
+    if (!startStatus) {
+      throw new RuntimeException("Error starting the server!");
+    }
+    try {
+      Thread.sleep(3 * 1000);   // add this sleep time to
+      // ensure that the server is fully started before proceeding with tests.
+    } catch (InterruptedException e) {
+      // ignore
+    }
+    logger.info("Completed the prepare phase.");
+  }
+
+  public void tearDown() {
+    logger.info("Shutting down the Kafka Consumer.");
+    if (consumer != null) {
+      consumer.close();
+    }
+    if (adminClient != null) {
+      adminClient.close();
+      adminClient = null;
+    }
+    try {
+      Thread.sleep(3 * 1000);   // add this sleep time to
+      // ensure that the server is fully started before proceeding with tests.
+    } catch (InterruptedException e) {
+      // ignore
+    }
+    if (kafkaServer != null) {
+      logger.info("Shutting down the kafka Server.");
+      kafkaServer.stop();
+    }
+    logger.info("Completed the tearDown phase.");
+  }
+
+  private synchronized int getNextPort() throws IOException {
+    try (ServerSocket socket = new ServerSocket(0)) {
+      return socket.getLocalPort();
+    }
+  }
+
+  public String getZkUrl() {
+    return zkServerUrl;
+  }
+
+  public String getKafkaServerUrl() {
+    return kafkaServerUrl;
+  }
+
+  public String getKafkaServerSslUrl() {
+    return kafkaServerSslUrl;
+  }
+
+  public Collection<TransactionListing> getTransactionState() {
+    ListTransactionsResult result = getAdminClient().listTransactions();
+    Throwable throwable = null;
+    for (int i = 0; i < 10; ++i) {
+      try {
+        return result.all().get(1, TimeUnit.SECONDS);
+      } catch (Exception e) {
+        throwable = e;
+      }
+    }
+    throw new RuntimeException("Error getting transactions info", throwable);
+
+  }
+
+  public void deleteTopics(List<String> topicsList) {
+    for (String topic: topicsList) {
+      deleteTopic(topic);
+    }
+  }
+}
diff --git a/flume-kafka-sink/src/test/java/org/apache/flume/sink/kafka/util/ZooKeeperLocal.java b/flume-kafka-sink/src/test/java/org/apache/flume/sink/kafka/util/ZooKeeperLocal.java
new file mode 100644
index 0000000..a1e5647
--- /dev/null
+++ b/flume-kafka-sink/src/test/java/org/apache/flume/sink/kafka/util/ZooKeeperLocal.java
@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ * limitations under the License.
+ */
+
+package org.apache.flume.sink.kafka.util;
+
+import org.apache.zookeeper.server.ServerConfig;
+import org.apache.zookeeper.server.ZooKeeperServerMain;
+import org.apache.zookeeper.server.admin.AdminServer;
+import org.apache.zookeeper.server.quorum.QuorumPeerConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Properties;
+
+/**
+ * A local Zookeeper server for running unit tests.
+ * Reference: https://gist.github.com/fjavieralba/7930018/
+ */
+public class ZooKeeperLocal {
+
+  private static final Logger logger = LoggerFactory.getLogger(ZooKeeperLocal.class);
+  private ZooKeeperServerMain zooKeeperServer;
+
+  public ZooKeeperLocal(Properties zkProperties) throws IOException {
+    QuorumPeerConfig quorumConfiguration = new QuorumPeerConfig();
+    try {
+      quorumConfiguration.parseProperties(zkProperties);
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+
+    zooKeeperServer = new ZooKeeperServerMain();
+    final ServerConfig configuration = new ServerConfig();
+    configuration.readFrom(quorumConfiguration);
+
+    new Thread() {
+      public void run() {
+        try {
+          zooKeeperServer.runFromConfig(configuration);
+        } catch (IOException | AdminServer.AdminServerException e) {
+          logger.error("Zookeeper startup failed.", e);
+        }
+      }
+    }.start();
+  }
+}
diff --git a/flume-kafka-sink/src/test/resources/kafka-server.properties b/flume-kafka-sink/src/test/resources/kafka-server.properties
new file mode 100644
index 0000000..7cc0675
--- /dev/null
+++ b/flume-kafka-sink/src/test/resources/kafka-server.properties
@@ -0,0 +1,131 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+# see kafka.server.KafkaConfig for additional details and defaults
+
+############################# Server Basics #############################
+
+# The id of the broker. This must be set to a unique integer for each broker.
+broker.id=0
+
+############################# Socket Server Settings #############################
+
+# The port the socket server listens on
+port=9092
+
+# Hostname the broker will bind to. If not set, the server will bind to all interfaces
+#host.name=localhost
+
+# Hostname the broker will advertise to producers and consumers. If not set, it uses the
+# value for "host.name" if configured.  Otherwise, it will use the value returned from
+# java.net.InetAddress.getCanonicalHostName().
+#advertised.host.name=<hostname routable by clients>
+
+# The port to publish to ZooKeeper for clients to use. If this is not set,
+# it will publish the same port that the broker binds to.
+#advertised.port=<port accessible by clients>
+
+# The number of threads handling network requests
+num.network.threads=2
+
+# The number of threads doing disk I/O
+num.io.threads=8
+
+# The send buffer (SO_SNDBUF) used by the socket server
+socket.send.buffer.bytes=1048576
+
+# The receive buffer (SO_RCVBUF) used by the socket server
+socket.receive.buffer.bytes=1048576
+
+# The maximum size of a request that the socket server will accept (protection against OOM)
+socket.request.max.bytes=104857600
+
+
+############################# Log Basics #############################
+
+# A comma seperated list of directories under which to store log files
+log.dirs=target/kafka-logs
+
+# The default number of log partitions per topic. More partitions allow greater
+# parallelism for consumption, but this will also result in more files across
+# the brokers.
+num.partitions=2
+
+############################# Log Flush Policy #############################
+
+# Messages are immediately written to the filesystem but by default we only fsync() to sync
+# the OS cache lazily. The following configurations control the flush of data to disk.
+# There are a few important trade-offs here:
+#    1. Durability: Unflushed data may be lost if you are not using replication.
+#    2. Latency: Very large flush intervals may lead to latency spikes when the flush does occur as
+#    there will be a lot of data to flush.
+#    3. Throughput: The flush is generally the most expensive operation, and a small flush interval
+#    may lead to exceessive seeks.
+# The settings below allow one to configure the flush policy to flush data after a period of time or
+# every N messages (or both). This can be done globally and overridden on a per-topic basis.
+
+# The number of messages to accept before forcing a flush of data to disk
+#log.flush.interval.messages=10000
+
+# The maximum amount of time a message can sit in a log before we force a flush
+#log.flush.interval.ms=1000
+
+############################# Log Retention Policy #############################
+
+# The following configurations control the disposal of log segments. The policy can
+# be set to delete segments after a period of time, or after a given size has accumulated.
+# A segment will be deleted whenever *either* of these criteria are met. Deletion always happens
+# from the end of the log.
+
+# The minimum age of a log file to be eligible for deletion
+log.retention.hours=168
+
+# A size-based retention policy for logs. Segments are pruned from the log as long as the remaining
+# segments don't drop below log.retention.bytes.
+#log.retention.bytes=1073741824
+
+# The maximum size of a log segment file. When this size is reached a new log segment will be
+# created.
+log.segment.bytes=536870912
+
+# The interval at which log segments are checked to see if they can be deleted according
+# to the retention policies
+log.retention.check.interval.ms=60000
+
+# By default the log cleaner is disabled and the log retention policy will default to just delete
+# segments after their retention expires.
+# If log.cleaner.enable=true is set the cleaner will be enabled and individual logs can then be
+# marked for log compaction.
+log.cleaner.enable=false
+
+############################# Transactions #############################
+
+# Settings only to be used for non-production
+transaction.state.log.replication.factor = 1
+transaction.state.log.min.isr = 1
+transaction.state.log.num.partitions = 1
+
+############################# Zookeeper #############################
+
+# Zookeeper connection string (see zookeeper docs for details).
+# This is a comma separated host:port pairs, each corresponding to a zk
+# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002".
+# You can also append an optional chroot string to the urls to specify the
+# root directory for all kafka znodes.
+zookeeper.connect=localhost:2181
+
+# Timeout in ms for connecting to zookeeper
+zookeeper.connection.timeout.ms=1000000
+
+offsets.topic.replication.factor=1
diff --git a/flume-kafka-sink/src/test/resources/keystorefile.jks b/flume-kafka-sink/src/test/resources/keystorefile.jks
new file mode 100644
index 0000000..20ac6a8
Binary files /dev/null and b/flume-kafka-sink/src/test/resources/keystorefile.jks differ
diff --git a/flume-kafka-sink/src/test/resources/log4j2.xml b/flume-kafka-sink/src/test/resources/log4j2.xml
new file mode 100644
index 0000000..b4b0067
--- /dev/null
+++ b/flume-kafka-sink/src/test/resources/log4j2.xml
@@ -0,0 +1,72 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements.  See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License.  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+
+-->
+<Configuration status="OFF">
+  <Properties>
+    <Property name="kafka.logs.dir">target/logs</Property>
+  </Properties>
+  <Appenders>
+    <Console name="Console" target="SYSTEM_OUT">
+      <PatternLayout pattern="%d (%t) [%p - %l] %m%n" />
+    </Console>
+    <RollingFile name="kafka" fileName="${kafka.logs.dir}/server.log" filePattern="${kafka.logs.dir}/server.log.%i">
+      <PatternLayout pattern="[%d] %p %m (%c)%n" />
+      <CronTriggeringPolicy schedule="0 0 0 * * ?"/>
+    </RollingFile>
+    <RollingFile name="stateChange" fileName="${kafka.logs.dir}/state-change.log" filePattern="${kafka.logs.dir}/state-change.log.%i">
+      <PatternLayout pattern="[%d] %p %m (%c)%n" />
+      <CronTriggeringPolicy schedule="0 0 0 * * ?"/>
+    </RollingFile>
+    <RollingFile name="request" fileName="${kafka.logs.dir}/kafka-request.log" filePattern="${kafka.logs.dir}/kafka-request.log.%i">
+      <PatternLayout pattern="[%d] %p %m (%c)%n" />
+      <CronTriggeringPolicy schedule="0 0 0 * * ?"/>
+    </RollingFile>
+    <RollingFile name="cleaner" fileName="${kafka.logs.dir}/log-cleaner.log" filePattern="${kafka.logs.dir}/log-cleaner.log.%i">
+      <PatternLayout pattern="[%d] %p %m (%c)%n" />
+      <CronTriggeringPolicy schedule="0 0 0 * * ?"/>
+    </RollingFile>
+    <RollingFile name="controller" fileName="${kafka.logs.dir}/controller.log" filePattern="${kafka.logs.dir}/controller.log.%i">
+      <PatternLayout pattern="[%d] %p %m (%c)%n" />
+      <CronTriggeringPolicy schedule="0 0 0 * * ?"/>
+    </RollingFile>
+  </Appenders>
+
+  <Loggers>
+    <Logger name="kafka" level="info">
+      <AppenderRef ref="kafka"/>
+    </Logger>
+    <Logger name="kafka.network.RequestChannel.*" level="WARN" additivity="false">
+      <AppenderRef ref="request"/>
+    </Logger>
+    <Logger name="kafka.request.logger" level="WARN" additivity="false">
+      <AppenderRef ref="request"/>
+    </Logger>
+    <Logger name="kafka.controller" level="TRACE" additivity="false">
+      <AppenderRef ref="controller"/>
+    </Logger>
+    <Logger name="kafka.log.cleaner" level="INFO" additivity="false">
+      <AppenderRef ref="cleaner"/>
+    </Logger>
+    <Logger name="state.change.logger" level="TRACE" additivity="false">
+      <AppenderRef ref="stateChange"/>
+    </Logger>
+    <Root level="INFO">
+      <AppenderRef ref="Console" />
+    </Root>
+  </Loggers>
+</Configuration>
\ No newline at end of file
diff --git a/flume-kafka-sink/src/test/resources/testutil.properties b/flume-kafka-sink/src/test/resources/testutil.properties
new file mode 100644
index 0000000..97bbf1b
--- /dev/null
+++ b/flume-kafka-sink/src/test/resources/testutil.properties
@@ -0,0 +1,3 @@
+external-servers=false
+kafka-server-url=localhost:9092
+zk-server-url=localhost:2181
diff --git a/flume-kafka-sink/src/test/resources/truststorefile.jks b/flume-kafka-sink/src/test/resources/truststorefile.jks
new file mode 100644
index 0000000..a98c490
Binary files /dev/null and b/flume-kafka-sink/src/test/resources/truststorefile.jks differ
diff --git a/flume-kafka-sink/src/test/resources/zookeeper.properties b/flume-kafka-sink/src/test/resources/zookeeper.properties
new file mode 100644
index 0000000..89e1b5e
--- /dev/null
+++ b/flume-kafka-sink/src/test/resources/zookeeper.properties
@@ -0,0 +1,20 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+# the directory where the snapshot is stored.
+dataDir=target
+# the port at which the clients will connect
+clientPort=2181
+# disable the per-ip limit on the number of connections since this is a non-production config
+maxClientCnxns=0
\ No newline at end of file
diff --git a/flume-kafka-source/pom.xml b/flume-kafka-source/pom.xml
new file mode 100644
index 0000000..b116595
--- /dev/null
+++ b/flume-kafka-source/pom.xml
@@ -0,0 +1,102 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+  -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <parent>
+    <artifactId>flume-kafka-parent</artifactId>
+    <groupId>org.apache.flume</groupId>
+    <version>2.0.0-SNAPSHOT</version>
+  </parent>
+  <modelVersion>4.0.0</modelVersion>
+
+  <groupId>org.apache.flume</groupId>
+  <artifactId>flume-kafka-source</artifactId>
+  <name>Flume Kafka Source</name>
+
+  <properties>
+    <!-- TODO fix spotbugs/pmd violations -->
+    <spotbugs.maxAllowedViolations>16</spotbugs.maxAllowedViolations>
+    <pmd.maxAllowedViolations>32</pmd.maxAllowedViolations>
+    <module.name>org.apache.flume.source.kafka</module.name>
+  </properties>
+
+  <dependencies>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.flume</groupId>
+      <artifactId>flume-ng-core</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.flume</groupId>
+      <artifactId>flume-shared-kafka</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.flume</groupId>
+      <artifactId>flume-shared-kafka-test</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.kafka</groupId>
+      <artifactId>kafka_${scala.version}</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.scala-lang</groupId>
+      <artifactId>scala-library</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>io.dropwizard.metrics</groupId>
+      <artifactId>metrics-core</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.zookeeper</groupId>
+      <artifactId>zookeeper</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-all</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.kafka</groupId>
+      <artifactId>kafka-clients</artifactId>
+      <version>${kafka.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.kafka</groupId>
+      <artifactId>kafka-clients</artifactId>
+      <classifier>test</classifier>
+      <version>${kafka.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.kafka</groupId>
+      <artifactId>kafka_${scala.version}</artifactId>
+      <version>${kafka.version}</version>
+      <classifier>test</classifier>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.logging.log4j</groupId>
+      <artifactId>log4j-slf4j-impl</artifactId>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+</project>
diff --git a/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSource.java b/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSource.java
new file mode 100644
index 0000000..a99b6cf
--- /dev/null
+++ b/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSource.java
@@ -0,0 +1,539 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flume.source.kafka;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Optional;
+import kafka.zk.KafkaZkClient;
+import org.apache.avro.io.BinaryDecoder;
+import org.apache.avro.io.DecoderFactory;
+import org.apache.avro.specific.SpecificDatumReader;
+import org.apache.flume.Context;
+import org.apache.flume.Event;
+import org.apache.flume.EventDeliveryException;
+import org.apache.flume.FlumeException;
+import org.apache.flume.conf.BatchSizeSupported;
+import org.apache.flume.conf.Configurable;
+import org.apache.flume.conf.ConfigurationException;
+import org.apache.flume.conf.LogPrivacyUtil;
+import org.apache.flume.event.EventBuilder;
+import org.apache.flume.instrumentation.kafka.KafkaSourceCounter;
+import org.apache.flume.shared.kafka.KafkaSSLUtil;
+import org.apache.flume.source.AbstractPollableSource;
+import org.apache.flume.source.avro.AvroFlumeEvent;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.common.PartitionInfo;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.config.SslConfigs;
+import org.apache.kafka.common.header.Header;
+import org.apache.kafka.common.header.Headers;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+
+import java.io.ByteArrayInputStream;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.UUID;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.regex.Pattern;
+
+import static org.apache.flume.shared.kafka.KafkaSSLUtil.SSL_DISABLE_FQDN_CHECK;
+import static org.apache.flume.shared.kafka.KafkaSSLUtil.isSSLEnabled;
+import static org.apache.flume.source.kafka.KafkaSourceConstants.AVRO_EVENT;
+import static org.apache.flume.source.kafka.KafkaSourceConstants.BATCH_DURATION_MS;
+import static org.apache.flume.source.kafka.KafkaSourceConstants.BATCH_SIZE;
+import static org.apache.flume.source.kafka.KafkaSourceConstants.BOOTSTRAP_SERVERS;
+import static org.apache.flume.source.kafka.KafkaSourceConstants.DEFAULT_AUTO_COMMIT;
+import static org.apache.flume.source.kafka.KafkaSourceConstants.DEFAULT_AVRO_EVENT;
+import static org.apache.flume.source.kafka.KafkaSourceConstants.DEFAULT_BATCH_DURATION;
+import static org.apache.flume.source.kafka.KafkaSourceConstants.DEFAULT_BATCH_SIZE;
+import static org.apache.flume.source.kafka.KafkaSourceConstants.DEFAULT_GROUP_ID;
+import static org.apache.flume.source.kafka.KafkaSourceConstants.DEFAULT_KEY_DESERIALIZER;
+import static org.apache.flume.source.kafka.KafkaSourceConstants.DEFAULT_SET_TOPIC_HEADER;
+import static org.apache.flume.source.kafka.KafkaSourceConstants.DEFAULT_TOPIC_HEADER;
+import static org.apache.flume.source.kafka.KafkaSourceConstants.DEFAULT_VALUE_DESERIALIZER;
+import static org.apache.flume.source.kafka.KafkaSourceConstants.KAFKA_CONSUMER_PREFIX;
+import static org.apache.flume.source.kafka.KafkaSourceConstants.KAFKA_HEADER;
+import static org.apache.flume.source.kafka.KafkaSourceConstants.KEY_HEADER;
+import static org.apache.flume.source.kafka.KafkaSourceConstants.OFFSET_HEADER;
+import static org.apache.flume.source.kafka.KafkaSourceConstants.PARTITION_HEADER;
+import static org.apache.flume.source.kafka.KafkaSourceConstants.SET_TOPIC_HEADER;
+import static org.apache.flume.source.kafka.KafkaSourceConstants.TIMESTAMP_HEADER;
+import static org.apache.flume.source.kafka.KafkaSourceConstants.TOPICS;
+import static org.apache.flume.source.kafka.KafkaSourceConstants.TOPICS_REGEX;
+import static org.apache.flume.source.kafka.KafkaSourceConstants.TOPIC_HEADER;
+
+/**
+ * A Source for Kafka which reads messages from kafka topics.
+ *
+ * <tt>kafka.bootstrap.servers: </tt> A comma separated list of host:port pairs
+ * to use for establishing the initial connection to the Kafka cluster.
+ * For example host1:port1,host2:port2,...
+ * <b>Required</b> for kafka.
+ * <p>
+ * <tt>kafka.consumer.group.id: </tt> the group ID of consumer group. <b>Required</b>
+ * <p>
+ * <tt>kafka.topics: </tt> the topic list separated by commas to consume messages from.
+ * <b>Required</b>
+ * <p>
+ * <tt>maxBatchSize: </tt> Maximum number of messages written to Channel in one
+ * batch. Default: 1000
+ * <p>
+ * <tt>maxBatchDurationMillis: </tt> Maximum number of milliseconds before a
+ * batch (of any size) will be written to a channel. Default: 1000
+ * <p>
+ * <tt>kafka.consumer.*: </tt> Any property starting with "kafka.consumer" will be
+ * passed to the kafka consumer So you can use any configuration supported by Kafka 0.9.0.X
+ * <tt>useFlumeEventFormat: </tt> Reads events from Kafka Topic as an Avro FlumeEvent. Used
+ * in conjunction with useFlumeEventFormat (Kafka Sink) or parseAsFlumeEvent (Kafka Channel)
+ * <p>
+ */
+public class KafkaSource extends AbstractPollableSource
+        implements Configurable, BatchSizeSupported {
+  private static final Logger log = LoggerFactory.getLogger(KafkaSource.class);
+
+  // Constants used only for offset migration zookeeper connections
+  private static final int ZK_SESSION_TIMEOUT = 30000;
+  private static final int ZK_CONNECTION_TIMEOUT = 30000;
+
+  private Context context;
+  private Properties kafkaProps;
+  private KafkaSourceCounter counter;
+  private KafkaConsumer<String, byte[]> consumer;
+  private Iterator<ConsumerRecord<String, byte[]>> it;
+
+  private final List<Event> eventList = new ArrayList<Event>();
+  private Map<TopicPartition, OffsetAndMetadata> tpAndOffsetMetadata;
+  private AtomicBoolean rebalanceFlag;
+
+  private Map<String, String> headers;
+
+  private Optional<SpecificDatumReader<AvroFlumeEvent>> reader = Optional.absent();
+  private BinaryDecoder decoder = null;
+
+  private boolean useAvroEventFormat;
+
+  private int batchUpperLimit;
+  private int maxBatchDurationMillis;
+
+  private Subscriber subscriber;
+
+  private String bootstrapServers;
+  private String groupId = DEFAULT_GROUP_ID;
+  private String topicHeader = null;
+  private boolean setTopicHeader;
+  private Map<String, String> headerMap;
+
+  @Override
+  public long getBatchSize() {
+    return batchUpperLimit;
+  }
+
+  /**
+   * This class is a helper to subscribe for topics by using
+   * different strategies
+   */
+  public abstract class Subscriber<T> {
+    public abstract void subscribe(KafkaConsumer<?, ?> consumer, SourceRebalanceListener listener);
+
+    public T get() {
+      return null;
+    }
+  }
+
+  private class TopicListSubscriber extends Subscriber<List<String>> {
+    private List<String> topicList;
+
+    public TopicListSubscriber(String commaSeparatedTopics) {
+      this.topicList = Arrays.asList(commaSeparatedTopics.split("^\\s+|\\s*,\\s*|\\s+$"));
+    }
+
+    @Override
+    public void subscribe(KafkaConsumer<?, ?> consumer, SourceRebalanceListener listener) {
+      consumer.subscribe(topicList, listener);
+    }
+
+    @Override
+    public List<String> get() {
+      return topicList;
+    }
+  }
+
+  private class PatternSubscriber extends Subscriber<Pattern> {
+    private Pattern pattern;
+
+    public PatternSubscriber(String regex) {
+      this.pattern = Pattern.compile(regex);
+    }
+
+    @Override
+    public void subscribe(KafkaConsumer<?, ?> consumer, SourceRebalanceListener listener) {
+      consumer.subscribe(pattern, listener);
+    }
+
+    @Override
+    public Pattern get() {
+      return pattern;
+    }
+  }
+
+
+  @Override
+  protected Status doProcess() throws EventDeliveryException {
+    final String batchUUID = UUID.randomUUID().toString();
+    String kafkaKey;
+    Event event;
+    byte[] eventBody;
+
+    try {
+      // prepare time variables for new batch
+      final long nanoBatchStartTime = System.nanoTime();
+      final long batchStartTime = System.currentTimeMillis();
+      final long maxBatchEndTime = System.currentTimeMillis() + maxBatchDurationMillis;
+
+      while (eventList.size() < batchUpperLimit &&
+              System.currentTimeMillis() < maxBatchEndTime) {
+
+        if (it == null || !it.hasNext()) {
+          // Obtaining new records
+          // Poll time is remainder time for current batch.
+          long durMs = Math.max(0L, maxBatchEndTime - System.currentTimeMillis());
+          Duration duration = Duration.ofMillis(durMs);
+          ConsumerRecords<String, byte[]> records = consumer.poll(duration);
+          it = records.iterator();
+
+          // this flag is set to true in a callback when some partitions are revoked.
+          // If there are any records we commit them.
+          if (rebalanceFlag.compareAndSet(true, false)) {
+            break;
+          }
+          // check records after poll
+          if (!it.hasNext()) {
+            counter.incrementKafkaEmptyCount();
+            log.debug("Returning with backoff. No more data to read");
+            // batch time exceeded
+            break;
+          }
+        }
+
+        // get next message
+        ConsumerRecord<String, byte[]> message = it.next();
+        kafkaKey = message.key();
+
+        if (useAvroEventFormat) {
+          //Assume the event is in Avro format using the AvroFlumeEvent schema
+          //Will need to catch the exception if it is not
+          ByteArrayInputStream in =
+                  new ByteArrayInputStream(message.value());
+          decoder = DecoderFactory.get().directBinaryDecoder(in, decoder);
+          if (!reader.isPresent()) {
+            reader = Optional.of(
+                    new SpecificDatumReader<AvroFlumeEvent>(AvroFlumeEvent.class));
+          }
+          //This may throw an exception but it will be caught by the
+          //exception handler below and logged at error
+          AvroFlumeEvent avroevent = reader.get().read(null, decoder);
+
+          eventBody = avroevent.getBody().array();
+          headers = toStringMap(avroevent.getHeaders());
+        } else {
+          eventBody = message.value();
+          headers.clear();
+          headers = new HashMap<String, String>(4);
+        }
+
+        // Add headers to event (timestamp, topic, partition, key) only if they don't exist
+        if (!headers.containsKey(TIMESTAMP_HEADER)) {
+          headers.put(TIMESTAMP_HEADER, String.valueOf(message.timestamp()));
+        }
+        if (!headerMap.isEmpty()) {
+          Headers kafkaHeaders = message.headers();
+          for (Map.Entry<String, String> entry : headerMap.entrySet()) {
+            for (Header kafkaHeader : kafkaHeaders.headers(entry.getValue())) {
+              headers.put(entry.getKey(), new String(kafkaHeader.value()));
+            }
+          }
+        }
+        // Only set the topic header if setTopicHeader and it isn't already populated
+        if (setTopicHeader && !headers.containsKey(topicHeader)) {
+          headers.put(topicHeader, message.topic());
+        }
+        if (!headers.containsKey(PARTITION_HEADER)) {
+          headers.put(PARTITION_HEADER, String.valueOf(message.partition()));
+        }
+        if (!headers.containsKey(OFFSET_HEADER)) {
+          headers.put(OFFSET_HEADER, String.valueOf(message.offset()));
+        }
+
+        if (kafkaKey != null) {
+          headers.put(KEY_HEADER, kafkaKey);
+        }
+
+        if (log.isTraceEnabled()) {
+          if (LogPrivacyUtil.allowLogRawData()) {
+            log.trace("Topic: {} Partition: {} Message: {}", new String[]{
+                message.topic(),
+                String.valueOf(message.partition()),
+                new String(eventBody)
+            });
+          } else {
+            log.trace("Topic: {} Partition: {} Message arrived.",
+                message.topic(),
+                String.valueOf(message.partition()));
+          }
+        }
+
+        event = EventBuilder.withBody(eventBody, headers);
+        eventList.add(event);
+
+        if (log.isDebugEnabled()) {
+          log.debug("Waited: {} ", System.currentTimeMillis() - batchStartTime);
+          log.debug("Event #: {}", eventList.size());
+        }
+
+        // For each partition store next offset that is going to be read.
+        tpAndOffsetMetadata.put(new TopicPartition(message.topic(), message.partition()),
+                new OffsetAndMetadata(message.offset() + 1, batchUUID));
+      }
+
+      if (eventList.size() > 0) {
+        counter.addToKafkaEventGetTimer((System.nanoTime() - nanoBatchStartTime) / (1000 * 1000));
+        counter.addToEventReceivedCount((long) eventList.size());
+        getChannelProcessor().processEventBatch(eventList);
+        counter.addToEventAcceptedCount(eventList.size());
+        if (log.isDebugEnabled()) {
+          log.debug("Wrote {} events to channel", eventList.size());
+        }
+        eventList.clear();
+
+        if (!tpAndOffsetMetadata.isEmpty()) {
+          long commitStartTime = System.nanoTime();
+          consumer.commitSync(tpAndOffsetMetadata);
+          long commitEndTime = System.nanoTime();
+          counter.addToKafkaCommitTimer((commitEndTime - commitStartTime) / (1000 * 1000));
+          tpAndOffsetMetadata.clear();
+        }
+        return Status.READY;
+      }
+
+      return Status.BACKOFF;
+    } catch (Exception e) {
+      log.error("KafkaSource EXCEPTION, {}", e);
+      counter.incrementEventReadOrChannelFail(e);
+      return Status.BACKOFF;
+    }
+  }
+
+  /**
+   * We configure the source and generate properties for the Kafka Consumer
+   *
+   * Kafka Consumer properties are generated as follows:
+   * 1. Generate a properties object with some static defaults that can be
+   * overridden if corresponding properties are specified
+   * 2. We add the configuration users added for Kafka (parameters starting
+   * with kafka.consumer and must be valid Kafka Consumer properties
+   * 3. Add source level properties (with no prefix)
+   * @param context
+   */
+  @Override
+  protected void doConfigure(Context context) throws FlumeException {
+    this.context = context;
+    headers = new HashMap<String, String>(4);
+    tpAndOffsetMetadata = new HashMap<TopicPartition, OffsetAndMetadata>();
+    rebalanceFlag = new AtomicBoolean(false);
+    kafkaProps = new Properties();
+
+    String topicProperty = context.getString(TOPICS_REGEX);
+    if (topicProperty != null && !topicProperty.isEmpty()) {
+      // create subscriber that uses pattern-based subscription
+      subscriber = new PatternSubscriber(topicProperty);
+    } else if ((topicProperty = context.getString(TOPICS)) != null &&
+               !topicProperty.isEmpty()) {
+      // create subscriber that uses topic list subscription
+      subscriber = new TopicListSubscriber(topicProperty);
+    } else if (subscriber == null) {
+      throw new ConfigurationException("At least one Kafka topic must be specified.");
+    }
+
+    batchUpperLimit = context.getInteger(BATCH_SIZE, DEFAULT_BATCH_SIZE);
+    maxBatchDurationMillis = context.getInteger(BATCH_DURATION_MS, DEFAULT_BATCH_DURATION);
+
+    useAvroEventFormat = context.getBoolean(AVRO_EVENT, DEFAULT_AVRO_EVENT);
+
+    if (log.isDebugEnabled()) {
+      log.debug(AVRO_EVENT + " set to: {}", useAvroEventFormat);
+    }
+
+    bootstrapServers = context.getString(BOOTSTRAP_SERVERS);
+    if (bootstrapServers == null || bootstrapServers.isEmpty()) {
+      throw new ConfigurationException("Bootstrap Servers must be specified");
+    }
+
+    String groupIdProperty =
+        context.getString(KAFKA_CONSUMER_PREFIX + ConsumerConfig.GROUP_ID_CONFIG);
+    if (groupIdProperty != null && !groupIdProperty.isEmpty()) {
+      groupId = groupIdProperty; // Use the new group id property
+    }
+
+    if (groupId == null || groupId.isEmpty()) {
+      groupId = DEFAULT_GROUP_ID;
+      log.info("Group ID was not specified. Using {} as the group id.", groupId);
+    }
+
+    setTopicHeader = context.getBoolean(SET_TOPIC_HEADER, DEFAULT_SET_TOPIC_HEADER);
+
+    topicHeader = context.getString(TOPIC_HEADER, DEFAULT_TOPIC_HEADER);
+
+    headerMap = context.getSubProperties(KAFKA_HEADER);
+
+    setConsumerProps(context);
+
+    if (log.isDebugEnabled() && LogPrivacyUtil.allowLogPrintConfig()) {
+      log.debug("Kafka consumer properties: {}", kafkaProps);
+    }
+
+    if (counter == null) {
+      counter = new KafkaSourceCounter(getName());
+    }
+  }
+
+  private void setConsumerProps(Context ctx) {
+    kafkaProps.clear();
+    kafkaProps.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, DEFAULT_KEY_DESERIALIZER);
+    kafkaProps.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, DEFAULT_VALUE_DESERIALIZER);
+    //Defaults overridden based on config
+    kafkaProps.putAll(ctx.getSubProperties(KAFKA_CONSUMER_PREFIX));
+    //These always take precedence over config
+    kafkaProps.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers);
+    if (groupId != null) {
+      kafkaProps.put(ConsumerConfig.GROUP_ID_CONFIG, groupId);
+    }
+    kafkaProps.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, DEFAULT_AUTO_COMMIT);
+    //  The default value of `ssl.endpoint.identification.algorithm`
+    //  is changed to `https`, since kafka client 2.0+
+    //  And because flume does not accept an empty string as property value,
+    //  so we need to use an alternative custom property
+    //  `ssl.disableTLSHostnameVerification` to check if enable fqdn check.
+    if (isSSLEnabled(kafkaProps) && "true".equalsIgnoreCase(kafkaProps.getProperty(SSL_DISABLE_FQDN_CHECK))) {
+      kafkaProps.put(SslConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG, "");
+    }
+    KafkaSSLUtil.addGlobalSSLParameters(kafkaProps);
+  }
+
+  @VisibleForTesting
+  String getBootstrapServers() {
+    return bootstrapServers;
+  }
+
+  Properties getConsumerProps() {
+    return kafkaProps;
+  }
+
+  /**
+   * Helper function to convert a map of CharSequence to a map of String.
+   */
+  private static Map<String, String> toStringMap(Map<CharSequence, CharSequence> charSeqMap) {
+    Map<String, String> stringMap = new HashMap<String, String>();
+    for (Map.Entry<CharSequence, CharSequence> entry : charSeqMap.entrySet()) {
+      stringMap.put(entry.getKey().toString(), entry.getValue().toString());
+    }
+    return stringMap;
+  }
+
+  <T> Subscriber<T> getSubscriber() {
+    return subscriber;
+  }
+
+  @Override
+  protected void doStart() throws FlumeException {
+    log.info("Starting {}...", this);
+
+    //initialize a consumer.
+    consumer = new KafkaConsumer<String, byte[]>(kafkaProps);
+
+    // Subscribe for topics by already specified strategy
+    subscriber.subscribe(consumer, new SourceRebalanceListener(rebalanceFlag));
+
+    log.info("Kafka source {} started.", getName());
+    counter.start();
+  }
+
+  @Override
+  protected void doStop() throws FlumeException {
+    if (consumer != null) {
+      consumer.wakeup();
+      consumer.close();
+    }
+    if (counter != null) {
+      counter.stop();
+    }
+    log.info("Kafka Source {} stopped. Metrics: {}", getName(), counter);
+  }
+
+  private Map<TopicPartition, OffsetAndMetadata> getZookeeperOffsets(
+          KafkaZkClient zkClient, KafkaConsumer<String, byte[]> consumer, String topicStr) {
+
+    Map<TopicPartition, OffsetAndMetadata> offsets = new HashMap<>();
+    List<PartitionInfo> partitions = consumer.partitionsFor(topicStr);
+    for (PartitionInfo partition : partitions) {
+      TopicPartition topicPartition = new TopicPartition(topicStr, partition.partition());
+      Option<Object> optionOffset = zkClient.getConsumerOffset(groupId, topicPartition);
+      if (optionOffset.nonEmpty()) {
+        Long offset = (Long) optionOffset.get();
+        OffsetAndMetadata offsetAndMetadata = new OffsetAndMetadata(offset);
+        offsets.put(topicPartition, offsetAndMetadata);
+      }
+    }
+    return offsets;
+  }
+}
+
+class SourceRebalanceListener implements ConsumerRebalanceListener {
+  private static final Logger log = LoggerFactory.getLogger(SourceRebalanceListener.class);
+  private AtomicBoolean rebalanceFlag;
+
+  public SourceRebalanceListener(AtomicBoolean rebalanceFlag) {
+    this.rebalanceFlag = rebalanceFlag;
+  }
+
+  // Set a flag that a rebalance has occurred. Then commit already read events to kafka.
+  public void onPartitionsRevoked(Collection<TopicPartition> partitions) {
+    for (TopicPartition partition : partitions) {
+      log.info("topic {} - partition {} revoked.", partition.topic(), partition.partition());
+      rebalanceFlag.set(true);
+    }
+  }
+
+  public void onPartitionsAssigned(Collection<TopicPartition> partitions) {
+    for (TopicPartition partition : partitions) {
+      log.info("topic {} - partition {} assigned.", partition.topic(), partition.partition());
+    }
+  }
+}
diff --git a/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSourceConstants.java b/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSourceConstants.java
new file mode 100644
index 0000000..97216e1
--- /dev/null
+++ b/flume-kafka-source/src/main/java/org/apache/flume/source/kafka/KafkaSourceConstants.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flume.source.kafka;
+
+import org.apache.kafka.clients.CommonClientConfigs;
+
+public class KafkaSourceConstants {
+
+  public static final String KAFKA_PREFIX = "kafka.";
+  public static final String KAFKA_CONSUMER_PREFIX = KAFKA_PREFIX + "consumer.";
+  public static final String DEFAULT_KEY_DESERIALIZER =
+      "org.apache.kafka.common.serialization.StringDeserializer";
+  public static final String DEFAULT_VALUE_DESERIALIZER =
+      "org.apache.kafka.common.serialization.ByteArrayDeserializer";
+  public static final String BOOTSTRAP_SERVERS =
+      KAFKA_PREFIX + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG;
+  public static final String TOPICS = KAFKA_PREFIX + "topics";
+  public static final String TOPICS_REGEX = TOPICS + "." + "regex";
+  public static final String DEFAULT_AUTO_COMMIT =  "false";
+  public static final String BATCH_SIZE = "batchSize";
+  public static final String BATCH_DURATION_MS = "batchDurationMillis";
+  public static final int DEFAULT_BATCH_SIZE = 1000;
+  public static final int DEFAULT_BATCH_DURATION = 1000;
+  public static final String DEFAULT_GROUP_ID = "flume";
+  public static final String KAFKA_HEADER = "header.";
+
+  public static final String AVRO_EVENT = "useFlumeEventFormat";
+  public static final boolean DEFAULT_AVRO_EVENT = false;
+
+  // flume event headers
+  public static final String DEFAULT_TOPIC_HEADER = "topic";
+  public static final String KEY_HEADER = "key";
+  public static final String TIMESTAMP_HEADER = "timestamp";
+  public static final String PARTITION_HEADER = "partition";
+  public static final String OFFSET_HEADER = "offset";
+
+  public static final String SET_TOPIC_HEADER = "setTopicHeader";
+  public static final boolean DEFAULT_SET_TOPIC_HEADER = true;
+  public static final String TOPIC_HEADER = "topicHeader";
+
+
+
+}
diff --git a/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceEmbeddedKafka.java b/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceEmbeddedKafka.java
new file mode 100644
index 0000000..e2c4d8f
--- /dev/null
+++ b/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceEmbeddedKafka.java
@@ -0,0 +1,209 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flume.source.kafka;
+
+import kafka.server.KafkaConfig;
+import kafka.server.KafkaServer;
+import kafka.utils.TestUtils;
+import org.apache.commons.io.FileUtils;
+import org.apache.kafka.clients.admin.AdminClient;
+import org.apache.kafka.clients.admin.CreateTopicsResult;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.common.header.Headers;
+import org.apache.kafka.common.serialization.ByteArraySerializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.common.utils.Time;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.ServerSocket;
+import java.util.Collections;
+import java.util.List;
+import java.util.Properties;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.kafka.common.config.SslConfigs.*;
+
+public class KafkaSourceEmbeddedKafka {
+
+  public static String HOST = InetAddress.getLoopbackAddress().getCanonicalHostName();
+
+  KafkaServer kafkaServer;
+  KafkaSourceEmbeddedZookeeper zookeeper;
+  private AdminClient adminClient;
+
+  private static int findFreePort() {
+    try (ServerSocket socket = new ServerSocket(0)) {
+      return socket.getLocalPort();
+    } catch (IOException e) {
+      throw new AssertionError("Can not find free port.", e);
+    }
+  }
+
+  private int zkPort = findFreePort(); // none-standard
+  private int serverPort = findFreePort();
+  private int serverSslPort = findFreePort();
+
+  KafkaProducer<String, byte[]> producer;
+  File dir;
+
+  public KafkaSourceEmbeddedKafka(Properties properties) {
+    zookeeper = new KafkaSourceEmbeddedZookeeper(zkPort);
+    dir = new File(System.getProperty("java.io.tmpdir"), "kafka_log-" + UUID.randomUUID());
+    try {
+      FileUtils.deleteDirectory(dir);
+    } catch (IOException e) {
+      e.printStackTrace();
+    }
+    Properties props = new Properties();
+    props.put("zookeeper.connect",zookeeper.getConnectString());
+    props.put("broker.id","1");
+    props.put("host.name", "localhost");
+    //  to enable ssl feature,
+    //  we need to use listeners instead of using port property
+    //  props.put("port", String.valueOf(serverPort));
+    props.put("listeners",
+            String.format("PLAINTEXT://%s:%d,SSL://%s:%d",
+                    HOST,
+                    serverPort,
+                    HOST,
+                    serverSslPort
+            )
+    );
+    props.put("log.dir", dir.getAbsolutePath());
+    props.put("offsets.topic.replication.factor", "1");
+    props.put("auto.create.topics.enable", "false");
+    //  ssl configuration
+    props.put(SSL_TRUSTSTORE_LOCATION_CONFIG, "src/test/resources/truststorefile.jks");
+    props.put(SSL_TRUSTSTORE_PASSWORD_CONFIG, "password");
+    props.put(SSL_KEYSTORE_LOCATION_CONFIG, "src/test/resources/keystorefile.jks");
+    props.put(SSL_KEYSTORE_PASSWORD_CONFIG, "password");
+    if (properties != null) {
+      props.putAll(properties);
+    }
+    KafkaConfig config = new KafkaConfig(props);
+    kafkaServer = TestUtils.createServer(config, Time.SYSTEM);
+    kafkaServer.startup();
+    initProducer();
+  }
+
+  public void stop() throws IOException {
+    producer.close();
+    kafkaServer.shutdown();
+    zookeeper.stopZookeeper();
+  }
+
+  public String getZkConnectString() {
+    return zookeeper.getConnectString();
+  }
+
+  public String getBootstrapServers() {
+    return HOST + ":" + serverPort;
+  }
+
+  public String getBootstrapSslServers() {
+    return String.format("%s:%s", HOST, serverSslPort);
+  }
+
+  public String getBootstrapSslIpPortServers() {
+    return String.format("%s:%s", "127.0.0.1", serverSslPort);
+  }
+
+  private void initProducer() {
+    Properties props = new Properties();
+    props.put("bootstrap.servers", HOST + ":" + serverPort);
+    props.put("acks", "1");
+    producer = new KafkaProducer<String,byte[]>(props,
+            new StringSerializer(), new ByteArraySerializer());
+  }
+
+  public void produce(String topic, String k, String v) {
+    produce(topic, k, v.getBytes());
+  }
+
+  public void produce(String topic, String k, byte[] v) {
+    ProducerRecord<String, byte[]> rec = new ProducerRecord<>(topic, k, v);
+    try {
+      producer.send(rec).get();
+    } catch (InterruptedException e) {
+      e.printStackTrace();
+    } catch (ExecutionException e) {
+      e.printStackTrace();
+    }
+  }
+
+  public void produce(String topic, int partition, String k, String v) {
+    produce(topic,partition,k,v.getBytes());
+  }
+
+  public void produce(String topic, int partition, String k, byte[] v) {
+    this.produce(topic, partition, null, k, v, null);
+  }
+
+  public void produce(String topic, int partition, Long timestamp, String k, byte[] v,
+      Headers headers) {
+    ProducerRecord<String, byte[]> rec = new ProducerRecord<>(topic, partition, timestamp, k, v,
+        headers);
+    try {
+      producer.send(rec).get();
+    } catch (InterruptedException e) {
+      e.printStackTrace();
+    } catch (ExecutionException e) {
+      e.printStackTrace();
+    }
+  }
+
+  public void createTopic(String topicName, int numPartitions) {
+    AdminClient adminClient = getAdminClient();
+    NewTopic newTopic = new NewTopic(topicName, numPartitions, (short) 1);
+    CreateTopicsResult result = adminClient.createTopics(Collections.singletonList(newTopic));
+    Throwable throwable = null;
+    for (int i = 0; i < 10; ++i) {
+      try {
+        result.all().get(1, TimeUnit.SECONDS);
+        throwable = null;
+        break;
+      } catch (Exception e) {
+        throwable = e;
+      }
+    }
+    if (throwable != null) {
+      throw new RuntimeException("Error getting topic info", throwable);
+    }
+  }
+
+  private AdminClient getAdminClient() {
+    if (adminClient == null) {
+      final Properties props = new Properties();
+      props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, HOST + ":" + serverPort);
+      props.put(ConsumerConfig.GROUP_ID_CONFIG, "group_1");
+      adminClient = AdminClient.create(props);
+    }
+    return adminClient;
+  }
+
+  public void deleteTopics(List<String> topic) {
+    getAdminClient().deleteTopics(topic);
+  }
+
+}
diff --git a/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceEmbeddedZookeeper.java b/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceEmbeddedZookeeper.java
new file mode 100644
index 0000000..f04fc64
--- /dev/null
+++ b/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/KafkaSourceEmbeddedZookeeper.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flume.source.kafka;
+
+import org.apache.zookeeper.server.NIOServerCnxnFactory;
+import org.apache.zookeeper.server.ZooKeeperServer;
+import org.apache.commons.io.FileUtils;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.UUID;
+
+public class KafkaSourceEmbeddedZookeeper {
+  private int zkPort;
+  private ZooKeeperServer zookeeper;
+  private NIOServerCnxnFactory factory;
+  File dir;
+
+  public KafkaSourceEmbeddedZookeeper(int zkPort) {
+    int tickTime = 2000;
+
+    this.zkPort = zkPort;
+
+    String dataDirectory = System.getProperty("java.io.tmpdir");
+    dir = new File(dataDirectory, "zookeeper" + UUID.randomUUID()).getAbsoluteFile();
+
+    try {
+      FileUtils.deleteDirectory(dir);
+    } catch (IOException e) {
+      e.printStackTrace();
+      System.exit(1);
+    }
+
+    try {
+      this.zookeeper = new ZooKeeperServer(dir,dir,tickTime);
+      this.factory = new NIOServerCnxnFactory();
+      factory.configure(new InetSocketAddress(KafkaSourceEmbeddedKafka.HOST, zkPort),0);
+      factory.startup(zookeeper);
+    } catch (IOException e) {
+      e.printStackTrace();
+    } catch (InterruptedException e) {
+      e.printStackTrace();
+    }
+  }
+
+  public void stopZookeeper() throws IOException {
+    zookeeper.shutdown();
+    factory.shutdown();
+    FileUtils.deleteDirectory(dir);
+  }
+
+  public String getConnectString() {
+    return KafkaSourceEmbeddedKafka.HOST + ":" + zkPort;
+  }
+}
diff --git a/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/TestKafkaSource.java b/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/TestKafkaSource.java
new file mode 100644
index 0000000..2d67f10
--- /dev/null
+++ b/flume-kafka-source/src/test/java/org/apache/flume/source/kafka/TestKafkaSource.java
@@ -0,0 +1,932 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flume.source.kafka;
+
+import com.google.common.base.Charsets;
+import com.google.common.collect.Lists;
+import org.apache.avro.io.BinaryEncoder;
+import org.apache.avro.io.EncoderFactory;
+import org.apache.avro.specific.SpecificDatumWriter;
+import org.apache.commons.lang.RandomStringUtils;
+import org.apache.flume.ChannelException;
+import org.apache.flume.Context;
+import org.apache.flume.Event;
+import org.apache.flume.EventDeliveryException;
+import org.apache.flume.FlumeException;
+import org.apache.flume.PollableSource.Status;
+import org.apache.flume.channel.ChannelProcessor;
+import org.apache.flume.instrumentation.SourceCounter;
+import org.apache.flume.lifecycle.LifecycleState;
+import org.apache.flume.source.avro.AvroFlumeEvent;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.common.errors.TopicExistsException;
+import org.apache.kafka.common.header.Headers;
+import org.apache.kafka.common.header.internals.RecordHeader;
+import org.apache.kafka.common.header.internals.RecordHeaders;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.mockito.Mockito;
+import org.mockito.internal.util.reflection.Whitebox;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.time.Duration;
+import java.time.ZoneId;
+import java.time.ZonedDateTime;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.regex.Pattern;
+
+import static org.apache.flume.shared.kafka.KafkaSSLUtil.SSL_DISABLE_FQDN_CHECK;
+import static org.apache.flume.source.kafka.KafkaSourceConstants.AVRO_EVENT;
+import static org.apache.flume.source.kafka.KafkaSourceConstants.BATCH_DURATION_MS;
+import static org.apache.flume.source.kafka.KafkaSourceConstants.BATCH_SIZE;
+import static org.apache.flume.source.kafka.KafkaSourceConstants.BOOTSTRAP_SERVERS;
+import static org.apache.flume.source.kafka.KafkaSourceConstants.DEFAULT_AUTO_COMMIT;
+import static org.apache.flume.source.kafka.KafkaSourceConstants.DEFAULT_TOPIC_HEADER;
+import static org.apache.flume.source.kafka.KafkaSourceConstants.KAFKA_CONSUMER_PREFIX;
+import static org.apache.flume.source.kafka.KafkaSourceConstants.PARTITION_HEADER;
+import static org.apache.flume.source.kafka.KafkaSourceConstants.TIMESTAMP_HEADER;
+import static org.apache.flume.source.kafka.KafkaSourceConstants.TOPICS;
+import static org.apache.flume.source.kafka.KafkaSourceConstants.TOPICS_REGEX;
+import static org.apache.kafka.clients.CommonClientConfigs.SECURITY_PROTOCOL_CONFIG;
+import static org.apache.kafka.common.config.SslConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG;
+import static org.apache.kafka.common.config.SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG;
+import static org.apache.kafka.common.config.SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.mock;
+
+public class TestKafkaSource {
+  private static final Logger log = LoggerFactory.getLogger(TestKafkaSource.class);
+
+  private KafkaSource kafkaSource;
+  private static KafkaSourceEmbeddedKafka kafkaServer;
+  private Context context;
+  private List<Event> events;
+
+  private final List<String> usedTopics = new ArrayList<>();
+  private String topic0;
+  private String topic1;
+
+
+  @BeforeClass
+  public static void startKafkaServer() {
+    kafkaServer = new KafkaSourceEmbeddedKafka(null);
+    startupCheck();
+  }
+
+  @SuppressWarnings("unchecked")
+  @Before
+  public void setup() throws Exception {
+    kafkaSource = new KafkaSource();
+    try {
+      topic0 = findUnusedTopic();
+      kafkaServer.createTopic(topic0, 1);
+      usedTopics.add(topic0);
+      topic1 = findUnusedTopic();
+      kafkaServer.createTopic(topic1, 3);
+      usedTopics.add(topic1);
+    } catch (TopicExistsException e) {
+      //do nothing
+      e.printStackTrace();
+    }
+    context = prepareDefaultContext("flume-group");
+    kafkaSource.setChannelProcessor(createGoodChannel());
+  }
+
+  private static void startupCheck() {
+    String startupTopic = "startupCheck";
+    KafkaConsumer<String, String> startupConsumer;
+    kafkaServer.createTopic(startupTopic, 1);
+    final Properties props = new Properties();
+    props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafkaServer.getBootstrapServers());
+    props.put(ConsumerConfig.GROUP_ID_CONFIG, "group_1");
+    props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName());
+    props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName());
+    KafkaConsumer<String, String> consumer = new KafkaConsumer<>(props);
+    consumer.subscribe(Collections.singletonList(startupTopic));
+    log.info("Checking Startup");
+    boolean success = false;
+    for (int i = 0; i < 20; i++) {
+      kafkaServer.produce(startupTopic, "", "record");
+      ConsumerRecords recs = consumer.poll(Duration.ofMillis(1000L));
+      if (!recs.isEmpty()) {
+        success = true;
+        break;
+      }
+    }
+    if (!success) {
+      fail("Kafka server startup failed");
+    }
+    log.info("Kafka server startup success");
+    consumer.close();
+    kafkaServer.deleteTopics(Collections.singletonList(startupTopic));
+  }
+
+  private Context prepareDefaultContext(String groupId) {
+    Context context = new Context();
+    context.put(BOOTSTRAP_SERVERS, kafkaServer.getBootstrapServers());
+    context.put(KAFKA_CONSUMER_PREFIX + "group.id", groupId);
+    return context;
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    try {
+      kafkaSource.stop();
+    } catch (Exception e) {
+      log.warn("Error stopping kafkaSource", e);
+    }
+    topic0 = null;
+    topic1 = null;
+    kafkaServer.deleteTopics(usedTopics);
+    usedTopics.clear();
+  }
+
+  @AfterClass
+  public static void stopKafkaServer() throws Exception {
+    kafkaServer.stop();
+  }
+
+  private void startKafkaSource() throws EventDeliveryException, InterruptedException {
+    kafkaSource.start();
+    /* Timing magic: We call the process method, that executes a consumer.poll()
+      A thread.sleep(10000L) does not work even though it takes longer */
+    for (int i = 0; i < 3; i++) {
+      kafkaSource.process();
+      Thread.sleep(1000);
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testOffsets() throws InterruptedException, EventDeliveryException {
+    long batchDuration = 2000;
+    context.put(TOPICS, topic1);
+    context.put(BATCH_DURATION_MS,
+            String.valueOf(batchDuration));
+    context.put(BATCH_SIZE, "3");
+    kafkaSource.configure(context);
+    startKafkaSource();
+    Thread.sleep(500L);
+    Status status = kafkaSource.process();
+    assertEquals(Status.BACKOFF, status);
+    assertEquals(0, events.size());
+    kafkaServer.produce(topic1, "", "record1");
+    kafkaServer.produce(topic1, "", "record2");
+    Thread.sleep(500L);
+    status = kafkaSource.process();
+    assertEquals(Status.READY, status);
+    assertEquals(2, events.size());
+    events.clear();
+    kafkaServer.produce(topic1, "", "record3");
+    kafkaServer.produce(topic1, "", "record4");
+    kafkaServer.produce(topic1, "", "record5");
+    Thread.sleep(500L);
+    assertEquals(Status.READY, kafkaSource.process());
+    assertEquals(3, events.size());
+    assertEquals("record3", new String(events.get(0).getBody(), Charsets.UTF_8));
+    assertEquals("record4", new String(events.get(1).getBody(), Charsets.UTF_8));
+    assertEquals("record5", new String(events.get(2).getBody(), Charsets.UTF_8));
+    events.clear();
+    kafkaServer.produce(topic1, "", "record6");
+    kafkaServer.produce(topic1, "", "record7");
+    kafkaServer.produce(topic1, "", "record8");
+    kafkaServer.produce(topic1, "", "record9");
+    kafkaServer.produce(topic1, "", "record10");
+    Thread.sleep(500L);
+    assertEquals(Status.READY, kafkaSource.process());
+    assertEquals(3, events.size());
+    assertEquals("record6", new String(events.get(0).getBody(), Charsets.UTF_8));
+    assertEquals("record7", new String(events.get(1).getBody(), Charsets.UTF_8));
+    assertEquals("record8", new String(events.get(2).getBody(), Charsets.UTF_8));
+    events.clear();
+    kafkaServer.produce(topic1, "", "record11");
+    // status must be READY due to time out exceed.
+    assertEquals(Status.READY, kafkaSource.process());
+    assertEquals(3, events.size());
+    assertEquals("record9", new String(events.get(0).getBody(), Charsets.UTF_8));
+    assertEquals("record10", new String(events.get(1).getBody(), Charsets.UTF_8));
+    assertEquals("record11", new String(events.get(2).getBody(), Charsets.UTF_8));
+    events.clear();
+    kafkaServer.produce(topic1, "", "record12");
+    kafkaServer.produce(topic1, "", "record13");
+    // stop kafka source
+    kafkaSource.stop();
+    // start again
+    kafkaSource = new KafkaSource();
+    kafkaSource.setChannelProcessor(createGoodChannel());
+    kafkaSource.configure(context);
+    startKafkaSource();
+    kafkaServer.produce(topic1, "", "record14");
+    Thread.sleep(1000L);
+    assertEquals(Status.READY, kafkaSource.process());
+    assertEquals(3, events.size());
+    assertEquals("record12", new String(events.get(0).getBody(), Charsets.UTF_8));
+    assertEquals("record13", new String(events.get(1).getBody(), Charsets.UTF_8));
+    assertEquals("record14", new String(events.get(2).getBody(), Charsets.UTF_8));
+    events.clear();
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testProcessItNotEmpty() throws EventDeliveryException,
+          SecurityException, NoSuchFieldException, IllegalArgumentException,
+          IllegalAccessException, InterruptedException {
+    context.put(TOPICS, topic0);
+    context.put(BATCH_SIZE, "1");
+    kafkaSource.configure(context);
+    startKafkaSource();
+
+    Thread.sleep(500L);
+
+    kafkaServer.produce(topic0, "", "hello, world");
+
+    Thread.sleep(500L);
+    Assert.assertEquals(Status.READY, kafkaSource.process());
+    Assert.assertEquals(Status.BACKOFF, kafkaSource.process());
+    Assert.assertEquals(1, events.size());
+
+    Assert.assertEquals("hello, world", new String(events.get(0).getBody(),
+            Charsets.UTF_8));
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testProcessItNotEmptyBatch() throws EventDeliveryException,
+          SecurityException, NoSuchFieldException, IllegalArgumentException,
+          IllegalAccessException, InterruptedException {
+    context.put(TOPICS, topic0);
+    context.put(BATCH_SIZE,"2");
+    kafkaSource.configure(context);
+    startKafkaSource();
+
+    Thread.sleep(500L);
+
+    kafkaServer.produce(topic0, "", "hello, world");
+    kafkaServer.produce(topic0, "", "foo, bar");
+
+    Thread.sleep(500L);
+
+    Status status = kafkaSource.process();
+    assertEquals(Status.READY, status);
+    Assert.assertEquals("hello, world", new String(events.get(0).getBody(),
+            Charsets.UTF_8));
+    Assert.assertEquals("foo, bar", new String(events.get(1).getBody(),
+            Charsets.UTF_8));
+
+  }
+
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testProcessItEmpty() throws EventDeliveryException,
+          SecurityException, NoSuchFieldException, IllegalArgumentException,
+          IllegalAccessException, InterruptedException {
+    context.put(TOPICS, topic0);
+    kafkaSource.configure(context);
+    startKafkaSource();
+    Thread.sleep(500L);
+
+    Status status = kafkaSource.process();
+    assertEquals(Status.BACKOFF, status);
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testNonExistingTopic() throws EventDeliveryException,
+          SecurityException, NoSuchFieldException, IllegalArgumentException,
+          IllegalAccessException, InterruptedException {
+    context.put(TOPICS,"faketopic");
+    kafkaSource.configure(context);
+    startKafkaSource();
+    Thread.sleep(500L);
+
+    assertEquals(LifecycleState.START, kafkaSource.getLifecycleState());
+
+    Status status = kafkaSource.process();
+    assertEquals(Status.BACKOFF, status);
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test(expected = FlumeException.class)
+  public void testNonExistingKafkaServer() throws EventDeliveryException, SecurityException,
+                                                  NoSuchFieldException, IllegalArgumentException,
+                                                  IllegalAccessException, InterruptedException {
+    context.put(TOPICS, topic0);
+    context.put(BOOTSTRAP_SERVERS,"blabla:666");
+    kafkaSource.configure(context);
+    startKafkaSource();
+    Thread.sleep(500L);
+
+    Status status = kafkaSource.process();
+    assertEquals(Status.BACKOFF, status);
+  }
+
+  @Test
+  public void testBatchTime() throws InterruptedException, EventDeliveryException {
+    context.put(TOPICS, topic0);
+    context.put(BATCH_DURATION_MS, "250");
+    kafkaSource.configure(context);
+    startKafkaSource();
+    kafkaSource.process(); // timing magic
+
+    Thread.sleep(500L);
+
+    for (int i = 1; i < 5000; i++) {
+      kafkaServer.produce(topic0, "", "hello, world " + i);
+    }
+    Thread.sleep(500L);
+
+    long error = 50;
+    long startTime = System.currentTimeMillis();
+    Status status = kafkaSource.process();
+    long endTime = System.currentTimeMillis();
+    assertEquals(Status.READY, status);
+    assertTrue(endTime - startTime < (context.getLong(BATCH_DURATION_MS) + error));
+  }
+
+  // Consume event, stop source, start again and make sure we are not
+  // consuming same event again
+  @Test
+  public void testCommit() throws InterruptedException, EventDeliveryException {
+    context.put(TOPICS, topic0);
+    context.put(BATCH_SIZE, "1");
+    kafkaSource.configure(context);
+    startKafkaSource();
+
+    Thread.sleep(500L);
+
+    kafkaServer.produce(topic0, "", "hello, world");
+
+    Thread.sleep(500L);
+
+    Assert.assertEquals(Status.READY, kafkaSource.process());
+    kafkaSource.stop();
+    Thread.sleep(500L);
+    startKafkaSource();
+    Thread.sleep(500L);
+    Assert.assertEquals(Status.BACKOFF, kafkaSource.process());
+  }
+
+  // Remove channel processor and test if we can consume events again
+  @Test
+  public void testNonCommit() throws EventDeliveryException, InterruptedException {
+    context.put(TOPICS, topic0);
+    context.put(BATCH_SIZE,"1");
+    context.put(BATCH_DURATION_MS,"30000");
+    kafkaSource.configure(context);
+    startKafkaSource();
+    Thread.sleep(500L);
+
+    kafkaServer.produce(topic0, "", "hello, world");
+    Thread.sleep(500L);
+
+    kafkaSource.setChannelProcessor(createBadChannel());
+    log.debug("processing from kafka to bad channel");
+    Assert.assertEquals(Status.BACKOFF, kafkaSource.process());
+
+    log.debug("repairing channel");
+    kafkaSource.setChannelProcessor(createGoodChannel());
+
+    log.debug("re-process to good channel - this should work");
+    kafkaSource.process();
+    Assert.assertEquals("hello, world", new String(events.get(0).getBody(), Charsets.UTF_8));
+  }
+
+  @Test
+  public void testTwoBatches() throws InterruptedException, EventDeliveryException {
+    context.put(TOPICS, topic0);
+    context.put(BATCH_SIZE,"1");
+    context.put(BATCH_DURATION_MS, "30000");
+    kafkaSource.configure(context);
+    startKafkaSource();
+    Thread.sleep(500L);
+
+    kafkaServer.produce(topic0, "", "event 1");
+    Thread.sleep(500L);
+
+    kafkaSource.process();
+    Assert.assertEquals("event 1", new String(events.get(0).getBody(), Charsets.UTF_8));
+    events.clear();
+
+    kafkaServer.produce(topic0, "", "event 2");
+    Thread.sleep(500L);
+    kafkaSource.process();
+    Assert.assertEquals("event 2", new String(events.get(0).getBody(), Charsets.UTF_8));
+  }
+
+  @Test
+  public void testTwoBatchesWithAutocommit() throws InterruptedException, EventDeliveryException {
+    context.put(TOPICS, topic0);
+    context.put(BATCH_SIZE,"1");
+    context.put(BATCH_DURATION_MS,"30000");
+    context.put(KAFKA_CONSUMER_PREFIX + "enable.auto.commit", "true");
+    kafkaSource.configure(context);
+    startKafkaSource();
+    Thread.sleep(500L);
+
+    kafkaServer.produce(topic0, "", "event 1");
+    Thread.sleep(500L);
+
+    kafkaSource.process();
+    Assert.assertEquals("event 1", new String(events.get(0).getBody(), Charsets.UTF_8));
+    events.clear();
+
+    kafkaServer.produce(topic0, "", "event 2");
+    Thread.sleep(500L);
+    kafkaSource.process();
+    Assert.assertEquals("event 2", new String(events.get(0).getBody(), Charsets.UTF_8));
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testNullKey() throws EventDeliveryException, SecurityException, NoSuchFieldException,
+                                   IllegalArgumentException, IllegalAccessException,
+                                   InterruptedException {
+    context.put(TOPICS, topic0);
+    context.put(BATCH_SIZE, "1");
+    kafkaSource.configure(context);
+    startKafkaSource();
+
+    Thread.sleep(500L);
+
+    kafkaServer.produce(topic0, null, "hello, world");
+
+    Thread.sleep(500L);
+
+    Assert.assertEquals(Status.READY, kafkaSource.process());
+    Assert.assertEquals(Status.BACKOFF, kafkaSource.process());
+    Assert.assertEquals(1, events.size());
+
+    Assert.assertEquals("hello, world", new String(events.get(0).getBody(), Charsets.UTF_8));
+  }
+
+  @Test
+  public void testErrorCounters() throws InterruptedException, EventDeliveryException {
+    context.put(TOPICS, topic0);
+    context.put(BATCH_SIZE, "1");
+    kafkaSource.configure(context);
+
+    ChannelProcessor cp = Mockito.mock(ChannelProcessor.class);
+    doThrow(new ChannelException("dummy")).doThrow(new RuntimeException("dummy"))
+        .when(cp).processEventBatch(any(List.class));
+    kafkaSource.setChannelProcessor(cp);
+
+    startKafkaSource();
+
+    Thread.sleep(500L);
+
+    kafkaServer.produce(topic0, "", "hello, world");
+
+    Thread.sleep(500L);
+
+    kafkaSource.doProcess();
+    kafkaSource.doProcess();
+
+    SourceCounter sc = (SourceCounter) Whitebox.getInternalState(kafkaSource, "counter");
+    Assert.assertEquals(1, sc.getChannelWriteFail());
+    Assert.assertEquals(1, sc.getEventReadFail());
+
+    kafkaSource.stop();
+  }
+
+
+  @Test
+  public void testSourceProperties() {
+    Context context = new Context();
+    context.put(TOPICS, "test1, test2");
+    context.put(TOPICS_REGEX, "^stream[0-9]$");
+    context.put(BOOTSTRAP_SERVERS, "bootstrap-servers-list");
+    KafkaSource source = new KafkaSource();
+    source.doConfigure(context);
+
+    //check that kafka.topics.regex has higher priority than topics
+    //type of subscriber should be PatternSubscriber
+    KafkaSource.Subscriber<Pattern> subscriber = source.getSubscriber();
+    Pattern pattern = subscriber.get();
+    Assert.assertTrue(pattern.matcher("stream1").find());
+  }
+
+  @Test
+  public void testKafkaProperties() {
+    Context context = new Context();
+    context.put(TOPICS, "test1, test2");
+    context.put(KAFKA_CONSUMER_PREFIX + ConsumerConfig.GROUP_ID_CONFIG,
+                "override.default.group.id");
+    context.put(KAFKA_CONSUMER_PREFIX + "fake.property", "kafka.property.value");
+    context.put(BOOTSTRAP_SERVERS, "real-bootstrap-servers-list");
+    context.put(KAFKA_CONSUMER_PREFIX + "bootstrap.servers", "bad-bootstrap-servers-list");
+    KafkaSource source = new KafkaSource();
+    source.doConfigure(context);
+    Properties kafkaProps = source.getConsumerProps();
+
+    //check that we have defaults set
+    assertEquals(String.valueOf(DEFAULT_AUTO_COMMIT),
+                 kafkaProps.getProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG));
+    //check that kafka properties override the default and get correct name
+    assertEquals("override.default.group.id",
+                 kafkaProps.getProperty(ConsumerConfig.GROUP_ID_CONFIG));
+    //check that any kafka property gets in
+    assertEquals("kafka.property.value",
+                 kafkaProps.getProperty("fake.property"));
+    //check that documented property overrides defaults
+    assertEquals("real-bootstrap-servers-list",
+                 kafkaProps.getProperty("bootstrap.servers"));
+  }
+
+  @Test
+  public void testPatternBasedSubscription() {
+    Context context = new Context();
+
+    context.put(TOPICS_REGEX, "^topic[0-9]$");
+    context.put(BOOTSTRAP_SERVERS, "real-bootstrap-servers-list");
+    KafkaSource source = new KafkaSource();
+    source.doConfigure(context);
+    KafkaSource.Subscriber<Pattern> subscriber = source.getSubscriber();
+    for (int i = 0; i < 10; i++) {
+      Assert.assertTrue(subscriber.get().matcher("topic" + i).find());
+    }
+    Assert.assertFalse(subscriber.get().matcher("topic").find());
+  }
+
+  @Test
+  public void testAvroEvent() throws InterruptedException, EventDeliveryException, IOException {
+    SpecificDatumWriter<AvroFlumeEvent> writer;
+    ByteArrayOutputStream tempOutStream;
+    BinaryEncoder encoder;
+    byte[] bytes;
+
+    context.put(TOPICS, topic0);
+    context.put(BATCH_SIZE, "1");
+    context.put(AVRO_EVENT, "true");
+    kafkaSource.configure(context);
+    startKafkaSource();
+
+    Thread.sleep(500L);
+
+    tempOutStream = new ByteArrayOutputStream();
+    writer = new SpecificDatumWriter<AvroFlumeEvent>(AvroFlumeEvent.class);
+
+    Map<CharSequence, CharSequence> headers = new HashMap<CharSequence,CharSequence>();
+    headers.put("header1", "value1");
+    headers.put("header2", "value2");
+
+    AvroFlumeEvent e = new AvroFlumeEvent(headers, ByteBuffer.wrap("hello, world".getBytes()));
+    encoder = EncoderFactory.get().directBinaryEncoder(tempOutStream, null);
+    writer.write(e, encoder);
+    encoder.flush();
+    bytes = tempOutStream.toByteArray();
+
+    kafkaServer.produce(topic0, "", bytes);
+
+    String currentTimestamp = Long.toString(System.currentTimeMillis());
+
+    headers.put(TIMESTAMP_HEADER, currentTimestamp);
+    headers.put(PARTITION_HEADER, "1");
+    headers.put(DEFAULT_TOPIC_HEADER, "topic0");
+
+    e = new AvroFlumeEvent(headers, ByteBuffer.wrap("hello, world2".getBytes()));
+    tempOutStream.reset();
+    encoder = EncoderFactory.get().directBinaryEncoder(tempOutStream, null);
+    writer.write(e, encoder);
+    encoder.flush();
+    bytes = tempOutStream.toByteArray();
+
+    kafkaServer.produce(topic0, "", bytes);
+
+    Thread.sleep(500L);
+    Assert.assertEquals(Status.READY, kafkaSource.process());
+    Assert.assertEquals(Status.READY, kafkaSource.process());
+    Assert.assertEquals(Status.BACKOFF, kafkaSource.process());
+
+    Assert.assertEquals(2, events.size());
+
+    Event event = events.get(0);
+
+    Assert.assertEquals("hello, world", new String(event.getBody(), Charsets.UTF_8));
+
+    Assert.assertEquals("value1", e.getHeaders().get("header1"));
+    Assert.assertEquals("value2", e.getHeaders().get("header2"));
+
+
+    event = events.get(1);
+
+    Assert.assertEquals("hello, world2", new String(event.getBody(), Charsets.UTF_8));
+
+    Assert.assertEquals("value1", e.getHeaders().get("header1"));
+    Assert.assertEquals("value2", e.getHeaders().get("header2"));
+    Assert.assertEquals(currentTimestamp, e.getHeaders().get(TIMESTAMP_HEADER));
+    Assert.assertEquals(e.getHeaders().get(PARTITION_HEADER), "1");
+    Assert.assertEquals(e.getHeaders().get(DEFAULT_TOPIC_HEADER),"topic0");
+
+  }
+
+  /**
+   * Tests that sub-properties (kafka.consumer.*) apply correctly across multiple invocations
+   * of configure() (fix for FLUME-2857).
+   * @throws Exception
+   */
+  @Test
+  public void testDefaultSettingsOnReConfigure() throws Exception {
+    String sampleConsumerProp = "auto.offset.reset";
+    String sampleConsumerVal = "earliest";
+    String group = "group";
+
+    Context context = prepareDefaultContext(group);
+    context.put(KafkaSourceConstants.KAFKA_CONSUMER_PREFIX + sampleConsumerProp,
+        sampleConsumerVal);
+    context.put(TOPICS, "random-topic");
+
+    kafkaSource.configure(context);
+
+    Assert.assertEquals(sampleConsumerVal,
+        kafkaSource.getConsumerProps().getProperty(sampleConsumerProp));
+
+    context = prepareDefaultContext(group);
+    context.put(TOPICS, "random-topic");
+
+    kafkaSource.configure(context);
+    Assert.assertNull(kafkaSource.getConsumerProps().getProperty(sampleConsumerProp));
+  }
+
+  /**
+   * Tests the availability of the topic header in the output events,
+   * based on the configuration parameters added in FLUME-3046
+   * @throws InterruptedException
+   * @throws EventDeliveryException
+   */
+  @Test
+  public void testTopicHeaderSet() throws InterruptedException, EventDeliveryException {
+    context.put(TOPICS, topic0);
+    kafkaSource.configure(context);
+    startKafkaSource();
+
+    Thread.sleep(500L);
+
+    kafkaServer.produce(topic0, "", "hello, world");
+
+    Thread.sleep(500L);
+
+    Status status = kafkaSource.process();
+    assertEquals(Status.READY, status);
+    Assert.assertEquals("hello, world", new String(events.get(0).getBody(),
+            Charsets.UTF_8));
+
+    Assert.assertEquals(topic0, events.get(0).getHeaders().get("topic"));
+
+    kafkaSource.stop();
+    events.clear();
+  }
+
+  /**
+   * Tests the availability of the custom topic header in the output events,
+   * based on the configuration parameters added in FLUME-3046
+   * @throws InterruptedException
+   * @throws EventDeliveryException
+   */
+  @Test
+  public void testTopicCustomHeaderSet() throws InterruptedException, EventDeliveryException {
+    context.put(TOPICS, topic0);
+    context.put(KafkaSourceConstants.TOPIC_HEADER, "customTopicHeader");
+    kafkaSource.configure(context);
+
+    startKafkaSource();
+
+    Thread.sleep(500L);
+
+    kafkaServer.produce(topic0, "", "hello, world2");
+
+    Thread.sleep(500L);
+
+    Status status = kafkaSource.process();
+    assertEquals(Status.READY, status);
+    Assert.assertEquals("hello, world2", new String(events.get(0).getBody(),
+            Charsets.UTF_8));
+
+    Assert.assertEquals(topic0, events.get(0).getHeaders().get("customTopicHeader"));
+
+    kafkaSource.stop();
+    events.clear();
+  }
+
+
+  /**
+   * Tests the availability of the custom topic header in the output events,
+   * based on the configuration parameters added in FLUME-3046
+   * @throws InterruptedException
+   * @throws EventDeliveryException
+   */
+  @Test
+  public void testTopicKafkaHeaderSet() throws InterruptedException, EventDeliveryException {
+    final String correlatorHeader = "FLUME_CORRELATOR";
+    context.put(TOPICS, topic0);
+    context.put(KafkaSourceConstants.TOPIC_HEADER, "customTopicHeader");
+    context.put(KafkaSourceConstants.KAFKA_HEADER + "correlator", correlatorHeader);
+    context.put(TIMESTAMP_HEADER, "true");
+    kafkaSource.configure(context);
+
+    startKafkaSource();
+
+    Thread.sleep(500L);
+
+    long date = ZonedDateTime.of(2022, 10, 7, 8, 0, 0, 0,
+            ZoneId.systemDefault()).toInstant().toEpochMilli();
+    Headers headers = new RecordHeaders();
+    headers.add(new RecordHeader(correlatorHeader, "12345".getBytes(StandardCharsets.UTF_8)));
+    kafkaServer.produce(topic0, 0, date, "", "hello, world2".getBytes(StandardCharsets.UTF_8),
+        headers);
+
+    Thread.sleep(500L);
+
+    Status status = kafkaSource.process();
+    assertEquals(Status.READY, status);
+    Assert.assertEquals("hello, world2", new String(events.get(0).getBody(),
+        Charsets.UTF_8));
+    Map<String, String> flumeHeaders = events.get(0).getHeaders();
+    Assert.assertEquals(Long.toString(date), flumeHeaders.get("timestamp"));
+    Assert.assertEquals(topic0, flumeHeaders.get("customTopicHeader"));
+    Assert.assertEquals("12345", flumeHeaders.get("correlator"));
+
+    kafkaSource.stop();
+    events.clear();
+  }
+
+
+  /**
+   * Tests the unavailability of the topic header in the output events,
+   * based on the configuration parameters added in FLUME-3046
+   * @throws InterruptedException
+   * @throws EventDeliveryException
+   */
+  @Test
+  public void testTopicCustomHeaderNotSet() throws InterruptedException, EventDeliveryException {
+    context.put(TOPICS, topic0);
+    context.put(KafkaSourceConstants.SET_TOPIC_HEADER, "false");
+    kafkaSource.configure(context);
+
+    startKafkaSource();
+
+    Thread.sleep(500L);
+
+    kafkaServer.produce(topic0, "", "hello, world3");
+
+    Thread.sleep(500L);
+
+    Status status = kafkaSource.process();
+    assertEquals(Status.READY, status);
+    Assert.assertEquals("hello, world3", new String(events.get(0).getBody(),
+            Charsets.UTF_8));
+
+    Assert.assertNull(events.get(0).getHeaders().get("customTopicHeader"));
+
+    kafkaSource.stop();
+  }
+
+  ChannelProcessor createGoodChannel() {
+
+    ChannelProcessor channelProcessor = mock(ChannelProcessor.class);
+
+    events = Lists.newArrayList();
+
+    doAnswer(new Answer<Void>() {
+      @Override
+      public Void answer(InvocationOnMock invocation) throws Throwable {
+        events.addAll((List<Event>)invocation.getArguments()[0]);
+        return null;
+      }
+    }).when(channelProcessor).processEventBatch(any(List.class));
+
+    return channelProcessor;
+  }
+
+  ChannelProcessor createBadChannel() {
+    ChannelProcessor channelProcessor = mock(ChannelProcessor.class);
+
+    doAnswer(new Answer<Void>() {
+      @Override
+      public Void answer(InvocationOnMock invocation) throws Throwable {
+        throw new ChannelException("channel intentional broken");
+      }
+    }).when(channelProcessor).processEventBatch(any(List.class));
+
+    return channelProcessor;
+  }
+
+  public String findUnusedTopic() {
+    String newTopic = null;
+    boolean topicFound = false;
+    while (!topicFound) {
+      newTopic = RandomStringUtils.randomAlphabetic(8);
+      if (!usedTopics.contains(newTopic)) {
+        usedTopics.add(newTopic);
+        topicFound = true;
+      }
+    }
+    return newTopic;
+  }
+
+  private Properties createProducerProps(String bootStrapServers) {
+    Properties props = new Properties();
+    props.put(ProducerConfig.ACKS_CONFIG, "-1");
+    props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG,
+        "org.apache.kafka.common.serialization.StringSerializer");
+    props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG,
+        "org.apache.kafka.common.serialization.ByteArraySerializer");
+    props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, bootStrapServers);
+    return props;
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testSslSource() throws EventDeliveryException,
+          SecurityException,
+          IllegalArgumentException,
+          InterruptedException {
+    context.put(TOPICS, topic0);
+    context.put(BATCH_SIZE, "1");
+    context.put(BOOTSTRAP_SERVERS, kafkaServer.getBootstrapSslServers());
+    context.put(KAFKA_CONSUMER_PREFIX + SECURITY_PROTOCOL_CONFIG, "SSL");
+    context.put(KAFKA_CONSUMER_PREFIX + SSL_TRUSTSTORE_LOCATION_CONFIG, "src/test/resources/truststorefile.jks");
+    context.put(KAFKA_CONSUMER_PREFIX + SSL_TRUSTSTORE_PASSWORD_CONFIG, "password");
+    context.put(KAFKA_CONSUMER_PREFIX + SSL_DISABLE_FQDN_CHECK, "true");
+    kafkaSource.configure(context);
+    startKafkaSource();
+
+    Thread.sleep(500L);
+    kafkaServer.produce(topic0, "", "hello, world");
+    Thread.sleep(500L);
+
+    Assert.assertEquals("", kafkaSource.getConsumerProps().get(SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG));
+    Assert.assertEquals(Status.READY, kafkaSource.process());
+    Assert.assertEquals(1, events.size());
+    Assert.assertEquals("hello, world",
+            new String(events.get(0).getBody(), Charsets.UTF_8)
+    );
+  }
+
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testSslWithFqdnValidationFailedSource() throws EventDeliveryException,
+          SecurityException,
+          IllegalArgumentException,
+          InterruptedException {
+    context.put(TOPICS, topic0);
+    context.put(BATCH_SIZE, "1");
+    context.put(BOOTSTRAP_SERVERS, kafkaServer.getBootstrapSslIpPortServers());
+    context.put(KAFKA_CONSUMER_PREFIX + SECURITY_PROTOCOL_CONFIG, "SSL");
+    context.put(KAFKA_CONSUMER_PREFIX + SSL_TRUSTSTORE_LOCATION_CONFIG, "src/test/resources/truststorefile.jks");
+    context.put(KAFKA_CONSUMER_PREFIX + SSL_TRUSTSTORE_PASSWORD_CONFIG, "password");
+    kafkaSource.configure(context);
+    startKafkaSource();
+
+    Thread.sleep(500L);
+    kafkaServer.produce(topic0, "", "hello, world");
+    Thread.sleep(500L);
+
+    assertNull(kafkaSource.getConsumerProps().get(SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG));
+    Assert.assertEquals(Status.BACKOFF, kafkaSource.process());
+  }
+
+
+}
diff --git a/flume-kafka-source/src/test/resources/keystorefile.jks b/flume-kafka-source/src/test/resources/keystorefile.jks
new file mode 100644
index 0000000..20ac6a8
Binary files /dev/null and b/flume-kafka-source/src/test/resources/keystorefile.jks differ
diff --git a/flume-kafka-source/src/test/resources/log4j2.xml b/flume-kafka-source/src/test/resources/log4j2.xml
new file mode 100644
index 0000000..7caf0d2
--- /dev/null
+++ b/flume-kafka-source/src/test/resources/log4j2.xml
@@ -0,0 +1,31 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements.  See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License.  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+
+-->
+<Configuration status="OFF">
+  <Appenders>
+    <Console name="Console" target="SYSTEM_OUT">
+      <PatternLayout pattern="%d (%t) [%p - %l] %m%n" />
+    </Console>
+  </Appenders>
+  <Loggers>
+    <Logger name="org.apache.flume" level="INFO"/>
+    <Root level="INFO">
+      <AppenderRef ref="Console" />
+    </Root>
+  </Loggers>
+</Configuration>
\ No newline at end of file
diff --git a/flume-kafka-source/src/test/resources/truststorefile.jks b/flume-kafka-source/src/test/resources/truststorefile.jks
new file mode 100644
index 0000000..a98c490
Binary files /dev/null and b/flume-kafka-source/src/test/resources/truststorefile.jks differ
diff --git a/flume-shared/flume-shared-kafka-test/pom.xml b/flume-shared/flume-shared-kafka-test/pom.xml
new file mode 100644
index 0000000..3940ffc
--- /dev/null
+++ b/flume-shared/flume-shared-kafka-test/pom.xml
@@ -0,0 +1,86 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!-- Licensed to the Apache Software Foundation (ASF) under one or more contributor
+  license agreements. See the NOTICE file distributed with this work for additional
+  information regarding copyright ownership. The ASF licenses this file to
+  You under the Apache License, Version 2.0 (the "License"); you may not use
+  this file except in compliance with the License. You may obtain a copy of
+  the License at http://www.apache.org/licenses/LICENSE-2.0 Unless required
+  by applicable law or agreed to in writing, software distributed under the
+  License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS
+  OF ANY KIND, either express or implied. See the License for the specific
+  language governing permissions and limitations under the License. -->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <artifactId>flume-shared</artifactId>
+    <groupId>org.apache.flume</groupId>
+    <version>2.0.0-SNAPSHOT</version>
+  </parent>
+  <groupId>org.apache.flume</groupId>
+  <artifactId>flume-shared-kafka-test</artifactId>
+  <name>Flume Shared Kafka Test Utils</name>
+
+  <properties>
+    <!-- TODO fix spotbugs/pmd violations -->
+    <spotbugs.maxAllowedViolations>1</spotbugs.maxAllowedViolations>
+    <pmd.maxAllowedViolations>1</pmd.maxAllowedViolations>
+    <module.name>org.apache.flume.shared.kafka.test</module.name>
+  </properties>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+        <executions>
+          <execution>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.flume</groupId>
+      <artifactId>flume-ng-sdk</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.flume</groupId>
+      <artifactId>flume-ng-core</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.flume</groupId>
+      <artifactId>flume-ng-configuration</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-api</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.kafka</groupId>
+      <artifactId>kafka_${scala.version}</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.kafka</groupId>
+      <artifactId>kafka-clients</artifactId>
+      <scope>provided</scope>
+    </dependency>
+  </dependencies>
+
+</project>
diff --git a/flume-shared/flume-shared-kafka-test/src/main/java/org/apache/flume/shared/kafka/test/KafkaPartitionTestUtil.java b/flume-shared/flume-shared-kafka-test/src/main/java/org/apache/flume/shared/kafka/test/KafkaPartitionTestUtil.java
new file mode 100644
index 0000000..946c0bb
--- /dev/null
+++ b/flume-shared/flume-shared-kafka-test/src/main/java/org/apache/flume/shared/kafka/test/KafkaPartitionTestUtil.java
@@ -0,0 +1,220 @@
+/**
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements.  See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License.  You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+ limitations under the License.
+ */
+
+package org.apache.flume.shared.kafka.test;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+import org.apache.flume.Event;
+import org.apache.flume.event.EventBuilder;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.common.TopicPartition;
+import org.junit.Assert;
+
+public class KafkaPartitionTestUtil {
+
+  public static final String PARTITION_HEADER = "partition-header";
+
+  /**
+   * This method checks the retrieved messages (passed as resultsMap) against the expected
+   * results (passed as partitionMap). The behaviour of this method is slightly different
+   * depending on the scenario:
+   *  - STATIC_HEADER_ONLY: Don't check partitionMap, just check that all messages are in the
+   *                        passed staticPtn partition.
+   *  - NO_PARTITION_HEADERS: Check that messages are evenly distributed between partitions
+   *                          (requires numMsgs to be a multiple of the number of partitons)
+   *  - else: Check that the contents of each partition list in resultsMap is the same as that
+   *          specified in partitionMap.
+   *
+   *  As this is a testing method, it will issue JUnit AssertionExceptions if the results are not
+   *  as expected.
+   *
+   * @param scenario
+   * @param partitionMap
+   * @param resultsMap
+   * @param staticPtn
+   * @param numMsgs
+   */
+  public static void checkResultsAgainstSkew(PartitionTestScenario scenario,
+      Map<Integer,List<Event>> partitionMap, Map<Integer, List<byte[]>> resultsMap,
+                                 int staticPtn, int numMsgs) {
+    int numPtns = partitionMap.size();
+
+    if (scenario == PartitionTestScenario.NO_PARTITION_HEADERS) {
+      if (numMsgs % numPtns != 0) {
+        throw new IllegalArgumentException("This method is not designed to work with scenarios" +
+                " where there is expected to be a non-even distribution of messages");
+      } else {
+        // Since Kafka 2.4 results with no partition are not distrubuted evenly.
+        int sum = resultsMap.values().stream().mapToInt(List::size).sum();
+        Assert.assertEquals("Scenario: " + scenario + " Incorrect number of messages", numMsgs, sum);
+        return;
+      }
+    }
+    for (int ptn = 0; ptn < numPtns; ptn++) {
+      List<Event> expectedResults = partitionMap.get(ptn);
+      List<byte[]> actualResults = resultsMap.get(ptn);
+      if (scenario == PartitionTestScenario.PARTITION_ID_HEADER_ONLY ||
+          scenario == PartitionTestScenario.STATIC_HEADER_AND_PARTITION_ID) {
+        // In these two scenarios we're checking against partitionMap
+        Assert.assertEquals("Scenario: " + scenario + " Partition " + ptn + " incorrect",
+                expectedResults.size(), actualResults.size());
+        //Go and check the message payload is what we wanted it to be
+        for (int idx = 0; idx < expectedResults.size(); idx++) {
+          Assert.assertArrayEquals("Scenario: " + scenario + " Partition " + ptn + " event " + idx
+                  + " incorrect", expectedResults.get(idx).getBody(), actualResults.get(idx));
+        }
+      } else if (scenario == PartitionTestScenario.STATIC_HEADER_ONLY) {
+        // Check that if we are looking in the statically assigned partition
+        // all messages are in it, else all other partitions are zero
+        if (ptn == staticPtn) {
+          Assert.assertEquals("Scenario: " + scenario + " incorrect number of messages in partition " +
+                  ptn, numMsgs, actualResults.size());
+        } else {
+          Assert.assertEquals("Scenario: " + scenario + " partition " + ptn + "should have no messages",
+                  0, actualResults.size());
+        }
+      }
+    }
+  }
+
+  /**
+   * This method is can be used to create a list of events for use in Kafka partition tests.
+   * Depending on the scenario it will deliberate generate an artificially skewed distribution
+   * of events-per-partition (populated into the passed partitionMap) and then ordered randomly
+   * into the resulting List of events.
+   * Four scenarios are catered for:
+   *  - STATIC_HEADER_ONLY: All events are put into the partition specified by the staticPtn param
+   *  - PARTITION_ID_HEADER_ONLY: Events are skewed into three partitions
+   *  - STATIC_HEADER_AND_PARTITION_ID: Events are skewed into two partitions, with all others
+   *                                    going into the partition specified by staticPtn
+   *  - NO_PARTITION_HEADERS: No partition header is set and the partitionMap is not populated
+   *
+   * @param scenario The scenario being catered for.
+   * @param numMsgs The number of messages to generate
+   * @param partitionMap A map of Integer (partitionId) and List of Events - to be populated
+   * @param numPtns The number of partitions to be populated.
+   * @param staticPtn The static partition to be assigned to.
+   * @return
+   */
+  public static List<Event> generateSkewedMessageList(PartitionTestScenario scenario, int numMsgs,
+      Map<Integer, List<Event>> partitionMap, int numPtns, int staticPtn) {
+    List<Event> msgs = new ArrayList<Event>(numMsgs);
+
+    // Pre-conditions
+    if (numMsgs < 0) {
+      throw new IllegalArgumentException("Number of messages must be greater than zero");
+    }
+    if (staticPtn >= numPtns) {
+      throw new IllegalArgumentException("The static partition must be less than the " +
+                                         "number of partitions");
+    }
+    if (numPtns < 5) {
+      throw new IllegalArgumentException("This method is designed to work with at least 5 " +
+                                         "partitions");
+    }
+    if (partitionMap.size() != numPtns) {
+      throw new IllegalArgumentException("partitionMap has not been correctly initialised");
+    }
+
+    for (int i = 0; i < numMsgs; i++) {
+      Map<String, String> headers = new HashMap<String, String>();
+      Integer partition = null;
+      // Simple code to artificially create a skew. In this scenario, with 5 partitions
+      // and 50 messages we end up with a ratio of 0:0:27:13:10 however these will be
+      // interleaved
+
+      if (scenario == PartitionTestScenario.NO_PARTITION_HEADERS) {
+        // Don't bother adding a header or computing a partition
+      } else if (scenario == PartitionTestScenario.STATIC_HEADER_ONLY) {
+        partition = staticPtn;
+      } else {
+        // We're going to artificially create a skew by putting every 5th event
+        // into partition 4, every 3rd event into partition 3 and everything else into
+        // partition 2 (unless a static partition is provided, in which case we'll
+        // put it into that partition instead, but without setting the header).
+        if (i % 5 == 0) {
+          partition = 4;
+          headers.put(PARTITION_HEADER, String.valueOf(partition));
+        } else if (i % 3 == 0 ) {
+          partition = 3;
+          headers.put(PARTITION_HEADER, String.valueOf(partition));
+        } else if (scenario == PartitionTestScenario.STATIC_HEADER_AND_PARTITION_ID) {
+          // In this case we're not going to set the header, but we are going
+          // to set partition which will then populate the partitionMap
+          partition = staticPtn;
+        } else if (scenario == PartitionTestScenario.PARTITION_ID_HEADER_ONLY) {
+          partition = 2;
+          headers.put(PARTITION_HEADER, String.valueOf(partition));
+        } // Logically no other scenarios
+      }
+
+      // Build the event
+      Event event = EventBuilder.withBody(String.valueOf(i).getBytes(), headers);
+
+      if (scenario != PartitionTestScenario.NO_PARTITION_HEADERS) {
+        // Save into partitionMap
+        partitionMap.get(partition).add(event);
+      }
+
+      // Add to ordered list
+      msgs.add(event);
+
+    }
+    return msgs;
+  }
+
+  /**
+   * Return a map containing one List of records per partition.
+   * This internally creates a Kafka Consumer using the provided consumer properties.
+   *
+   * @param numPtns
+   * @param consumerProperties
+   * @return A Map of Partitions(Integer) and the resulting List of messages (byte[]) retrieved
+   */
+  public static Map<Integer, List<byte[]>> retrieveRecordsFromPartitions(String topic, int numPtns,
+                                                                   Properties consumerProperties) {
+
+    Map<Integer, List<byte[]>> resultsMap = new HashMap<Integer, List<byte[]>>();
+    for (int i = 0; i < numPtns; i++) {
+      List<byte[]> partitionResults = new ArrayList<byte[]>();
+      resultsMap.put(i, partitionResults);
+      KafkaConsumer<String, byte[]> consumer =
+          new KafkaConsumer<String, byte[]>(consumerProperties);
+
+      TopicPartition partition = new TopicPartition(topic, i);
+
+      consumer.assign(Arrays.asList(partition));
+
+      ConsumerRecords<String, byte[]> records = consumer.poll(1000);
+      for (ConsumerRecord<String, byte[]> record : records) {
+        partitionResults.add(record.value());
+      }
+      consumer.close();
+    }
+    return resultsMap;
+  }
+
+}
diff --git a/flume-shared/flume-shared-kafka-test/src/main/java/org/apache/flume/shared/kafka/test/PartitionOption.java b/flume-shared/flume-shared-kafka-test/src/main/java/org/apache/flume/shared/kafka/test/PartitionOption.java
new file mode 100644
index 0000000..97bfe33
--- /dev/null
+++ b/flume-shared/flume-shared-kafka-test/src/main/java/org/apache/flume/shared/kafka/test/PartitionOption.java
@@ -0,0 +1,25 @@
+/**
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements.  See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License.  You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+ limitations under the License.
+ */
+
+package org.apache.flume.shared.kafka.test;
+
+public enum PartitionOption {
+  VALIDBUTOUTOFRANGE,
+  NOTSET,
+  NOTANUMBER
+}
diff --git a/flume-shared/flume-shared-kafka-test/src/main/java/org/apache/flume/shared/kafka/test/PartitionTestScenario.java b/flume-shared/flume-shared-kafka-test/src/main/java/org/apache/flume/shared/kafka/test/PartitionTestScenario.java
new file mode 100644
index 0000000..8cafd14
--- /dev/null
+++ b/flume-shared/flume-shared-kafka-test/src/main/java/org/apache/flume/shared/kafka/test/PartitionTestScenario.java
@@ -0,0 +1,26 @@
+/**
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements.  See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License.  You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+ limitations under the License.
+ */
+
+package org.apache.flume.shared.kafka.test;
+
+public enum PartitionTestScenario {
+  STATIC_HEADER_ONLY,
+  PARTITION_ID_HEADER_ONLY,
+  STATIC_HEADER_AND_PARTITION_ID,
+  NO_PARTITION_HEADERS
+}
\ No newline at end of file
diff --git a/flume-shared/flume-shared-kafka/pom.xml b/flume-shared/flume-shared-kafka/pom.xml
new file mode 100644
index 0000000..dcfc7af
--- /dev/null
+++ b/flume-shared/flume-shared-kafka/pom.xml
@@ -0,0 +1,54 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+
+  <modelVersion>4.0.0</modelVersion>
+
+  <parent>
+    <artifactId>flume-shared</artifactId>
+    <groupId>org.apache.flume</groupId>
+    <version>2.0.0-SNAPSHOT</version>
+  </parent>
+
+  <groupId>org.apache.flume</groupId>
+  <artifactId>flume-shared-kafka</artifactId>
+  <name>Flume Shared Kafka</name>
+
+  <properties>
+    <module.name>org.apache.flume.shared.kafka</module.name>
+  </properties>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.flume</groupId>
+      <artifactId>flume-ng-sdk</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.kafka</groupId>
+      <artifactId>kafka-clients</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+
+</project>
\ No newline at end of file
diff --git a/flume-shared/flume-shared-kafka/src/main/java/org/apache/flume/shared/kafka/KafkaSSLUtil.java b/flume-shared/flume-shared-kafka/src/main/java/org/apache/flume/shared/kafka/KafkaSSLUtil.java
new file mode 100644
index 0000000..4f03008
--- /dev/null
+++ b/flume-shared/flume-shared-kafka/src/main/java/org/apache/flume/shared/kafka/KafkaSSLUtil.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.flume.shared.kafka;
+
+import org.apache.flume.util.SSLUtil;
+import org.apache.kafka.clients.CommonClientConfigs;
+import org.apache.kafka.common.config.SslConfigs;
+import org.apache.kafka.common.security.auth.SecurityProtocol;
+
+import java.util.Properties;
+
+public class KafkaSSLUtil {
+
+  public static final String SSL_DISABLE_FQDN_CHECK = "ssl.disableTLSHostnameVerification";
+
+  private KafkaSSLUtil() {
+  }
+
+  /**
+   * Adds the global keystore/truststore SSL parameters to Kafka properties
+   * if SSL is enabled but the keystore/truststore SSL parameters
+   * are not defined explicitly in Kafka properties.
+   *
+   * @param kafkaProps Kafka properties
+   */
+  public static void addGlobalSSLParameters(Properties kafkaProps) {
+    if (isSSLEnabled(kafkaProps)) {
+      addGlobalSSLParameter(kafkaProps,
+          SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG, SSLUtil.getGlobalKeystorePath());
+      addGlobalSSLParameter(kafkaProps,
+          SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, SSLUtil.getGlobalKeystorePassword());
+      addGlobalSSLParameter(kafkaProps,
+          SslConfigs.SSL_KEYSTORE_TYPE_CONFIG, SSLUtil.getGlobalKeystoreType(null));
+      addGlobalSSLParameter(kafkaProps,
+          SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, SSLUtil.getGlobalTruststorePath());
+      addGlobalSSLParameter(kafkaProps,
+          SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, SSLUtil.getGlobalTruststorePassword());
+      addGlobalSSLParameter(kafkaProps,
+          SslConfigs.SSL_TRUSTSTORE_TYPE_CONFIG, SSLUtil.getGlobalTruststoreType(null));
+    }
+  }
+
+  private static void addGlobalSSLParameter(Properties kafkaProps,
+                                            String propName, String globalValue) {
+    if (!kafkaProps.containsKey(propName) && globalValue != null) {
+      kafkaProps.put(propName, globalValue);
+    }
+  }
+
+  public static boolean isSSLEnabled(Properties kafkaProps) {
+    String securityProtocol =
+        kafkaProps.getProperty(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG);
+
+    return securityProtocol != null &&
+        (securityProtocol.equals(SecurityProtocol.SSL.name) ||
+            securityProtocol.equals(SecurityProtocol.SASL_SSL.name));
+  }
+
+}
diff --git a/flume-shared/flume-shared-kafka/src/test/java/org/apache/flume/shared/kafka/KafkaSSLUtilTest.java b/flume-shared/flume-shared-kafka/src/test/java/org/apache/flume/shared/kafka/KafkaSSLUtilTest.java
new file mode 100644
index 0000000..1119152
--- /dev/null
+++ b/flume-shared/flume-shared-kafka/src/test/java/org/apache/flume/shared/kafka/KafkaSSLUtilTest.java
@@ -0,0 +1,174 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.flume.shared.kafka;
+
+import org.apache.kafka.clients.CommonClientConfigs;
+import org.apache.kafka.common.config.SslConfigs;
+import org.apache.kafka.common.security.auth.SecurityProtocol;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Properties;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+public class KafkaSSLUtilTest {
+
+  @Before
+  public void initSystemProperties() {
+    System.setProperty("javax.net.ssl.keyStore", "global-keystore-path");
+    System.setProperty("javax.net.ssl.keyStorePassword", "global-keystore-password");
+    System.setProperty("javax.net.ssl.keyStoreType", "global-keystore-type");
+    System.setProperty("javax.net.ssl.trustStore", "global-truststore-path");
+    System.setProperty("javax.net.ssl.trustStorePassword", "global-truststore-password");
+    System.setProperty("javax.net.ssl.trustStoreType", "global-truststore-type");
+  }
+
+  @After
+  public void clearSystemProperties() {
+    System.clearProperty("javax.net.ssl.keyStore");
+    System.clearProperty("javax.net.ssl.keyStorePassword");
+    System.clearProperty("javax.net.ssl.keyStoreType");
+    System.clearProperty("javax.net.ssl.trustStore");
+    System.clearProperty("javax.net.ssl.trustStorePassword");
+    System.clearProperty("javax.net.ssl.trustStoreType");
+  }
+
+  @Test
+  public void testSecurityProtocol_PLAINTEXT() {
+    Properties kafkaProps = new Properties();
+    kafkaProps.put(
+        CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, SecurityProtocol.PLAINTEXT.name);
+
+    KafkaSSLUtil.addGlobalSSLParameters(kafkaProps);
+
+    assertNoSSLParameters(kafkaProps);
+  }
+
+  @Test
+  public void testSecurityProtocol_SASL_PLAINTEXT() {
+    Properties kafkaProps = new Properties();
+    kafkaProps.put(
+        CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, SecurityProtocol.SASL_PLAINTEXT.name);
+
+    KafkaSSLUtil.addGlobalSSLParameters(kafkaProps);
+
+    assertNoSSLParameters(kafkaProps);
+  }
+
+  @Test
+  public void testSecurityProtocol_SSL() {
+    Properties kafkaProps = new Properties();
+    kafkaProps.put(
+        CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, SecurityProtocol.SSL.name);
+
+    KafkaSSLUtil.addGlobalSSLParameters(kafkaProps);
+
+    assertGlobalSSLParameters(kafkaProps);
+  }
+
+  @Test
+  public void testSecurityProtocol_SASL_SSL() {
+    Properties kafkaProps = new Properties();
+    kafkaProps.put(
+        CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, SecurityProtocol.SASL_SSL.name);
+
+    KafkaSSLUtil.addGlobalSSLParameters(kafkaProps);
+
+    assertGlobalSSLParameters(kafkaProps);
+  }
+
+  @Test
+  public void testComponentParametersNotOverridden() {
+    Properties kafkaProps = new Properties();
+    kafkaProps.put(
+        CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, SecurityProtocol.SSL.name);
+
+    kafkaProps.put(
+        SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG, "component-keystore-path");
+    kafkaProps.put(
+        SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, "component-keystore-password");
+    kafkaProps.put(
+        SslConfigs.SSL_KEYSTORE_TYPE_CONFIG, "component-keystore-type");
+    kafkaProps.put(
+        SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, "component-truststore-path");
+    kafkaProps.put(
+        SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, "component-truststore-password");
+    kafkaProps.put(
+        SslConfigs.SSL_TRUSTSTORE_TYPE_CONFIG, "component-truststore-type");
+
+    KafkaSSLUtil.addGlobalSSLParameters(kafkaProps);
+
+    assertComponentSSLParameters(kafkaProps);
+  }
+
+  @Test
+  public void testEmptyGlobalParametersNotAdded() {
+    Properties kafkaProps = new Properties();
+    kafkaProps.put(
+        CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, SecurityProtocol.SSL.name);
+
+    clearSystemProperties();
+
+    KafkaSSLUtil.addGlobalSSLParameters(kafkaProps);
+
+    assertNoSSLParameters(kafkaProps);
+  }
+
+  private void assertNoSSLParameters(Properties kafkaProps) {
+    assertFalse(kafkaProps.containsKey(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG));
+    assertFalse(kafkaProps.containsKey(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG));
+    assertFalse(kafkaProps.containsKey(SslConfigs.SSL_KEYSTORE_TYPE_CONFIG));
+    assertFalse(kafkaProps.containsKey(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG));
+    assertFalse(kafkaProps.containsKey(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG));
+    assertFalse(kafkaProps.containsKey(SslConfigs.SSL_TRUSTSTORE_TYPE_CONFIG));
+  }
+
+  private void assertGlobalSSLParameters(Properties kafkaProps) {
+    assertEquals("global-keystore-path",
+        kafkaProps.getProperty(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG));
+    assertEquals("global-keystore-password",
+        kafkaProps.getProperty(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG));
+    assertEquals("global-keystore-type",
+        kafkaProps.getProperty(SslConfigs.SSL_KEYSTORE_TYPE_CONFIG));
+    assertEquals("global-truststore-path",
+        kafkaProps.getProperty(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG));
+    assertEquals("global-truststore-password",
+        kafkaProps.getProperty(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG));
+    assertEquals("global-truststore-type",
+        kafkaProps.getProperty(SslConfigs.SSL_TRUSTSTORE_TYPE_CONFIG));
+  }
+
+  private void assertComponentSSLParameters(Properties kafkaProps) {
+    assertEquals("component-keystore-path",
+        kafkaProps.getProperty(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG));
+    assertEquals("component-keystore-password",
+        kafkaProps.getProperty(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG));
+    assertEquals("component-keystore-type",
+        kafkaProps.getProperty(SslConfigs.SSL_KEYSTORE_TYPE_CONFIG));
+    assertEquals("component-truststore-path",
+        kafkaProps.getProperty(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG));
+    assertEquals("component-truststore-password",
+        kafkaProps.getProperty(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG));
+    assertEquals("component-truststore-type",
+        kafkaProps.getProperty(SslConfigs.SSL_TRUSTSTORE_TYPE_CONFIG));
+  }
+}
diff --git a/flume-shared/pom.xml b/flume-shared/pom.xml
new file mode 100644
index 0000000..377715f
--- /dev/null
+++ b/flume-shared/pom.xml
@@ -0,0 +1,40 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+
+  <modelVersion>4.0.0</modelVersion>
+
+  <parent>
+    <artifactId>flume-kafka-parent</artifactId>
+    <groupId>org.apache.flume</groupId>
+    <version>2.0.0-SNAPSHOT</version>
+  </parent>
+
+  <groupId>org.apache.flume</groupId>
+  <artifactId>flume-shared</artifactId>
+  <name>Flume Shared Utils</name>
+  <packaging>pom</packaging>
+
+
+  <modules>
+    <module>flume-shared-kafka</module>
+    <module>flume-shared-kafka-test</module>
+  </modules>
+
+</project>
diff --git a/pom.xml b/pom.xml
new file mode 100644
index 0000000..dc9a20c
--- /dev/null
+++ b/pom.xml
@@ -0,0 +1,384 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+
+  <parent>
+    <groupId>org.apache</groupId>
+    <artifactId>apache</artifactId>
+    <version>29</version>
+  </parent>
+
+  <groupId>org.apache.flume</groupId>
+  <artifactId>flume-kafka-parent</artifactId>
+  <name>Flume Kafka Parent</name>
+  <version>2.0.0-SNAPSHOT</version>
+  <packaging>pom</packaging>
+
+  <properties>
+    <ReleaseVersion>2.0.0</ReleaseVersion>
+    <ReleaseManager>Ralph Goers</ReleaseManager>
+    <ReleaseKey>B3D8E1BA</ReleaseKey>
+    <SigningUserName>rgoers@apache.org</SigningUserName>
+    <checksum-maven-plugin.version>1.11</checksum-maven-plugin.version>
+    <dropwizard-metrics.version>4.1.18</dropwizard-metrics.version>
+    <findsecbugs-plugin.version>1.12.0</findsecbugs-plugin.version>
+    <flume-bom.version>1.11.1-SNAPSHOT</flume-bom.version>
+    <junit.version>4.13.2</junit.version>
+    <kafka.version>3.3.1</kafka.version>
+    <log4j.version>2.20.0</log4j.version>
+    <maven.compiler.source>1.8</maven.compiler.source>
+    <maven.compiler.target>1.8</maven.compiler.target>
+    <mockito.version>1.9.0</mockito.version>
+    <module.name>org.apache.flume.kafka</module.name>
+    <mvn-gpg-plugin.version>1.6</mvn-gpg-plugin.version>
+    <mvn-javadoc-plugin.version>2.9</mvn-javadoc-plugin.version>
+    <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+    <rat.version>0.12</rat.version>
+    <scala.version>2.13</scala.version>
+    <scala-library.version>2.13.9</scala-library.version>
+    <slf4j.version>1.7.32</slf4j.version>
+    <spotbugs-maven-plugin.version>4.7.2.1</spotbugs-maven-plugin.version>
+    <spotless-maven-plugin.version>2.27.2</spotless-maven-plugin.version>
+    <zookeeper.version>3.6.2</zookeeper.version>
+  </properties>
+  <dependencyManagement>
+    <dependencies>
+      <dependency>
+        <groupId>org.apache.flume</groupId>
+        <artifactId>flume-dependencies</artifactId>
+        <version>${flume-bom.version}</version>
+        <scope>import</scope>
+        <type>pom</type>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.flume</groupId>
+        <artifactId>flume-shared-kafka</artifactId>
+        <version>${project.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.flume</groupId>
+        <artifactId>flume-shared-kafka-test</artifactId>
+        <scope>test</scope>
+        <version>${project.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.flume</groupId>
+        <artifactId>flume-kafka-channel</artifactId>
+        <version>${project.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.flume</groupId>
+        <artifactId>flume-kafka-source</artifactId>
+        <version>${project.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.flume</groupId>
+        <artifactId>flume-kafka-sink</artifactId>
+        <version>${project.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.flume</groupId>
+        <artifactId>flume-kafka-sink</artifactId>
+        <version>${project.version}</version>
+        <classifier>tests</classifier>
+        <scope>test</scope>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.kafka</groupId>
+        <artifactId>kafka_${scala.version}</artifactId>
+        <version>${kafka.version}</version>
+        <exclusions>
+          <exclusion>
+            <groupId>log4j</groupId>
+            <artifactId>log4j</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-log4j12</artifactId>
+          </exclusion>
+        </exclusions>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.kafka</groupId>
+        <artifactId>kafka_${scala.version}</artifactId>
+        <version>${kafka.version}</version>
+        <classifier>test</classifier>
+        <scope>test</scope>
+        <exclusions>
+          <exclusion>
+            <groupId>log4j</groupId>
+            <artifactId>log4j</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-log4j12</artifactId>
+          </exclusion>
+        </exclusions>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.kafka</groupId>
+        <artifactId>kafka-clients</artifactId>
+        <version>${kafka.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.scala-lang</groupId>
+        <artifactId>scala-library</artifactId>
+        <version>${scala-library.version}</version>
+      </dependency>
+
+      <dependency>
+        <groupId>io.dropwizard.metrics</groupId>
+        <artifactId>metrics-core</artifactId>
+        <version>${dropwizard-metrics.version}</version>
+      </dependency>
+
+      <dependency>
+        <groupId>org.apache.zookeeper</groupId>
+        <artifactId>zookeeper</artifactId>
+        <version>${zookeeper.version}</version>
+        <exclusions>
+          <exclusion>
+            <groupId>log4j</groupId>
+            <artifactId>log4j</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-log4j12</artifactId>
+          </exclusion>
+        </exclusions>
+      </dependency>
+
+      <dependency>
+        <groupId>org.apache.logging.log4j</groupId>
+        <artifactId>log4j-api</artifactId>
+        <version>${log4j.version}</version>
+      </dependency>
+
+      <dependency>
+        <groupId>org.apache.logging.log4j</groupId>
+        <artifactId>log4j-core</artifactId>
+        <version>${log4j.version}</version>
+      </dependency>
+
+      <dependency>
+        <groupId>org.apache.logging.log4j</groupId>
+        <artifactId>log4j-slf4j-impl</artifactId>
+        <version>${log4j.version}</version>
+        <scope>test</scope>
+      </dependency>
+
+      <dependency>
+        <groupId>org.apache.logging.log4j</groupId>
+        <artifactId>log4j-1.2-api</artifactId>
+        <version>${log4j.version}</version>
+      </dependency>
+
+      <dependency>
+        <groupId>org.apache.logging.log4j</groupId>
+        <artifactId>log4j-jcl</artifactId>
+        <version>${log4j.version}</version>
+      </dependency>
+
+      <dependency>
+        <groupId>org.apache.logging.log4j</groupId>
+        <artifactId>log4j-jul</artifactId>
+        <version>${log4j.version}</version>
+      </dependency>
+
+      <dependency>
+        <groupId>org.slf4j</groupId>
+        <artifactId>slf4j-api</artifactId>
+        <version>${slf4j.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>junit</groupId>
+        <artifactId>junit</artifactId>
+        <version>${junit.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.mockito</groupId>
+        <artifactId>mockito-all</artifactId>
+        <version>${mockito.version}</version>
+        <scope>test</scope>
+      </dependency>
+    </dependencies>
+  </dependencyManagement>
+
+  <inceptionYear>2022</inceptionYear>
+
+  <issueManagement>
+    <system>JIRA</system>
+    <url>https://issues.apache.org/jira/browse/FLUME</url>
+  </issueManagement>
+
+  <licenses>
+    <license>
+      <name>The Apache Software License, Version 2.0</name>
+      <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url>
+    </license>
+  </licenses>
+
+  <mailingLists>
+    <mailingList>
+      <archive>http://mail-archives.apache.org/mod_mbox/flume-user/</archive>
+      <name>Flume User List</name>
+      <post>user@flume.apache.org</post>
+      <subscribe>user-subscribe@flume.apache.org</subscribe>
+      <unsubscribe>user-unsubscribe@flume.apache.org</unsubscribe>
+    </mailingList>
+    <mailingList>
+      <archive>http://mail-archives.apache.org/mod_mbox/flume-dev/</archive>
+      <name>Flume Developer List</name>
+      <post>dev@flume.apache.org</post>
+      <subscribe>dev-subscribe@flume.apache.org</subscribe>
+      <unsubscribe>dev-unsubscribe@flume.apache.org</unsubscribe>
+    </mailingList>
+    <mailingList>
+      <archive>http://mail-archives.apache.org/mod_mbox/flume-commits/</archive>
+      <name>Flume Commits</name>
+      <post>commits@flume.apache.org</post>
+      <subscribe>commits-subscribe@flume.apache.org</subscribe>
+      <unsubscribe>commits-unsubscribe@flume.apache.org</unsubscribe>
+    </mailingList>
+  </mailingLists>
+
+  <scm>
+    <url>https://gitbox.apache.org/repos/asf/flume-spring-boot.git</url>
+    <developerConnection>https://gitbox.apache.org/repos/asf/flume-spring-boot.git</developerConnection>
+    <connection>https://gitbox.apache.org/repos/asf/flume-spring-boot.git</connection>
+  </scm>
+
+  <developers>
+    <developer>
+      <name>Ralph Goers</name>
+      <id>rgoers</id>
+      <email>rgoers@apache.org</email>
+      <organization>Intuit</organization>
+    </developer>
+  </developers>
+
+  <organization>
+    <name>Apache Software Foundation</name>
+    <url>http://www.apache.org</url>
+  </organization>
+  <modules>
+    <module>flume-shared</module>
+    <module>flume-kafka-channel</module>
+    <module>flume-kafka-sink</module>
+    <module>flume-kafka-source</module>
+  </modules>
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.rat</groupId>
+        <artifactId>apache-rat-plugin</artifactId>
+        <version>${rat.version}</version>
+        <configuration>
+          <excludes>
+            <exclude>**/.idea/</exclude>
+            <exclude>**/*.iml</exclude>
+            <exclude>src/main/resources/META-INF/services/**/*</exclude>
+            <exclude>**/nb-configuration.xml</exclude>
+            <exclude>.git/</exclude>
+            <exclude>patchprocess/</exclude>
+            <exclude>.gitignore</exclude>
+            <exclude>**/*.yml</exclude>
+            <exclude>**/*.yaml</exclude>
+            <exclude>**/*.json</exclude>
+            <!-- ASF jenkins box puts the Maven repo in our root directory. -->
+            <exclude>.repository/</exclude>
+            <exclude>**/*.diff</exclude>
+            <exclude>**/*.patch</exclude>
+            <exclude>**/*.avsc</exclude>
+            <exclude>**/*.avro</exclude>
+            <exclude>**/docs/**</exclude>
+            <exclude>**/test/resources/**</exclude>
+            <exclude>**/.settings/*</exclude>
+            <exclude>**/.classpath</exclude>
+            <exclude>**/.project</exclude>
+            <exclude>**/target/**</exclude>
+            <exclude>**/derby.log</exclude>
+            <exclude>**/metastore_db/</exclude>
+            <exclude>.mvn/**</exclude>
+            <exclude>**/exclude-pmd.properties</exclude>
+          </excludes>
+          <consoleOutput>true</consoleOutput>
+        </configuration>
+        <executions>
+          <execution>
+            <id>verify.rat</id>
+            <phase>verify</phase>
+            <goals>
+              <goal>check</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+
+  <profiles>
+    <profile>
+      <id>release</id>
+      <modules>
+        <module>flume-kafka-dist</module>
+      </modules>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-gpg-plugin</artifactId>
+            <version>${mvn-gpg-plugin.version}</version>
+            <executions>
+              <execution>
+                <phase>verify</phase>
+                <goals>
+                  <goal>sign</goal>
+                </goals>
+                <configuration>
+                  <keyname>${SigningUserName}</keyname>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-javadoc-plugin</artifactId>
+            <version>${mvn-javadoc-plugin.version}</version>
+            <executions>
+              <execution>
+                <id>javadoc-jar</id>
+                <phase>package</phase>
+                <goals>
+                  <goal>aggregate-jar</goal>
+                </goals>
+              </execution>
+            </executions>
+            <configuration>
+              <additionalparam>-Xdoclint:none</additionalparam>
+            </configuration>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+  </profiles>
+
+</project>