You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by bb...@apache.org on 2015/11/10 18:28:55 UTC

[2/3] nifi git commit: NIFI-817 Processors for interacting with HBase - Refactoring PutHBaseCell to batch Puts by table - Adding optional Columns property to GetHBase to return only selected column families or columns - Making GetHBase cluster friendly b

http://git-wip-us.apache.org/repos/asf/nifi/blob/e748fd58/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/test/java/org/apache/nifi/hbase/TestGetHBase.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/test/java/org/apache/nifi/hbase/TestGetHBase.java b/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/test/java/org/apache/nifi/hbase/TestGetHBase.java
new file mode 100644
index 0000000..92f42f2
--- /dev/null
+++ b/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/test/java/org/apache/nifi/hbase/TestGetHBase.java
@@ -0,0 +1,459 @@
+/*
+ * 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.nifi.hbase;
+
+import org.apache.nifi.annotation.notification.PrimaryNodeState;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.distributed.cache.client.Deserializer;
+import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient;
+import org.apache.nifi.distributed.cache.client.Serializer;
+import org.apache.nifi.hbase.scan.Column;
+import org.apache.nifi.hbase.util.StringSerDe;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+public class TestGetHBase {
+
+    private TestRunner runner;
+    private MockGetHBase proc;
+    private MockCacheClient cacheClient;
+    private MockHBaseClientService hBaseClient;
+
+    @Before
+    public void setup() throws InitializationException {
+        proc = new MockGetHBase();
+        runner = TestRunners.newTestRunner(proc);
+
+        cacheClient = new MockCacheClient();
+        runner.addControllerService("cacheClient", cacheClient);
+        runner.enableControllerService(cacheClient);
+
+        hBaseClient = new MockHBaseClientService();
+        runner.addControllerService("hbaseClient", hBaseClient);
+        runner.enableControllerService(hBaseClient);
+
+        runner.setProperty(GetHBase.TABLE_NAME, "nifi");
+        runner.setProperty(GetHBase.DISTRIBUTED_CACHE_SERVICE, "cacheClient");
+        runner.setProperty(GetHBase.HBASE_CLIENT_SERVICE, "hbaseClient");
+    }
+
+    @After
+    public void cleanup() {
+        final File file = proc.getStateFile();
+        if (file.exists()) {
+            file.delete();
+        }
+        Assert.assertFalse(file.exists());
+    }
+
+    @Test
+    public void testColumnsValidation() {
+        runner.assertValid();
+
+        runner.setProperty(GetHBase.COLUMNS, "cf1:cq1");
+        runner.assertValid();
+
+        runner.setProperty(GetHBase.COLUMNS, "cf1");
+        runner.assertValid();
+
+        runner.setProperty(GetHBase.COLUMNS, "cf1:cq1,cf2:cq2,cf3:cq3");
+        runner.assertValid();
+
+        runner.setProperty(GetHBase.COLUMNS, "cf1,cf2:cq1,cf3");
+        runner.assertValid();
+
+        runner.setProperty(GetHBase.COLUMNS, "cf1 cf2,cf3");
+        runner.assertNotValid();
+
+        runner.setProperty(GetHBase.COLUMNS, "cf1:,cf2,cf3");
+        runner.assertNotValid();
+
+        runner.setProperty(GetHBase.COLUMNS, "cf1:cq1,");
+        runner.assertNotValid();
+    }
+
+    @Test
+    public void testRowCounts() {
+        final long now = System.currentTimeMillis();
+
+        final Map<String, String> cells = new HashMap<>();
+        cells.put("greeting", "hello");
+        cells.put("name", "nifi");
+
+        hBaseClient.addResult("row0", cells, now - 2);
+        hBaseClient.addResult("row1", cells, now - 1);
+        hBaseClient.addResult("row2", cells, now - 1);
+        hBaseClient.addResult("row3", cells, now);
+
+        runner.run(100);
+        runner.assertAllFlowFilesTransferred(GetHBase.REL_SUCCESS, 4);
+
+        hBaseClient.addResult("row4", cells, now + 1);
+        runner.run();
+        runner.assertAllFlowFilesTransferred(GetHBase.REL_SUCCESS, 5);
+    }
+
+    @Test
+    public void testPersistAndRecoverFromLocalState() throws InitializationException {
+        final File stateFile = new File("target/test-recover-state.bin");
+        if (!stateFile.delete() && stateFile.exists()) {
+            Assert.fail("Could not delete state file " + stateFile);
+        }
+        proc.setStateFile(stateFile);
+
+        final long now = System.currentTimeMillis();
+
+        final Map<String, String> cells = new HashMap<>();
+        cells.put("greeting", "hello");
+        cells.put("name", "nifi");
+
+        hBaseClient.addResult("row0", cells, now - 2);
+        hBaseClient.addResult("row1", cells, now - 1);
+        hBaseClient.addResult("row2", cells, now - 1);
+        hBaseClient.addResult("row3", cells, now);
+
+        runner.run(100);
+        runner.assertAllFlowFilesTransferred(GetHBase.REL_SUCCESS, 4);
+
+        hBaseClient.addResult("row4", cells, now + 1);
+        runner.run();
+        runner.assertAllFlowFilesTransferred(GetHBase.REL_SUCCESS, 5);
+
+        proc = new MockGetHBase(stateFile);
+        final TestRunner newRunner = TestRunners.newTestRunner(proc);
+
+        newRunner.addControllerService("cacheClient", cacheClient);
+        newRunner.enableControllerService(cacheClient);
+
+        newRunner.addControllerService("hbaseClient", hBaseClient);
+        newRunner.enableControllerService(hBaseClient);
+
+        newRunner.setProperty(GetHBase.TABLE_NAME, "nifi");
+        newRunner.setProperty(GetHBase.DISTRIBUTED_CACHE_SERVICE, "cacheClient");
+        newRunner.setProperty(GetHBase.HBASE_CLIENT_SERVICE, "hbaseClient");
+
+        hBaseClient.addResult("row0", cells, now - 2);
+        hBaseClient.addResult("row1", cells, now - 1);
+        hBaseClient.addResult("row2", cells, now - 1);
+        hBaseClient.addResult("row3", cells, now);
+
+        newRunner.run(100);
+        newRunner.assertAllFlowFilesTransferred(GetHBase.REL_SUCCESS, 0);
+    }
+
+    @Test
+    public void testBecomePrimaryWithNoLocalState() throws InitializationException {
+        final long now = System.currentTimeMillis();
+
+        final Map<String, String> cells = new HashMap<>();
+        cells.put("greeting", "hello");
+        cells.put("name", "nifi");
+
+        hBaseClient.addResult("row0", cells, now - 2);
+        hBaseClient.addResult("row1", cells, now - 1);
+        hBaseClient.addResult("row2", cells, now - 1);
+        hBaseClient.addResult("row3", cells, now);
+
+        runner.run(100);
+        runner.assertAllFlowFilesTransferred(GetHBase.REL_SUCCESS, 4);
+
+        hBaseClient.addResult("row4", cells, now + 1);
+        runner.run();
+        runner.assertAllFlowFilesTransferred(GetHBase.REL_SUCCESS, 5);
+
+        // delete the processor's local state to simulate becoming the primary node
+        // for the first time, should use the state from distributed cache
+        final File stateFile = proc.getStateFile();
+        if (!stateFile.delete() && stateFile.exists()) {
+            Assert.fail("Could not delete state file " + stateFile);
+        }
+        proc.onPrimaryNodeChange(PrimaryNodeState.ELECTED_PRIMARY_NODE);
+
+        hBaseClient.addResult("row0", cells, now - 2);
+        hBaseClient.addResult("row1", cells, now - 1);
+        hBaseClient.addResult("row2", cells, now - 1);
+        hBaseClient.addResult("row3", cells, now);
+        hBaseClient.addResult("row4", cells, now + 1);
+
+        runner.clearTransferState();
+        runner.run(100);
+        runner.assertAllFlowFilesTransferred(GetHBase.REL_SUCCESS, 0);
+    }
+
+    @Test
+    public void testBecomePrimaryWithNewerLocalState() throws InitializationException {
+        final long now = System.currentTimeMillis();
+
+        final Map<String, String> cells = new HashMap<>();
+        cells.put("greeting", "hello");
+        cells.put("name", "nifi");
+
+        hBaseClient.addResult("row0", cells, now - 2);
+        hBaseClient.addResult("row1", cells, now - 1);
+        hBaseClient.addResult("row2", cells, now - 1);
+        hBaseClient.addResult("row3", cells, now);
+
+        runner.run(100);
+        runner.assertAllFlowFilesTransferred(GetHBase.REL_SUCCESS, 4);
+
+        // trick for testing so that row4 gets written to local state but not to the real cache
+        final MockCacheClient otherCacheClient = new MockCacheClient();
+        runner.addControllerService("otherCacheClient", otherCacheClient);
+        runner.enableControllerService(otherCacheClient);
+        runner.setProperty(GetHBase.DISTRIBUTED_CACHE_SERVICE, "otherCacheClient");
+
+        hBaseClient.addResult("row4", cells, now + 1);
+        runner.run();
+        runner.assertAllFlowFilesTransferred(GetHBase.REL_SUCCESS, 5);
+
+        // set back the original cache cacheClient which is missing row4
+        runner.setProperty(GetHBase.DISTRIBUTED_CACHE_SERVICE, "cacheClient");
+
+        // become the primary node, but we have existing local state with rows 0-4
+        // so we shouldn't get any output because we should use the local state
+        proc.onPrimaryNodeChange(PrimaryNodeState.ELECTED_PRIMARY_NODE);
+
+        hBaseClient.addResult("row0", cells, now - 2);
+        hBaseClient.addResult("row1", cells, now - 1);
+        hBaseClient.addResult("row2", cells, now - 1);
+        hBaseClient.addResult("row3", cells, now);
+        hBaseClient.addResult("row4", cells, now + 1);
+
+        runner.clearTransferState();
+        runner.run(100);
+        runner.assertAllFlowFilesTransferred(GetHBase.REL_SUCCESS, 0);
+    }
+
+    @Test
+    public void testOnRemovedClearsState() throws IOException {
+        final long now = System.currentTimeMillis();
+
+        final Map<String, String> cells = new HashMap<>();
+        cells.put("greeting", "hello");
+        cells.put("name", "nifi");
+
+        hBaseClient.addResult("row0", cells, now - 2);
+        hBaseClient.addResult("row1", cells, now - 1);
+        hBaseClient.addResult("row2", cells, now - 1);
+        hBaseClient.addResult("row3", cells, now);
+
+        runner.run(100);
+        runner.assertAllFlowFilesTransferred(GetHBase.REL_SUCCESS, 4);
+
+        // should have a local state file and a cache entry before removing
+        Assert.assertTrue(proc.getStateFile().exists());
+        Assert.assertTrue(cacheClient.containsKey(proc.getKey(), new StringSerDe()));
+
+        proc.onRemoved(runner.getProcessContext());
+
+        // onRemoved should have cleared both
+        Assert.assertFalse(proc.getStateFile().exists());
+        Assert.assertFalse(cacheClient.containsKey(proc.getKey(), new StringSerDe()));
+    }
+
+    @Test
+    public void testChangeTableNameClearsState() {
+        final long now = System.currentTimeMillis();
+
+        final Map<String, String> cells = new HashMap<>();
+        cells.put("greeting", "hello");
+        cells.put("name", "nifi");
+
+        hBaseClient.addResult("row0", cells, now - 2);
+        hBaseClient.addResult("row1", cells, now - 1);
+        hBaseClient.addResult("row2", cells, now - 1);
+        hBaseClient.addResult("row3", cells, now);
+
+        runner.run(100);
+        runner.assertAllFlowFilesTransferred(GetHBase.REL_SUCCESS, 4);
+
+        // change the table name and run again, should get all the data coming out
+        // again because previous state will be wiped
+        runner.setProperty(GetHBase.TABLE_NAME, "otherTable");
+
+        hBaseClient.addResult("row0", cells, now - 2);
+        hBaseClient.addResult("row1", cells, now - 1);
+        hBaseClient.addResult("row2", cells, now - 1);
+        hBaseClient.addResult("row3", cells, now);
+
+        runner.run(100);
+        runner.assertAllFlowFilesTransferred(GetHBase.REL_SUCCESS, 4);
+    }
+
+    @Test
+    public void testInitialTimeCurrentTime() {
+        runner.setProperty(GetHBase.INITIAL_TIMERANGE, GetHBase.CURRENT_TIME);
+
+        final long now = System.currentTimeMillis();
+
+        final Map<String, String> cells = new HashMap<>();
+        cells.put("greeting", "hello");
+        cells.put("name", "nifi");
+
+        hBaseClient.addResult("row0", cells, now - 4000);
+        hBaseClient.addResult("row1", cells, now - 3000);
+        hBaseClient.addResult("row2", cells, now - 2000);
+        hBaseClient.addResult("row3", cells, now - 1000);
+
+        // should not get any output because the mock results have a time before current time
+        runner.run(100);
+        runner.assertAllFlowFilesTransferred(GetHBase.REL_SUCCESS, 0);
+    }
+
+    @Test
+    public void testParseColumns() {
+        runner.setProperty(GetHBase.COLUMNS, "cf1,cf2:cq1,cf3");
+        proc.parseColumns(runner.getProcessContext());
+
+        final List<Column> expectedCols = new ArrayList<>();
+        expectedCols.add(new Column("cf1".getBytes(Charset.forName("UTF-8")), null));
+        expectedCols.add(new Column("cf2".getBytes(Charset.forName("UTF-8")), "cq1".getBytes(Charset.forName("UTF-8"))));
+        expectedCols.add(new Column("cf3".getBytes(Charset.forName("UTF-8")), null));
+
+        final List<Column> actualColumns = proc.getColumns();
+        Assert.assertNotNull(actualColumns);
+        Assert.assertEquals(expectedCols.size(), actualColumns.size());
+
+        for (final Column expectedCol : expectedCols) {
+            boolean found = false;
+            for (final Column providedCol : actualColumns) {
+                if (expectedCol.equals(providedCol)) {
+                    found = true;
+                    break;
+                }
+            }
+            Assert.assertTrue("Didn't find expected column", found);
+        }
+    }
+
+    @Test
+    public void testCustomValidate() throws CharacterCodingException {
+        runner.setProperty(GetHBase.FILTER_EXPRESSION, "PrefixFilter ('Row') AND PageFilter (1) AND FirstKeyOnlyFilter ()");
+        runner.assertValid();
+
+        runner.setProperty(GetHBase.COLUMNS, "colA");
+        runner.assertNotValid();
+    }
+
+    // Mock processor to override the location of the state file
+    private static class MockGetHBase extends GetHBase {
+
+        private static final String DEFAULT_STATE_FILE_NAME = "target/TestGetHBase.bin";
+
+        private File stateFile;
+
+        public MockGetHBase() {
+            this(new File(DEFAULT_STATE_FILE_NAME));
+        }
+
+        public MockGetHBase(final File stateFile) {
+            this.stateFile = stateFile;
+        }
+
+        public void setStateFile(final File stateFile) {
+            this.stateFile = stateFile;
+        }
+
+        @Override
+        protected int getBatchSize() {
+            return 2;
+        }
+
+        @Override
+        protected File getStateDir() {
+            return new File("target");
+        }
+
+        @Override
+        protected File getStateFile() {
+            return stateFile;
+        }
+    }
+
+    private class MockCacheClient extends AbstractControllerService implements DistributedMapCacheClient {
+        private final ConcurrentMap<Object, Object> values = new ConcurrentHashMap<>();
+        private boolean failOnCalls = false;
+
+        private void verifyNotFail() throws IOException {
+            if ( failOnCalls ) {
+                throw new IOException("Could not call to remote cacheClient because Unit Test marked cacheClient unavailable");
+            }
+        }
+
+        @Override
+        public <K, V> boolean putIfAbsent(final K key, final V value, final Serializer<K> keySerializer, final Serializer<V> valueSerializer) throws IOException {
+            verifyNotFail();
+            final Object retValue = values.putIfAbsent(key, value);
+            return (retValue == null);
+        }
+
+        @Override
+        @SuppressWarnings("unchecked")
+        public <K, V> V getAndPutIfAbsent(final K key, final V value, final Serializer<K> keySerializer, final Serializer<V> valueSerializer,
+                                          final Deserializer<V> valueDeserializer) throws IOException {
+            verifyNotFail();
+            return (V) values.putIfAbsent(key, value);
+        }
+
+        @Override
+        public <K> boolean containsKey(final K key, final Serializer<K> keySerializer) throws IOException {
+            verifyNotFail();
+            return values.containsKey(key);
+        }
+
+        @Override
+        public <K, V> void put(final K key, final V value, final Serializer<K> keySerializer, final Serializer<V> valueSerializer) throws IOException {
+            verifyNotFail();
+            values.put(key, value);
+        }
+
+        @Override
+        @SuppressWarnings("unchecked")
+        public <K, V> V get(final K key, final Serializer<K> keySerializer, final Deserializer<V> valueDeserializer) throws IOException {
+            verifyNotFail();
+            return (V) values.get(key);
+        }
+
+        @Override
+        public void close() throws IOException {
+        }
+
+        @Override
+        public <K> boolean remove(final K key, final Serializer<K> serializer) throws IOException {
+            verifyNotFail();
+            values.remove(key);
+            return true;
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/e748fd58/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/test/java/org/apache/nifi/hbase/TestPutHBaseCell.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/test/java/org/apache/nifi/hbase/TestPutHBaseCell.java b/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/test/java/org/apache/nifi/hbase/TestPutHBaseCell.java
new file mode 100644
index 0000000..62fa9a6
--- /dev/null
+++ b/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/test/java/org/apache/nifi/hbase/TestPutHBaseCell.java
@@ -0,0 +1,274 @@
+/*
+ * 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.nifi.hbase;
+
+import org.apache.nifi.hbase.put.PutFlowFile;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+public class TestPutHBaseCell {
+
+    @Test
+    public void testSingleFlowFile() throws IOException, InitializationException {
+        final String tableName = "nifi";
+        final String row = "row1";
+        final String columnFamily = "family1";
+        final String columnQualifier = "qualifier1";
+
+        final TestRunner runner = TestRunners.newTestRunner(PutHBaseCell.class);
+        runner.setProperty(PutHBaseCell.TABLE_NAME, tableName);
+        runner.setProperty(PutHBaseCell.ROW, row);
+        runner.setProperty(PutHBaseCell.COLUMN_FAMILY, columnFamily);
+        runner.setProperty(PutHBaseCell.COLUMN_QUALIFIER, columnQualifier);
+        runner.setProperty(PutHBaseCell.BATCH_SIZE, "1");
+
+        final MockHBaseClientService hBaseClient = getHBaseClientService(runner);
+
+        final String content = "some content";
+        runner.enqueue(content.getBytes("UTF-8"));
+        runner.run();
+        runner.assertAllFlowFilesTransferred(PutHBaseCell.REL_SUCCESS);
+
+        final MockFlowFile outFile = runner.getFlowFilesForRelationship(PutHBaseCell.REL_SUCCESS).get(0);
+        outFile.assertContentEquals(content);
+
+        assertNotNull(hBaseClient.getPuts());
+        assertEquals(1, hBaseClient.getPuts().size());
+
+        List<PutFlowFile> puts = hBaseClient.getPuts().get(tableName);
+        assertEquals(1, puts.size());
+        verifyPut(row, columnFamily, columnQualifier, content, puts.get(0));
+    }
+
+    @Test
+    public void testSingleFlowFileWithEL() throws IOException, InitializationException {
+        final String tableName = "nifi";
+        final String row = "row1";
+        final String columnFamily = "family1";
+        final String columnQualifier = "qualifier1";
+
+        final PutHBaseCell proc = new PutHBaseCell();
+        final TestRunner runner = getTestRunnerWithEL(proc);
+        runner.setProperty(PutHBaseCell.BATCH_SIZE, "1");
+
+        final MockHBaseClientService hBaseClient = getHBaseClientService(runner);
+
+        final String content = "some content";
+        final Map<String, String> attributes = getAtrributeMapWithEL(tableName, row, columnFamily, columnQualifier);
+        runner.enqueue(content.getBytes("UTF-8"), attributes);
+
+        runner.run();
+        runner.assertAllFlowFilesTransferred(PutHBaseCell.REL_SUCCESS);
+
+        final MockFlowFile outFile = runner.getFlowFilesForRelationship(PutHBaseCell.REL_SUCCESS).get(0);
+        outFile.assertContentEquals(content);
+
+        assertNotNull(hBaseClient.getPuts());
+        assertEquals(1, hBaseClient.getPuts().size());
+
+        List<PutFlowFile> puts = hBaseClient.getPuts().get(tableName);
+        assertEquals(1, puts.size());
+        verifyPut(row, columnFamily, columnQualifier, content, puts.get(0));
+    }
+
+    @Test
+    public void testSingleFlowFileWithELMissingAttributes() throws IOException, InitializationException {
+        final PutHBaseCell proc = new PutHBaseCell();
+        final TestRunner runner = getTestRunnerWithEL(proc);
+        runner.setProperty(PutHBaseCell.BATCH_SIZE, "1");
+
+        final MockHBaseClientService hBaseClient = new MockHBaseClientService();
+        runner.addControllerService("hbaseClient", hBaseClient);
+        runner.enableControllerService(hBaseClient);
+        runner.setProperty(PutHBaseCell.HBASE_CLIENT_SERVICE, "hbaseClient");
+
+        getHBaseClientService(runner);
+
+        final String content = "some content";
+        runner.enqueue(content.getBytes("UTF-8"), new HashMap<String, String>());
+        runner.run();
+
+        runner.assertTransferCount(PutHBaseCell.REL_SUCCESS, 0);
+        runner.assertTransferCount(PutHBaseCell.FAILURE, 1);
+    }
+
+    @Test
+    public void testMultipleFlowFileWithELOneMissingAttributes() throws IOException, InitializationException {
+        final PutHBaseCell proc = new PutHBaseCell();
+        final TestRunner runner = getTestRunnerWithEL(proc);
+        runner.setProperty(PutHBaseCell.BATCH_SIZE, "10");
+
+        final MockHBaseClientService hBaseClient = new MockHBaseClientService();
+        runner.addControllerService("hbaseClient", hBaseClient);
+        runner.enableControllerService(hBaseClient);
+        runner.setProperty(PutHBaseCell.HBASE_CLIENT_SERVICE, "hbaseClient");
+
+        getHBaseClientService(runner);
+
+        // this one will go to failure
+        final String content = "some content";
+        runner.enqueue(content.getBytes("UTF-8"), new HashMap<String, String>());
+
+        // this will go to success
+        final String content2 = "some content2";
+        final Map<String, String> attributes = getAtrributeMapWithEL("table", "row", "cf", "cq");
+        runner.enqueue(content2.getBytes("UTF-8"), attributes);
+
+        runner.run();
+        runner.assertTransferCount(PutHBaseCell.REL_SUCCESS, 1);
+        runner.assertTransferCount(PutHBaseCell.FAILURE, 1);
+    }
+
+    @Test
+    public void testMultipleFlowFilesSameTableDifferentRow() throws IOException, InitializationException {
+        final String tableName = "nifi";
+        final String row1 = "row1";
+        final String row2 = "row2";
+        final String columnFamily = "family1";
+        final String columnQualifier = "qualifier1";
+
+        final PutHBaseCell proc = new PutHBaseCell();
+        final TestRunner runner = getTestRunnerWithEL(proc);
+        final MockHBaseClientService hBaseClient = getHBaseClientService(runner);
+
+        final String content1 = "some content1";
+        final Map<String, String> attributes1 = getAtrributeMapWithEL(tableName, row1, columnFamily, columnQualifier);
+        runner.enqueue(content1.getBytes("UTF-8"), attributes1);
+
+        final String content2 = "some content1";
+        final Map<String, String> attributes2 = getAtrributeMapWithEL(tableName, row2, columnFamily, columnQualifier);
+        runner.enqueue(content2.getBytes("UTF-8"), attributes2);
+
+        runner.run();
+        runner.assertAllFlowFilesTransferred(PutHBaseCell.REL_SUCCESS);
+
+        final MockFlowFile outFile = runner.getFlowFilesForRelationship(PutHBaseCell.REL_SUCCESS).get(0);
+        outFile.assertContentEquals(content1);
+
+        assertNotNull(hBaseClient.getPuts());
+        assertEquals(1, hBaseClient.getPuts().size());
+
+        List<PutFlowFile> puts = hBaseClient.getPuts().get(tableName);
+        assertEquals(2, puts.size());
+        verifyPut(row1, columnFamily, columnQualifier, content1, puts.get(0));
+        verifyPut(row2, columnFamily, columnQualifier, content2, puts.get(1));
+    }
+
+    @Test
+    public void testMultipleFlowFilesSameTableDifferentRowFailure() throws IOException, InitializationException {
+        final String tableName = "nifi";
+        final String row1 = "row1";
+        final String row2 = "row2";
+        final String columnFamily = "family1";
+        final String columnQualifier = "qualifier1";
+
+        final PutHBaseCell proc = new PutHBaseCell();
+        final TestRunner runner = getTestRunnerWithEL(proc);
+        final MockHBaseClientService hBaseClient = getHBaseClientService(runner);
+        hBaseClient.setThrowException(true);
+
+        final String content1 = "some content1";
+        final Map<String, String> attributes1 = getAtrributeMapWithEL(tableName, row1, columnFamily, columnQualifier);
+        runner.enqueue(content1.getBytes("UTF-8"), attributes1);
+
+        final String content2 = "some content1";
+        final Map<String, String> attributes2 = getAtrributeMapWithEL(tableName, row2, columnFamily, columnQualifier);
+        runner.enqueue(content2.getBytes("UTF-8"), attributes2);
+
+        runner.run();
+        runner.assertAllFlowFilesTransferred(PutHBaseCell.FAILURE, 2);
+    }
+
+    @Test
+    public void testMultipleFlowFilesSameTableSameRow() throws IOException, InitializationException {
+        final String tableName = "nifi";
+        final String row = "row1";
+        final String columnFamily = "family1";
+        final String columnQualifier = "qualifier1";
+
+        final PutHBaseCell proc = new PutHBaseCell();
+        final TestRunner runner = getTestRunnerWithEL(proc);
+        final MockHBaseClientService hBaseClient = getHBaseClientService(runner);
+
+        final String content1 = "some content1";
+        final Map<String, String> attributes1 = getAtrributeMapWithEL(tableName, row, columnFamily, columnQualifier);
+        runner.enqueue(content1.getBytes("UTF-8"), attributes1);
+
+        final String content2 = "some content1";
+        runner.enqueue(content2.getBytes("UTF-8"), attributes1);
+
+        runner.run();
+        runner.assertAllFlowFilesTransferred(PutHBaseCell.REL_SUCCESS);
+
+        final MockFlowFile outFile = runner.getFlowFilesForRelationship(PutHBaseCell.REL_SUCCESS).get(0);
+        outFile.assertContentEquals(content1);
+
+        assertNotNull(hBaseClient.getPuts());
+        assertEquals(1, hBaseClient.getPuts().size());
+
+        List<PutFlowFile> puts = hBaseClient.getPuts().get(tableName);
+        assertEquals(2, puts.size());
+        verifyPut(row, columnFamily, columnQualifier, content1, puts.get(0));
+        verifyPut(row, columnFamily, columnQualifier, content2, puts.get(1));
+    }
+
+    private Map<String, String> getAtrributeMapWithEL(String tableName, String row, String columnFamily, String columnQualifier) {
+        final Map<String,String> attributes1 = new HashMap<>();
+        attributes1.put("hbase.tableName", tableName);
+        attributes1.put("hbase.row", row);
+        attributes1.put("hbase.columnFamily", columnFamily);
+        attributes1.put("hbase.columnQualifier", columnQualifier);
+        return attributes1;
+    }
+
+    private TestRunner getTestRunnerWithEL(PutHBaseCell proc) {
+        final TestRunner runner = TestRunners.newTestRunner(proc);
+        runner.setProperty(PutHBaseCell.TABLE_NAME, "${hbase.tableName}");
+        runner.setProperty(PutHBaseCell.ROW, "${hbase.row}");
+        runner.setProperty(PutHBaseCell.COLUMN_FAMILY, "${hbase.columnFamily}");
+        runner.setProperty(PutHBaseCell.COLUMN_QUALIFIER, "${hbase.columnQualifier}");
+        return runner;
+    }
+
+    private MockHBaseClientService getHBaseClientService(TestRunner runner) throws InitializationException {
+        final MockHBaseClientService hBaseClient = new MockHBaseClientService();
+        runner.addControllerService("hbaseClient", hBaseClient);
+        runner.enableControllerService(hBaseClient);
+        runner.setProperty(PutHBaseCell.HBASE_CLIENT_SERVICE, "hbaseClient");
+        return hBaseClient;
+    }
+
+    private void verifyPut(String row, String columnFamily, String columnQualifier, String content, PutFlowFile put) {
+        assertEquals(row, put.getRow());
+        assertEquals(columnFamily, put.getColumnFamily());
+        assertEquals(columnQualifier, put.getColumnQualifier());
+        assertEquals(content, new String(put.getBuffer(), StandardCharsets.UTF_8));
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/e748fd58/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/test/java/org/apache/nifi/hbase/util/TestObjectSerDe.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/test/java/org/apache/nifi/hbase/util/TestObjectSerDe.java b/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/test/java/org/apache/nifi/hbase/util/TestObjectSerDe.java
new file mode 100644
index 0000000..c2badb8
--- /dev/null
+++ b/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/test/java/org/apache/nifi/hbase/util/TestObjectSerDe.java
@@ -0,0 +1,71 @@
+/*
+ * 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.nifi.hbase.util;
+
+import org.apache.nifi.stream.io.ByteArrayInputStream;
+import org.apache.nifi.stream.io.ByteArrayOutputStream;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+
+public class TestObjectSerDe {
+
+   @Test
+    public void testDeserializeSuccessful() throws IOException {
+       final ObjectSerDe serDe = new ObjectSerDe();
+
+       final String myObject = "myObject";
+       final ByteArrayOutputStream bOut = new ByteArrayOutputStream();
+       final ObjectOutputStream out = new ObjectOutputStream(bOut);
+       out.writeObject(myObject);
+
+       byte[] myObjectBytes = bOut.toByteArray();
+       Assert.assertNotNull(myObjectBytes);
+       Assert.assertTrue(myObjectBytes.length > 0);
+
+       final Object deserialized = serDe.deserialize(myObjectBytes);
+       Assert.assertTrue(deserialized instanceof String);
+       Assert.assertEquals(myObject, deserialized);
+   }
+
+    @Test
+    public void testDeserializeNull() throws IOException {
+        final ObjectSerDe serDe = new ObjectSerDe();
+        final Object deserialized = serDe.deserialize(null);
+        Assert.assertNull(deserialized);
+    }
+
+    @Test
+    public void testSerialize() throws IOException, ClassNotFoundException {
+        final ByteArrayOutputStream out = new ByteArrayOutputStream();
+        final String myObject = "myObject";
+
+        final ObjectSerDe serDe = new ObjectSerDe();
+        serDe.serialize(myObject, out);
+
+        final ByteArrayInputStream bIn = new ByteArrayInputStream(out.toByteArray());
+        final ObjectInputStream in = new ObjectInputStream(bIn);
+
+        final Object deserialized = in.readObject();
+        Assert.assertTrue(deserialized instanceof String);
+        Assert.assertEquals(myObject, deserialized);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/e748fd58/nifi-nar-bundles/nifi-hbase-bundle/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hbase-bundle/pom.xml b/nifi-nar-bundles/nifi-hbase-bundle/pom.xml
new file mode 100644
index 0000000..f18918c
--- /dev/null
+++ b/nifi-nar-bundles/nifi-hbase-bundle/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/maven-v4_0_0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-nar-bundles</artifactId>
+        <version>0.4.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>nifi-hbase-bundle</artifactId>
+    <packaging>pom</packaging>
+
+    <modules>
+        <module>nifi-hbase-processors</module>
+        <module>nifi-hbase-nar</module>
+    </modules>
+
+    <dependencyManagement>
+        <dependencies>
+            <dependency>
+                <groupId>org.apache.nifi</groupId>
+                <artifactId>nifi-hbase-processors</artifactId>
+                <version>0.4.0-SNAPSHOT</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.hbase</groupId>
+                <artifactId>hbase-client</artifactId>
+                <version>1.1.2</version>
+            </dependency>
+            <!-- the top-level pom forces 18.0, but HBase 2.6 expects 12.0.1 -->
+            <dependency>
+                <groupId>com.google.guava</groupId>
+                <artifactId>guava</artifactId>
+                <version>${hadoop.guava.version}</version>
+            </dependency>
+        </dependencies>
+    </dependencyManagement>
+</project>

http://git-wip-us.apache.org/repos/asf/nifi/blob/e748fd58/nifi-nar-bundles/nifi-standard-services/nifi-hbase-client-service-api/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-hbase-client-service-api/pom.xml b/nifi-nar-bundles/nifi-standard-services/nifi-hbase-client-service-api/pom.xml
new file mode 100644
index 0000000..f871ee0
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-hbase-client-service-api/pom.xml
@@ -0,0 +1,39 @@
+<?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.nifi</groupId>
+        <artifactId>nifi-standard-services</artifactId>
+        <version>0.4.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>nifi-hbase-client-service-api</artifactId>
+    <packaging>jar</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-processor-utils</artifactId>
+        </dependency>
+    </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/nifi/blob/e748fd58/nifi-nar-bundles/nifi-standard-services/nifi-hbase-client-service-api/src/main/java/org/apache/nifi/hbase/HBaseClientService.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-hbase-client-service-api/src/main/java/org/apache/nifi/hbase/HBaseClientService.java b/nifi-nar-bundles/nifi-standard-services/nifi-hbase-client-service-api/src/main/java/org/apache/nifi/hbase/HBaseClientService.java
new file mode 100644
index 0000000..a3f2040
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-hbase-client-service-api/src/main/java/org/apache/nifi/hbase/HBaseClientService.java
@@ -0,0 +1,64 @@
+/*
+ * 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.nifi.hbase;
+
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.hbase.put.PutFlowFile;
+import org.apache.nifi.hbase.scan.Column;
+import org.apache.nifi.hbase.scan.ResultHandler;
+import org.apache.nifi.hbase.validate.ConfigFilesValidator;
+
+import java.io.IOException;
+import java.util.Collection;
+
+@Tags({"hbase", "client"})
+@CapabilityDescription("A controller service for accessing an HBase client.")
+public interface HBaseClientService extends ControllerService {
+
+    PropertyDescriptor HADOOP_CONF_FILES = new PropertyDescriptor.Builder()
+            .name("Hadoop Configuration Files")
+            .description("Comma-separated list of Hadoop Configuration files, such as hbase-site.xml")
+            .required(true)
+            .defaultValue("./conf/hbase-site.xml")
+            .addValidator(new ConfigFilesValidator())
+            .build();
+
+    /**
+     * Puts a batch of mutations to the given table.
+     *
+     * @param tableName the name of an HBase table
+     * @param puts a list of put mutations for the given table
+     * @throws IOException thrown when there are communication errors with HBase
+     */
+    void put(String tableName, Collection<PutFlowFile> puts) throws IOException;
+
+    /**
+     * Scans the given table using the optional filter criteria and passing each result to the provided handler.
+     *
+     * @param tableName the name of an HBase table to scan
+     * @param columns optional columns to return, if not specified all columns are returned
+     * @param filterExpression optional filter expression, if not specified no filtering is performed
+     * @param minTime the minimum timestamp of cells to return, passed to the HBase scanner timeRange
+     * @param handler a handler to process rows of the result set
+     * @throws IOException thrown when there are communication errors with HBase
+     */
+    void scan(String tableName, Collection<Column> columns, String filterExpression, long minTime, ResultHandler handler) throws IOException;
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/e748fd58/nifi-nar-bundles/nifi-standard-services/nifi-hbase-client-service-api/src/main/java/org/apache/nifi/hbase/put/PutFlowFile.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-hbase-client-service-api/src/main/java/org/apache/nifi/hbase/put/PutFlowFile.java b/nifi-nar-bundles/nifi-standard-services/nifi-hbase-client-service-api/src/main/java/org/apache/nifi/hbase/put/PutFlowFile.java
new file mode 100644
index 0000000..ed6319e
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-hbase-client-service-api/src/main/java/org/apache/nifi/hbase/put/PutFlowFile.java
@@ -0,0 +1,67 @@
+/*
+ * 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.nifi.hbase.put;
+
+import org.apache.nifi.flowfile.FlowFile;
+
+/**
+ * Wrapper to encapsulate all of the information for the Put along with the FlowFile.
+ */
+public class PutFlowFile {
+
+    private final String tableName;
+    private final String row;
+    private final String columnFamily;
+    private final String columnQualifier;
+    private final byte[] buffer;
+    private final FlowFile flowFile;
+
+    public PutFlowFile(String tableName, String row, String columnFamily, String columnQualifier,
+                       byte[] buffer, FlowFile flowFile) {
+        this.tableName = tableName;
+        this.row = row;
+        this.columnFamily = columnFamily;
+        this.columnQualifier = columnQualifier;
+        this.buffer = buffer;
+        this.flowFile = flowFile;
+    }
+
+    public String getTableName() {
+        return tableName;
+    }
+
+    public String getRow() {
+        return row;
+    }
+
+    public String getColumnFamily() {
+        return columnFamily;
+    }
+
+    public String getColumnQualifier() {
+        return columnQualifier;
+    }
+
+    public byte[] getBuffer() {
+        return buffer;
+    }
+
+    public FlowFile getFlowFile() {
+        return flowFile;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/e748fd58/nifi-nar-bundles/nifi-standard-services/nifi-hbase-client-service-api/src/main/java/org/apache/nifi/hbase/scan/Column.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-hbase-client-service-api/src/main/java/org/apache/nifi/hbase/scan/Column.java b/nifi-nar-bundles/nifi-standard-services/nifi-hbase-client-service-api/src/main/java/org/apache/nifi/hbase/scan/Column.java
new file mode 100644
index 0000000..ba5efda
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-hbase-client-service-api/src/main/java/org/apache/nifi/hbase/scan/Column.java
@@ -0,0 +1,71 @@
+/*
+ * 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.nifi.hbase.scan;
+
+import java.util.Arrays;
+
+/**
+ * Wrapper to encapsulate a column family and qualifier.
+ */
+public class Column {
+
+    private final byte[] family;
+    private final byte[] qualifier;
+
+    public Column(byte[] family, byte[] qualifier) {
+        this.family = family;
+        this.qualifier = qualifier;
+    }
+
+    public byte[] getFamily() {
+        return family;
+    }
+
+    public byte[] getQualifier() {
+        return qualifier;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (!(obj instanceof Column)) {
+            return false;
+        }
+
+        final Column other = (Column) obj;
+        return ((this.family == null && other.family == null)
+                || (this.family != null && other.family != null && Arrays.equals(this.family, other.family)))
+                && ((this.qualifier == null && other.qualifier == null)
+                || (this.qualifier != null && other.qualifier != null && Arrays.equals(this.qualifier, other.qualifier)));
+    }
+
+    @Override
+    public int hashCode() {
+        int result = 37;
+        if (family != null) {
+            for (byte b : family) {
+                result += (int)b;
+            }
+        }
+        if (qualifier != null) {
+            for (byte b : qualifier) {
+                result += (int)b;
+            }
+        }
+        return result;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/e748fd58/nifi-nar-bundles/nifi-standard-services/nifi-hbase-client-service-api/src/main/java/org/apache/nifi/hbase/scan/ResultCell.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-hbase-client-service-api/src/main/java/org/apache/nifi/hbase/scan/ResultCell.java b/nifi-nar-bundles/nifi-standard-services/nifi-hbase-client-service-api/src/main/java/org/apache/nifi/hbase/scan/ResultCell.java
new file mode 100644
index 0000000..950552b
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-hbase-client-service-api/src/main/java/org/apache/nifi/hbase/scan/ResultCell.java
@@ -0,0 +1,188 @@
+/*
+ * 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.nifi.hbase.scan;
+
+public class ResultCell {
+
+    byte[] rowArray;
+    int rowOffset;
+    short rowLength;
+
+    byte[] familyArray;
+    int familyOffset;
+    byte familyLength;
+
+    byte[] qualifierArray;
+    int qualifierOffset;
+    int qualifierLength;
+
+    long timestamp;
+    byte typeByte;
+    long sequenceId;
+
+    byte[] valueArray;
+    int valueOffset;
+    int valueLength;
+
+    byte[] tagsArray;
+    int tagsOffset;
+    int tagsLength;
+
+    public byte[] getRowArray() {
+        return rowArray;
+    }
+
+    public void setRowArray(byte[] rowArray) {
+        this.rowArray = rowArray;
+    }
+
+    public int getRowOffset() {
+        return rowOffset;
+    }
+
+    public void setRowOffset(int rowOffset) {
+        this.rowOffset = rowOffset;
+    }
+
+    public short getRowLength() {
+        return rowLength;
+    }
+
+    public void setRowLength(short rowLength) {
+        this.rowLength = rowLength;
+    }
+
+    public byte[] getFamilyArray() {
+        return familyArray;
+    }
+
+    public void setFamilyArray(byte[] familyArray) {
+        this.familyArray = familyArray;
+    }
+
+    public int getFamilyOffset() {
+        return familyOffset;
+    }
+
+    public void setFamilyOffset(int familyOffset) {
+        this.familyOffset = familyOffset;
+    }
+
+    public byte getFamilyLength() {
+        return familyLength;
+    }
+
+    public void setFamilyLength(byte familyLength) {
+        this.familyLength = familyLength;
+    }
+
+    public byte[] getQualifierArray() {
+        return qualifierArray;
+    }
+
+    public void setQualifierArray(byte[] qualifierArray) {
+        this.qualifierArray = qualifierArray;
+    }
+
+    public int getQualifierOffset() {
+        return qualifierOffset;
+    }
+
+    public void setQualifierOffset(int qualifierOffset) {
+        this.qualifierOffset = qualifierOffset;
+    }
+
+    public int getQualifierLength() {
+        return qualifierLength;
+    }
+
+    public void setQualifierLength(int qualifierLength) {
+        this.qualifierLength = qualifierLength;
+    }
+
+    public long getTimestamp() {
+        return timestamp;
+    }
+
+    public void setTimestamp(long timestamp) {
+        this.timestamp = timestamp;
+    }
+
+    public byte getTypeByte() {
+        return typeByte;
+    }
+
+    public void setTypeByte(byte typeByte) {
+        this.typeByte = typeByte;
+    }
+
+    public long getSequenceId() {
+        return sequenceId;
+    }
+
+    public void setSequenceId(long sequenceId) {
+        this.sequenceId = sequenceId;
+    }
+
+    public byte[] getValueArray() {
+        return valueArray;
+    }
+
+    public void setValueArray(byte[] valueArray) {
+        this.valueArray = valueArray;
+    }
+
+    public int getValueOffset() {
+        return valueOffset;
+    }
+
+    public void setValueOffset(int valueOffset) {
+        this.valueOffset = valueOffset;
+    }
+
+    public int getValueLength() {
+        return valueLength;
+    }
+
+    public void setValueLength(int valueLength) {
+        this.valueLength = valueLength;
+    }
+
+    public byte[] getTagsArray() {
+        return tagsArray;
+    }
+
+    public void setTagsArray(byte[] tagsArray) {
+        this.tagsArray = tagsArray;
+    }
+
+    public int getTagsOffset() {
+        return tagsOffset;
+    }
+
+    public void setTagsOffset(int tagsOffset) {
+        this.tagsOffset = tagsOffset;
+    }
+
+    public int getTagsLength() {
+        return tagsLength;
+    }
+
+    public void setTagsLength(int tagsLength) {
+        this.tagsLength = tagsLength;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/e748fd58/nifi-nar-bundles/nifi-standard-services/nifi-hbase-client-service-api/src/main/java/org/apache/nifi/hbase/scan/ResultHandler.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-hbase-client-service-api/src/main/java/org/apache/nifi/hbase/scan/ResultHandler.java b/nifi-nar-bundles/nifi-standard-services/nifi-hbase-client-service-api/src/main/java/org/apache/nifi/hbase/scan/ResultHandler.java
new file mode 100644
index 0000000..d0f1eab
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-hbase-client-service-api/src/main/java/org/apache/nifi/hbase/scan/ResultHandler.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.
+ */
+package org.apache.nifi.hbase.scan;
+
+/**
+ * Handles a single row from an HBase scan.
+ */
+public interface ResultHandler {
+
+    void handle(byte[] row, ResultCell[] resultCells);
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/e748fd58/nifi-nar-bundles/nifi-standard-services/nifi-hbase-client-service-api/src/main/java/org/apache/nifi/hbase/validate/ConfigFilesValidator.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-hbase-client-service-api/src/main/java/org/apache/nifi/hbase/validate/ConfigFilesValidator.java b/nifi-nar-bundles/nifi-standard-services/nifi-hbase-client-service-api/src/main/java/org/apache/nifi/hbase/validate/ConfigFilesValidator.java
new file mode 100644
index 0000000..9421440
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-hbase-client-service-api/src/main/java/org/apache/nifi/hbase/validate/ConfigFilesValidator.java
@@ -0,0 +1,38 @@
+/*
+ * 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.nifi.hbase.validate;
+
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.components.Validator;
+import org.apache.nifi.processor.util.StandardValidators;
+
+public class ConfigFilesValidator implements Validator {
+
+    @Override
+    public ValidationResult validate(final String subject, final String value, final ValidationContext context) {
+        final String[] filenames = value.split(",");
+        for (final String filename : filenames) {
+            final ValidationResult result = StandardValidators.FILE_EXISTS_VALIDATOR.validate(subject, filename.trim(), context);
+            if (!result.isValid()) {
+                return result;
+            }
+        }
+
+        return new ValidationResult.Builder().subject(subject).input(value).valid(true).build();
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/e748fd58/nifi-nar-bundles/nifi-standard-services/nifi-hbase_1_1_2-client-service-bundle/nifi-hbase_1_1_2-client-service-nar/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-hbase_1_1_2-client-service-bundle/nifi-hbase_1_1_2-client-service-nar/pom.xml b/nifi-nar-bundles/nifi-standard-services/nifi-hbase_1_1_2-client-service-bundle/nifi-hbase_1_1_2-client-service-nar/pom.xml
new file mode 100644
index 0000000..7ed5604
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-hbase_1_1_2-client-service-bundle/nifi-hbase_1_1_2-client-service-nar/pom.xml
@@ -0,0 +1,42 @@
+<?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.nifi</groupId>
+        <artifactId>nifi-hbase_1_1_2-client-service-bundle</artifactId>
+        <version>0.4.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>nifi-hbase_1_1_2-client-service-nar</artifactId>
+    <version>0.4.0-SNAPSHOT</version>
+    <packaging>nar</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-standard-services-api-nar</artifactId>
+            <type>nar</type>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-hbase_1_1_2-client-service</artifactId>
+            <version>0.4.0-SNAPSHOT</version>
+        </dependency>
+    </dependencies>
+
+</project>

http://git-wip-us.apache.org/repos/asf/nifi/blob/e748fd58/nifi-nar-bundles/nifi-standard-services/nifi-hbase_1_1_2-client-service-bundle/nifi-hbase_1_1_2-client-service-nar/src/main/resources/META-INF/LICENSE
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-hbase_1_1_2-client-service-bundle/nifi-hbase_1_1_2-client-service-nar/src/main/resources/META-INF/LICENSE b/nifi-nar-bundles/nifi-standard-services/nifi-hbase_1_1_2-client-service-bundle/nifi-hbase_1_1_2-client-service-nar/src/main/resources/META-INF/LICENSE
new file mode 100644
index 0000000..d167721
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-hbase_1_1_2-client-service-bundle/nifi-hbase_1_1_2-client-service-nar/src/main/resources/META-INF/LICENSE
@@ -0,0 +1,357 @@
+
+                                 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.
+
+APACHE NIFI SUBCOMPONENTS:
+
+The Apache NiFi project contains subcomponents with separate copyright
+notices and license terms. Your use of the source code for the these
+subcomponents is subject to the terms and conditions of the following
+licenses. 
+
+The binary distribution of this product bundles 'Jcodings' under an MIT style
+license.
+
+  Permission is hereby granted, free of charge, to any person obtaining a copy of
+  this software and associated documentation files (the "Software"), to deal in
+  the Software without restriction, including without limitation the rights to
+  use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies
+  of the Software, and to permit persons to whom the Software is furnished to do
+  so, subject to the following conditions:
+
+  The above copyright notice and this permission notice shall be included in all
+  copies or substantial portions of the Software.
+
+  THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+  IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+  FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+  AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+  LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+  OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+  SOFTWARE.
+
+The binary distribution of this product bundles 'Joni' under an MIT style
+license.
+
+  Permission is hereby granted, free of charge, to any person obtaining a copy of
+  this software and associated documentation files (the "Software"), to deal in
+  the Software without restriction, including without limitation the rights to
+  use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies
+  of the Software, and to permit persons to whom the Software is furnished to do
+  so, subject to the following conditions:
+
+  The above copyright notice and this permission notice shall be included in all
+  copies or substantial portions of the Software.
+
+  THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+  IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+  FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+  AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+  LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+  OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+  SOFTWARE.
+
+The binary distribution of this product bundles 'Google Protocol Buffers Java 2.5.0'
+which is licensed under a BSD license.
+
+  This license applies to all parts of Protocol Buffers except the following:
+
+    - Atomicops support for generic gcc, located in
+      src/google/protobuf/stubs/atomicops_internals_generic_gcc.h.
+      This file is copyrighted by Red Hat Inc.
+
+    - Atomicops support for AIX/POWER, located in
+      src/google/protobuf/stubs/atomicops_internals_aix.h.
+      This file is copyrighted by Bloomberg Finance LP.
+
+  Copyright 2014, Google Inc.  All rights reserved.
+
+  Redistribution and use in source and binary forms, with or without
+  modification, are permitted provided that the following conditions are
+  met:
+
+      * Redistributions of source code must retain the above copyright
+  notice, this list of conditions and the following disclaimer.
+      * 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.
+      * Neither the name of Google Inc. nor the names of its
+  contributors may be used to endorse or promote products derived from
+  this software without specific prior written permission.
+
+  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
+  OWNER 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.
+
+  Code generated by the Protocol Buffer compiler is owned by the owner
+  of the input file used when generating it.  This code is not
+  standalone and requires a support library to be linked with it.  This
+  support library is itself covered by the above license.
+
+The binary distribution of this product bundles 'Paranamer Core' which is available
+under a BSD style license.
+
+    Copyright (c) 2006 Paul Hammant & ThoughtWorks Inc
+     All rights reserved.
+
+     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.
+     3. Neither the name of the copyright holders nor the names of its
+        contributors may be used to endorse or promote products derived from
+        this software without specific prior written permission.
+
+     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 OWNER 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.
+
+The binary distribution of this product bundles 'JCraft Jsch' which is available
+under a BSD style license.
+
+     Copyright (c) 2002-2014 Atsuhiko Yamanaka, JCraft,Inc.
+     All rights reserved.
+
+     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.
+
+       3. The names of the authors may not be used to endorse or promote products
+          derived from this software without specific prior written permission.
+
+     THIS SOFTWARE IS PROVIDED ``AS IS'' AND ANY EXPRESSED 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 JCRAFT,
+     INC. OR ANY CONTRIBUTORS TO THIS SOFTWARE 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.
\ No newline at end of file