You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by jo...@apache.org on 2018/01/16 18:27:57 UTC

[3/5] nifi git commit: NIFI-4428: Replaced JSON input with RecordReader using per-record flowfiles NIFI-4428: Added initial L&N, fixed dependency hierarchy

http://git-wip-us.apache.org/repos/asf/nifi/blob/ecb80678/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service-api/src/main/java/org/apache/nifi/controller/api/druid/DruidTranquilityService.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service-api/src/main/java/org/apache/nifi/controller/api/druid/DruidTranquilityService.java b/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service-api/src/main/java/org/apache/nifi/controller/api/druid/DruidTranquilityService.java
new file mode 100644
index 0000000..ca83207
--- /dev/null
+++ b/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service-api/src/main/java/org/apache/nifi/controller/api/druid/DruidTranquilityService.java
@@ -0,0 +1,30 @@
+/*
+ * 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.controller.api.druid;
+
+import java.util.Map;
+
+import org.apache.nifi.controller.ControllerService;
+
+import com.metamx.tranquility.tranquilizer.Tranquilizer;
+
+public interface DruidTranquilityService extends ControllerService {
+    Tranquilizer<Map<String, Object>> getTranquilizer();
+
+    String getTransitUri();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/ecb80678/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service/pom.xml b/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service/pom.xml
index 2714969..d852fe1 100644
--- a/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service/pom.xml
+++ b/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service/pom.xml
@@ -37,59 +37,20 @@
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-processor-utils</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-druid-controller-service-api</artifactId>
             <version>1.5.0-SNAPSHOT</version>
+            <scope>provided</scope>
         </dependency>
         <dependency>
             <groupId>io.druid</groupId>
-            <artifactId>tranquility-core_2.10</artifactId>
-            <version>0.8.2</version>
-            <exclusions>
-                <exclusion>
-                    <groupId>io.druid</groupId>
-                    <artifactId>druid-console</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>io.druid</groupId>
-                    <artifactId>druid-aws-common</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>c3p0</groupId>
-                    <artifactId>c3p0</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>io.tesla.aether</groupId>
-                    <artifactId>tesla-aether</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>org.glassfish</groupId>
-                    <artifactId>javax.el</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>com.sun.jersey</groupId>
-                    <artifactId>*</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>org.eclipse.jetty</groupId>
-                    <artifactId>*</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>org.eclipse.aether</groupId>
-                    <artifactId>aether-api</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>com.maxmind.geoip2</groupId>
-                    <artifactId>geoip2</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>net.java.dev.jets3t</groupId>
-                    <artifactId>jets3t</artifactId>
-                </exclusion>
-            </exclusions>
+            <artifactId>tranquility-core_2.11</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-mock</artifactId>
+            <version>1.5.0-SNAPSHOT</version>
+            <scope>test</scope>
         </dependency>
     </dependencies>
 </project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/ecb80678/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service/src/main/java/org/apache/nifi/controller/DruidTranquilityController.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service/src/main/java/org/apache/nifi/controller/DruidTranquilityController.java b/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service/src/main/java/org/apache/nifi/controller/DruidTranquilityController.java
deleted file mode 100644
index 6d4ee19..0000000
--- a/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service/src/main/java/org/apache/nifi/controller/DruidTranquilityController.java
+++ /dev/null
@@ -1,452 +0,0 @@
-/*
- * 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.controller;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.curator.framework.CuratorFramework;
-import org.apache.curator.framework.CuratorFrameworkFactory;
-import org.apache.curator.retry.ExponentialBackoffRetry;
-import org.apache.nifi.annotation.documentation.CapabilityDescription;
-import org.apache.nifi.annotation.documentation.Tags;
-import org.apache.nifi.annotation.lifecycle.OnEnabled;
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.controller.api.DruidTranquilityService;
-import org.apache.nifi.logging.ComponentLog;
-import org.apache.nifi.processor.util.StandardValidators;
-import org.apache.nifi.reporting.InitializationException;
-import org.codehaus.jackson.map.ObjectMapper;
-import org.joda.time.DateTime;
-import org.joda.time.Period;
-
-import com.metamx.common.Granularity;
-import com.metamx.tranquility.beam.Beam;
-import com.metamx.tranquility.beam.ClusteredBeamTuning;
-import com.metamx.tranquility.druid.DruidBeamConfig;
-import com.metamx.tranquility.druid.DruidBeams;
-import com.metamx.tranquility.druid.DruidDimensions;
-import com.metamx.tranquility.druid.DruidEnvironment;
-import com.metamx.tranquility.druid.DruidLocation;
-import com.metamx.tranquility.druid.DruidRollup;
-import com.metamx.tranquility.tranquilizer.Tranquilizer;
-import com.metamx.tranquility.typeclass.Timestamper;
-
-import io.druid.data.input.impl.TimestampSpec;
-import io.druid.granularity.QueryGranularity;
-import io.druid.query.aggregation.AggregatorFactory;
-import io.druid.query.aggregation.CountAggregatorFactory;
-import io.druid.query.aggregation.DoubleMaxAggregatorFactory;
-import io.druid.query.aggregation.DoubleMinAggregatorFactory;
-import io.druid.query.aggregation.DoubleSumAggregatorFactory;
-import io.druid.query.aggregation.LongMaxAggregatorFactory;
-import io.druid.query.aggregation.LongMinAggregatorFactory;
-import io.druid.query.aggregation.LongSumAggregatorFactory;
-
-@Tags({"Druid", "Timeseries", "OLAP", "ingest"})
-@CapabilityDescription("Asynchronously sends flowfiles to Druid Indexing Task using Tranquility API. "
-        + "If aggregation and roll-up of data is required, an Aggregator JSON descriptor needs to be provided."
-        + "Details on how describe aggregation using JSON can be found at: http://druid.io/docs/latest/querying/aggregations.html")
-public class DruidTranquilityController extends AbstractControllerService implements DruidTranquilityService {
-    private String firehosePattern = "druid:firehose:%s";
-    private int clusterPartitions = 1;
-    private int clusterReplication = 1;
-    private String indexRetryPeriod = "PT10M";
-
-    private Tranquilizer tranquilizer = null;
-
-    public static final PropertyDescriptor DATASOURCE = new PropertyDescriptor.Builder()
-            .name("druid-cs-data-source")
-            .displayName("Druid Data Source")
-            .description("Druid Data Source") //TODO description, example
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .required(true)
-            .expressionLanguageSupported(true)
-            .build();
-
-    public static final PropertyDescriptor CONNECT_STRING = new PropertyDescriptor.Builder()
-            .name("druid-cs-zk-connect-string")
-            .displayName("Zookeeper Connection String")
-            .description("ZK Connect String for Druid") //TODO example
-            .required(true)
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
-            .build();
-
-    public static final PropertyDescriptor DRUID_INDEX_SERVICE_PATH = new PropertyDescriptor.Builder()
-            .name("druid-cs-index-service-path")
-            .displayName("Index Service Path")
-            .description("Druid Index Service path as defined via the Druid Overlord druid.service property.")
-            .required(true)
-            .defaultValue("druid/overlord")
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
-            .build();
-
-    public static final PropertyDescriptor DRUID_DISCOVERY_PATH = new PropertyDescriptor.Builder()
-            .name("druid-cs-discovery-path")
-            .displayName("Discovery Path")
-            .description("Druid Discovery Path as configured in Druid Common druid.discovery.curator.path property")
-            .required(true)
-            .defaultValue("/druid/discovery")
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
-            .build();
-
-    public static final PropertyDescriptor TIMESTAMP_FIELD = new PropertyDescriptor.Builder()
-            .name("druid-cs-timestamp-field")
-            .displayName("Timestamp field")
-            .description("The name of the field that will be used as the timestamp. Should be in ISO format.")
-            .required(true)
-            .defaultValue("timestamp")
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
-            .build();
-
-    public static final PropertyDescriptor AGGREGATOR_JSON = new PropertyDescriptor.Builder()
-            .name("druid-cs-aggregators-descriptor")
-            .displayName("Aggregator JSON")
-            .description("Tranquility-compliant JSON string that defines what aggregators to apply on ingest."
-                    + "Example: "
-                    + "["
-                    + "{"
-                    + "\t\"type\" : \"count\","
-                    + "\t\"name\" : \"count\","
-                    + "},"
-                    + "{"
-                    + "\t\"name\" : \"value_sum\","
-                    + "\t\"type\" : \"doubleSum\","
-                    + "\t\"fieldName\" : \"value\""
-                    + "},"
-                    + "{"
-                    + "\t\"fieldName\" : \"value\","
-                    + "\t\"name\" : \"value_min\","
-                    + "\t\"type\" : \"doubleMin\""
-                    + "},"
-                    + "{"
-                    + "\t\"type\" : \"doubleMax\","
-                    + "\t\"name\" : \"value_max\","
-                    + "\t\"fieldName\" : \"value\""
-                    + "}"
-                    + "]")
-            .required(true)
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
-            .build();
-
-    public static final PropertyDescriptor DIMENSIONS_LIST = new PropertyDescriptor.Builder()
-            .name("druid-cs-dimensions-list")
-            .displayName("Dimension Fields")
-            .description("A comma separated list of field names that will be stored as dimensions on ingest.")
-            .required(true)
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
-            .build();
-
-    public static final PropertyDescriptor SEGMENT_GRANULARITY = new PropertyDescriptor.Builder()
-            .name("druid-cs-segment-granularity")
-            .displayName("Segment Granularity")
-            .description("Time unit by which to group and aggregate/rollup events.")
-            .required(true)
-            .allowableValues("NONE", "SECOND", "MINUTE", "TEN_MINUTE", "HOUR", "DAY", "MONTH", "YEAR", "Use druid.segment.granularity variable")
-            .defaultValue("MINUTE")
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .build();
-
-    public static final PropertyDescriptor QUERY_GRANULARITY = new PropertyDescriptor.Builder()
-            .name("druid-cs-query-granularity")
-            .displayName("Query Granularity")
-            .description("Time unit by which to group and aggregate/rollup events. The value must be at least as large as the value of Segment Granularity.")
-            .required(true)
-            .allowableValues("NONE", "SECOND", "MINUTE", "TEN_MINUTE", "HOUR", "DAY", "MONTH", "YEAR", "Use druid.query.granularity variable")
-            .defaultValue("TEN_MINUTE")
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .build();
-
-    public static final PropertyDescriptor WINDOW_PERIOD = new PropertyDescriptor.Builder()
-            .name("druid-cs-window-period")
-            .displayName("Late Event Grace Period")
-            .description("Grace period to allow late arriving events for real time ingest.")
-            .required(true)
-            .allowableValues("PT1M", "PT10M", "PT60M")// TODO possibly friendly name
-            .defaultValue("PT10M")
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .build();
-
-    public static final PropertyDescriptor MAX_BATCH_SIZE = new PropertyDescriptor.Builder()
-            .name("druid-cs-batch-size")
-            .displayName("Batch Size")
-            .description("Maximum number of messages to send at once.")
-            .required(true)
-            .defaultValue("2000")
-            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
-            .build();
-
-    public static final PropertyDescriptor MAX_PENDING_BATCHES = new PropertyDescriptor.Builder()
-            .name("druid-cs-max-pending-batches")
-            .displayName("Max Pending Batches")
-            .description("Maximum number of batches that may be in flight before service blocks and waits for one to finish.")
-            .required(true)
-            .defaultValue("5")
-            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
-            .build();
-
-    public static final PropertyDescriptor LINGER_MILLIS = new PropertyDescriptor.Builder()
-            .name("druid-cs-linger-millis")
-            .displayName("Linger (milliseconds)")
-            .description("Wait this long for batches to collect more messages (up to Batch Size) before sending them. "
-                    + "Set to zero to disable waiting. "
-                    + "Set to -1 to always wait for complete batches before sending. ")
-            .required(true)
-            .defaultValue("1000")
-            .addValidator(StandardValidators.INTEGER_VALIDATOR)
-            .build();
-
-    private static final List<PropertyDescriptor> properties;
-
-    static {
-        final List<PropertyDescriptor> props = new ArrayList<>();
-        props.add(DATASOURCE);
-        props.add(CONNECT_STRING);
-        props.add(DRUID_INDEX_SERVICE_PATH);
-        props.add(DRUID_DISCOVERY_PATH);
-        props.add(DIMENSIONS_LIST);
-        props.add(AGGREGATOR_JSON);
-        props.add(SEGMENT_GRANULARITY);
-        props.add(QUERY_GRANULARITY);
-        props.add(WINDOW_PERIOD);
-        props.add(TIMESTAMP_FIELD);
-        props.add(MAX_BATCH_SIZE);
-        props.add(MAX_PENDING_BATCHES);
-        props.add(LINGER_MILLIS);
-
-        properties = Collections.unmodifiableList(props);
-    }
-
-    @Override
-    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
-        return properties;
-    }
-
-    @OnEnabled
-    public void onConfigured(final ConfigurationContext context) throws InitializationException {
-        ComponentLog log = getLogger();
-        log.info("Starting Druid Tranquility Controller Service...");
-
-        final String dataSource = context.getProperty(DATASOURCE).getValue();
-        final String zkConnectString = context.getProperty(CONNECT_STRING).getValue();
-        final String indexService = context.getProperty(DRUID_INDEX_SERVICE_PATH).getValue();
-        final String discoveryPath = context.getProperty(DRUID_DISCOVERY_PATH).getValue();
-        final String timestampField = context.getProperty(TIMESTAMP_FIELD).getValue();
-        final String segmentGranularity = context.getProperty(SEGMENT_GRANULARITY).getValue();
-        final String queryGranularity = context.getProperty(QUERY_GRANULARITY).getValue();
-        final String windowPeriod = context.getProperty(WINDOW_PERIOD).getValue();
-        final String aggregatorJSON = context.getProperty(AGGREGATOR_JSON).getValue();
-        final String dimensionsStringList = context.getProperty(DIMENSIONS_LIST).getValue();
-        final int maxBatchSize = Integer.valueOf(context.getProperty(MAX_BATCH_SIZE).getValue());
-        final int maxPendingBatches = Integer.valueOf(context.getProperty(MAX_PENDING_BATCHES).getValue());
-        final int lingerMillis = Integer.valueOf(context.getProperty(LINGER_MILLIS).getValue());
-
-        final List<String> dimensions = getDimensions(dimensionsStringList);
-        final List<AggregatorFactory> aggregator = getAggregatorList(aggregatorJSON);
-
-        final Timestamper<Map<String, Object>> timestamper = new Timestamper<Map<String, Object>>() {
-            private static final long serialVersionUID = 1L;
-
-            @Override
-            public DateTime timestamp(Map<String, Object> theMap) {
-                return new DateTime(theMap.get(timestampField));
-            }
-        };
-
-        Iterator<AggregatorFactory> aggIterator = aggregator.iterator();
-        AggregatorFactory currFactory;
-        log.debug("Number of Aggregations Defined: " + aggregator.size());
-        while (aggIterator.hasNext()) {
-            currFactory = aggIterator.next();
-            log.debug("Verifying Aggregator Definition");
-            log.debug("Aggregator Name: " + currFactory.getName());
-            log.debug("Aggregator Type: " + currFactory.getTypeName());
-            log.debug("Aggregator Req Fields: " + currFactory.requiredFields());
-        }
-        // Tranquility uses ZooKeeper (through Curator) for coordination.
-        final CuratorFramework curator = CuratorFrameworkFactory
-                .builder()
-                .connectString(zkConnectString)
-                .retryPolicy(new ExponentialBackoffRetry(1000, 20, 30000)) // TODO expose as properties, maybe fibonacci backoff
-                .build();
-        curator.start();
-
-        // The JSON serialization of your object must have a timestamp field in a format that Druid understands. By default,
-        // Druid expects the field to be called "timestamp" and to be an ISO8601 timestamp.
-        final TimestampSpec timestampSpec = new TimestampSpec(timestampField, "auto", null);
-
-        final Beam<Map<String, Object>> beam = DruidBeams.builder(timestamper)
-                .curator(curator)
-                .discoveryPath(discoveryPath)
-                .location(DruidLocation.create(DruidEnvironment.create(indexService, firehosePattern), dataSource))
-                .timestampSpec(timestampSpec)
-                .rollup(DruidRollup.create(DruidDimensions.specific(dimensions), aggregator, QueryGranularity.fromString(queryGranularity)))
-                .tuning(
-                        ClusteredBeamTuning
-                                .builder()
-                                .segmentGranularity(getSegmentGranularity(segmentGranularity))
-                                .windowPeriod(new Period(windowPeriod))
-                                .partitions(clusterPartitions)
-                                .replicants(clusterReplication)
-                                .build()
-                )
-                .druidBeamConfig(
-                        DruidBeamConfig
-                                .builder()
-                                .indexRetryPeriod(new Period(indexRetryPeriod))
-                                .build())
-                .buildBeam();
-
-        tranquilizer = Tranquilizer.builder()
-                .maxBatchSize(maxBatchSize)
-                .maxPendingBatches(maxPendingBatches)
-                .lingerMillis(lingerMillis)
-                .blockOnFull(true)
-                .build(beam);
-
-        tranquilizer.start();
-    }
-
-    public Tranquilizer getTranquilizer() {
-        return tranquilizer;
-    }
-
-    private List<Map<String, String>> parseJsonString(String aggregatorJson) {
-        ObjectMapper mapper = new ObjectMapper();
-        List<Map<String, String>> aggSpecList = null;
-        try {
-            aggSpecList = mapper.readValue(aggregatorJson, List.class);
-            return aggSpecList;
-        } catch (IOException e) {
-            throw new IllegalArgumentException("Exception while parsing the aggregrator JSON");
-        }
-    }
-
-    private List<String> getDimensions(String dimensionStringList) {
-        List<String> dimensionList = new LinkedList(Arrays.asList(dimensionStringList.split(",")));
-        return dimensionList;
-    }
-
-    private List<AggregatorFactory> getAggregatorList(String aggregatorJSON) {
-        List<AggregatorFactory> aggregatorList = new LinkedList<>();
-        List<Map<String, String>> aggregatorInfo = parseJsonString(aggregatorJSON);
-        for (Map<String, String> aggregator : aggregatorInfo) {
-
-            if (aggregator.get("type").equalsIgnoreCase("count")) {
-                aggregatorList.add(getCountAggregator(aggregator));
-            } else if (aggregator.get("type").equalsIgnoreCase("doublesum")) {
-                aggregatorList.add(getDoubleSumAggregator(aggregator));
-            } else if (aggregator.get("type").equalsIgnoreCase("doublemax")) {
-                aggregatorList.add(getDoubleMaxAggregator(aggregator));
-            } else if (aggregator.get("type").equalsIgnoreCase("doublemin")) {
-                aggregatorList.add(getDoubleMinAggregator(aggregator));
-            } else if (aggregator.get("type").equalsIgnoreCase("longsum")) {
-                aggregatorList.add(getLongSumAggregator(aggregator));
-            } else if (aggregator.get("type").equalsIgnoreCase("longmax")) {
-                aggregatorList.add(getLongMaxAggregator(aggregator));
-            } else if (aggregator.get("type").equalsIgnoreCase("longmin")) {
-                aggregatorList.add(getLongMinAggregator(aggregator));
-            }
-        }
-
-        return aggregatorList;
-    }
-
-    private AggregatorFactory getLongMinAggregator(Map<String, String> map) {
-        return new LongMinAggregatorFactory(map.get("name"), map.get("fieldName"));
-    }
-
-    private AggregatorFactory getLongMaxAggregator(Map<String, String> map) {
-        return new LongMaxAggregatorFactory(map.get("name"), map.get("fieldName"));
-    }
-
-    private AggregatorFactory getLongSumAggregator(Map<String, String> map) {
-        return new LongSumAggregatorFactory(map.get("name"), map.get("fieldName"));
-    }
-
-    private AggregatorFactory getDoubleMinAggregator(Map<String, String> map) {
-        return new DoubleMinAggregatorFactory(map.get("name"), map.get("fieldName"));
-    }
-
-    private AggregatorFactory getDoubleMaxAggregator(Map<String, String> map) {
-        return new DoubleMaxAggregatorFactory(map.get("name"), map.get("fieldName"));
-    }
-
-    private AggregatorFactory getDoubleSumAggregator(Map<String, String> map) {
-        return new DoubleSumAggregatorFactory(map.get("name"), map.get("fieldName"));
-    }
-
-    private AggregatorFactory getCountAggregator(Map<String, String> map) {
-        return new CountAggregatorFactory(map.get("name"));
-    }
-
-    private Granularity getSegmentGranularity(String segmentGranularity) {
-        Granularity granularity = Granularity.HOUR;
-
-        switch (segmentGranularity) {
-            case "SECOND":
-                granularity = Granularity.SECOND;
-                break;
-            case "MINUTE":
-                granularity = Granularity.MINUTE;
-                break;
-            case "FIVE_MINUTE":
-                granularity = Granularity.FIVE_MINUTE;
-                break;
-            case "TEN_MINUTE":
-                granularity = Granularity.TEN_MINUTE;
-                break;
-            case "FIFTEEN_MINUTE":
-                granularity = Granularity.FIFTEEN_MINUTE;
-                break;
-            case "HOUR":
-                granularity = Granularity.HOUR;
-                break;
-            case "SIX_HOUR":
-                granularity = Granularity.SIX_HOUR;
-                break;
-            case "DAY":
-                granularity = Granularity.DAY;
-                break;
-            case "WEEK":
-                granularity = Granularity.WEEK;
-                break;
-            case "MONTH":
-                granularity = Granularity.MONTH;
-                break;
-            case "YEAR":
-                granularity = Granularity.YEAR;
-                break;
-            default:
-                break;
-        }
-        return granularity;
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/ecb80678/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service/src/main/java/org/apache/nifi/controller/druid/DruidTranquilityController.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service/src/main/java/org/apache/nifi/controller/druid/DruidTranquilityController.java b/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service/src/main/java/org/apache/nifi/controller/druid/DruidTranquilityController.java
new file mode 100644
index 0000000..e5af8ea
--- /dev/null
+++ b/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service/src/main/java/org/apache/nifi/controller/druid/DruidTranquilityController.java
@@ -0,0 +1,614 @@
+/*
+ * 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.controller.druid;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.retry.ExponentialBackoffRetry;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnDisabled;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.controller.api.druid.DruidTranquilityService;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.codehaus.jackson.map.ObjectMapper;
+
+import com.metamx.common.Granularity;
+import com.metamx.tranquility.beam.Beam;
+import com.metamx.tranquility.beam.ClusteredBeamTuning;
+import com.metamx.tranquility.druid.DruidBeamConfig;
+import com.metamx.tranquility.druid.DruidBeams;
+import com.metamx.tranquility.druid.DruidDimensions;
+import com.metamx.tranquility.druid.DruidEnvironment;
+import com.metamx.tranquility.druid.DruidLocation;
+import com.metamx.tranquility.druid.DruidRollup;
+import com.metamx.tranquility.tranquilizer.Tranquilizer;
+import com.metamx.tranquility.typeclass.Timestamper;
+
+import io.druid.data.input.impl.TimestampSpec;
+import io.druid.granularity.QueryGranularity;
+import io.druid.query.aggregation.AggregatorFactory;
+import io.druid.query.aggregation.CountAggregatorFactory;
+import io.druid.query.aggregation.DoubleMaxAggregatorFactory;
+import io.druid.query.aggregation.DoubleMinAggregatorFactory;
+import io.druid.query.aggregation.DoubleSumAggregatorFactory;
+import io.druid.query.aggregation.LongMaxAggregatorFactory;
+import io.druid.query.aggregation.LongMinAggregatorFactory;
+import io.druid.query.aggregation.LongSumAggregatorFactory;
+import org.joda.time.DateTime;
+import org.joda.time.Period;
+
+
+@Tags({"Druid", "Timeseries", "OLAP", "ingest"})
+@CapabilityDescription("Asynchronously sends flowfiles to Druid Indexing Task using Tranquility API. "
+        + "If aggregation and roll-up of data is required, an Aggregator JSON descriptor needs to be provided."
+        + "Details on how describe aggregation using JSON can be found at: http://druid.io/docs/latest/querying/aggregations.html")
+public class DruidTranquilityController extends AbstractControllerService implements DruidTranquilityService {
+    private final static String FIREHOSE_PATTERN = "druid:firehose:%s";
+
+    private final static AllowableValue PT1M = new AllowableValue("PT1M", "1 minute", "1 minute");
+    private final static AllowableValue PT10M = new AllowableValue("PT10M", "10 minutes", "10 minutes");
+    private final static AllowableValue PT60M = new AllowableValue("PT60M", "60 minutes", "1 hour");
+
+    private final static List<String> TIME_ORDINALS = Arrays.asList("SECOND", "MINUTE", "FIVE_MINUTE", "TEN_MINUTE", "FIFTEEN_MINUTE", "HOUR", "SIX_HOUR", "DAY", "WEEK", "MONTH", "YEAR");
+
+    private Tranquilizer tranquilizer = null;
+    private String transitUri = "";
+
+    public static final PropertyDescriptor DATASOURCE = new PropertyDescriptor.Builder()
+            .name("druid-cs-data-source")
+            .displayName("Druid Data Source")
+            .description("A data source is the Druid equivalent of a database table.")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .required(true)
+            .expressionLanguageSupported(true)
+            .build();
+
+    public static final PropertyDescriptor ZOOKEEPER_CONNECTION_STRING = new PropertyDescriptor.Builder()
+            .name("druid-cs-zk-connect-string")
+            .displayName("Zookeeper Connection String")
+            .description("A comma-separated list of host:port pairs, each corresponding to a ZooKeeper server. Ex: localhost:2181")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(true)
+            .build();
+
+    public static final PropertyDescriptor ZOOKEEPER_RETRY_BASE_SLEEP_TIME = new PropertyDescriptor.Builder()
+            .name("druid-cs-zk-retry-base-sleep")
+            .displayName("Zookeeper Retry Base Sleep Time")
+            .description("When a connection to Zookeeper needs to be retried, this property specifies the amount of time (in milliseconds) to wait at first before retrying.")
+            .required(true)
+            .defaultValue("1000")
+            .expressionLanguageSupported(true)
+            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor ZOOKEEPER_RETRY_MAX_RETRIES = new PropertyDescriptor.Builder()
+            .name("druid-cs-zk-retry-max-retries")
+            .displayName("Zookeeper Retry Max Retries")
+            .description("When a connection to Zookeeper needs to be retried, this property specifies how many times to attempt reconnection.")
+            .required(true)
+            .defaultValue("20")
+            .expressionLanguageSupported(true)
+            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor ZOOKEEPER_RETRY_SLEEP_TIME = new PropertyDescriptor.Builder()
+            .name("druid-cs-zk-retry-sleep")
+            .displayName("Zookeeper Retry Sleep Time")
+            .description("When a connection to Zookeeper needs to be retried, this property specifies the amount of time to sleep (in milliseconds) between retries.")
+            .required(true)
+            .defaultValue("30000")
+            .expressionLanguageSupported(true)
+            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor DRUID_INDEX_SERVICE_PATH = new PropertyDescriptor.Builder()
+            .name("druid-cs-index-service-path")
+            .displayName("Index Service Path")
+            .description("Druid Index Service path as defined via the Druid Overlord druid.service property.")
+            .required(true)
+            .defaultValue("druid/overlord")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(true)
+            .build();
+
+    public static final PropertyDescriptor DRUID_DISCOVERY_PATH = new PropertyDescriptor.Builder()
+            .name("druid-cs-discovery-path")
+            .displayName("Discovery Path")
+            .description("Druid Discovery Path as configured in Druid Common druid.discovery.curator.path property")
+            .required(true)
+            .defaultValue("/druid/discovery")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(true)
+            .build();
+
+    public static final PropertyDescriptor CLUSTER_PARTITIONS = new PropertyDescriptor.Builder()
+            .name("druid-cs-cluster-partitions")
+            .displayName("Cluster Partitions")
+            .description("The number of partitions in the Druid cluster.")
+            .required(true)
+            .defaultValue("1")
+            .expressionLanguageSupported(true)
+            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor CLUSTER_REPLICATION = new PropertyDescriptor.Builder()
+            .name("druid-cs-cluster-replication")
+            .displayName("Cluster Replication")
+            .description("The replication factor for the Druid cluster.")
+            .required(true)
+            .defaultValue("1")
+            .expressionLanguageSupported(true)
+            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor TIMESTAMP_FIELD = new PropertyDescriptor.Builder()
+            .name("druid-cs-timestamp-field")
+            .displayName("Timestamp field")
+            .description("The name of the field that will be used as the timestamp. Should be in ISO8601 format.")
+            .required(true)
+            .defaultValue("timestamp")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(true)
+            .build();
+
+    public static final PropertyDescriptor AGGREGATOR_JSON = new PropertyDescriptor.Builder()
+            .name("druid-cs-aggregators-descriptor")
+            .displayName("Aggregator JSON")
+            .description("Tranquility-compliant JSON string that defines what aggregators to apply on ingest."
+                    + "Example: "
+                    + "["
+                    + "{"
+                    + "\t\"type\" : \"count\","
+                    + "\t\"name\" : \"count\","
+                    + "},"
+                    + "{"
+                    + "\t\"name\" : \"value_sum\","
+                    + "\t\"type\" : \"doubleSum\","
+                    + "\t\"fieldName\" : \"value\""
+                    + "},"
+                    + "{"
+                    + "\t\"fieldName\" : \"value\","
+                    + "\t\"name\" : \"value_min\","
+                    + "\t\"type\" : \"doubleMin\""
+                    + "},"
+                    + "{"
+                    + "\t\"type\" : \"doubleMax\","
+                    + "\t\"name\" : \"value_max\","
+                    + "\t\"fieldName\" : \"value\""
+                    + "}"
+                    + "]")
+            .required(true)
+            .addValidator((subject, value, context) -> { // Non-empty and valid JSON validator
+                if (value == null || value.isEmpty()) {
+                    return new ValidationResult.Builder().subject(subject).input(value).valid(false).explanation(subject + " cannot be empty").build();
+                }
+                try {
+                    DruidTranquilityController.parseJsonString(value);
+                    return new ValidationResult.Builder().subject(subject).input(value).valid(true).build();
+                } catch (IllegalArgumentException iae) {
+                    return new ValidationResult.Builder().subject(subject).input(value).valid(false).explanation(subject + " is not valid Aggregator JSON").build();
+                }
+            })
+            .expressionLanguageSupported(true)
+            .build();
+
+    public static final PropertyDescriptor DIMENSIONS_LIST = new PropertyDescriptor.Builder()
+            .name("druid-cs-dimensions-list")
+            .displayName("Dimension Fields")
+            .description("A comma separated list of field names that will be stored as dimensions on ingest.")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(true)
+            .build();
+
+    public static final PropertyDescriptor SEGMENT_GRANULARITY = new PropertyDescriptor.Builder()
+            .name("druid-cs-segment-granularity")
+            .displayName("Segment Granularity")
+            .description("Time unit by which to group and aggregate/rollup events. The value must be at least as large as the value of Query Granularity.")
+            .required(true)
+            .allowableValues("NONE", "SECOND", "MINUTE", "TEN_MINUTE", "HOUR", "DAY", "MONTH", "YEAR")
+            .defaultValue("TEN_MINUTE")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor QUERY_GRANULARITY = new PropertyDescriptor.Builder()
+            .name("druid-cs-query-granularity")
+            .displayName("Query Granularity")
+            .description("Time unit by which to group and aggregate/rollup events. The value must be less than or equal to the value of Segment Granularity.")
+            .required(true)
+            .allowableValues("NONE", "SECOND", "MINUTE", "FIFTEEN_MINUTE", "THIRTY_MINUTE", "HOUR", "DAY", "MONTH", "YEAR")
+            .defaultValue("MINUTE")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor INDEX_RETRY_PERIOD = new PropertyDescriptor.Builder()
+            .name("druid-cs-index-retry-period")
+            .displayName("Index Retry Period")
+            .description("Grace period to allow late arriving events for real time ingest.")
+            .required(true)
+            .allowableValues(PT1M, PT10M, PT60M)
+            .defaultValue(PT10M.getValue())
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor WINDOW_PERIOD = new PropertyDescriptor.Builder()
+            .name("druid-cs-window-period")
+            .displayName("Late Event Grace Period")
+            .description("Grace period to allow late arriving events for real time ingest.")
+            .required(true)
+            .allowableValues(PT1M, PT10M, PT60M)
+            .defaultValue(PT10M.getValue())
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor MAX_BATCH_SIZE = new PropertyDescriptor.Builder()
+            .name("druid-cs-batch-size")
+            .displayName("Batch Size")
+            .description("Maximum number of messages to send at once.")
+            .required(true)
+            .defaultValue("2000")
+            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
+            .expressionLanguageSupported(true)
+            .build();
+
+    public static final PropertyDescriptor MAX_PENDING_BATCHES = new PropertyDescriptor.Builder()
+            .name("druid-cs-max-pending-batches")
+            .displayName("Max Pending Batches")
+            .description("Maximum number of batches that may be in flight before service blocks and waits for one to finish.")
+            .required(true)
+            .defaultValue("5")
+            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
+            .expressionLanguageSupported(true)
+            .build();
+
+    public static final PropertyDescriptor LINGER_MILLIS = new PropertyDescriptor.Builder()
+            .name("druid-cs-linger-millis")
+            .displayName("Linger (milliseconds)")
+            .description("Wait this long for batches to collect more messages (up to Batch Size) before sending them. "
+                    + "Set to zero to disable waiting. "
+                    + "Set to -1 to always wait for complete batches before sending. ")
+            .required(true)
+            .defaultValue("1000")
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .expressionLanguageSupported(true)
+            .build();
+
+    private static final List<PropertyDescriptor> properties;
+
+    private volatile CuratorFramework curator;
+    private volatile int zkBaseSleepMillis;
+    private volatile int zkMaxRetries;
+    private volatile int zkSleepMillis;
+
+    static {
+        final List<PropertyDescriptor> props = new ArrayList<>();
+        props.add(DATASOURCE);
+        props.add(ZOOKEEPER_CONNECTION_STRING);
+        props.add(ZOOKEEPER_RETRY_BASE_SLEEP_TIME);
+        props.add(ZOOKEEPER_RETRY_MAX_RETRIES);
+        props.add(ZOOKEEPER_RETRY_SLEEP_TIME);
+        props.add(DRUID_INDEX_SERVICE_PATH);
+        props.add(DRUID_DISCOVERY_PATH);
+        props.add(CLUSTER_PARTITIONS);
+        props.add(CLUSTER_REPLICATION);
+        props.add(DIMENSIONS_LIST);
+        props.add(AGGREGATOR_JSON);
+        props.add(SEGMENT_GRANULARITY);
+        props.add(QUERY_GRANULARITY);
+        props.add(WINDOW_PERIOD);
+        props.add(TIMESTAMP_FIELD);
+        props.add(MAX_BATCH_SIZE);
+        props.add(MAX_PENDING_BATCHES);
+        props.add(LINGER_MILLIS);
+
+        properties = Collections.unmodifiableList(props);
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(ValidationContext validationContext) {
+        Set<ValidationResult> results = new HashSet<>();
+        final String segmentGranularity = validationContext.getProperty(SEGMENT_GRANULARITY).getValue();
+        final String queryGranularity = validationContext.getProperty(QUERY_GRANULARITY).getValue();
+
+        // Verify that segment granularity is as least as large as query granularity
+        if (TIME_ORDINALS.indexOf(segmentGranularity) < TIME_ORDINALS.indexOf(queryGranularity)) {
+            results.add(new ValidationResult.Builder().valid(false).explanation(
+                    "Segment Granularity must be at least as large as Query Granularity").build());
+        }
+
+        return results;
+    }
+
+    @OnEnabled
+    public void onConfigured(final ConfigurationContext context) {
+        ComponentLog log = getLogger();
+        log.info("Starting Druid Tranquility Controller Service...");
+
+        final String dataSource = context.getProperty(DATASOURCE).evaluateAttributeExpressions().getValue();
+        final String zkConnectString = context.getProperty(ZOOKEEPER_CONNECTION_STRING).evaluateAttributeExpressions().getValue();
+        zkBaseSleepMillis = context.getProperty(ZOOKEEPER_RETRY_BASE_SLEEP_TIME).evaluateAttributeExpressions().asInteger();
+        zkMaxRetries = context.getProperty(ZOOKEEPER_RETRY_BASE_SLEEP_TIME).evaluateAttributeExpressions().asInteger();
+        zkSleepMillis = context.getProperty(ZOOKEEPER_RETRY_SLEEP_TIME).evaluateAttributeExpressions().asInteger();
+        final String indexService = context.getProperty(DRUID_INDEX_SERVICE_PATH).evaluateAttributeExpressions().getValue();
+        final String discoveryPath = context.getProperty(DRUID_DISCOVERY_PATH).evaluateAttributeExpressions().getValue();
+        final int clusterPartitions = context.getProperty(CLUSTER_PARTITIONS).evaluateAttributeExpressions().asInteger();
+        final int clusterReplication = context.getProperty(CLUSTER_REPLICATION).evaluateAttributeExpressions().asInteger();
+        final String timestampField = context.getProperty(TIMESTAMP_FIELD).evaluateAttributeExpressions().getValue();
+        final String segmentGranularity = context.getProperty(SEGMENT_GRANULARITY).getValue();
+        final String queryGranularity = context.getProperty(QUERY_GRANULARITY).getValue();
+        final String windowPeriod = context.getProperty(WINDOW_PERIOD).getValue();
+        final String indexRetryPeriod = context.getProperty(INDEX_RETRY_PERIOD).getValue();
+        final String aggregatorJSON = context.getProperty(AGGREGATOR_JSON).evaluateAttributeExpressions().getValue();
+        final String dimensionsStringList = context.getProperty(DIMENSIONS_LIST).getValue();
+        final int maxBatchSize = context.getProperty(MAX_BATCH_SIZE).evaluateAttributeExpressions().asInteger();
+        final int maxPendingBatches = context.getProperty(MAX_PENDING_BATCHES).evaluateAttributeExpressions().asInteger();
+        final int lingerMillis = context.getProperty(LINGER_MILLIS).evaluateAttributeExpressions().asInteger();
+
+        transitUri = String.format(FIREHOSE_PATTERN, dataSource) + ";indexServicePath=" + indexService;
+
+        final List<String> dimensions = getDimensions(dimensionsStringList);
+        final List<AggregatorFactory> aggregator = getAggregatorList(aggregatorJSON);
+
+        final Timestamper<Map<String, Object>> timestamper = new Timestamper<Map<String, Object>>() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public DateTime timestamp(Map<String, Object> theMap) {
+                return new DateTime(theMap.get(timestampField));
+            }
+        };
+
+        Iterator<AggregatorFactory> aggIterator = aggregator.iterator();
+        AggregatorFactory currFactory;
+        log.debug("Number of Aggregations Defined: {}", new Object[]{aggregator.size()});
+        while (aggIterator.hasNext()) {
+            currFactory = aggIterator.next();
+            log.debug("Verifying Aggregator Definition\n\tAggregator Name: {}\n\tAggregator Type: {}\n\tAggregator Req Fields: {}",
+                    new Object[]{currFactory.getName(), currFactory.getTypeName(), currFactory.requiredFields()});
+        }
+        // Tranquility uses ZooKeeper (through Curator) for coordination.
+        curator = getCurator(zkConnectString);
+        curator.start();
+
+        // The JSON serialization of your object must have a timestamp field in a format that Druid understands. By default,
+        // Druid expects the field to be called "timestamp" and to be an ISO8601 timestamp.
+        final TimestampSpec timestampSpec = new TimestampSpec(timestampField, "auto", null);
+
+        final Beam<Map<String, Object>> beam = buildBeam(dataSource, indexService, discoveryPath, clusterPartitions, clusterReplication,
+                segmentGranularity, queryGranularity, windowPeriod, indexRetryPeriod, dimensions, aggregator, timestamper, timestampSpec);
+
+        tranquilizer = buildTranquilizer(maxBatchSize, maxPendingBatches, lingerMillis, beam);
+
+        tranquilizer.start();
+    }
+
+    Tranquilizer<Map<String, Object>> buildTranquilizer(int maxBatchSize, int maxPendingBatches, int lingerMillis, Beam<Map<String, Object>> beam) {
+        return Tranquilizer.builder()
+                .maxBatchSize(maxBatchSize)
+                .maxPendingBatches(maxPendingBatches)
+                .lingerMillis(lingerMillis)
+                .blockOnFull(true)
+                .build(beam);
+    }
+
+    Beam<Map<String, Object>> buildBeam(String dataSource, String indexService, String discoveryPath, int clusterPartitions, int clusterReplication,
+                                        String segmentGranularity, String queryGranularity, String windowPeriod, String indexRetryPeriod, List<String> dimensions,
+                                        List<AggregatorFactory> aggregator, Timestamper<Map<String, Object>> timestamper, TimestampSpec timestampSpec) {
+        return DruidBeams.builder(timestamper)
+                .curator(curator)
+                .discoveryPath(discoveryPath)
+                .location(DruidLocation.create(DruidEnvironment.create(indexService, FIREHOSE_PATTERN), dataSource))
+                .timestampSpec(timestampSpec)
+                .rollup(DruidRollup.create(DruidDimensions.specific(dimensions), aggregator, QueryGranularity.fromString(queryGranularity)))
+                .tuning(
+                        ClusteredBeamTuning
+                                .builder()
+                                .segmentGranularity(getGranularity(segmentGranularity))
+                                .windowPeriod(new Period(windowPeriod))
+                                .partitions(clusterPartitions)
+                                .replicants(clusterReplication)
+                                .build()
+                )
+                .druidBeamConfig(
+                        DruidBeamConfig
+                                .builder()
+                                .indexRetryPeriod(new Period(indexRetryPeriod))
+                                .build())
+                .buildBeam();
+    }
+
+    @OnDisabled
+    public void onDisabled() {
+        if (tranquilizer != null) {
+            tranquilizer.flush();
+            tranquilizer.stop();
+            tranquilizer = null;
+        }
+
+        if (curator != null) {
+            curator.close();
+            curator = null;
+        }
+    }
+
+    public Tranquilizer getTranquilizer() {
+        return tranquilizer;
+    }
+
+    CuratorFramework getCurator(String zkConnectString) {
+        return CuratorFrameworkFactory
+                .builder()
+                .connectString(zkConnectString)
+                .retryPolicy(new ExponentialBackoffRetry(zkBaseSleepMillis, zkMaxRetries, zkSleepMillis))
+                .build();
+    }
+
+    @Override
+    public String getTransitUri() {
+        return transitUri;
+    }
+
+    @SuppressWarnings("unchecked")
+    private static List<Map<String, String>> parseJsonString(String aggregatorJson) {
+        if (aggregatorJson == null) {
+            return Collections.EMPTY_LIST;
+        }
+        ObjectMapper mapper = new ObjectMapper();
+        final List<Map<String, String>> aggSpecList;
+        try {
+            aggSpecList = mapper.readValue(aggregatorJson, List.class);
+            return aggSpecList;
+        } catch (IOException e) {
+            throw new IllegalArgumentException("Exception while parsing the aggregrator JSON");
+        }
+    }
+
+    private List<String> getDimensions(String dimensionStringList) {
+        List<String> dimensionList = new ArrayList<>();
+        if (dimensionStringList != null) {
+            Arrays.stream(dimensionStringList.split(","))
+                    .filter(StringUtils::isNotBlank)
+                    .map(String::trim)
+                    .forEach(dimensionList::add);
+        }
+        return dimensionList;
+    }
+
+    private List<AggregatorFactory> getAggregatorList(String aggregatorJSON) {
+        List<AggregatorFactory> aggregatorList = new LinkedList<>();
+        List<Map<String, String>> aggregatorInfo = parseJsonString(aggregatorJSON);
+        for (Map<String, String> aggregator : aggregatorInfo) {
+
+            if (aggregator.get("type").equalsIgnoreCase("count")) {
+                aggregatorList.add(getCountAggregator(aggregator));
+            } else if (aggregator.get("type").equalsIgnoreCase("doublesum")) {
+                aggregatorList.add(getDoubleSumAggregator(aggregator));
+            } else if (aggregator.get("type").equalsIgnoreCase("doublemax")) {
+                aggregatorList.add(getDoubleMaxAggregator(aggregator));
+            } else if (aggregator.get("type").equalsIgnoreCase("doublemin")) {
+                aggregatorList.add(getDoubleMinAggregator(aggregator));
+            } else if (aggregator.get("type").equalsIgnoreCase("longsum")) {
+                aggregatorList.add(getLongSumAggregator(aggregator));
+            } else if (aggregator.get("type").equalsIgnoreCase("longmax")) {
+                aggregatorList.add(getLongMaxAggregator(aggregator));
+            } else if (aggregator.get("type").equalsIgnoreCase("longmin")) {
+                aggregatorList.add(getLongMinAggregator(aggregator));
+            }
+        }
+
+        return aggregatorList;
+    }
+
+    private AggregatorFactory getLongMinAggregator(Map<String, String> map) {
+        return new LongMinAggregatorFactory(map.get("name"), map.get("fieldName"));
+    }
+
+    private AggregatorFactory getLongMaxAggregator(Map<String, String> map) {
+        return new LongMaxAggregatorFactory(map.get("name"), map.get("fieldName"));
+    }
+
+    private AggregatorFactory getLongSumAggregator(Map<String, String> map) {
+        return new LongSumAggregatorFactory(map.get("name"), map.get("fieldName"));
+    }
+
+    private AggregatorFactory getDoubleMinAggregator(Map<String, String> map) {
+        return new DoubleMinAggregatorFactory(map.get("name"), map.get("fieldName"));
+    }
+
+    private AggregatorFactory getDoubleMaxAggregator(Map<String, String> map) {
+        return new DoubleMaxAggregatorFactory(map.get("name"), map.get("fieldName"));
+    }
+
+    private AggregatorFactory getDoubleSumAggregator(Map<String, String> map) {
+        return new DoubleSumAggregatorFactory(map.get("name"), map.get("fieldName"));
+    }
+
+    private AggregatorFactory getCountAggregator(Map<String, String> map) {
+        return new CountAggregatorFactory(map.get("name"));
+    }
+
+    private Granularity getGranularity(String granularityString) {
+        Granularity granularity = Granularity.HOUR;
+
+        switch (granularityString) {
+            case "SECOND":
+                granularity = Granularity.SECOND;
+                break;
+            case "MINUTE":
+                granularity = Granularity.MINUTE;
+                break;
+            case "FIVE_MINUTE":
+                granularity = Granularity.FIVE_MINUTE;
+                break;
+            case "TEN_MINUTE":
+                granularity = Granularity.TEN_MINUTE;
+                break;
+            case "FIFTEEN_MINUTE":
+                granularity = Granularity.FIFTEEN_MINUTE;
+                break;
+            case "HOUR":
+                granularity = Granularity.HOUR;
+                break;
+            case "SIX_HOUR":
+                granularity = Granularity.SIX_HOUR;
+                break;
+            case "DAY":
+                granularity = Granularity.DAY;
+                break;
+            case "WEEK":
+                granularity = Granularity.WEEK;
+                break;
+            case "MONTH":
+                granularity = Granularity.MONTH;
+                break;
+            case "YEAR":
+                granularity = Granularity.YEAR;
+                break;
+            default:
+                break;
+        }
+        return granularity;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/ecb80678/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService b/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService
index 53d6d06..201af17 100644
--- a/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService
+++ b/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService
@@ -13,4 +13,4 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-org.apache.nifi.controller.DruidTranquilityController
\ No newline at end of file
+org.apache.nifi.controller.druid.DruidTranquilityController
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/ecb80678/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service/src/test/java/org/apache/nifi/controller/DruidTranquilityControllerTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service/src/test/java/org/apache/nifi/controller/DruidTranquilityControllerTest.java b/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service/src/test/java/org/apache/nifi/controller/DruidTranquilityControllerTest.java
deleted file mode 100644
index 95d5e9d..0000000
--- a/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service/src/test/java/org/apache/nifi/controller/DruidTranquilityControllerTest.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/*
- * 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.controller;
-
-
-public class DruidTranquilityControllerTest {
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/ecb80678/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service/src/test/java/org/apache/nifi/controller/druid/DruidTranquilityControllerTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service/src/test/java/org/apache/nifi/controller/druid/DruidTranquilityControllerTest.java b/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service/src/test/java/org/apache/nifi/controller/druid/DruidTranquilityControllerTest.java
new file mode 100644
index 0000000..56c2616
--- /dev/null
+++ b/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service/src/test/java/org/apache/nifi/controller/druid/DruidTranquilityControllerTest.java
@@ -0,0 +1,100 @@
+/*
+ * 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.controller.druid;
+
+
+import com.metamx.tranquility.tranquilizer.Tranquilizer;
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.controller.api.druid.DruidTranquilityService;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.mockito.Mockito.mock;
+
+public class DruidTranquilityControllerTest {
+
+    private TestRunner runner;
+    private MockDruidTranquilityController service;
+
+    @Before
+    public void setup() throws Exception {
+        runner = TestRunners.newTestRunner(TestControllerServiceProcessor.class);
+        service = new MockDruidTranquilityController();
+        runner.addControllerService("Client Service", service);
+    }
+
+    @Test
+    public void testValid() {
+        runner.assertNotValid(service);
+        runner.setProperty(service, DruidTranquilityController.DATASOURCE, "test");
+        runner.assertNotValid(service);
+        runner.setProperty(service, DruidTranquilityController.ZOOKEEPER_CONNECTION_STRING, "localhost:2181");
+        runner.assertNotValid(service);
+        runner.setProperty(service, DruidTranquilityController.AGGREGATOR_JSON, "[{\"type\": \"count\", \"name\": \"count\"}]");
+        runner.assertNotValid(service);
+        runner.setProperty(service, DruidTranquilityController.DIMENSIONS_LIST, "dim1,dim2");
+        runner.assertValid(service);
+    }
+
+    public static class MockDruidTranquilityController extends DruidTranquilityController {
+
+        Tranquilizer t = mock(Tranquilizer.class);
+        CuratorFramework c = mock(CuratorFramework.class);
+
+        @Override
+        public Tranquilizer getTranquilizer() {
+            return t;
+        }
+
+        @Override
+        CuratorFramework getCurator(String zkConnectString) {
+            return c;
+        }
+    }
+
+    public static class TestControllerServiceProcessor extends AbstractProcessor {
+
+        static final PropertyDescriptor CLIENT_SERVICE = new PropertyDescriptor.Builder()
+                .name("Client Service")
+                .description("DruidTranquilityService")
+                .identifiesControllerService(DruidTranquilityService.class)
+                .required(true)
+                .build();
+
+        @Override
+        public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+        }
+
+        @Override
+        protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+            List<PropertyDescriptor> propertyDescriptors = new ArrayList<>();
+            propertyDescriptors.add(CLIENT_SERVICE);
+            return propertyDescriptors;
+        }
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/ecb80678/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-nar/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-nar/pom.xml b/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-nar/pom.xml
index 746a077..6a384c6 100644
--- a/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-nar/pom.xml
@@ -25,6 +25,11 @@
   <packaging>nar</packaging>
   
 	<dependencies>
+		<dependency>
+			<groupId>org.apache.nifi</groupId>
+			<artifactId>nifi-druid-controller-service-api-nar</artifactId>
+			<type>nar</type>
+		</dependency>
    		<dependency>
 			<groupId>org.apache.nifi</groupId>
 			<artifactId>nifi-druid-controller-service</artifactId>

http://git-wip-us.apache.org/repos/asf/nifi/blob/ecb80678/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-nar/src/main/resources/META-INF/LICENSE
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-nar/src/main/resources/META-INF/LICENSE b/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-nar/src/main/resources/META-INF/LICENSE
new file mode 100644
index 0000000..f3c8ece
--- /dev/null
+++ b/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-nar/src/main/resources/META-INF/LICENSE
@@ -0,0 +1,209 @@
+
+                                 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.

http://git-wip-us.apache.org/repos/asf/nifi/blob/ecb80678/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-nar/src/main/resources/META-INF/NOTICE
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-nar/src/main/resources/META-INF/NOTICE b/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-nar/src/main/resources/META-INF/NOTICE
new file mode 100644
index 0000000..b8c958c
--- /dev/null
+++ b/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-nar/src/main/resources/META-INF/NOTICE
@@ -0,0 +1,5 @@
+nifi-druid-nar
+Copyright 2015-2017 The Apache Software Foundation
+
+This product includes software developed at
+The Apache Software Foundation (http://www.apache.org/).

http://git-wip-us.apache.org/repos/asf/nifi/blob/ecb80678/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-processors/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-processors/pom.xml b/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-processors/pom.xml
index 6ce35f6..f596c6f 100644
--- a/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-processors/pom.xml
@@ -27,69 +27,48 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-api</artifactId>
-            <version>1.3.0</version>
+            <scope>provided</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.3.0</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-processor-utils</artifactId>
-            <version>1.3.0</version>
         </dependency>
+       <dependency>
+           <groupId>org.apache.nifi</groupId>
+           <artifactId>nifi-record</artifactId>
+       </dependency>
+       <dependency>
+           <groupId>org.apache.nifi</groupId>
+           <artifactId>nifi-record-serialization-service-api</artifactId>
+       </dependency>
         <dependency>
         	<groupId>org.apache.nifi</groupId>
   			<artifactId>nifi-druid-controller-service-api</artifactId>
   			<version>1.5.0-SNAPSHOT</version>
+            <scope>provided</scope>
 		</dependency>
 		<dependency>
     		<groupId>io.druid</groupId>
-    		<artifactId>tranquility-core_2.10</artifactId>
-    		<version>0.8.2</version>
-            <exclusions>
-                <exclusion>
-                    <groupId>io.druid</groupId>
-                    <artifactId>druid-console</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>io.druid</groupId>
-                    <artifactId>druid-aws-common</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>c3p0</groupId>
-                    <artifactId>c3p0</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>io.tesla.aether</groupId>
-                    <artifactId>tesla-aether</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>org.glassfish</groupId>
-                    <artifactId>javax.el</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>com.sun.jersey</groupId>
-                    <artifactId>*</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>org.eclipse.jetty</groupId>
-                    <artifactId>*</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>org.eclipse.aether</groupId>
-                    <artifactId>aether-api</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>com.maxmind.geoip2</groupId>
-                    <artifactId>geoip2</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>net.java.dev.jets3t</groupId>
-                    <artifactId>jets3t</artifactId>
-                </exclusion>
-            </exclusions>
+    		<artifactId>tranquility-core_2.11</artifactId>
+    		<scope>provided</scope>
 		</dependency>
+       <dependency>
+           <groupId>org.apache.nifi</groupId>
+           <artifactId>nifi-mock</artifactId>
+           <version>1.5.0-SNAPSHOT</version>
+           <scope>test</scope>
+       </dependency>
+       <dependency>
+           <groupId>org.apache.nifi</groupId>
+           <artifactId>nifi-mock-record-utils</artifactId>
+           <version>1.5.0-SNAPSHOT</version>
+           <scope>test</scope>
+       </dependency>
+       <dependency>
+           <groupId>org.apache.nifi</groupId>
+           <artifactId>nifi-druid-controller-service</artifactId>
+           <version>1.5.0-SNAPSHOT</version>
+           <scope>test</scope>
+       </dependency>
 	</dependencies>	
 </project>
\ No newline at end of file